ClickHouse/tests/integration/test_table_function_redis/test.py

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

220 lines
4.6 KiB
Python
Raw Normal View History

2023-05-26 02:34:37 +00:00
import time
2023-05-25 04:33:07 +00:00
2023-05-26 02:34:37 +00:00
import redis
2023-05-25 04:33:07 +00:00
import pytest
2023-05-26 02:34:37 +00:00
from helpers.client import QueryRuntimeException
2023-05-25 04:33:07 +00:00
from helpers.cluster import ClickHouseCluster
2023-05-26 02:34:37 +00:00
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", with_redis=True)
2023-05-25 04:33:07 +00:00
@pytest.fixture(scope="module")
2023-05-26 02:34:37 +00:00
def started_cluster():
2023-05-25 04:33:07 +00:00
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
2023-05-26 02:34:37 +00:00
def get_redis_connection(db_id=0):
client = redis.Redis(
host='localhost', port=cluster.redis_port, password="clickhouse", db=db_id
2023-05-25 04:33:07 +00:00
)
2023-05-26 02:34:37 +00:00
return client
def get_address_for_ch():
return cluster.redis_host + ':6379'
def test_storage_simple(started_cluster):
client = get_redis_connection()
address = get_address_for_ch()
# clean all
client.flushall()
data = {}
for i in range(100):
data[str(i)] = str(i)
client.mset(data)
client.close()
response = TSV.toMat(node.query(
f"""
SELECT
key, value
FROM
redis('{address}', 0, 'clickhouse')
WHERE
key='0'
FORMAT TSV
"""))
assert (len(response) == 1)
assert (response[0] == ['0', '0'])
response = TSV.toMat(node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse')
ORDER BY
key
FORMAT TSV
"""))
assert (len(response) == 100)
assert (response[0] == ['0', '0'])
def test_storage_hash_map(started_cluster):
client = get_redis_connection()
address = get_address_for_ch()
# clean all
client.flushall()
key = 'k'
data = {}
for i in range(100):
data[str(i)] = str(i)
client.hset(key, mapping=data)
client.close()
response = TSV.toMat(node.query(
f"""
SELECT
key, field, value
FROM
redis('{address}', 0, 'clickhouse','hash_map')
WHERE
field='0'
FORMAT TSV
"""))
assert (len(response) == 1)
assert (response[0] == ['k', '0', '0'])
response = TSV.toMat(node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse','hash_map')
ORDER BY
field
FORMAT TSV
"""))
assert (len(response) == 100)
assert (response[0] == ['k', '0', '0'])
def test_customized_table_structure(started_cluster):
address = get_address_for_ch()
node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', "simple", 10, "k String, v UInt8")
""")
node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, f UInt8, v String")
""")
# illegal columns
2023-05-25 04:33:07 +00:00
with pytest.raises(QueryRuntimeException):
node.query(
2023-05-26 02:34:37 +00:00
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', "hash_map", 10, "k String, v String")
""")
2023-05-25 04:33:07 +00:00
2023-05-26 02:34:37 +00:00
# illegal data type
with pytest.raises(QueryRuntimeException):
2023-05-25 04:33:07 +00:00
node.query(
2023-05-26 02:34:37 +00:00
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', "simple", 10, "k Ss, v String")
""")
2023-05-29 07:22:29 +00:00
def test_data_type(started_cluster):
client = get_redis_connection()
address = get_address_for_ch()
# string
client.flushall()
client.set('0', '0')
response = TSV.toMat(node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', 'simple', 10, "k String, v UInt8")
WHERE
k='0'
FORMAT TSV
"""))
assert (len(response) == 1)
assert (response[0] == ['0', '0'])
# number
client.flushall()
client.set('0', '0')
response = TSV.toMat(node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', 'simple', 10, "k UInt8, v UInt8")
WHERE
k=0
FORMAT TSV
"""))
assert (len(response) == 1)
assert (response[0] == ['0', '0'])
# datetime
client.flushall()
client.set('2023-06-01 00:00:00', '0')
response = TSV.toMat(node.query(
f"""
SELECT
*
FROM
redis('{address}', 0, 'clickhouse', 'simple', 10, "k DateTime, v UInt8")
WHERE
k='2023-06-01 00:00:00'
FORMAT TSV
"""))
# TODO open
# assert (len(response) == 1)
# assert (response[0] == ['2023-06-01 00:00:00', '0'])