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
|
|
|
|
{
|
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)
|
|
|
|
{
|
2023-05-29 07:22:29 +00:00
|
|
|
/// TODO use scan to avoid performance issue
|
2023-05-23 07:31:50 +00:00
|
|
|
RedisCommand command_for_keys("KEYS");
|
2023-05-23 12:54:26 +00:00
|
|
|
command_for_keys << "*";
|
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());
|
|
|
|
|
2023-05-23 12:54:26 +00:00
|
|
|
num_threads = std::min<size_t>(num_threads, configuration.pool_size);
|
2023-05-23 07:31:50 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2023-05-23 12:54:26 +00:00
|
|
|
delete connection.release();
|
|
|
|
|
2023-05-23 07:31:50 +00:00
|
|
|
/// TODO reduce keys copy
|
|
|
|
pipes.emplace_back(std::make_shared<RedisSource>(
|
2023-05-23 12:54:26 +00:00
|
|
|
getRedisConnection(pool, configuration), keys,
|
|
|
|
configuration.storage_type, sample_block, redis_types, max_block_size));
|
2023-05-23 07:31:50 +00:00
|
|
|
}
|
|
|
|
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());
|
|
|
|
|
2023-05-23 12:54:26 +00:00
|
|
|
num_threads = std::min<size_t>(num_threads, configuration.pool_size);
|
2023-05-23 07:31:50 +00:00
|
|
|
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++)
|
2023-05-29 07:22:29 +00:00
|
|
|
{
|
|
|
|
if (WhichDataType(*primary_key_data_type).isStringOrFixedString())
|
|
|
|
{
|
|
|
|
keys.add(fields->at(pos).get<String>());
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
keys.add(toString(fields->at(pos))); /// TODO redis source deserialize
|
|
|
|
}
|
|
|
|
}
|
2023-05-23 07:31:50 +00:00
|
|
|
|
|
|
|
if (configuration.storage_type == RedisStorageType::HASH_MAP)
|
|
|
|
{
|
|
|
|
keys = *getRedisHashMapKeys(connection, keys);
|
|
|
|
}
|
|
|
|
|
2023-05-23 12:54:26 +00:00
|
|
|
delete connection.release();
|
|
|
|
|
2023-05-23 07:31:50 +00:00
|
|
|
pipes.emplace_back(std::make_shared<RedisSource>(
|
2023-05-23 12:54:26 +00:00
|
|
|
getRedisConnection(pool, configuration), keys,
|
|
|
|
configuration.storage_type, sample_block, redis_types, max_block_size));
|
2023-05-23 07:31:50 +00:00
|
|
|
}
|
|
|
|
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*/)
|
|
|
|
{
|
2023-05-23 12:54:26 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is unsupported for StorageRedis");
|
2023-05-17 02:42:52 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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)
|
|
|
|
{
|
2023-05-26 02:34:37 +00:00
|
|
|
auto configuration = getRedisConfiguration(args.engine_args, args.getLocalContext());
|
2023-05-17 02:42:52 +00:00
|
|
|
|
2023-05-23 12:54:26 +00:00
|
|
|
checkRedisTableStructure(args.columns, configuration);
|
|
|
|
|
2023-05-17 02:42:52 +00:00
|
|
|
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-24 10:06:42 +00:00
|
|
|
.source_access_type = AccessType::REDIS,
|
2023-05-17 02:42:52 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|