Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
Apache RocketMQ
Rocketmq
提交
37b6dd02
R
Rocketmq
项目概览
Apache RocketMQ
/
Rocketmq
上一次同步 3 年多
通知
275
Star
16140
Fork
68
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
DevOps
流水线
流水线任务
计划
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
R
Rocketmq
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
DevOps
DevOps
流水线
流水线任务
计划
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
流水线任务
提交
Issue看板
提交
37b6dd02
编写于
12月 27, 2018
作者:
C
chengxiangwang
浏览文件
操作
浏览文件
下载
差异文件
Merge branch 'develop' of
https://github.com/apache/rocketmq
上级
d4d02b5e
d45fda46
变更
16
隐藏空白更改
内联
并排
Showing
16 changed file
with
293 addition
and
99 deletion
+293
-99
.gitignore
.gitignore
+2
-1
broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java
...er/transaction/queue/TransactionalMessageServiceImpl.java
+15
-22
client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java
...pache/rocketmq/client/consumer/DefaultMQPullConsumer.java
+26
-0
client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java
...a/org/apache/rocketmq/client/consumer/MQPullConsumer.java
+47
-0
client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
...ketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
+73
-25
client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
.../apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
+0
-28
client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
...apache/rocketmq/client/impl/factory/MQClientInstance.java
+13
-0
client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
.../rocketmq/client/impl/producer/DefaultMQProducerImpl.java
+31
-5
client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java
...rg/apache/rocketmq/client/producer/DefaultMQProducer.java
+10
-0
client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java
...pache/rocketmq/client/producer/DefaultMQProducerTest.java
+1
-12
common/src/main/java/org/apache/rocketmq/common/constant/DBMsgConstants.java
...a/org/apache/rocketmq/common/constant/DBMsgConstants.java
+1
-1
docs/cn/index.md
docs/cn/index.md
+0
-0
docs/en/index.md
docs/en/index.md
+0
-0
store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
.../org/apache/rocketmq/store/config/MessageStoreConfig.java
+1
-1
test/src/test/java/org/apache/rocketmq/test/client/consumer/filter/SqlFilterIT.java
...che/rocketmq/test/client/consumer/filter/SqlFilterIT.java
+72
-3
tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
...mq/tools/command/consumer/ConsumerProgressSubCommand.java
+1
-1
未找到文件。
.gitignore
浏览文件 @
37b6dd02
...
...
@@ -10,4 +10,5 @@ devenv
*.versionsBackup
!NOTICE-BIN
!LICENSE-BIN
.DS_Store
\ No newline at end of file
.DS_Store
localbin
broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java
浏览文件 @
37b6dd02
...
...
@@ -198,7 +198,7 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
if
(
null
!=
checkImmunityTimeStr
)
{
checkImmunityTime
=
getImmunityTime
(
checkImmunityTimeStr
,
transactionTimeout
);
if
(
valueOfCurrentMinusBorn
<
checkImmunityTime
)
{
if
(
checkPrepareQueueOffset
(
removeMap
,
doneOpOffset
,
msgExt
,
checkImmunityTime
))
{
if
(
checkPrepareQueueOffset
(
removeMap
,
doneOpOffset
,
msgExt
))
{
newOffset
=
i
+
1
;
i
++;
continue
;
...
...
@@ -315,33 +315,26 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
* @param removeMap Op message map to determine whether a half message was responded by producer.
* @param doneOpOffset Op Message which has been checked.
* @param msgExt Half message
* @param checkImmunityTime User defined time to avoid being detected early.
* @return Return true if put success, otherwise return false.
*/
private
boolean
checkPrepareQueueOffset
(
HashMap
<
Long
,
Long
>
removeMap
,
List
<
Long
>
doneOpOffset
,
MessageExt
msgExt
,
long
checkImmunityTime
)
{
if
(
System
.
currentTimeMillis
()
-
msgExt
.
getBornTimestamp
()
<
checkImmunityTime
)
{
String
prepareQueueOffsetStr
=
msgExt
.
getUserProperty
(
MessageConst
.
PROPERTY_TRANSACTION_PREPARED_QUEUE_OFFSET
);
if
(
null
==
prepareQueueOffsetStr
)
{
return
putImmunityMsgBackToHalfQueue
(
msgExt
);
private
boolean
checkPrepareQueueOffset
(
HashMap
<
Long
,
Long
>
removeMap
,
List
<
Long
>
doneOpOffset
,
MessageExt
msgExt
)
{
String
prepareQueueOffsetStr
=
msgExt
.
getUserProperty
(
MessageConst
.
PROPERTY_TRANSACTION_PREPARED_QUEUE_OFFSET
);
if
(
null
==
prepareQueueOffsetStr
)
{
return
putImmunityMsgBackToHalfQueue
(
msgExt
);
}
else
{
long
prepareQueueOffset
=
getLong
(
prepareQueueOffsetStr
);
if
(-
1
==
prepareQueueOffset
)
{
return
false
;
}
else
{
long
prepareQueueOffset
=
getLong
(
prepareQueueOffsetStr
);
if
(-
1
==
prepareQueueOffset
)
{
return
false
;
if
(
removeMap
.
containsKey
(
prepareQueueOffset
))
{
long
tmpOpOffset
=
removeMap
.
remove
(
prepareQueueOffset
);
doneOpOffset
.
add
(
tmpOpOffset
);
return
true
;
}
else
{
if
(
removeMap
.
containsKey
(
prepareQueueOffset
))
{
long
tmpOpOffset
=
removeMap
.
remove
(
prepareQueueOffset
);
doneOpOffset
.
add
(
tmpOpOffset
);
return
true
;
}
else
{
return
putImmunityMsgBackToHalfQueue
(
msgExt
);
}
return
putImmunityMsgBackToHalfQueue
(
msgExt
);
}
}
}
else
{
return
true
;
}
}
...
...
client/src/main/java/org/apache/rocketmq/client/consumer/DefaultMQPullConsumer.java
浏览文件 @
37b6dd02
...
...
@@ -257,6 +257,18 @@ public class DefaultMQPullConsumer extends ClientConfig implements MQPullConsume
return
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
subExpression
,
offset
,
maxNums
,
timeout
);
}
@Override
public
PullResult
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
messageSelector
,
offset
,
maxNums
);
}
@Override
public
PullResult
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
messageSelector
,
offset
,
maxNums
,
timeout
);
}
@Override
public
void
pull
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
...
...
@@ -270,6 +282,20 @@ public class DefaultMQPullConsumer extends ClientConfig implements MQPullConsume
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
subExpression
,
offset
,
maxNums
,
pullCallback
,
timeout
);
}
@Override
public
void
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
messageSelector
,
offset
,
maxNums
,
pullCallback
);
}
@Override
public
void
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
this
.
defaultMQPullConsumerImpl
.
pull
(
mq
,
messageSelector
,
offset
,
maxNums
,
pullCallback
,
timeout
);
}
@Override
public
PullResult
pullBlockIfNotFound
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
...
...
client/src/main/java/org/apache/rocketmq/client/consumer/MQPullConsumer.java
浏览文件 @
37b6dd02
...
...
@@ -66,6 +66,39 @@ public interface MQPullConsumer extends MQConsumer {
final
int
maxNums
,
final
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
;
/**
* Pulling the messages, not blocking
* <p>
* support other message selection, such as {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92}
* </p>
*
* @param mq from which message queue
* @param selector message selector({@link MessageSelector}), can be null.
* @param offset from where to pull
* @param maxNums max pulling numbers
* @return The resulting {@code PullRequest}
*/
PullResult
pull
(
final
MessageQueue
mq
,
final
MessageSelector
selector
,
final
long
offset
,
final
int
maxNums
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
;
/**
* Pulling the messages in the specified timeout
* <p>
* support other message selection, such as {@link org.apache.rocketmq.common.filter.ExpressionType#SQL92}
* </p>
*
* @param mq from which message queue
* @param selector message selector({@link MessageSelector}), can be null.
* @param offset from where to pull
* @param maxNums max pulling numbers
* @param timeout Pulling the messages in the specified timeout
* @return The resulting {@code PullRequest}
*/
PullResult
pull
(
final
MessageQueue
mq
,
final
MessageSelector
selector
,
final
long
offset
,
final
int
maxNums
,
final
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
;
/**
* Pulling the messages in a async. way
*/
...
...
@@ -80,6 +113,20 @@ public interface MQPullConsumer extends MQConsumer {
final
PullCallback
pullCallback
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
;
/**
* Pulling the messages in a async. way. Support message selection
*/
void
pull
(
final
MessageQueue
mq
,
final
MessageSelector
selector
,
final
long
offset
,
final
int
maxNums
,
final
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
InterruptedException
;
/**
* Pulling the messages in a async. way. Support message selection
*/
void
pull
(
final
MessageQueue
mq
,
final
MessageSelector
selector
,
final
long
offset
,
final
int
maxNums
,
final
PullCallback
pullCallback
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
;
/**
* Pulling the messages,if no message arrival,blocking some time
*
...
...
client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultMQPullConsumerImpl.java
浏览文件 @
37b6dd02
...
...
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentMap;
import
org.apache.rocketmq.client.QueryResult
;
import
org.apache.rocketmq.client.Validators
;
import
org.apache.rocketmq.client.consumer.DefaultMQPullConsumer
;
import
org.apache.rocketmq.client.consumer.MessageSelector
;
import
org.apache.rocketmq.client.consumer.PullCallback
;
import
org.apache.rocketmq.client.consumer.PullResult
;
import
org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus
;
...
...
@@ -46,6 +47,7 @@ import org.apache.rocketmq.common.MixAll;
import
org.apache.rocketmq.common.ServiceState
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.consumer.ConsumeFromWhere
;
import
org.apache.rocketmq.common.filter.ExpressionType
;
import
org.apache.rocketmq.common.filter.FilterAPI
;
import
org.apache.rocketmq.common.help.FAQUrl
;
import
org.apache.rocketmq.logging.InternalLogger
;
...
...
@@ -158,17 +160,58 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
public
PullResult
pull
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
this
.
pullSyncImpl
(
mq
,
subExpression
,
offset
,
maxNums
,
false
,
timeout
);
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
subExpression
);
return
this
.
pullSyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
false
,
timeout
);
}
private
PullResult
pullSyncImpl
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
,
boolean
block
,
public
PullResult
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
pull
(
mq
,
messageSelector
,
offset
,
maxNums
,
this
.
defaultMQPullConsumer
.
getConsumerPullTimeoutMillis
());
}
public
PullResult
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
messageSelector
);
return
this
.
pullSyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
false
,
timeout
);
}
private
SubscriptionData
getSubscriptionData
(
MessageQueue
mq
,
String
subExpression
)
throws
MQClientException
{
if
(
null
==
mq
)
{
throw
new
MQClientException
(
"mq is null"
,
null
);
}
try
{
return
FilterAPI
.
buildSubscriptionData
(
this
.
defaultMQPullConsumer
.
getConsumerGroup
(),
mq
.
getTopic
(),
subExpression
);
}
catch
(
Exception
e
)
{
throw
new
MQClientException
(
"parse subscription error"
,
e
);
}
}
private
SubscriptionData
getSubscriptionData
(
MessageQueue
mq
,
MessageSelector
messageSelector
)
throws
MQClientException
{
if
(
null
==
mq
)
{
throw
new
MQClientException
(
"mq is null"
,
null
);
}
try
{
return
FilterAPI
.
build
(
mq
.
getTopic
(),
messageSelector
.
getExpression
(),
messageSelector
.
getExpressionType
());
}
catch
(
Exception
e
)
{
throw
new
MQClientException
(
"parse subscription error"
,
e
);
}
}
private
PullResult
pullSyncImpl
(
MessageQueue
mq
,
SubscriptionData
subscriptionData
,
long
offset
,
int
maxNums
,
boolean
block
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
this
.
makeSureStateOK
();
if
(
null
==
mq
)
{
throw
new
MQClientException
(
"mq is null"
,
null
);
}
if
(
offset
<
0
)
{
...
...
@@ -183,20 +226,14 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
int
sysFlag
=
PullSysFlag
.
buildSysFlag
(
false
,
block
,
true
,
false
);
SubscriptionData
subscriptionData
;
try
{
subscriptionData
=
FilterAPI
.
buildSubscriptionData
(
this
.
defaultMQPullConsumer
.
getConsumerGroup
(),
mq
.
getTopic
(),
subExpression
);
}
catch
(
Exception
e
)
{
throw
new
MQClientException
(
"parse subscription error"
,
e
);
}
long
timeoutMillis
=
block
?
this
.
defaultMQPullConsumer
.
getConsumerTimeoutMillisWhenSuspend
()
:
timeout
;
boolean
isTagType
=
ExpressionType
.
isTagType
(
subscriptionData
.
getExpressionType
());
PullResult
pullResult
=
this
.
pullAPIWrapper
.
pullKernelImpl
(
mq
,
subscriptionData
.
getSubString
(),
0L
,
subscriptionData
.
getExpressionType
(),
isTagType
?
0L
:
subscriptionData
.
getSubVersion
(),
offset
,
maxNums
,
sysFlag
,
...
...
@@ -369,12 +406,27 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
public
void
pull
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
this
.
pullAsyncImpl
(
mq
,
subExpression
,
offset
,
maxNums
,
pullCallback
,
false
,
timeout
);
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
subExpression
);
this
.
pullAsyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
pullCallback
,
false
,
timeout
);
}
public
void
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
pull
(
mq
,
messageSelector
,
offset
,
maxNums
,
pullCallback
,
this
.
defaultMQPullConsumer
.
getConsumerPullTimeoutMillis
());
}
public
void
pull
(
MessageQueue
mq
,
MessageSelector
messageSelector
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
,
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
messageSelector
);
this
.
pullAsyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
pullCallback
,
false
,
timeout
);
}
private
void
pullAsyncImpl
(
final
MessageQueue
mq
,
final
S
tring
subExpression
,
final
S
ubscriptionData
subscriptionData
,
final
long
offset
,
final
int
maxNums
,
final
PullCallback
pullCallback
,
...
...
@@ -403,20 +455,14 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
try
{
int
sysFlag
=
PullSysFlag
.
buildSysFlag
(
false
,
block
,
true
,
false
);
final
SubscriptionData
subscriptionData
;
try
{
subscriptionData
=
FilterAPI
.
buildSubscriptionData
(
this
.
defaultMQPullConsumer
.
getConsumerGroup
(),
mq
.
getTopic
(),
subExpression
);
}
catch
(
Exception
e
)
{
throw
new
MQClientException
(
"parse subscription error"
,
e
);
}
long
timeoutMillis
=
block
?
this
.
defaultMQPullConsumer
.
getConsumerTimeoutMillisWhenSuspend
()
:
timeout
;
boolean
isTagType
=
ExpressionType
.
isTagType
(
subscriptionData
.
getExpressionType
());
this
.
pullAPIWrapper
.
pullKernelImpl
(
mq
,
subscriptionData
.
getSubString
(),
0L
,
subscriptionData
.
getExpressionType
(),
isTagType
?
0L
:
subscriptionData
.
getSubVersion
(),
offset
,
maxNums
,
sysFlag
,
...
...
@@ -444,7 +490,8 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
public
PullResult
pullBlockIfNotFound
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
this
.
pullSyncImpl
(
mq
,
subExpression
,
offset
,
maxNums
,
true
,
this
.
getDefaultMQPullConsumer
().
getConsumerPullTimeoutMillis
());
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
subExpression
);
return
this
.
pullSyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
true
,
this
.
getDefaultMQPullConsumer
().
getConsumerPullTimeoutMillis
());
}
public
DefaultMQPullConsumer
getDefaultMQPullConsumer
()
{
...
...
@@ -454,7 +501,8 @@ public class DefaultMQPullConsumerImpl implements MQConsumerInner {
public
void
pullBlockIfNotFound
(
MessageQueue
mq
,
String
subExpression
,
long
offset
,
int
maxNums
,
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
this
.
pullAsyncImpl
(
mq
,
subExpression
,
offset
,
maxNums
,
pullCallback
,
true
,
SubscriptionData
subscriptionData
=
getSubscriptionData
(
mq
,
subExpression
);
this
.
pullAsyncImpl
(
mq
,
subscriptionData
,
offset
,
maxNums
,
pullCallback
,
true
,
this
.
getDefaultMQPullConsumer
().
getConsumerPullTimeoutMillis
());
}
...
...
client/src/main/java/org/apache/rocketmq/client/impl/consumer/PullAPIWrapper.java
浏览文件 @
37b6dd02
...
...
@@ -209,34 +209,6 @@ public class PullAPIWrapper {
throw
new
MQClientException
(
"The broker["
+
mq
.
getBrokerName
()
+
"] not exist"
,
null
);
}
public
PullResult
pullKernelImpl
(
final
MessageQueue
mq
,
final
String
subExpression
,
final
long
subVersion
,
final
long
offset
,
final
int
maxNums
,
final
int
sysFlag
,
final
long
commitOffset
,
final
long
brokerSuspendMaxTimeMillis
,
final
long
timeoutMillis
,
final
CommunicationMode
communicationMode
,
final
PullCallback
pullCallback
)
throws
MQClientException
,
RemotingException
,
MQBrokerException
,
InterruptedException
{
return
pullKernelImpl
(
mq
,
subExpression
,
ExpressionType
.
TAG
,
subVersion
,
offset
,
maxNums
,
sysFlag
,
commitOffset
,
brokerSuspendMaxTimeMillis
,
timeoutMillis
,
communicationMode
,
pullCallback
);
}
public
long
recalculatePullFromWhichNode
(
final
MessageQueue
mq
)
{
if
(
this
.
isConnectBrokerByUser
())
{
return
this
.
defaultBrokerId
;
...
...
client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
浏览文件 @
37b6dd02
...
...
@@ -1046,6 +1046,19 @@ public class MQClientInstance {
if
(
this
.
brokerVersionTable
.
get
(
brokerName
).
containsKey
(
brokerAddr
))
{
return
this
.
brokerVersionTable
.
get
(
brokerName
).
get
(
brokerAddr
);
}
}
else
{
HeartbeatData
heartbeatData
=
prepareHeartbeatData
();
try
{
int
version
=
this
.
mQClientAPIImpl
.
sendHearbeat
(
brokerAddr
,
heartbeatData
,
3000
);
return
version
;
}
catch
(
Exception
e
)
{
if
(
this
.
isBrokerInNameServer
(
brokerAddr
))
{
log
.
info
(
"send heart beat to broker[{} {}] failed"
,
brokerName
,
brokerAddr
);
}
else
{
log
.
info
(
"send heart beat to broker[{} {}] exception, because the broker not up, forget it"
,
brokerName
,
brokerAddr
);
}
}
}
return
0
;
}
...
...
client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
浏览文件 @
37b6dd02
...
...
@@ -30,8 +30,10 @@ import java.util.concurrent.ConcurrentMap;
import
java.util.concurrent.ExecutorService
;
import
java.util.concurrent.LinkedBlockingQueue
;
import
java.util.concurrent.RejectedExecutionException
;
import
java.util.concurrent.ThreadFactory
;
import
java.util.concurrent.ThreadPoolExecutor
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.atomic.AtomicInteger
;
import
org.apache.rocketmq.client.QueryResult
;
import
org.apache.rocketmq.client.Validators
;
import
org.apache.rocketmq.client.common.ClientErrorCode
;
...
...
@@ -101,6 +103,10 @@ public class DefaultMQProducerImpl implements MQProducerInner {
private
MQFaultStrategy
mqFaultStrategy
=
new
MQFaultStrategy
();
private
final
BlockingQueue
<
Runnable
>
asyncSenderThreadPoolQueue
;
private
final
ExecutorService
defaultAsyncSenderExecutor
;
private
ExecutorService
asyncSenderExecutor
;
public
DefaultMQProducerImpl
(
final
DefaultMQProducer
defaultMQProducer
)
{
this
(
defaultMQProducer
,
null
);
}
...
...
@@ -108,6 +114,22 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public
DefaultMQProducerImpl
(
final
DefaultMQProducer
defaultMQProducer
,
RPCHook
rpcHook
)
{
this
.
defaultMQProducer
=
defaultMQProducer
;
this
.
rpcHook
=
rpcHook
;
this
.
asyncSenderThreadPoolQueue
=
new
LinkedBlockingQueue
<
Runnable
>(
50000
);
this
.
defaultAsyncSenderExecutor
=
new
ThreadPoolExecutor
(
Runtime
.
getRuntime
().
availableProcessors
(),
Runtime
.
getRuntime
().
availableProcessors
(),
1000
*
60
,
TimeUnit
.
MILLISECONDS
,
this
.
asyncSenderThreadPoolQueue
,
new
ThreadFactory
()
{
private
AtomicInteger
threadIndex
=
new
AtomicInteger
(
0
);
@Override
public
Thread
newThread
(
Runnable
r
)
{
return
new
Thread
(
r
,
"AsyncSenderExecutor_"
+
this
.
threadIndex
.
incrementAndGet
());
}
});
}
public
void
registerCheckForbiddenHook
(
CheckForbiddenHook
checkForbiddenHook
)
{
...
...
@@ -456,7 +478,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public
void
send
(
final
Message
msg
,
final
SendCallback
sendCallback
,
final
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
final
long
beginStartTime
=
System
.
currentTimeMillis
();
ExecutorService
executor
=
this
.
get
Callback
Executor
();
ExecutorService
executor
=
this
.
get
AsyncSender
Executor
();
try
{
executor
.
submit
(
new
Runnable
()
{
@Override
...
...
@@ -957,7 +979,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public
void
send
(
final
Message
msg
,
final
MessageQueue
mq
,
final
SendCallback
sendCallback
,
final
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
final
long
beginStartTime
=
System
.
currentTimeMillis
();
ExecutorService
executor
=
this
.
get
Callback
Executor
();
ExecutorService
executor
=
this
.
get
AsyncSender
Executor
();
try
{
executor
.
submit
(
new
Runnable
()
{
@Override
...
...
@@ -1079,7 +1101,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public
void
send
(
final
Message
msg
,
final
MessageQueueSelector
selector
,
final
Object
arg
,
final
SendCallback
sendCallback
,
final
long
timeout
)
throws
MQClientException
,
RemotingException
,
InterruptedException
{
final
long
beginStartTime
=
System
.
currentTimeMillis
();
ExecutorService
executor
=
this
.
get
Callback
Executor
();
ExecutorService
executor
=
this
.
get
AsyncSender
Executor
();
try
{
executor
.
submit
(
new
Runnable
()
{
@Override
...
...
@@ -1243,9 +1265,13 @@ public class DefaultMQProducerImpl implements MQProducerInner {
public
void
setCallbackExecutor
(
final
ExecutorService
callbackExecutor
)
{
this
.
mQClientFactory
.
getMQClientAPIImpl
().
getRemotingClient
().
setCallbackExecutor
(
callbackExecutor
);
}
public
ExecutorService
getCallbackExecutor
()
{
return
this
.
mQClientFactory
.
getMQClientAPIImpl
().
getRemotingClient
().
getCallbackExecutor
();
public
ExecutorService
getAsyncSenderExecutor
()
{
return
null
==
asyncSenderExecutor
?
defaultAsyncSenderExecutor
:
asyncSenderExecutor
;
}
public
void
setAsyncSenderExecutor
(
ExecutorService
asyncSenderExecutor
)
{
this
.
asyncSenderExecutor
=
asyncSenderExecutor
;
}
public
SendResult
send
(
Message
msg
,
...
...
client/src/main/java/org/apache/rocketmq/client/producer/DefaultMQProducer.java
浏览文件 @
37b6dd02
...
...
@@ -655,6 +655,16 @@ public class DefaultMQProducer extends ClientConfig implements MQProducer {
this
.
defaultMQProducerImpl
.
setCallbackExecutor
(
callbackExecutor
);
}
/**
* Sets an Executor to be used for executing asynchronous send. If the Executor is not set, {@link
* DefaultMQProducerImpl#defaultAsyncSenderExecutor} will be used.
*
* @param asyncSenderExecutor the instance of Executor
*/
public
void
setAsyncSenderExecutor
(
final
ExecutorService
asyncSenderExecutor
)
{
this
.
defaultMQProducerImpl
.
setAsyncSenderExecutor
(
asyncSenderExecutor
);
}
private
MessageBatch
batch
(
Collection
<
Message
>
msgs
)
throws
MQClientException
{
MessageBatch
msgBatch
;
try
{
...
...
client/src/test/java/org/apache/rocketmq/client/producer/DefaultMQProducerTest.java
浏览文件 @
37b6dd02
...
...
@@ -167,10 +167,7 @@ public class DefaultMQProducerTest {
@Test
public
void
testSendMessageAsync_Success
()
throws
RemotingException
,
InterruptedException
,
MQBrokerException
,
MQClientException
{
ExecutorService
callbackExecutor
=
Executors
.
newSingleThreadExecutor
();
final
CountDownLatch
countDownLatch
=
new
CountDownLatch
(
1
);
when
(
mQClientAPIImpl
.
getRemotingClient
()).
thenReturn
((
nettyRemotingClient
));
when
(
nettyRemotingClient
.
getCallbackExecutor
()).
thenReturn
(
callbackExecutor
);
producer
.
send
(
message
,
new
SendCallback
()
{
@Override
public
void
onSuccess
(
SendResult
sendResult
)
{
...
...
@@ -186,15 +183,11 @@ public class DefaultMQProducerTest {
}
});
countDownLatch
.
await
(
3000L
,
TimeUnit
.
MILLISECONDS
);
callbackExecutor
.
shutdown
();
}
@Test
public
void
testSendMessageAsync
()
throws
RemotingException
,
MQClientException
,
InterruptedException
{
final
AtomicInteger
cc
=
new
AtomicInteger
(
0
);
final
CountDownLatch
countDownLatch
=
new
CountDownLatch
(
6
);
ExecutorService
callbackExecutor
=
Executors
.
newSingleThreadExecutor
();
when
(
mQClientAPIImpl
.
getRemotingClient
()).
thenReturn
((
nettyRemotingClient
));
when
(
nettyRemotingClient
.
getCallbackExecutor
()).
thenReturn
(
callbackExecutor
);
SendCallback
sendCallback
=
new
SendCallback
()
{
@Override
...
...
@@ -226,16 +219,13 @@ public class DefaultMQProducerTest {
producer
.
send
(
message
,
messageQueueSelector
,
null
,
sendCallback
,
1000
);
countDownLatch
.
await
(
3000L
,
TimeUnit
.
MILLISECONDS
);
callbackExecutor
.
shutdown
();
assertThat
(
cc
.
get
()).
isEqualTo
(
6
);
}
@Test
public
void
testSendMessageAsync_BodyCompressed
()
throws
RemotingException
,
InterruptedException
,
MQBrokerException
,
MQClientException
{
ExecutorService
callbackExecutor
=
Executors
.
newSingleThreadExecutor
();
final
CountDownLatch
countDownLatch
=
new
CountDownLatch
(
1
);
when
(
mQClientAPIImpl
.
getRemotingClient
()).
thenReturn
((
nettyRemotingClient
));
when
(
nettyRemotingClient
.
getCallbackExecutor
()).
thenReturn
(
callbackExecutor
);
producer
.
send
(
bigMessage
,
new
SendCallback
()
{
@Override
public
void
onSuccess
(
SendResult
sendResult
)
{
...
...
@@ -251,7 +241,6 @@ public class DefaultMQProducerTest {
}
});
countDownLatch
.
await
(
3000L
,
TimeUnit
.
MILLISECONDS
);
callbackExecutor
.
shutdown
();
}
@Test
...
...
common/src/main/java/org/apache/rocketmq/common/constant/DBMsgConstants.java
浏览文件 @
37b6dd02
...
...
@@ -18,5 +18,5 @@
package
org.apache.rocketmq.common.constant
;
public
class
DBMsgConstants
{
public
static
final
int
MAX_BODY_SIZE
=
64
*
1024
*
1
20
4
;
//64KB
public
static
final
int
MAX_BODY_SIZE
=
64
*
1024
*
1
02
4
;
//64KB
}
docs/cn/index.md
0 → 100644
浏览文件 @
37b6dd02
docs/en/index.md
0 → 100644
浏览文件 @
37b6dd02
store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
浏览文件 @
37b6dd02
...
...
@@ -604,7 +604,7 @@ public class MessageStoreConfig {
}
/**
* Enable transient commitLog store po
l
l only if transientStorePoolEnable is true and the FlushDiskType is
* Enable transient commitLog store po
o
l only if transientStorePoolEnable is true and the FlushDiskType is
* ASYNC_FLUSH
*
* @return <tt>true</tt> or <tt>false</tt>
...
...
test/src/test/java/org/apache/rocketmq/test/client/consumer/filter/SqlFilterIT.java
浏览文件 @
37b6dd02
...
...
@@ -17,12 +17,18 @@
package
org.apache.rocketmq.test.client.consumer.filter
;
import
java.util.ArrayList
;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Set
;
import
org.apache.log4j.Logger
;
import
org.apache.rocketmq.client.consumer.DefaultMQPullConsumer
;
import
org.apache.rocketmq.client.consumer.MessageSelector
;
import
org.apache.rocketmq.client.consumer.PullResult
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageQueue
;
import
org.apache.rocketmq.test.base.BaseConf
;
import
org.apache.rocketmq.test.client.consumer.broadcast.BaseBroadCastIT
;
import
org.apache.rocketmq.test.client.consumer.broadcast.normal.NormalMsgTwoSameGroupConsumerIT
;
import
org.apache.rocketmq.test.client.rmq.RMQBroadCastConsumer
;
import
org.apache.rocketmq.test.client.rmq.RMQNormalProducer
;
import
org.apache.rocketmq.test.client.rmq.RMQSqlConsumer
;
import
org.apache.rocketmq.test.factory.ConsumerFactory
;
...
...
@@ -39,12 +45,14 @@ public class SqlFilterIT extends BaseConf {
private
static
Logger
logger
=
Logger
.
getLogger
(
SqlFilterIT
.
class
);
private
RMQNormalProducer
producer
=
null
;
private
String
topic
=
null
;
private
static
final
Map
<
MessageQueue
,
Long
>
OFFSE_TABLE
=
new
HashMap
<
MessageQueue
,
Long
>();
@Before
public
void
setUp
()
{
topic
=
initTopic
();
logger
.
info
(
String
.
format
(
"use topic: %s;"
,
topic
));
producer
=
getProducer
(
nsAddr
,
topic
);
OFFSE_TABLE
.
clear
();
}
@After
...
...
@@ -71,4 +79,65 @@ public class SqlFilterIT extends BaseConf {
assertThat
(
consumer
.
getListener
().
getAllMsgBody
().
size
()).
isEqualTo
(
msgSize
*
2
);
}
@Test
public
void
testFilterPullConsumer
()
throws
Exception
{
int
msgSize
=
16
;
String
group
=
initConsumerGroup
();
MessageSelector
selector
=
MessageSelector
.
bySql
(
"(TAGS is not null and TAGS in ('TagA', 'TagB'))"
);
DefaultMQPullConsumer
consumer
=
new
DefaultMQPullConsumer
(
group
);
consumer
.
setNamesrvAddr
(
nsAddr
);
consumer
.
start
();
Thread
.
sleep
(
3000
);
producer
.
send
(
"TagA"
,
msgSize
);
producer
.
send
(
"TagB"
,
msgSize
);
producer
.
send
(
"TagC"
,
msgSize
);
Assert
.
assertEquals
(
"Not all sent succeeded"
,
msgSize
*
3
,
producer
.
getAllUndupMsgBody
().
size
());
List
<
String
>
receivedMessage
=
new
ArrayList
<>(
2
);
Set
<
MessageQueue
>
mqs
=
consumer
.
fetchSubscribeMessageQueues
(
topic
);
for
(
MessageQueue
mq
:
mqs
)
{
SINGLE_MQ:
while
(
true
)
{
try
{
PullResult
pullResult
=
consumer
.
pull
(
mq
,
selector
,
getMessageQueueOffset
(
mq
),
32
);
putMessageQueueOffset
(
mq
,
pullResult
.
getNextBeginOffset
());
switch
(
pullResult
.
getPullStatus
())
{
case
FOUND:
List
<
MessageExt
>
msgs
=
pullResult
.
getMsgFoundList
();
for
(
MessageExt
msg
:
msgs
)
{
receivedMessage
.
add
(
new
String
(
msg
.
getBody
()));
}
break
;
case
NO_MATCHED_MSG:
break
;
case
NO_NEW_MSG:
break
SINGLE_MQ
;
case
OFFSET_ILLEGAL:
break
;
default
:
break
;
}
}
catch
(
Exception
e
)
{
e
.
printStackTrace
();
}
}
}
assertThat
(
receivedMessage
.
size
()).
isEqualTo
(
msgSize
*
2
);
}
private
static
long
getMessageQueueOffset
(
MessageQueue
mq
)
{
Long
offset
=
OFFSE_TABLE
.
get
(
mq
);
if
(
offset
!=
null
)
return
offset
;
return
0
;
}
private
static
void
putMessageQueueOffset
(
MessageQueue
mq
,
long
offset
)
{
OFFSE_TABLE
.
put
(
mq
,
offset
);
}
}
tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
浏览文件 @
37b6dd02
...
...
@@ -155,7 +155,7 @@ public class ConsumerProgressSubCommand implements SubCommand {
}
System
.
out
.
printf
(
"%n"
);
System
.
out
.
printf
(
"Consume TPS: %
s
%n"
,
consumeStats
.
getConsumeTps
());
System
.
out
.
printf
(
"Consume TPS: %
.2f
%n"
,
consumeStats
.
getConsumeTps
());
System
.
out
.
printf
(
"Diff Total: %d%n"
,
diffTotal
);
}
else
{
System
.
out
.
printf
(
"%-32s %-6s %-24s %-5s %-14s %-7s %s%n"
,
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录