Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Ivan Blinkov 2018-12-11 11:29:58 +03:00
commit 1d37d86f60
232 changed files with 6897 additions and 3547 deletions

View File

@ -50,7 +50,8 @@ IncludeCategories:
- Regex: '.*'
Priority: 40
ReflowComments: false
AlignEscapedNewlinesLeft: true
AlignEscapedNewlinesLeft: false
AlignEscapedNewlines: DontAlign
# Not changed:
AccessModifierOffset: -4

View File

@ -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 "")

View File

@ -43,6 +43,7 @@
#include <IO/WriteHelpers.h>
#include <IO/UseSSL.h>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <DataStreams/InternalTextLogsRowOutputStream.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/ASTSetQuery.h>
@ -60,6 +61,7 @@
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <Common/Config/configReadClient.h>
#include <Storages/ColumnsDescription.h>
#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<const ASTInsertQuery *>(&*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<AddingDefaultsBlockInputStream>(block_input, column_defaults, context);
BlockInputStreamPtr async_block_input = std::make_shared<AsynchronousBlockInputStream>(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);

View File

@ -30,6 +30,7 @@
#include <Storages/StorageMemory.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Core/ExternalTable.h>
#include <Storages/ColumnDefault.h>
#include <DataTypes/DataTypeLowCardinality.h>
#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)
{

View File

@ -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();

View File

@ -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<Exception> Connection::receiveException()
}
std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
{
size_t num = Protocol::Server::stringsInMessage(msg_type);
std::vector<String> 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");

View File

@ -1,5 +1,7 @@
#pragma once
#include <optional>
#include <common/logger_useful.h>
#include <Poco/Net/StreamSocket.h>
@ -96,6 +98,7 @@ public:
Block block;
std::unique_ptr<Exception> exception;
std::vector<String> multistring_message;
Progress progress;
BlockStreamProfileInfo profile_info;
@ -254,6 +257,7 @@ private:
Block receiveLogData();
Block receiveDataImpl(BlockInputStreamPtr & stream);
std::vector<String> receiveMultistringMessage(UInt64 msg_type);
std::unique_ptr<Exception> receiveException();
Progress receiveProgress();
BlockStreamProfileInfo receiveProfileInfo();

View File

@ -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); }
};

View File

@ -99,6 +99,13 @@ void Block::insertUnique(ColumnWithTypeAndName && elem)
}
void Block::erase(const std::set<size_t> & positions)
{
for (auto it = positions.rbegin(); it != positions.rend(); ++it)
erase(*it);
}
void Block::erase(size_t position)
{
if (data.empty())

View File

@ -2,6 +2,7 @@
#include <vector>
#include <list>
#include <set>
#include <map>
#include <initializer_list>
@ -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<size_t> & positions);
/// remove the column with the specified name
void erase(const String & name);

View File

@ -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;
}
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <unordered_map>
#include <Core/Types.h>
@ -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<bool>; /// 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<size_t, RowsBitMask>;
/// 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;
};
}

View File

@ -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

View File

@ -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";
}

View File

@ -0,0 +1,205 @@
#include <Common/typeid_cast.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateMissingDefaults.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnConst.h>
#include <Columns/FilterDescription.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
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<size_t, MutableColumnPtr> 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<decltype(types)>;
using DataType = typename Types::LeftType;
if constexpr (!std::is_same_v<DataType, DataTypeString> && !std::is_same_v<DataType, DataTypeFixedString>)
{
using FieldType = typename DataType::FieldType;
using ColVecType = std::conditional_t<IsDecimalNumber<FieldType>, ColumnDecimal<FieldType>, ColumnVector<FieldType>>;
auto col_read = typeid_cast<ColVecType *>(column_mixed.get());
if (!col_read)
return false;
typename ColVecType::Container & dst = col_read->getData();
if (auto const_col_defs = checkAndGetColumnConst<ColVecType>(column_defs.get()))
{
FieldType value = checkAndGetColumn<ColVecType>(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<ColVecType>(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<void>(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;
}
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Storages/ColumnDefault.h>
#include <Interpreters/Context.h>
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;
};
}

View File

@ -1,11 +1,11 @@
#include <DataStreams/AddingDefaultBlockInputStream.h>
#include <DataStreams/AddingMissedBlockInputStream.h>
#include <Interpreters/addMissingDefaults.h>
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)

View File

@ -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:

View File

@ -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().

View File

@ -4,7 +4,8 @@
#include <IO/ReadBufferFromMemory.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/InputStreamFromASTInsertQuery.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <Storages/ColumnsDescription.h>
namespace DB
{
@ -44,6 +45,10 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
input_buffer_contacenated = std::make_unique<ConcatReadBuffer>(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<AddingDefaultsBlockInputStream>(res_stream, columns_description->defaults, context);
}
}

View File

@ -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;

View File

@ -9,6 +9,7 @@
#include <DataStreams/NativeBlockOutputStream.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeLowCardinality.h>
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);

View File

@ -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;

View File

@ -514,6 +514,13 @@ inline bool isNumber(const T & data_type)
return which.isInt() || which.isUInt() || which.isFloat();
}
template <typename T>
inline bool isColumnedAsNumber(const T & data_type)
{
WhichDataType which(data_type);
return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID();
}
template <typename T>
inline bool isString(const T & data_type)
{

View File

@ -1,24 +1,24 @@
#include "CacheDictionary.h"
#include <functional>
#include <sstream>
#include <memory>
#include <Columns/ColumnsNumber.h>
#include <sstream>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Common/BitHelpers.h>
#include <Common/randomSeed.h>
#include <Common/HashTable/Hash.h>
#include <Common/Stopwatch.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Common/HashTable/Hash.h>
#include <Common/ProfileEvents.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/Stopwatch.h>
#include <Common/randomSeed.h>
#include <Common/typeid_cast.h>
#include "DictionaryBlockInputStream.h"
#include <ext/size.h>
#include <ext/range.h>
#include <ext/map.h>
#include "DictionaryFactory.h"
#include <ext/range.h>
#include <ext/size.h>
#include "CacheDictionary.inc.h"
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace ProfileEvents
{
@ -42,7 +42,6 @@ namespace CurrentMetrics
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,7 +83,8 @@ 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<Key> & ids, PaddedPODArray<Key> & out) const
@ -91,15 +96,18 @@ void CacheDictionary::toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<K
/// Allow to use single value in same way as array.
static inline CacheDictionary::Key getAt(const PaddedPODArray<CacheDictionary::Key> & 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<CacheDictionary::Key> & arr, const size_t idx)
{
return arr[idx];
}
static inline CacheDictionary::Key getAt(const CacheDictionary::Key & value, const size_t)
{
return value;
}
template <typename AncestorType>
void CacheDictionary::isInImpl(
const PaddedPODArray<Key> & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void CacheDictionary::isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
/// Transform all children to parents until ancestor id or null_value will be reached.
@ -164,25 +172,17 @@ void CacheDictionary::isInImpl(
}
void CacheDictionary::isInVectorVector(
const PaddedPODArray<Key> & child_ids,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_ids, out);
}
void CacheDictionary::isInVectorConstant(
const PaddedPODArray<Key> & child_ids,
const Key ancestor_id,
PaddedPODArray<UInt8> & out) const
void CacheDictionary::isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_id, out);
}
void CacheDictionary::isInConstantVector(
const Key child_id,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
/// Special case with single child value.
@ -213,7 +213,8 @@ 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<String>(attribute.null_values)};
@ -221,23 +222,23 @@ void CacheDictionary::getString(const std::string & attribute_name, const Padded
}
void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & 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); });
}
void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & 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}; });
}
@ -329,11 +330,11 @@ void CacheDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
return;
std::vector<Key> 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,
update(
required_ids,
[&](const auto id, const auto)
{
for (const auto row : outdated_ids[id])
@ -413,17 +414,39 @@ void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::UInt128: std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = std::get<UInt128>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::UInt8:
std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt16:
std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt32:
std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt64:
std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt128:
std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = std::get<UInt128>(attribute.null_values);
break;
case AttributeUnderlyingType::Int8:
std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values);
break;
case AttributeUnderlyingType::Int16:
std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values);
break;
case AttributeUnderlyingType::Int32:
std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values);
break;
case AttributeUnderlyingType::Int64:
std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values);
break;
case AttributeUnderlyingType::Float32:
std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values);
break;
case AttributeUnderlyingType::Float64:
std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values);
break;
case AttributeUnderlyingType::Decimal32:
std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = std::get<Decimal32>(attribute.null_values);
@ -457,21 +480,49 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt128: std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = value.get<UInt128>(); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::UInt8:
std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt16:
std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt32:
std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt64:
std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt128:
std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = value.get<UInt128>();
break;
case AttributeUnderlyingType::Int8:
std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int16:
std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int32:
std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int64:
std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Float32:
std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>();
break;
case AttributeUnderlyingType::Float64:
std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>();
break;
case AttributeUnderlyingType::Decimal32: std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = value.get<Decimal32>(); break;
case AttributeUnderlyingType::Decimal64: std::get<ContainerPtrType<Decimal64>>(attribute.arrays)[idx] = value.get<Decimal64>(); break;
case AttributeUnderlyingType::Decimal128: std::get<ContainerPtrType<Decimal128>>(attribute.arrays)[idx] = value.get<Decimal128>(); break;
case AttributeUnderlyingType::Decimal32:
std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = value.get<Decimal32>();
break;
case AttributeUnderlyingType::Decimal64:
std::get<ContainerPtrType<Decimal64>>(attribute.arrays)[idx] = value.get<Decimal64>();
break;
case AttributeUnderlyingType::Decimal128:
std::get<ContainerPtrType<Decimal128>>(attribute.arrays)[idx] = value.get<Decimal128>();
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_urep_t>(CellMetadata::time_point_t()));
return (idx != zero_cell_idx && cells[idx].id == 0)
|| (cells[idx].data == ext::safe_bit_cast<CellMetadata::time_point_urep_t>(CellMetadata::time_point_t()));
}
PaddedPODArray<CacheDictionary::Key> CacheDictionary::getCachedIds() const
@ -537,14 +588,12 @@ BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_na
void registerDictionaryCache(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD};
@ -565,8 +614,6 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("cache", create_layout);
}

View File

@ -1,31 +1,33 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <Common/ArenaWithFreeLists.h>
#include <Common/CurrentMetrics.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <ext/bit_cast.h>
#include <cmath>
#include <atomic>
#include <chrono>
#include <vector>
#include <cmath>
#include <map>
#include <variant>
#include <pcg_random.hpp>
#include <shared_mutex>
#include <variant>
#include <vector>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <pcg_random.hpp>
#include <Common/ArenaWithFreeLists.h>
#include <Common/CurrentMetrics.h>
#include <ext/bit_cast.h>
#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<double>(hit_count.load(std::memory_order_acquire)) /
query_count.load(std::memory_order_relaxed);
return static_cast<double>(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<double>(element_count.load(std::memory_order_relaxed)) / size;
}
double getLoadFactor() const override { return static_cast<double>(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<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -77,7 +72,8 @@ public:
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
void isInVectorVector(const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
@ -106,7 +102,9 @@ public:
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -124,9 +122,9 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const;
void
getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out)
const;
#define DECLARE(TYPE) \
void get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const;
@ -146,17 +144,17 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const;
void getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const;
void has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8> & out) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value> using ContainerType = Value[];
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
template <typename Value>
using ContainerType = Value[];
template <typename Value>
using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
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<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<UInt8>,
ContainerPtrType<UInt16>,
ContainerPtrType<UInt32>,
ContainerPtrType<UInt64>,
ContainerPtrType<UInt128>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>, ContainerPtrType<Decimal64>, ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> arrays;
ContainerPtrType<Int8>,
ContainerPtrType<Int16>,
ContainerPtrType<Int32>,
ContainerPtrType<Int64>,
ContainerPtrType<Decimal32>,
ContainerPtrType<Decimal64>,
ContainerPtrType<Decimal128>,
ContainerPtrType<Float32>,
ContainerPtrType<Float64>,
ContainerPtrType<StringRef>>
arrays;
};
void createAttributes();
@ -205,29 +223,17 @@ private:
template <typename OutputType, typename DefaultGetter>
void getItemsNumber(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const;
Attribute & attribute, const PaddedPODArray<Key> & ids, ResultArrayType<OutputType> & out, DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename DefaultGetter>
void getItemsNumberImpl(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const;
Attribute & attribute, const PaddedPODArray<Key> & ids, ResultArrayType<OutputType> & out, DefaultGetter && get_default) const;
template <typename DefaultGetter>
void getItemsString(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ColumnString * out,
DefaultGetter && get_default) const;
void getItemsString(Attribute & attribute, const PaddedPODArray<Key> & ids, ColumnString * out, DefaultGetter && get_default) const;
template <typename PresentIdHandler, typename AbsentIdHandler>
void update(
const std::vector<Key> & requested_ids, PresentIdHandler && on_cell_updated,
AbsentIdHandler && on_id_not_found) const;
void update(const std::vector<Key> & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const;
PaddedPODArray<Key> getCachedIds() const;
@ -251,10 +257,7 @@ private:
FindResult findCellIdx(const Key & id, const CellMetadata::time_point_t now) const;
template <typename AncestorType>
void isInImpl(
const PaddedPODArray<Key> & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const;
void isInImpl(const PaddedPODArray<Key> & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
const std::string name;
const DictionaryStructure dict_struct;

View File

@ -1,11 +1,11 @@
#include "CacheDictionary.h"
#include <ext/size.h>
#include <ext/map.h>
#include <ext/range.h>
#include <Columns/ColumnsNumber.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnsNumber.h>
#include <ext/map.h>
#include <ext/range.h>
#include <ext/size.h>
namespace ProfileEvents
{
@ -28,7 +28,6 @@ namespace CurrentMetrics
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
@ -36,12 +35,11 @@ namespace ErrorCodes
template <typename OutputType, typename DefaultGetter>
void CacheDictionary::getItemsNumber(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const
Attribute & attribute, const PaddedPODArray<Key> & ids, ResultArrayType<OutputType> & out, DefaultGetter && get_default) const
{
if (false) {}
if (false)
{
}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsNumberImpl<TYPE, OutputType>(attribute, ids, out, std::forward<DefaultGetter>(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 <typename AttributeType, typename OutputType, typename DefaultGetter>
void CacheDictionary::getItemsNumberImpl(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const
Attribute & attribute, const PaddedPODArray<Key> & ids, ResultArrayType<OutputType> & out, DefaultGetter && get_default) const
{
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std::unordered_map<Key, std::vector<size_t>> outdated_ids;
@ -122,11 +116,11 @@ void CacheDictionary::getItemsNumberImpl(
return;
std::vector<Key> 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,
update(
required_ids,
[&](const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
@ -143,10 +137,7 @@ void CacheDictionary::getItemsNumberImpl(
template <typename DefaultGetter>
void CacheDictionary::getItemsString(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ColumnString * out,
DefaultGetter && get_default) const
Attribute & attribute, const PaddedPODArray<Key> & ids, ColumnString * out, DefaultGetter && get_default) const
{
const auto rows = ext::size(ids);
@ -245,10 +236,10 @@ void CacheDictionary::getItemsString(
if (!outdated_ids.empty())
{
std::vector<Key> 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,
update(
required_ids,
[&](const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
@ -277,19 +268,13 @@ void CacheDictionary::getItemsString(
template <typename PresentIdHandler, typename AbsentIdHandler>
void CacheDictionary::update(
const std::vector<Key> & requested_ids,
PresentIdHandler && on_cell_updated,
AbsentIdHandler && on_id_not_found) const
const std::vector<Key> & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const
{
std::unordered_map<Key, UInt8> remaining_ids{requested_ids.size()};
for (const auto id : requested_ids)
remaining_ids.insert({id, 0});
std::uniform_int_distribution<UInt64> distribution
{
dict_lifetime.min_sec,
dict_lifetime.max_sec
};
std::uniform_int_distribution<UInt64> 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<std::vector>(ext::range(0, attributes.size()), [&block] (size_t i)
{
return block.safeGetByPosition(i + 1).column.get();
});
const auto column_ptrs = ext::map<std::vector>(
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()))
{

View File

@ -1,21 +1,20 @@
#include "ClickHouseDictionarySource.h"
#include "ExternalQueryBuilder.h"
#include "writeParenthesisedString.h"
#include <memory>
#include <Client/ConnectionPool.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include "readInvalidateQuery.h"
#include <IO/ConnectionTimeouts.h>
#include <Interpreters/executeQuery.h>
#include <Common/isLocalAddress.h>
#include <memory>
#include <ext/range.h>
#include <IO/ConnectionTimeouts.h>
#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,13 +24,25 @@ 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<ConnectionPool>(
MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource",
MAX_CONNECTIONS,
host,
port,
db,
user,
password,
timeouts,
"ClickHouseDictionarySource",
Protocol::Compression::Enable,
secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
return std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
@ -42,44 +53,52 @@ 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 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<UInt64> & ids)
{
return createStreamForSelectiveLoad(
query_builder.composeLoadIdsQuery(ids));
return createStreamForSelectiveLoad(query_builder.composeLoadIdsQuery(ids));
}
BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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

View File

@ -1,15 +1,14 @@
#pragma once
#include "IDictionarySource.h"
#include <memory>
#include <Client/ConnectionPoolWithFailover.h>
#include "DictionaryStructure.h"
#include "ExternalQueryBuilder.h"
#include <Client/ConnectionPoolWithFailover.h>
#include <memory>
#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<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override;
bool supportsSelectiveLoad() const override { return true; }

View File

@ -1,20 +1,19 @@
#include "ComplexKeyCacheDictionary.h"
#include "DictionaryBlockInputStream.h"
#include <Common/Arena.h>
#include <Common/BitHelpers.h>
#include <Common/randomSeed.h>
#include <Common/Stopwatch.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <ext/range.h>
#include <Common/ProfileEvents.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/Stopwatch.h>
#include <Common/randomSeed.h>
#include <ext/map.h>
#include <ext/range.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace ProfileEvents
{
extern const Event DictCacheKeysRequested;
extern const Event DictCacheKeysRequestedMiss;
extern const Event DictCacheKeysRequestedFound;
@ -34,7 +33,6 @@ namespace CurrentMetrics
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,17 +72,18 @@ 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<String>(attribute.null_values)};
@ -86,27 +91,35 @@ void ComplexKeyCacheDictionary::getString(
}
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); });
}
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}; });
}
@ -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,11 +225,14 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes
return;
std::vector<size_t> 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,
update(
key_columns,
keys_array,
required_rows,
[&](const StringRef key, const auto)
{
for (const auto out_idx : outdated_keys[key])
@ -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 <typename Pool>
StringRef ComplexKeyCacheDictionary::placeKeysInPool(
const size_t row, const Columns & key_columns, StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes, Pool & pool)
const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector<DictionaryAttribute> & key_attributes, Pool & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
@ -319,16 +336,21 @@ StringRef ComplexKeyCacheDictionary::placeKeysInPool(
/// Explicit instantiations.
template StringRef ComplexKeyCacheDictionary::placeKeysInPool<Arena>(
const size_t row, const Columns & key_columns, StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes, Arena & pool);
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes,
Arena & pool);
template StringRef ComplexKeyCacheDictionary::placeKeysInPool<ArenaWithFreeLists>(
const size_t row, const Columns & key_columns, StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes, ArenaWithFreeLists & pool);
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & 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;
@ -360,8 +382,9 @@ StringRef ComplexKeyCacheDictionary::copyKey(const StringRef key) const
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_urep_t>(CellMetadata::time_point_t())));
return (
cells[idx].key == StringRef{}
&& (idx != zero_cell_idx || cells[idx].data == ext::safe_bit_cast<CellMetadata::time_point_urep_t>(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,13 +404,12 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names &
void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};
const auto & layout_prefix = config_prefix + ".layout";

View File

@ -3,23 +3,23 @@
#include <atomic>
#include <chrono>
#include <map>
#include <shared_mutex>
#include <variant>
#include <vector>
#include <shared_mutex>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <pcg_random.hpp>
#include <Common/ArenaWithFreeLists.h>
#include <Common/HashTable/HashMap.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/SmallObjectPool.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include <common/StringRef.h>
#include <ext/bit_cast.h>
#include <ext/map.h>
#include <ext/scope_guard.h>
#include <pcg_random.hpp>
#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<double>(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<double>(element_count.load(std::memory_order_relaxed)) / size;
}
double getLoadFactor() const override { return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size; }
bool isCached() const override
{
return true;
}
bool isCached() const override { return true; }
std::unique_ptr<IExternalLoadable> clone() const override
{
return std::make_unique<ComplexKeyCacheDictionary>(*this);
}
std::unique_ptr<IExternalLoadable> clone() const override { return std::make_unique<ComplexKeyCacheDictionary>(*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<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -156,7 +118,8 @@ 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, \
void get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const PaddedPODArray<TYPE> & def, \
@ -177,14 +140,16 @@ 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, \
void get##TYPE( \
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const TYPE def, \
@ -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 <typename Value> using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value> using ContainerType = Value[];
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
template <typename Value>
using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value>
using ContainerType = Value[];
template <typename Value>
using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
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<time_point_t>(data & EXPIRES_AT_MASK);
}
void setExpiresAt(const time_point_t & t)
{
data = ext::safe_bit_cast<time_point_urep_t>(t);
}
time_point_t expiresAt() const { return ext::safe_bit_cast<time_point_t>(data & EXPIRES_AT_MASK); }
void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast<time_point_urep_t>(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<UInt8, UInt16, UInt32, UInt64, UInt128, Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64, String> null_values;
std::variant<ContainerPtrType<UInt8>,
std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
String>
null_values;
std::variant<
ContainerPtrType<UInt8>,
ContainerPtrType<UInt16>,
ContainerPtrType<UInt32>,
ContainerPtrType<UInt64>,
@ -283,8 +255,8 @@ private:
Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value);
template <typename OutputType, typename DefaultGetter>
void getItemsNumber(
Attribute & attribute, const Columns & key_columns, PaddedPODArray<OutputType> & out, DefaultGetter && get_default) const
void
getItemsNumber(Attribute & attribute, const Columns & key_columns, PaddedPODArray<OutputType> & out, DefaultGetter && get_default) const
{
if (false)
{
@ -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 <typename PresentKeyHandler, typename AbsentKeyHandler>
void update(const Columns & in_key_columns,
void update(
const Columns & in_key_columns,
const PODArray<StringRef> & in_keys,
const std::vector<size_t> & in_requested_rows,
PresentKeyHandler && on_cell_updated,
@ -561,8 +536,10 @@ private:
const auto key_columns = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
const auto attribute_columns = ext::map<Columns>(ext::range(0, attributes_size),
[&](const size_t attribute_idx) { return block.safeGetByPosition(keys_size + attribute_idx).column; });
const auto attribute_columns = ext::map<Columns>(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 <typename Arena>
static StringRef placeKeysInPool(const size_t row,
static StringRef placeKeysInPool(
const size_t row,
const Columns & key_columns,
StringRefs & keys,
const std::vector<DictionaryAttribute> & key_attributes,

View File

@ -2,8 +2,8 @@
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, {}, {}};

View File

@ -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<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt128: std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = value.get<UInt128>(); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::UInt8:
std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt16:
std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt32:
std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt64:
std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>();
break;
case AttributeUnderlyingType::UInt128:
std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = value.get<UInt128>();
break;
case AttributeUnderlyingType::Int8:
std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int16:
std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int32:
std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Int64:
std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>();
break;
case AttributeUnderlyingType::Float32:
std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>();
break;
case AttributeUnderlyingType::Float64:
std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>();
break;
case AttributeUnderlyingType::Decimal32: std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = value.get<Decimal32>(); break;
case AttributeUnderlyingType::Decimal64: std::get<ContainerPtrType<Decimal64>>(attribute.arrays)[idx] = value.get<Decimal64>(); break;
case AttributeUnderlyingType::Decimal128: std::get<ContainerPtrType<Decimal128>>(attribute.arrays)[idx] = value.get<Decimal128>(); break;
case AttributeUnderlyingType::Decimal32:
std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = value.get<Decimal32>();
break;
case AttributeUnderlyingType::Decimal64:
std::get<ContainerPtrType<Decimal64>>(attribute.arrays)[idx] = value.get<Decimal64>();
break;
case AttributeUnderlyingType::Decimal128:
std::get<ContainerPtrType<Decimal128>>(attribute.arrays)[idx] = value.get<Decimal128>();
break;
case AttributeUnderlyingType::String:
{

View File

@ -2,22 +2,43 @@
namespace DB
{
void ComplexKeyCacheDictionary::setDefaultAttributeValue(Attribute & attribute, const size_t idx) const
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::UInt128: std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = std::get<UInt128>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::UInt8:
std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt16:
std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt32:
std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt64:
std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values);
break;
case AttributeUnderlyingType::UInt128:
std::get<ContainerPtrType<UInt128>>(attribute.arrays)[idx] = std::get<UInt128>(attribute.null_values);
break;
case AttributeUnderlyingType::Int8:
std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values);
break;
case AttributeUnderlyingType::Int16:
std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values);
break;
case AttributeUnderlyingType::Int32:
std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values);
break;
case AttributeUnderlyingType::Int64:
std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values);
break;
case AttributeUnderlyingType::Float32:
std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values);
break;
case AttributeUnderlyingType::Float64:
std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values);
break;
case AttributeUnderlyingType::Decimal32:
std::get<ContainerPtrType<Decimal32>>(attribute.arrays)[idx] = std::get<Decimal32>(attribute.null_values);

View File

@ -1,12 +1,11 @@
#include "ComplexKeyHashedDictionary.h"
#include <ext/map.h>
#include <ext/range.h>
#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,24 +44,27 @@ 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<TYPE> & out) const\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & 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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
getItemsNumber<TYPE>(attribute, key_columns,\
getItemsNumber<TYPE>( \
attribute, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t) { return null_value; }); \
}
@ -76,34 +85,42 @@ 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<String>(attribute.null_values)};
getItemsImpl<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
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<TYPE> & def, ResultArrayType<TYPE> & out) const\
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & 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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns,\
getItemsNumber<TYPE>( \
attribute, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t row) { return def[row]; }); \
}
@ -124,34 +141,43 @@ 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<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
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<TYPE> & out) const\
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const TYPE def, \
ResultArrayType<TYPE> & 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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return def; });\
getItemsNumber<TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -170,16 +196,22 @@ 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<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
attribute,
key_columns,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return StringRef{def}; });
}
@ -192,22 +224,52 @@ void ComplexKeyHashedDictionary::has(const Columns & key_columns, const DataType
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt128: has<UInt128>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::String: has<StringRef>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt8:
has<UInt8>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt16:
has<UInt16>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt32:
has<UInt32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt64:
has<UInt64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt128:
has<UInt128>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int8:
has<Int8>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int16:
has<Int16>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int32:
has<Int32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int64:
has<Int64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Float32:
has<Float32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Float64:
has<Float64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::String:
has<StringRef>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal32: has<Decimal32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal64: has<Decimal64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal128: has<Decimal128>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal32:
has<Decimal32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal64:
has<Decimal64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal128:
has<Decimal128>(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,14 +299,10 @@ void ComplexKeyHashedDictionary::blockToAttributes(const Block & block)
const auto rows = block.rows();
element_count += rows;
const auto key_column_ptrs = ext::map<Columns>(ext::range(0, keys_size),
[&](const size_t attribute_idx)
{
return block.safeGetByPosition(attribute_idx).column;
});
const auto key_column_ptrs = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size),
[&](const size_t attribute_idx)
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size), [&](const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
@ -304,15 +363,11 @@ void ComplexKeyHashedDictionary::updateData()
stream->readPrefix();
while (Block block = stream->read())
{
const auto saved_key_column_ptrs = ext::map<Columns>(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<Columns>(
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<Columns>(ext::range(0, keys_size), [&](const size_t key_idx)
{
return block.safeGetByPosition(key_idx).column;
});
const auto update_key_column_ptrs = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t key_idx) { return block.safeGetByPosition(key_idx).column; });
Arena temp_key_pool;
ContainerType<std::vector<size_t>> update_key_hash;
@ -389,21 +444,49 @@ void ComplexKeyHashedDictionary::calculateBytesAllocated()
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::UInt128: addAttributeSize<UInt128>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::UInt8:
addAttributeSize<UInt8>(attribute);
break;
case AttributeUnderlyingType::UInt16:
addAttributeSize<UInt16>(attribute);
break;
case AttributeUnderlyingType::UInt32:
addAttributeSize<UInt32>(attribute);
break;
case AttributeUnderlyingType::UInt64:
addAttributeSize<UInt64>(attribute);
break;
case AttributeUnderlyingType::UInt128:
addAttributeSize<UInt128>(attribute);
break;
case AttributeUnderlyingType::Int8:
addAttributeSize<Int8>(attribute);
break;
case AttributeUnderlyingType::Int16:
addAttributeSize<Int16>(attribute);
break;
case AttributeUnderlyingType::Int32:
addAttributeSize<Int32>(attribute);
break;
case AttributeUnderlyingType::Int64:
addAttributeSize<Int64>(attribute);
break;
case AttributeUnderlyingType::Float32:
addAttributeSize<Float32>(attribute);
break;
case AttributeUnderlyingType::Float64:
addAttributeSize<Float64>(attribute);
break;
case AttributeUnderlyingType::Decimal32: addAttributeSize<Decimal32>(attribute); break;
case AttributeUnderlyingType::Decimal64: addAttributeSize<Decimal64>(attribute); break;
case AttributeUnderlyingType::Decimal128: addAttributeSize<Decimal128>(attribute); break;
case AttributeUnderlyingType::Decimal32:
addAttributeSize<Decimal32>(attribute);
break;
case AttributeUnderlyingType::Decimal64:
addAttributeSize<Decimal64>(attribute);
break;
case AttributeUnderlyingType::Decimal128:
addAttributeSize<Decimal128>(attribute);
break;
case AttributeUnderlyingType::String:
{
@ -425,27 +508,56 @@ void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, cons
attribute.maps.emplace<ContainerType<T>>();
}
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<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::UInt128: createAttributeImpl<UInt128>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::UInt8:
createAttributeImpl<UInt8>(attr, null_value);
break;
case AttributeUnderlyingType::UInt16:
createAttributeImpl<UInt16>(attr, null_value);
break;
case AttributeUnderlyingType::UInt32:
createAttributeImpl<UInt32>(attr, null_value);
break;
case AttributeUnderlyingType::UInt64:
createAttributeImpl<UInt64>(attr, null_value);
break;
case AttributeUnderlyingType::UInt128:
createAttributeImpl<UInt128>(attr, null_value);
break;
case AttributeUnderlyingType::Int8:
createAttributeImpl<Int8>(attr, null_value);
break;
case AttributeUnderlyingType::Int16:
createAttributeImpl<Int16>(attr, null_value);
break;
case AttributeUnderlyingType::Int32:
createAttributeImpl<Int32>(attr, null_value);
break;
case AttributeUnderlyingType::Int64:
createAttributeImpl<Int64>(attr, null_value);
break;
case AttributeUnderlyingType::Float32:
createAttributeImpl<Float32>(attr, null_value);
break;
case AttributeUnderlyingType::Float64:
createAttributeImpl<Float64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal32: createAttributeImpl<Decimal32>(attr, null_value); break;
case AttributeUnderlyingType::Decimal64: createAttributeImpl<Decimal64>(attr, null_value); break;
case AttributeUnderlyingType::Decimal128: createAttributeImpl<Decimal128>(attr, null_value); break;
case AttributeUnderlyingType::Decimal32:
createAttributeImpl<Decimal32>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal64:
createAttributeImpl<Decimal64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal128:
createAttributeImpl<Decimal128>(attr, null_value);
break;
case AttributeUnderlyingType::String:
{
@ -462,15 +574,14 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
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<TYPE, OutputType>(attribute, key_columns, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(get_default));
else if (attribute.type == AttributeUnderlyingType::TYPE) getItemsImpl<TYPE, OutputType>( \
attribute, key_columns, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
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<ContainerType<AttributeType>>(attribute.maps);
@ -532,21 +639,35 @@ bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: return setAttributeValueImpl<UInt8>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt16: return setAttributeValueImpl<UInt16>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt32: return setAttributeValueImpl<UInt32>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt64: return setAttributeValueImpl<UInt64>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt128: return setAttributeValueImpl<UInt128>(attribute, key, value.get<UInt128>());
case AttributeUnderlyingType::Int8: return setAttributeValueImpl<Int8>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int16: return setAttributeValueImpl<Int16>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int32: return setAttributeValueImpl<Int32>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int64: return setAttributeValueImpl<Int64>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Float32: return setAttributeValueImpl<Float32>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Float64: return setAttributeValueImpl<Float64>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::UInt8:
return setAttributeValueImpl<UInt8>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt16:
return setAttributeValueImpl<UInt16>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt32:
return setAttributeValueImpl<UInt32>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt64:
return setAttributeValueImpl<UInt64>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt128:
return setAttributeValueImpl<UInt128>(attribute, key, value.get<UInt128>());
case AttributeUnderlyingType::Int8:
return setAttributeValueImpl<Int8>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int16:
return setAttributeValueImpl<Int16>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int32:
return setAttributeValueImpl<Int32>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int64:
return setAttributeValueImpl<Int64>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Float32:
return setAttributeValueImpl<Float32>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Float64:
return setAttributeValueImpl<Float64>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Decimal32: return setAttributeValueImpl<Decimal32>(attribute, key, value.get<Decimal32>());
case AttributeUnderlyingType::Decimal64: return setAttributeValueImpl<Decimal64>(attribute, key, value.get<Decimal64>());
case AttributeUnderlyingType::Decimal128: return setAttributeValueImpl<Decimal128>(attribute, key, value.get<Decimal128>());
case AttributeUnderlyingType::Decimal32:
return setAttributeValueImpl<Decimal32>(attribute, key, value.get<Decimal32>());
case AttributeUnderlyingType::Decimal64:
return setAttributeValueImpl<Decimal64>(attribute, key, value.get<Decimal64>());
case AttributeUnderlyingType::Decimal128:
return setAttributeValueImpl<Decimal128>(attribute, key, value.get<Decimal128>());
case AttributeUnderlyingType::String:
{
@ -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{};
@ -623,22 +743,37 @@ std::vector<StringRef> ComplexKeyHashedDictionary::getKeys() const
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: return getKeys<UInt8>(attribute);
case AttributeUnderlyingType::UInt16: return getKeys<UInt16>(attribute);
case AttributeUnderlyingType::UInt32: return getKeys<UInt32>(attribute);
case AttributeUnderlyingType::UInt64: return getKeys<UInt64>(attribute);
case AttributeUnderlyingType::UInt128: return getKeys<UInt128>(attribute);
case AttributeUnderlyingType::Int8: return getKeys<Int8>(attribute);
case AttributeUnderlyingType::Int16: return getKeys<Int16>(attribute);
case AttributeUnderlyingType::Int32: return getKeys<Int32>(attribute);
case AttributeUnderlyingType::Int64: return getKeys<Int64>(attribute);
case AttributeUnderlyingType::Float32: return getKeys<Float32>(attribute);
case AttributeUnderlyingType::Float64: return getKeys<Float64>(attribute);
case AttributeUnderlyingType::String: return getKeys<StringRef>(attribute);
case AttributeUnderlyingType::UInt8:
return getKeys<UInt8>(attribute);
case AttributeUnderlyingType::UInt16:
return getKeys<UInt16>(attribute);
case AttributeUnderlyingType::UInt32:
return getKeys<UInt32>(attribute);
case AttributeUnderlyingType::UInt64:
return getKeys<UInt64>(attribute);
case AttributeUnderlyingType::UInt128:
return getKeys<UInt128>(attribute);
case AttributeUnderlyingType::Int8:
return getKeys<Int8>(attribute);
case AttributeUnderlyingType::Int16:
return getKeys<Int16>(attribute);
case AttributeUnderlyingType::Int32:
return getKeys<Int32>(attribute);
case AttributeUnderlyingType::Int64:
return getKeys<Int64>(attribute);
case AttributeUnderlyingType::Float32:
return getKeys<Float32>(attribute);
case AttributeUnderlyingType::Float64:
return getKeys<Float64>(attribute);
case AttributeUnderlyingType::String:
return getKeys<StringRef>(attribute);
case AttributeUnderlyingType::Decimal32: return getKeys<Decimal32>(attribute);
case AttributeUnderlyingType::Decimal64: return getKeys<Decimal64>(attribute);
case AttributeUnderlyingType::Decimal128: return getKeys<Decimal128>(attribute);
case AttributeUnderlyingType::Decimal32:
return getKeys<Decimal32>(attribute);
case AttributeUnderlyingType::Decimal64:
return getKeys<Decimal64>(attribute);
case AttributeUnderlyingType::Decimal128:
return getKeys<Decimal128>(attribute);
}
return {};
}
@ -663,13 +798,12 @@ BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};

View File

@ -1,30 +1,33 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <common/StringRef.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <variant>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <common/StringRef.h>
#include <ext/range.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
namespace DB
{
using BlockPtr = std::shared_ptr<Block>;
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<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -71,7 +71,31 @@ public:
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & 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<TYPE> & def, \
ResultArrayType<TYPE> & 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<TYPE> & def, ResultArrayType<TYPE> & out) const;
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const TYPE def, \
ResultArrayType<TYPE> & 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<TYPE> & 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<UInt8> & out) const;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value> using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value>
using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
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<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt8>,
ContainerType<UInt16>,
ContainerType<UInt32>,
ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> maps;
ContainerType<Int8>,
ContainerType<Int16>,
ContainerType<Int32>,
ContainerType<Int64>,
ContainerType<Decimal32>,
ContainerType<Decimal64>,
ContainerType<Decimal128>,
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>
maps;
std::unique_ptr<Arena> string_arena;
};
@ -188,18 +218,12 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
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 <typename T>
@ -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 <typename T>
void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray<UInt8> & out) const;

View File

@ -1,22 +1,21 @@
#pragma once
#include <Columns/ColumnVector.h>
#include <memory>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/IColumn.h>
#include <Core/Names.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/DataTypesNumber.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include "DictionaryBlockInputStreamBase.h"
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include <ext/range.h>
#include <common/logger_useful.h>
#include <Core/Names.h>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -32,28 +31,30 @@ class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
public:
using DictionaryPtr = std::shared_ptr<DictionaryType const>;
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
PaddedPODArray<Key> && ids, const Names & column_names);
DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size, PaddedPODArray<Key> && ids, const Names & column_names);
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const std::vector<StringRef> & keys, const Names & column_names);
DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary,
size_t max_block_size,
const std::vector<StringRef> & keys,
const Names & column_names);
using GetColumnsFunction =
std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & attributes)>;
using GetColumnsFunction = std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & 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<const IDictionaryBase> dictionary, size_t max_block_size,
const Columns & data_columns, const Names & column_names,
DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> 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<Key> &, PaddedPODArray<Type> &) const;
template <typename Type>
using DictionaryDecimalGetter =
void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &, DecimalPaddedPODArray<Type> &) const;
using DictionaryDecimalGetter
= void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &, DecimalPaddedPODArray<Type> &) const;
using DictionaryStringGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &, ColumnString *) const;
@ -75,61 +76,103 @@ private:
using GetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, PaddedPODArray<Type> & out) const;
template <typename Type>
using DecimalGetterByKey =
void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, DecimalPaddedPODArray<Type> & out) const;
using DecimalGetterByKey
= void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, DecimalPaddedPODArray<Type> & out) const;
using StringGetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, ColumnString * out) const;
// call getXXX
// for single key dictionaries
template <typename Type, typename Container>
void callGetter(DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
void callGetter(
DictionaryGetter<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <typename Type, typename Container>
void callGetter(DictionaryDecimalGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
void callGetter(
DictionaryDecimalGetter<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <typename Container>
void callGetter(DictionaryStringGetter getter, const PaddedPODArray<Key> & 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<Key> & 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 <typename Type, typename Container>
void callGetter(GetterByKey<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
void callGetter(
GetterByKey<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <typename Type, typename Container>
void callGetter(DecimalGetterByKey<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
void callGetter(
DecimalGetterByKey<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <typename Container>
void callGetter(StringGetterByKey getter, const PaddedPODArray<Key> & 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<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <template <typename> class Getter, template <typename> class DecimalGetter, typename StringGetter>
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill, const Columns & keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
Block
fillBlock(const PaddedPODArray<Key> & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const;
template <typename AttributeType, typename Getter>
ColumnPtr getColumnFromAttribute(Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
ColumnPtr getColumnFromAttribute(
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
template <typename Getter>
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
ColumnPtr getColumnFromStringAttribute(
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
const DictionaryAttribute & attribute,
const DictionaryType & dictionary) const;
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const;
void fillKeyColumns(const std::vector<StringRef> & keys, size_t start, size_t size,
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & columns) const;
void fillKeyColumns(
const std::vector<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
ColumnsWithTypeAndName & columns) const;
DictionaryPtr dictionary;
Names column_names;
@ -138,8 +181,7 @@ private:
Poco::Logger * logger;
using FillBlockFunction = Block (DictionaryBlockInputStream<DictionaryType, Key>::*)(
const PaddedPODArray<Key> & ids_to_fill, const Columns & keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
const PaddedPODArray<Key> & ids_to_fill, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const;
FillBlockFunction fill_block_function;
@ -160,27 +202,30 @@ private:
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
PaddedPODArray<Key> && ids, const Names & column_names)
: DictionaryBlockInputStreamBase(ids.size(), max_block_size),
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)),
column_names(column_names), ids(std::move(ids)),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fill_block_function(
&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<DictionaryGetter, DictionaryDecimalGetter, DictionaryStringGetter>),
key_type(DictionaryKeyType::Id)
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size, PaddedPODArray<Key> && ids, const Names & column_names)
: DictionaryBlockInputStreamBase(ids.size(), max_block_size)
, dictionary(std::static_pointer_cast<const DictionaryType>(dictionary))
, column_names(column_names)
, ids(std::move(ids))
, logger(&Poco::Logger::get("DictionaryBlockInputStream"))
, fill_block_function(
&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<DictionaryGetter, DictionaryDecimalGetter, DictionaryStringGetter>)
, key_type(DictionaryKeyType::Id)
{
}
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const std::vector<StringRef> & keys, const Names & column_names)
: DictionaryBlockInputStreamBase(keys.size(), max_block_size),
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, DecimalGetterByKey, StringGetterByKey>),
key_type(DictionaryKeyType::ComplexKey)
std::shared_ptr<const IDictionaryBase> dictionary,
size_t max_block_size,
const std::vector<StringRef> & keys,
const Names & column_names)
: DictionaryBlockInputStreamBase(keys.size(), max_block_size)
, dictionary(std::static_pointer_cast<const DictionaryType>(dictionary))
, column_names(column_names)
, logger(&Poco::Logger::get("DictionaryBlockInputStream"))
, fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, DecimalGetterByKey, StringGetterByKey>)
, key_type(DictionaryKeyType::ComplexKey)
{
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
fillKeyColumns(keys, 0, keys.size(), dictionaty_structure, key_columns);
@ -188,17 +233,21 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const Columns & data_columns, const Names & column_names,
std::shared_ptr<const IDictionaryBase> dictionary,
size_t max_block_size,
const Columns & data_columns,
const Names & column_names,
GetColumnsFunction && get_key_columns_function,
GetColumnsFunction && get_view_columns_function)
: DictionaryBlockInputStreamBase(data_columns.front()->size(), max_block_size),
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, DecimalGetterByKey, StringGetterByKey>),
data_columns(data_columns),
get_key_columns_function(get_key_columns_function), get_view_columns_function(get_view_columns_function),
key_type(DictionaryKeyType::Callback)
: DictionaryBlockInputStreamBase(data_columns.front()->size(), max_block_size)
, dictionary(std::static_pointer_cast<const DictionaryType>(dictionary))
, column_names(column_names)
, logger(&Poco::Logger::get("DictionaryBlockInputStream"))
, fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, DecimalGetterByKey, StringGetterByKey>)
, data_columns(data_columns)
, get_key_columns_function(get_key_columns_function)
, get_view_columns_function(get_view_columns_function)
, key_type(DictionaryKeyType::Callback)
{
}
@ -256,9 +305,13 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/, const DataTypes & /*data_types*/,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
DictionaryGetter<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/,
const DataTypes & /*data_types*/,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, ids_to_fill, container);
}
@ -266,9 +319,13 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryDecimalGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/, const DataTypes & /*data_types*/,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
DictionaryDecimalGetter<Type> getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/,
const DataTypes & /*data_types*/,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, ids_to_fill, container);
}
@ -276,9 +333,13 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DictionaryStringGetter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/, const DataTypes & /*data_types*/,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
DictionaryStringGetter getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & /*keys*/,
const DataTypes & /*data_types*/,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, ids_to_fill, container);
}
@ -286,9 +347,13 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
GetterByKey<Type> getter, const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
GetterByKey<Type> getter,
const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, keys, data_types, container);
}
@ -296,9 +361,13 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
DecimalGetterByKey<Type> getter, const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
DecimalGetterByKey<Type> getter,
const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, keys, data_types, container);
}
@ -306,9 +375,13 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
StringGetterByKey getter, const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dict) const
StringGetterByKey getter,
const PaddedPODArray<Key> & /*ids_to_fill*/,
const Columns & keys,
const DataTypes & data_types,
Container & container,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
(dict.*getter)(attribute.name, keys, data_types, container);
}
@ -346,8 +419,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
{
ColumnPtr column;
#define GET_COLUMN_FORM_ATTRIBUTE(TYPE) \
column = getColumnFromAttribute<TYPE, Getter<TYPE>>( \
&DictionaryType::get##TYPE, ids_to_fill, keys, data_types, attribute, *dictionary)
column = getColumnFromAttribute<TYPE, Getter<TYPE>>(&DictionaryType::get##TYPE, ids_to_fill, keys, data_types, attribute, *dictionary)
switch (attribute.underlying_type)
{
case AttributeUnderlyingType::UInt8:
@ -419,9 +491,12 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
template <typename DictionaryType, typename Key>
template <typename AttributeType, typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dict) const
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
if constexpr (IsDecimalNumber<AttributeType>)
{
@ -447,9 +522,12 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribut
template <typename DictionaryType, typename Key>
template <typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute(
Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dict) const
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys,
const DataTypes & data_types,
const DictionaryAttribute & attribute,
const DictionaryType & dict) const
{
auto column_string = ColumnString::create();
auto ptr = column_string.get();
@ -471,8 +549,11 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(cons
template <typename DictionaryType, typename Key>
void DictionaryBlockInputStream<DictionaryType, Key>::fillKeyColumns(
const std::vector<StringRef> & keys, size_t start, size_t size,
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & res) const
const std::vector<StringRef> & keys,
size_t start,
size_t size,
const DictionaryStructure & dictionary_structure,
ColumnsWithTypeAndName & res) const
{
MutableColumns columns;
columns.reserve(dictionary_structure.key->size());
@ -489,7 +570,8 @@ void DictionaryBlockInputStream<DictionaryType, Key>::fillKeyColumns(
}
for (size_t i = 0, num_columns = columns.size(); i < num_columns; ++i)
res.emplace_back(ColumnWithTypeAndName{ std::move(columns[i]), (*dictionary_structure.key)[i].type, (*dictionary_structure.key)[i].name });
res.emplace_back(
ColumnWithTypeAndName{std::move(columns[i]), (*dictionary_structure.key)[i].type, (*dictionary_structure.key)[i].name});
}
}

View File

@ -2,7 +2,6 @@
namespace DB
{
DictionaryBlockInputStreamBase::DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size)
: rows_count(rows_count), max_block_size(max_block_size)
{

View File

@ -4,7 +4,6 @@
namespace DB
{
class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream
{
protected:

View File

@ -1,16 +1,15 @@
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Block.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h>
#include "DictionaryStructure.h"
namespace DB
{
/// For simple key
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids)
{
@ -26,8 +25,11 @@ void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids)
}
/// For composite key
void formatKeys(const DictionaryStructure & dict_struct, BlockOutputStreamPtr & out,
const Columns & key_columns, const std::vector<size_t> & requested_rows)
void formatKeys(
const DictionaryStructure & dict_struct,
BlockOutputStreamPtr & out,
const Columns & key_columns,
const std::vector<size_t> & requested_rows)
{
Block block;
for (size_t i = 0, size = key_columns.size(); i < size; ++i)

View File

@ -1,13 +1,12 @@
#pragma once
#include <vector>
#include <common/Types.h>
#include <Columns/IColumn.h>
#include <common/Types.h>
namespace DB
{
class IBlockOutputStream;
using BlockOutputStreamPtr = std::shared_ptr<IBlockOutputStream>;
@ -19,7 +18,10 @@ struct DictionaryStructure;
void formatIDs(BlockOutputStreamPtr & out, const std::vector<UInt64> & ids);
/// For composite key
void formatKeys(const DictionaryStructure & dict_struct, BlockOutputStreamPtr & out,
const Columns & key_columns, const std::vector<size_t> & requested_rows);
void formatKeys(
const DictionaryStructure & dict_struct,
BlockOutputStreamPtr & out,
const Columns & key_columns,
const std::vector<size_t> & requested_rows);
}

View File

@ -1,20 +1,19 @@
#include "DictionaryStructure.h"
#include <Formats/FormatSettings.h>
#include <Columns/IColumn.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/IColumn.h>
#include <Common/StringUtils/StringUtils.h>
#include <Formats/FormatSettings.h>
#include <IO/WriteHelpers.h>
#include <Common/StringUtils/StringUtils.h>
#include <ext/range.h>
#include <numeric>
#include <unordered_set>
#include <unordered_map>
#include <unordered_set>
#include <ext/range.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
@ -26,9 +25,7 @@ namespace ErrorCodes
namespace
{
DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const std::string& default_type)
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_type)
{
const auto name = config.getString(config_prefix + ".name", "");
const auto expression = config.getString(config_prefix + ".expression", "");
@ -123,21 +120,36 @@ std::string toString(const AttributeUnderlyingType type)
{
switch (type)
{
case AttributeUnderlyingType::UInt8: return "UInt8";
case AttributeUnderlyingType::UInt16: return "UInt16";
case AttributeUnderlyingType::UInt32: return "UInt32";
case AttributeUnderlyingType::UInt64: return "UInt64";
case AttributeUnderlyingType::UInt128: return "UUID";
case AttributeUnderlyingType::Int8: return "Int8";
case AttributeUnderlyingType::Int16: return "Int16";
case AttributeUnderlyingType::Int32: return "Int32";
case AttributeUnderlyingType::Int64: return "Int64";
case AttributeUnderlyingType::Float32: return "Float32";
case AttributeUnderlyingType::Float64: return "Float64";
case AttributeUnderlyingType::Decimal32: return "Decimal32";
case AttributeUnderlyingType::Decimal64: return "Decimal64";
case AttributeUnderlyingType::Decimal128: return "Decimal128";
case AttributeUnderlyingType::String: return "String";
case AttributeUnderlyingType::UInt8:
return "UInt8";
case AttributeUnderlyingType::UInt16:
return "UInt16";
case AttributeUnderlyingType::UInt32:
return "UInt32";
case AttributeUnderlyingType::UInt64:
return "UInt64";
case AttributeUnderlyingType::UInt128:
return "UUID";
case AttributeUnderlyingType::Int8:
return "Int8";
case AttributeUnderlyingType::Int16:
return "Int16";
case AttributeUnderlyingType::Int32:
return "Int32";
case AttributeUnderlyingType::Int64:
return "Int64";
case AttributeUnderlyingType::Float32:
return "Float32";
case AttributeUnderlyingType::Float64:
return "Float64";
case AttributeUnderlyingType::Decimal32:
return "Decimal32";
case AttributeUnderlyingType::Decimal64:
return "Decimal64";
case AttributeUnderlyingType::Decimal128:
return "Decimal128";
case AttributeUnderlyingType::String:
return "String";
}
throw Exception{"Unknown attribute_type " + toString(static_cast<int>(type)), ErrorCodes::ARGUMENT_OUT_OF_BOUND};
@ -145,8 +157,7 @@ std::string toString(const AttributeUnderlyingType type)
DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
: name{config.getString(config_prefix + ".name", "")},
expression{config.getString(config_prefix + ".expression", "")}
: name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")}
{
if (name.empty() && !expression.empty())
throw Exception{"Element " + config_prefix + ".name is empty", ErrorCodes::BAD_ARGUMENTS};
@ -186,14 +197,18 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
if (range_min.has_value() != range_max.has_value())
{
throw Exception{"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.", ErrorCodes::BAD_ARGUMENTS};
throw Exception{"Dictionary structure should have both 'range_min' and 'range_max' either specified or not.",
ErrorCodes::BAD_ARGUMENTS};
}
if (range_min && range_max && !range_min->type->equals(*range_max->type))
{
throw Exception{"Dictionary structure 'range_min' and 'range_max' should have same type, "
"'range_min' type: " + range_min->type->getName() + ", "
"'range_max' type: " + range_max->type->getName(),
"'range_min' type: "
+ range_min->type->getName()
+ ", "
"'range_max' type: "
+ range_max->type->getName(),
ErrorCodes::BAD_ARGUMENTS};
}
@ -201,13 +216,12 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
{
if (!range_min->type->isValueRepresentedByInteger())
throw Exception{"Dictionary structure type of 'range_min' and 'range_max' should be an integer, Date, DateTime, or Enum."
" Actual 'range_min' and 'range_max' type is " + range_min->type->getName(),
" Actual 'range_min' and 'range_max' type is "
+ range_min->type->getName(),
ErrorCodes::BAD_ARGUMENTS};
}
if (!id->expression.empty() ||
(range_min && !range_min->expression.empty()) ||
(range_max && !range_max->expression.empty()))
if (!id->expression.empty() || (range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty()))
has_expressions = true;
}
@ -228,8 +242,9 @@ void DictionaryStructure::validateKeyTypes(const DataTypes & key_types) const
const auto & actual_type = key_types[i]->getName();
if (expected_type != actual_type)
throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type +
", found " + actual_type, ErrorCodes::TYPE_MISMATCH};
throw Exception{"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type + ", found "
+ actual_type,
ErrorCodes::TYPE_MISMATCH};
}
}
@ -274,15 +289,17 @@ bool DictionaryStructure::isKeySizeFixed() const
size_t DictionaryStructure::getKeySize() const
{
return std::accumulate(std::begin(*key), std::end(*key), size_t{},
[] (const auto running_size, const auto & key_i) {return running_size + key_i.type->getSizeOfValueInMemory(); });
return std::accumulate(std::begin(*key), std::end(*key), size_t{}, [](const auto running_size, const auto & key_i)
{
return running_size + key_i.type->getSizeOfValueInMemory();
});
}
static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & keys)
{
static const std::unordered_set<std::string> valid_keys =
{ "name", "type", "expression", "null_value", "hierarchical", "injective", "is_object_id" };
static const std::unordered_set<std::string> valid_keys
= {"name", "type", "expression", "null_value", "hierarchical", "injective", "is_object_id"};
for (const auto & key : keys)
{
@ -293,8 +310,10 @@ static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & k
std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
const bool hierarchy_allowed, const bool allow_null_values)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed,
const bool allow_null_values)
{
Poco::Util::AbstractConfiguration::Keys config_elems;
config.keys(config_prefix, config_elems);
@ -361,9 +380,8 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
has_hierarchy = has_hierarchy || hierarchical;
res_attributes.emplace_back(DictionaryAttribute{
name, underlying_type, type, expression, null_value, hierarchical, injective, is_object_id
});
res_attributes.emplace_back(
DictionaryAttribute{name, underlying_type, type, expression, null_value, hierarchical, injective, is_object_id});
}
return res_attributes;

View File

@ -5,15 +5,14 @@
#include <Interpreters/IExternalLoadable.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <vector>
#include <string>
#include <map>
#include <optional>
#include <string>
#include <vector>
namespace DB
{
enum class AttributeUnderlyingType
{
UInt8,
@ -104,8 +103,10 @@ struct DictionaryStructure final
private:
std::vector<DictionaryAttribute> getAttributes(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
const bool hierarchy_allowed = true, const bool allow_null_values = true);
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const bool hierarchy_allowed = true,
const bool allow_null_values = true);
};
}

View File

@ -4,8 +4,7 @@
#include "GeodataProviders/HierarchiesProvider.h"
#include "GeodataProviders/NamesProvider.h"
std::unique_ptr<RegionsHierarchies> GeoDictionariesLoader::reloadRegionsHierarchies(
const Poco::Util::AbstractConfiguration & config)
std::unique_ptr<RegionsHierarchies> GeoDictionariesLoader::reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config)
{
static constexpr auto config_key = "path_to_regions_hierarchy_file";
@ -17,8 +16,7 @@ std::unique_ptr<RegionsHierarchies> GeoDictionariesLoader::reloadRegionsHierarch
return std::make_unique<RegionsHierarchies>(std::move(data_provider));
}
std::unique_ptr<RegionsNames> GeoDictionariesLoader::reloadRegionsNames(
const Poco::Util::AbstractConfiguration & config)
std::unique_ptr<RegionsNames> GeoDictionariesLoader::reloadRegionsNames(const Poco::Util::AbstractConfiguration & config)
{
static constexpr auto config_key = "path_to_regions_names_files";

View File

@ -7,9 +7,7 @@
class GeoDictionariesLoader : public IGeoDictionariesLoader
{
public:
std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(
const Poco::Util::AbstractConfiguration & config) override;
std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config) override;
std::unique_ptr<RegionsNames> reloadRegionsNames(
const Poco::Util::AbstractConfiguration & config) override;
std::unique_ptr<RegionsNames> reloadRegionsNames(const Poco::Util::AbstractConfiguration & config) override;
};

View File

@ -1,7 +1,7 @@
#pragma once
#include "Types.h"
#include <string>
#include "Types.h"
struct RegionEntry
{
@ -17,4 +17,3 @@ struct RegionNameEntry
RegionID id;
std::string name;
};

View File

@ -1,10 +1,10 @@
#include "HierarchiesProvider.h"
#include "HierarchyFormatReader.h"
#include <IO/ReadBufferFromFile.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/Exception.h>
#include <Poco/Util/Application.h>
#include "HierarchyFormatReader.h"
bool RegionsHierarchyDataSource::isModified() const
@ -20,8 +20,7 @@ IRegionsHierarchyReaderPtr RegionsHierarchyDataSource::createReader()
}
RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string & path)
: path(path)
RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string & path) : path(path)
{
discoverFilesWithCustomHierarchies();
}
@ -37,9 +36,8 @@ void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies()
{
std::string candidate_basename = dir_it.path().getBaseName();
if ((0 == candidate_basename.compare(0, basename.size(), basename)) &&
(candidate_basename.size() > basename.size() + 1) &&
(candidate_basename[basename.size()] == '_'))
if ((0 == candidate_basename.compare(0, basename.size(), basename)) && (candidate_basename.size() > basename.size() + 1)
&& (candidate_basename[basename.size()] == '_'))
{
const std::string suffix = candidate_basename.substr(basename.size() + 1);
hierarchy_files.emplace(suffix, dir_it->path());

View File

@ -2,23 +2,19 @@
#include "IHierarchiesProvider.h"
#include <Common/FileUpdatesTracker.h>
#include <unordered_map>
#include <Common/FileUpdatesTracker.h>
// Represents local file with regions hierarchy dump
class RegionsHierarchyDataSource
: public IRegionsHierarchyDataSource
class RegionsHierarchyDataSource : public IRegionsHierarchyDataSource
{
private:
std::string path;
FileUpdatesTracker updates_tracker;
public:
RegionsHierarchyDataSource(const std::string & path_)
: path(path_)
, updates_tracker(path_)
{}
RegionsHierarchyDataSource(const std::string & path_) : path(path_), updates_tracker(path_) {}
bool isModified() const override;
@ -27,8 +23,7 @@ public:
// Provides access to directory with multiple data source files: one file per regions hierarchy
class RegionsHierarchiesDataProvider
: public IRegionsHierarchiesDataProvider
class RegionsHierarchiesDataProvider : public IRegionsHierarchiesDataProvider
{
private:
// path to file with default regions hierarchy
@ -55,4 +50,3 @@ public:
private:
void discoverFilesWithCustomHierarchies();
};

View File

@ -30,8 +30,7 @@ bool RegionsHierarchyFormatReader::readNext(RegionEntry & entry)
++input->position();
UInt64 population_big = 0;
DB::readIntText(population_big, *input);
population = population_big > std::numeric_limits<RegionPopulation>::max()
? std::numeric_limits<RegionPopulation>::max()
population = population_big > std::numeric_limits<RegionPopulation>::max() ? std::numeric_limits<RegionPopulation>::max()
: population_big;
}
DB::assertChar('\n', *input);

View File

@ -1,7 +1,7 @@
#pragma once
#include "IHierarchiesProvider.h"
#include <IO/ReadBuffer.h>
#include "IHierarchiesProvider.h"
// Reads regions hierarchy in geoexport format
@ -11,10 +11,7 @@ private:
DB::ReadBufferPtr input;
public:
RegionsHierarchyFormatReader(DB::ReadBufferPtr input_)
: input(std::move(input_))
{}
RegionsHierarchyFormatReader(DB::ReadBufferPtr input_) : input(std::move(input_)) {}
bool readNext(RegionEntry & entry) override;
};

View File

@ -1,9 +1,9 @@
#pragma once
#include "Entries.h"
#include <memory>
#include <string>
#include <vector>
#include "Entries.h"
// Iterates over all regions in data source
@ -46,4 +46,3 @@ public:
};
using IRegionsHierarchiesDataProviderPtr = std::shared_ptr<IRegionsHierarchiesDataProvider>;

View File

@ -1,7 +1,7 @@
#pragma once
#include "Entries.h"
#include <memory>
#include "Entries.h"
// Iterates over all name entries in data source
@ -42,11 +42,9 @@ using ILanguageRegionsNamesDataSourcePtr = std::unique_ptr<ILanguageRegionsNames
class IRegionsNamesDataProvider
{
public:
virtual ILanguageRegionsNamesDataSourcePtr getLanguageRegionsNamesSource(
const std::string & language) const = 0;
virtual ILanguageRegionsNamesDataSourcePtr getLanguageRegionsNamesSource(const std::string & language) const = 0;
virtual ~IRegionsNamesDataProvider() {}
};
using IRegionsNamesDataProviderPtr = std::unique_ptr<IRegionsNamesDataProvider>;

View File

@ -1,7 +1,7 @@
#pragma once
#include "INamesProvider.h"
#include <IO/ReadBuffer.h>
#include "INamesProvider.h"
// Reads regions names list in geoexport format
@ -11,9 +11,7 @@ private:
DB::ReadBufferPtr input;
public:
LanguageRegionsNamesFormatReader(DB::ReadBufferPtr input_)
: input(std::move(input_))
{}
LanguageRegionsNamesFormatReader(DB::ReadBufferPtr input_) : input(std::move(input_)) {}
bool readNext(RegionNameEntry & entry) override;
};

View File

@ -1,7 +1,7 @@
#include "NamesProvider.h"
#include "NamesFormatReader.h"
#include <IO/ReadBufferFromFile.h>
#include "NamesFormatReader.h"
bool LanguageRegionsNamesDataSource::isModified() const
@ -32,12 +32,11 @@ std::string LanguageRegionsNamesDataSource::getSourceName() const
}
RegionsNamesDataProvider::RegionsNamesDataProvider(const std::string & directory_)
: directory(directory_)
{}
RegionsNamesDataProvider::RegionsNamesDataProvider(const std::string & directory_) : directory(directory_)
{
}
ILanguageRegionsNamesDataSourcePtr RegionsNamesDataProvider::getLanguageRegionsNamesSource(
const std::string & language) const
ILanguageRegionsNamesDataSourcePtr RegionsNamesDataProvider::getLanguageRegionsNamesSource(const std::string & language) const
{
const auto data_file = getDataFilePath(language);
return std::make_unique<LanguageRegionsNamesDataSource>(data_file, language);

View File

@ -1,7 +1,7 @@
#pragma once
#include "INamesProvider.h"
#include <Common/FileUpdatesTracker.h>
#include "INamesProvider.h"
// Represents local file with list of regions ids / names
@ -14,10 +14,9 @@ private:
public:
LanguageRegionsNamesDataSource(const std::string & path_, const std::string & language_)
: path(path_)
, updates_tracker(path_)
, language(language_)
{}
: path(path_), updates_tracker(path_), language(language_)
{
}
bool isModified() const override;
@ -42,8 +41,7 @@ private:
public:
RegionsNamesDataProvider(const std::string & directory_);
ILanguageRegionsNamesDataSourcePtr getLanguageRegionsNamesSource(
const std::string & language) const override;
ILanguageRegionsNamesDataSourcePtr getLanguageRegionsNamesSource(const std::string & language) const override;
private:
std::string getDataFilePath(const std::string & language) const;

View File

@ -1,8 +1,8 @@
#pragma once
#include <memory>
#include "RegionsHierarchies.h"
#include "RegionsNames.h"
#include <memory>
namespace Poco
{
@ -20,11 +20,9 @@ namespace Poco
class IGeoDictionariesLoader
{
public:
virtual std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(
const Poco::Util::AbstractConfiguration & config) = 0;
virtual std::unique_ptr<RegionsHierarchies> reloadRegionsHierarchies(const Poco::Util::AbstractConfiguration & config) = 0;
virtual std::unique_ptr<RegionsNames> reloadRegionsNames(
const Poco::Util::AbstractConfiguration & config) = 0;
virtual std::unique_ptr<RegionsNames> reloadRegionsNames(const Poco::Util::AbstractConfiguration & config) = 0;
virtual ~IGeoDictionariesLoader() {}
};

View File

@ -1,7 +1,7 @@
#include "RegionsHierarchies.h"
#include <common/logger_useful.h>
#include <Poco/DirectoryIterator.h>
#include <common/logger_useful.h>
RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider)

View File

@ -1,9 +1,9 @@
#pragma once
#include "RegionsHierarchy.h"
#include "GeodataProviders/IHierarchiesProvider.h"
#include <Poco/Exception.h>
#include <unordered_map>
#include <Poco/Exception.h>
#include "GeodataProviders/IHierarchiesProvider.h"
#include "RegionsHierarchy.h"
/** Contains several hierarchies of regions.

View File

@ -1,11 +1,11 @@
#include "RegionsHierarchy.h"
#include "GeodataProviders/IHierarchiesProvider.h"
#include <Poco/Util/Application.h>
#include <IO/WriteHelpers.h>
#include <Poco/Exception.h>
#include <Poco/Util/Application.h>
#include <common/logger_useful.h>
#include <ext/singleton.h>
#include <IO/WriteHelpers.h>
#include "GeodataProviders/IHierarchiesProvider.h"
namespace DB
@ -17,8 +17,7 @@ namespace DB
}
RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_)
: data_source(data_source_)
RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_) : data_source(data_source_)
{
}
@ -56,7 +55,8 @@ void RegionsHierarchy::reload()
if (region_entry.id > max_region_id)
{
if (region_entry.id > max_size)
throw DB::Exception("Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size),
throw DB::Exception(
"Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
max_region_id = region_entry.id;
@ -113,14 +113,16 @@ void RegionsHierarchy::reload()
++depth;
if (depth == std::numeric_limits<RegionDepth>::max())
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " possible is inside infinite loop");
throw Poco::Exception(
"Logical error in regions hierarchy: region " + DB::toString(current) + " possible is inside infinite loop");
current = new_parents[current];
if (current == 0)
break;
if (current > max_region_id)
throw Poco::Exception("Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist");
throw Poco::Exception(
"Logical error in regions hierarchy: region " + DB::toString(current) + " (specified as parent) doesn't exist");
if (types[current] == RegionType::City)
new_city[i] = current;

View File

@ -1,9 +1,9 @@
#pragma once
#include "GeodataProviders/IHierarchiesProvider.h"
#include <vector>
#include <boost/noncopyable.hpp>
#include <common/Types.h>
#include "GeodataProviders/IHierarchiesProvider.h"
class IRegionsHierarchyDataProvider;

View File

@ -1,10 +1,10 @@
#include "RegionsNames.h"
#include "GeodataProviders/INamesProvider.h"
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <common/logger_useful.h>
#include <IO/WriteHelpers.h>
#include <Poco/Exception.h>
#include <Poco/Util/Application.h>
#include <common/logger_useful.h>
#include "GeodataProviders/INamesProvider.h"
namespace DB
{
@ -84,7 +84,8 @@ void RegionsNames::reload()
max_region_id = name_entry.id;
if (name_entry.id > max_size)
throw DB::Exception("Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size),
throw DB::Exception(
"Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
}

View File

@ -1,11 +1,11 @@
#pragma once
#include "GeodataProviders/INamesProvider.h"
#include <Poco/Exception.h>
#include <common/Types.h>
#include <common/StringRef.h>
#include <string>
#include <vector>
#include <Poco/Exception.h>
#include <common/StringRef.h>
#include <common/Types.h>
#include "GeodataProviders/INamesProvider.h"
/** A class that allows you to recognize by region id its text name in one of the supported languages: ru, en, ua, by, kz, tr.
@ -41,19 +41,20 @@ private:
return res;
}
struct language_alias { const char * const name; const Language lang; };
struct language_alias
{
const char * const name;
const Language lang;
};
static const language_alias * getLanguageAliases()
{
static constexpr const language_alias language_aliases[]
{
{ "ru", Language::RU },
static constexpr const language_alias language_aliases[]{{"ru", Language::RU},
{"en", Language::EN},
{"ua", Language::UA},
{"uk", Language::UA},
{"by", Language::BY},
{"kz", Language::KZ},
{ "tr", Language::TR }
};
{"tr", Language::TR}};
return language_aliases;
}

View File

@ -49,15 +49,9 @@ public:
}
UInt8 OSToParent(UInt8 x) const
{
return os_parent[x];
}
UInt8 OSToParent(UInt8 x) const { return os_parent[x]; }
UInt8 SEToParent(UInt8 x) const
{
return se_parent[x];
}
UInt8 SEToParent(UInt8 x) const { return se_parent[x]; }
/// To the topmost ancestor.
@ -77,4 +71,6 @@ public:
};
class TechDataHierarchySingleton : public ext::singleton<TechDataHierarchySingleton>, public TechDataHierarchy {};
class TechDataHierarchySingleton : public ext::singleton<TechDataHierarchySingleton>, public TechDataHierarchy
{
};

View File

@ -1,26 +1,24 @@
#include "ExecutableDictionarySource.h"
#include <thread>
#include <future>
#include <Common/ShellCommand.h>
#include <Interpreters/Context.h>
#include <DataStreams/OwningBlockInputStream.h>
#include "DictionarySourceHelpers.h"
#include <thread>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Common/ShellCommand.h>
#include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
namespace DB
{
static const size_t max_block_size = 8192;
namespace
{
/// Owns ShellCommand and calls wait for it.
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
{
@ -40,29 +38,32 @@ public:
}
ExecutableDictionarySource::ExecutableDictionarySource(const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
Block & sample_block, const Context & context)
: log(&Logger::get("ExecutableDictionarySource")),
update_time{std::chrono::system_clock::from_time_t(0)},
dict_struct{dict_struct_},
command{config.getString(config_prefix + ".command")},
update_field{config.getString(config_prefix + ".update_field", "")},
format{config.getString(config_prefix + ".format")},
sample_block{sample_block},
context(context)
ExecutableDictionarySource::ExecutableDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context)
: log(&Logger::get("ExecutableDictionarySource"))
, update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, command{config.getString(config_prefix + ".command")}
, update_field{config.getString(config_prefix + ".update_field", "")}
, format{config.getString(config_prefix + ".format")}
, sample_block{sample_block}
, context(context)
{
}
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
: log(&Logger::get("ExecutableDictionarySource")),
update_time{other.update_time},
dict_struct{other.dict_struct},
command{other.command},
update_field{other.update_field},
format{other.format},
sample_block{other.sample_block},
context(other.context)
: log(&Logger::get("ExecutableDictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, command{other.command}
, update_field{other.update_field}
, format{other.format}
, sample_block{other.sample_block}
, context(other.context)
{
}
@ -110,7 +111,6 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
namespace
{
/** A stream, that also runs and waits for background thread
* (that will feed data into pipe to be read from the other side of the pipe).
*/
@ -118,10 +118,11 @@ class BlockInputStreamWithBackgroundThread final : public IProfilingBlockInputSt
{
public:
BlockInputStreamWithBackgroundThread(
const BlockInputStreamPtr & stream_, std::unique_ptr<ShellCommand> && command_,
std::packaged_task<void()> && task_)
: stream{stream_}, command{std::move(command_)}, task(std::move(task_)),
thread([this]{ task(); command->in.close(); })
const BlockInputStreamPtr & stream_, std::unique_ptr<ShellCommand> && command_, std::packaged_task<void()> && task_)
: stream{stream_}, command{std::move(command_)}, task(std::move(task_)), thread([this] {
task();
command->in.close();
})
{
children.push_back(stream);
}
@ -180,15 +181,10 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<BlockInputStreamWithBackgroundThread>(
input_stream, std::move(process), std::packaged_task<void()>(
[output_stream, &ids]() mutable
{
formatIDs(output_stream, ids);
}));
input_stream, std::move(process), std::packaged_task<void()>([output_stream, &ids]() mutable { formatIDs(output_stream, ids); }));
}
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
auto process = ShellCommand::execute(command);
@ -197,8 +193,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadKeys(
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
return std::make_shared<BlockInputStreamWithBackgroundThread>(
input_stream, std::move(process), std::packaged_task<void()>(
[output_stream, key_columns, &requested_rows, this]() mutable
input_stream, std::move(process), std::packaged_task<void()>([output_stream, key_columns, &requested_rows, this]() mutable
{
formatKeys(dict_struct, output_stream, key_columns, requested_rows);
}));

View File

@ -1,15 +1,17 @@
#pragma once
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include "IDictionarySource.h"
namespace Poco { class Logger; }
namespace Poco
{
class Logger;
}
namespace DB
{
/// Allows loading dictionaries from executable
class ExecutableDictionarySource final : public IDictionarySource
{
@ -29,8 +31,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override;

View File

@ -1,16 +1,15 @@
#include <ext/range.h>
#include <boost/range/join.hpp>
#include "ExternalQueryBuilder.h"
#include <IO/WriteBuffer.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include "writeParenthesisedString.h"
#include <boost/range/join.hpp>
#include <ext/range.h>
#include "DictionaryStructure.h"
#include "ExternalQueryBuilder.h"
#include "writeParenthesisedString.h"
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
@ -238,10 +237,8 @@ std::string ExternalQueryBuilder::composeLoadIdsQuery(const std::vector<UInt64>
}
std::string ExternalQueryBuilder::composeLoadKeysQuery(
const Columns & key_columns,
const std::vector<size_t> & requested_rows,
LoadKeysMethod method)
std::string
ExternalQueryBuilder::composeLoadKeysQuery(const Columns & key_columns, const std::vector<size_t> & requested_rows, LoadKeysMethod method)
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};

View File

@ -1,14 +1,13 @@
#pragma once
#include <string>
#include <Formats/FormatSettings.h>
#include <Columns/IColumn.h>
#include <Formats/FormatSettings.h>
#include <Parsers/IdentifierQuotingStyle.h>
namespace DB
{
struct DictionaryStructure;
class WriteBuffer;
@ -53,10 +52,7 @@ struct ExternalQueryBuilder
IN_WITH_TUPLES,
};
std::string composeLoadKeysQuery(
const Columns & key_columns,
const std::vector<size_t> & requested_rows,
LoadKeysMethod method);
std::string composeLoadKeysQuery(const Columns & key_columns, const std::vector<size_t> & requested_rows, LoadKeysMethod method);
private:

View File

@ -1,17 +1,16 @@
#include <ext/range.h>
#include "ExternalResultDescription.h"
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;

View File

@ -5,7 +5,6 @@
namespace DB
{
/** Common part for implementation of MySQLBlockInputStream, MongoDBBlockInputStream and others.
*/
struct ExternalResultDescription

View File

@ -1,36 +1,38 @@
#include "FileDictionarySource.h"
#include <Interpreters/Context.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB
{
static const size_t max_block_size = 8192;
FileDictionarySource::FileDictionarySource(const std::string & filename, const std::string & format, Block & sample_block,
const Context & context)
FileDictionarySource::FileDictionarySource(
const std::string & filename, const std::string & format, Block & sample_block, const Context & context)
: filename{filename}, format{format}, sample_block{sample_block}, context(context)
{}
{
}
FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
: filename{other.filename}, format{other.format},
sample_block{other.sample_block}, context(other.context),
last_modification{other.last_modification}
{}
: filename{other.filename}
, format{other.format}
, sample_block{other.sample_block}
, context(other.context)
, last_modification{other.last_modification}
{
}
BlockInputStreamPtr FileDictionarySource::loadAll()
{
auto in_ptr = std::make_unique<ReadBufferFromFile>(filename);
auto stream = context.getInputFormat(
format, *in_ptr, sample_block, max_block_size);
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
last_modification = getLastModification();
return std::make_shared<OwningBlockInputStream<ReadBuffer>>(stream, std::move(in_ptr));

View File

@ -1,20 +1,18 @@
#pragma once
#include "IDictionarySource.h"
#include <Poco/Timestamp.h>
#include "IDictionarySource.h"
namespace DB
{
class Context;
/// Allows loading dictionaries from a file with given format, does not support "random access"
class FileDictionarySource final : public IDictionarySource
{
public:
FileDictionarySource(const std::string & filename, const std::string & format, Block & sample_block,
const Context & context);
FileDictionarySource(const std::string & filename, const std::string & format, Block & sample_block, const Context & context);
FileDictionarySource(const FileDictionarySource & other);
@ -30,8 +28,7 @@ public:
throw Exception{"Method loadIds is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
}
BlockInputStreamPtr loadKeys(
const Columns & /*key_columns*/, const std::vector<size_t> & /*requested_rows*/) override
BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector<size_t> & /*requested_rows*/) override
{
throw Exception{"Method loadKeys is unsupported for FileDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
}

View File

@ -1,11 +1,10 @@
#include "FlatDictionary.h"
#include "DictionaryBlockInputStream.h"
#include <IO/WriteHelpers.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
@ -21,12 +20,20 @@ static const auto initial_array_size = 1024;
static const auto max_array_size = 500000;
FlatDictionary::FlatDictionary(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),
loaded_ids(initial_array_size, false), saved_block{std::move(saved_block)}
FlatDictionary::FlatDictionary(
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)
, loaded_ids(initial_array_size, false)
, saved_block{std::move(saved_block)}
{
createAttributes();
@ -44,7 +51,8 @@ FlatDictionary::FlatDictionary(const std::string & name, const DictionaryStructu
}
FlatDictionary::FlatDictionary(const FlatDictionary & other)
: FlatDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block}
: FlatDictionary{
other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block}
{
}
@ -53,21 +61,26 @@ void FlatDictionary::toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Ke
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
getItemsNumber<UInt64>(*hierarchical_attribute, ids,
getItemsNumber<UInt64>(
*hierarchical_attribute,
ids,
[&](const size_t row, const UInt64 value) { out[row] = value; },
[&](const size_t) { return null_value; });
}
/// Allow to use single value in same way as array.
static inline FlatDictionary::Key getAt(const PaddedPODArray<FlatDictionary::Key> & arr, const size_t idx) { return arr[idx]; }
static inline FlatDictionary::Key getAt(const FlatDictionary::Key & value, const size_t) { return value; }
static inline FlatDictionary::Key getAt(const PaddedPODArray<FlatDictionary::Key> & arr, const size_t idx)
{
return arr[idx];
}
static inline FlatDictionary::Key getAt(const FlatDictionary::Key & value, const size_t)
{
return value;
}
template <typename ChildType, typename AncestorType>
void FlatDictionary::isInImpl(
const ChildType & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void FlatDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
const auto & attr = std::get<ContainerType<Key>>(hierarchical_attribute->arrays);
@ -90,25 +103,17 @@ void FlatDictionary::isInImpl(
void FlatDictionary::isInVectorVector(
const PaddedPODArray<Key> & child_ids,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_ids, out);
}
void FlatDictionary::isInVectorConstant(
const PaddedPODArray<Key> & child_ids,
const Key ancestor_id,
PaddedPODArray<UInt8> & out) const
void FlatDictionary::isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_id, out);
}
void FlatDictionary::isInConstantVector(
const Key child_id,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void FlatDictionary::isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_id, ancestor_ids, out);
}
@ -119,13 +124,13 @@ void FlatDictionary::get##TYPE(const std::string & attribute_name, const PaddedP
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return null_value; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return null_value; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -147,27 +152,32 @@ void FlatDictionary::getString(const std::string & attribute_name, const PaddedP
{
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 = std::get<StringRef>(attribute.null_values);
getItemsImpl<StringRef, StringRef>(attribute, ids,
getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return null_value; });
}
#define DECLARE(TYPE) \
void FlatDictionary::get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t row) { return def[row]; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -186,14 +196,16 @@ DECLARE(Decimal128)
#undef DECLARE
void FlatDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
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<StringRef, StringRef>(attribute, ids,
getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t row) { return def->getDataAt(row); });
}
@ -204,11 +216,11 @@ void FlatDictionary::get##TYPE(\
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return def; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -227,14 +239,16 @@ DECLARE(Decimal128)
#undef DECLARE
void FlatDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
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};
FlatDictionary::getItemsImpl<StringRef, StringRef>(attribute, ids,
FlatDictionary::getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return StringRef{def}; });
}
@ -246,22 +260,52 @@ void FlatDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8>
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt128: has<UInt128>(attribute, ids, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, ids, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, ids, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, ids, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, ids, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, ids, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, ids, out); break;
case AttributeUnderlyingType::String: has<String>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt8:
has<UInt8>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt16:
has<UInt16>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt32:
has<UInt32>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt64:
has<UInt64>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt128:
has<UInt128>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int8:
has<Int8>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int16:
has<Int16>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int32:
has<Int32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int64:
has<Int64>(attribute, ids, out);
break;
case AttributeUnderlyingType::Float32:
has<Float32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Float64:
has<Float64>(attribute, ids, out);
break;
case AttributeUnderlyingType::String:
has<String>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal32: has<Decimal32>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal64: has<Decimal64>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal128: has<Decimal128>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal32:
has<Decimal32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal64:
has<Decimal64>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal128:
has<Decimal128>(attribute, ids, out);
break;
}
}
@ -409,21 +453,49 @@ void FlatDictionary::calculateBytesAllocated()
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::UInt128: addAttributeSize<UInt128>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::UInt8:
addAttributeSize<UInt8>(attribute);
break;
case AttributeUnderlyingType::UInt16:
addAttributeSize<UInt16>(attribute);
break;
case AttributeUnderlyingType::UInt32:
addAttributeSize<UInt32>(attribute);
break;
case AttributeUnderlyingType::UInt64:
addAttributeSize<UInt64>(attribute);
break;
case AttributeUnderlyingType::UInt128:
addAttributeSize<UInt128>(attribute);
break;
case AttributeUnderlyingType::Int8:
addAttributeSize<Int8>(attribute);
break;
case AttributeUnderlyingType::Int16:
addAttributeSize<Int16>(attribute);
break;
case AttributeUnderlyingType::Int32:
addAttributeSize<Int32>(attribute);
break;
case AttributeUnderlyingType::Int64:
addAttributeSize<Int64>(attribute);
break;
case AttributeUnderlyingType::Float32:
addAttributeSize<Float32>(attribute);
break;
case AttributeUnderlyingType::Float64:
addAttributeSize<Float64>(attribute);
break;
case AttributeUnderlyingType::Decimal32: addAttributeSize<Decimal32>(attribute); break;
case AttributeUnderlyingType::Decimal64: addAttributeSize<Decimal64>(attribute); break;
case AttributeUnderlyingType::Decimal128: addAttributeSize<Decimal128>(attribute); break;
case AttributeUnderlyingType::Decimal32:
addAttributeSize<Decimal32>(attribute);
break;
case AttributeUnderlyingType::Decimal64:
addAttributeSize<Decimal64>(attribute);
break;
case AttributeUnderlyingType::Decimal128:
addAttributeSize<Decimal128>(attribute);
break;
case AttributeUnderlyingType::String:
{
@ -462,22 +534,52 @@ FlatDictionary::Attribute FlatDictionary::createAttributeWithType(const Attribut
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::UInt128: createAttributeImpl<UInt128>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::String: createAttributeImpl<String>(attr, null_value); break;
case AttributeUnderlyingType::UInt8:
createAttributeImpl<UInt8>(attr, null_value);
break;
case AttributeUnderlyingType::UInt16:
createAttributeImpl<UInt16>(attr, null_value);
break;
case AttributeUnderlyingType::UInt32:
createAttributeImpl<UInt32>(attr, null_value);
break;
case AttributeUnderlyingType::UInt64:
createAttributeImpl<UInt64>(attr, null_value);
break;
case AttributeUnderlyingType::UInt128:
createAttributeImpl<UInt128>(attr, null_value);
break;
case AttributeUnderlyingType::Int8:
createAttributeImpl<Int8>(attr, null_value);
break;
case AttributeUnderlyingType::Int16:
createAttributeImpl<Int16>(attr, null_value);
break;
case AttributeUnderlyingType::Int32:
createAttributeImpl<Int32>(attr, null_value);
break;
case AttributeUnderlyingType::Int64:
createAttributeImpl<Int64>(attr, null_value);
break;
case AttributeUnderlyingType::Float32:
createAttributeImpl<Float32>(attr, null_value);
break;
case AttributeUnderlyingType::Float64:
createAttributeImpl<Float64>(attr, null_value);
break;
case AttributeUnderlyingType::String:
createAttributeImpl<String>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal32: createAttributeImpl<Decimal32>(attr, null_value); break;
case AttributeUnderlyingType::Decimal64: createAttributeImpl<Decimal64>(attr, null_value); break;
case AttributeUnderlyingType::Decimal128: createAttributeImpl<Decimal128>(attr, null_value); break;
case AttributeUnderlyingType::Decimal32:
createAttributeImpl<Decimal32>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal64:
createAttributeImpl<Decimal64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal128:
createAttributeImpl<Decimal128>(attr, null_value);
break;
}
return attr;
@ -486,12 +588,11 @@ FlatDictionary::Attribute FlatDictionary::createAttributeWithType(const Attribut
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void FlatDictionary::getItemsNumber(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const
{
if (false) {}
if (false)
{
}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsImpl<TYPE, OutputType>(attribute, ids, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(get_default));
@ -510,17 +611,13 @@ void FlatDictionary::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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void FlatDictionary::getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const
{
const auto & attr = std::get<ContainerType<AttributeType>>(attribute.arrays);
const auto rows = ext::size(ids);
@ -572,22 +669,52 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, cons
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt128: setAttributeValueImpl<UInt128>(attribute, id, value.get<UInt128>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::String: setAttributeValueImpl<String>(attribute, id, value.get<String>()); break;
case AttributeUnderlyingType::UInt8:
setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt16:
setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt32:
setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt64:
setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt128:
setAttributeValueImpl<UInt128>(attribute, id, value.get<UInt128>());
break;
case AttributeUnderlyingType::Int8:
setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int16:
setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int32:
setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int64:
setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Float32:
setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>());
break;
case AttributeUnderlyingType::Float64:
setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>());
break;
case AttributeUnderlyingType::String:
setAttributeValueImpl<String>(attribute, id, value.get<String>());
break;
case AttributeUnderlyingType::Decimal32: setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal128>()); break;
case AttributeUnderlyingType::Decimal64: setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal128>()); break;
case AttributeUnderlyingType::Decimal128: setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>()); break;
case AttributeUnderlyingType::Decimal32:
setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal128>());
break;
case AttributeUnderlyingType::Decimal64:
setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal128>());
break;
case AttributeUnderlyingType::Decimal128:
setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>());
break;
}
}
@ -636,14 +763,12 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam
void registerDictionaryFlat(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'flat'", ErrorCodes::UNSUPPORTED_METHOD};
@ -655,12 +780,9 @@ void registerDictionaryFlat(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("flat", create_layout);
}
}

View File

@ -1,28 +1,32 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <atomic>
#include <variant>
#include <vector>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <ext/range.h>
#include <ext/size.h>
#include <atomic>
#include <vector>
#include <variant>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
namespace DB
{
using BlockPtr = std::shared_ptr<Block>;
class FlatDictionary final : public IDictionary
{
public:
FlatDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty, BlockPtr saved_block = nullptr);
FlatDictionary(
const std::string & name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty,
BlockPtr saved_block = nullptr);
FlatDictionary(const FlatDictionary & other);
@ -52,10 +56,7 @@ public:
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -66,7 +67,8 @@ public:
void toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<Key> & out) const override;
void isInVectorVector(const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
@ -95,7 +97,9 @@ public:
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -113,14 +117,12 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const;
void
getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out)
const;
#define DECLARE(TYPE) \
void get##TYPE(\
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def,\
ResultArrayType<TYPE> & out) const;
void get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
@ -137,34 +139,53 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const;
void getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const;
void has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8> & out) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value> using ContainerType = PaddedPODArray<Value>;
template <typename Value>
using ContainerType = PaddedPODArray<Value>;
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,
StringRef> null_values;
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
StringRef>
null_values;
std::variant<
ContainerType<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt8>,
ContainerType<UInt16>,
ContainerType<UInt32>,
ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> arrays;
ContainerType<Int8>,
ContainerType<Int16>,
ContainerType<Int32>,
ContainerType<Int64>,
ContainerType<Decimal32>,
ContainerType<Decimal64>,
ContainerType<Decimal128>,
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>
arrays;
std::unique_ptr<Arena> string_arena;
};
@ -185,17 +206,11 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsNumber(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const;
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const;
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const;
template <typename T>
void resize(Attribute & attribute, const Key id);
@ -211,10 +226,7 @@ private:
void has(const Attribute & attribute, const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8> & out) const;
template <typename ChildType, typename AncestorType>
void isInImpl(
const ChildType & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const;
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
PaddedPODArray<Key> getIds() const;

View File

@ -1,49 +1,51 @@
#include "HTTPDictionarySource.h"
#include <Poco/Net/HTTPRequest.h>
#include <Interpreters/Context.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <DataStreams/IBlockOutputStream.h>
#include <IO/WriteBufferFromOStream.h>
#include "DictionarySourceHelpers.h"
#include <common/logger_useful.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/WriteBufferFromOStream.h>
#include <Interpreters/Context.h>
#include <Poco/Net/HTTPRequest.h>
#include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionarySourceHelpers.h"
#include "DictionaryStructure.h"
namespace DB
{
static const size_t max_block_size = 8192;
HTTPDictionarySource::HTTPDictionarySource(const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
Block & sample_block, const Context & context)
: log(&Logger::get("HTTPDictionarySource")),
update_time{std::chrono::system_clock::from_time_t(0)},
dict_struct{dict_struct_},
url{config.getString(config_prefix + ".url", "")},
update_field{config.getString(config_prefix + ".update_field", "")},
format{config.getString(config_prefix + ".format")},
sample_block{sample_block},
context(context),
timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef()))
HTTPDictionarySource::HTTPDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context)
: log(&Logger::get("HTTPDictionarySource"))
, update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, url{config.getString(config_prefix + ".url", "")}
, update_field{config.getString(config_prefix + ".update_field", "")}
, format{config.getString(config_prefix + ".format")}
, sample_block{sample_block}
, context(context)
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef()))
{
}
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
: log(&Logger::get("HTTPDictionarySource")),
update_time{other.update_time},
dict_struct{other.dict_struct},
url{other.url},
update_field{other.update_field},
format{other.format},
sample_block{other.sample_block},
context(other.context),
timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef()))
: log(&Logger::get("HTTPDictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, url{other.url}
, update_field{other.update_field}
, format{other.format}
, sample_block{other.sample_block}
, context(other.context)
, timeouts(ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef()))
{
}
@ -72,8 +74,8 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll()
{
LOG_TRACE(log, "loadAll " + toString());
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
@ -83,8 +85,8 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
Poco::URI uri(url);
getUpdateFieldAndDate(uri);
LOG_TRACE(log, "loadUpdatedAll " + uri.toString());
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_GET,
ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
@ -101,14 +103,12 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & id
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST,
out_stream_callback, timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}
BlockInputStreamPtr HTTPDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
@ -120,8 +120,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(
};
Poco::URI uri(url);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST,
out_stream_callback, timeouts);
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, out_stream_callback, timeouts);
auto input_stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
return std::make_shared<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(input_stream, std::move(in_ptr));
}

View File

@ -1,22 +1,25 @@
#pragma once
#include <Poco/URI.h>
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <common/LocalDateTime.h>
#include <IO/ConnectionTimeouts.h>
#include <Poco/URI.h>
#include <common/LocalDateTime.h>
#include "DictionaryStructure.h"
#include "IDictionarySource.h"
namespace Poco { class Logger; }
namespace Poco
{
class Logger;
}
namespace DB
{
/// Allows loading dictionaries from http[s] source
class HTTPDictionarySource final : public IDictionarySource
{
public:
HTTPDictionarySource(const DictionaryStructure & dict_struct_,
HTTPDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -30,8 +33,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override;

View File

@ -1,11 +1,10 @@
#include <ext/size.h>
#include "HashedDictionary.h"
#include <ext/size.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
@ -16,10 +15,19 @@ namespace ErrorCodes
}
HashedDictionary::HashedDictionary(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)}
HashedDictionary::HashedDictionary(
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();
@ -37,7 +45,8 @@ HashedDictionary::HashedDictionary(const std::string & name, const DictionaryStr
}
HashedDictionary::HashedDictionary(const HashedDictionary & other)
: HashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block}
: HashedDictionary{
other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block}
{
}
@ -46,21 +55,26 @@ void HashedDictionary::toParent(const PaddedPODArray<Key> & ids, PaddedPODArray<
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
getItemsNumber<UInt64>(*hierarchical_attribute, ids,
getItemsNumber<UInt64>(
*hierarchical_attribute,
ids,
[&](const size_t row, const UInt64 value) { out[row] = value; },
[&](const size_t) { return null_value; });
}
/// Allow to use single value in same way as array.
static inline HashedDictionary::Key getAt(const PaddedPODArray<HashedDictionary::Key> & arr, const size_t idx) { return arr[idx]; }
static inline HashedDictionary::Key getAt(const HashedDictionary::Key & value, const size_t) { return value; }
static inline HashedDictionary::Key getAt(const PaddedPODArray<HashedDictionary::Key> & arr, const size_t idx)
{
return arr[idx];
}
static inline HashedDictionary::Key getAt(const HashedDictionary::Key & value, const size_t)
{
return value;
}
template <typename ChildType, typename AncestorType>
void HashedDictionary::isInImpl(
const ChildType & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
const auto & attr = *std::get<CollectionPtrType<Key>>(hierarchical_attribute->maps);
@ -87,42 +101,35 @@ void HashedDictionary::isInImpl(
}
void HashedDictionary::isInVectorVector(
const PaddedPODArray<Key> & child_ids,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_ids, out);
}
void HashedDictionary::isInVectorConstant(
const PaddedPODArray<Key> & child_ids,
const Key ancestor_id,
PaddedPODArray<UInt8> & out) const
void HashedDictionary::isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_ids, ancestor_id, out);
}
void HashedDictionary::isInConstantVector(
const Key child_id,
const PaddedPODArray<Key> & ancestor_ids,
PaddedPODArray<UInt8> & out) const
void HashedDictionary::isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const
{
isInImpl(child_id, ancestor_ids, out);
}
#define DECLARE(TYPE) \
void HashedDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const\
void HashedDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) \
const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return null_value; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return null_value; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -144,27 +151,32 @@ void HashedDictionary::getString(const std::string & attribute_name, const Padde
{
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<String>(attribute.null_values)};
getItemsImpl<StringRef, StringRef>(attribute, ids,
getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return null_value; });
}
#define DECLARE(TYPE) \
void HashedDictionary::get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t row) { return def[row]; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t row) { return def[row]; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -183,14 +195,16 @@ DECLARE(Decimal128)
#undef DECLARE
void HashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out) const
{
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<StringRef, StringRef>(attribute, ids,
getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t row) { return def->getDataAt(row); });
}
@ -201,11 +215,11 @@ void HashedDictionary::get##TYPE(\
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, ids,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return def; });\
getItemsNumber<TYPE>( \
attribute, ids, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -224,14 +238,16 @@ DECLARE(Decimal128)
#undef DECLARE
void HashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const
{
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<StringRef, StringRef>(attribute, ids,
getItemsImpl<StringRef, StringRef>(
attribute,
ids,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return StringRef{def}; });
}
@ -242,22 +258,52 @@ void HashedDictionary::has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt128: has<UInt128>(attribute, ids, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, ids, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, ids, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, ids, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, ids, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, ids, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, ids, out); break;
case AttributeUnderlyingType::String: has<StringRef>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt8:
has<UInt8>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt16:
has<UInt16>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt32:
has<UInt32>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt64:
has<UInt64>(attribute, ids, out);
break;
case AttributeUnderlyingType::UInt128:
has<UInt128>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int8:
has<Int8>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int16:
has<Int16>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int32:
has<Int32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Int64:
has<Int64>(attribute, ids, out);
break;
case AttributeUnderlyingType::Float32:
has<Float32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Float64:
has<Float64>(attribute, ids, out);
break;
case AttributeUnderlyingType::String:
has<StringRef>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal32: has<Decimal32>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal64: has<Decimal64>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal128: has<Decimal128>(attribute, ids, out); break;
case AttributeUnderlyingType::Decimal32:
has<Decimal32>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal64:
has<Decimal64>(attribute, ids, out);
break;
case AttributeUnderlyingType::Decimal128:
has<Decimal128>(attribute, ids, out);
break;
}
}
@ -402,21 +448,49 @@ void HashedDictionary::calculateBytesAllocated()
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::UInt128: addAttributeSize<UInt128>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::UInt8:
addAttributeSize<UInt8>(attribute);
break;
case AttributeUnderlyingType::UInt16:
addAttributeSize<UInt16>(attribute);
break;
case AttributeUnderlyingType::UInt32:
addAttributeSize<UInt32>(attribute);
break;
case AttributeUnderlyingType::UInt64:
addAttributeSize<UInt64>(attribute);
break;
case AttributeUnderlyingType::UInt128:
addAttributeSize<UInt128>(attribute);
break;
case AttributeUnderlyingType::Int8:
addAttributeSize<Int8>(attribute);
break;
case AttributeUnderlyingType::Int16:
addAttributeSize<Int16>(attribute);
break;
case AttributeUnderlyingType::Int32:
addAttributeSize<Int32>(attribute);
break;
case AttributeUnderlyingType::Int64:
addAttributeSize<Int64>(attribute);
break;
case AttributeUnderlyingType::Float32:
addAttributeSize<Float32>(attribute);
break;
case AttributeUnderlyingType::Float64:
addAttributeSize<Float64>(attribute);
break;
case AttributeUnderlyingType::Decimal32: addAttributeSize<Decimal32>(attribute); break;
case AttributeUnderlyingType::Decimal64: addAttributeSize<Decimal64>(attribute); break;
case AttributeUnderlyingType::Decimal128: addAttributeSize<Decimal128>(attribute); break;
case AttributeUnderlyingType::Decimal32:
addAttributeSize<Decimal32>(attribute);
break;
case AttributeUnderlyingType::Decimal64:
addAttributeSize<Decimal64>(attribute);
break;
case AttributeUnderlyingType::Decimal128:
addAttributeSize<Decimal128>(attribute);
break;
case AttributeUnderlyingType::String:
{
@ -442,21 +516,49 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::UInt128: createAttributeImpl<UInt128>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::UInt8:
createAttributeImpl<UInt8>(attr, null_value);
break;
case AttributeUnderlyingType::UInt16:
createAttributeImpl<UInt16>(attr, null_value);
break;
case AttributeUnderlyingType::UInt32:
createAttributeImpl<UInt32>(attr, null_value);
break;
case AttributeUnderlyingType::UInt64:
createAttributeImpl<UInt64>(attr, null_value);
break;
case AttributeUnderlyingType::UInt128:
createAttributeImpl<UInt128>(attr, null_value);
break;
case AttributeUnderlyingType::Int8:
createAttributeImpl<Int8>(attr, null_value);
break;
case AttributeUnderlyingType::Int16:
createAttributeImpl<Int16>(attr, null_value);
break;
case AttributeUnderlyingType::Int32:
createAttributeImpl<Int32>(attr, null_value);
break;
case AttributeUnderlyingType::Int64:
createAttributeImpl<Int64>(attr, null_value);
break;
case AttributeUnderlyingType::Float32:
createAttributeImpl<Float32>(attr, null_value);
break;
case AttributeUnderlyingType::Float64:
createAttributeImpl<Float64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal32: createAttributeImpl<Decimal32>(attr, null_value); break;
case AttributeUnderlyingType::Decimal64: createAttributeImpl<Decimal64>(attr, null_value); break;
case AttributeUnderlyingType::Decimal128: createAttributeImpl<Decimal128>(attr, null_value); break;
case AttributeUnderlyingType::Decimal32:
createAttributeImpl<Decimal32>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal64:
createAttributeImpl<Decimal64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal128:
createAttributeImpl<Decimal128>(attr, null_value);
break;
case AttributeUnderlyingType::String:
{
@ -473,12 +575,11 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void HashedDictionary::getItemsNumber(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const
{
if (false) {}
if (false)
{
}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsImpl<TYPE, OutputType>(attribute, ids, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(get_default));
@ -497,16 +598,12 @@ void HashedDictionary::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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void HashedDictionary::getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const
{
const auto & attr = *std::get<CollectionPtrType<AttributeType>>(attribute.maps);
const auto rows = ext::size(ids);
@ -532,21 +629,49 @@ void HashedDictionary::setAttributeValue(Attribute & attribute, const Key id, co
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt128: setAttributeValueImpl<UInt128>(attribute, id, value.get<UInt128>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::UInt8:
setAttributeValueImpl<UInt8>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt16:
setAttributeValueImpl<UInt16>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt32:
setAttributeValueImpl<UInt32>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt64:
setAttributeValueImpl<UInt64>(attribute, id, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt128:
setAttributeValueImpl<UInt128>(attribute, id, value.get<UInt128>());
break;
case AttributeUnderlyingType::Int8:
setAttributeValueImpl<Int8>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int16:
setAttributeValueImpl<Int16>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int32:
setAttributeValueImpl<Int32>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Int64:
setAttributeValueImpl<Int64>(attribute, id, value.get<Int64>());
break;
case AttributeUnderlyingType::Float32:
setAttributeValueImpl<Float32>(attribute, id, value.get<Float64>());
break;
case AttributeUnderlyingType::Float64:
setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>());
break;
case AttributeUnderlyingType::Decimal32: setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal32>()); break;
case AttributeUnderlyingType::Decimal64: setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal64>()); break;
case AttributeUnderlyingType::Decimal128: setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>()); break;
case AttributeUnderlyingType::Decimal32:
setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal32>());
break;
case AttributeUnderlyingType::Decimal64:
setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal64>());
break;
case AttributeUnderlyingType::Decimal128:
setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>());
break;
case AttributeUnderlyingType::String:
{
@ -599,22 +724,37 @@ PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds() const
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: return getIds<UInt8>(attribute);
case AttributeUnderlyingType::UInt16: return getIds<UInt16>(attribute);
case AttributeUnderlyingType::UInt32: return getIds<UInt32>(attribute);
case AttributeUnderlyingType::UInt64: return getIds<UInt64>(attribute);
case AttributeUnderlyingType::UInt128: return getIds<UInt128>(attribute);
case AttributeUnderlyingType::Int8: return getIds<Int8>(attribute);
case AttributeUnderlyingType::Int16: return getIds<Int16>(attribute);
case AttributeUnderlyingType::Int32: return getIds<Int32>(attribute);
case AttributeUnderlyingType::Int64: return getIds<Int64>(attribute);
case AttributeUnderlyingType::Float32: return getIds<Float32>(attribute);
case AttributeUnderlyingType::Float64: return getIds<Float64>(attribute);
case AttributeUnderlyingType::String: return getIds<StringRef>(attribute);
case AttributeUnderlyingType::UInt8:
return getIds<UInt8>(attribute);
case AttributeUnderlyingType::UInt16:
return getIds<UInt16>(attribute);
case AttributeUnderlyingType::UInt32:
return getIds<UInt32>(attribute);
case AttributeUnderlyingType::UInt64:
return getIds<UInt64>(attribute);
case AttributeUnderlyingType::UInt128:
return getIds<UInt128>(attribute);
case AttributeUnderlyingType::Int8:
return getIds<Int8>(attribute);
case AttributeUnderlyingType::Int16:
return getIds<Int16>(attribute);
case AttributeUnderlyingType::Int32:
return getIds<Int32>(attribute);
case AttributeUnderlyingType::Int64:
return getIds<Int64>(attribute);
case AttributeUnderlyingType::Float32:
return getIds<Float32>(attribute);
case AttributeUnderlyingType::Float64:
return getIds<Float64>(attribute);
case AttributeUnderlyingType::String:
return getIds<StringRef>(attribute);
case AttributeUnderlyingType::Decimal32: return getIds<Decimal32>(attribute);
case AttributeUnderlyingType::Decimal64: return getIds<Decimal64>(attribute);
case AttributeUnderlyingType::Decimal128: return getIds<Decimal128>(attribute);
case AttributeUnderlyingType::Decimal32:
return getIds<Decimal32>(attribute);
case AttributeUnderlyingType::Decimal64:
return getIds<Decimal64>(attribute);
case AttributeUnderlyingType::Decimal128:
return getIds<Decimal128>(attribute);
}
return PaddedPODArray<Key>();
}
@ -627,13 +767,12 @@ BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_n
void registerDictionaryHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'hashed'", ErrorCodes::UNSUPPORTED_METHOD};
@ -645,7 +784,6 @@ void registerDictionaryHashed(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("hashed", create_layout);
}

View File

@ -1,27 +1,31 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <variant>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/HashTable/HashMap.h>
#include <ext/range.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
namespace DB
{
using BlockPtr = std::shared_ptr<Block>;
class HashedDictionary final : public IDictionary
{
public:
HashedDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty, BlockPtr saved_block = nullptr);
HashedDictionary(
const std::string & name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty,
BlockPtr saved_block = nullptr);
HashedDictionary(const HashedDictionary & other);
@ -51,10 +55,7 @@ public:
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -90,7 +91,9 @@ public:
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -108,13 +111,13 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const;
void
getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, ColumnString * const out)
const;
#define DECLARE(TYPE) \
void get##TYPE(\
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE & def, ResultArrayType<TYPE> & out) const;
void get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE & def, ResultArrayType<TYPE> & out) \
const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
@ -131,39 +134,60 @@ public:
DECLARE(Decimal128)
#undef DECLARE
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const;
void getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, ColumnString * const out) const;
void has(const PaddedPODArray<Key> & ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorVector(const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorVector(
const PaddedPODArray<Key> & child_ids, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
void isInVectorConstant(const PaddedPODArray<Key> & child_ids, const Key ancestor_id, PaddedPODArray<UInt8> & out) const override;
void isInConstantVector(const Key child_id, const PaddedPODArray<Key> & ancestor_ids, PaddedPODArray<UInt8> & out) const override;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value> using CollectionType = HashMap<UInt64, Value>;
template <typename Value> using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
template <typename Value>
using CollectionType = HashMap<UInt64, Value>;
template <typename Value>
using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
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<
CollectionPtrType<UInt8>, CollectionPtrType<UInt16>, CollectionPtrType<UInt32>, CollectionPtrType<UInt64>,
CollectionPtrType<UInt8>,
CollectionPtrType<UInt16>,
CollectionPtrType<UInt32>,
CollectionPtrType<UInt64>,
CollectionPtrType<UInt128>,
CollectionPtrType<Int8>, CollectionPtrType<Int16>, CollectionPtrType<Int32>, CollectionPtrType<Int64>,
CollectionPtrType<Decimal32>, CollectionPtrType<Decimal64>, CollectionPtrType<Decimal128>,
CollectionPtrType<Float32>, CollectionPtrType<Float64>,
CollectionPtrType<StringRef>> maps;
CollectionPtrType<Int8>,
CollectionPtrType<Int16>,
CollectionPtrType<Int32>,
CollectionPtrType<Int64>,
CollectionPtrType<Decimal32>,
CollectionPtrType<Decimal64>,
CollectionPtrType<Decimal128>,
CollectionPtrType<Float32>,
CollectionPtrType<Float64>,
CollectionPtrType<StringRef>>
maps;
std::unique_ptr<Arena> string_arena;
};
@ -187,17 +211,11 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsNumber(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const;
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const;
template <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void getItemsImpl(
const Attribute & attribute,
const PaddedPODArray<Key> & ids,
ValueSetter && set_value,
DefaultGetter && get_default) const;
const Attribute & attribute, const PaddedPODArray<Key> & ids, ValueSetter && set_value, DefaultGetter && get_default) const;
template <typename T>
void setAttributeValueImpl(Attribute & attribute, const Key id, const T value);
@ -215,10 +233,7 @@ private:
PaddedPODArray<Key> getIds() const;
template <typename ChildType, typename AncestorType>
void isInImpl(
const ChildType & child_ids,
const AncestorType & ancestor_ids,
PaddedPODArray<UInt8> & out) const;
void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray<UInt8> & out) const;
const std::string name;
const DictionaryStructure dict_struct;

View File

@ -1,18 +1,17 @@
#pragma once
#include <chrono>
#include <memory>
#include <Core/Field.h>
#include <Interpreters/IExternalLoadable.h>
#include <common/StringRef.h>
#include <Core/Names.h>
#include <Interpreters/IExternalLoadable.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Common/PODArray.h>
#include <memory>
#include <chrono>
#include <common/StringRef.h>
#include "IDictionarySource.h"
namespace DB
{
struct IDictionaryBase;
using DictionaryPtr = std::unique_ptr<IDictionaryBase>;
@ -79,17 +78,20 @@ struct IDictionary : IDictionaryBase
/// Methods for hierarchy.
virtual void isInVectorVector(const PaddedPODArray<Key> & /*child_ids*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
virtual void isInVectorVector(
const PaddedPODArray<Key> & /*child_ids*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
virtual void isInVectorConstant(const PaddedPODArray<Key> & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray<UInt8> & /*out*/) const
virtual void
isInVectorConstant(const PaddedPODArray<Key> & /*child_ids*/, const Key /*ancestor_id*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
virtual void isInConstantVector(const Key /*child_id*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
virtual void
isInConstantVector(const Key /*child_id*/, const PaddedPODArray<Key> & /*ancestor_ids*/, PaddedPODArray<UInt8> & /*out*/) const
{
throw Exception("Hierarchy is not supported for " + getName() + " dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -1,11 +1,10 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <vector>
#include <DataStreams/IBlockInputStream.h>
namespace DB
{
class IDictionarySource;
using DictionarySourcePtr = std::unique_ptr<IDictionarySource>;
@ -36,8 +35,7 @@ public:
* `requested_rows` contains indices of all rows containing unique keys.
* It must be guaranteed, that 'requested_rows' array will live at least until all data will be read from returned stream.
*/
virtual BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) = 0;
virtual BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) = 0;
/// indicates whether the source has been modified since last load* operation
virtual bool isModified() const = 0;

View File

@ -1,6 +1,5 @@
#include <DataStreams/OneBlockInputStream.h>
#include "LibraryDictionarySource.h"
#include "LibraryDictionarySourceExternal.h"
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
@ -9,6 +8,7 @@
#include <ext/scope_guard.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "LibraryDictionarySourceExternal.h"
namespace DB
@ -78,7 +78,8 @@ namespace
auto columns_received = static_cast<const ClickHouseLibrary::Table *>(data);
if (columns_received->error_code)
throw Exception("LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " "
throw Exception(
"LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " "
+ (columns_received->error_string ? columns_received->error_string : ""),
ErrorCodes::EXTERNAL_LIBRARY_ERROR);
@ -89,8 +90,9 @@ namespace
for (size_t col_n = 0; col_n < columns_received->size; ++col_n)
{
if (columns.size() != columns_received->data[col_n].size)
throw Exception("LibraryDictionarySource: Returned unexpected number of columns: "
+ std::to_string(columns_received->data[col_n].size) + ", must be " + std::to_string(columns.size()),
throw Exception(
"LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size)
+ ", must be " + std::to_string(columns.size()),
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
for (size_t row_n = 0; row_n < columns_received->data[col_n].size; ++row_n)
@ -115,7 +117,8 @@ namespace
}
LibraryDictionarySource::LibraryDictionarySource(const DictionaryStructure & dict_struct_,
LibraryDictionarySource::LibraryDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -128,7 +131,8 @@ LibraryDictionarySource::LibraryDictionarySource(const DictionaryStructure & dic
, context(context)
{
if (!Poco::File(path).exists())
throw Exception("LibraryDictionarySource: Can't load lib " + toString() + ": " + Poco::File(path).path() + " - File doesn't exist",
throw Exception(
"LibraryDictionarySource: Can't load lib " + toString() + ": " + Poco::File(path).path() + " - File doesn't exist",
ErrorCodes::FILE_DOESNT_EXIST);
description.init(sample_block);
library = std::make_shared<SharedLibrary>(path);
@ -151,7 +155,8 @@ LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource &
{
if (auto libClone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v3_libClone"))
lib_data = libClone(other.lib_data);
else if (auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
else if (
auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings), decltype(&ClickHouseLibrary::log))>(
"ClickHouseDictionary_v3_libNew"))
lib_data = libNew(&settings->strings, ClickHouseLibrary::log);
}
@ -167,8 +172,8 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll()
LOG_TRACE(log, "loadAll " + toString());
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
ClickHouseLibrary::CStrings columns{
static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()), dict_struct.attributes.size()};
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),
dict_struct.attributes.size()};
size_t i = 0;
for (auto & a : dict_struct.attributes)
{
@ -193,8 +198,8 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> &
const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast<decltype(ClickHouseLibrary::VectorUInt64::data)>(ids.data()), ids.size()};
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
ClickHouseLibrary::CStrings columns_pass{
static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()), dict_struct.attributes.size()};
ClickHouseLibrary::CStrings columns_pass{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),
dict_struct.attributes.size()};
size_t i = 0;
for (auto & a : dict_struct.attributes)
{

View File

@ -1,10 +1,10 @@
#pragma once
#include <Common/SharedLibrary.h>
#include <common/LocalDateTime.h>
#include "DictionaryStructure.h"
#include "ExternalResultDescription.h"
#include "IDictionarySource.h"
#include <Common/SharedLibrary.h>
#include <common/LocalDateTime.h>
namespace Poco
@ -28,7 +28,8 @@ class CStringsHolder;
class LibraryDictionarySource final : public IDictionarySource
{
public:
LibraryDictionarySource(const DictionaryStructure & dict_struct_,
LibraryDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
@ -54,10 +55,7 @@ public:
bool supportsSelectiveLoad() const override;
///Not yet supported
bool hasUpdateField() const override
{
return false;
}
bool hasUpdateField() const override { return false; }
DictionarySourcePtr clone() const override;

View File

@ -1,29 +1,28 @@
#include <Common/config.h>
#if USE_POCO_MONGODB
#include <vector>
#include <string>
# include <sstream>
# include <string>
# include <vector>
# include <Poco/MongoDB/Connection.h>
# include <Poco/MongoDB/Cursor.h>
# include <Poco/MongoDB/Element.h>
# include <Poco/MongoDB/ObjectId.h>
#include "DictionaryStructure.h"
#include "MongoDBBlockInputStream.h"
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnString.h>
# include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h>
#include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <Common/FieldVisitors.h>
# include <ext/range.h>
# include "DictionaryStructure.h"
# include "MongoDBBlockInputStream.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
@ -55,16 +54,20 @@ namespace
switch (value.type())
{
case Poco::MongoDB::ElementTraits<Int32>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Int32> &>(value).value());
static_cast<ColumnVector<T> &>(column).getData().push_back(
static_cast<const Poco::MongoDB::ConcreteElement<Int32> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<Poco::Int64>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Poco::Int64> &>(value).value());
static_cast<ColumnVector<T> &>(column).getData().push_back(
static_cast<const Poco::MongoDB::ConcreteElement<Poco::Int64> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<Float64>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Float64> &>(value).value());
static_cast<ColumnVector<T> &>(column).getData().push_back(
static_cast<const Poco::MongoDB::ConcreteElement<Float64> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<bool>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<bool> &>(value).value());
static_cast<ColumnVector<T> &>(column).getData().push_back(
static_cast<const Poco::MongoDB::ConcreteElement<bool> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<Poco::MongoDB::NullValue>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().emplace_back();
@ -74,26 +77,46 @@ namespace
parse<T>(static_cast<const Poco::MongoDB::ConcreteElement<String> &>(value).value()));
break;
default:
throw Exception("Type mismatch, expected a number, got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH);
throw Exception(
"Type mismatch, expected a number, got type id = " + toString(value.type()) + " for column " + name,
ErrorCodes::TYPE_MISMATCH);
}
}
void insertValue(
IColumn & column, const ValueType type, const Poco::MongoDB::Element & value, const std::string & name)
void insertValue(IColumn & column, const ValueType type, const Poco::MongoDB::Element & value, const std::string & name)
{
switch (type)
{
case ValueType::UInt8: insertNumber<UInt8>(column, value, name); break;
case ValueType::UInt16: insertNumber<UInt16>(column, value, name); break;
case ValueType::UInt32: insertNumber<UInt32>(column, value, name); break;
case ValueType::UInt64: insertNumber<UInt64>(column, value, name); break;
case ValueType::Int8: insertNumber<Int8>(column, value, name); break;
case ValueType::Int16: insertNumber<Int16>(column, value, name); break;
case ValueType::Int32: insertNumber<Int32>(column, value, name); break;
case ValueType::Int64: insertNumber<Int64>(column, value, name); break;
case ValueType::Float32: insertNumber<Float32>(column, value, name); break;
case ValueType::Float64: insertNumber<Float64>(column, value, name); break;
case ValueType::UInt8:
insertNumber<UInt8>(column, value, name);
break;
case ValueType::UInt16:
insertNumber<UInt16>(column, value, name);
break;
case ValueType::UInt32:
insertNumber<UInt32>(column, value, name);
break;
case ValueType::UInt64:
insertNumber<UInt64>(column, value, name);
break;
case ValueType::Int8:
insertNumber<Int8>(column, value, name);
break;
case ValueType::Int16:
insertNumber<Int16>(column, value, name);
break;
case ValueType::Int32:
insertNumber<Int32>(column, value, name);
break;
case ValueType::Int64:
insertNumber<Int64>(column, value, name);
break;
case ValueType::Float32:
insertNumber<Float32>(column, value, name);
break;
case ValueType::Float64:
insertNumber<Float64>(column, value, name);
break;
case ValueType::String:
{
@ -110,18 +133,17 @@ namespace
break;
}
throw Exception{"Type mismatch, expected String, got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH};
throw Exception{"Type mismatch, expected String, got type id = " + toString(value.type()) + " for column " + name,
ErrorCodes::TYPE_MISMATCH};
}
case ValueType::Date:
{
if (value.type() != Poco::MongoDB::ElementTraits<Poco::Timestamp>::TypeId)
throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH};
throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) + " for column " + name,
ErrorCodes::TYPE_MISMATCH};
static_cast<ColumnUInt16 &>(column).getData().push_back(
UInt16{DateLUT::instance().toDayNum(
static_cast<ColumnUInt16 &>(column).getData().push_back(UInt16{DateLUT::instance().toDayNum(
static_cast<const Poco::MongoDB::ConcreteElement<Poco::Timestamp> &>(value).value().epochTime())});
break;
}
@ -129,8 +151,8 @@ namespace
case ValueType::DateTime:
{
if (value.type() != Poco::MongoDB::ElementTraits<Poco::Timestamp>::TypeId)
throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH};
throw Exception{"Type mismatch, expected Timestamp, got type id = " + toString(value.type()) + " for column " + name,
ErrorCodes::TYPE_MISMATCH};
static_cast<ColumnUInt32 &>(column).getData().push_back(
static_cast<const Poco::MongoDB::ConcreteElement<Poco::Timestamp> &>(value).value().epochTime());
@ -144,17 +166,15 @@ namespace
static_cast<ColumnUInt128 &>(column).getData().push_back(parse<UUID>(string));
}
else
throw Exception{"Type mismatch, expected String (UUID), got type id = " + toString(value.type()) +
" for column " + name, ErrorCodes::TYPE_MISMATCH};
throw Exception{"Type mismatch, expected String (UUID), got type id = " + toString(value.type()) + " for column "
+ name,
ErrorCodes::TYPE_MISMATCH};
break;
}
}
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column)
{
column.insertFrom(sample_column, 0);
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); }
}

View File

@ -17,7 +17,6 @@ namespace Poco
namespace DB
{
/// Converts MongoDB Cursor to a stream of Blocks
class MongoDBBlockInputStream final : public IProfilingBlockInputStream
{

View File

@ -1,10 +1,9 @@
#include "MongoDBDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "MongoDBDictionarySource.h"
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
@ -36,27 +35,26 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
#if USE_POCO_MONGODB
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/MongoDB/Connection.h>
#include <Poco/MongoDB/Database.h>
#include <Poco/MongoDB/Cursor.h>
# include <Poco/MongoDB/Array.h>
# include <Poco/MongoDB/Connection.h>
# include <Poco/MongoDB/Cursor.h>
# include <Poco/MongoDB/Database.h>
# include <Poco/MongoDB/ObjectId.h>
# include <Poco/Util/AbstractConfiguration.h>
# include <Poco/Version.h>
// only after poco
// naming conflict:
// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value);
// dbms/src/IO/WriteHelpers.h:146 #define writeCString(s, buf)
#include "MongoDBBlockInputStream.h"
#include <Common/FieldVisitors.h>
# include <IO/WriteHelpers.h>
# include <Common/FieldVisitors.h>
# include <ext/enumerate.h>
# include "MongoDBBlockInputStream.h"
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
@ -70,8 +68,8 @@ static const size_t max_block_size = 8192;
# if POCO_VERSION < 0x01070800
/// See https://pocoproject.org/forum/viewtopic.php?f=10&t=6326&p=11426&hilit=mongodb+auth#p11485
static void authenticate(Poco::MongoDB::Connection & connection,
const std::string & database, const std::string & user, const std::string & password)
static void
authenticate(Poco::MongoDB::Connection & connection, const std::string & database, const std::string & user, const std::string & password)
{
Poco::MongoDB::Database db(database);
@ -88,7 +86,8 @@ static void authenticate(Poco::MongoDB::Connection & connection,
connection.sendRequest(*command, response);
if (response.documents().empty())
throw Exception("Cannot authenticate in MongoDB: server returned empty response for 'getnonce' command",
throw Exception(
"Cannot authenticate in MongoDB: server returned empty response for 'getnonce' command",
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
auto doc = response.documents()[0];
@ -96,18 +95,24 @@ static void authenticate(Poco::MongoDB::Connection & connection,
{
double ok = doc->get<double>("ok", 0);
if (ok != 1)
throw Exception("Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
" has field 'ok' missing or having wrong value", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
" has field 'ok' missing or having wrong value",
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
nonce = doc->get<std::string>("nonce", "");
if (nonce.empty())
throw Exception("Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
" has field 'nonce' missing or empty", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
" has field 'nonce' missing or empty",
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
catch (Poco::NotFoundException & e)
{
throw Exception("Cannot authenticate in MongoDB: server returned response for 'getnonce' command that has missing required field: "
+ e.displayText(), ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that has missing required field: "
+ e.displayText(),
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
}
@ -136,7 +141,8 @@ static void authenticate(Poco::MongoDB::Connection & connection,
connection.sendRequest(*command, response);
if (response.empty())
throw Exception("Cannot authenticate in MongoDB: server returned empty response for 'authenticate' command",
throw Exception(
"Cannot authenticate in MongoDB: server returned empty response for 'authenticate' command",
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
auto doc = response.documents()[0];
@ -144,13 +150,17 @@ static void authenticate(Poco::MongoDB::Connection & connection,
{
double ok = doc->get<double>("ok", 0);
if (ok != 1)
throw Exception("Cannot authenticate in MongoDB: server returned response for 'authenticate' command that"
" has field 'ok' missing or having wrong value", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(
"Cannot authenticate in MongoDB: server returned response for 'authenticate' command that"
" has field 'ok' missing or having wrong value",
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
catch (Poco::NotFoundException & e)
{
throw Exception("Cannot authenticate in MongoDB: server returned response for 'authenticate' command that has missing required field: "
+ e.displayText(), ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
throw Exception(
"Cannot authenticate in MongoDB: server returned response for 'authenticate' command that has missing required field: "
+ e.displayText(),
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
}
}
@ -158,15 +168,25 @@ static void authenticate(Poco::MongoDB::Connection & connection,
MongoDBDictionarySource::MongoDBDictionarySource(
const DictionaryStructure & dict_struct, const std::string & host, UInt16 port,
const std::string & user, const std::string & password,
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
const std::string & user,
const std::string & password,
const std::string & method,
const std::string & db, const std::string & collection,
const std::string & db,
const std::string & collection,
const Block & sample_block)
: dict_struct{dict_struct}, host{host}, port{port}, user{user}, password{password},
method{method},
db{db}, collection{collection}, sample_block{sample_block},
connection{std::make_shared<Poco::MongoDB::Connection>(host, port)}
: dict_struct{dict_struct}
, host{host}
, port{port}
, user{user}
, password{password}
, method{method}
, db{db}
, collection{collection}
, sample_block{sample_block}
, connection{std::make_shared<Poco::MongoDB::Connection>(host, port)}
{
if (!user.empty())
{
@ -176,14 +196,15 @@ MongoDBDictionarySource::MongoDBDictionarySource(
# else
authenticate(*connection, db, user, password);
# endif
}
}
MongoDBDictionarySource::MongoDBDictionarySource(
const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, Block & sample_block)
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block)
: MongoDBDictionarySource(
dict_struct,
config.getString(config_prefix + ".host"),
@ -199,10 +220,15 @@ MongoDBDictionarySource::MongoDBDictionarySource(
MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource & other)
: MongoDBDictionarySource{
other.dict_struct, other.host, other.port, other.user, other.password,
: MongoDBDictionarySource{other.dict_struct,
other.host,
other.port,
other.user,
other.password,
other.method,
other.db, other.collection, other.sample_block}
other.db,
other.collection,
other.sample_block}
{
}
@ -210,8 +236,8 @@ MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource &
MongoDBDictionarySource::~MongoDBDictionarySource() = default;
static std::unique_ptr<Poco::MongoDB::Cursor> createCursor(
const std::string & database, const std::string & collection, const Block & sample_block_to_select)
static std::unique_ptr<Poco::MongoDB::Cursor>
createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select)
{
auto cursor = std::make_unique<Poco::MongoDB::Cursor>(database, collection);
@ -228,8 +254,7 @@ static std::unique_ptr<Poco::MongoDB::Cursor> createCursor(
BlockInputStreamPtr MongoDBDictionarySource::loadAll()
{
return std::make_shared<MongoDBBlockInputStream>(
connection, createCursor(db, collection, sample_block), sample_block, max_block_size);
return std::make_shared<MongoDBBlockInputStream>(connection, createCursor(db, collection, sample_block), sample_block, max_block_size);
}
@ -248,16 +273,13 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> &
for (const UInt64 id : ids)
ids_array->add(DB::toString(id), Int32(id));
cursor->query().selector().addNewDocument(dict_struct.id->name)
.add("$in", ids_array);
cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array);
return std::make_shared<MongoDBBlockInputStream>(
connection, std::move(cursor), sample_block, max_block_size);
return std::make_shared<MongoDBBlockInputStream>(connection, std::move(cursor), sample_block, max_block_size);
}
BlockInputStreamPtr MongoDBDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
if (!dict_struct.key)
throw Exception{"'key' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
@ -314,8 +336,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(
/// If more than one key we should use $or
cursor->query().selector().add("$or", keys_array);
return std::make_shared<MongoDBBlockInputStream>(
connection, std::move(cursor), sample_block, max_block_size);
return std::make_shared<MongoDBBlockInputStream>(connection, std::move(cursor), sample_block, max_block_size);
}

View File

@ -3,8 +3,8 @@
#include <Common/config.h>
#if USE_POCO_MONGODB
#include "IDictionarySource.h"
# include "DictionaryStructure.h"
# include "IDictionarySource.h"
namespace Poco
{
@ -22,21 +22,26 @@ namespace Poco
namespace DB
{
/// Allows loading dictionaries from a MongoDB collection
class MongoDBDictionarySource final : public IDictionarySource
{
MongoDBDictionarySource(
const DictionaryStructure & dict_struct, const std::string & host, UInt16 port,
const std::string & user, const std::string & password,
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
const std::string & user,
const std::string & password,
const std::string & method,
const std::string & db, const std::string & collection,
const std::string & db,
const std::string & collection,
const Block & sample_block);
public:
MongoDBDictionarySource(
const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, Block & sample_block);
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block);
MongoDBDictionarySource(const MongoDBDictionarySource & other);
@ -53,8 +58,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field
bool isModified() const override { return true; }

View File

@ -1,19 +1,18 @@
#include <Common/config.h>
#if USE_MYSQL
#include "MySQLBlockInputStream.h"
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
# include <vector>
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnString.h>
# include <Columns/ColumnsNumber.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <ext/range.h>
#include <vector>
# include "MySQLBlockInputStream.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
@ -21,13 +20,12 @@ namespace ErrorCodes
MySQLBlockInputStream::MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block,
const size_t max_block_size)
: entry{entry}, query{this->entry->query(query_str)}, result{query.use()},
max_block_size{max_block_size}
const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, const size_t max_block_size)
: entry{entry}, query{this->entry->query(query_str)}, result{query.use()}, max_block_size{max_block_size}
{
if (sample_block.columns() != result.getNumFields())
throw Exception{"mysqlxx::UseQueryResult contains " + toString(result.getNumFields()) + " columns while " + toString(sample_block.columns()) + " expected",
throw Exception{"mysqlxx::UseQueryResult contains " + toString(result.getNumFields()) + " columns while "
+ toString(sample_block.columns()) + " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
description.init(sample_block);
@ -42,27 +40,52 @@ namespace
{
switch (type)
{
case ValueType::UInt8: static_cast<ColumnUInt8 &>(column).insertValue(value.getUInt()); break;
case ValueType::UInt16: static_cast<ColumnUInt16 &>(column).insertValue(value.getUInt()); break;
case ValueType::UInt32: static_cast<ColumnUInt32 &>(column).insertValue(value.getUInt()); break;
case ValueType::UInt64: static_cast<ColumnUInt64 &>(column).insertValue(value.getUInt()); break;
case ValueType::Int8: static_cast<ColumnInt8 &>(column).insertValue(value.getInt()); break;
case ValueType::Int16: static_cast<ColumnInt16 &>(column).insertValue(value.getInt()); break;
case ValueType::Int32: static_cast<ColumnInt32 &>(column).insertValue(value.getInt()); break;
case ValueType::Int64: static_cast<ColumnInt64 &>(column).insertValue(value.getInt()); break;
case ValueType::Float32: static_cast<ColumnFloat32 &>(column).insertValue(value.getDouble()); break;
case ValueType::Float64: static_cast<ColumnFloat64 &>(column).insertValue(value.getDouble()); break;
case ValueType::String: static_cast<ColumnString &>(column).insertData(value.data(), value.size()); break;
case ValueType::Date: static_cast<ColumnUInt16 &>(column).insertValue(UInt16(value.getDate().getDayNum())); break;
case ValueType::DateTime: static_cast<ColumnUInt32 &>(column).insertValue(UInt32(value.getDateTime())); break;
case ValueType::UUID: static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.data(), value.size())); break;
case ValueType::UInt8:
static_cast<ColumnUInt8 &>(column).insertValue(value.getUInt());
break;
case ValueType::UInt16:
static_cast<ColumnUInt16 &>(column).insertValue(value.getUInt());
break;
case ValueType::UInt32:
static_cast<ColumnUInt32 &>(column).insertValue(value.getUInt());
break;
case ValueType::UInt64:
static_cast<ColumnUInt64 &>(column).insertValue(value.getUInt());
break;
case ValueType::Int8:
static_cast<ColumnInt8 &>(column).insertValue(value.getInt());
break;
case ValueType::Int16:
static_cast<ColumnInt16 &>(column).insertValue(value.getInt());
break;
case ValueType::Int32:
static_cast<ColumnInt32 &>(column).insertValue(value.getInt());
break;
case ValueType::Int64:
static_cast<ColumnInt64 &>(column).insertValue(value.getInt());
break;
case ValueType::Float32:
static_cast<ColumnFloat32 &>(column).insertValue(value.getDouble());
break;
case ValueType::Float64:
static_cast<ColumnFloat64 &>(column).insertValue(value.getDouble());
break;
case ValueType::String:
static_cast<ColumnString &>(column).insertData(value.data(), value.size());
break;
case ValueType::Date:
static_cast<ColumnUInt16 &>(column).insertValue(UInt16(value.getDate().getDayNum()));
break;
case ValueType::DateTime:
static_cast<ColumnUInt32 &>(column).insertValue(UInt32(value.getDateTime()));
break;
case ValueType::UUID:
static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.data(), value.size()));
break;
}
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column)
{
column.insertFrom(sample_column, 0);
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); }
}

View File

@ -1,22 +1,23 @@
#pragma once
#include <string>
#include <Core/Block.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include "ExternalResultDescription.h"
#include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h>
#include <string>
#include <mysqlxx/Query.h>
#include "ExternalResultDescription.h"
namespace DB
{
/// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining
class MySQLBlockInputStream final : public IProfilingBlockInputStream
{
public:
MySQLBlockInputStream(
const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block,
const mysqlxx::PoolWithFailover::Entry & entry,
const std::string & query_str,
const Block & sample_block,
const size_t max_block_size);
String getName() const override { return "MySQL"; }

View File

@ -1,14 +1,13 @@
#include "MySQLDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/config.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
@ -39,56 +38,59 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
#if USE_MYSQL
#include <IO/WriteBufferFromString.h>
#include <DataTypes/DataTypeString.h>
# include <Columns/ColumnString.h>
#include <common/logger_useful.h>
# include <DataTypes/DataTypeString.h>
# include <IO/WriteBufferFromString.h>
# include <IO/WriteHelpers.h>
# include <common/LocalDateTime.h>
# include <common/logger_useful.h>
# include "MySQLBlockInputStream.h"
# include "readInvalidateQuery.h"
#include <IO/WriteHelpers.h>
namespace DB
{
static const size_t max_block_size = 8192;
MySQLDictionarySource::MySQLDictionarySource(const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
MySQLDictionarySource::MySQLDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Block & sample_block)
: log(&Logger::get("MySQLDictionarySource")),
update_time{std::chrono::system_clock::from_time_t(0)},
dict_struct{dict_struct_},
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", "")},
dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)},
sample_block{sample_block},
pool{config, config_prefix},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
: log(&Logger::get("MySQLDictionarySource"))
, update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, 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", "")}
, dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)}
, sample_block{sample_block}
, pool{config, config_prefix}
, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, load_all_query{query_builder.composeLoadAllQuery()}
, invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
{
}
/// copy-constructor is provided in order to support cloneability
MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other)
: log(&Logger::get("MySQLDictionarySource")),
update_time{other.update_time},
dict_struct{other.dict_struct},
db{other.db},
table{other.table},
where{other.where},
update_field{other.update_field},
dont_check_update_time{other.dont_check_update_time},
sample_block{other.sample_block},
pool{other.pool},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
load_all_query{other.load_all_query}, last_modification{other.last_modification},
invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response}
: log(&Logger::get("MySQLDictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, db{other.db}
, table{other.table}
, where{other.where}
, update_field{other.update_field}
, dont_check_update_time{other.dont_check_update_time}
, sample_block{other.sample_block}
, pool{other.pool}
, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}
, load_all_query{other.load_all_query}
, last_modification{other.last_modification}
, invalidate_query{other.invalidate_query}
, invalidate_query_response{other.invalidate_query_response}
{
}
@ -135,8 +137,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & i
return std::make_shared<MySQLBlockInputStream>(pool.Get(), query, sample_block, max_block_size);
}
BlockInputStreamPtr MySQLDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
/// We do not log in here and do not update the modification time, as the request can be large, and often called.

View File

@ -3,11 +3,11 @@
#include <Common/config.h>
#if USE_MYSQL
#include "IDictionarySource.h"
#include "ExternalQueryBuilder.h"
#include "DictionaryStructure.h"
# include <common/LocalDateTime.h>
# include <mysqlxx/PoolWithFailover.h>
# include "DictionaryStructure.h"
# include "ExternalQueryBuilder.h"
# include "IDictionarySource.h"
namespace Poco
@ -23,14 +23,14 @@ namespace Poco
namespace DB
{
/// Allows loading dictionaries from a MySQL database
class MySQLDictionarySource final : public IDictionarySource
{
public:
MySQLDictionarySource(const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
MySQLDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Block & sample_block);
/// copy-constructor is provided in order to support cloneability
@ -42,8 +42,7 @@ public:
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override;

View File

@ -1,17 +1,16 @@
#include "ODBCBlockInputStream.h"
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <vector>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include <vector>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
@ -19,19 +18,18 @@ namespace ErrorCodes
ODBCBlockInputStream::ODBCBlockInputStream(
Poco::Data::Session && session, const std::string & query_str, const Block & sample_block,
const size_t max_block_size)
:
session{session},
statement{(this->session << query_str, Poco::Data::Keywords::now)},
result{statement},
iterator{result.begin()},
max_block_size{max_block_size},
log(&Logger::get("ODBCBlockInputStream"))
Poco::Data::Session && session, const std::string & query_str, const Block & sample_block, const size_t max_block_size)
: session{session}
, statement{(this->session << query_str, Poco::Data::Keywords::now)}
, result{statement}
, iterator{result.begin()}
, max_block_size{max_block_size}
, log(&Logger::get("ODBCBlockInputStream"))
{
if (sample_block.columns() != result.columnCount())
throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " +
toString(sample_block.columns()) + " expected", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " + toString(sample_block.columns())
+ " expected",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
description.init(sample_block);
}
@ -45,28 +43,52 @@ namespace
{
switch (type)
{
case ValueType::UInt8: static_cast<ColumnUInt8 &>(column).insertValue(value.convert<UInt64>()); break;
case ValueType::UInt16: static_cast<ColumnUInt16 &>(column).insertValue(value.convert<UInt64>()); break;
case ValueType::UInt32: static_cast<ColumnUInt32 &>(column).insertValue(value.convert<UInt64>()); break;
case ValueType::UInt64: static_cast<ColumnUInt64 &>(column).insertValue(value.convert<UInt64>()); break;
case ValueType::Int8: static_cast<ColumnInt8 &>(column).insertValue(value.convert<Int64>()); break;
case ValueType::Int16: static_cast<ColumnInt16 &>(column).insertValue(value.convert<Int64>()); break;
case ValueType::Int32: static_cast<ColumnInt32 &>(column).insertValue(value.convert<Int64>()); break;
case ValueType::Int64: static_cast<ColumnInt64 &>(column).insertValue(value.convert<Int64>()); break;
case ValueType::Float32: static_cast<ColumnFloat32 &>(column).insertValue(value.convert<Float64>()); break;
case ValueType::Float64: static_cast<ColumnFloat64 &>(column).insertValue(value.convert<Float64>()); break;
case ValueType::String: static_cast<ColumnString &>(column).insert(value.convert<String>()); break;
case ValueType::Date: static_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{value.convert<String>()}.getDayNum()}); break;
case ValueType::DateTime: static_cast<ColumnUInt32 &>(column).insertValue(time_t{LocalDateTime{value.convert<String>()}}); break;
case ValueType::UUID: static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.convert<std::string>())); break;
case ValueType::UInt8:
static_cast<ColumnUInt8 &>(column).insertValue(value.convert<UInt64>());
break;
case ValueType::UInt16:
static_cast<ColumnUInt16 &>(column).insertValue(value.convert<UInt64>());
break;
case ValueType::UInt32:
static_cast<ColumnUInt32 &>(column).insertValue(value.convert<UInt64>());
break;
case ValueType::UInt64:
static_cast<ColumnUInt64 &>(column).insertValue(value.convert<UInt64>());
break;
case ValueType::Int8:
static_cast<ColumnInt8 &>(column).insertValue(value.convert<Int64>());
break;
case ValueType::Int16:
static_cast<ColumnInt16 &>(column).insertValue(value.convert<Int64>());
break;
case ValueType::Int32:
static_cast<ColumnInt32 &>(column).insertValue(value.convert<Int64>());
break;
case ValueType::Int64:
static_cast<ColumnInt64 &>(column).insertValue(value.convert<Int64>());
break;
case ValueType::Float32:
static_cast<ColumnFloat32 &>(column).insertValue(value.convert<Float64>());
break;
case ValueType::Float64:
static_cast<ColumnFloat64 &>(column).insertValue(value.convert<Float64>());
break;
case ValueType::String:
static_cast<ColumnString &>(column).insert(value.convert<String>());
break;
case ValueType::Date:
static_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{value.convert<String>()}.getDayNum()});
break;
case ValueType::DateTime:
static_cast<ColumnUInt32 &>(column).insertValue(time_t{LocalDateTime{value.convert<String>()}});
break;
case ValueType::UUID:
static_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.convert<std::string>()));
break;
}
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column)
{
column.insertFrom(sample_column, 0);
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); }
}
@ -108,7 +130,6 @@ Block ODBCBlockInputStream::readImpl()
++num_rows;
if (num_rows == max_block_size)
break;
}
return description.sample_block.cloneWithColumns(std::move(columns));

View File

@ -1,24 +1,22 @@
#pragma once
#include <string>
#include <Core/Block.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include "ExternalResultDescription.h"
#include <Poco/Data/RecordSet.h>
#include <Poco/Data/Session.h>
#include <Poco/Data/Statement.h>
#include <Poco/Data/RecordSet.h>
#include <string>
#include "ExternalResultDescription.h"
namespace DB
{
/// Allows processing results of a query to ODBC source as a sequence of Blocks, simplifies chaining
class ODBCBlockInputStream final : public IProfilingBlockInputStream
{
public:
ODBCBlockInputStream(
Poco::Data::Session && session, const std::string & query_str, const Block & sample_block,
const size_t max_block_size);
Poco::Data::Session && session, const std::string & query_str, const Block & sample_block, const size_t max_block_size);
String getName() const override { return "ODBC"; }

View File

@ -1,19 +1,18 @@
#pragma once
#include <Columns/ColumnVector.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/IColumn.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypesNumber.h>
#include <ext/range.h>
#include "DictionaryBlockInputStreamBase.h"
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "RangeHashedDictionary.h"
#include <ext/range.h>
namespace DB
{
/*
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
@ -25,46 +24,58 @@ public:
using DictionaryPtr = std::shared_ptr<DictionaryType const>;
RangeDictionaryBlockInputStream(
DictionaryPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids_to_fill,
PaddedPODArray<RangeType> && start_dates, PaddedPODArray<RangeType> && end_dates);
DictionaryPtr dictionary,
size_t max_block_size,
const Names & column_names,
PaddedPODArray<Key> && ids_to_fill,
PaddedPODArray<RangeType> && start_dates,
PaddedPODArray<RangeType> && end_dates);
String getName() const override
{
return "RangeDictionary";
}
String getName() const override { return "RangeDictionary"; }
protected:
Block getBlock(size_t start, size_t length) const override;
private:
template <typename Type>
using DictionaryGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &,
const PaddedPODArray<Int64> &, PaddedPODArray<Type> &) const;
using DictionaryGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, const PaddedPODArray<Int64> &, PaddedPODArray<Type> &) const;
template <typename Type>
using DictionaryDecimalGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &,
const PaddedPODArray<Int64> &, DecimalPaddedPODArray<Type> &) const;
using DictionaryDecimalGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, const PaddedPODArray<Int64> &, DecimalPaddedPODArray<Type> &) const;
template <typename AttributeType, typename Getter>
ColumnPtr getColumnFromAttribute(Getter getter,
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const;
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const;
ColumnPtr getColumnFromAttribute(
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute,
const DictionaryType & concrete_dictionary) const;
ColumnPtr getColumnFromAttributeString(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute,
const DictionaryType & concrete_dictionary) const;
template <typename T>
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array) const;
template <typename DictionarySpecialAttributeType, typename T>
void addSpecialColumn(
const std::optional<DictionarySpecialAttributeType> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const;
const std::optional<DictionarySpecialAttributeType> & attribute,
DataTypePtr type,
const std::string & default_name,
const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values,
ColumnsWithTypeAndName & columns) const;
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
Block fillBlock(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const;
PaddedPODArray<Int64> makeDateKey(
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
PaddedPODArray<Int64>
makeDateKey(const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const;
DictionaryPtr dictionary;
Names column_names;
@ -76,11 +87,18 @@ private:
template <typename DictionaryType, typename RangeType, typename Key>
RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::RangeDictionaryBlockInputStream(
DictionaryPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<RangeType> && block_start_dates, PaddedPODArray<RangeType> && block_end_dates)
: DictionaryBlockInputStreamBase(ids.size(), max_column_size),
dictionary(dictionary), column_names(column_names),
ids(std::move(ids)), start_dates(std::move(block_start_dates)), end_dates(std::move(block_end_dates))
DictionaryPtr dictionary,
size_t max_column_size,
const Names & column_names,
PaddedPODArray<Key> && ids,
PaddedPODArray<RangeType> && block_start_dates,
PaddedPODArray<RangeType> && block_end_dates)
: DictionaryBlockInputStreamBase(ids.size(), max_column_size)
, dictionary(dictionary)
, column_names(column_names)
, ids(std::move(ids))
, start_dates(std::move(block_start_dates))
, end_dates(std::move(block_end_dates))
{
}
@ -107,8 +125,11 @@ Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getBlock(
template <typename DictionaryType, typename RangeType, typename Key>
template <typename AttributeType, typename Getter>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttribute(
Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates, const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
Getter getter,
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute,
const DictionaryType & concrete_dictionary) const
{
if constexpr (IsDecimalNumber<AttributeType>)
{
@ -126,8 +147,10 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getCo
template <typename DictionaryType, typename RangeType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute, const DictionaryType & concrete_dictionary) const
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<Int64> & dates,
const DictionaryAttribute & attribute,
const DictionaryType & concrete_dictionary) const
{
auto column_string = ColumnString::create();
concrete_dictionary.getString(attribute.name, ids_to_fill, dates, column_string.get());
@ -149,9 +172,12 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::getCo
template <typename DictionaryType, typename RangeType, typename Key>
template <typename DictionarySpecialAttributeType, typename T>
void RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::addSpecialColumn(
const std::optional<DictionarySpecialAttributeType> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
const std::optional<DictionarySpecialAttributeType> & attribute,
DataTypePtr type,
const std::string & default_name,
const std::unordered_set<std::string> & column_names_set,
const PaddedPODArray<T> & values,
ColumnsWithTypeAndName & columns) const
{
std::string name = default_name;
if (attribute)
@ -181,7 +207,8 @@ PaddedPODArray<Int64> RangeDictionaryBlockInputStream<DictionaryType, RangeType,
template <typename DictionaryType, typename RangeType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, RangeType, Key>::fillBlock(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const
{
ColumnsWithTypeAndName columns;
const DictionaryStructure & structure = dictionary->getStructure();

View File

@ -1,14 +1,13 @@
#include "RangeHashedDictionary.h"
#include "RangeDictionaryBlockInputStream.h"
#include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <Common/TypeList.h>
#include <ext/range.h>
#include "DictionaryFactory.h"
#include "RangeDictionaryBlockInputStream.h"
namespace
{
using RangeStorageType = DB::RangeHashedDictionary::RangeStorageType;
// Null values mean that specified boundary, either min or max is not set on range.
@ -44,7 +43,6 @@ const DB::IColumn & unwrapNullableColumn(const DB::IColumn & column)
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
@ -70,11 +68,16 @@ bool operator<(const RangeHashedDictionary::Range & left, const RangeHashedDicti
RangeHashedDictionary::RangeHashedDictionary(
const std::string & dictionary_name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: dictionary_name{dictionary_name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
const std::string & dictionary_name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty)
: dictionary_name{dictionary_name}
, dict_struct(dict_struct)
, source_ptr{std::move(source_ptr)}
, dict_lifetime(dict_lifetime)
, require_nonempty(require_nonempty)
{
createAttributes();
@ -92,14 +95,17 @@ RangeHashedDictionary::RangeHashedDictionary(
}
RangeHashedDictionary::RangeHashedDictionary(const RangeHashedDictionary & other)
: RangeHashedDictionary{other.dictionary_name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
: RangeHashedDictionary{
other.dictionary_name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{
}
#define DECLARE_MULTIPLE_GETTER(TYPE) \
void RangeHashedDictionary::get##TYPE( \
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<RangeStorageType> & dates,\
const std::string & attribute_name, \
const PaddedPODArray<Key> & ids, \
const PaddedPODArray<RangeStorageType> & dates, \
ResultArrayType<TYPE> & out) const \
{ \
const auto & attribute = getAttributeWithType(attribute_name, AttributeUnderlyingType::TYPE); \
@ -122,7 +128,9 @@ DECLARE_MULTIPLE_GETTER(Decimal128)
#undef DECLARE_MULTIPLE_GETTER
void RangeHashedDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<RangeStorageType> & dates,
const std::string & attribute_name,
const PaddedPODArray<Key> & ids,
const PaddedPODArray<RangeStorageType> & dates,
ColumnString * out) const
{
const auto & attribute = getAttributeWithType(attribute_name, AttributeUnderlyingType::String);
@ -136,8 +144,11 @@ void RangeHashedDictionary::getString(
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values),
[date] (const Value<StringRef> & v) { return v.range.contains(date); });
const auto val_it
= std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value<StringRef> & v)
{
return v.range.contains(date);
});
const auto string_ref = val_it != std::end(ranges_and_values) ? val_it->value : StringRef{null_value};
out->insertData(string_ref.data, string_ref.size);
@ -161,7 +172,8 @@ void RangeHashedDictionary::createAttributes()
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{dictionary_name + ": hierarchical attributes not supported by " + getName() + " dictionary.", ErrorCodes::BAD_ARGUMENTS};
throw Exception{dictionary_name + ": hierarchical attributes not supported by " + getName() + " dictionary.",
ErrorCodes::BAD_ARGUMENTS};
}
}
@ -203,9 +215,7 @@ void RangeHashedDictionary::loadData()
upper_bound = getColumnIntValueOrDefault(max_range_column, row_idx, is_date, RANGE_MAX_NULL_VALUE);
}
setAttributeValue(attribute, id_column.getUInt(row_idx),
Range{lower_bound, upper_bound},
attribute_column[row_idx]);
setAttributeValue(attribute, id_column.getUInt(row_idx), Range{lower_bound, upper_bound}, attribute_column[row_idx]);
}
}
}
@ -213,7 +223,8 @@ void RangeHashedDictionary::loadData()
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception{dictionary_name + ": dictionary source is empty and 'require_nonempty' property is set.", ErrorCodes::DICTIONARY_IS_EMPTY};
throw Exception{dictionary_name + ": dictionary source is empty and 'require_nonempty' property is set.",
ErrorCodes::DICTIONARY_IS_EMPTY};
}
template <typename T>
@ -232,21 +243,49 @@ void RangeHashedDictionary::calculateBytesAllocated()
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::UInt128: addAttributeSize<UInt128>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::UInt8:
addAttributeSize<UInt8>(attribute);
break;
case AttributeUnderlyingType::UInt16:
addAttributeSize<UInt16>(attribute);
break;
case AttributeUnderlyingType::UInt32:
addAttributeSize<UInt32>(attribute);
break;
case AttributeUnderlyingType::UInt64:
addAttributeSize<UInt64>(attribute);
break;
case AttributeUnderlyingType::UInt128:
addAttributeSize<UInt128>(attribute);
break;
case AttributeUnderlyingType::Int8:
addAttributeSize<Int8>(attribute);
break;
case AttributeUnderlyingType::Int16:
addAttributeSize<Int16>(attribute);
break;
case AttributeUnderlyingType::Int32:
addAttributeSize<Int32>(attribute);
break;
case AttributeUnderlyingType::Int64:
addAttributeSize<Int64>(attribute);
break;
case AttributeUnderlyingType::Float32:
addAttributeSize<Float32>(attribute);
break;
case AttributeUnderlyingType::Float64:
addAttributeSize<Float64>(attribute);
break;
case AttributeUnderlyingType::Decimal32: addAttributeSize<Decimal32>(attribute); break;
case AttributeUnderlyingType::Decimal64: addAttributeSize<Decimal64>(attribute); break;
case AttributeUnderlyingType::Decimal128: addAttributeSize<Decimal128>(attribute); break;
case AttributeUnderlyingType::Decimal32:
addAttributeSize<Decimal32>(attribute);
break;
case AttributeUnderlyingType::Decimal64:
addAttributeSize<Decimal64>(attribute);
break;
case AttributeUnderlyingType::Decimal128:
addAttributeSize<Decimal128>(attribute);
break;
case AttributeUnderlyingType::String:
{
@ -266,27 +305,56 @@ void RangeHashedDictionary::createAttributeImpl(Attribute & attribute, const Fie
attribute.maps = std::make_unique<Collection<T>>();
}
RangeHashedDictionary::Attribute RangeHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
RangeHashedDictionary::Attribute
RangeHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
Attribute attr{type, {}, {}, {}};
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::UInt128: createAttributeImpl<UInt128>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::UInt8:
createAttributeImpl<UInt8>(attr, null_value);
break;
case AttributeUnderlyingType::UInt16:
createAttributeImpl<UInt16>(attr, null_value);
break;
case AttributeUnderlyingType::UInt32:
createAttributeImpl<UInt32>(attr, null_value);
break;
case AttributeUnderlyingType::UInt64:
createAttributeImpl<UInt64>(attr, null_value);
break;
case AttributeUnderlyingType::UInt128:
createAttributeImpl<UInt128>(attr, null_value);
break;
case AttributeUnderlyingType::Int8:
createAttributeImpl<Int8>(attr, null_value);
break;
case AttributeUnderlyingType::Int16:
createAttributeImpl<Int16>(attr, null_value);
break;
case AttributeUnderlyingType::Int32:
createAttributeImpl<Int32>(attr, null_value);
break;
case AttributeUnderlyingType::Int64:
createAttributeImpl<Int64>(attr, null_value);
break;
case AttributeUnderlyingType::Float32:
createAttributeImpl<Float32>(attr, null_value);
break;
case AttributeUnderlyingType::Float64:
createAttributeImpl<Float64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal32: createAttributeImpl<Decimal32>(attr, null_value); break;
case AttributeUnderlyingType::Decimal64: createAttributeImpl<Decimal64>(attr, null_value); break;
case AttributeUnderlyingType::Decimal128: createAttributeImpl<Decimal128>(attr, null_value); break;
case AttributeUnderlyingType::Decimal32:
createAttributeImpl<Decimal32>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal64:
createAttributeImpl<Decimal64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal128:
createAttributeImpl<Decimal128>(attr, null_value);
break;
case AttributeUnderlyingType::String:
{
@ -308,10 +376,10 @@ void RangeHashedDictionary::getItems(
const PaddedPODArray<RangeStorageType> & dates,
PaddedPODArray<OutputType> & out) const
{
if (false) {}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsImpl<TYPE, OutputType>(attribute, ids, dates, out);
if (false)
{
}
#define DISPATCH(TYPE) else if (attribute.type == AttributeUnderlyingType::TYPE) getItemsImpl<TYPE, OutputType>(attribute, ids, dates, out);
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
@ -327,8 +395,7 @@ void RangeHashedDictionary::getItems(
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 <typename AttributeType, typename OutputType>
@ -348,8 +415,11 @@ void RangeHashedDictionary::getItemsImpl(
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values),
[date] (const Value<AttributeType> & v) { return v.range.contains(date); });
const auto val_it
= std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value<AttributeType> & v)
{
return v.range.contains(date);
});
out[i] = static_cast<OutputType>(val_it != std::end(ranges_and_values) ? val_it->value : null_value);
}
@ -373,8 +443,8 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K
{
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const Value<T> & lhs, const Range & rhs_range)
const auto insert_it
= std::lower_bound(std::begin(values), std::end(values), range, [](const Value<T> & lhs, const Range & rhs_range)
{
return lhs.range < rhs_range;
});
@ -389,17 +459,39 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: setAttributeValueImpl<UInt8>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt16: setAttributeValueImpl<UInt16>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt32: setAttributeValueImpl<UInt32>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt64: setAttributeValueImpl<UInt64>(attribute, id, range, value.get<UInt64>()); break;
case AttributeUnderlyingType::UInt128: setAttributeValueImpl<UInt128>(attribute, id, range, value.get<UInt128>()); break;
case AttributeUnderlyingType::Int8: setAttributeValueImpl<Int8>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int16: setAttributeValueImpl<Int16>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int32: setAttributeValueImpl<Int32>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Int64: setAttributeValueImpl<Int64>(attribute, id, range, value.get<Int64>()); break;
case AttributeUnderlyingType::Float32: setAttributeValueImpl<Float32>(attribute, id, range, value.get<Float64>()); break;
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, range, value.get<Float64>()); break;
case AttributeUnderlyingType::UInt8:
setAttributeValueImpl<UInt8>(attribute, id, range, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt16:
setAttributeValueImpl<UInt16>(attribute, id, range, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt32:
setAttributeValueImpl<UInt32>(attribute, id, range, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt64:
setAttributeValueImpl<UInt64>(attribute, id, range, value.get<UInt64>());
break;
case AttributeUnderlyingType::UInt128:
setAttributeValueImpl<UInt128>(attribute, id, range, value.get<UInt128>());
break;
case AttributeUnderlyingType::Int8:
setAttributeValueImpl<Int8>(attribute, id, range, value.get<Int64>());
break;
case AttributeUnderlyingType::Int16:
setAttributeValueImpl<Int16>(attribute, id, range, value.get<Int64>());
break;
case AttributeUnderlyingType::Int32:
setAttributeValueImpl<Int32>(attribute, id, range, value.get<Int64>());
break;
case AttributeUnderlyingType::Int64:
setAttributeValueImpl<Int64>(attribute, id, range, value.get<Int64>());
break;
case AttributeUnderlyingType::Float32:
setAttributeValueImpl<Float32>(attribute, id, range, value.get<Float64>());
break;
case AttributeUnderlyingType::Float64:
setAttributeValueImpl<Float64>(attribute, id, range, value.get<Float64>());
break;
case AttributeUnderlyingType::Decimal32:
setAttributeValueImpl<Decimal32>(attribute, id, range, value.get<Decimal32>());
@ -424,8 +516,8 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i
{
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const Value<StringRef> & lhs, const Range & rhs_range)
const auto insert_it = std::lower_bound(
std::begin(values), std::end(values), range, [](const Value<StringRef> & lhs, const Range & rhs_range)
{
return lhs.range < rhs_range;
});
@ -449,45 +541,78 @@ const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttribute(con
return attributes[it->second];
}
const RangeHashedDictionary::Attribute & RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name, const AttributeUnderlyingType type) const
const RangeHashedDictionary::Attribute &
RangeHashedDictionary::getAttributeWithType(const std::string & attribute_name, const AttributeUnderlyingType type) const
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != type)
throw Exception{attribute_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};
throw Exception{attribute_name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
return attribute;
}
template <typename RangeType>
void RangeHashedDictionary::getIdsAndDates(PaddedPODArray<Key> & ids,
PaddedPODArray<RangeType> & start_dates,
PaddedPODArray<RangeType> & end_dates) const
void RangeHashedDictionary::getIdsAndDates(
PaddedPODArray<Key> & ids, PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const
{
const auto & attribute = attributes.front();
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: getIdsAndDates<UInt8>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::UInt16: getIdsAndDates<UInt16>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::UInt32: getIdsAndDates<UInt32>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::UInt64: getIdsAndDates<UInt64>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::UInt128: getIdsAndDates<UInt128>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Int8: getIdsAndDates<Int8>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Int16: getIdsAndDates<Int16>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Int32: getIdsAndDates<Int32>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Int64: getIdsAndDates<Int64>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Float32: getIdsAndDates<Float32>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Float64: getIdsAndDates<Float64>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::String: getIdsAndDates<StringRef>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::UInt8:
getIdsAndDates<UInt8>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::UInt16:
getIdsAndDates<UInt16>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::UInt32:
getIdsAndDates<UInt32>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::UInt64:
getIdsAndDates<UInt64>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::UInt128:
getIdsAndDates<UInt128>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Int8:
getIdsAndDates<Int8>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Int16:
getIdsAndDates<Int16>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Int32:
getIdsAndDates<Int32>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Int64:
getIdsAndDates<Int64>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Float32:
getIdsAndDates<Float32>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Float64:
getIdsAndDates<Float64>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::String:
getIdsAndDates<StringRef>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Decimal32: getIdsAndDates<Decimal32>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Decimal64: getIdsAndDates<Decimal64>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Decimal128: getIdsAndDates<Decimal128>(attribute, ids, start_dates, end_dates); break;
case AttributeUnderlyingType::Decimal32:
getIdsAndDates<Decimal32>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Decimal64:
getIdsAndDates<Decimal64>(attribute, ids, start_dates, end_dates);
break;
case AttributeUnderlyingType::Decimal128:
getIdsAndDates<Decimal128>(attribute, ids, start_dates, end_dates);
break;
}
}
template <typename T, typename RangeType>
void RangeHashedDictionary::getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
void RangeHashedDictionary::getIdsAndDates(
const Attribute & attribute,
PaddedPODArray<Key> & ids,
PaddedPODArray<RangeType> & start_dates,
PaddedPODArray<RangeType> & end_dates) const
{
@ -556,8 +681,8 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col
ListType::forEach(callable);
if (!callable.stream)
throw Exception("Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(),
ErrorCodes::LOGICAL_ERROR);
throw Exception(
"Unexpected range type for RangeHashed dictionary: " + dict_struct.range_min->type->getName(), ErrorCodes::LOGICAL_ERROR);
return callable.stream;
}
@ -565,13 +690,12 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col
void registerDictionaryRangeHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'range_hashed'", ErrorCodes::UNSUPPORTED_METHOD};

View File

@ -1,11 +1,11 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/HashTable/HashMap.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include <atomic>
#include <memory>
@ -14,13 +14,15 @@
namespace DB
{
class RangeHashedDictionary final : public IDictionaryBase
{
public:
RangeHashedDictionary(
const std::string & dictionary_name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty);
const std::string & dictionary_name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty);
RangeHashedDictionary(const RangeHashedDictionary & other);
@ -50,10 +52,7 @@ public:
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -88,7 +87,9 @@ public:
#undef DECLARE_MULTIPLE_GETTER
void getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<RangeStorageType> & dates,
const std::string & attribute_name,
const PaddedPODArray<Key> & ids,
const PaddedPODArray<RangeStorageType> & dates,
ColumnString * out) const;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
@ -110,25 +111,51 @@ private:
T value;
};
template <typename T> using Values = std::vector<Value<T>>;
template <typename T> using Collection = HashMap<UInt64, Values<T>>;
template <typename T> using Ptr = std::unique_ptr<Collection<T>>;
template <typename T>
using Values = std::vector<Value<T>>;
template <typename T>
using Collection = HashMap<UInt64, Values<T>>;
template <typename T>
using Ptr = std::unique_ptr<Collection<T>>;
struct Attribute final
{
public:
AttributeUnderlyingType type;
std::variant<UInt8, UInt16, UInt32, UInt64,
std::variant<
UInt8,
UInt16,
UInt32,
UInt64,
UInt128,
Int8, Int16, Int32, Int64,
Decimal32, Decimal64, Decimal128,
Float32, Float64,
String> null_values;
std::variant<Ptr<UInt8>, Ptr<UInt16>, Ptr<UInt32>, Ptr<UInt64>,
Int8,
Int16,
Int32,
Int64,
Decimal32,
Decimal64,
Decimal128,
Float32,
Float64,
String>
null_values;
std::variant<
Ptr<UInt8>,
Ptr<UInt16>,
Ptr<UInt32>,
Ptr<UInt64>,
Ptr<UInt128>,
Ptr<Int8>, Ptr<Int16>, Ptr<Int32>, Ptr<Int64>,
Ptr<Decimal32>, Ptr<Decimal64>, Ptr<Decimal128>,
Ptr<Float32>, Ptr<Float64>, Ptr<StringRef>> maps;
Ptr<Int8>,
Ptr<Int16>,
Ptr<Int32>,
Ptr<Int64>,
Ptr<Decimal32>,
Ptr<Decimal64>,
Ptr<Decimal128>,
Ptr<Float32>,
Ptr<Float64>,
Ptr<StringRef>>
maps;
std::unique_ptr<Arena> string_arena;
};
@ -172,12 +199,14 @@ private:
const Attribute & getAttributeWithType(const std::string & name, const AttributeUnderlyingType type) const;
template <typename RangeType>
void getIdsAndDates(PaddedPODArray<Key> & ids,
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
void getIdsAndDates(PaddedPODArray<Key> & ids, PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
template <typename T, typename RangeType>
void getIdsAndDates(const Attribute & attribute, PaddedPODArray<Key> & ids,
PaddedPODArray<RangeType> & start_dates, PaddedPODArray<RangeType> & end_dates) const;
void getIdsAndDates(
const Attribute & attribute,
PaddedPODArray<Key> & ids,
PaddedPODArray<RangeType> & start_dates,
PaddedPODArray<RangeType> & end_dates) const;
template <typename RangeType>
BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const;

View File

@ -1,24 +1,23 @@
#include <stack>
#include <ext/map.h>
#include <ext/range.h>
#include <Poco/Net/IPAddress.h>
#include <Poco/ByteOrder.h>
#include "TrieDictionary.h"
#include <Columns/ColumnVector.h>
#include <iostream>
#include <stack>
#include <btrie.h>
#include <Columns/ColumnFixedString.h>
#include "DictionaryBlockInputStream.h"
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h>
#include <IO/WriteIntText.h>
#include <Poco/ByteOrder.h>
#include <Poco/Net/IPAddress.h>
#include <Common/formatIPv6.h>
#include <iostream>
#include <btrie.h>
#include <ext/map.h>
#include <ext/range.h>
#include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
@ -29,10 +28,17 @@ namespace ErrorCodes
}
TrieDictionary::TrieDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty), logger(&Poco::Logger::get("TrieDictionary"))
const std::string & name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty)
: name{name}
, dict_struct(dict_struct)
, source_ptr{std::move(source_ptr)}
, dict_lifetime(dict_lifetime)
, require_nonempty(require_nonempty)
, logger(&Poco::Logger::get("TrieDictionary"))
{
createAttributes();
trie = btrie_create();
@ -62,18 +68,20 @@ TrieDictionary::~TrieDictionary()
#define DECLARE(TYPE) \
void TrieDictionary::get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
ResultArrayType<TYPE> & out) const\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & out) const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
getItemsNumber<TYPE>(attribute, key_columns,\
getItemsNumber<TYPE>( \
attribute, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t) { return null_value; }); \
}
@ -94,34 +102,42 @@ DECLARE(Decimal128)
#undef DECLARE
void TrieDictionary::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
{
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<String>(attribute.null_values)};
getItemsImpl<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
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 TrieDictionary::get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const PaddedPODArray<TYPE> & def, ResultArrayType<TYPE> & out) const\
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const PaddedPODArray<TYPE> & def, \
ResultArrayType<TYPE> & out) const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns,\
getItemsNumber<TYPE>( \
attribute, \
key_columns, \
[&](const size_t row, const auto value) { out[row] = value; }, \
[&](const size_t row) { return def[row]; }); \
}
@ -142,34 +158,43 @@ DECLARE(Decimal128)
#undef DECLARE
void TrieDictionary::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
{
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<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
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 TrieDictionary::get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const TYPE def, ResultArrayType<TYPE> & out) const\
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const TYPE def, \
ResultArrayType<TYPE> & out) const \
{ \
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};\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns,\
[&] (const size_t row, const auto value) { out[row] = value; },\
[&] (const size_t) { return def; });\
getItemsNumber<TYPE>( \
attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \
}
DECLARE(UInt8)
DECLARE(UInt16)
@ -188,16 +213,22 @@ DECLARE(Decimal128)
#undef DECLARE
void TrieDictionary::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
{
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<StringRef, StringRef>(attribute, key_columns,
getItemsImpl<StringRef, StringRef>(
attribute,
key_columns,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t) { return StringRef{def}; });
}
@ -210,22 +241,52 @@ void TrieDictionary::has(const Columns & key_columns, const DataTypes & key_type
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt128: has<UInt128>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::String: has<StringRef>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt8:
has<UInt8>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt16:
has<UInt16>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt32:
has<UInt32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt64:
has<UInt64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::UInt128:
has<UInt128>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int8:
has<Int8>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int16:
has<Int16>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int32:
has<Int32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Int64:
has<Int64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Float32:
has<Float32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Float64:
has<Float64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::String:
has<StringRef>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal32: has<Decimal32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal64: has<Decimal64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal128: has<Decimal128>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Decimal32:
has<Decimal32>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal64:
has<Decimal64>(attribute, key_columns, out);
break;
case AttributeUnderlyingType::Decimal128:
has<Decimal128>(attribute, key_columns, out);
break;
}
}
@ -240,7 +301,8 @@ void TrieDictionary::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};
}
}
@ -260,14 +322,10 @@ void TrieDictionary::loadData()
const auto rows = block.rows();
element_count += rows;
const auto key_column_ptrs = ext::map<Columns>(ext::range(0, keys_size),
[&] (const size_t attribute_idx)
{
return block.safeGetByPosition(attribute_idx).column;
});
const auto key_column_ptrs = ext::map<Columns>(
ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; });
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size),
[&] (const size_t attribute_idx)
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size), [&](const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});
@ -284,7 +342,6 @@ void TrieDictionary::loadData()
setAttributeValue(attribute, key_column->getDataAt(row_idx), attribute_column[row_idx]);
}
}
}
stream->readSuffix();
@ -309,21 +366,49 @@ void TrieDictionary::calculateBytesAllocated()
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::UInt128: addAttributeSize<UInt128>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::UInt8:
addAttributeSize<UInt8>(attribute);
break;
case AttributeUnderlyingType::UInt16:
addAttributeSize<UInt16>(attribute);
break;
case AttributeUnderlyingType::UInt32:
addAttributeSize<UInt32>(attribute);
break;
case AttributeUnderlyingType::UInt64:
addAttributeSize<UInt64>(attribute);
break;
case AttributeUnderlyingType::UInt128:
addAttributeSize<UInt128>(attribute);
break;
case AttributeUnderlyingType::Int8:
addAttributeSize<Int8>(attribute);
break;
case AttributeUnderlyingType::Int16:
addAttributeSize<Int16>(attribute);
break;
case AttributeUnderlyingType::Int32:
addAttributeSize<Int32>(attribute);
break;
case AttributeUnderlyingType::Int64:
addAttributeSize<Int64>(attribute);
break;
case AttributeUnderlyingType::Float32:
addAttributeSize<Float32>(attribute);
break;
case AttributeUnderlyingType::Float64:
addAttributeSize<Float64>(attribute);
break;
case AttributeUnderlyingType::Decimal32: addAttributeSize<Decimal32>(attribute); break;
case AttributeUnderlyingType::Decimal64: addAttributeSize<Decimal64>(attribute); break;
case AttributeUnderlyingType::Decimal128: addAttributeSize<Decimal128>(attribute); break;
case AttributeUnderlyingType::Decimal32:
addAttributeSize<Decimal32>(attribute);
break;
case AttributeUnderlyingType::Decimal64:
addAttributeSize<Decimal64>(attribute);
break;
case AttributeUnderlyingType::Decimal128:
addAttributeSize<Decimal128>(attribute);
break;
case AttributeUnderlyingType::String:
{
@ -363,21 +448,49 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::UInt128: createAttributeImpl<UInt128>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::UInt8:
createAttributeImpl<UInt8>(attr, null_value);
break;
case AttributeUnderlyingType::UInt16:
createAttributeImpl<UInt16>(attr, null_value);
break;
case AttributeUnderlyingType::UInt32:
createAttributeImpl<UInt32>(attr, null_value);
break;
case AttributeUnderlyingType::UInt64:
createAttributeImpl<UInt64>(attr, null_value);
break;
case AttributeUnderlyingType::UInt128:
createAttributeImpl<UInt128>(attr, null_value);
break;
case AttributeUnderlyingType::Int8:
createAttributeImpl<Int8>(attr, null_value);
break;
case AttributeUnderlyingType::Int16:
createAttributeImpl<Int16>(attr, null_value);
break;
case AttributeUnderlyingType::Int32:
createAttributeImpl<Int32>(attr, null_value);
break;
case AttributeUnderlyingType::Int64:
createAttributeImpl<Int64>(attr, null_value);
break;
case AttributeUnderlyingType::Float32:
createAttributeImpl<Float32>(attr, null_value);
break;
case AttributeUnderlyingType::Float64:
createAttributeImpl<Float64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal32: createAttributeImpl<Decimal32>(attr, null_value); break;
case AttributeUnderlyingType::Decimal64: createAttributeImpl<Decimal64>(attr, null_value); break;
case AttributeUnderlyingType::Decimal128: createAttributeImpl<Decimal128>(attr, null_value); break;
case AttributeUnderlyingType::Decimal32:
createAttributeImpl<Decimal32>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal64:
createAttributeImpl<Decimal64>(attr, null_value);
break;
case AttributeUnderlyingType::Decimal128:
createAttributeImpl<Decimal128>(attr, null_value);
break;
case AttributeUnderlyingType::String:
{
@ -394,15 +507,14 @@ TrieDictionary::Attribute TrieDictionary::createAttributeWithType(const Attribut
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
void TrieDictionary::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<TYPE, OutputType>(attribute, key_columns, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(get_default));
else if (attribute.type == AttributeUnderlyingType::TYPE) getItemsImpl<TYPE, OutputType>( \
attribute, key_columns, std::forward<ValueSetter>(set_value), std::forward<DefaultGetter>(get_default));
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
@ -418,16 +530,12 @@ void TrieDictionary::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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
void TrieDictionary::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
{
auto & vec = std::get<ContainerType<AttributeType>>(attribute.maps);
@ -473,7 +581,6 @@ bool TrieDictionary::setAttributeValueImpl(Attribute & attribute, const StringRe
size_t pos = addr_str.find('/');
if (pos != std::string::npos)
{
addr = Poco::Net::IPAddress(addr_str.substr(0, pos));
mask = Poco::Net::IPAddress(std::stoi(addr_str.substr(pos + 1), nullptr, 10), addr.family());
}
@ -506,21 +613,35 @@ bool TrieDictionary::setAttributeValue(Attribute & attribute, const StringRef ke
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: return setAttributeValueImpl<UInt8>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt16: return setAttributeValueImpl<UInt16>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt32: return setAttributeValueImpl<UInt32>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt64: return setAttributeValueImpl<UInt64>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt128: return setAttributeValueImpl<UInt128>(attribute, key, value.get<UInt128>());
case AttributeUnderlyingType::Int8: return setAttributeValueImpl<Int8>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int16: return setAttributeValueImpl<Int16>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int32: return setAttributeValueImpl<Int32>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int64: return setAttributeValueImpl<Int64>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Float32: return setAttributeValueImpl<Float32>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Float64: return setAttributeValueImpl<Float64>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::UInt8:
return setAttributeValueImpl<UInt8>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt16:
return setAttributeValueImpl<UInt16>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt32:
return setAttributeValueImpl<UInt32>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt64:
return setAttributeValueImpl<UInt64>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt128:
return setAttributeValueImpl<UInt128>(attribute, key, value.get<UInt128>());
case AttributeUnderlyingType::Int8:
return setAttributeValueImpl<Int8>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int16:
return setAttributeValueImpl<Int16>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int32:
return setAttributeValueImpl<Int32>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int64:
return setAttributeValueImpl<Int64>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Float32:
return setAttributeValueImpl<Float32>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Float64:
return setAttributeValueImpl<Float64>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Decimal32: return setAttributeValueImpl<Decimal32>(attribute, key, value.get<Decimal32>());
case AttributeUnderlyingType::Decimal64: return setAttributeValueImpl<Decimal64>(attribute, key, value.get<Decimal64>());
case AttributeUnderlyingType::Decimal128: return setAttributeValueImpl<Decimal128>(attribute, key, value.get<Decimal128>());
case AttributeUnderlyingType::Decimal32:
return setAttributeValueImpl<Decimal32>(attribute, key, value.get<Decimal32>());
case AttributeUnderlyingType::Decimal64:
return setAttributeValueImpl<Decimal64>(attribute, key, value.get<Decimal64>());
case AttributeUnderlyingType::Decimal128:
return setAttributeValueImpl<Decimal128>(attribute, key, value.get<Decimal128>());
case AttributeUnderlyingType::String:
{
@ -642,8 +763,8 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
auto getKeys = [](const Columns & columns, const std::vector<DictionaryAttribute> & attributes)
{
const auto & attr = attributes.front();
return ColumnsWithTypeAndName({ColumnWithTypeAndName(columns.front(),
std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH), attr.name)});
return ColumnsWithTypeAndName(
{ColumnWithTypeAndName(columns.front(), std::make_shared<DataTypeFixedString>(IPV6_BINARY_LENGTH), attr.name)});
};
auto getView = [](const Columns & columns, const std::vector<DictionaryAttribute> & attributes)
{
@ -660,22 +781,22 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
auto size = detail::writeUIntText(mask, ptr);
column->insertData(buffer, size + (ptr - buffer));
}
return ColumnsWithTypeAndName{ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), attributes.front().name)};
return ColumnsWithTypeAndName{
ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), attributes.front().name)};
};
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getKeyColumns(), column_names,
std::move(getKeys), std::move(getView));
return std::make_shared<BlockInputStreamType>(
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(getKeys), std::move(getView));
}
void registerDictionaryTrie(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
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 {
DictionarySourcePtr source_ptr) -> DictionaryPtr
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'ip_trie'", ErrorCodes::BAD_ARGUMENTS};

View File

@ -1,31 +1,33 @@
#pragma once
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
#include <common/StringRef.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <ext/range.h>
#include <atomic>
#include <memory>
#include <variant>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <common/StringRef.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
struct btrie_s;
typedef struct btrie_s btrie_t;
namespace DB
{
class TrieDictionary final : public IDictionaryBase
{
public:
TrieDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty);
const std::string & name,
const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime,
bool require_nonempty);
TrieDictionary(const TrieDictionary & other);
@ -59,10 +61,7 @@ public:
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override { return creation_time; }
bool isInjective(const std::string & attribute_name) const override
{
@ -74,7 +73,31 @@ public:
#define DECLARE(TYPE) \
void get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & 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<TYPE> & def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
@ -93,13 +116,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<TYPE> & def, ResultArrayType<TYPE> & out) const;
const std::string & attribute_name, \
const Columns & key_columns, \
const DataTypes & key_types, \
const TYPE def, \
ResultArrayType<TYPE> & out) const;
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
@ -117,57 +146,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<TYPE> & 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<UInt8> & out) const;
BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override;
private:
template <typename Value> using ContainerType = std::vector<Value>;
template <typename Value>
using ContainerType = std::vector<Value>;
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<UInt8>, ContainerType<UInt16>, ContainerType<UInt32>, ContainerType<UInt64>,
ContainerType<UInt8>,
ContainerType<UInt16>,
ContainerType<UInt32>,
ContainerType<UInt64>,
ContainerType<UInt128>,
ContainerType<Int8>, ContainerType<Int16>, ContainerType<Int32>, ContainerType<Int64>,
ContainerType<Decimal32>, ContainerType<Decimal64>, ContainerType<Decimal128>,
ContainerType<Float32>, ContainerType<Float64>,
ContainerType<StringRef>> maps;
ContainerType<Int8>,
ContainerType<Int16>,
ContainerType<Int32>,
ContainerType<Int64>,
ContainerType<Decimal32>,
ContainerType<Decimal64>,
ContainerType<Decimal128>,
ContainerType<Float32>,
ContainerType<Float64>,
ContainerType<StringRef>>
maps;
std::unique_ptr<Arena> string_arena;
};
@ -189,18 +218,12 @@ private:
template <typename OutputType, typename ValueSetter, typename DefaultGetter>
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 <typename AttributeType, typename OutputType, typename ValueSetter, typename DefaultGetter>
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 <typename T>

View File

@ -1,21 +1,21 @@
#include "XDBCDictionarySource.h"
#include <common/logger_useful.h>
#include <common/LocalDateTime.h>
#include <Poco/Ext/SessionPoolHelpers.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Net/HTTPRequest.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include "readInvalidateQuery.h"
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/FormatFactory.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Poco/Ext/SessionPoolHelpers.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/XDBCBridgeHelper.h>
#include <common/LocalDateTime.h>
#include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "readInvalidateQuery.h"
#include <Common/config.h>
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
@ -24,7 +24,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
@ -35,32 +34,27 @@ namespace
class XDBCBridgeBlockInputStream : public IProfilingBlockInputStream
{
public:
XDBCBridgeBlockInputStream(const Poco::URI & uri,
XDBCBridgeBlockInputStream(
const Poco::URI & uri,
std::function<void(std::ostream &)> callback,
const Block & sample_block,
const Context & context,
size_t max_block_size,
const ConnectionTimeouts & timeouts, const String name) : name(name)
const ConnectionTimeouts & timeouts,
const String name)
: name(name)
{
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, callback, timeouts);
reader = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, context, max_block_size);
reader
= FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, context, max_block_size);
}
Block getHeader() const override
{
return reader->getHeader();
}
Block getHeader() const override { return reader->getHeader(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
private:
Block readImpl() override
{
return reader->read();
}
Block readImpl() override { return reader->read(); }
String name;
std::unique_ptr<ReadWriteBufferFromHTTP> read_buf;
@ -71,23 +65,27 @@ namespace
static const size_t max_block_size = 8192;
XDBCDictionarySource::XDBCDictionarySource(const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix_,
const Block & sample_block_, const Context & context_, const BridgeHelperPtr bridge_)
: log(&Logger::get(bridge_->getName() + "DictionarySource")),
update_time{std::chrono::system_clock::from_time_t(0)},
dict_struct{dict_struct_},
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", "")},
sample_block{sample_block_},
query_builder{dict_struct, db, table, where, bridge_->getIdentifierQuotingStyle()},
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config_.getString(config_prefix_ + ".invalidate_query", "")},
bridge_helper{bridge_},
timeouts{ConnectionTimeouts::getHTTPTimeouts(context_.getSettingsRef())},
global_context(context_)
XDBCDictionarySource::XDBCDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config_,
const std::string & config_prefix_,
const Block & sample_block_,
const Context & context_,
const BridgeHelperPtr bridge_)
: log(&Logger::get(bridge_->getName() + "DictionarySource"))
, update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, 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", "")}
, sample_block{sample_block_}
, query_builder{dict_struct, db, table, where, bridge_->getIdentifierQuotingStyle()}
, load_all_query{query_builder.composeLoadAllQuery()}
, invalidate_query{config_.getString(config_prefix_ + ".invalidate_query", "")}
, bridge_helper{bridge_}
, timeouts{ConnectionTimeouts::getHTTPTimeouts(context_.getSettingsRef())}
, global_context(context_)
{
bridge_url = bridge_helper->getMainURI();
@ -98,24 +96,23 @@ XDBCDictionarySource::XDBCDictionarySource(const DictionaryStructure & dict_stru
/// copy-constructor is provided in order to support cloneability
XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other)
: log(&Logger::get(other.bridge_helper->getName() + "DictionarySource")),
update_time{other.update_time},
dict_struct{other.dict_struct},
db{other.db},
table{other.table},
where{other.where},
update_field{other.update_field},
sample_block{other.sample_block},
query_builder{dict_struct, db, table, where, other.bridge_helper->getIdentifierQuotingStyle()},
load_all_query{other.load_all_query},
invalidate_query{other.invalidate_query},
invalidate_query_response{other.invalidate_query_response},
bridge_helper{other.bridge_helper},
bridge_url{other.bridge_url},
timeouts{other.timeouts},
global_context{other.global_context}
: log(&Logger::get(other.bridge_helper->getName() + "DictionarySource"))
, update_time{other.update_time}
, dict_struct{other.dict_struct}
, db{other.db}
, table{other.table}
, where{other.where}
, update_field{other.update_field}
, sample_block{other.sample_block}
, query_builder{dict_struct, db, table, where, other.bridge_helper->getIdentifierQuotingStyle()}
, load_all_query{other.load_all_query}
, invalidate_query{other.invalidate_query}
, invalidate_query_response{other.invalidate_query_response}
, bridge_helper{other.bridge_helper}
, bridge_url{other.bridge_url}
, timeouts{other.timeouts}
, global_context{other.global_context}
{
}
std::string XDBCDictionarySource::getUpdateFieldAndDate()
@ -156,8 +153,7 @@ BlockInputStreamPtr XDBCDictionarySource::loadIds(const std::vector<UInt64> & id
return loadBase(query);
}
BlockInputStreamPtr XDBCDictionarySource::loadKeys(
const Columns & key_columns, const std::vector<size_t> & requested_rows)
BlockInputStreamPtr XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
return loadBase(query);
@ -215,7 +211,8 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request)
invalidate_sample_block,
global_context,
max_block_size,
timeouts, bridge_helper->getName() + "BlockInputStream");
timeouts,
bridge_helper->getName() + "BlockInputStream");
return readInvalidateQuery(stream);
}
@ -223,12 +220,14 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request)
BlockInputStreamPtr XDBCDictionarySource::loadBase(const std::string & query) const
{
bridge_helper->startBridgeSync();
return std::make_shared<XDBCBridgeBlockInputStream>(bridge_url,
return std::make_shared<XDBCBridgeBlockInputStream>(
bridge_url,
[query](std::ostream & os) { os << "query=" << query; },
sample_block,
global_context,
max_block_size,
timeouts, bridge_helper->getName() + "BlockInputStream");
timeouts,
bridge_helper->getName() + "BlockInputStream");
}
void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
@ -243,7 +242,8 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
Block & sample_block,
Context & context) -> DictionarySourcePtr {
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge);
#else
(void)dict_struct;

View File

@ -1,12 +1,12 @@
#pragma once
#include <IO/ConnectionTimeouts.h>
#include <Poco/Data/SessionPool.h>
#include <Poco/URI.h>
#include <Common/XDBCBridgeHelper.h>
#include "DictionaryStructure.h"
#include "ExternalQueryBuilder.h"
#include "IDictionarySource.h"
#include <IO/ConnectionTimeouts.h>
#include <Common/XDBCBridgeHelper.h>
namespace Poco
@ -26,7 +26,8 @@ namespace DB
class XDBCDictionarySource final : public IDictionarySource
{
public:
XDBCDictionarySource(const DictionaryStructure & dict_struct_,
XDBCDictionarySource(
const DictionaryStructure & dict_struct_,
const Poco::Util::AbstractConfiguration & config_,
const std::string & config_prefix_,
const Block & sample_block_,

View File

@ -3,7 +3,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MANY_COLUMNS;

View File

@ -5,7 +5,6 @@ class IProfilingBlockInputStream;
namespace DB
{
// Using in MySQLDictionarySource and XDBCDictionarySource after processing invalidate_query
std::string readInvalidateQuery(IProfilingBlockInputStream & block_input_stream);

Some files were not shown because too many files have changed in this diff Show More