ClickHouse/src/Storages/StorageRedis.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

219 lines
7.8 KiB
C++
Raw Normal View History

2023-05-17 02:42:52 +00:00
#include <Storages/StorageRedis.h>
2023-05-20 03:48:57 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/NamedCollectionsHelpers.h>
2023-05-17 02:42:52 +00:00
#include <Storages/checkAndGetLiteralArgument.h>
2023-05-23 07:31:50 +00:00
#include <Storages/KVStorageUtils.h>
2023-05-17 02:42:52 +00:00
#include <unordered_set>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <QueryPipeline/Pipe.h>
2023-05-23 07:31:50 +00:00
#include <Common/logger_useful.h>
2023-05-17 02:42:52 +00:00
#include <Common/parseAddress.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2023-05-23 07:31:50 +00:00
extern const int INVALID_REDIS_STORAGE_TYPE;
2023-05-17 02:42:52 +00:00
extern const int NOT_IMPLEMENTED;
}
StorageRedis::StorageRedis(
const StorageID & table_id_,
2023-05-20 03:48:57 +00:00
const RedisConfiguration & configuration_,
2023-05-17 02:42:52 +00:00
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
2023-05-20 03:48:57 +00:00
const String & comment_)
: IStorage(table_id_)
, table_id(table_id_)
, configuration(configuration_)
2023-05-23 07:31:50 +00:00
, log(&Poco::Logger::get("StorageRedis"))
2023-05-17 02:42:52 +00:00
{
2023-05-20 03:48:57 +00:00
pool = std::make_shared<RedisPool>(configuration.pool_size);
2023-05-23 07:31:50 +00:00
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment_);
setInMemoryMetadata(storage_metadata);
2023-05-17 02:42:52 +00:00
}
Pipe StorageRedis::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
2023-05-23 07:31:50 +00:00
SelectQueryInfo & query_info,
ContextPtr context,
2023-05-17 02:42:52 +00:00
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
2023-05-23 07:31:50 +00:00
size_t num_streams)
2023-05-17 02:42:52 +00:00
{
2023-05-20 03:48:57 +00:00
auto connection = getRedisConnection(pool, configuration);
2023-05-17 02:42:52 +00:00
storage_snapshot->check(column_names);
Block sample_block;
2023-05-23 07:31:50 +00:00
RedisColumnTypes redis_types;
auto all_columns = storage_snapshot->metadata->getColumns().getNamesOfPhysical();
2023-05-17 02:42:52 +00:00
for (const String & column_name : column_names)
{
auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name);
sample_block.insert({column_data.type, column_data.name});
2023-05-23 07:31:50 +00:00
redis_types.push_back(getRedisColumnType(configuration.storage_type, all_columns, column_name));
2023-05-17 02:42:52 +00:00
}
2023-05-23 07:31:50 +00:00
FieldVectorPtr fields;
bool all_scan = false;
String primary_key = all_columns.at(0);
auto primary_key_data_type = sample_block.getByName(primary_key).type;
std::tie(fields, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info, context);
if (all_scan)
{
RedisCommand command_for_keys("KEYS");
/// generate keys by table name prefix
2023-05-23 08:42:46 +00:00
command_for_keys << table_id.getTableName() + ":" + storageTypeToKeyType(configuration.storage_type) + ":*";
2023-05-23 07:31:50 +00:00
auto all_keys = connection->client->execute<RedisArray>(command_for_keys);
2023-05-23 08:42:46 +00:00
if (all_keys.isNull() || all_keys.size() == 0)
2023-05-23 07:31:50 +00:00
return {};
2023-05-20 03:48:57 +00:00
2023-05-23 07:31:50 +00:00
Pipes pipes;
size_t num_keys = all_keys.size();
size_t num_threads = std::min<size_t>(num_streams, all_keys.size());
assert(num_keys <= std::numeric_limits<uint32_t>::max());
for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx)
{
size_t begin = num_keys * thread_idx / num_threads;
size_t end = num_keys * (thread_idx + 1) / num_threads;
RedisArray keys;
for (size_t pos=begin; pos<std::min(end, num_keys); pos++)
keys.add(all_keys.get<RedisBulkString>(pos));
if (configuration.storage_type == RedisStorageType::HASH_MAP)
{
keys = *getRedisHashMapKeys(connection, keys);
}
/// TODO reduce keys copy
pipes.emplace_back(std::make_shared<RedisSource>(
std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size));
}
return Pipe::unitePipes(std::move(pipes));
}
else
{
if (fields->empty())
return {};
Pipes pipes;
size_t num_keys = fields->size();
size_t num_threads = std::min<size_t>(num_streams, fields->size());
assert(num_keys <= std::numeric_limits<uint32_t>::max());
for (size_t thread_idx = 0; thread_idx < num_threads; ++thread_idx)
{
size_t begin = num_keys * thread_idx / num_threads;
size_t end = num_keys * (thread_idx + 1) / num_threads;
RedisArray keys;
for (size_t pos=begin; pos<std::min(end, num_keys); pos++)
keys.add(fields->at(pos).get<String>());
if (configuration.storage_type == RedisStorageType::HASH_MAP)
{
keys = *getRedisHashMapKeys(connection, keys);
}
pipes.emplace_back(std::make_shared<RedisSource>(
std::move(connection), keys, configuration.storage_type, sample_block, redis_types, max_block_size));
}
return Pipe::unitePipes(std::move(pipes));
}
2023-05-17 02:42:52 +00:00
}
SinkToStoragePtr StorageRedis::write(
const ASTPtr & /*query*/,
const StorageMetadataPtr & /*metadata_snapshot*/,
ContextPtr /*context*/)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is unsupported for StorageRedis");
}
2023-05-20 03:48:57 +00:00
RedisConfiguration StorageRedis::getConfiguration(ASTs engine_args, ContextPtr context)
2023-05-17 02:42:52 +00:00
{
2023-05-20 03:48:57 +00:00
RedisConfiguration configuration;
2023-05-17 02:42:52 +00:00
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context))
{
validateNamedCollection(
*named_collection,
2023-05-23 07:31:50 +00:00
ValidateKeysMultiset<RedisEqualKeysSet>{"host", "port", "hostname", "password", "db_index", "storage_type", "pool_size"},
2023-05-17 02:42:52 +00:00
{});
configuration.host = named_collection->getAny<String>({"host", "hostname"});
2023-05-20 03:48:57 +00:00
configuration.port = static_cast<uint32_t>(named_collection->get<UInt64>("port"));
2023-05-17 02:42:52 +00:00
configuration.password = named_collection->get<String>("password");
2023-05-20 03:48:57 +00:00
configuration.db_index = static_cast<uint32_t>(named_collection->get<UInt64>({"db_index"}));
2023-05-23 07:31:50 +00:00
configuration.storage_type = keyTypeToStorageType(named_collection->getOrDefault<String>("storage_type", ""));
configuration.pool_size = static_cast<uint32_t>(named_collection->get<UInt64>("pool_size"));
2023-05-17 02:42:52 +00:00
}
else
{
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
/// 6379 is the default Redis port.
auto parsed_host_port = parseAddress(checkAndGetLiteralArgument<String>(engine_args[0], "host:port"), 6379);
configuration.host = parsed_host_port.first;
configuration.port = parsed_host_port.second;
2023-05-20 03:48:57 +00:00
configuration.db_index = static_cast<uint32_t>(checkAndGetLiteralArgument<UInt64>(engine_args[1], "db_index"));
2023-05-17 02:42:52 +00:00
configuration.password = checkAndGetLiteralArgument<String>(engine_args[2], "password");
2023-05-23 07:31:50 +00:00
configuration.storage_type = keyTypeToStorageType(checkAndGetLiteralArgument<String>(engine_args[3], "storage_type"));
configuration.pool_size = static_cast<uint32_t>(checkAndGetLiteralArgument<UInt64>(engine_args[4], "pool_size"));
2023-05-17 02:42:52 +00:00
}
2023-05-23 07:31:50 +00:00
if (configuration.storage_type == RedisStorageType::UNKNOWN)
throw Exception(ErrorCodes::INVALID_REDIS_STORAGE_TYPE, "Invalid Redis storage type");
2023-05-17 02:42:52 +00:00
context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port));
return configuration;
}
void registerStorageRedis(StorageFactory & factory)
{
factory.registerStorage(
2023-05-20 03:48:57 +00:00
"Redis",
2023-05-17 02:42:52 +00:00
[](const StorageFactory::Arguments & args)
{
auto configuration = StorageRedis::getConfiguration(args.engine_args, args.getLocalContext());
return std::make_shared<StorageRedis>(
args.table_id,
2023-05-20 03:48:57 +00:00
configuration,
2023-05-17 02:42:52 +00:00
args.columns,
args.constraints,
args.comment);
},
{
2023-05-20 03:48:57 +00:00
.source_access_type = AccessType::Redis,
2023-05-17 02:42:52 +00:00
});
}
}