Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
s920243400
Rocketmq
提交
11b686e4
R
Rocketmq
项目概览
s920243400
/
Rocketmq
与 Fork 源项目一致
Fork自
Apache RocketMQ / Rocketmq
通知
1
Star
1
Fork
0
代码
文件
提交
分支
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看板
提交
11b686e4
编写于
7月 17, 2019
作者:
D
duhenglucky
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Add lite pull consumer example
上级
af04557a
变更
4
隐藏空白更改
内联
并排
Showing
4 changed file
with
192 addition
and
19 deletion
+192
-19
client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLiteMQPullConsumer.java
...e/rocketmq/client/consumer/DefaultLiteMQPullConsumer.java
+75
-7
client/src/main/java/org/apache/rocketmq/client/consumer/LiteMQPullConsumer.java
...g/apache/rocketmq/client/consumer/LiteMQPullConsumer.java
+0
-3
client/src/main/java/org/apache/rocketmq/client/impl/consumer/LiteMQPullConsumerImpl.java
...rocketmq/client/impl/consumer/LiteMQPullConsumerImpl.java
+68
-9
example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerTest.java
.../apache/rocketmq/example/simple/LitePullConsumerTest.java
+49
-0
未找到文件。
client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLiteMQPullConsumer.java
浏览文件 @
11b686e4
...
...
@@ -27,18 +27,50 @@ import org.apache.rocketmq.remoting.RPCHook;
public
class
DefaultLiteMQPullConsumer
extends
DefaultMQPullConsumer
implements
LiteMQPullConsumer
{
private
LiteMQPullConsumerImpl
liteMQPullConsumer
;
/**
* Maximum amount of time in minutes a message may block the consuming thread.
*/
private
long
consumeTimeout
=
15
;
/**
* Is auto commit offset
*/
private
boolean
autoCommit
=
true
;
private
int
pullThreadNumbers
=
20
;
/**
* Maximum commit offset interval time in seconds.
*/
private
long
autoCommitInterval
=
20
;
public
DefaultLiteMQPullConsumer
(
String
consumerGroup
,
RPCHook
rpcHook
)
{
this
.
setConsumerGroup
(
consumerGroup
);
this
.
liteMQPullConsumer
=
new
LiteMQPullConsumerImpl
(
this
,
rpcHook
);
}
@Override
public
void
subscribe
(
String
topic
,
String
subExpression
)
throws
MQClientException
{
public
DefaultLiteMQPullConsumer
(
String
consumerGroup
)
{
this
.
setConsumerGroup
(
consumerGroup
);
this
.
liteMQPullConsumer
=
new
LiteMQPullConsumerImpl
(
this
,
null
);
}
@Override
public
void
start
()
throws
MQClientException
{
this
.
liteMQPullConsumer
.
start
();
}
@Override
public
void
subscribe
(
String
topic
,
String
subExpression
)
throws
MQClientException
{
this
.
liteMQPullConsumer
.
subscribe
(
topic
,
subExpression
);
}
@Override
public
void
unsubscribe
(
String
topic
)
{
@Override
public
void
unsubscribe
(
String
topic
)
{
this
.
liteMQPullConsumer
.
unsubscribe
(
topic
);
}
@Override
public
List
<
MessageExt
>
poll
()
{
@Override
public
List
<
MessageExt
>
poll
()
{
return
poll
(
this
.
getConsumerPullTimeoutMillis
());
}
...
...
@@ -46,19 +78,55 @@ public class DefaultLiteMQPullConsumer extends DefaultMQPullConsumer implements
return
liteMQPullConsumer
.
poll
(
timeout
);
}
@Override
public
void
seek
(
MessageQueue
messageQueue
,
long
offset
)
throws
MQClientException
{
@Override
public
void
seek
(
MessageQueue
messageQueue
,
long
offset
)
throws
MQClientException
{
this
.
liteMQPullConsumer
.
seek
(
messageQueue
,
offset
);
}
@Override
public
void
pause
(
Collection
<
MessageQueue
>
messageQueues
)
{
this
.
liteMQPullConsumer
.
pause
(
messageQueues
);
}
@Override
public
void
resume
(
Collection
<
MessageQueue
>
messageQueues
)
{
this
.
liteMQPullConsumer
.
resume
(
messageQueues
);
}
@Override
public
void
commitSync
()
{
this
.
liteMQPullConsumer
.
commit
();
}
public
long
getConsumeTimeout
()
{
return
consumeTimeout
;
}
@Override
public
void
pause
(
Collection
<
MessageQueue
>
messageQueueCollection
)
{
public
void
setConsumeTimeout
(
long
consumeTimeout
)
{
this
.
consumeTimeout
=
consumeTimeout
;
}
public
boolean
isAutoCommit
()
{
return
autoCommit
;
}
@Override
public
void
resume
(
Collection
<
MessageQueue
>
partitions
)
{
public
void
setAutoCommit
(
boolean
autoCommit
)
{
this
.
autoCommit
=
autoCommit
;
}
public
int
getPullThreadNumbers
()
{
return
pullThreadNumbers
;
}
@Override
public
void
commitSync
()
{
public
void
setPullThreadNumbers
(
int
pullThreadNumbers
)
{
this
.
pullThreadNumbers
=
pullThreadNumbers
;
}
public
long
getAutoCommitInterval
()
{
return
autoCommitInterval
;
}
public
void
setAutoCommitInterval
(
long
autoCommitInterval
)
{
this
.
autoCommitInterval
=
autoCommitInterval
;
}
}
client/src/main/java/org/apache/rocketmq/client/consumer/LiteMQPullConsumer.java
浏览文件 @
11b686e4
...
...
@@ -38,9 +38,6 @@ public interface LiteMQPullConsumer {
*/
void
unsubscribe
(
final
String
topic
);
/**
* @return
*/
List
<
MessageExt
>
poll
();
List
<
MessageExt
>
poll
(
long
timeout
);
...
...
client/src/main/java/org/apache/rocketmq/client/impl/consumer/LiteMQPullConsumerImpl.java
浏览文件 @
11b686e4
...
...
@@ -27,12 +27,14 @@ import java.util.TreeMap;
import
java.util.concurrent.BlockingQueue
;
import
java.util.concurrent.ConcurrentHashMap
;
import
java.util.concurrent.ConcurrentMap
;
import
java.util.concurrent.Executors
;
import
java.util.concurrent.LinkedBlockingQueue
;
import
java.util.concurrent.ScheduledExecutorService
;
import
java.util.concurrent.ScheduledThreadPoolExecutor
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.locks.ReadWriteLock
;
import
org.apache.commons.lang3.reflect.FieldUtils
;
import
org.apache.rocketmq.client.consumer.DefaultMQPullConsumer
;
import
org.apache.rocketmq.client.consumer.Default
Lite
MQPullConsumer
;
import
org.apache.rocketmq.client.consumer.MessageQueueListener
;
import
org.apache.rocketmq.client.consumer.PullResult
;
import
org.apache.rocketmq.client.exception.MQClientException
;
...
...
@@ -50,6 +52,8 @@ import org.apache.rocketmq.remoting.RPCHook;
public
class
LiteMQPullConsumerImpl
extends
DefaultMQPullConsumerImpl
{
private
final
InternalLogger
log
=
ClientLogger
.
getLog
();
private
DefaultLiteMQPullConsumer
defaultLiteMQPullConsumer
;
private
final
ConcurrentMap
<
MessageQueue
,
PullTaskImpl
>
taskTable
=
new
ConcurrentHashMap
<
MessageQueue
,
PullTaskImpl
>();
...
...
@@ -58,12 +62,21 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
private
List
<
ConsumeRequest
>
allConsumed
=
new
ArrayList
<
ConsumeRequest
>(
256
);
private
final
BlockingQueue
<
ConsumeRequest
>
consumeRequestCache
=
new
LinkedBlockingQueue
<
ConsumeRequest
>();
;
private
final
ScheduledExecutorService
cleanExpireMsgExecutors
;
private
ScheduledThreadPoolExecutor
scheduledThreadPoolExecutor
;
public
LiteMQPullConsumerImpl
(
final
DefaultMQPullConsumer
defaultMQPullConsumer
,
final
RPCHook
rpcHook
)
{
private
ScheduledExecutorService
autoCommitExecutors
;
public
LiteMQPullConsumerImpl
(
final
DefaultLiteMQPullConsumer
defaultMQPullConsumer
,
final
RPCHook
rpcHook
)
{
super
(
defaultMQPullConsumer
,
rpcHook
);
this
.
defaultLiteMQPullConsumer
=
defaultMQPullConsumer
;
this
.
cleanExpireMsgExecutors
=
Executors
.
newSingleThreadScheduledExecutor
(
new
ThreadFactoryImpl
(
"Lite_CleanExpireMsgScheduledThread_"
));
this
.
autoCommitExecutors
=
Executors
.
newSingleThreadScheduledExecutor
(
new
ThreadFactoryImpl
(
"Lite_AutoCommitScheduledThread_"
));
}
public
void
updateAssignedMessageQueue
(
String
topic
,
Set
<
MessageQueue
>
assignedMessageQueue
)
{
...
...
@@ -115,18 +128,43 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
@Override
public
synchronized
void
start
()
throws
MQClientException
{
this
.
defaultMQPullConsumer
.
setMessageQueueListener
(
new
MessageQueueListenerImpl
());
super
.
start
();
final
String
group
=
this
.
defaultMQPullConsumer
.
getConsumerGroup
();
this
.
scheduledThreadPoolExecutor
=
new
ScheduledThreadPoolExecutor
(
10
,
//this.pullThreadNums
,
this
.
defaultLiteMQPullConsumer
.
getPullThreadNumbers
()
,
new
ThreadFactoryImpl
(
"PullMsgThread-"
+
group
)
);
this
.
defaultMQPullConsumer
.
setMessageQueueListener
(
new
MessageQueueListenerImpl
());
this
.
cleanExpireMsgExecutors
.
scheduleAtFixedRate
(
new
Runnable
()
{
@Override
public
void
run
()
{
cleanExpireMsg
();
}
},
this
.
defaultLiteMQPullConsumer
.
getConsumeTimeout
(),
this
.
defaultLiteMQPullConsumer
.
getConsumeTimeout
(),
TimeUnit
.
MINUTES
);
this
.
autoCommitExecutors
.
scheduleAtFixedRate
(
new
Runnable
()
{
@Override
public
void
run
()
{
if
(
defaultLiteMQPullConsumer
.
isAutoCommit
())
{
commit
();
}
}
},
this
.
defaultLiteMQPullConsumer
.
getAutoCommitInterval
(),
this
.
defaultLiteMQPullConsumer
.
getAutoCommitInterval
(),
TimeUnit
.
SECONDS
);
updateTopicSubscribeInfoWhenSubscriptionChanged
();
}
private
void
updateTopicSubscribeInfoWhenSubscriptionChanged
()
{
Map
<
String
,
SubscriptionData
>
subTable
=
rebalanceImpl
.
getSubscriptionInner
();
if
(
subTable
!=
null
)
{
for
(
final
Map
.
Entry
<
String
,
SubscriptionData
>
entry
:
subTable
.
entrySet
())
{
final
String
topic
=
entry
.
getKey
();
this
.
mQClientFactory
.
updateTopicRouteInfoFromNameServer
(
topic
);
}
}
}
public
List
<
MessageExt
>
poll
(
long
timeout
)
{
try
{
ConsumeRequest
consumeRequest
=
consumeRequestCache
.
poll
(
timeout
,
TimeUnit
.
MILLI
SECONDS
);
ConsumeRequest
consumeRequest
=
consumeRequestCache
.
poll
(
timeout
,
TimeUnit
.
SECONDS
);
if
(
consumeRequest
!=
null
)
{
List
<
MessageExt
>
messages
=
consumeRequest
.
getMessageExts
();
for
(
MessageExt
messageExt
:
messages
)
{
...
...
@@ -148,6 +186,16 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
assignedMessageQueue
.
resume
(
messageQueues
);
}
public
void
seek
(
MessageQueue
messageQueue
,
long
offset
)
throws
MQClientException
{
this
.
updatePullOffset
(
messageQueue
,
offset
);
try
{
updateConsumeOffset
(
messageQueue
,
offset
);
}
catch
(
MQClientException
ex
)
{
log
.
error
(
"Seek offset to remote message queue error!"
,
ex
);
throw
ex
;
}
}
public
void
unsubscribe
(
final
String
topic
)
{
unsubscribe
(
topic
);
removePullTaskCallback
(
topic
);
...
...
@@ -270,7 +318,7 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
if
(!
msgTreeMap
.
isEmpty
())
{
msg
=
msgTreeMap
.
firstEntry
().
getValue
();
if
(
System
.
currentTimeMillis
()
-
Long
.
parseLong
(
MessageAccessor
.
getConsumeStartTimeStamp
(
msg
))
>
10
*
60
*
1000
)
{
>
this
.
defaultLiteMQPullConsumer
.
getConsumeTimeout
()
*
60
*
1000
)
{
//Expired, ack and remove it.
}
else
{
break
;
...
...
@@ -316,16 +364,19 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
@Override
public
void
run
()
{
System
.
out
.
println
(
"begin pull message"
);
String
topic
=
this
.
messageQueue
.
getTopic
();
if
(!
this
.
isCancelled
())
{
if
(
assignedMessageQueue
.
isPaused
(
messageQueue
))
{
scheduledThreadPoolExecutor
.
schedule
(
this
,
1000
,
TimeUnit
.
MILLISECONDS
);
log
.
debug
(
"Message Queue: {} has been paused!"
,
messageQueue
);
return
;
}
SubscriptionData
subscriptionData
=
rebalanceImpl
.
getSubscriptionInner
().
get
(
topic
);
long
offset
=
nextPullOffset
(
messageQueue
);
long
pullDelayTimeMills
=
0
;
try
{
PullResult
pullResult
=
defaultMQPullConsumer
.
pull
(
messageQueue
,
subscriptionData
.
getSubString
(),
offset
,
nextPullBatchNums
());
PullResult
pullResult
=
pull
(
messageQueue
,
subscriptionData
.
getSubString
(),
offset
,
nextPullBatchNums
());
ProcessQueue
processQueue
=
rebalanceImpl
.
getProcessQueueTable
().
get
(
messageQueue
);
switch
(
pullResult
.
getPullStatus
())
{
case
FOUND:
...
...
@@ -338,9 +389,17 @@ public class LiteMQPullConsumerImpl extends DefaultMQPullConsumerImpl {
break
;
}
updatePullOffset
(
messageQueue
,
pullResult
.
getNextBeginOffset
());
}
catch
(
Exception
e
)
{
}
catch
(
Throwable
e
)
{
pullDelayTimeMills
=
1000
;
e
.
printStackTrace
();
log
.
error
(
"An error occurred in pull message process."
,
e
);
}
if
(!
this
.
isCancelled
())
{
scheduledThreadPoolExecutor
.
schedule
(
this
,
pullDelayTimeMills
,
TimeUnit
.
MILLISECONDS
);
}
else
{
log
.
warn
(
"The Pull Task is cancelled after doPullTask, {}"
,
messageQueue
);
}
}
}
...
...
example/src/main/java/org/apache/rocketmq/example/simple/LitePullConsumerTest.java
0 → 100644
浏览文件 @
11b686e4
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package
org.apache.rocketmq.example.simple
;
import
java.util.Arrays
;
import
java.util.List
;
import
org.apache.rocketmq.client.consumer.DefaultLiteMQPullConsumer
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageQueue
;
public
class
LitePullConsumerTest
{
public
static
void
main
(
String
[]
args
)
throws
Exception
{
DefaultLiteMQPullConsumer
litePullConsumer
=
new
DefaultLiteMQPullConsumer
(
"test"
,
null
);
litePullConsumer
.
subscribe
(
"test"
,
null
);
litePullConsumer
.
start
();
MessageQueue
messageQueue
=
new
MessageQueue
(
"test"
,
"duhengdeMacBook-Pro.local"
,
1
);
int
i
=
0
;
while
(
true
)
{
List
<
MessageExt
>
messageExts
=
litePullConsumer
.
poll
();
System
.
out
.
println
(
"-----------"
);
System
.
out
.
println
(
messageExts
);
System
.
out
.
println
(
"-----------"
);
i
++;
if
(
i
==
3
)
{
System
.
out
.
println
(
"pause"
);
litePullConsumer
.
pause
(
Arrays
.
asList
(
messageQueue
));
}
if
(
i
==
10
)
{
System
.
out
.
println
(
"resume"
);
litePullConsumer
.
resume
(
Arrays
.
asList
(
messageQueue
));
}
litePullConsumer
.
commitSync
();
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录