ClickHouse/dbms/src/Dictionaries/RedisDictionarySource.cpp

140 lines
4.0 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
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/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"
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
static const size_t max_block_size = 8192;
RedisDictionarySource::RedisDictionarySource(
const DictionaryStructure & dict_struct,
const std::string & host,
UInt16 port,
const Block & sample_block)
: dict_struct{dict_struct}
, host{host}
, port{port}
, sample_block{sample_block}
, client{std::make_shared<Poco::Redis::Client>(host, port)}
{
}
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"),
sample_block)
{
}
RedisDictionarySource::RedisDictionarySource(const RedisDictionarySource & other)
: RedisDictionarySource{other.dict_struct,
other.host,
other.port,
other.sample_block}
{
}
RedisDictionarySource::~RedisDictionarySource() = default;
BlockInputStreamPtr RedisDictionarySource::loadAll()
{
2019-01-27 15:30:51 +00:00
Poco::Redis::Array commandForKeys;
commandForKeys << "KEYS" << "*";
2019-02-13 00:05:43 +00:00
2019-01-27 15:30:51 +00:00
Poco::Redis::Array keys = client->execute<Poco::Redis::Array>(commandForKeys);
2019-01-27 13:14:02 +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
BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector<UInt64> & ids)
2019-01-15 22:08:56 +00:00
{
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-01-27 13:14:02 +00:00
for (const UInt64 id : ids)
keys << static_cast<Int64>(id);
2019-01-15 22:08:56 +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);
}
}
#endif