ClickHouse/dbms/src/Dictionaries/RedisDictionarySource.cpp

227 lines
8.2 KiB
C++
Raw Normal View History

2019-01-15 22:08:56 +00:00
#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
2019-03-30 21:42:13 +00:00
return std::make_unique<RedisDictionarySource>(dict_struct, config, config_prefix + ".redis", sample_block);
2019-01-15 22:08:56 +00:00
#else
2019-03-30 21:42:13 +00:00
(void)dict_struct;
2019-01-15 22:08:56 +00:00
(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/AsyncReader.h>
# include <Poco/Redis/Client.h>
# include <Poco/Redis/Command.h>
# include <Poco/Redis/Error.h>
# include <Poco/Redis/Exception.h>
# include <Poco/Redis/RedisEventArgs.h>
# include <Poco/Redis/RedisStream.h>
# include <Poco/Redis/Type.h>
# include <Poco/Util/AbstractConfiguration.h>
# include <Poco/Version.h>
# include <IO/WriteHelpers.h>
# include <Common/FieldVisitors.h>
# include <ext/enumerate.h>
# include "RedisBlockInputStream.h"
2019-04-15 01:34:10 +00:00
# include "Poco/Logger.h"
# include "common/logger_useful.h"
2019-01-15 22:08:56 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int CANNOT_SELECT;
2019-04-16 23:13:07 +00:00
extern const int INVALID_CONFIG_PARAMETER;
2019-01-15 22:08:56 +00:00
}
static const size_t max_block_size = 8192;
RedisDictionarySource::RedisDictionarySource(
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
UInt8 db_index,
2019-04-16 23:13:07 +00:00
RedisStorageType::Id storage_type,
2019-01-15 22:08:56 +00:00
const Block & sample_block)
: dict_struct{dict_struct}
, host{host}
, port{port}
, db_index{db_index}
2019-04-16 23:13:07 +00:00
, storage_type{storage_type}
2019-01-15 22:08:56 +00:00
, sample_block{sample_block}
, client{std::make_shared<Poco::Redis::Client>(host, port)}
{
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "in ctor");
LOG_INFO(&Logger::get("Redis"), dict_struct.attributes.size());
2019-04-16 23:13:07 +00:00
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};
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "After first check");
2019-04-16 23:13:07 +00:00
if (storage_type == RedisStorageType::HASH_MAP)
{
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "SET STORAGE_TYPE");
2019-04-16 23:13:07 +00:00
if (!dict_struct.key.has_value())
2019-05-25 00:28:09 +00:00
throw Exception{"Redis source with storage type \'hash_map\' must have key",
2019-04-16 23:13:07 +00:00
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
}
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "After second check");
if (db_index != 0)
{
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "SET DB_INDEX");
2019-04-16 23:13:07 +00:00
Poco::Redis::Command command("SELECT");
command << static_cast<Int64>(db_index);
2019-04-14 17:44:44 +00:00
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};
}
2019-05-25 00:28:09 +00:00
LOG_INFO(&Logger::get("Redis"), "After third check");
2019-01-15 22:08:56 +00:00
}
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),
2019-04-16 23:13:07 +00:00
parseStorageType(config.getString(config_prefix + ".storage_type", "")),
2019-01-15 22:08:56 +00:00
sample_block)
{
}
RedisDictionarySource::RedisDictionarySource(const RedisDictionarySource & other)
: RedisDictionarySource{other.dict_struct,
other.host,
other.port,
other.db_index,
2019-04-16 23:13:07 +00:00
other.storage_type,
2019-01-15 22:08:56 +00:00
other.sample_block}
{
}
RedisDictionarySource::~RedisDictionarySource() = default;
BlockInputStreamPtr RedisDictionarySource::loadAll()
{
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "Redis in loadAll");
2019-04-15 01:34:10 +00:00
2019-04-16 23:13:07 +00:00
Poco::Redis::Command command_for_keys("KEYS");
command_for_keys << "*";
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "Command for keys: " + command_for_keys.toString());
2019-02-13 00:05:43 +00:00
2019-04-16 23:13:07 +00:00
Poco::Redis::Array keys = client->execute<Poco::Redis::Array>(command_for_keys);
2019-01-27 13:14:02 +00:00
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "Command for keys executed");
LOG_INFO(&Logger::get("Redis"), "KEYS: " + keys.toString());
2019-04-15 01:34:10 +00:00
2019-04-16 23:13:07 +00:00
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);
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "Command for hkeys executed");
2019-04-16 23:13:07 +00:00
Poco::SharedPtr<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);
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "HKEYS: " + primary_with_secondary->toString());
2019-04-16 23:26:57 +00:00
hkeys.add(*primary_with_secondary);
2019-04-16 23:13:07 +00:00
}
keys = hkeys;
}
2019-01-27 22:22:18 +00:00
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
2019-01-15 22:08:56 +00:00
}
2019-01-27 13:14:02 +00:00
BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector<UInt64> & ids)
2019-01-15 22:08:56 +00:00
{
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "Redis in loadIds");
2019-04-15 01:34:10 +00:00
2019-04-16 23:13:07 +00:00
if (storage_type != RedisStorageType::SIMPLE)
throw Exception{"Cannot use loadIds with \'simple\' storage type", ErrorCodes::UNSUPPORTED_METHOD};
2019-01-27 13:14:02 +00:00
if (!dict_struct.id)
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
2019-01-15 22:08:56 +00:00
2019-01-27 13:14:02 +00:00
Poco::Redis::Array keys;
2019-01-15 22:08:56 +00:00
2019-03-30 21:42:13 +00:00
for (UInt64 id : ids)
2019-01-27 13:14:02 +00:00
keys << static_cast<Int64>(id);
2019-01-15 22:08:56 +00:00
2019-05-23 23:42:21 +00:00
LOG_INFO(&Logger::get("Redis"), "KEYS: " + keys.toString());
2019-04-15 01:34:10 +00:00
2019-01-27 22:22:18 +00:00
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), sample_block, max_block_size);
2019-01-15 22:08:56 +00:00
}
2019-01-27 13:14:02 +00:00
2019-01-15 22:08:56 +00:00
std::string RedisDictionarySource::toString() const
{
return "Redis: " + host + ':' + DB::toString(port);
}
2019-04-17 10:14:07 +00:00
RedisStorageType::Id RedisDictionarySource::parseStorageType(const std::string & storage_type)
{
2019-04-16 23:13:07 +00:00
RedisStorageType::Id storage_type_id = RedisStorageType::valueOf(storage_type);
2019-04-17 10:14:07 +00:00
if (storage_type_id == RedisStorageType::UNKNOWN)
2019-04-16 23:13:07 +00:00
storage_type_id = RedisStorageType::SIMPLE;
return storage_type_id;
}
2019-01-15 22:08:56 +00:00
}
#endif