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>
|
|
|
|
|
|
|
|
#include <unordered_set>
|
|
|
|
#include <Core/Settings.h>
|
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
|
|
|
#include <QueryPipeline/Pipe.h>
|
|
|
|
#include <Common/NamedCollections/NamedCollections.h>
|
|
|
|
#include <Common/parseAddress.h>
|
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
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_)
|
|
|
|
, columns(columns_)
|
|
|
|
, constraints(constraints_)
|
|
|
|
, comment(comment_)
|
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-17 02:42:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Pipe StorageRedis::read(
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
|
|
|
SelectQueryInfo & /*query_info*/,
|
|
|
|
ContextPtr /*context*/,
|
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t max_block_size,
|
|
|
|
size_t /*num_streams*/)
|
|
|
|
{
|
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;
|
|
|
|
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-20 03:48:57 +00:00
|
|
|
RedisArray keys;
|
|
|
|
RedisCommand command_for_keys("KEYS");
|
|
|
|
/// generate keys by table name prefix
|
|
|
|
command_for_keys << table_id.getTableName() + ":" + toString(configuration.storage_type) + ":*";
|
|
|
|
|
|
|
|
/// Get only keys for specified storage type.
|
|
|
|
auto all_keys = connection->client->execute<RedisArray>(command_for_keys);
|
|
|
|
return Pipe(std::make_shared<RedisSource>(std::move(connection), all_keys, configuration.storage_type, sample_block, max_block_size));
|
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-20 03:48:57 +00:00
|
|
|
ValidateKeysMultiset<RedisEqualKeysSet>{"host", "port", "hostname", "password", "db_index", "storage_type"},
|
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"}));
|
|
|
|
configuration.storage_type = toRedisStorageType(named_collection->getOrDefault<String>("storage_type", ""));
|
|
|
|
configuration.pool_size = 16; /// TODO
|
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-20 03:48:57 +00:00
|
|
|
configuration.storage_type = toRedisStorageType(checkAndGetLiteralArgument<String>(engine_args[3], "storage_type"));
|
|
|
|
configuration.pool_size = 16; /// TODO
|
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
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|