add truncate to redis storage

This commit is contained in:
JackyWoo 2023-05-31 16:31:06 +08:00
parent 1df1dfc3e5
commit 0106704573
2 changed files with 43 additions and 5 deletions

View File

@ -87,6 +87,7 @@ public:
return storage.getBySerializedKeys(raw_keys, nullptr);
}
/// TODO scan may get duplicated keys
Chunk generateFullScan()
{
/// redis scan ending
@ -480,17 +481,16 @@ SinkToStoragePtr StorageRedis::write(
return std::make_shared<RedisSink>(*this, metadata_snapshot);
}
/// TODO use scan to reduce latency
void StorageRedis::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &)
{
auto connection = getRedisConnection(pool, configuration);
RedisCommand cmd_flush_db("FLUSHDB");
cmd_flush_db << toString(configuration.db_index);
auto ret = connection->client->execute<RedisBulkString>(cmd_flush_db);
cmd_flush_db.add("ASYNC");
auto ret = connection->client->execute<RedisSimpleString>(cmd_flush_db);
if (ret.isNull() || ret.value() != "OK")
throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to truncate redis table {}, for {}", table_id.getFullNameNotQuoted(), ret.value());
if (ret != "OK")
throw Exception(ErrorCodes::INTERNAL_REDIS_ERROR, "Fail to truncate redis table {}, for {}", table_id.getFullNameNotQuoted(), ret);
}
void StorageRedis::checkMutationIsPossible(const MutationCommands & commands, const Settings & /* settings */) const
@ -580,6 +580,7 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_
}
}
/// TODO support ttl
void registerStorageRedis(StorageFactory & factory)
{
StorageFactory::StorageFeatures features{

View File

@ -332,3 +332,40 @@ def test_delete(started_cluster):
response = TSV.toMat(node.query("SELECT k, m, n FROM test_delete FORMAT TSV"))
assert (len(response) == 0)
def test_truncate(started_cluster):
client = get_redis_connection()
address = get_address_for_ch()
# clean all
client.flushall()
drop_table('test_truncate')
node.query(
f"""
CREATE TABLE test_truncate(
k UInt32,
m DateTime,
n String
) Engine=Redis('{address}', 0, 'clickhouse') PRIMARY KEY (k)
"""
)
node.query(
"""
INSERT INTO test_truncate Values
(1, '2023-06-01 00:00:00', 'lili'), (2, '2023-06-02 00:00:00', 'lucy')
"""
)
response = node.query(
"""
TRUNCATE TABLE test_truncate
"""
)
print("truncate table response: ", response)
response = TSV.toMat(node.query("SELECT COUNT(*) FROM test_truncate FORMAT TSV"))
assert (len(response) == 1)
assert (response[0] == ['0'])