Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
2dot5
ClickHouse
提交
626eb53b
C
ClickHouse
项目概览
2dot5
/
ClickHouse
通知
3
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
DevOps
流水线
流水线任务
计划
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
C
ClickHouse
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
DevOps
DevOps
流水线
流水线任务
计划
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
流水线任务
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
提交
626eb53b
编写于
6月 11, 2020
作者:
K
kssenii
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Fix multiple bindings for single queue & rm hardcoded strings
上级
9350472e
变更
3
隐藏空白更改
内联
并排
Showing
3 changed file
with
65 addition
and
31 deletion
+65
-31
src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp
src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp
+32
-16
src/Storages/RabbitMQ/StorageRabbitMQ.cpp
src/Storages/RabbitMQ/StorageRabbitMQ.cpp
+1
-1
tests/integration/test_storage_rabbitmq/test.py
tests/integration/test_storage_rabbitmq/test.py
+32
-14
未找到文件。
src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp
浏览文件 @
626eb53b
...
...
@@ -14,6 +14,17 @@ namespace DB
{
namespace
Exchange
{
/// Note that default here means default by implementation and not by rabbitmq settings
static
const
String
DEFAULT
=
"default"
;
static
const
String
FANOUT
=
"fanout"
;
static
const
String
DIRECT
=
"direct"
;
static
const
String
TOPIC
=
"topic"
;
static
const
String
HASH
=
"consistent_hash"
;
}
ReadBufferFromRabbitMQConsumer
::
ReadBufferFromRabbitMQConsumer
(
ChannelPtr
consumer_channel_
,
RabbitMQHandler
&
eventHandler_
,
...
...
@@ -44,7 +55,7 @@ ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer(
messages
.
clear
();
current
=
messages
.
begin
();
exchange_type_set
=
exchange_type
!=
"default"
?
true
:
false
;
exchange_type_set
=
exchange_type
!=
Exchange
::
DEFAULT
?
true
:
false
;
/* One queue per consumer can handle up to 50000 messages. More queues per consumer can be added.
* By default there is one queue per consumer.
...
...
@@ -72,14 +83,14 @@ void ReadBufferFromRabbitMQConsumer::initExchange()
/* If exchange_type is not set - then direct-exchange is used - this type of exchange is the fastest (also due to different
* binding algorithm this default behaviuor is much faster). It is also used in INSERT query.
*/
String
producer_exchange
=
exchange_type_set
?
exchange_name
+
"_
default"
:
exchange_name
;
String
producer_exchange
=
exchange_type_set
?
exchange_name
+
"_
"
+
Exchange
::
DEFAULT
:
exchange_name
;
consumer_channel
->
declareExchange
(
producer_exchange
,
AMQP
::
fanout
).
onError
([
&
](
const
char
*
message
)
{
internal_exchange_declared
=
false
;
LOG_ERROR
(
log
,
"Failed to declare exchange: {}"
,
message
);
});
internal_exchange_name
=
producer_exchange
+
"_
direct"
;
internal_exchange_name
=
producer_exchange
+
"_
"
+
Exchange
::
DIRECT
;
consumer_channel
->
declareExchange
(
internal_exchange_name
,
AMQP
::
direct
).
onError
([
&
](
const
char
*
message
)
{
internal_exchange_declared
=
false
;
...
...
@@ -99,11 +110,11 @@ void ReadBufferFromRabbitMQConsumer::initExchange()
/// For special purposes to use the flexibility of routing provided by rabbitmq - choosing exchange types is supported.
AMQP
::
ExchangeType
type
;
if
(
exchange_type
==
"fanout"
)
type
=
AMQP
::
ExchangeType
::
fanout
;
else
if
(
exchange_type
==
"direct"
)
type
=
AMQP
::
ExchangeType
::
direct
;
else
if
(
exchange_type
==
"topic"
)
type
=
AMQP
::
ExchangeType
::
topic
;
else
if
(
exchange_type
==
"consistent_hash"
)
type
=
AMQP
::
ExchangeType
::
consistent_hash
;
else
return
;
if
(
exchange_type
==
Exchange
::
FANOUT
)
type
=
AMQP
::
ExchangeType
::
fanout
;
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
return
;
/* Declare exchange of the specified type and bind it to hash-exchange, which will evenly distribute messages
* between all consumers. (This enables better scaling as without hash-echange - the only oprion to avoid getting the same
...
...
@@ -115,12 +126,12 @@ void ReadBufferFromRabbitMQConsumer::initExchange()
LOG_ERROR
(
log
,
"Failed to declare {} exchange: {}"
,
exchange_type
,
message
);
});
hash_exchange
=
true
;
/// No need for declaring hash-exchange if there is only one consumer with one queue and exchange type is not hash
if
(
!
bind_by_id
&&
exchange_type
!=
"consistent_hash"
)
if
(
!
bind_by_id
&&
exchange_type
!=
Exchange
::
HASH
)
return
;
hash_exchange
=
true
;
AMQP
::
Table
exchange_arguments
;
exchange_arguments
[
"hash-property"
]
=
"message_id"
;
...
...
@@ -153,6 +164,10 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
internal_exchange_declared
=
true
;
}
/* Internal exchange is a default exchange (by implementstion, not by rabbitmq settings) and is used for INSERT query
* and if exchange_type is not set - there is no local exchange. If it is set - then local exchange is a distributor
* exchange, which is bound to the exchange specified by the client.
*/
bool
internal_bindings_created
=
false
,
internal_bindings_error
=
false
;
bool
local_bindings_created
=
false
,
local_bindings_error
=
false
;
...
...
@@ -188,7 +203,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
.
onError
([
&
](
const
char
*
message
)
{
internal_bindings_error
=
true
;
LOG_ERROR
(
log
,
"Failed to bind to key {}
, the reason is
: {}"
,
binding_key
,
message
);
LOG_ERROR
(
log
,
"Failed to bind to key {}
. Reason
: {}"
,
binding_key
,
message
);
});
/// Must be done here and not in readPrefix() because library might fail to handle async subscription on the same connection
...
...
@@ -209,15 +224,16 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
.
onError
([
&
](
const
char
*
message
)
{
local_bindings_error
=
true
;
LOG_ERROR
(
log
,
"Failed to create queue binding
: {}"
,
message
);
LOG_ERROR
(
log
,
"Failed to create queue binding
to key {}. Reason: {}"
,
binding_key
,
message
);
});
}
else
{
///
m
eans there is only one queue with one consumer - no even distribution needed - no hash-exchange
///
M
eans there is only one queue with one consumer - no even distribution needed - no hash-exchange
for
(
auto
&
routing_key
:
routing_keys
)
{
consumer_channel
->
bindQueue
(
local_exchange_name
,
queue_name_
,
routing_key
)
/// Binding directly to exchange, specified by the client
consumer_channel
->
bindQueue
(
exchange_name
,
queue_name_
,
routing_key
)
.
onSuccess
([
&
]
{
local_bindings_created
=
true
;
...
...
@@ -225,7 +241,7 @@ void ReadBufferFromRabbitMQConsumer::initQueueBindings(const size_t queue_id)
.
onError
([
&
](
const
char
*
message
)
{
local_bindings_error
=
true
;
LOG_ERROR
(
log
,
"Failed to create queue binding
: {}"
,
message
);
LOG_ERROR
(
log
,
"Failed to create queue binding
to key {}. Reason: {}"
,
routing_key
,
message
);
});
}
}
...
...
src/Storages/RabbitMQ/StorageRabbitMQ.cpp
浏览文件 @
626eb53b
...
...
@@ -213,7 +213,7 @@ ConsumerBufferPtr StorageRabbitMQ::createReadBuffer()
ChannelPtr
consumer_channel
=
std
::
make_shared
<
AMQP
::
TcpChannel
>
(
&
connection
);
auto
table_id
=
getStorageID
();
String
table_name
=
table_id
.
getNameForLogs
();
String
table_name
=
table_id
.
getNameForLogs
();
return
std
::
make_shared
<
ReadBufferFromRabbitMQConsumer
>
(
consumer_channel
,
eventHandler
,
exchange_name
,
routing_keys
,
next_channel_id
,
log
,
row_delimiter
,
bind_by_id
,
num_queues
,
exchange_type
,
table_name
,
stream_cancelled
);
...
...
tests/integration/test_storage_rabbitmq/test.py
浏览文件 @
626eb53b
...
...
@@ -927,7 +927,6 @@ def test_rabbitmq_sharding_between_channels_insert(rabbitmq_cluster):
while
True
:
result
=
instance
.
query
(
'SELECT count() FROM test.view_sharding'
)
time
.
sleep
(
1
)
print
result
if
int
(
result
)
==
messages_num
*
threads_num
:
break
...
...
@@ -1288,9 +1287,17 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster):
@
pytest
.
mark
.
timeout
(
420
)
def
test_rabbitmq_multiple_bindings
(
rabbitmq_cluster
):
instance
.
query
(
'''
DROP TABLE IF EXISTS test.bindings;
DROP TABLE IF EXISTS test.bindings_mv;
CREATE TABLE test.bindings (key UInt64, value UInt64)
DROP TABLE IF EXISTS test.destination;
CREATE TABLE test.destination(key UInt64, value UInt64,
_consumed_by LowCardinality(String))
ENGINE = MergeTree()
ORDER BY key;
'''
)
instance
.
query
(
'''
DROP TABLE IF EXISTS test.bindings_1;
DROP TABLE IF EXISTS test.bindings_1_mv;
CREATE TABLE test.bindings_1 (key UInt64, value UInt64)
ENGINE = RabbitMQ
SETTINGS rabbitmq_host_port = 'rabbitmq1:5672',
rabbitmq_num_consumers = 5,
...
...
@@ -1300,13 +1307,25 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster):
rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5',
rabbitmq_format = 'JSONEachRow',
rabbitmq_row_delimiter = '
\\
n';
CREATE TABLE test.view_bindings (key UInt64, value UInt64)
ENGINE = MergeTree
ORDER BY key;
CREATE MATERIALIZED VIEW test.bindings_mv TO test.view_bindings AS
SELECT * FROM test.bindings;
CREATE MATERIALIZED VIEW test.bindings_1_mv TO test.destination AS
SELECT * FROM test.bindings_1;
'''
)
# in case num_consumers and num_queues are not set - multiple bindings are implemented differently, so test them too
instance
.
query
(
'''
DROP TABLE IF EXISTS test.bindings_2;
DROP TABLE IF EXISTS test.bindings_2_mv;
CREATE TABLE test.bindings_2 (key UInt64, value UInt64)
ENGINE = RabbitMQ
SETTINGS rabbitmq_host_port = 'rabbitmq1:5672',
rabbitmq_exchange_name = 'multiple_bindings_testing',
rabbitmq_exchange_type = 'direct',
rabbitmq_routing_key_list = 'key1,key2,key3,key4,key5',
rabbitmq_format = 'JSONEachRow',
rabbitmq_row_delimiter = '
\\
n';
CREATE MATERIALIZED VIEW test.bindings_2_mv TO test.destination AS
SELECT * FROM test.bindings_2;
'''
)
i
=
[
0
]
messages_num
=
500
...
...
@@ -1318,7 +1337,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster):
# init connection here because otherwise python rabbitmq client might fail
connection
=
pika
.
BlockingConnection
(
parameters
)
channel
=
connection
.
channel
()
channel
.
exchange_declare
(
exchange
=
'
hash_exchange_testing'
,
exchange_type
=
'x-consistent-hash
'
)
channel
.
exchange_declare
(
exchange
=
'
multiple_bindings_testing'
,
exchange_type
=
'direct
'
)
messages
=
[]
for
_
in
range
(
messages_num
):
...
...
@@ -1343,16 +1362,15 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster):
thread
.
start
()
while
True
:
result
=
instance
.
query
(
'SELECT count() FROM test.
view_bindings
'
)
result
=
instance
.
query
(
'SELECT count() FROM test.
destination
'
)
time
.
sleep
(
1
)
print
result
if
int
(
result
)
==
messages_num
*
threads_num
*
5
:
if
int
(
result
)
==
messages_num
*
threads_num
*
5
*
2
:
break
for
thread
in
threads
:
thread
.
join
()
assert
int
(
result
)
==
messages_num
*
threads_num
*
5
,
'ClickHouse lost some messages: {}'
.
format
(
result
)
assert
int
(
result
)
==
messages_num
*
threads_num
*
5
*
2
,
'ClickHouse lost some messages: {}'
.
format
(
result
)
if
__name__
==
'__main__'
:
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录