Support headers-exchange type

This commit is contained in:
kssenii 2020-06-13 21:37:37 +00:00
parent 9c49398728
commit dcd7b7351c
3 changed files with 152 additions and 12 deletions

View File

@ -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.

View File

@ -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;

View File

@ -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...")