mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Add test on lost messages
This commit is contained in:
parent
7723a63d0a
commit
d479836362
@ -12,7 +12,7 @@ services:
|
||||
- label:disable
|
||||
|
||||
kafka1:
|
||||
image: confluentinc/cp-kafka:4.1.0
|
||||
image: confluentinc/cp-kafka:5.2.0
|
||||
hostname: kafka1
|
||||
ports:
|
||||
- "9092:9092"
|
||||
|
@ -7,7 +7,8 @@ from helpers.test_tools import TSV
|
||||
|
||||
import json
|
||||
import subprocess
|
||||
from kafka import KafkaProducer
|
||||
import kafka.errors
|
||||
from kafka import KafkaAdminClient, KafkaProducer
|
||||
from google.protobuf.internal.encoder import _VarintBytes
|
||||
|
||||
"""
|
||||
@ -318,6 +319,52 @@ def test_kafka_materialized_view(kafka_cluster):
|
||||
''')
|
||||
|
||||
|
||||
def test_kafka_flush_on_big_message(kafka_cluster):
|
||||
# Create batchs of messages of size ~100Kb
|
||||
kafka_messages = 10000
|
||||
batch_messages = 1000
|
||||
messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(kafka_messages)]
|
||||
kafka_produce('flush', messages)
|
||||
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.view;
|
||||
DROP TABLE IF EXISTS test.consumer;
|
||||
CREATE TABLE test.kafka (key UInt64, value String)
|
||||
ENGINE = Kafka
|
||||
SETTINGS
|
||||
kafka_broker_list = 'kafka1:19092',
|
||||
kafka_topic_list = 'flush',
|
||||
kafka_group_name = 'flush',
|
||||
kafka_format = 'JSONEachRow',
|
||||
kafka_max_block_size = 10;
|
||||
CREATE TABLE test.view (key UInt64, value String)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key;
|
||||
CREATE MATERIALIZED VIEW test.consumer TO test.view AS
|
||||
SELECT * FROM test.kafka;
|
||||
''')
|
||||
|
||||
client = KafkaAdminClient(bootstrap_servers="localhost:9092")
|
||||
received = False
|
||||
while not received:
|
||||
try:
|
||||
offsets = client.list_consumer_group_offsets('flush')
|
||||
for topic, offset in offsets.items():
|
||||
if topic.topic == 'flush' and offset.offset == kafka_messages:
|
||||
received = True
|
||||
break
|
||||
except kafka.errors.GroupCoordinatorNotAvailableError:
|
||||
continue
|
||||
|
||||
for _ in range(20):
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT count() FROM test.view')
|
||||
if int(result) == kafka_messages*batch_messages:
|
||||
break
|
||||
|
||||
assert int(result) == kafka_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result)
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
cluster.start()
|
||||
raw_input("Cluster created, press any key to destroy...")
|
||||
|
Loading…
Reference in New Issue
Block a user