mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Support headers-exchange type
This commit is contained in:
parent
9c49398728
commit
dcd7b7351c
@ -6,6 +6,7 @@
|
||||
#include <memory>
|
||||
#include <Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.h>
|
||||
#include <Storages/RabbitMQ/RabbitMQHandler.h>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <common/logger_useful.h>
|
||||
#include "Poco/Timer.h"
|
||||
#include <amqpcpp.h>
|
||||
@ -122,8 +123,7 @@ void ReadBufferFromRabbitMQConsumer::initExchange()
|
||||
else if (exchange_type == Exchange::DIRECT) type = AMQP::ExchangeType::direct;
|
||||
else if (exchange_type == Exchange::TOPIC) type = AMQP::ExchangeType::topic;
|
||||
else if (exchange_type == Exchange::HASH) type = AMQP::ExchangeType::consistent_hash;
|
||||
else if (exchange_type == Exchange::HEADERS)
|
||||
throw Exception("Headers exchange is not supported", ErrorCodes::BAD_ARGUMENTS);
|
||||
else if (exchange_type == Exchange::HEADERS) type = AMQP::ExchangeType::headers;
|
||||
else throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/* Declare exchange of the specified type and bind it to hash-exchange, which will evenly distribute messages
|
||||
@ -156,14 +156,37 @@ void ReadBufferFromRabbitMQConsumer::initExchange()
|
||||
LOG_ERROR(log, "Failed to declare {} exchange: {}", exchange_type, message);
|
||||
});
|
||||
|
||||
for (auto & routing_key : routing_keys)
|
||||
if (exchange_type == Exchange::HEADERS)
|
||||
{
|
||||
consumer_channel->bindExchange(exchange_name, local_hash_exchange_name, routing_key).onError([&](const char * message)
|
||||
AMQP::Table binding_arguments;
|
||||
std::vector<String> matching;
|
||||
|
||||
for (auto & header : routing_keys)
|
||||
{
|
||||
boost::split(matching, header, [](char c){ return c == '='; });
|
||||
binding_arguments[matching[0]] = matching[1];
|
||||
matching.clear();
|
||||
}
|
||||
|
||||
/// Routing key can be arbitrary here.
|
||||
consumer_channel->bindExchange(exchange_name, local_hash_exchange_name, routing_keys[0], binding_arguments)
|
||||
.onError([&](const char * message)
|
||||
{
|
||||
local_exchange_declared = false;
|
||||
LOG_ERROR(log, "Failed to bind {} exchange to {} exchange: {}", local_exchange_name, exchange_name, message);
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto & routing_key : routing_keys)
|
||||
{
|
||||
consumer_channel->bindExchange(exchange_name, local_hash_exchange_name, routing_key).onError([&](const char * message)
|
||||
{
|
||||
local_exchange_declared = false;
|
||||
LOG_ERROR(log, "Failed to bind {} exchange to {} exchange: {}", local_exchange_name, exchange_name, message);
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -232,7 +255,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
|
||||
*/
|
||||
String hash_exchange_name = exchange_type == Exchange::HASH ? exchange_name : local_exchange_name + "_hash";
|
||||
|
||||
/// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary
|
||||
/// If hash-exchange is used for messages distribution, then the binding key is ignored - can be arbitrary.
|
||||
consumer_channel->bindQueue(hash_exchange_name, queue_name_, binding_key)
|
||||
.onSuccess([&]
|
||||
{
|
||||
@ -244,6 +267,30 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
|
||||
LOG_ERROR(log, "Failed to create queue binding to key {}. Reason: {}", binding_key, message);
|
||||
});
|
||||
}
|
||||
else if (exchange_type == Exchange::HEADERS)
|
||||
{
|
||||
AMQP::Table binding_arguments;
|
||||
std::vector<String> matching;
|
||||
|
||||
/// It is not parsed for the second time - if it was parsed above, then it would go to the first if statement, not here.
|
||||
for (auto & header : routing_keys)
|
||||
{
|
||||
boost::split(matching, header, [](char c){ return c == '='; });
|
||||
binding_arguments[matching[0]] = matching[1];
|
||||
matching.clear();
|
||||
}
|
||||
|
||||
consumer_channel->bindQueue(exchange_name, queue_name_, routing_keys[0], binding_arguments)
|
||||
.onSuccess([&]
|
||||
{
|
||||
bindings_created = true;
|
||||
})
|
||||
.onError([&](const char * message)
|
||||
{
|
||||
bindings_error = true;
|
||||
LOG_ERROR(log, "Failed to create queue binding to key {}. Reason: {}", routing_keys[0], message);
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Means there is only one queue with one consumer - no even distribution needed - no hash-exchange.
|
||||
|
@ -458,13 +458,9 @@ void registerStorageRabbitMQ(StorageFactory & factory)
|
||||
exchange_type = safeGet<String>(ast->value);
|
||||
}
|
||||
|
||||
if (exchange_type != "fanout" && exchange_type != "direct" && exchange_type != "topic" && exchange_type != "consistent_hash")
|
||||
{
|
||||
if (exchange_type == "headers")
|
||||
throw Exception("Headers exchange is not supported", ErrorCodes::BAD_ARGUMENTS);
|
||||
else
|
||||
throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
if (exchange_type != "fanout" && exchange_type != "direct" && exchange_type != "topic"
|
||||
&& exchange_type != "headers" && exchange_type != "consistent_hash")
|
||||
throw Exception("Invalid exchange type", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
UInt64 num_consumers = rabbitmq_settings.rabbitmq_num_consumers;
|
||||
|
@ -1420,6 +1420,103 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster):
|
||||
assert int(result) == messages_num * threads_num * 5 * 2, 'ClickHouse lost some messages: {}'.format(result)
|
||||
|
||||
|
||||
@pytest.mark.timeout(420)
|
||||
def test_rabbitmq_headers_exchange(rabbitmq_cluster):
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.destination;
|
||||
CREATE TABLE test.destination(key UInt64, value UInt64,
|
||||
_consumed_by LowCardinality(String))
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY key;
|
||||
''')
|
||||
|
||||
num_tables_to_receive = 3
|
||||
for consumer_id in range(num_tables_to_receive):
|
||||
print("Setting up table {}".format(consumer_id))
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.headers_exchange_{0};
|
||||
DROP TABLE IF EXISTS test.headers_exchange_{0}_mv;
|
||||
CREATE TABLE test.headers_exchange_{0} (key UInt64, value UInt64)
|
||||
ENGINE = RabbitMQ
|
||||
SETTINGS rabbitmq_host_port = 'rabbitmq1:5672',
|
||||
rabbitmq_num_consumers = 4,
|
||||
rabbitmq_exchange_name = 'headers_exchange_testing',
|
||||
rabbitmq_exchange_type = 'headers',
|
||||
rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2020',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_row_delimiter = '\\n';
|
||||
CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS
|
||||
SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0};
|
||||
'''.format(consumer_id))
|
||||
|
||||
num_tables_to_ignore = 2
|
||||
for consumer_id in range(num_tables_to_ignore):
|
||||
print("Setting up table {}".format(consumer_id + num_tables_to_receive))
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.headers_exchange_{0};
|
||||
DROP TABLE IF EXISTS test.headers_exchange_{0}_mv;
|
||||
CREATE TABLE test.headers_exchange_{0} (key UInt64, value UInt64)
|
||||
ENGINE = RabbitMQ
|
||||
SETTINGS rabbitmq_host_port = 'rabbitmq1:5672',
|
||||
rabbitmq_exchange_name = 'headers_exchange_testing',
|
||||
rabbitmq_exchange_type = 'headers',
|
||||
rabbitmq_routing_key_list = 'x-match=all,format=logs,type=report,year=2019',
|
||||
rabbitmq_format = 'JSONEachRow',
|
||||
rabbitmq_row_delimiter = '\\n';
|
||||
CREATE MATERIALIZED VIEW test.headers_exchange_{0}_mv TO test.destination AS
|
||||
SELECT key, value, '{0}' as _consumed_by FROM test.headers_exchange_{0};
|
||||
'''.format(consumer_id + num_tables_to_receive))
|
||||
|
||||
i = [0]
|
||||
messages_num = 1000
|
||||
|
||||
credentials = pika.PlainCredentials('root', 'clickhouse')
|
||||
parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials)
|
||||
connection = pika.BlockingConnection(parameters)
|
||||
channel = connection.channel()
|
||||
channel.exchange_declare(exchange='headers_exchange_testing', exchange_type='headers')
|
||||
|
||||
messages = []
|
||||
for _ in range(messages_num):
|
||||
messages.append(json.dumps({'key': i[0], 'value': i[0]}))
|
||||
i[0] += 1
|
||||
|
||||
fields={}
|
||||
fields['format']='logs'
|
||||
fields['type']='report'
|
||||
fields['year']='2020'
|
||||
|
||||
key_num = 0
|
||||
for message in messages:
|
||||
channel.basic_publish(exchange='headers_exchange_testing', routing_key='',
|
||||
properties=pika.BasicProperties(headers=fields), body=message)
|
||||
|
||||
connection.close()
|
||||
|
||||
while True:
|
||||
result = instance.query('SELECT count() FROM test.destination')
|
||||
time.sleep(1)
|
||||
if int(result) == messages_num * num_tables_to_receive:
|
||||
break
|
||||
|
||||
for consumer_id in range(num_tables_to_receive):
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.direct_exchange_{0}_mv;
|
||||
DROP TABLE IF EXISTS test.direct_exchange_{0};
|
||||
'''.format(consumer_id))
|
||||
for consumer_id in range(num_tables_to_ignore):
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.direct_exchange_{0}_mv;
|
||||
DROP TABLE IF EXISTS test.direct_exchange_{0};
|
||||
'''.format(consumer_id + num_tables_to_receive))
|
||||
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.destination;
|
||||
''')
|
||||
|
||||
assert int(result) == messages_num * num_tables_to_receive, '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