Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
apache
pulsar
提交
f6ae797f
pulsar
项目概览
apache
/
pulsar
通知
129
Star
40
Fork
3
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Wiki
1
Wiki
分析
仓库
DevOps
项目成员
Pages
pulsar
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Pages
分析
分析
仓库分析
DevOps
Wiki
1
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
提交
前往新版Gitcode,体验更适合开发者的 AI 搜索 >>
提交
f6ae797f
编写于
10月 11, 2017
作者:
M
massakam
提交者:
Matteo Merli
10月 10, 2017
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Prevent message duplication when active consumer is changed (#807)
上级
34ae881a
变更
12
显示空白变更内容
内联
并排
Showing
12 changed file
with
186 addition
and
24 deletion
+186
-24
conf/broker.conf
conf/broker.conf
+3
-0
conf/standalone.conf
conf/standalone.conf
+3
-0
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
...n/java/org/apache/pulsar/broker/ServiceConfiguration.java
+10
-0
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java
.../persistent/PersistentDispatcherSingleActiveConsumer.java
+57
-17
pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
...pache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
+1
-0
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
...ache/pulsar/broker/service/PersistentFailoverE2ETest.java
+84
-0
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java
...java/org/apache/pulsar/broker/service/ReplicatorTest.java
+7
-0
pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java
...apache/pulsar/client/api/DispatcherBlockConsumerTest.java
+2
-2
pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
.../apache/pulsar/client/api/SimpleProducerConsumerTest.java
+1
-1
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
...pache/pulsar/client/impl/BrokerClientIntegrationTest.java
+12
-4
site/_data/config/broker.yaml
site/_data/config/broker.yaml
+3
-0
site/_data/config/standalone.yaml
site/_data/config/standalone.yaml
+3
-0
未找到文件。
conf/broker.conf
浏览文件 @
f6ae797f
...
...
@@ -70,6 +70,9 @@ brokerDeleteInactiveTopicsFrequencySeconds=60
# How frequently to proactively check and purge expired messages
messageExpiryCheckIntervalInMinutes
=
5
# How long to delay rewinding cursor and dispatching messages when active consumer is changed
activeConsumerFailoverDelayTimeMillis
=
1000
# Set the default behavior for message deduplication in the broker
# This can be overridden per-namespace. If enabled, broker will reject
# messages that were already stored in the topic
...
...
conf/standalone.conf
浏览文件 @
f6ae797f
...
...
@@ -63,6 +63,9 @@ brokerDeleteInactiveTopicsFrequencySeconds=60
# How frequently to proactively check and purge expired messages
messageExpiryCheckIntervalInMinutes
=
5
# How long to delay rewinding cursor and dispatching messages when active consumer is changed
activeConsumerFailoverDelayTimeMillis
=
1000
# Set the default behavior for message deduplication in the broker
# This can be overridden per-namespace. If enabled, broker will reject
# messages that were already stored in the topic
...
...
pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java
浏览文件 @
f6ae797f
...
...
@@ -78,6 +78,8 @@ public class ServiceConfiguration implements PulsarConfiguration {
private
long
brokerDeleteInactiveTopicsFrequencySeconds
=
60
;
// How frequently to proactively check and purge expired messages
private
int
messageExpiryCheckIntervalInMinutes
=
5
;
// How long to delay rewinding cursor and dispatching messages when active consumer is changed
private
int
activeConsumerFailoverDelayTimeMillis
=
1000
;
// Set the default behavior for message deduplication in the broker
// This can be overridden per-namespace. If enabled, broker will reject
...
...
@@ -521,6 +523,14 @@ public class ServiceConfiguration implements PulsarConfiguration {
this
.
brokerDeduplicationEnabled
=
brokerDeduplicationEnabled
;
}
public
int
getActiveConsumerFailoverDelayTimeMillis
()
{
return
activeConsumerFailoverDelayTimeMillis
;
}
public
void
setActiveConsumerFailoverDelayTimeMillis
(
int
activeConsumerFailoverDelayTimeMillis
)
{
this
.
activeConsumerFailoverDelayTimeMillis
=
activeConsumerFailoverDelayTimeMillis
;
}
public
boolean
isClientLibraryVersionCheckEnabled
()
{
return
clientLibraryVersionCheckEnabled
;
}
...
...
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java
浏览文件 @
f6ae797f
...
...
@@ -23,6 +23,7 @@ import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAG
import
java.util.List
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.ScheduledFuture
;
import
org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback
;
import
org.apache.bookkeeper.mledger.Entry
;
...
...
@@ -54,6 +55,7 @@ public final class PersistentDispatcherSingleActiveConsumer extends AbstractDisp
private
int
readBatchSize
;
private
final
Backoff
readFailureBackoff
=
new
Backoff
(
15
,
TimeUnit
.
SECONDS
,
1
,
TimeUnit
.
MINUTES
,
0
,
TimeUnit
.
MILLISECONDS
);
private
final
ServiceConfiguration
serviceConfig
;
private
ScheduledFuture
<?>
readOnActiveConsumerTask
=
null
;
public
PersistentDispatcherSingleActiveConsumer
(
ManagedCursor
cursor
,
SubType
subscriptionType
,
int
partitionIndex
,
PersistentTopic
topic
)
{
...
...
@@ -71,12 +73,38 @@ public final class PersistentDispatcherSingleActiveConsumer extends AbstractDisp
havePendingRead
=
false
;
}
// When a new consumer is chosen, start delivery from unacked message. If there is any pending read operation,
// let it finish and then rewind
if
(!
havePendingRead
)
{
if
(
havePendingRead
)
{
return
;
}
// When a new consumer is chosen, start delivery from unacked message.
// If there is any pending read operation, let it finish and then rewind
if
(
subscriptionType
!=
SubType
.
Failover
||
serviceConfig
.
getActiveConsumerFailoverDelayTimeMillis
()
<=
0
)
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}] Rewind cursor and read more entries without delay"
,
name
);
}
cursor
.
rewind
();
readMoreEntries
(
ACTIVE_CONSUMER_UPDATER
.
get
(
this
));
return
;
}
// If subscription type is Failover, delay rewinding cursor and
// reading more entries in order to prevent message duplication
if
(
readOnActiveConsumerTask
!=
null
)
{
return
;
}
readOnActiveConsumerTask
=
topic
.
getBrokerService
().
executor
().
schedule
(()
->
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}] Rewind cursor and read more entries after {} ms delay"
,
name
,
serviceConfig
.
getActiveConsumerFailoverDelayTimeMillis
());
}
cursor
.
rewind
();
readMoreEntries
(
ACTIVE_CONSUMER_UPDATER
.
get
(
this
));
readOnActiveConsumerTask
=
null
;
},
serviceConfig
.
getActiveConsumerFailoverDelayTimeMillis
(),
TimeUnit
.
MILLISECONDS
);
}
protected
void
cancelPendingRead
()
{
...
...
@@ -148,35 +176,47 @@ public final class PersistentDispatcherSingleActiveConsumer extends AbstractDisp
@Override
public
synchronized
void
consumerFlow
(
Consumer
consumer
,
int
additionalNumberOfMessages
)
{
if
(!
havePendingRead
)
{
if
(
ACTIVE_CONSUMER_UPDATER
.
get
(
this
)
==
consumer
)
{
if
(
havePendingRead
)
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}-{}] Trigger new read after receiving flow control message"
,
name
,
consumer
);
log
.
debug
(
"[{}-{}] Ignoring flow control message since we already have a pending read req"
,
name
,
consumer
);
}
readMoreEntries
(
consumer
);
}
else
{
}
else
if
(
ACTIVE_CONSUMER_UPDATER
.
get
(
this
)
!=
consumer
)
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}-{}] Ignoring flow control message since consumer is not active partition consumer"
,
name
,
consumer
);
log
.
debug
(
"[{}-{}] Ignoring flow control message since consumer is not active partition consumer"
,
name
,
consumer
);
}
}
else
if
(
readOnActiveConsumerTask
!=
null
)
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}-{}] Ignoring flow control message since consumer is waiting for cursor to be rewinded"
,
name
,
consumer
);
}
}
else
{
if
(
log
.
isDebugEnabled
())
{
log
.
debug
(
"[{}-{}]
Ignoring flow control message since we already have a pending read req
"
,
name
,
consumer
);
log
.
debug
(
"[{}-{}]
Trigger new read after receiving flow control message
"
,
name
,
consumer
);
}
readMoreEntries
(
consumer
);
}
}
@Override
public
synchronized
void
redeliverUnacknowledgedMessages
(
Consumer
consumer
)
{
if
(
consumer
!=
ACTIVE_CONSUMER_UPDATER
.
get
(
this
))
{
log
.
info
(
"[{}] Ignoring reDeliverUnAcknowledgedMessages: Only the active consumer can call resend"
,
consumer
);
log
.
info
(
"[{}
-{}
] Ignoring reDeliverUnAcknowledgedMessages: Only the active consumer can call resend"
,
name
,
consumer
);
return
;
}
if
(
readOnActiveConsumerTask
!=
null
)
{
log
.
info
(
"[{}-{}] Ignoring reDeliverUnAcknowledgedMessages: consumer is waiting for cursor to be rewinded"
,
name
,
consumer
);
return
;
}
if
(
havePendingRead
&&
cursor
.
cancelPendingReadRequest
())
{
havePendingRead
=
false
;
}
if
(!
havePendingRead
)
{
cursor
.
rewind
();
if
(
log
.
isDebugEnabled
())
{
...
...
pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java
浏览文件 @
f6ae797f
...
...
@@ -90,6 +90,7 @@ public abstract class MockedPulsarServiceBaseTest {
this
.
conf
.
setClusterName
(
"test"
);
this
.
conf
.
setAdvertisedAddress
(
"localhost"
);
// there are TLS tests in here, they need to use localhost because of the certificate
this
.
conf
.
setManagedLedgerCacheSizeMB
(
8
);
this
.
conf
.
setActiveConsumerFailoverDelayTimeMillis
(
0
);
}
protected
final
void
internalSetup
()
throws
Exception
{
...
...
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java
浏览文件 @
f6ae797f
...
...
@@ -433,4 +433,88 @@ public class PersistentFailoverE2ETest extends BrokerTestBase {
admin
.
persistentTopics
().
deletePartitionedTopic
(
topicName
);
}
@Test
public
void
testActiveConsumerFailoverWithDelay
()
throws
Exception
{
final
String
topicName
=
"persistent://prop/use/ns-abc/failover-topic3"
;
final
String
subName
=
"sub1"
;
final
int
numMsgs
=
100
;
List
<
Message
>
receivedMessages
=
Lists
.
newArrayList
();
ConsumerConfiguration
consumerConf1
=
new
ConsumerConfiguration
();
consumerConf1
.
setSubscriptionType
(
SubscriptionType
.
Failover
);
consumerConf1
.
setConsumerName
(
"1"
);
consumerConf1
.
setMessageListener
((
consumer
,
msg
)
->
{
try
{
synchronized
(
receivedMessages
)
{
receivedMessages
.
add
(
msg
);
}
consumer
.
acknowledge
(
msg
);
}
catch
(
Exception
e
)
{
fail
(
"Should not fail"
);
}
});
ConsumerConfiguration
consumerConf2
=
new
ConsumerConfiguration
();
consumerConf2
.
setSubscriptionType
(
SubscriptionType
.
Failover
);
consumerConf2
.
setConsumerName
(
"2"
);
consumerConf2
.
setMessageListener
((
consumer
,
msg
)
->
{
try
{
synchronized
(
receivedMessages
)
{
receivedMessages
.
add
(
msg
);
}
consumer
.
acknowledge
(
msg
);
}
catch
(
Exception
e
)
{
fail
(
"Should not fail"
);
}
});
conf
.
setActiveConsumerFailoverDelayTimeMillis
(
500
);
restartBroker
();
// create subscription
Consumer
consumer
=
pulsarClient
.
subscribe
(
topicName
,
subName
,
consumerConf1
);
consumer
.
close
();
PersistentTopic
topicRef
=
(
PersistentTopic
)
pulsar
.
getBrokerService
().
getTopicReference
(
topicName
);
PersistentSubscription
subRef
=
topicRef
.
getSubscription
(
subName
);
// enqueue messages
List
<
CompletableFuture
<
MessageId
>>
futures
=
Lists
.
newArrayListWithCapacity
(
numMsgs
);
Producer
producer
=
pulsarClient
.
createProducer
(
topicName
);
for
(
int
i
=
0
;
i
<
numMsgs
;
i
++)
{
String
message
=
"my-message-"
+
i
;
futures
.
add
(
producer
.
sendAsync
(
message
.
getBytes
()));
}
FutureUtil
.
waitForAll
(
futures
).
get
();
futures
.
clear
();
producer
.
close
();
// two consumers subscribe at almost the same time
CompletableFuture
<
Consumer
>
subscribeFuture2
=
pulsarClient
.
subscribeAsync
(
topicName
,
subName
,
consumerConf2
);
CompletableFuture
<
Consumer
>
subscribeFuture1
=
pulsarClient
.
subscribeAsync
(
topicName
,
subName
,
consumerConf1
);
// wait for all messages to be dequeued
int
retry
=
20
;
for
(
int
i
=
0
;
i
<
retry
;
i
++)
{
if
(
receivedMessages
.
size
()
>=
numMsgs
&&
subRef
.
getNumberOfEntriesInBacklog
()
==
0
)
{
break
;
}
else
if
(
i
!=
retry
-
1
)
{
Thread
.
sleep
(
100
);
}
}
// check if message duplication has occurred
assertEquals
(
receivedMessages
.
size
(),
numMsgs
);
assertEquals
(
subRef
.
getNumberOfEntriesInBacklog
(),
0
);
for
(
int
i
=
0
;
i
<
receivedMessages
.
size
();
i
++)
{
Assert
.
assertNotNull
(
receivedMessages
.
get
(
i
));
Assert
.
assertEquals
(
new
String
(
receivedMessages
.
get
(
i
).
getData
()),
"my-message-"
+
i
);
}
subscribeFuture1
.
get
().
close
();
subscribeFuture2
.
get
().
unsubscribe
();
admin
.
persistentTopics
().
delete
(
topicName
);
resetConfig
();
restartBroker
();
}
}
pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTest.java
浏览文件 @
f6ae797f
...
...
@@ -739,6 +739,13 @@ public class ReplicatorTest extends ReplicatorTestBase {
// Wait until the 2nd message got delivered to consumer
consumer2
.
receive
(
1
);
int
retry
=
10
;
for
(
int
i
=
0
;
i
<
retry
&&
replicator
.
getStats
().
replicationBacklog
>
0
;
i
++)
{
if
(
i
!=
retry
-
1
)
{
Thread
.
sleep
(
100
);
}
}
assertEquals
(
replicator
.
getStats
().
replicationBacklog
,
0
);
producer1
.
close
();
...
...
pulsar-broker/src/test/java/org/apache/pulsar/client/api/DispatcherBlockConsumerTest.java
浏览文件 @
f6ae797f
...
...
@@ -633,7 +633,7 @@ public class DispatcherBlockConsumerTest extends ProducerConsumerBase {
Set
<
Integer
>
unackMessages
=
Sets
.
newHashSet
(
5
,
10
,
20
,
21
,
22
,
23
,
25
,
26
,
30
,
32
,
40
,
80
,
160
,
320
);
int
receivedMsgCount
=
0
;
for
(
int
i
=
0
;
i
<
totalProducedMsgs
;
i
++)
{
Message
msg
=
consumer
.
receive
(
1
00
,
TimeUnit
.
MILLISECONDS
);
Message
msg
=
consumer
.
receive
(
5
00
,
TimeUnit
.
MILLISECONDS
);
if
(!
unackMessages
.
contains
(
i
))
{
consumer
.
acknowledge
(
msg
);
}
...
...
pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerTest.java
浏览文件 @
f6ae797f
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java
浏览文件 @
f6ae797f
...
...
@@ -394,6 +394,10 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
final
NavigableMap
<
Long
,
TimestampEntryCount
>
publishTimeIdMap
=
new
ConcurrentSkipListMap
<>();
// set delay time to start dispatching messages to active consumer in order to avoid message duplication
conf
.
setActiveConsumerFailoverDelayTimeMillis
(
500
);
restartBroker
();
consConfig
.
setSubscriptionType
(
subType
);
consConfig
.
setMessageListener
((
MessageListener
)
(
Consumer
consumer
,
Message
msg
)
->
{
try
{
...
...
@@ -413,7 +417,8 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
admin
.
namespaces
().
setRetention
(
destName
.
getNamespace
(),
policy
);
Consumer
consumer
=
pulsarClient
.
subscribe
(
destName
.
toString
(),
subsId
,
consConfig
);
Consumer
consumer1
=
pulsarClient
.
subscribe
(
destName
.
toString
(),
subsId
,
consConfig
);
Consumer
consumer2
=
pulsarClient
.
subscribe
(
destName
.
toString
(),
subsId
,
consConfig
);
final
Producer
producer
=
pulsarClient
.
createProducer
(
destName
.
toString
());
log
.
info
(
"warm up started for "
+
destName
.
toString
());
...
...
@@ -426,7 +431,7 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
// sleep to ensure receiving of msgs
for
(
int
n
=
0
;
n
<
10
&&
received
.
size
()
<
warmup
;
n
++)
{
Thread
.
sleep
(
1
00
);
Thread
.
sleep
(
2
00
);
}
// validate received msgs
...
...
@@ -465,7 +470,6 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
Assert
.
assertTrue
(
subList
.
contains
(
subsId
));
admin
.
persistentTopics
().
resetCursor
(
destName
.
toString
(),
subsId
,
timestamp
);
consumer
=
pulsarClient
.
subscribe
(
destName
.
toString
(),
subsId
,
consConfig
);
Thread
.
sleep
(
3000
);
int
totalExpected
=
0
;
for
(
TimestampEntryCount
tec
:
expectedMessages
.
values
())
{
...
...
@@ -473,7 +477,8 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
}
// validate that replay happens after the timestamp
Assert
.
assertTrue
(
publishTimeIdMap
.
firstEntry
().
getKey
()
>=
timestamp
);
consumer
.
close
();
consumer1
.
close
();
consumer2
.
close
();
producer
.
close
();
// validate that expected and received counts match
int
totalReceived
=
0
;
...
...
@@ -481,6 +486,9 @@ public class BrokerClientIntegrationTest extends ProducerConsumerBase {
totalReceived
+=
tec
.
numMessages
;
}
Assert
.
assertEquals
(
totalReceived
,
totalExpected
,
"did not receive all messages on replay after reset"
);
resetConfig
();
restartBroker
();
}
/**
...
...
site/_data/config/broker.yaml
浏览文件 @
f6ae797f
...
...
@@ -72,6 +72,9 @@ configs:
-
name
:
messageExpiryCheckIntervalInMinutes
default
:
'
5'
description
:
How frequently to proactively check and purge expired messages
-
name
:
activeConsumerFailoverDelayTimeMillis
default
:
'
1000'
description
:
How long to delay rewinding cursor and dispatching messages when active consumer is changed.
-
name
:
clientLibraryVersionCheckEnabled
default
:
'
false'
description
:
Enable check for minimum allowed client library version
...
...
site/_data/config/standalone.yaml
浏览文件 @
f6ae797f
...
...
@@ -63,6 +63,9 @@ configs:
-
name
:
messageExpiryCheckIntervalInMinutes
default
:
'
5'
description
:
How often to proactively check and purged expired messages.
-
name
:
activeConsumerFailoverDelayTimeMillis
default
:
'
1000'
description
:
How long to delay rewinding cursor and dispatching messages when active consumer is changed.
-
name
:
clientLibraryVersionCheckEnabled
default
:
'
false'
description
:
Enable checks for minimum allowed client library version.
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录