Merge branch 'add-ext-dict-redis' of https://github.com/comunodi/ClickHouse into merging_redis

This commit is contained in:
CurtizJ 2019-09-09 13:51:50 +03:00
commit 3b3657ff1e
18 changed files with 1228 additions and 50 deletions

View File

@ -18,6 +18,9 @@ if (NOT DEFINED ENABLE_POCO_MONGODB OR ENABLE_POCO_MONGODB)
else ()
set(ENABLE_POCO_MONGODB 0 CACHE BOOL "")
endif ()
if (NOT DEFINED ENABLE_POCO_REDIS OR ENABLE_POCO_REDIS)
list (APPEND POCO_COMPONENTS Redis)
endif ()
# TODO: after new poco release with SQL library rename ENABLE_POCO_ODBC -> ENABLE_POCO_SQLODBC
if (NOT DEFINED ENABLE_POCO_ODBC OR ENABLE_POCO_ODBC)
list (APPEND POCO_COMPONENTS DataODBC)
@ -35,7 +38,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (ENABLE_ZIP 0 CACHE BOOL "")
set (ENABLE_PAGECOMPILER 0 CACHE BOOL "")
set (ENABLE_PAGECOMPILER_FILE2PAGE 0 CACHE BOOL "")
set (ENABLE_REDIS 0 CACHE BOOL "")
set (ENABLE_DATA_SQLITE 0 CACHE BOOL "")
set (ENABLE_DATA_MYSQL 0 CACHE BOOL "")
set (ENABLE_DATA_POSTGRESQL 0 CACHE BOOL "")
@ -46,7 +48,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (POCO_ENABLE_ZIP 0 CACHE BOOL "")
set (POCO_ENABLE_PAGECOMPILER 0 CACHE BOOL "")
set (POCO_ENABLE_PAGECOMPILER_FILE2PAGE 0 CACHE BOOL "")
set (POCO_ENABLE_REDIS 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_SQLITE 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_MYSQL 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_POSTGRESQL 0 CACHE BOOL "")
@ -69,6 +70,11 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (Poco_MongoDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/poco/MongoDB/include/")
endif ()
if (NOT DEFINED ENABLE_POCO_REDIS OR ENABLE_POCO_REDIS)
set (Poco_Redis_LIBRARY PocoRedis)
set (Poco_Redis_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/poco/Redis/include/")
endif ()
if (EXISTS "${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/ODBC/include/")
set (Poco_SQL_FOUND 1)
set (Poco_SQL_LIBRARY PocoSQL)
@ -122,6 +128,9 @@ endif ()
if (Poco_MongoDB_LIBRARY)
set (USE_POCO_MONGODB 1)
endif ()
if (Poco_Redis_LIBRARY)
set (USE_POCO_REDIS 1)
endif ()
if (Poco_DataODBC_LIBRARY AND ODBC_FOUND)
set (USE_POCO_DATAODBC 1)
endif ()
@ -129,7 +138,7 @@ if (Poco_SQLODBC_LIBRARY AND ODBC_FOUND)
set (USE_POCO_SQLODBC 1)
endif ()
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Foundation_LIBRARY},${Poco_Util_LIBRARY},${Poco_Net_LIBRARY},${Poco_NetSSL_LIBRARY},${Poco_Crypto_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_SQL_LIBRARY},${Poco_SQLODBC_LIBRARY},${Poco_MongoDB_LIBRARY}; MongoDB=${USE_POCO_MONGODB}, DataODBC=${USE_POCO_DATAODBC}, NetSSL=${USE_POCO_NETSSL}")
message(STATUS "Using Poco: ${Poco_INCLUDE_DIRS} : ${Poco_Foundation_LIBRARY},${Poco_Util_LIBRARY},${Poco_Net_LIBRARY},${Poco_NetSSL_LIBRARY},${Poco_Crypto_LIBRARY},${Poco_XML_LIBRARY},${Poco_Data_LIBRARY},${Poco_DataODBC_LIBRARY},${Poco_SQL_LIBRARY},${Poco_SQLODBC_LIBRARY},${Poco_MongoDB_LIBRARY},${Poco_Redis_LIBRARY}; MongoDB=${USE_POCO_MONGODB}, Redis=${USE_POCO_REDIS}, DataODBC=${USE_POCO_DATAODBC}, NetSSL=${USE_POCO_NETSSL}")
# How to make sutable poco:
# use branch:

View File

@ -3,6 +3,17 @@
// .h autogenerated by cmake!
#cmakedefine01 USE_RE2_ST
<<<<<<< HEAD
=======
#cmakedefine01 USE_VECTORCLASS
#cmakedefine01 USE_RDKAFKA
#cmakedefine01 USE_CAPNP
#cmakedefine01 USE_EMBEDDED_COMPILER
#cmakedefine01 USE_POCO_SQLODBC
#cmakedefine01 USE_POCO_DATAODBC
#cmakedefine01 USE_POCO_MONGODB
#cmakedefine01 USE_POCO_REDIS
>>>>>>> 102967015e8813129384dcd0f6e377e5b730f167
#cmakedefine01 USE_POCO_NETSSL
#cmakedefine01 USE_HDFS
#cmakedefine01 USE_CPUID

View File

@ -39,4 +39,12 @@ if(USE_POCO_MONGODB)
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_MongoDB_LIBRARY})
endif()
if(USE_POCO_REDIS)
# for code highlighting in CLion
# target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${Poco_Redis_INCLUDE_DIR})
# for build
target_link_libraries(clickhouse_dictionaries PRIVATE ${Poco_Redis_LIBRARY})
endif()
add_subdirectory(Embedded)

View File

@ -0,0 +1,279 @@
#include <Common/config.h>
#if USE_POCO_REDIS
# include <string>
# include <vector>
# include <Poco/Redis/Array.h>
# include <Poco/Redis/Client.h>
# include <Poco/Redis/Command.h>
# include <Poco/Redis/Type.h>
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnString.h>
# include <Columns/ColumnsNumber.h>
# include <IO/ReadHelpers.h>
# include <IO/WriteHelpers.h>
# include <ext/range.h>
# include "DictionaryStructure.h"
# include "RedisBlockInputStream.h"
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR;
extern const int LIMIT_EXCEEDED;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
RedisBlockInputStream::RedisBlockInputStream(
const std::shared_ptr<Poco::Redis::Client> & client_,
const Poco::Redis::Array & keys_,
const DB::Block & sample_block,
const size_t max_block_size)
: client(client_), keys(keys_), max_block_size{max_block_size}
{
description.init(sample_block);
}
RedisBlockInputStream::~RedisBlockInputStream() = default;
namespace
{
using ValueType = ExternalResultDescription::ValueType;
using RedisArray = Poco::Redis::Array;
bool isNull(const Poco::Redis::RedisType::Ptr & value)
{
return value->isBulkString() &&
static_cast<const Poco::Redis::Type<Poco::Redis::BulkString> *>(value.get())->value().isNull();
}
std::string getStringOrThrow(const Poco::Redis::RedisType::Ptr & value, const std::string & column_name)
{
switch (value->type())
{
case Poco::Redis::RedisTypeTraits<Poco::Redis::BulkString>::TypeId:
{
const auto & bs = static_cast<const Poco::Redis::Type<Poco::Redis::BulkString> *>(value.get())->value();
if (bs.isNull())
throw Exception{"Type mismatch, expected not null String for column " + column_name,
ErrorCodes::TYPE_MISMATCH};
return bs.value();
}
case Poco::Redis::RedisTypeTraits<std::string>::TypeId:
return static_cast<const Poco::Redis::Type<std::string> *>(value.get())->value();
default:
throw Exception{"Type mismatch, expected std::string, got type id = " + toString(value->type()) + " for column " + column_name,
ErrorCodes::TYPE_MISMATCH};
}
}
template <typename T>
inline void insert(IColumn & column, const String & stringValue)
{
static_cast<ColumnVector<T> &>(column).insertValue(parse<T>(stringValue));
}
void insertValue(IColumn & column, const ValueType type, const Poco::Redis::RedisType::Ptr & value, const std::string & name)
{
String stringValue = getStringOrThrow(value, name);
switch (type)
{
case ValueType::UInt8:
insert<UInt8>(column, stringValue);
break;
case ValueType::UInt16:
insert<UInt16>(column, stringValue);
break;
case ValueType::UInt32:
insert<UInt32>(column, stringValue);
break;
case ValueType::UInt64:
insert<UInt64>(column, stringValue);
break;
case ValueType::Int8:
insert<Int8>(column, stringValue);
break;
case ValueType::Int16:
insert<Int16>(column, stringValue);
break;
case ValueType::Int32:
insert<Int32>(column, stringValue);
break;
case ValueType::Int64:
insert<Int64>(column, stringValue);
break;
case ValueType::Float32:
insert<Float32>(column, stringValue);
break;
case ValueType::Float64:
insert<Float64>(column, stringValue);
break;
case ValueType::String:
static_cast<ColumnString &>(column).insert(parse<String>(stringValue));
break;
case ValueType::Date:
static_cast<ColumnUInt16 &>(column).insertValue(parse<LocalDate>(stringValue).getDayNum());
break;
case ValueType::DateTime:
static_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(parse<LocalDateTime>(stringValue)));
break;
case ValueType::UUID:
static_cast<ColumnUInt128 &>(column).insertValue(parse<UUID>(stringValue));
break;
}
}
void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); }
}
Block RedisBlockInputStream::readImpl()
{
if (description.sample_block.rows() == 0 || keys.size() == 0)
all_read = true;
if (all_read)
return {};
const size_t size = description.sample_block.columns();
MutableColumns columns(size);
for (const auto i : ext::range(0, size))
columns[i] = description.sample_block.getByPosition(i).column->cloneEmpty();
const auto insertValueByIdx = [this, &columns](size_t idx, const auto & value)
{
const auto & name = description.sample_block.getByPosition(idx).name;
if (description.types[idx].second)
{
ColumnNullable & column_nullable = static_cast<ColumnNullable &>(*columns[idx]);
insertValue(column_nullable.getNestedColumn(), description.types[idx].first, value, name);
column_nullable.getNullMapData().emplace_back(0);
}
else
insertValue(*columns[idx], description.types[idx].first, value, name);
};
if (keys.begin()->get()->isArray())
{
size_t num_rows = 0;
while (num_rows < max_block_size && !all_read)
{
if (cursor >= keys.size())
{
all_read = true;
break;
}
const auto & primary_with_secondary = *(keys.begin() + cursor);
const auto & keys_array =
static_cast<const Poco::Redis::Type<Poco::Redis::Array> *>(primary_with_secondary.get())->value();
if (keys_array.size() < 2)
{
throw Exception{"Too low keys in request to source: " + DB::toString(keys_array.size())
+ ", expected 2 or more",
ErrorCodes::LOGICAL_ERROR};
}
if (num_rows + keys_array.size() - 1 > max_block_size)
{
if (num_rows == 0)
throw Exception{"Too many (" + DB::toString(keys_array.size()) + ") key attributes",
ErrorCodes::LIMIT_EXCEEDED};
break;
}
Poco::Redis::Command commandForValues("HMGET");
for (size_t i = 0; i < keys_array.size(); ++i)
{
const auto & secondary_key = *(keys_array.begin() + i);
commandForValues.addRedisType(secondary_key);
}
++cursor;
Poco::Redis::Array values = client->execute<Poco::Redis::Array>(commandForValues);
if (keys_array.size() != values.size() + 1) // 'HMGET' primary_key secondary_keys
throw Exception{"Inconsistent sizes of keys and values in Redis request",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
const auto & primary_key = *keys_array.begin();
for (size_t i = 0; i < values.size(); ++i)
{
const auto & secondary_key = *(keys_array.begin() + i + 1);
const auto & value = *(values.begin() + i);
if (value.isNull())
{
insertValueByIdx(0, primary_key);
insertValueByIdx(1, secondary_key);
insertDefaultValue(*columns[2], *description.sample_block.getByPosition(2).column);
++num_rows;
}
else if (!isNull(value)) // null string means 'no value for requested key'
{
insertValueByIdx(0, primary_key);
insertValueByIdx(1, secondary_key);
insertValueByIdx(2, value);
++num_rows;
}
}
}
}
else
{
size_t num_rows = 0;
while (num_rows < max_block_size && !all_read)
{
Poco::Redis::Command commandForValues("MGET");
// keys.size() > 0
for (size_t i = 0; i < max_block_size && cursor < keys.size(); ++i)
{
const auto & key = *(keys.begin() + cursor);
commandForValues.addRedisType(key);
++cursor;
}
if (commandForValues.size() == 1) // only 'MGET'
{
all_read = true;
break;
}
Poco::Redis::Array values = client->execute<Poco::Redis::Array>(commandForValues);
if (commandForValues.size() != values.size() + 1) // 'MGET' keys
throw Exception{"Inconsistent sizes of keys and values in Redis request",
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
for (size_t i = 0; i < values.size(); ++i)
{
const auto & key = *(keys.begin() + cursor - i - 1);
const auto & value = *(values.begin() + values.size() - i - 1);
if (value.isNull())
{
insertValueByIdx(0, key);
insertDefaultValue(*columns[1], *description.sample_block.getByPosition(1).column);
++num_rows;
}
else if (!isNull(value)) // null string means 'no value for requested key'
{
insertValueByIdx(0, key);
insertValueByIdx(1, value);
++num_rows;
}
}
}
}
return description.sample_block.cloneWithColumns(std::move(columns));
}
}
#endif

View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/Block.h>
#include <Core/ExternalResultDescription.h>
#include <DataStreams/IBlockInputStream.h>
namespace Poco
{
namespace Redis
{
class Array;
class Client;
}
}
namespace DB
{
class RedisBlockInputStream final : public IBlockInputStream
{
public:
RedisBlockInputStream(
const std::shared_ptr<Poco::Redis::Client> & client_,
const Poco::Redis::Array & keys_,
const Block & sample_block,
const size_t max_block_size);
~RedisBlockInputStream() override;
String getName() const override { return "Redis"; }
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
private:
Block readImpl() override;
std::shared_ptr<Poco::Redis::Client> client;
Poco::Redis::Array keys;
const size_t max_block_size;
ExternalResultDescription description;
size_t cursor = 0;
bool all_read = false;
};
}

View File

@ -0,0 +1,197 @@
#include "RedisDictionarySource.h"
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
void registerDictionarySourceRedis(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_REDIS
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
#else
(void)dict_struct;
(void)config;
(void)config_prefix;
(void)sample_block;
throw Exception{"Dictionary source of type `redis` is disabled because poco library was built without redis support.",
ErrorCodes::SUPPORT_IS_DISABLED};
#endif
};
factory.registerSource("redis", createTableSource);
}
}
#if USE_POCO_REDIS
# include <Poco/Redis/Array.h>
# include <Poco/Redis/Client.h>
# include <Poco/Redis/Command.h>
# include <Poco/Redis/Type.h>
# include <Poco/Util/AbstractConfiguration.h>
# include <Common/FieldVisitors.h>
# include <IO/WriteHelpers.h>
# include "RedisBlockInputStream.h"
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int CANNOT_SELECT;
extern const int INVALID_CONFIG_PARAMETER;
}
static const size_t max_block_size = 8192;
RedisDictionarySource::RedisDictionarySource(
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
UInt8 db_index,
RedisStorageType::Id storage_type,
const Block & sample_block)
: dict_struct{dict_struct}
, host{host}
, port{port}
, db_index{db_index}
, storage_type{storage_type}
, sample_block{sample_block}
, client{std::make_shared<Poco::Redis::Client>(host, port)}
{
if (dict_struct.attributes.size() != 1)
throw Exception{"Invalid number of non key columns for Redis source: " +
DB::toString(dict_struct.attributes.size()) + ", expected 1",
ErrorCodes::INVALID_CONFIG_PARAMETER};
if (storage_type == RedisStorageType::HASH_MAP)
{
if (!dict_struct.key.has_value())
throw Exception{"Redis source with storage type \'hash_map\' must have key",
ErrorCodes::INVALID_CONFIG_PARAMETER};
if (dict_struct.key.value().size() > 2)
throw Exception{"Redis source with complex keys having more than 2 attributes are unsupported",
ErrorCodes::INVALID_CONFIG_PARAMETER};
// suppose key[0] is primary key, key[1] is secondary key
}
if (db_index != 0)
{
Poco::Redis::Command command("SELECT");
command << static_cast<Int64>(db_index);
std::string reply = client->execute<std::string>(command);
if (reply != "+OK\r\n")
throw Exception{"Selecting db with index " + DB::toString(db_index) + " failed with reason " + reply,
ErrorCodes::CANNOT_SELECT};
}
}
RedisDictionarySource::RedisDictionarySource(
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block)
: RedisDictionarySource(
dict_struct,
config.getString(config_prefix + ".host"),
config.getUInt(config_prefix + ".port"),
config.getUInt(config_prefix + ".db_index", 0),
parseStorageType(config.getString(config_prefix + ".storage_type", "")),
sample_block)
{
}
RedisDictionarySource::RedisDictionarySource(const RedisDictionarySource & other)
: RedisDictionarySource{other.dict_struct,
other.host,
other.port,
other.db_index,
other.storage_type,
other.sample_block}
{
}
RedisDictionarySource::~RedisDictionarySource() = default;
BlockInputStreamPtr RedisDictionarySource::loadAll()
{
Poco::Redis::Command command_for_keys("KEYS");
command_for_keys << "*";
Poco::Redis::Array keys = client->execute<Poco::Redis::Array>(command_for_keys);
if (storage_type == RedisStorageType::HASH_MAP && dict_struct.key->size() == 2)
{
Poco::Redis::Array hkeys;
for (const auto & key : keys)
{
Poco::Redis::Command command_for_secondary_keys("HKEYS");
command_for_secondary_keys.addRedisType(key);
Poco::Redis::Array reply_for_primary_key = client->execute<Poco::Redis::Array>(command_for_secondary_keys);
Poco::Redis::Array primary_with_secondary;
primary_with_secondary.addRedisType(key);
for (const auto & secondary_key : reply_for_primary_key)
primary_with_secondary.addRedisType(secondary_key);
hkeys.add(primary_with_secondary);
}
keys = hkeys;
}
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
}
BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
if (storage_type != RedisStorageType::SIMPLE)
throw Exception{"Cannot use loadIds with \'simple\' storage type", ErrorCodes::UNSUPPORTED_METHOD};
if (!dict_struct.id)
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
Poco::Redis::Array keys;
for (UInt64 id : ids)
keys << DB::toString(id);
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
}
std::string RedisDictionarySource::toString() const
{
return "Redis: " + host + ':' + DB::toString(port);
}
RedisStorageType::Id RedisDictionarySource::parseStorageType(const std::string & storage_type)
{
RedisStorageType::Id storage_type_id = RedisStorageType::valueOf(storage_type);
if (storage_type_id == RedisStorageType::UNKNOWN)
storage_type_id = RedisStorageType::SIMPLE;
return storage_type_id;
}
}
#endif

View File

@ -0,0 +1,107 @@
#pragma once
#include <Common/config.h>
#include <Core/Block.h>
#if USE_POCO_REDIS
# include "DictionaryStructure.h"
# include "IDictionarySource.h"
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
namespace Redis
{
class Client;
}
}
namespace DB
{
namespace RedisStorageType
{
enum Id
{
SIMPLE,
HASH_MAP,
UNKNOWN
};
Id valueOf(const std::string & value)
{
if (value == "simple")
return SIMPLE;
if (value == "hash_map")
return HASH_MAP;
return UNKNOWN;
}
}
class RedisDictionarySource final : public IDictionarySource
{
RedisDictionarySource(
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
UInt8 db_index,
RedisStorageType::Id storage_type,
const Block & sample_block);
public:
RedisDictionarySource(
const DictionaryStructure & dict_struct,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block);
RedisDictionarySource(const RedisDictionarySource & other);
~RedisDictionarySource() override;
BlockInputStreamPtr loadAll() override;
BlockInputStreamPtr loadUpdatedAll() override
{
throw Exception{"Method loadUpdatedAll is unsupported for RedisDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
}
bool supportsSelectiveLoad() const override { return true; }
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & /* key_columns */, const std::vector<size_t> & /* requested_rows */) override
{
// Redis does not support native indexing
throw Exception{"Method loadKeys is unsupported for RedisDictionarySource", ErrorCodes::NOT_IMPLEMENTED};
}
bool isModified() const override { return true; }
bool hasUpdateField() const override { return false; }
DictionarySourcePtr clone() const override { return std::make_unique<RedisDictionarySource>(*this); }
std::string toString() const override;
private:
static RedisStorageType::Id parseStorageType(const std::string& storage_type);
private:
const DictionaryStructure dict_struct;
const std::string host;
const UInt16 port;
const UInt8 db_index;
const RedisStorageType::Id storage_type;
Block sample_block;
std::shared_ptr<Poco::Redis::Client> client;
};
}
#endif

View File

@ -7,6 +7,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & source_factory);
void registerDictionarySourceMysql(DictionarySourceFactory & source_factory);
void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory);
void registerDictionarySourceMongoDB(DictionarySourceFactory & source_factory);
void registerDictionarySourceRedis(DictionarySourceFactory & source_factory);
void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory);
void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory);
void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory);
@ -30,6 +31,7 @@ void registerDictionaries()
registerDictionarySourceMysql(source_factory);
registerDictionarySourceClickHouse(source_factory);
registerDictionarySourceMongoDB(source_factory);
registerDictionarySourceRedis(source_factory);
registerDictionarySourceXDBC(source_factory);
registerDictionarySourceJDBC(source_factory);
registerDictionarySourceExecutable(source_factory);

View File

@ -103,6 +103,7 @@ class ClickHouseCluster:
self.with_hdfs = False
self.with_mongo = False
self.with_net_trics = False
self.with_redis = False
self.docker_client = None
self.is_up = False
@ -114,7 +115,7 @@ class ClickHouseCluster:
cmd += " client"
return cmd
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False):
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None):
"""Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -132,7 +133,7 @@ class ClickHouseCluster:
instance = ClickHouseInstance(
self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper,
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, self.base_configs_dir, self.server_bin_path,
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, self.base_configs_dir, self.server_bin_path,
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary)
@ -208,6 +209,13 @@ class ClickHouseCluster:
for cmd in cmds:
cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_net.yml')])
if with_redis and not self.with_redis:
self.with_redis = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')])
self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]
return instance
@ -367,6 +375,11 @@ class ClickHouseCluster:
subprocess_check_call(self.base_mongo_cmd + common_opts)
self.wait_mongo_to_start(30)
if self.with_redis and self.base_redis_cmd:
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10)
subprocess_check_call(self.base_cmd + ['up', '-d', '--no-recreate'])
start_deadline = time.time() + 20.0 # seconds
@ -454,7 +467,7 @@ class ClickHouseInstance:
def __init__(
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, base_configs_dir, server_bin_path, odbc_bridge_bin_path,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, base_configs_dir, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False):
@ -479,6 +492,7 @@ class ClickHouseInstance:
self.with_mysql = with_mysql
self.with_kafka = with_kafka
self.with_mongo = with_mongo
self.with_redis = with_redis
self.path = p.join(self.cluster.instances_dir, name)
self.docker_compose_path = p.join(self.path, 'docker_compose.yml')

View File

@ -0,0 +1,7 @@
version: '2.2'
services:
redis1:
image: redis
restart: always
ports:
- 6380:6379

View File

@ -1,4 +1,4 @@
[pytest]
python_files = test.py
python_files = test*.py
norecursedirs = _instances
timeout = 600

View File

@ -1,4 +1,4 @@
#-*- coding: utf-8 -*-
# -*- coding: utf-8 -*-
import copy
@ -8,8 +8,10 @@ class Layout(object):
'hashed': '<hashed/>',
'cache': '<cache><size_in_cells>128</size_in_cells></cache>',
'complex_key_hashed': '<complex_key_hashed/>',
'complex_key_hashed_one_key': '<complex_key_hashed/>',
'complex_key_hashed_two_keys': '<complex_key_hashed/>',
'complex_key_cache': '<complex_key_cache><size_in_cells>128</size_in_cells></complex_key_cache>',
'range_hashed': '<range_hashed/>'
'range_hashed': '<range_hashed/>',
}
def __init__(self, name):
@ -18,13 +20,13 @@ class Layout(object):
self.is_simple = False
self.is_ranged = False
if self.name.startswith('complex'):
self.layout_type = "complex"
self.layout_type = 'complex'
self.is_complex = True
elif name.startswith("range"):
self.layout_type = "ranged"
elif name.startswith('range'):
self.layout_type = 'ranged'
self.is_ranged = True
else:
self.layout_type = "simple"
self.layout_type = 'simple'
self.is_simple = True
def get_str(self):
@ -33,8 +35,7 @@ class Layout(object):
def get_key_block_name(self):
if self.is_complex:
return 'key'
else:
return 'id'
return 'id'
class Row(object):
@ -46,6 +47,9 @@ class Row(object):
def get_value_by_name(self, name):
return self.data[name]
def set_value(self, name, value):
self.data[name] = value
class Field(object):
def __init__(self, name, field_type, is_key=False, is_range_key=False, default=None, hierarchical=False, range_hash_type=None, default_value_for_get=None):
@ -93,6 +97,7 @@ class DictionaryStructure(object):
self.range_key = None
self.ordinary_fields = []
self.range_fields = []
for field in fields:
if field.is_key:
self.keys.append(field)
@ -116,11 +121,12 @@ class DictionaryStructure(object):
fields_strs = []
for field in self.ordinary_fields:
fields_strs.append(field.get_attribute_str())
key_strs = []
if self.layout.is_complex:
for key_field in self.keys:
key_strs.append(key_field.get_attribute_str())
else: # same for simple and ranged
else: # same for simple and ranged
for key_field in self.keys:
key_strs.append(key_field.get_simple_index_str())
@ -179,7 +185,7 @@ class DictionaryStructure(object):
if isinstance(val, str):
val = "'" + val + "'"
key_exprs_strs.append('to{type}({value})'.format(type=key.field_type, value=val))
key_expr = ', (' + ','.join(key_exprs_strs) + ')'
key_expr = ', tuple(' + ','.join(key_exprs_strs) + ')'
date_expr = ''
if self.layout.is_ranged:
@ -280,12 +286,14 @@ class DictionaryStructure(object):
class Dictionary(object):
def __init__(self, name, structure, source, config_path, table_name):
def __init__(self, name, structure, source, config_path, table_name, fields=None, values=None):
self.name = name
self.structure = copy.deepcopy(structure)
self.source = copy.deepcopy(source)
self.config_path = config_path
self.table_name = table_name
self.fields = fields
self.values = values
def generate_config(self):
with open(self.config_path, 'w') as result:

View File

@ -2,6 +2,8 @@
import warnings
import pymysql.cursors
import pymongo
import redis
import aerospike
from tzlocal import get_localzone
import datetime
import os
@ -372,3 +374,116 @@ class SourceHTTP(SourceHTTPBase):
class SourceHTTPS(SourceHTTPBase):
def _get_schema(self):
return "https"
class SourceRedis(ExternalSource):
def __init__(
self, name, internal_hostname, internal_port, docker_hostname, docker_port, user, password, storage_type
):
super(SourceRedis, self).__init__(
name, internal_hostname, internal_port, docker_hostname, docker_port, user, password
)
self.storage_type = storage_type
def get_source_str(self, table_name):
return '''
<redis>
<host>{host}</host>
<port>{port}</port>
<db_index>0</db_index>
<storage_type>{storage_type}</storage_type>
</redis>
'''.format(
host=self.docker_hostname,
port=self.docker_port,
storage_type=self.storage_type, # simple or hash_map
)
def prepare(self, structure, table_name, cluster):
self.client = redis.StrictRedis(host=self.internal_hostname, port=self.internal_port)
self.prepared = True
def load_kv_data(self, values):
self.client.flushdb()
if len(values[0]) == 2:
self.client.mset({value[0]: value[1] for value in values})
else:
for value in values:
self.client.hset(value[0], value[1], value[2])
def compatible_with_layout(self, layout):
if (
layout.is_simple and self.storage_type == "simple" or
layout.is_complex and self.storage_type == "simple" and layout.name == "complex_key_hashed_one_key" or
layout.is_complex and self.storage_type == "hash_map" and layout.name == "complex_key_hashed_two_keys"
):
return True
return False
class SourceAerospike(ExternalSource):
def __init__(self, name, internal_hostname, internal_port,
docker_hostname, docker_port, user, password):
ExternalSource.__init__(self, name, internal_hostname, internal_port,
docker_hostname, docker_port, user, password)
self.namespace = "test"
self.set = "test_set"
def get_source_str(self, table_name):
print("AEROSPIKE get source str")
return '''
<aerospike>
<host>{host}</host>
<port>{port}</port>
</aerospike>
'''.format(
host=self.docker_hostname,
port=self.docker_port,
)
def prepare(self, structure, table_name, cluster):
config = {
'hosts': [ (self.internal_hostname, self.internal_port) ]
}
self.client = aerospike.client(config).connect()
self.prepared = True
print("PREPARED AEROSPIKE")
print(config)
def compatible_with_layout(self, layout):
print("compatible AEROSPIKE")
return layout.is_simple
def _flush_aerospike_db(self):
keys = []
def handle_record((key, metadata, record)):
print("Handle record {} {}".format(key, record))
keys.append(key)
def print_record((key, metadata, record)):
print("Print record {} {}".format(key, record))
scan = self.client.scan(self.namespace, self.set)
scan.foreach(handle_record)
[self.client.remove(key) for key in keys]
def load_kv_data(self, values):
self._flush_aerospike_db()
print("Load KV Data Aerospike")
if len(values[0]) == 2:
for value in values:
key = (self.namespace, self.set, value[0])
print(key)
self.client.put(key, {"bin_value": value[1]}, policy={"key": aerospike.POLICY_KEY_SEND})
assert self.client.exists(key)
else:
assert("VALUES SIZE != 2")
# print(values)
def load_data(self, data, table_name):
print("Load Data Aerospike")
# print(data)

View File

@ -1,11 +1,10 @@
import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
from dictionary import Field, Row, Dictionary, DictionaryStructure, Layout
from external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, SourceExecutableHashed, SourceMongo
from external_sources import SourceHTTP, SourceHTTPS
from external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, SourceExecutableHashed
from external_sources import SourceMongo, SourceHTTP, SourceHTTPS
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
@ -95,6 +94,7 @@ DICTIONARIES = []
cluster = None
node = None
def setup_module(module):
global DICTIONARIES
global cluster
@ -123,6 +123,7 @@ def setup_module(module):
node = cluster.add_instance('node', main_configs=main_configs, with_mysql=True, with_mongo=True)
cluster.add_instance('clickhouse1')
@pytest.fixture(scope="module")
def started_cluster():
try:
@ -143,11 +144,11 @@ def test_simple_dictionaries(started_cluster):
data = [
Row(fields,
[1, 22, 333, 4444, 55555, -6, -77,
-888, -999, '550e8400-e29b-41d4-a716-446655440003',
-888, -999, '550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25', 'hello', 22.543, 3332154213.4, 0]),
Row(fields,
[2, 3, 4, 5, 6, -7, -8,
-9, -10, '550e8400-e29b-41d4-a716-446655440002',
-9, -10, '550e8400-e29b-41d4-a716-446655440002',
'1978-06-28', '1986-02-28 23:42:25', 'hello', 21.543, 3222154213.4, 1]),
]
@ -188,18 +189,19 @@ def test_simple_dictionaries(started_cluster):
answer = str(answer).replace(' ', '')
assert node.query(query) == str(answer) + '\n'
def test_complex_dictionaries(started_cluster):
fields = FIELDS["complex"]
data = [
Row(fields,
[1, 'world', 22, 333, 4444, 55555, -6,
-77, -888, -999, '550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25',
'hello', 22.543, 3332154213.4]),
-77, -888, -999, '550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25',
'hello', 22.543, 3332154213.4]),
Row(fields,
[2, 'qwerty2', 52, 2345, 6544, 9191991, -2,
-717, -81818, -92929, '550e8400-e29b-41d4-a716-446655440007',
'1975-09-28', '2000-02-28 23:33:24',
-717, -81818, -92929, '550e8400-e29b-41d4-a716-446655440007',
'1975-09-28', '2000-02-28 23:33:24',
'my', 255.543, 3332221.44]),
]
@ -227,21 +229,22 @@ def test_complex_dictionaries(started_cluster):
print query
assert node.query(query) == str(answer) + '\n'
def test_ranged_dictionaries(started_cluster):
fields = FIELDS["ranged"]
data = [
Row(fields,
[1, '2019-02-10', '2019-02-01', '2019-02-28',
22, 333, 4444, 55555, -6, -77, -888, -999,
'550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25', 'hello',
22.543, 3332154213.4]),
22, 333, 4444, 55555, -6, -77, -888, -999,
'550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25', 'hello',
22.543, 3332154213.4]),
Row(fields,
[2, '2019-04-10', '2019-04-01', '2019-04-28',
11, 3223, 41444, 52515, -65, -747, -8388, -9099,
'550e8400-e29b-41d4-a716-446655440004',
'1973-06-29', '2002-02-28 23:23:25', '!!!!',
32.543, 3332543.4]),
11, 3223, 41444, 52515, -65, -747, -8388, -9099,
'550e8400-e29b-41d4-a716-446655440004',
'1973-06-29', '2002-02-28 23:23:25', '!!!!',
32.543, 3332543.4]),
]
ranged_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "ranged"]

View File

@ -0,0 +1,325 @@
import os
import pytest
from dictionary import Field, Row, Dictionary, DictionaryStructure, Layout
from external_sources import SourceRedis, SourceAerospike
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
FIELDS = {
"simple": [
Field("KeyField", 'UInt64', is_key=True, default_value_for_get=9999999),
Field("UInt8_", 'UInt8', default_value_for_get=55),
Field("UInt16_", 'UInt16', default_value_for_get=66),
Field("UInt32_", 'UInt32', default_value_for_get=77),
Field("UInt64_", 'UInt64', default_value_for_get=88),
Field("Int8_", 'Int8', default_value_for_get=-55),
Field("Int16_", 'Int16', default_value_for_get=-66),
Field("Int32_", 'Int32', default_value_for_get=-77),
Field("Int64_", 'Int64', default_value_for_get=-88),
Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'),
Field("Date_", 'Date', default_value_for_get='2018-12-30'),
Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'),
Field("String_", 'String', default_value_for_get='hi'),
Field("Float32_", 'Float32', default_value_for_get=555.11),
Field("Float64_", 'Float64', default_value_for_get=777.11),
Field("ParentKeyField", "UInt64", default_value_for_get=444, hierarchical=True),
],
"complex": [
Field("KeyField1", 'UInt64', is_key=True, default_value_for_get=9999999),
Field("KeyField2", 'String', is_key=True, default_value_for_get='xxxxxxxxx'),
Field("UInt8_", 'UInt8', default_value_for_get=55),
Field("UInt16_", 'UInt16', default_value_for_get=66),
Field("UInt32_", 'UInt32', default_value_for_get=77),
Field("UInt64_", 'UInt64', default_value_for_get=88),
Field("Int8_", 'Int8', default_value_for_get=-55),
Field("Int16_", 'Int16', default_value_for_get=-66),
Field("Int32_", 'Int32', default_value_for_get=-77),
Field("Int64_", 'Int64', default_value_for_get=-88),
Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'),
Field("Date_", 'Date', default_value_for_get='2018-12-30'),
Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'),
Field("String_", 'String', default_value_for_get='hi'),
Field("Float32_", 'Float32', default_value_for_get=555.11),
Field("Float64_", 'Float64', default_value_for_get=777.11),
],
"ranged": [
Field("KeyField1", 'UInt64', is_key=True),
Field("KeyField2", 'Date', is_range_key=True),
Field("StartDate", 'Date', range_hash_type='min'),
Field("EndDate", 'Date', range_hash_type='max'),
Field("UInt8_", 'UInt8', default_value_for_get=55),
Field("UInt16_", 'UInt16', default_value_for_get=66),
Field("UInt32_", 'UInt32', default_value_for_get=77),
Field("UInt64_", 'UInt64', default_value_for_get=88),
Field("Int8_", 'Int8', default_value_for_get=-55),
Field("Int16_", 'Int16', default_value_for_get=-66),
Field("Int32_", 'Int32', default_value_for_get=-77),
Field("Int64_", 'Int64', default_value_for_get=-88),
Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'),
Field("Date_", 'Date', default_value_for_get='2018-12-30'),
Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'),
Field("String_", 'String', default_value_for_get='hi'),
Field("Float32_", 'Float32', default_value_for_get=555.11),
Field("Float64_", 'Float64', default_value_for_get=777.11),
],
}
VALUES = {
"simple": [
[
1, 22, 333, 4444, 55555, -6, -77,
-888, -999, '550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25', 'hello', 22.543, 3332154213.4, 0,
],
[
2, 3, 4, 5, 6, -7, -8,
-9, -10, '550e8400-e29b-41d4-a716-446655440002',
'1978-06-28', '1986-02-28 23:42:25', 'hello', 21.543, 3222154213.4, 1,
],
],
"complex": [
[
1, 'world', 22, 333, 4444, 55555, -6,
-77, -888, -999, '550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25',
'hello', 22.543, 3332154213.4,
],
[
2, 'qwerty2', 52, 2345, 6544, 9191991, -2,
-717, -81818, -92929, '550e8400-e29b-41d4-a716-446655440007',
'1975-09-28', '2000-02-28 23:33:24',
'my', 255.543, 3332221.44,
],
],
"ranged": [
[
1, '2019-02-10', '2019-02-01', '2019-02-28',
22, 333, 4444, 55555, -6, -77, -888, -999,
'550e8400-e29b-41d4-a716-446655440003',
'1973-06-28', '1985-02-28 23:43:25', 'hello',
22.543, 3332154213.4,
],
[
2, '2019-04-10', '2019-04-01', '2019-04-28',
11, 3223, 41444, 52515, -65, -747, -8388, -9099,
'550e8400-e29b-41d4-a716-446655440004',
'1973-06-29', '2002-02-28 23:23:25', '!!!!',
32.543, 3332543.4,
],
],
}
LAYOUTS = [
Layout("flat"),
Layout("hashed"),
Layout("cache"),
Layout('complex_key_hashed_one_key'),
Layout('complex_key_hashed_two_keys'),
Layout("complex_key_cache"),
Layout("range_hashed"),
]
SOURCES = [
SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "", storage_type="simple"),
SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "", storage_type="hash_map"),
# SourceAerospike("Aerospike", "localhost", "3000", "aerospike1", "3000", "", ""),
]
DICTIONARIES = []
cluster = None
node = None
def setup_kv_dict(suffix, layout, fields, kv_source, dict_configs_path, values):
global DICTIONARIES
structure = DictionaryStructure(layout, fields)
dict_name = "{}_{}_{}".format(kv_source.name, layout.name, suffix)
dict_path = os.path.join(dict_configs_path, dict_name + '.xml')
dictionary = Dictionary(dict_name, structure, kv_source, dict_path, "table_" + dict_name, fields, values)
dictionary.generate_config()
DICTIONARIES.append(dictionary)
def setup_module(module):
global DICTIONARIES
global cluster
global node
dict_configs_path = os.path.join(SCRIPT_DIR, 'configs/dictionaries')
for f in os.listdir(dict_configs_path):
os.remove(os.path.join(dict_configs_path, f))
for layout in LAYOUTS:
for source in SOURCES:
if source.compatible_with_layout(layout):
if layout.layout_type == "simple":
fields_len = len(FIELDS["simple"])
for i in range(fields_len - 1):
local_fields = [FIELDS["simple"][0], FIELDS["simple"][i + 1]]
local_values = [[value[0], value[i + 1]] for value in VALUES["simple"]]
setup_kv_dict(i + 1, layout, local_fields, source, dict_configs_path, local_values)
elif layout.layout_type == "complex":
fields_len = len(FIELDS["complex"])
for i in range(fields_len - 2):
if layout.name == 'complex_key_hashed_two_keys':
local_fields = [FIELDS['complex'][0], FIELDS['complex'][1], FIELDS['complex'][i + 2]]
local_values = [[value[0], value[1], value[i + 2]] for value in VALUES["complex"]]
else:
local_fields = [FIELDS['complex'][1], FIELDS['complex'][i + 2]]
local_values = [[value[1], value[i + 2]] for value in VALUES["complex"]]
setup_kv_dict(i + 2, layout, local_fields, source, dict_configs_path, local_values)
elif layout.layout_type == "ranged":
fields_len = len(FIELDS["ranged"])
local_fields = FIELDS["ranged"][0:5]
local_values = VALUES["ranged"][0:5]
for i in range(fields_len - 4):
local_fields[4] = FIELDS["ranged"][i + 4]
for j, value in enumerate(VALUES["ranged"]):
local_values[j][4] = value[i + 4]
setup_kv_dict(i + 2, layout, local_fields, source, dict_configs_path, local_values)
else:
print "Source", source.name, "incompatible with layout", layout.name
main_configs = []
for fname in os.listdir(dict_configs_path):
main_configs.append(os.path.join(dict_configs_path, fname))
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
node = cluster.add_instance('node', main_configs=main_configs, with_redis=True)
cluster.add_instance('clickhouse1')
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for dictionary in DICTIONARIES:
print "Preparing", dictionary.name
dictionary.prepare_source(cluster)
print "Prepared"
yield cluster
finally:
cluster.shutdown()
def prepare_data(fields, values_by_row):
return [Row(fields, values) for values in values_by_row]
def test_simple_kv_dictionaries(started_cluster):
simple_kv_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "simple"]
for dct in simple_kv_dicts:
queries_with_answers = []
fields = dct.fields
print("FIELDS AND VALUES FOR " + dct.name)
print(fields)
print(dct.values)
data = prepare_data(fields, dct.values)
dct.source.load_kv_data(dct.values)
try:
node.query("system reload dictionary '{}'".format(dct.name))
except Exception:
print(dct.name)
raise
for row in data:
for field in fields:
if not field.is_key:
for query in dct.get_select_get_queries(field, row):
queries_with_answers.append((query, row.get_value_by_name(field.name)))
for query in dct.get_select_has_queries(field, row):
queries_with_answers.append((query, 1))
for query in dct.get_select_get_or_default_queries(field, row):
queries_with_answers.append((query, field.default_value_for_get))
if dct.fields[1].hierarchical:
for query in dct.get_hierarchical_queries(data[0]):
queries_with_answers.append((query, [1]))
for query in dct.get_hierarchical_queries(data[1]):
queries_with_answers.append((query, [2, 1]))
for query in dct.get_is_in_queries(data[0], data[1]):
queries_with_answers.append((query, 0))
for query in dct.get_is_in_queries(data[1], data[0]):
queries_with_answers.append((query, 1))
for query, answer in queries_with_answers:
if isinstance(answer, list):
answer = str(answer).replace(' ', '')
print query
assert node.query(query) == str(answer) + '\n', query
def test_complex_dictionaries(started_cluster):
complex_kv_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex"]
for dct in complex_kv_dicts:
queries_with_answers = []
fields = dct.fields
print("FIELDS AND VALUES FOR " + dct.name)
print(fields)
print(dct.values)
data = prepare_data(fields, dct.values)
dct.source.load_kv_data(dct.values)
try:
node.query("system reload dictionary '{}'".format(dct.name))
except Exception:
print(dct.name)
raise
for row in data:
for field in fields:
if not field.is_key:
for query in dct.get_select_get_queries(field, row):
queries_with_answers.append((query, row.get_value_by_name(field.name)))
for query in dct.get_select_has_queries(field, row):
queries_with_answers.append((query, 1))
for query in dct.get_select_get_or_default_queries(field, row):
queries_with_answers.append((query, field.default_value_for_get))
for query, answer in queries_with_answers:
print query
assert node.query(query) == str(answer) + '\n'
def test_ranged_dictionaries(started_cluster):
complex_kv_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "ranged"]
for dct in complex_kv_dicts:
queries_with_answers = []
fields = dct.fields
print("FIELDS AND VALUES FOR " + dct.name)
print(fields)
print(dct.values)
data = prepare_data(fields, dct.values)
dct.source.load_kv_data(dct.values)
try:
node.query("system reload dictionary '{}'".format(dct.name))
except Exception:
print(dct.name)
raise
for row in data:
for field in fields:
if not field.is_key and not field.is_range:
for query in dct.get_select_get_queries(field, row):
queries_with_answers.append((query, row.get_value_by_name(field.name)))
for query, answer in queries_with_answers:
print query
assert node.query(query) == str(answer) + '\n'

View File

@ -27,10 +27,11 @@ Types of sources (`source_type`):
- [Executable file](#dicts-external_dicts_dict_sources-executable)
- [HTTP(s)](#dicts-external_dicts_dict_sources-http)
- DBMS
- [ODBC](#dicts-external_dicts_dict_sources-odbc)
- [MySQL](#dicts-external_dicts_dict_sources-mysql)
- [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [ODBC](#dicts-external_dicts_dict_sources-odbc)
- [Redis](#dicts-external_dicts_dict_sources-redis)
## Local File {#dicts-external_dicts_dict_sources-local_file}
@ -423,4 +424,27 @@ Setting fields:
- `db` Name of the database.
- `collection` Name of the collection.
### Redis {#dicts-external_dicts_dict_sources-redis}
Example of settings:
```xml
<source>
<redis>
<host>localhost</host>
<port>6379</port>
<storage_type>simple</storage_type>
<db_index>0</db_index>
</redis>
</source>
```
Setting fields:
- `host` The Redis host.
- `port` The port on the Redis server.
- `storage_type` The structure of internal Redis storage using for work with keys. `simple` is for simple sources and for hashed single key sources, `hash_map` is for hashed sources with two keys. Ranged sources and cache sources with complex key are unsupported. May be omitted, default value is `simple`.
- `db_index` The specific numeric index of Redis logical database. May be omitted, default value is 0.
[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) <!--hide-->

View File

@ -1,5 +1,5 @@
# Источники внешних словарей
# Источники внешних словарей {#dicts-external_dicts_dict_sources}
Внешний словарь можно подключить из множества источников.
@ -24,17 +24,18 @@
Типы источников (`source_type`):
- [Локальный файл](#ispolniaemyi-fail)
- [Исполняемый файл](#ispolniaemyi-fail)
- [HTTP(s)](#http-s)
- [Локальный файл](#dicts-external_dicts_dict_sources-local_file)
- [Исполняемый файл](#dicts-external_dicts_dict_sources-executable)
- [HTTP(s)](#dicts-external_dicts_dict_sources-http)
- СУБД:
- [ODBC](#dicts-external_dicts_dict_sources-odbc)
- [MySQL](#mysql)
- [ClickHouse](#clickhouse)
- [MongoDB](#mongodb)
- [MySQL](#dicts-external_dicts_dict_sources-mysql)
- [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
- [MongoDB](#dicts-external_dicts_dict_sources-mongodb)
- [Redis](#dicts-external_dicts_dict_sources-redis)
## Локальный файл
## Локальный файл {#dicts-external_dicts_dict_sources-local_file}
Пример настройки:
@ -53,7 +54,7 @@
- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
## Исполняемый файл
## Исполняемый файл {#dicts-external_dicts_dict_sources-executable}
Работа с исполняемым файлом зависит от [размещения словаря в памяти](external_dicts_dict_layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос в `STDIN` исполняемого файла.
@ -74,7 +75,7 @@
- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)".
## HTTP(s)
## HTTP(s) {#dicts-external_dicts_dict_sources-http}
Работа с HTTP(s) сервером зависит от [размещения словаря в памяти](external_dicts_dict_layout.md). Если тип размещения словаря `cache` и `complex_key_cache`, то ClickHouse запрашивает необходимые ключи, отправляя запрос методом `POST`.
@ -362,7 +363,7 @@ MySQL можно подключить на локальном хосте чер
```
### ClickHouse
### ClickHouse {#dicts-external_dicts_dict_sources-clickhouse}
Пример настройки:
@ -392,7 +393,7 @@ MySQL можно подключить на локальном хосте чер
- `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md).
### MongoDB
### MongoDB {#dicts-external_dicts_dict_sources-mongodb}
Пример настройки:
@ -418,4 +419,26 @@ MySQL можно подключить на локальном хосте чер
- `db` - имя базы данных.
- `collection` - имя коллекции.
### Redis {#dicts-external_dicts_dict_sources-redis}
Пример настройки:
```xml
<source>
<redis>
<host>localhost</host>
<port>6379</port>
<storage_type>simple</storage_type>
<db_index>0</db_index>
</redis>
</source>
```
Поля настройки:
- `host` хост Redis.
- `port` порт сервера Redis.
- `storage_type` способ хранения ключей. Необходимо использовать `simple` для источников с одним столбцом ключей, `hash_map` -- для источников с двумя столбцами ключей. Источники с более, чем двумя столбцами ключей, не поддерживаются. Может отсутствовать, значение по умолчанию `simple`.
- `db_index` номер базы данных. Может отсутствовать, значение по умолчанию 0.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_sources/) <!--hide-->

View File

@ -21,7 +21,7 @@ BUILD_TARGETS=clickhouse
BUILD_TYPE=Debug
ENABLE_EMBEDDED_COMPILER=0
CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_JEMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_SSL=0 -D ENABLE_POCO_NETSSL=0"
CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_JEMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_REDIS=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_SSL=0 -D ENABLE_POCO_NETSSL=0"
[[ $(uname) == "FreeBSD" ]] && COMPILER_PACKAGE_VERSION=devel && export COMPILER_PATH=/usr/local/bin