CLICKHOUSE-4137 DictionaryFactory, DictionarySourceFactory (#3653)

* Split ComplexKeyCacheDictionary to faster compile (part2)

* Dictionaries as lib WIP

* wip

* clean

* Fix build with old capnp

* fix

* wip

* fixes

* fix

* clean

* clean

* clean

* wip

* wip

* wip

* flat

* wip

* cache

* clean

* wip

* faster

* fix style

* fixes

* clean

* clean

* Split CacheDictionary.cpp for faster compile

* fix

* fix

* Less memory usage while compiling

* missing file

* format

* Update registerDictionaries.h

* clean
This commit is contained in:
proller 2018-11-28 14:37:12 +03:00 committed by alexey-milovidov
parent 0dfa73221c
commit dd2371e071
101 changed files with 1407 additions and 1057 deletions

View File

@ -0,0 +1,5 @@
function(generate_code TEMPLATE_FILE)
foreach(NAME IN LISTS ARGN)
configure_file (${TEMPLATE_FILE}.cpp.in ${CMAKE_CURRENT_BINARY_DIR}/generated/${TEMPLATE_FILE}_${NAME}.cpp)
endforeach()
endfunction()

View File

@ -63,9 +63,6 @@ add_headers_and_sources(dbms src/Core)
add_headers_and_sources(dbms src/DataStreams) add_headers_and_sources(dbms src/DataStreams)
add_headers_and_sources(dbms src/DataTypes) add_headers_and_sources(dbms src/DataTypes)
add_headers_and_sources(dbms src/Databases) add_headers_and_sources(dbms src/Databases)
add_headers_and_sources(dbms src/Dictionaries)
add_headers_and_sources(dbms src/Dictionaries/Embedded)
add_headers_and_sources(dbms src/Dictionaries/Embedded/GeodataProviders)
add_headers_and_sources(dbms src/Interpreters) add_headers_and_sources(dbms src/Interpreters)
add_headers_and_sources(dbms src/Interpreters/ClusterProxy) add_headers_and_sources(dbms src/Interpreters/ClusterProxy)
add_headers_and_sources(dbms src/Columns) add_headers_and_sources(dbms src/Columns)
@ -184,8 +181,11 @@ target_link_libraries (dbms
clickhouse_common_config clickhouse_common_config
PUBLIC PUBLIC
clickhouse_common_io clickhouse_common_io
pocoext PRIVATE
clickhouse_dictionaries
clickhouse_dictionaries_embedded
PUBLIC PUBLIC
pocoext
${MYSQLXX_LIBRARY} ${MYSQLXX_LIBRARY}
PRIVATE PRIVATE
${BTRIE_LIBRARIES} ${BTRIE_LIBRARIES}

View File

@ -30,6 +30,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h> #include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h> #include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h> #include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
#include <boost/program_options/options_description.hpp> #include <boost/program_options/options_description.hpp>
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
@ -142,6 +143,7 @@ try
registerAggregateFunctions(); registerAggregateFunctions();
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries();
/// Maybe useless /// Maybe useless
if (config().has("macros")) if (config().has("macros"))

View File

@ -36,6 +36,7 @@
#include <Functions/registerFunctions.h> #include <Functions/registerFunctions.h>
#include <TableFunctions/registerTableFunctions.h> #include <TableFunctions/registerTableFunctions.h>
#include <Storages/registerStorages.h> #include <Storages/registerStorages.h>
#include <Dictionaries/registerDictionaries.h>
#include <Common/Config/ConfigReloader.h> #include <Common/Config/ConfigReloader.h>
#include "HTTPHandlerFactory.h" #include "HTTPHandlerFactory.h"
#include "MetricsTransmitter.h" #include "MetricsTransmitter.h"
@ -109,6 +110,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
registerAggregateFunctions(); registerAggregateFunctions();
registerTableFunctions(); registerTableFunctions();
registerStorages(); registerStorages();
registerDictionaries();
CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get());
CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger());

View File

@ -0,0 +1,43 @@
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
include(${ClickHouse_SOURCE_DIR}/cmake/generate_code.cmake)
add_headers_and_sources(clickhouse_dictionaries .)
generate_code(ComplexKeyCacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
generate_code(ComplexKeyCacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
generate_code(ComplexKeyCacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
generate_code(CacheDictionary_generate1 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/)
add_library(clickhouse_dictionaries ${LINK_MODE} ${clickhouse_dictionaries_sources})
target_link_libraries(clickhouse_dictionaries PRIVATE clickhouse_common_io pocoext ${MYSQLXX_LIBRARY} ${BTRIE_LIBRARIES})
if(Poco_SQL_FOUND AND NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_SQL_INCLUDE_DIR})
endif()
if(USE_POCO_SQLODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} ${Poco_SQL_INCLUDE_DIR})
endif()
endif()
if(Poco_Data_FOUND)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_Data_INCLUDE_DIR})
endif()
if(USE_POCO_DATAODBC)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_DataODBC_LIBRARY} ${Poco_Data_LIBRARY})
if (NOT USE_INTERNAL_POCO_LIBRARY)
target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR})
endif()
endif()
if(USE_POCO_MONGODB)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_MongoDB_LIBRARY})
endif()
add_subdirectory(Embedded)

View File

@ -1,3 +1,5 @@
#include "CacheDictionary.h"
#include <functional> #include <functional>
#include <sstream> #include <sstream>
#include <memory> #include <memory>
@ -11,12 +13,12 @@
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Dictionaries/CacheDictionary.h> #include "DictionaryBlockInputStream.h"
#include <Dictionaries/DictionaryBlockInputStream.h>
#include <ext/size.h> #include <ext/size.h>
#include <ext/range.h> #include <ext/range.h>
#include <ext/map.h> #include <ext/map.h>
#include "DictionaryFactory.h"
#include "CacheDictionary.inc.h"
namespace ProfileEvents namespace ProfileEvents
{ {
@ -47,6 +49,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD; extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int TOO_SMALL_BUFFER_SIZE;
} }
@ -206,34 +209,6 @@ void CacheDictionary::isInConstantVector(
out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end();
} }
#define DECLARE(TYPE)\
void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const\
{\
auto & attribute = getAttribute(attribute_name);\
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\
\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItemsNumber<TYPE>(attribute, ids, out, [&] (const size_t) { return null_value; });\
}
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 CacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const void CacheDictionary::getString(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ColumnString * out) const
{ {
auto & attribute = getAttribute(attribute_name); auto & attribute = getAttribute(attribute_name);
@ -245,33 +220,6 @@ void CacheDictionary::getString(const std::string & attribute_name, const Padded
getItemsString(attribute, ids, out, [&] (const size_t) { return null_value; }); getItemsString(attribute, ids, out, [&] (const size_t) { return null_value; });
} }
#define DECLARE(TYPE)\
void CacheDictionary::get##TYPE(\
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const PaddedPODArray<TYPE> & def,\
ResultArrayType<TYPE> & out) const\
{\
auto & attribute = getAttribute(attribute_name);\
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\
\
getItemsNumber<TYPE>(attribute, ids, out, [&] (const size_t row) { return def[row]; });\
}
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 CacheDictionary::getString( void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def, const std::string & attribute_name, const PaddedPODArray<Key> & ids, const ColumnString * const def,
ColumnString * const out) const ColumnString * const out) const
@ -283,32 +231,6 @@ void CacheDictionary::getString(
getItemsString(attribute, ids, out, [&] (const size_t row) { return def->getDataAt(row); }); getItemsString(attribute, ids, out, [&] (const size_t row) { return def->getDataAt(row); });
} }
#define DECLARE(TYPE)\
void CacheDictionary::get##TYPE(\
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const\
{\
auto & attribute = getAttribute(attribute_name);\
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\
\
getItemsNumber<TYPE>(attribute, ids, out, [&] (const size_t) { return def; });\
}
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 CacheDictionary::getString( void CacheDictionary::getString(
const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def, const std::string & attribute_name, const PaddedPODArray<Key> & ids, const String & def,
ColumnString * const out) const ColumnString * const out) const
@ -487,374 +409,6 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib
return attr; return attr;
} }
template <typename OutputType, typename DefaultGetter>
void CacheDictionary::getItemsNumber(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const
{
if (false) {}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsNumberImpl<TYPE, OutputType>(attribute, ids, out, std::forward<DefaultGetter>(get_default));
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
DISPATCH(UInt64)
DISPATCH(UInt128)
DISPATCH(Int8)
DISPATCH(Int16)
DISPATCH(Int32)
DISPATCH(Int64)
DISPATCH(Float32)
DISPATCH(Float64)
DISPATCH(Decimal32)
DISPATCH(Decimal64)
DISPATCH(Decimal128)
#undef DISPATCH
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
{
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std::unordered_map<Key, std::vector<size_t>> outdated_ids;
auto & attribute_array = std::get<ContainerPtrType<AttributeType>>(attribute.arrays);
const auto rows = ext::size(ids);
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows))
{
const auto id = ids[row];
/** cell should be updated if either:
* 1. ids do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
outdated_ids[id].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
out[row] = cell.isDefault() ? get_default(row) : static_cast<OutputType>(attribute_array[cell_idx]);
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
if (outdated_ids.empty())
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; });
/// request new values
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
for (const size_t row : outdated_ids[id])
out[row] = static_cast<OutputType>(attribute_value);
},
[&] (const auto id, const auto)
{
for (const size_t row : outdated_ids[id])
out[row] = get_default(row);
});
}
template <typename DefaultGetter>
void CacheDictionary::getItemsString(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ColumnString * out,
DefaultGetter && get_default) const
{
const auto rows = ext::size(ids);
/// save on some allocations
out->getOffsets().reserve(rows);
auto & attribute_array = std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto found_outdated_values = false;
/// perform optimistic version, fallback to pessimistic if failed
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto row : ext::range(0, rows))
{
const auto id = ids[row];
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
found_outdated_values = true;
break;
}
else
{
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
}
/// optimistic code completed successfully
if (!found_outdated_values)
{
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows, std::memory_order_release);
return;
}
/// now onto the pessimistic one, discard possible partial results from the optimistic path
out->getChars().resize_assume_reserved(0);
out->getOffsets().resize_assume_reserved(0);
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std::unordered_map<Key, std::vector<size_t>> outdated_ids;
/// we are going to store every string separately
std::unordered_map<Key, String> map;
size_t total_length = 0;
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
for (const auto row : ext::range(0, ids.size()))
{
const auto id = ids[row];
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
outdated_ids[id].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
if (!cell.isDefault())
map[id] = String{string_ref};
total_length += string_ref.size + 1;
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
/// request new values
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; });
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id].size();
},
[&] (const auto id, const auto)
{
for (const auto row : outdated_ids[id])
total_length += get_default(row).size + 1;
});
}
out->getChars().reserve(total_length);
for (const auto row : ext::range(0, ext::size(ids)))
{
const auto id = ids[row];
const auto it = map.find(id);
const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row);
out->insertData(string_ref.data, string_ref.size);
}
}
template <typename PresentIdHandler, typename AbsentIdHandler>
void CacheDictionary::update(
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
};
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests};
Stopwatch watch;
auto stream = source_ptr->loadIds(requested_ids);
stream->readPrefix();
const auto now = std::chrono::system_clock::now();
while (const auto block = stream->read())
{
const auto id_column = typeid_cast<const ColumnUInt64 *>(block.safeGetByPosition(0).column.get());
if (!id_column)
throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH};
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();
});
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
const auto find_result = findCellIdx(id, now);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *column_ptrs[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[i]);
}
/// if cell id is zero and zero does not map to this cell, then the cell is unused
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// inform caller
on_cell_updated(id, cell_idx);
/// mark corresponding id as found
remaining_ids[id] = 1;
}
}
stream->readSuffix();
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size());
ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed());
}
size_t not_found_num = 0, found_num = 0;
const auto now = std::chrono::system_clock::now();
/// Check which ids have not been found and require setting null_value
for (const auto & id_found_pair : remaining_ids)
{
if (id_found_pair.second)
{
++found_num;
continue;
}
++not_found_num;
const auto id = id_found_pair.first;
const auto find_result = findCellIdx(id, now);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
/// Set null_value for each attribute
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
cell.setDefault();
/// inform caller that the cell has not been found
on_id_not_found(id, cell_idx);
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num);
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num);
ProfileEvents::increment(ProfileEvents::DictCacheRequests);
}
void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const
{ {
switch (attribute.type) switch (attribute.type)
@ -981,5 +535,41 @@ BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_na
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getCachedIds(), column_names); return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getCachedIds(), column_names);
} }
void registerDictionaryCache(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (dict_struct.key)
throw Exception {"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception {name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const auto & layout_prefix = config_prefix + ".layout";
const auto size = config.getInt(layout_prefix + ".cache.size_in_cells");
if (size == 0)
throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
if (require_nonempty)
throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
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,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Common/ArenaWithFreeLists.h> #include <Common/ArenaWithFreeLists.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>

View File

@ -0,0 +1,403 @@
#include "CacheDictionary.h"
#include <ext/size.h>
#include <ext/map.h>
#include <ext/range.h>
#include <Common/ProfilingScopedRWLock.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnsNumber.h>
namespace ProfileEvents
{
extern const Event DictCacheKeysRequested;
extern const Event DictCacheKeysRequestedMiss;
extern const Event DictCacheKeysRequestedFound;
extern const Event DictCacheKeysExpired;
extern const Event DictCacheKeysNotFound;
extern const Event DictCacheKeysHit;
extern const Event DictCacheRequestTimeNs;
extern const Event DictCacheRequests;
extern const Event DictCacheLockWriteNs;
extern const Event DictCacheLockReadNs;
}
namespace CurrentMetrics
{
extern const Metric DictCacheRequests;
}
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
template <typename OutputType, typename DefaultGetter>
void CacheDictionary::getItemsNumber(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ResultArrayType<OutputType> & out,
DefaultGetter && get_default) const
{
if (false) {}
#define DISPATCH(TYPE) \
else if (attribute.type == AttributeUnderlyingType::TYPE) \
getItemsNumberImpl<TYPE, OutputType>(attribute, ids, out, std::forward<DefaultGetter>(get_default));
DISPATCH(UInt8)
DISPATCH(UInt16)
DISPATCH(UInt32)
DISPATCH(UInt64)
DISPATCH(UInt128)
DISPATCH(Int8)
DISPATCH(Int16)
DISPATCH(Int32)
DISPATCH(Int64)
DISPATCH(Float32)
DISPATCH(Float64)
DISPATCH(Decimal32)
DISPATCH(Decimal64)
DISPATCH(Decimal128)
#undef DISPATCH
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
{
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std::unordered_map<Key, std::vector<size_t>> outdated_ids;
auto & attribute_array = std::get<ContainerPtrType<AttributeType>>(attribute.arrays);
const auto rows = ext::size(ids);
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows))
{
const auto id = ids[row];
/** cell should be updated if either:
* 1. ids do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
outdated_ids[id].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
out[row] = cell.isDefault() ? get_default(row) : static_cast<OutputType>(attribute_array[cell_idx]);
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
if (outdated_ids.empty())
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; });
/// request new values
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
for (const size_t row : outdated_ids[id])
out[row] = static_cast<OutputType>(attribute_value);
},
[&] (const auto id, const auto)
{
for (const size_t row : outdated_ids[id])
out[row] = get_default(row);
});
}
template <typename DefaultGetter>
void CacheDictionary::getItemsString(
Attribute & attribute,
const PaddedPODArray<Key> & ids,
ColumnString * out,
DefaultGetter && get_default) const
{
const auto rows = ext::size(ids);
/// save on some allocations
out->getOffsets().reserve(rows);
auto & attribute_array = std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto found_outdated_values = false;
/// perform optimistic version, fallback to pessimistic if failed
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto row : ext::range(0, rows))
{
const auto id = ids[row];
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
found_outdated_values = true;
break;
}
else
{
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
}
/// optimistic code completed successfully
if (!found_outdated_values)
{
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows, std::memory_order_release);
return;
}
/// now onto the pessimistic one, discard possible partial results from the optimistic path
out->getChars().resize_assume_reserved(0);
out->getOffsets().resize_assume_reserved(0);
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
std::unordered_map<Key, std::vector<size_t>> outdated_ids;
/// we are going to store every string separately
std::unordered_map<Key, String> map;
size_t total_length = 0;
size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0;
{
const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs};
const auto now = std::chrono::system_clock::now();
for (const auto row : ext::range(0, ids.size()))
{
const auto id = ids[row];
const auto find_result = findCellIdx(id, now);
if (!find_result.valid)
{
outdated_ids[id].push_back(row);
if (find_result.outdated)
++cache_expired;
else
++cache_not_found;
}
else
{
++cache_hit;
const auto & cell_idx = find_result.cell_idx;
const auto & cell = cells[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
if (!cell.isDefault())
map[id] = String{string_ref};
total_length += string_ref.size + 1;
}
}
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired);
ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found);
ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
/// request new values
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; });
update(required_ids,
[&] (const auto id, const auto cell_idx)
{
const auto attribute_value = attribute_array[cell_idx];
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id].size();
},
[&] (const auto id, const auto)
{
for (const auto row : outdated_ids[id])
total_length += get_default(row).size + 1;
});
}
out->getChars().reserve(total_length);
for (const auto row : ext::range(0, ext::size(ids)))
{
const auto id = ids[row];
const auto it = map.find(id);
const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row);
out->insertData(string_ref.data, string_ref.size);
}
}
template <typename PresentIdHandler, typename AbsentIdHandler>
void CacheDictionary::update(
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
};
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests};
Stopwatch watch;
auto stream = source_ptr->loadIds(requested_ids);
stream->readPrefix();
const auto now = std::chrono::system_clock::now();
while (const auto block = stream->read())
{
const auto id_column = typeid_cast<const ColumnUInt64 *>(block.safeGetByPosition(0).column.get());
if (!id_column)
throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH};
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();
});
for (const auto i : ext::range(0, ids.size()))
{
const auto id = ids[i];
const auto find_result = findCellIdx(id, now);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *column_ptrs[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[i]);
}
/// if cell id is zero and zero does not map to this cell, then the cell is unused
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// inform caller
on_cell_updated(id, cell_idx);
/// mark corresponding id as found
remaining_ids[id] = 1;
}
}
stream->readSuffix();
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size());
ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed());
}
size_t not_found_num = 0, found_num = 0;
const auto now = std::chrono::system_clock::now();
/// Check which ids have not been found and require setting null_value
for (const auto & id_found_pair : remaining_ids)
{
if (id_found_pair.second)
{
++found_num;
continue;
}
++not_found_num;
const auto id = id_found_pair.first;
const auto find_result = findCellIdx(id, now);
const auto & cell_idx = find_result.cell_idx;
auto & cell = cells[cell_idx];
/// Set null_value for each attribute
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
cell.setDefault();
/// inform caller that the cell has not been found
on_id_not_found(id, cell_idx);
}
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num);
ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num);
ProfileEvents::increment(ProfileEvents::DictCacheRequests);
}
}

View File

@ -0,0 +1,24 @@
#include <Dictionaries/CacheDictionary.h>
#include <Dictionaries/CacheDictionary.inc.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray<Key> & ids, ResultArrayType<TYPE> & out) const
{
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
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, out, [&](const size_t) { return null_value; });
}
}

View File

@ -0,0 +1,25 @@
#include <Dictionaries/CacheDictionary.h>
#include <Dictionaries/CacheDictionary.inc.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void CacheDictionary::get@NAME@(const std::string & attribute_name,
const PaddedPODArray<Key> & ids,
const PaddedPODArray<TYPE> & def,
ResultArrayType<TYPE> & out) const
{
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
getItemsNumber<TYPE>(attribute, ids, out, [&](const size_t row) { return def[row]; });
}
}

View File

@ -0,0 +1,22 @@
#include <Dictionaries/CacheDictionary.h>
#include <Dictionaries/CacheDictionary.inc.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void CacheDictionary::get@NAME@(const std::string & attribute_name, const PaddedPODArray<Key> & ids, const TYPE def, ResultArrayType<TYPE> & out) const
{
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
getItemsNumber<TYPE>(attribute, ids, out, [&](const size_t) { return def; });
}
}

View File

@ -1,14 +1,17 @@
#include <Dictionaries/ClickHouseDictionarySource.h> #include "ClickHouseDictionarySource.h"
#include <Dictionaries/ExternalQueryBuilder.h> #include "ExternalQueryBuilder.h"
#include <Dictionaries/writeParenthesisedString.h> #include "writeParenthesisedString.h"
#include <Client/ConnectionPool.h> #include <Client/ConnectionPool.h>
#include <DataStreams/RemoteBlockInputStream.h> #include <DataStreams/RemoteBlockInputStream.h>
#include <Dictionaries/readInvalidateQuery.h> #include "readInvalidateQuery.h"
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <memory> #include <memory>
#include <ext/range.h> #include <ext/range.h>
#include <IO/ConnectionTimeouts.h> #include <IO/ConnectionTimeouts.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB namespace DB
{ {
@ -175,4 +178,17 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
} }
} }
void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr {
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
};
factory.registerSource("clickhouse", createTableSource);
}
} }

View File

@ -1,10 +1,9 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/ExternalQueryBuilder.h> #include "ExternalQueryBuilder.h"
#include <Client/ConnectionPoolWithFailover.h> #include <Client/ConnectionPoolWithFailover.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <memory> #include <memory>

View File

@ -1,5 +1,5 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h> #include "ComplexKeyCacheDictionary.h"
#include <Dictionaries/DictionaryBlockInputStream.h> #include "DictionaryBlockInputStream.h"
#include <Common/Arena.h> #include <Common/Arena.h>
#include <Common/BitHelpers.h> #include <Common/BitHelpers.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
@ -9,6 +9,7 @@
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <ext/range.h> #include <ext/range.h>
#include <ext/map.h> #include <ext/map.h>
#include "DictionaryFactory.h"
namespace ProfileEvents namespace ProfileEvents
@ -39,6 +40,7 @@ namespace ErrorCodes
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_METHOD; extern const int UNSUPPORTED_METHOD;
extern const int TOO_SMALL_BUFFER_SIZE;
} }
@ -378,4 +380,32 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names &
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names); return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, keys, column_names);
} }
void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
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";
const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells");
if (size == 0)
throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
if (require_nonempty)
throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"};
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("complex_key_cache", create_layout);
}
} }

View File

@ -12,9 +12,9 @@
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Common/ProfilingScopedRWLock.h> #include <Common/ProfilingScopedRWLock.h>
#include <Common/SmallObjectPool.h> #include <Common/SmallObjectPool.h>
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <common/StringRef.h> #include <common/StringRef.h>
#include <ext/bit_cast.h> #include <ext/bit_cast.h>
#include <ext/map.h> #include <ext/map.h>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h> #include "ComplexKeyCacheDictionary.h"
namespace DB namespace DB
{ {

View File

@ -1,40 +0,0 @@
#include "ComplexKeyCacheDictionary.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
#define DECLARE(TYPE) \
void ComplexKeyCacheDictionary::get##TYPE( \
const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & out) const \
{ \
dict_struct.validateKeyTypes(key_types); \
\
auto & attribute = getAttribute(attribute_name); \
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
const auto null_value = std::get<TYPE>(attribute.null_values); \
\
getItemsNumber<TYPE>(attribute, key_columns, out, [&](const size_t) { return null_value; }); \
}
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
}

View File

@ -0,0 +1,24 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void ComplexKeyCacheDictionary::get@NAME@(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType<TYPE> & out) const
{
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
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, out, [&](const size_t) { return null_value; });
}
}

View File

@ -1,41 +0,0 @@
#include "ComplexKeyCacheDictionary.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
#define DECLARE(TYPE) \
void ComplexKeyCacheDictionary::get##TYPE(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); \
\
auto & attribute = getAttribute(attribute_name); \
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns, out, [&](const size_t row) { return def[row]; }); \
}
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
}

View File

@ -0,0 +1,27 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void ComplexKeyCacheDictionary::get@NAME@(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);
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
getItemsNumber<TYPE>(attribute, key_columns, out, [&](const size_t row) { return def[row]; });
}
}

View File

@ -1,41 +0,0 @@
#include "ComplexKeyCacheDictionary.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
#define DECLARE(TYPE) \
void ComplexKeyCacheDictionary::get##TYPE(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); \
\
auto & attribute = getAttribute(attribute_name); \
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \
throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \
ErrorCodes::TYPE_MISMATCH}; \
\
getItemsNumber<TYPE>(attribute, key_columns, out, [&](const size_t) { return def; }); \
}
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
}

View File

@ -0,0 +1,27 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
}
using TYPE = @NAME@;
void ComplexKeyCacheDictionary::get@NAME@(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);
auto & attribute = getAttribute(attribute_name);
if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::@NAME@))
throw Exception {name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH};
getItemsNumber<TYPE>(attribute, key_columns, out, [&](const size_t) { return def; });
}
}

View File

@ -1,4 +1,4 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h> #include "ComplexKeyCacheDictionary.h"
namespace DB namespace DB
{ {

View File

@ -1,4 +1,4 @@
#include <Dictionaries/ComplexKeyCacheDictionary.h> #include "ComplexKeyCacheDictionary.h"
namespace DB namespace DB
{ {

View File

@ -1,8 +1,8 @@
#include <ext/map.h> #include <ext/map.h>
#include <ext/range.h> #include <ext/range.h>
#include <Dictionaries/ComplexKeyHashedDictionary.h> #include "ComplexKeyHashedDictionary.h"
#include <Dictionaries/DictionaryBlockInputStream.h> #include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace DB namespace DB
{ {
@ -661,5 +661,24 @@ BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getKeys(), column_names); return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getKeys(), column_names);
} }
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (!dict_struct.key)
throw Exception {"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS};
const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("complex_key_hashed", create_layout);
}
} }

View File

@ -1,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <common/StringRef.h> #include <common/StringRef.h>
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
@ -243,5 +243,4 @@ private:
BlockPtr saved_block; BlockPtr saved_block;
}; };
} }

View File

@ -6,9 +6,9 @@
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Dictionaries/DictionaryBlockInputStreamBase.h> #include "DictionaryBlockInputStreamBase.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <ext/range.h> #include <ext/range.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Core/Names.h> #include <Core/Names.h>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/DictionaryBlockInputStreamBase.h> #include "DictionaryBlockInputStreamBase.h"
namespace DB namespace DB
{ {

View File

@ -0,0 +1,51 @@
#include "DictionaryFactory.h"
#include <memory>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB
{
namespace ErrorCodes
{
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
}
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout)
{
//LOG_DEBUG(log, "Register dictionary layout type `" + layout_type + "`");
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
}
DictionaryPtr DictionaryFactory::create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const
{
Poco::Util::AbstractConfiguration::Keys keys;
const auto & layout_prefix = config_prefix + ".layout";
config.keys(layout_prefix, keys);
if (keys.size() != 1)
throw Exception {name + ": element dictionary.layout should have exactly one child element",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
const DictionaryStructure dict_struct {config, config_prefix + ".structure"};
auto source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
const auto & layout_type = keys.front();
{
const auto found = registered_layouts.find(layout_type);
if (found != registered_layouts.end())
{
const auto & create_layout = found->second;
return create_layout(name, dict_struct, config, config_prefix, std::move(source_ptr));
}
}
throw Exception {name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}
}

View File

@ -1,20 +1,41 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <ext/singleton.h> #include <ext/singleton.h>
#include "IDictionary.h"
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
class Logger;
}
namespace DB namespace DB
{ {
class Context; class Context;
class DictionaryFactory : public ext::singleton<DictionaryFactory> class DictionaryFactory : public ext::singleton<DictionaryFactory>
{ {
public: public:
DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, DictionaryPtr
const std::string & config_prefix, Context & context) const; create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context)
const;
using Creator = std::function<DictionaryPtr(
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr)>;
void registerLayout(const std::string & layout_type, Creator create_layout);
private:
using LayoutRegistry = std::unordered_map<std::string, Creator>;
LayoutRegistry registered_layouts;
}; };
} }

View File

@ -1,41 +1,16 @@
#include <Dictionaries/DictionarySourceFactory.h> #include "DictionarySourceFactory.h"
#include <Core/Block.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/FileDictionarySource.h>
#include <Dictionaries/ClickHouseDictionarySource.h>
#include <Dictionaries/ExecutableDictionarySource.h>
#include <Dictionaries/HTTPDictionarySource.h>
#include <Dictionaries/LibraryDictionarySource.h>
#include <Dictionaries/XDBCDictionarySource.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/FieldVisitors.h>
#include <Common/XDBCBridgeHelper.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <IO/HTTPCommon.h> #include <Core/Block.h>
#include <memory> #include <Core/ColumnWithTypeAndName.h>
#include <mutex> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/config.h>
#if USE_POCO_MONGODB
#include <Dictionaries/MongoDBDictionarySource.h>
#endif
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
#include <Poco/Data/ODBC/Connector.h>
#endif
#if USE_MYSQL
#include <Dictionaries/MySQLDictionarySource.h>
#endif
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include "DictionaryStructure.h"
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int UNKNOWN_ELEMENT_IN_CONFIG; extern const int UNKNOWN_ELEMENT_IN_CONFIG;
@ -46,149 +21,78 @@ namespace ErrorCodes
namespace namespace
{ {
Block createSampleBlock(const DictionaryStructure & dict_struct)
Block createSampleBlock(const DictionaryStructure & dict_struct)
{
Block block;
if (dict_struct.id)
block.insert(ColumnWithTypeAndName{ColumnUInt64::create(1, 0), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
if (dict_struct.key)
{ {
for (const auto & attribute : *dict_struct.key) Block block;
if (dict_struct.id)
block.insert(ColumnWithTypeAndName {ColumnUInt64::create(1, 0), std::make_shared<DataTypeUInt64>(), dict_struct.id->name});
if (dict_struct.key)
{
for (const auto & attribute : *dict_struct.key)
{
auto column = attribute.type->createColumn();
column->insertDefault();
block.insert(ColumnWithTypeAndName {std::move(column), attribute.type, attribute.name});
}
}
if (dict_struct.range_min)
{
for (const auto & attribute : {dict_struct.range_min, dict_struct.range_max})
{
const auto & type = std::make_shared<DataTypeNullable>(attribute->type);
auto column = type->createColumn();
column->insertDefault();
block.insert(ColumnWithTypeAndName {std::move(column), type, attribute->name});
}
}
for (const auto & attribute : dict_struct.attributes)
{ {
auto column = attribute.type->createColumn(); auto column = attribute.type->createColumn();
column->insertDefault(); column->insert(attribute.null_value);
block.insert(ColumnWithTypeAndName{std::move(column), attribute.type, attribute.name}); block.insert(ColumnWithTypeAndName {std::move(column), attribute.type, attribute.name});
} }
return block;
} }
if (dict_struct.range_min)
{
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
{
const auto & type = std::make_shared<DataTypeNullable>(attribute->type);
auto column = type->createColumn();
column->insertDefault();
block.insert(ColumnWithTypeAndName{std::move(column), type, attribute->name});
}
}
for (const auto & attribute : dict_struct.attributes)
{
auto column = attribute.type->createColumn();
column->insert(attribute.null_value);
block.insert(ColumnWithTypeAndName{std::move(column), attribute.type, attribute.name});
}
return block;
}
} }
DictionarySourceFactory::DictionarySourceFactory() DictionarySourceFactory::DictionarySourceFactory() : log(&Poco::Logger::get("DictionarySourceFactory"))
: log(&Poco::Logger::get("DictionarySourceFactory"))
{ {
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
Poco::Data::ODBC::Connector::registerConnector();
#endif
} }
void DictionarySourceFactory::registerSource(const std::string & source_type, Creator create_source) void DictionarySourceFactory::registerSource(const std::string & source_type, Creator create_source)
{ {
LOG_DEBUG(log, "Register dictionary source type `" + source_type + "`"); LOG_DEBUG(log, "Register dictionary source type `" + source_type + "`");
if (!registered_sources.emplace(source_type, std::move(create_source)).second) if (!registered_sources.emplace(source_type, std::move(create_source)).second)
throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", throw Exception("DictionarySourceFactory: the source name '" + source_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
ErrorCodes::LOGICAL_ERROR);
} }
DictionarySourcePtr DictionarySourceFactory::create( DictionarySourcePtr DictionarySourceFactory::create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & name,
const DictionaryStructure & dict_struct, Context & context) const const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
Context & context) const
{ {
Poco::Util::AbstractConfiguration::Keys keys; Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys); config.keys(config_prefix, keys);
if (keys.size() != 1) if (keys.size() != 1)
throw Exception{name +": element dictionary.source should have exactly one child element", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG}; throw Exception {name + ": element dictionary.source should have exactly one child element",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
auto sample_block = createSampleBlock(dict_struct); auto sample_block = createSampleBlock(dict_struct);
const auto & source_type = keys.front(); const auto & source_type = keys.front();
if ("file" == source_type)
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
const auto filename = config.getString(config_prefix + ".file.path");
const auto format = config.getString(config_prefix + ".file.format");
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
}
else if ("mysql" == source_type)
{
#if USE_MYSQL
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);
#else
throw Exception{"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
}
else if ("clickhouse" == source_type)
{
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse",
sample_block, context);
}
else if ("mongodb" == source_type)
{
#if USE_POCO_MONGODB
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb", sample_block);
#else
throw Exception{"Dictionary source of type `mongodb` is disabled because poco library was built without mongodb support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
}
else if ("odbc" == source_type)
{
#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"));
return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge);
#else
throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
}
else if ("jdbc" == source_type)
{
throw Exception{"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
ErrorCodes::SUPPORT_IS_DISABLED};
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
// return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge);
}
else if ("executable" == source_type)
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
return std::make_unique<ExecutableDictionarySource>(dict_struct, config, config_prefix + ".executable", sample_block, context);
}
else if ("http" == source_type)
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
}
else if ("library" == source_type)
{
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context);
}
else
{ {
const auto found = registered_sources.find(source_type); const auto found = registered_sources.find(source_type);
if (found != registered_sources.end()) if (found != registered_sources.end())
@ -198,7 +102,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
} }
} }
throw Exception{name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG}; throw Exception {name + ": unknown dictionary source type: " + source_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
} }
} }

View File

@ -1,23 +1,22 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <ext/singleton.h>
#include <unordered_map> #include <unordered_map>
#include <ext/singleton.h>
namespace Poco namespace Poco
{ {
namespace Util namespace Util
{ {
class AbstractConfiguration; class AbstractConfiguration;
} }
class Logger; class Logger;
} }
namespace DB namespace DB
{ {
class Context; class Context;
struct DictionaryStructure; struct DictionaryStructure;
@ -30,15 +29,18 @@ public:
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
Block & sample_block, Block & sample_block,
const Context & context)>; Context & context)>;
DictionarySourceFactory(); DictionarySourceFactory();
void registerSource(const std::string & source_type, Creator create_source); void registerSource(const std::string & source_type, Creator create_source);
DictionarySourcePtr create( DictionarySourcePtr create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & name,
const DictionaryStructure & dict_struct, Context & context) const; const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
Context & context) const;
private: private:
using SourceRegistry = std::unordered_map<std::string, Creator>; using SourceRegistry = std::unordered_map<std::string, Creator>;

View File

@ -1,5 +1,5 @@
#include <Dictionaries/DictionarySourceHelpers.h> #include "DictionarySourceHelpers.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Core/ColumnWithTypeAndName.h> #include <Core/ColumnWithTypeAndName.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <DataTypes/DataTypeFactory.h> #include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>

View File

@ -0,0 +1,5 @@
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(clickhouse_dictionaries_embedded .)
add_headers_and_sources(clickhouse_dictionaries_embedded GeodataProviders)
add_library(clickhouse_dictionaries_embedded ${LINK_MODE} ${clickhouse_dictionaries_embedded_sources})
target_link_libraries(clickhouse_dictionaries_embedded PRIVATE clickhouse_common_io ${MYSQLXX_LIBRARY})

View File

@ -1,8 +1,8 @@
#include <Dictionaries/Embedded/GeoDictionariesLoader.h> #include "GeoDictionariesLoader.h"
#include <Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h>
#include <Dictionaries/Embedded/GeodataProviders/NamesProvider.h>
#include <Poco/Util/AbstractConfiguration.h>
#include "GeodataProviders/HierarchiesProvider.h"
#include "GeodataProviders/NamesProvider.h"
std::unique_ptr<RegionsHierarchies> GeoDictionariesLoader::reloadRegionsHierarchies( std::unique_ptr<RegionsHierarchies> GeoDictionariesLoader::reloadRegionsHierarchies(
const Poco::Util::AbstractConfiguration & config) const Poco::Util::AbstractConfiguration & config)

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/IGeoDictionariesLoader.h> #include "IGeoDictionariesLoader.h"
// Default implementation of geo dictionaries loader used by native server application // Default implementation of geo dictionaries loader used by native server application

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/Types.h> #include "Types.h"
#include <string> #include <string>
struct RegionEntry struct RegionEntry

View File

@ -1,8 +1,7 @@
#include <Dictionaries/Embedded/GeodataProviders/HierarchiesProvider.h> #include "HierarchiesProvider.h"
#include <Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h>
#include "HierarchyFormatReader.h"
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>

View File

@ -1,9 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h> #include "IHierarchiesProvider.h"
#include <Common/FileUpdatesTracker.h> #include <Common/FileUpdatesTracker.h>
#include <unordered_map> #include <unordered_map>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/Embedded/GeodataProviders/HierarchyFormatReader.h> #include "HierarchyFormatReader.h"
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h> #include "IHierarchiesProvider.h"
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/Entries.h> #include "Entries.h"
#include <memory> #include <memory>
#include <string> #include <string>
#include <vector> #include <vector>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/Entries.h> #include "Entries.h"
#include <memory> #include <memory>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h> #include "NamesFormatReader.h"
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/INamesProvider.h> #include "INamesProvider.h"
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>

View File

@ -1,6 +1,6 @@
#include <Dictionaries/Embedded/GeodataProviders/NamesProvider.h> #include "NamesProvider.h"
#include <Dictionaries/Embedded/GeodataProviders/NamesFormatReader.h>
#include "NamesFormatReader.h"
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/INamesProvider.h> #include "INamesProvider.h"
#include <Common/FileUpdatesTracker.h> #include <Common/FileUpdatesTracker.h>

View File

@ -1,12 +1,19 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/RegionsHierarchies.h> #include "RegionsHierarchies.h"
#include <Dictionaries/Embedded/RegionsNames.h> #include "RegionsNames.h"
#include <Poco/Util/AbstractConfiguration.h>
#include <memory> #include <memory>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
class Logger;
}
// Provides actual versions of geo dictionaries (regions hierarchies, regions names) // Provides actual versions of geo dictionaries (regions hierarchies, regions names)
// Bind data structures (RegionsHierarchies, RegionsNames) with data providers // Bind data structures (RegionsHierarchies, RegionsNames) with data providers

View File

@ -1,7 +1,6 @@
#include <Dictionaries/Embedded/RegionsHierarchies.h> #include "RegionsHierarchies.h"
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>

View File

@ -1,10 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/RegionsHierarchy.h> #include "RegionsHierarchy.h"
#include <Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h> #include "GeodataProviders/IHierarchiesProvider.h"
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <unordered_map> #include <unordered_map>

View File

@ -1,12 +1,10 @@
#include <Dictionaries/Embedded/RegionsHierarchy.h> #include "RegionsHierarchy.h"
#include <Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h>
#include "GeodataProviders/IHierarchiesProvider.h"
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <ext/singleton.h> #include <ext/singleton.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/IHierarchiesProvider.h> #include "GeodataProviders/IHierarchiesProvider.h"
#include <vector> #include <vector>
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
#include <common/Types.h> #include <common/Types.h>

View File

@ -1,11 +1,9 @@
#include <Dictionaries/Embedded/RegionsNames.h> #include "RegionsNames.h"
#include <Dictionaries/Embedded/GeodataProviders/INamesProvider.h>
#include "GeodataProviders/INamesProvider.h"
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
namespace DB namespace DB

View File

@ -1,12 +1,9 @@
#pragma once #pragma once
#include <Dictionaries/Embedded/GeodataProviders/INamesProvider.h> #include "GeodataProviders/INamesProvider.h"
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <common/Types.h> #include <common/Types.h>
#include <common/StringRef.h> #include <common/StringRef.h>
#include <string> #include <string>
#include <vector> #include <vector>

View File

@ -1,7 +1,7 @@
#include <Common/config.h> #include <Common/config.h>
#if USE_MYSQL #if USE_MYSQL
#include <Dictionaries/Embedded/TechDataHierarchy.h> #include "TechDataHierarchy.h"
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <mysqlxx/PoolWithFailover.h> #include <mysqlxx/PoolWithFailover.h>

View File

@ -1,12 +1,18 @@
#pragma once #pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Exception.h>
#include <common/Types.h> #include <common/Types.h>
#include <ext/singleton.h> #include <ext/singleton.h>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
class Logger;
}
/** @brief Class that lets you know if a search engine or operating system belongs /** @brief Class that lets you know if a search engine or operating system belongs
* another search engine or operating system, respectively. * another search engine or operating system, respectively.

View File

@ -1,12 +1,15 @@
#include "ExecutableDictionarySource.h"
#include <thread> #include <thread>
#include <future> #include <future>
#include <Dictionaries/ExecutableDictionarySource.h>
#include <Common/ShellCommand.h> #include <Common/ShellCommand.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <DataStreams/OwningBlockInputStream.h> #include <DataStreams/OwningBlockInputStream.h>
#include <Dictionaries/DictionarySourceHelpers.h> #include "DictionarySourceHelpers.h"
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB namespace DB
@ -229,4 +232,19 @@ std::string ExecutableDictionarySource::toString() const
return "Executable: " + command; return "Executable: " + command;
} }
void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
if (dict_struct.has_expressions)
throw Exception {"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
return std::make_unique<ExecutableDictionarySource>(dict_struct, config, config_prefix + ".executable", sample_block, context);
};
factory.registerSource("executable", createTableSource);
}
} }

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
namespace Poco { class Logger; } namespace Poco { class Logger; }

View File

@ -3,9 +3,9 @@
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Dictionaries/writeParenthesisedString.h> #include "writeParenthesisedString.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/ExternalQueryBuilder.h> #include "ExternalQueryBuilder.h"
namespace DB namespace DB

View File

@ -1,5 +1,5 @@
#include <ext/range.h> #include <ext/range.h>
#include <Dictionaries/ExternalResultDescription.h> #include "ExternalResultDescription.h"
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>

View File

@ -1,9 +1,11 @@
#include "FileDictionarySource.h"
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <DataStreams/OwningBlockInputStream.h> #include <DataStreams/OwningBlockInputStream.h>
#include <Dictionaries/FileDictionarySource.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Poco/File.h> #include <Poco/File.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB namespace DB
{ {
@ -46,4 +48,23 @@ Poco::Timestamp FileDictionarySource::getLastModification() const
return Poco::File{filename}.getLastModified(); return Poco::File{filename}.getLastModified();
} }
void registerDictionarySourceFile(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
if (dict_struct.has_expressions)
throw Exception {"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
const auto filename = config.getString(config_prefix + ".file.path");
const auto format = config.getString(config_prefix + ".file.format");
return std::make_unique<FileDictionarySource>(filename, format, sample_block, context);
};
factory.registerSource("file", createTableSource);
}
} }

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Poco/Timestamp.h> #include <Poco/Timestamp.h>
@ -9,7 +9,6 @@ namespace DB
class Context; class Context;
/// Allows loading dictionaries from a file with given format, does not support "random access" /// Allows loading dictionaries from a file with given format, does not support "random access"
class FileDictionarySource final : public IDictionarySource class FileDictionarySource final : public IDictionarySource
{ {

View File

@ -1,7 +1,7 @@
#include <Dictionaries/FlatDictionary.h> #include "FlatDictionary.h"
#include <Dictionaries/DictionaryBlockInputStream.h> #include "DictionaryBlockInputStream.h"
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include "DictionaryFactory.h"
namespace DB namespace DB
{ {
@ -14,9 +14,9 @@ namespace ErrorCodes
extern const int DICTIONARY_IS_EMPTY; extern const int DICTIONARY_IS_EMPTY;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int UNKNOWN_TYPE; extern const int UNKNOWN_TYPE;
extern const int UNSUPPORTED_METHOD;
} }
static const auto initial_array_size = 1024; static const auto initial_array_size = 1024;
static const auto max_array_size = 500000; static const auto max_array_size = 500000;
@ -634,5 +634,33 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getIds() ,column_names); return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getIds() ,column_names);
} }
void registerDictionaryFlat(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (dict_struct.key)
throw Exception {"'key' is not supported for dictionary of layout 'flat'", ErrorCodes::UNSUPPORTED_METHOD};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception {name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
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,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Common/Arena.h> #include <Common/Arena.h>

View File

@ -1,4 +1,4 @@
#include <Dictionaries/HTTPDictionarySource.h> #include "HTTPDictionarySource.h"
#include <Poco/Net/HTTPRequest.h> #include <Poco/Net/HTTPRequest.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
@ -6,9 +6,12 @@
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
#include <Dictionaries/DictionarySourceHelpers.h> #include "DictionarySourceHelpers.h"
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <IO/ConnectionTimeouts.h> #include <IO/ConnectionTimeouts.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB namespace DB
{ {
@ -149,4 +152,19 @@ std::string HTTPDictionarySource::toString() const
return uri.toString(); return uri.toString();
} }
void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
if (dict_struct.has_expressions)
throw Exception {"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};
return std::make_unique<HTTPDictionarySource>(dict_struct, config, config_prefix + ".http", sample_block, context);
};
factory.registerSource("http", createTableSource);
}
} }

View File

@ -1,8 +1,8 @@
#pragma once #pragma once
#include <Poco/URI.h> #include <Poco/URI.h>
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
#include <IO/ConnectionTimeouts.h> #include <IO/ConnectionTimeouts.h>

View File

@ -1,6 +1,7 @@
#include <ext/size.h> #include <ext/size.h>
#include <Dictionaries/HashedDictionary.h> #include "HashedDictionary.h"
#include <Dictionaries/DictionaryBlockInputStream.h> #include "DictionaryBlockInputStream.h"
#include "DictionaryFactory.h"
namespace DB namespace DB
{ {
@ -11,6 +12,7 @@ namespace ErrorCodes
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY; extern const int DICTIONARY_IS_EMPTY;
extern const int UNSUPPORTED_METHOD;
} }
@ -623,4 +625,29 @@ BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_n
return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getIds(), column_names); return std::make_shared<BlockInputStreamType>(shared_from_this(), max_block_size, getIds(), column_names);
} }
void registerDictionaryHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (dict_struct.key)
throw Exception {"'key' is not supported for dictionary of layout 'hashed'", ErrorCodes::UNSUPPORTED_METHOD};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception {name
+ ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
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,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>

View File

@ -8,7 +8,7 @@
#include <Common/PODArray.h> #include <Common/PODArray.h>
#include <memory> #include <memory>
#include <chrono> #include <chrono>
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
namespace DB namespace DB
{ {

View File

@ -1,12 +1,15 @@
#include <DataStreams/OneBlockInputStream.h> #include <DataStreams/OneBlockInputStream.h>
#include <Dictionaries/LibraryDictionarySource.h> #include "LibraryDictionarySource.h"
#include <Dictionaries/LibraryDictionarySourceExternal.h> #include "LibraryDictionarySourceExternal.h"
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Poco/File.h> #include <Poco/File.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <ext/bit_cast.h> #include <ext/bit_cast.h>
#include <ext/range.h> #include <ext/range.h>
#include <ext/scope_guard.h> #include <ext/scope_guard.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB namespace DB
{ {
@ -269,4 +272,17 @@ std::string LibraryDictionarySource::toString() const
{ {
return path; return path;
} }
void registerDictionarySourceLibrary(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & context) -> DictionarySourcePtr {
return std::make_unique<LibraryDictionarySource>(dict_struct, config, config_prefix + ".library", sample_block, context);
};
factory.registerSource("library", createTableSource);
}
} }

View File

@ -1,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/ExternalResultDescription.h> #include "ExternalResultDescription.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Common/SharedLibrary.h> #include <Common/SharedLibrary.h>
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
@ -78,4 +78,5 @@ private:
std::shared_ptr<CStringsHolder> settings; std::shared_ptr<CStringsHolder> settings;
void * lib_data = nullptr; void * lib_data = nullptr;
}; };
} }

View File

@ -1,4 +1,4 @@
#include <Dictionaries/LibraryDictionarySourceExternal.h> #include "LibraryDictionarySourceExternal.h"
#include <common/logger_useful.h> #include <common/logger_useful.h>
namespace namespace

View File

@ -10,8 +10,8 @@
#include <Poco/MongoDB/Element.h> #include <Poco/MongoDB/Element.h>
#include <Poco/MongoDB/ObjectId.h> #include <Poco/MongoDB/ObjectId.h>
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/MongoDBBlockInputStream.h> #include "MongoDBBlockInputStream.h"
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>

View File

@ -2,7 +2,7 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Dictionaries/ExternalResultDescription.h> #include "ExternalResultDescription.h"
namespace Poco namespace Poco

View File

@ -1,21 +1,54 @@
#include <Common/config.h> #include "DictionarySourceFactory.h"
#if USE_POCO_MONGODB #include "DictionaryStructure.h"
#include <Poco/Util/AbstractConfiguration.h> #include "MongoDBDictionarySource.h"
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & /* context */) -> DictionarySourcePtr {
#if USE_POCO_MONGODB
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb", sample_block);
#else
(void)dict_struct;
(void)config;
(void)config_prefix;
(void)sample_block;
throw Exception {"Dictionary source of type `mongodb` is disabled because poco library was built without mongodb support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("mongodb", createTableSource);
}
}
#if USE_POCO_MONGODB
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/MongoDB/Connection.h> #include <Poco/MongoDB/Connection.h>
#include <Poco/MongoDB/Database.h> #include <Poco/MongoDB/Database.h>
#include <Poco/MongoDB/Cursor.h> #include <Poco/MongoDB/Cursor.h>
#include <Poco/MongoDB/Array.h> #include <Poco/MongoDB/Array.h>
#include <Poco/MongoDB/ObjectId.h> #include <Poco/MongoDB/ObjectId.h>
#include <Poco/Version.h> #include <Poco/Version.h>
// only after poco // only after poco
// naming conflict: // naming conflict:
// Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); // Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value);
// dbms/src/IO/WriteHelpers.h:146 #define writeCString(s, buf) // dbms/src/IO/WriteHelpers.h:146 #define writeCString(s, buf)
#include <Dictionaries/MongoDBDictionarySource.h> #include "MongoDBBlockInputStream.h"
#include <Dictionaries/MongoDBBlockInputStream.h>
#include <Common/FieldVisitors.h> #include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <ext/enumerate.h> #include <ext/enumerate.h>

View File

@ -1,8 +1,10 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include <Common/config.h>
#include <Dictionaries/DictionaryStructure.h> #if USE_POCO_MONGODB
#include "IDictionarySource.h"
#include "DictionaryStructure.h"
namespace Poco namespace Poco
{ {
@ -79,3 +81,10 @@ private:
}; };
} }
#endif
/*namespace DB
{
class DictionarySourceFactory;
void registerDictionarySourceMongoDB(DictionarySourceFactory & factory);
}*/

View File

@ -1,7 +1,7 @@
#include <Common/config.h> #include <Common/config.h>
#if USE_MYSQL #if USE_MYSQL
#include <Dictionaries/MySQLBlockInputStream.h> #include "MySQLBlockInputStream.h"
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>

View File

@ -2,7 +2,7 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Dictionaries/ExternalResultDescription.h> #include "ExternalResultDescription.h"
#include <mysqlxx/Query.h> #include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h> #include <mysqlxx/PoolWithFailover.h>
#include <string> #include <string>

View File

@ -1,18 +1,51 @@
#include <Common/config.h> #include "MySQLDictionarySource.h"
#if USE_MYSQL
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/config.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
void registerDictionarySourceMysql(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
const Context & /* context */) -> DictionarySourcePtr {
#if USE_MYSQL
return std::make_unique<MySQLDictionarySource>(dict_struct, config, config_prefix + ".mysql", sample_block);
#else
(void)dict_struct;
(void)config;
(void)config_prefix;
(void)sample_block;
throw Exception {"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("mysql", createTableSource);
}
}
#if USE_MYSQL
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
#include "MySQLBlockInputStream.h"
#include <Dictionaries/MySQLDictionarySource.h> #include "readInvalidateQuery.h"
#include <Dictionaries/MySQLBlockInputStream.h>
#include <Dictionaries/readInvalidateQuery.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>

View File

@ -1,8 +1,11 @@
#pragma once #pragma once
#include <Dictionaries/IDictionarySource.h> #include <Common/config.h>
#include <Dictionaries/ExternalQueryBuilder.h> #if USE_MYSQL
#include <Dictionaries/DictionaryStructure.h>
#include "IDictionarySource.h"
#include "ExternalQueryBuilder.h"
#include "DictionaryStructure.h"
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
#include <mysqlxx/PoolWithFailover.h> #include <mysqlxx/PoolWithFailover.h>
@ -81,3 +84,5 @@ private:
}; };
} }
#endif

View File

@ -1,12 +1,9 @@
#include <Dictionaries/ODBCBlockInputStream.h> #include "ODBCBlockInputStream.h"
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <ext/range.h> #include <ext/range.h>
#include <vector> #include <vector>

View File

@ -2,12 +2,10 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Dictionaries/ExternalResultDescription.h> #include "ExternalResultDescription.h"
#include <Poco/Data/Session.h> #include <Poco/Data/Session.h>
#include <Poco/Data/Statement.h> #include <Poco/Data/Statement.h>
#include <Poco/Data/RecordSet.h> #include <Poco/Data/RecordSet.h>
#include <string> #include <string>

View File

@ -5,10 +5,10 @@
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <Dictionaries/DictionaryBlockInputStreamBase.h> #include "DictionaryBlockInputStreamBase.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/RangeHashedDictionary.h> #include "RangeHashedDictionary.h"
#include <ext/range.h> #include <ext/range.h>
namespace DB namespace DB

View File

@ -1,10 +1,10 @@
#include <Dictionaries/RangeHashedDictionary.h> #include "RangeHashedDictionary.h"
#include <Dictionaries/RangeDictionaryBlockInputStream.h> #include "RangeDictionaryBlockInputStream.h"
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Common/TypeList.h> #include <Common/TypeList.h>
#include <ext/range.h> #include <ext/range.h>
#include "DictionaryFactory.h"
namespace namespace
{ {
@ -50,6 +50,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int DICTIONARY_IS_EMPTY; extern const int DICTIONARY_IS_EMPTY;
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
extern const int UNSUPPORTED_METHOD;
} }
bool RangeHashedDictionary::Range::isCorrectDate(const RangeStorageType & date) bool RangeHashedDictionary::Range::isCorrectDate(const RangeStorageType & date)
@ -559,4 +560,28 @@ BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & col
return callable.stream; return callable.stream;
} }
void registerDictionaryRangeHashed(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (dict_struct.key)
throw Exception {"'key' is not supported for dictionary of layout 'range_hashed'", ErrorCodes::UNSUPPORTED_METHOD};
if (!dict_struct.range_min || !dict_struct.range_max)
throw Exception {name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
ErrorCodes::BAD_ARGUMENTS};
const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("range_hashed", create_layout);
}
} }

View File

@ -1,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>

View File

@ -3,16 +3,17 @@
#include <ext/range.h> #include <ext/range.h>
#include <Poco/Net/IPAddress.h> #include <Poco/Net/IPAddress.h>
#include <Poco/ByteOrder.h> #include <Poco/ByteOrder.h>
#include <Dictionaries/TrieDictionary.h> #include "TrieDictionary.h"
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Columns/ColumnFixedString.h> #include <Columns/ColumnFixedString.h>
#include <Dictionaries/DictionaryBlockInputStream.h> #include "DictionaryBlockInputStream.h"
#include <DataTypes/DataTypeFixedString.h> #include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <IO/WriteIntText.h> #include <IO/WriteIntText.h>
#include <Common/formatIPv6.h> #include <Common/formatIPv6.h>
#include <iostream> #include <iostream>
#include <btrie.h> #include <btrie.h>
#include "DictionaryFactory.h"
namespace DB namespace DB
@ -665,4 +666,25 @@ BlockInputStreamPtr TrieDictionary::getBlockInputStream(const Names & column_nam
std::move(getKeys), std::move(getView)); std::move(getKeys), std::move(getView));
} }
void registerDictionaryTrie(DictionaryFactory & factory)
{
auto create_layout = [=](
const std::string & name,
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
DictionarySourcePtr source_ptr
) -> DictionaryPtr {
if (!dict_struct.key)
throw Exception {"'key' is required for dictionary of layout 'ip_trie'", ErrorCodes::BAD_ARGUMENTS};
const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
// This is specialised trie for storing IPv4 and IPv6 prefixes.
return std::make_unique<TrieDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("ip_trie", create_layout);
}
} }

View File

@ -1,8 +1,8 @@
#pragma once #pragma once
#include <Dictionaries/IDictionary.h> #include "IDictionary.h"
#include <Dictionaries/IDictionarySource.h> #include "IDictionarySource.h"
#include <Dictionaries/DictionaryStructure.h> #include "DictionaryStructure.h"
#include <common/StringRef.h> #include <common/StringRef.h>
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
#include <Columns/ColumnDecimal.h> #include <Columns/ColumnDecimal.h>
@ -242,5 +242,4 @@ private:
Logger * logger; Logger * logger;
}; };
} }

View File

@ -1,4 +1,5 @@
#include <Dictionaries/XDBCDictionarySource.h> #include "XDBCDictionarySource.h"
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <common/LocalDateTime.h> #include <common/LocalDateTime.h>
#include <Poco/Ext/SessionPoolHelpers.h> #include <Poco/Ext/SessionPoolHelpers.h>
@ -7,17 +8,28 @@
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Dictionaries/readInvalidateQuery.h> #include "readInvalidateQuery.h"
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Common/XDBCBridgeHelper.h> #include <Common/XDBCBridgeHelper.h>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include <Common/config.h>
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
#include <Poco/Data/ODBC/Connector.h>
#endif
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
namespace namespace
{ {
class XDBCBridgeBlockInputStream : public IProfilingBlockInputStream class XDBCBridgeBlockInputStream : public IProfilingBlockInputStream
@ -219,4 +231,42 @@ BlockInputStreamPtr XDBCDictionarySource::loadBase(const std::string & query) co
timeouts, bridge_helper->getName() + "BlockInputStream"); timeouts, bridge_helper->getName() + "BlockInputStream");
} }
void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
{
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
Poco::Data::ODBC::Connector::registerConnector();
#endif
auto createTableSource = [=](const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
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"));
return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge);
#else
throw Exception {"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("odbc", createTableSource);
}
void registerDictionarySourceJDBC(DictionarySourceFactory & factory)
{
auto createTableSource = [=](const DictionaryStructure & /* dict_struct */,
const Poco::Util::AbstractConfiguration & /* config */,
const std::string & /* config_prefix */,
Block & /* sample_block */,
const Context & /* context */) -> DictionarySourcePtr {
throw Exception {"Dictionary source of type `jdbc` is disabled until consistent support for nullable fields.",
ErrorCodes::SUPPORT_IS_DISABLED};
// BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<JDBCBridgeMixin>>(config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".connection_string"));
// return std::make_unique<XDBCDictionarySource>(dict_struct, config, config_prefix + ".jdbc", sample_block, context, bridge);
};
factory.registerSource("jdbc", createTableSource);
}
} }

View File

@ -2,11 +2,9 @@
#include <Poco/Data/SessionPool.h> #include <Poco/Data/SessionPool.h>
#include <Poco/URI.h> #include <Poco/URI.h>
#include "DictionaryStructure.h"
#include <Dictionaries/DictionaryStructure.h> #include "ExternalQueryBuilder.h"
#include <Dictionaries/ExternalQueryBuilder.h> #include "IDictionarySource.h"
#include <Dictionaries/IDictionarySource.h>
#include <IO/ConnectionTimeouts.h> #include <IO/ConnectionTimeouts.h>
#include <Common/XDBCBridgeHelper.h> #include <Common/XDBCBridgeHelper.h>
@ -83,4 +81,5 @@ private:
ConnectionTimeouts timeouts; ConnectionTimeouts timeouts;
const Context & global_context; const Context & global_context;
}; };
} }

View File

@ -1,4 +1,4 @@
#include <Dictionaries/readInvalidateQuery.h> #include "readInvalidateQuery.h"
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
namespace DB namespace DB

View File

@ -0,0 +1,52 @@
#include "DictionaryFactory.h"
#include "DictionarySourceFactory.h"
namespace DB
{
void registerDictionarySourceFile(DictionarySourceFactory & source_factory);
void registerDictionarySourceMysql(DictionarySourceFactory & source_factory);
void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory);
void registerDictionarySourceMongoDB(DictionarySourceFactory & source_factory);
void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory);
void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory);
void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory);
void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory);
void registerDictionarySourceLibrary(DictionarySourceFactory & source_factory);
void registerDictionaryRangeHashed(DictionaryFactory & factory);
void registerDictionaryComplexKeyHashed(DictionaryFactory & factory);
void registerDictionaryComplexKeyCache(DictionaryFactory & factory);
void registerDictionaryTrie(DictionaryFactory & factory);
void registerDictionaryFlat(DictionaryFactory & factory);
void registerDictionaryHashed(DictionaryFactory & factory);
void registerDictionaryCache(DictionaryFactory & factory);
void registerDictionaries()
{
{
auto & source_factory = DictionarySourceFactory::instance();
registerDictionarySourceFile(source_factory);
registerDictionarySourceMysql(source_factory);
registerDictionarySourceClickHouse(source_factory);
registerDictionarySourceMongoDB(source_factory);
registerDictionarySourceXDBC(source_factory);
registerDictionarySourceJDBC(source_factory);
registerDictionarySourceExecutable(source_factory);
registerDictionarySourceHTTP(source_factory);
registerDictionarySourceLibrary(source_factory);
}
{
auto & factory = DictionaryFactory::instance();
registerDictionaryRangeHashed(factory);
registerDictionaryComplexKeyHashed(factory);
registerDictionaryComplexKeyCache(factory);
registerDictionaryTrie(factory);
registerDictionaryFlat(factory);
registerDictionaryHashed(factory);
registerDictionaryCache(factory);
}
}
}

View File

@ -0,0 +1,6 @@
#pragma once
namespace DB
{
void registerDictionaries();
}

View File

@ -1,4 +1,4 @@
#include <Dictionaries/writeParenthesisedString.h> #include "writeParenthesisedString.h"
namespace DB namespace DB
{ {

View File

@ -10,7 +10,18 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func
add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources}) add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources})
target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) target_link_libraries(clickhouse_functions
PUBLIC
dbms
PRIVATE
clickhouse_dictionaries
${CONSISTENT_HASHING_LIBRARY}
consistent-hashing-sumbur
${FARMHASH_LIBRARIES}
${METROHASH_LIBRARIES}
murmurhash
${BASE64_LIBRARY}
${OPENSSL_CRYPTO_LIBRARY})
target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR})

View File

@ -84,6 +84,8 @@ namespace
/// doesn't work properly without patch /// doesn't work properly without patch
#if POCO_CLICKHOUSE_PATCH #if POCO_CLICKHOUSE_PATCH
session->setKeepAlive(keep_alive); session->setKeepAlive(keep_alive);
#else
(void)keep_alive; // Avoid warning: unused parameter
#endif #endif
return session; return session;

View File

@ -1,4 +1,5 @@
#include <Dictionaries/CatBoostModel.h> #include "CatBoostModel.h"
#include <Common/FieldVisitors.h> #include <Common/FieldVisitors.h>
#include <mutex> #include <mutex>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>

View File

@ -1,133 +0,0 @@
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/DictionarySourceFactory.h>
#include <Dictionaries/FlatDictionary.h>
#include <Dictionaries/HashedDictionary.h>
#include <Dictionaries/CacheDictionary.h>
#include <Dictionaries/RangeHashedDictionary.h>
#include <Dictionaries/ComplexKeyHashedDictionary.h>
#include <Dictionaries/ComplexKeyCacheDictionary.h>
#include <Dictionaries/TrieDictionary.h>
#include <Dictionaries/DictionaryStructure.h>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int UNSUPPORTED_METHOD;
extern const int TOO_SMALL_BUFFER_SIZE;
extern const int BAD_ARGUMENTS;
}
DictionaryPtr DictionaryFactory::create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, Context & context) const
{
Poco::Util::AbstractConfiguration::Keys keys;
const auto & layout_prefix = config_prefix + ".layout";
config.keys(layout_prefix, keys);
if (keys.size() != 1)
throw Exception{name + ": element dictionary.layout should have exactly one child element",
ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG};
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
auto source_ptr = DictionarySourceFactory::instance().create(
name, config, config_prefix + ".source", dict_struct, context);
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
const auto & layout_type = keys.front();
if ("range_hashed" == layout_type)
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout 'range_hashed'",
ErrorCodes::UNSUPPORTED_METHOD};
if (!dict_struct.range_min || !dict_struct.range_max)
throw Exception{name + ": dictionary of layout 'range_hashed' requires .structure.range_min and .structure.range_max",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("complex_key_hashed" == layout_type)
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("complex_key_cache" == layout_type)
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'",
ErrorCodes::BAD_ARGUMENTS};
const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells");
if (size == 0)
throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells",
ErrorCodes::TOO_SMALL_BUFFER_SIZE};
if (require_nonempty)
throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
}
else if ("ip_trie" == layout_type)
{
if (!dict_struct.key)
throw Exception{"'key' is required for dictionary of layout 'ip_trie'",
ErrorCodes::BAD_ARGUMENTS};
// This is specialised trie for storing IPv4 and IPv6 prefixes.
return std::make_unique<TrieDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else
{
if (dict_struct.key)
throw Exception{"'key' is not supported for dictionary of layout '" + layout_type + "'",
ErrorCodes::UNSUPPORTED_METHOD};
if (dict_struct.range_min || dict_struct.range_max)
throw Exception{name + ": elements .structure.range_min and .structure.range_max should be defined only "
"for a dictionary of layout 'range_hashed'",
ErrorCodes::BAD_ARGUMENTS};
if ("flat" == layout_type)
{
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("hashed" == layout_type)
{
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
}
else if ("cache" == layout_type)
{
const auto size = config.getInt(layout_prefix + ".cache.size_in_cells");
if (size == 0)
throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells",
ErrorCodes::TOO_SMALL_BUFFER_SIZE};
if (require_nonempty)
throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set",
ErrorCodes::BAD_ARGUMENTS};
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
}
}
throw Exception{name + ": unknown dictionary layout type: " + layout_type,
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}
}

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <Dictionaries/CatBoostModel.h> #include <Interpreters/CatBoostModel.h>
#include <Interpreters/ExternalLoader.h> #include <Interpreters/ExternalLoader.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <memory> #include <memory>

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