Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
Apache RocketMQ
Rocketmq
提交
d21266d9
R
Rocketmq
项目概览
Apache RocketMQ
/
Rocketmq
上一次同步 大约 3 年
通知
267
Star
16139
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看板
未验证
提交
d21266d9
编写于
7月 31, 2021
作者:
Y
yuz10
提交者:
GitHub
7月 31, 2021
浏览文件
操作
浏览文件
下载
差异文件
Merge branch 'develop' into develop3
上级
3aa49437
d067bfc2
变更
19
展开全部
显示空白变更内容
内联
并排
Showing
19 changed file
with
378 addition
and
803 deletion
+378
-803
.travis.yml
.travis.yml
+2
-2
broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java
...cketmq/broker/processor/AbstractSendMessageProcessor.java
+1
-1
broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
...pache/rocketmq/broker/processor/AdminBrokerProcessor.java
+1
-1
broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
...pache/rocketmq/broker/processor/SendMessageProcessor.java
+1
-0
client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
...apache/rocketmq/client/impl/factory/MQClientInstance.java
+3
-21
distribution/bin/runbroker.cmd
distribution/bin/runbroker.cmd
+1
-1
distribution/bin/runbroker.sh
distribution/bin/runbroker.sh
+1
-1
distribution/bin/runserver.cmd
distribution/bin/runserver.cmd
+1
-1
distribution/bin/runserver.sh
distribution/bin/runserver.sh
+3
-3
remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java
...org/apache/rocketmq/remoting/netty/NettySystemConfig.java
+0
-1
store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
...java/org/apache/rocketmq/store/AppendMessageCallback.java
+3
-2
store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java
...n/java/org/apache/rocketmq/store/AppendMessageResult.java
+17
-0
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
+288
-472
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
...n/java/org/apache/rocketmq/store/DefaultMessageStore.java
+9
-46
store/src/main/java/org/apache/rocketmq/store/MappedFile.java
...e/src/main/java/org/apache/rocketmq/store/MappedFile.java
+13
-7
store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java
...java/org/apache/rocketmq/store/MessageExtBrokerInner.java
+12
-0
store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
...a/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
+0
-232
store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java
...st/java/org/apache/rocketmq/store/AppendCallbackTest.java
+19
-9
tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
...ketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
+3
-3
未找到文件。
.travis.yml
浏览文件 @
d21266d9
...
@@ -16,8 +16,8 @@ matrix:
...
@@ -16,8 +16,8 @@ matrix:
# On Linux we install latest OpenJDK 1.8 from Ubuntu repositories
# On Linux we install latest OpenJDK 1.8 from Ubuntu repositories
-
name
:
Linux x86_64
-
name
:
Linux x86_64
arch
:
amd64
arch
:
amd64
-
name
:
Linux aarch64
#
- name: Linux aarch64
arch
:
arm64
#
arch: arm64
cache
:
cache
:
directories
:
directories
:
...
...
broker/src/main/java/org/apache/rocketmq/broker/processor/AbstractSendMessageProcessor.java
浏览文件 @
d21266d9
...
@@ -257,8 +257,8 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
...
@@ -257,8 +257,8 @@ public abstract class AbstractSendMessageProcessor extends AsyncNettyRequestProc
try
{
try
{
final
SendMessageRequestHeader
requestHeader
=
parseRequestHeader
(
request
);
final
SendMessageRequestHeader
requestHeader
=
parseRequestHeader
(
request
);
String
namespace
=
NamespaceUtil
.
getNamespaceFromResource
(
requestHeader
.
getTopic
());
if
(
null
!=
requestHeader
)
{
if
(
null
!=
requestHeader
)
{
String
namespace
=
NamespaceUtil
.
getNamespaceFromResource
(
requestHeader
.
getTopic
());
context
.
setNamespace
(
namespace
);
context
.
setNamespace
(
namespace
);
context
.
setProducerGroup
(
requestHeader
.
getProducerGroup
());
context
.
setProducerGroup
(
requestHeader
.
getProducerGroup
());
context
.
setTopic
(
requestHeader
.
getTopic
());
context
.
setTopic
(
requestHeader
.
getTopic
());
...
...
broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
浏览文件 @
d21266d9
...
@@ -1445,7 +1445,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
...
@@ -1445,7 +1445,7 @@ public class AdminBrokerProcessor extends AsyncNettyRequestProcessor implements
java
.
io
.
File
commitLogDir
=
new
java
.
io
.
File
(
this
.
brokerController
.
getMessageStoreConfig
().
getStorePathRootDir
());
java
.
io
.
File
commitLogDir
=
new
java
.
io
.
File
(
this
.
brokerController
.
getMessageStoreConfig
().
getStorePathRootDir
());
if
(
commitLogDir
.
exists
())
{
if
(
commitLogDir
.
exists
())
{
runtimeInfo
.
put
(
"commitLogDirCapacity"
,
String
.
format
(
"Total : %s, Free : %s."
,
MixAll
.
humanReadableByteCount
(
commitLogDir
.
getTotalSpace
(),
false
),
MixAll
.
humanReadableByteCount
(
commitLogDir
.
get
Fre
eSpace
(),
false
)));
runtimeInfo
.
put
(
"commitLogDirCapacity"
,
String
.
format
(
"Total : %s, Free : %s."
,
MixAll
.
humanReadableByteCount
(
commitLogDir
.
getTotalSpace
(),
false
),
MixAll
.
humanReadableByteCount
(
commitLogDir
.
get
Usabl
eSpace
(),
false
)));
}
}
return
runtimeInfo
;
return
runtimeInfo
;
...
...
broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
浏览文件 @
d21266d9
...
@@ -612,6 +612,7 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
...
@@ -612,6 +612,7 @@ public class SendMessageProcessor extends AbstractSendMessageProcessor implement
}
}
}
}
@Override
public
SocketAddress
getStoreHost
()
{
public
SocketAddress
getStoreHost
()
{
return
storeHost
;
return
storeHost
;
}
}
...
...
client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
浏览文件 @
d21266d9
...
@@ -882,24 +882,6 @@ public class MQClientInstance {
...
@@ -882,24 +882,6 @@ public class MQClientInstance {
this
.
unregisterClient
(
null
,
group
);
this
.
unregisterClient
(
null
,
group
);
}
}
private
void
unregisterClientWithLock
(
final
String
producerGroup
,
final
String
consumerGroup
)
{
try
{
if
(
this
.
lockHeartbeat
.
tryLock
(
LOCK_TIMEOUT_MILLIS
,
TimeUnit
.
MILLISECONDS
))
{
try
{
this
.
unregisterClient
(
producerGroup
,
consumerGroup
);
}
catch
(
Exception
e
)
{
log
.
error
(
"unregisterClient exception"
,
e
);
}
finally
{
this
.
lockHeartbeat
.
unlock
();
}
}
else
{
log
.
warn
(
"lock heartBeat, but failed. [{}]"
,
this
.
clientId
);
}
}
catch
(
InterruptedException
e
)
{
log
.
warn
(
"unregisterClientWithLock exception"
,
e
);
}
}
private
void
unregisterClient
(
final
String
producerGroup
,
final
String
consumerGroup
)
{
private
void
unregisterClient
(
final
String
producerGroup
,
final
String
consumerGroup
)
{
Iterator
<
Entry
<
String
,
HashMap
<
Long
,
String
>>>
it
=
this
.
brokerAddrTable
.
entrySet
().
iterator
();
Iterator
<
Entry
<
String
,
HashMap
<
Long
,
String
>>>
it
=
this
.
brokerAddrTable
.
entrySet
().
iterator
();
while
(
it
.
hasNext
())
{
while
(
it
.
hasNext
())
{
...
@@ -927,7 +909,7 @@ public class MQClientInstance {
...
@@ -927,7 +909,7 @@ public class MQClientInstance {
}
}
}
}
public
boolean
registerProducer
(
final
String
group
,
final
DefaultMQProducerImpl
producer
)
{
public
synchronized
boolean
registerProducer
(
final
String
group
,
final
DefaultMQProducerImpl
producer
)
{
if
(
null
==
group
||
null
==
producer
)
{
if
(
null
==
group
||
null
==
producer
)
{
return
false
;
return
false
;
}
}
...
@@ -941,9 +923,9 @@ public class MQClientInstance {
...
@@ -941,9 +923,9 @@ public class MQClientInstance {
return
true
;
return
true
;
}
}
public
void
unregisterProducer
(
final
String
group
)
{
public
synchronized
void
unregisterProducer
(
final
String
group
)
{
this
.
producerTable
.
remove
(
group
);
this
.
producerTable
.
remove
(
group
);
this
.
unregisterClient
WithLock
(
group
,
null
);
this
.
unregisterClient
(
group
,
null
);
}
}
public
boolean
registerAdminExt
(
final
String
group
,
final
MQAdminExtInner
admin
)
{
public
boolean
registerAdminExt
(
final
String
group
,
final
MQAdminExtInner
admin
)
{
...
...
distribution/bin/runbroker.cmd
浏览文件 @
d21266d9
...
@@ -28,7 +28,7 @@ set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH%
...
@@ -28,7 +28,7 @@ set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH%
rem ===========================================================================================
rem ===========================================================================================
rem JVM Configuration
rem JVM Configuration
rem ===========================================================================================
rem ===========================================================================================
set
"JAVA_OPT=
%JAVA_OPT%
-server -Xms2g -Xmx2g
-Xmn1g
"
set
"JAVA_OPT=
%JAVA_OPT%
-server -Xms2g -Xmx2g"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0 -XX:SurvivorRatio=8"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0 -XX:SurvivorRatio=8"
set
"JAVA_OPT=
%JAVA_OPT%
-verbose:gc -Xloggc:
%USERPROFILE%
\mq_gc.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy"
set
"JAVA_OPT=
%JAVA_OPT%
-verbose:gc -Xloggc:
%USERPROFILE%
\mq_gc.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
...
...
distribution/bin/runbroker.sh
浏览文件 @
d21266d9
...
@@ -64,7 +64,7 @@ choose_gc_log_directory()
...
@@ -64,7 +64,7 @@ choose_gc_log_directory()
choose_gc_log_directory
choose_gc_log_directory
JAVA_OPT
=
"
${
JAVA_OPT
}
-server -Xms8g -Xmx8g
-Xmn4g
"
JAVA_OPT
=
"
${
JAVA_OPT
}
-server -Xms8g -Xmx8g"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0"
JAVA_OPT
=
"
${
JAVA_OPT
}
-verbose:gc -Xloggc:
${
GC_LOG_DIR
}
/rmq_broker_gc_%p_%t.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy"
JAVA_OPT
=
"
${
JAVA_OPT
}
-verbose:gc -Xloggc:
${
GC_LOG_DIR
}
/rmq_broker_gc_%p_%t.log -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintAdaptiveSizePolicy"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
...
...
distribution/bin/runserver.cmd
浏览文件 @
d21266d9
...
@@ -28,7 +28,7 @@ set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH%
...
@@ -28,7 +28,7 @@ set CLASSPATH=.;%BASE_DIR%conf;%CLASSPATH%
set
"JAVA_OPT=
%JAVA_OPT%
-server -Xms2g -Xmx2g -Xmn1g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
set
"JAVA_OPT=
%JAVA_OPT%
-server -Xms2g -Xmx2g -Xmn1g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
set
"JAVA_OPT=
%JAVA_OPT%
-verbose:gc -Xloggc:"
%USERPROFILE%
\rmq_srv_gc.log
" -XX:+PrintGCDetails"
set
"JAVA_OPT=
%JAVA_OPT%
-verbose:gc -Xloggc:"
%USERPROFILE%
\rmq_srv_gc.log
" -XX:+PrintGCDetails
-XX:+PrintGCDateStamps
"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:-OmitStackTraceInFastThrow"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:-OmitStackTraceInFastThrow"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:-UseLargePages"
set
"JAVA_OPT=
%JAVA_OPT%
-XX:-UseLargePages"
set
"JAVA_OPT=
%JAVA_OPT%
-Djava.ext.dirs=
%BASE_DIR%
lib;
%JAVA_HOME%
\jre\lib\ext"
set
"JAVA_OPT=
%JAVA_OPT%
-Djava.ext.dirs=
%BASE_DIR%
lib;
%JAVA_HOME%
\jre\lib\ext"
...
...
distribution/bin/runserver.sh
浏览文件 @
d21266d9
...
@@ -68,18 +68,18 @@ choose_gc_options()
...
@@ -68,18 +68,18 @@ choose_gc_options()
# '1' means releases befor Java 9
# '1' means releases befor Java 9
JAVA_MAJOR_VERSION
=
$(
"
$JAVA
"
-version
2>&1 |
sed
-r
-n
's/.* version "([0-9]*).*$/\1/p'
)
JAVA_MAJOR_VERSION
=
$(
"
$JAVA
"
-version
2>&1 |
sed
-r
-n
's/.* version "([0-9]*).*$/\1/p'
)
if
[
-z
"
$JAVA_MAJOR_VERSION
"
]
||
[
"
$JAVA_MAJOR_VERSION
"
-lt
"9"
]
;
then
if
[
-z
"
$JAVA_MAJOR_VERSION
"
]
||
[
"
$JAVA_MAJOR_VERSION
"
-lt
"9"
]
;
then
JAVA_OPT
=
"
${
JAVA_OPT
}
-server -Xms4g -Xmx4g -Xmn2g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseConcMarkSweepGC -XX:+UseCMSCompactAtFullCollection -XX:CMSInitiatingOccupancyFraction=70 -XX:+CMSParallelRemarkEnabled -XX:SoftRefLRUPolicyMSPerMB=0 -XX:+CMSClassUnloadingEnabled -XX:SurvivorRatio=8 -XX:-UseParNewGC"
JAVA_OPT
=
"
${
JAVA_OPT
}
-verbose:gc -Xloggc:
${
GC_LOG_DIR
}
/rmq_srv_gc_%p_%t.log -XX:+PrintGCDetails"
JAVA_OPT
=
"
${
JAVA_OPT
}
-verbose:gc -Xloggc:
${
GC_LOG_DIR
}
/rmq_srv_gc_%p_%t.log -XX:+PrintGCDetails
-XX:+PrintGCDateStamps
"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=30m"
else
else
JAVA_OPT
=
"
${
JAVA_OPT
}
-server -Xms4g -Xmx4g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:+UseG1GC -XX:G1HeapRegionSize=16m -XX:G1ReservePercent=25 -XX:InitiatingHeapOccupancyPercent=30 -XX:SoftRefLRUPolicyMSPerMB=0"
JAVA_OPT
=
"
${
JAVA_OPT
}
-Xlog:gc*:file=
${
GC_LOG_DIR
}
/rmq_srv_gc_%p_%t.log:time,tags:filecount=5,filesize=30M"
JAVA_OPT
=
"
${
JAVA_OPT
}
-Xlog:gc*:file=
${
GC_LOG_DIR
}
/rmq_srv_gc_%p_%t.log:time,tags:filecount=5,filesize=30M"
fi
fi
}
}
choose_gc_log_directory
choose_gc_log_directory
JAVA_OPT
=
"
${
JAVA_OPT
}
-server -Xms4g -Xmx4g -Xmn2g -XX:MetaspaceSize=128m -XX:MaxMetaspaceSize=320m"
choose_gc_options
choose_gc_options
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:-OmitStackTraceInFastThrow"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:-OmitStackTraceInFastThrow"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:-UseLargePages"
JAVA_OPT
=
"
${
JAVA_OPT
}
-XX:-UseLargePages"
...
...
remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettySystemConfig.java
浏览文件 @
d21266d9
...
@@ -37,7 +37,6 @@ public class NettySystemConfig {
...
@@ -37,7 +37,6 @@ public class NettySystemConfig {
public
static
final
String
COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT
=
public
static
final
String
COM_ROCKETMQ_REMOTING_CLIENT_CLOSE_SOCKET_IF_TIMEOUT
=
"com.rocketmq.remoting.client.closeSocketIfTimeout"
;
"com.rocketmq.remoting.client.closeSocketIfTimeout"
;
public
static
final
boolean
NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE
=
//
public
static
final
boolean
NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE
=
//
Boolean
.
parseBoolean
(
System
.
getProperty
(
COM_ROCKETMQ_REMOTING_NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE
,
"false"
));
Boolean
.
parseBoolean
(
System
.
getProperty
(
COM_ROCKETMQ_REMOTING_NETTY_POOLED_BYTE_BUF_ALLOCATOR_ENABLE
,
"false"
));
public
static
final
int
CLIENT_ASYNC_SEMAPHORE_VALUE
=
//
public
static
final
int
CLIENT_ASYNC_SEMAPHORE_VALUE
=
//
...
...
store/src/main/java/org/apache/rocketmq/store/AppendMessageCallback.java
浏览文件 @
d21266d9
...
@@ -18,6 +18,7 @@ package org.apache.rocketmq.store;
...
@@ -18,6 +18,7 @@ package org.apache.rocketmq.store;
import
java.nio.ByteBuffer
;
import
java.nio.ByteBuffer
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.store.CommitLog.PutMessageContext
;
/**
/**
* Write messages callback interface
* Write messages callback interface
...
@@ -30,7 +31,7 @@ public interface AppendMessageCallback {
...
@@ -30,7 +31,7 @@ public interface AppendMessageCallback {
* @return How many bytes to write
* @return How many bytes to write
*/
*/
AppendMessageResult
doAppend
(
final
long
fileFromOffset
,
final
ByteBuffer
byteBuffer
,
AppendMessageResult
doAppend
(
final
long
fileFromOffset
,
final
ByteBuffer
byteBuffer
,
final
int
maxBlank
,
final
MessageExtBrokerInner
msg
);
final
int
maxBlank
,
final
MessageExtBrokerInner
msg
,
PutMessageContext
putMessageContext
);
/**
/**
* After batched message serialization, write MapedByteBuffer
* After batched message serialization, write MapedByteBuffer
...
@@ -39,5 +40,5 @@ public interface AppendMessageCallback {
...
@@ -39,5 +40,5 @@ public interface AppendMessageCallback {
* @return How many bytes to write
* @return How many bytes to write
*/
*/
AppendMessageResult
doAppend
(
final
long
fileFromOffset
,
final
ByteBuffer
byteBuffer
,
AppendMessageResult
doAppend
(
final
long
fileFromOffset
,
final
ByteBuffer
byteBuffer
,
final
int
maxBlank
,
final
MessageExtBatch
messageExtBatch
);
final
int
maxBlank
,
final
MessageExtBatch
messageExtBatch
,
PutMessageContext
putMessageContext
);
}
}
store/src/main/java/org/apache/rocketmq/store/AppendMessageResult.java
浏览文件 @
d21266d9
...
@@ -16,6 +16,8 @@
...
@@ -16,6 +16,8 @@
*/
*/
package
org.apache.rocketmq.store
;
package
org.apache.rocketmq.store
;
import
java.util.function.Supplier
;
/**
/**
* When write a message to the commit log, returns results
* When write a message to the commit log, returns results
*/
*/
...
@@ -28,6 +30,7 @@ public class AppendMessageResult {
...
@@ -28,6 +30,7 @@ public class AppendMessageResult {
private
int
wroteBytes
;
private
int
wroteBytes
;
// Message ID
// Message ID
private
String
msgId
;
private
String
msgId
;
private
Supplier
<
String
>
msgIdSupplier
;
// Message storage timestamp
// Message storage timestamp
private
long
storeTimestamp
;
private
long
storeTimestamp
;
// Consume queue's offset(step by one)
// Consume queue's offset(step by one)
...
@@ -51,6 +54,17 @@ public class AppendMessageResult {
...
@@ -51,6 +54,17 @@ public class AppendMessageResult {
this
.
pagecacheRT
=
pagecacheRT
;
this
.
pagecacheRT
=
pagecacheRT
;
}
}
public
AppendMessageResult
(
AppendMessageStatus
status
,
long
wroteOffset
,
int
wroteBytes
,
Supplier
<
String
>
msgIdSupplier
,
long
storeTimestamp
,
long
logicsOffset
,
long
pagecacheRT
)
{
this
.
status
=
status
;
this
.
wroteOffset
=
wroteOffset
;
this
.
wroteBytes
=
wroteBytes
;
this
.
msgIdSupplier
=
msgIdSupplier
;
this
.
storeTimestamp
=
storeTimestamp
;
this
.
logicsOffset
=
logicsOffset
;
this
.
pagecacheRT
=
pagecacheRT
;
}
public
long
getPagecacheRT
()
{
public
long
getPagecacheRT
()
{
return
pagecacheRT
;
return
pagecacheRT
;
}
}
...
@@ -88,6 +102,9 @@ public class AppendMessageResult {
...
@@ -88,6 +102,9 @@ public class AppendMessageResult {
}
}
public
String
getMsgId
()
{
public
String
getMsgId
()
{
if
(
msgId
==
null
&&
msgIdSupplier
!=
null
)
{
msgId
=
msgIdSupplier
.
get
();
}
return
msgId
;
return
msgId
;
}
}
...
...
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
浏览文件 @
d21266d9
此差异已折叠。
点击以展开。
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
浏览文件 @
d21266d9
...
@@ -34,6 +34,7 @@ import java.util.Set;
...
@@ -34,6 +34,7 @@ import java.util.Set;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.ConcurrentHashMap
;
import
java.util.concurrent.ConcurrentHashMap
;
import
java.util.concurrent.ConcurrentMap
;
import
java.util.concurrent.ConcurrentMap
;
import
java.util.concurrent.ExecutionException
;
import
java.util.concurrent.Executors
;
import
java.util.concurrent.Executors
;
import
java.util.concurrent.ScheduledExecutorService
;
import
java.util.concurrent.ScheduledExecutorService
;
import
java.util.concurrent.TimeUnit
;
import
java.util.concurrent.TimeUnit
;
...
@@ -476,58 +477,20 @@ public class DefaultMessageStore implements MessageStore {
...
@@ -476,58 +477,20 @@ public class DefaultMessageStore implements MessageStore {
@Override
@Override
public
PutMessageResult
putMessage
(
MessageExtBrokerInner
msg
)
{
public
PutMessageResult
putMessage
(
MessageExtBrokerInner
msg
)
{
PutMessageStatus
checkStoreStatus
=
this
.
checkStoreStatus
();
try
{
if
(
checkStoreStatus
!=
PutMessageStatus
.
PUT_OK
)
{
return
asyncPutMessage
(
msg
).
get
();
return
new
PutMessageResult
(
checkStoreStatus
,
null
);
}
catch
(
InterruptedException
|
ExecutionException
e
)
{
}
return
new
PutMessageResult
(
PutMessageStatus
.
UNKNOWN_ERROR
,
null
);
PutMessageStatus
msgCheckStatus
=
this
.
checkMessage
(
msg
);
if
(
msgCheckStatus
==
PutMessageStatus
.
MESSAGE_ILLEGAL
)
{
return
new
PutMessageResult
(
msgCheckStatus
,
null
);
}
long
beginTime
=
this
.
getSystemClock
().
now
();
PutMessageResult
result
=
this
.
commitLog
.
putMessage
(
msg
);
long
elapsedTime
=
this
.
getSystemClock
().
now
()
-
beginTime
;
if
(
elapsedTime
>
500
)
{
log
.
warn
(
"not in lock elapsed time(ms)={}, bodyLength={}"
,
elapsedTime
,
msg
.
getBody
().
length
);
}
this
.
storeStatsService
.
setPutMessageEntireTimeMax
(
elapsedTime
);
if
(
null
==
result
||
!
result
.
isOk
())
{
this
.
storeStatsService
.
getPutMessageFailedTimes
().
incrementAndGet
();
}
}
return
result
;
}
}
@Override
@Override
public
PutMessageResult
putMessages
(
MessageExtBatch
messageExtBatch
)
{
public
PutMessageResult
putMessages
(
MessageExtBatch
messageExtBatch
)
{
PutMessageStatus
checkStoreStatus
=
this
.
checkStoreStatus
();
try
{
if
(
checkStoreStatus
!=
PutMessageStatus
.
PUT_OK
)
{
return
asyncPutMessages
(
messageExtBatch
).
get
();
return
new
PutMessageResult
(
checkStoreStatus
,
null
);
}
catch
(
InterruptedException
|
ExecutionException
e
)
{
}
return
new
PutMessageResult
(
PutMessageStatus
.
UNKNOWN_ERROR
,
null
);
PutMessageStatus
msgCheckStatus
=
this
.
checkMessages
(
messageExtBatch
);
if
(
msgCheckStatus
==
PutMessageStatus
.
MESSAGE_ILLEGAL
)
{
return
new
PutMessageResult
(
msgCheckStatus
,
null
);
}
long
beginTime
=
this
.
getSystemClock
().
now
();
PutMessageResult
result
=
this
.
commitLog
.
putMessages
(
messageExtBatch
);
long
elapsedTime
=
this
.
getSystemClock
().
now
()
-
beginTime
;
if
(
elapsedTime
>
500
)
{
log
.
warn
(
"not in lock elapsed time(ms)={}, bodyLength={}"
,
elapsedTime
,
messageExtBatch
.
getBody
().
length
);
}
this
.
storeStatsService
.
setPutMessageEntireTimeMax
(
elapsedTime
);
if
(
null
==
result
||
!
result
.
isOk
())
{
this
.
storeStatsService
.
getPutMessageFailedTimes
().
incrementAndGet
();
}
}
return
result
;
}
}
@Override
@Override
...
...
store/src/main/java/org/apache/rocketmq/store/MappedFile.java
浏览文件 @
d21266d9
...
@@ -37,6 +37,7 @@ import org.apache.rocketmq.logging.InternalLogger;
...
@@ -37,6 +37,7 @@ import org.apache.rocketmq.logging.InternalLogger;
import
org.apache.rocketmq.logging.InternalLoggerFactory
;
import
org.apache.rocketmq.logging.InternalLoggerFactory
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.store.CommitLog.PutMessageContext
;
import
org.apache.rocketmq.store.config.FlushDiskType
;
import
org.apache.rocketmq.store.config.FlushDiskType
;
import
org.apache.rocketmq.store.util.LibC
;
import
org.apache.rocketmq.store.util.LibC
;
import
sun.nio.ch.DirectBuffer
;
import
sun.nio.ch.DirectBuffer
;
...
@@ -188,15 +189,18 @@ public class MappedFile extends ReferenceResource {
...
@@ -188,15 +189,18 @@ public class MappedFile extends ReferenceResource {
return
fileChannel
;
return
fileChannel
;
}
}
public
AppendMessageResult
appendMessage
(
final
MessageExtBrokerInner
msg
,
final
AppendMessageCallback
cb
)
{
public
AppendMessageResult
appendMessage
(
final
MessageExtBrokerInner
msg
,
final
AppendMessageCallback
cb
,
return
appendMessagesInner
(
msg
,
cb
);
PutMessageContext
putMessageContext
)
{
return
appendMessagesInner
(
msg
,
cb
,
putMessageContext
);
}
}
public
AppendMessageResult
appendMessages
(
final
MessageExtBatch
messageExtBatch
,
final
AppendMessageCallback
cb
)
{
public
AppendMessageResult
appendMessages
(
final
MessageExtBatch
messageExtBatch
,
final
AppendMessageCallback
cb
,
return
appendMessagesInner
(
messageExtBatch
,
cb
);
PutMessageContext
putMessageContext
)
{
return
appendMessagesInner
(
messageExtBatch
,
cb
,
putMessageContext
);
}
}
public
AppendMessageResult
appendMessagesInner
(
final
MessageExt
messageExt
,
final
AppendMessageCallback
cb
)
{
public
AppendMessageResult
appendMessagesInner
(
final
MessageExt
messageExt
,
final
AppendMessageCallback
cb
,
PutMessageContext
putMessageContext
)
{
assert
messageExt
!=
null
;
assert
messageExt
!=
null
;
assert
cb
!=
null
;
assert
cb
!=
null
;
...
@@ -207,9 +211,11 @@ public class MappedFile extends ReferenceResource {
...
@@ -207,9 +211,11 @@ public class MappedFile extends ReferenceResource {
byteBuffer
.
position
(
currentPos
);
byteBuffer
.
position
(
currentPos
);
AppendMessageResult
result
;
AppendMessageResult
result
;
if
(
messageExt
instanceof
MessageExtBrokerInner
)
{
if
(
messageExt
instanceof
MessageExtBrokerInner
)
{
result
=
cb
.
doAppend
(
this
.
getFileFromOffset
(),
byteBuffer
,
this
.
fileSize
-
currentPos
,
(
MessageExtBrokerInner
)
messageExt
);
result
=
cb
.
doAppend
(
this
.
getFileFromOffset
(),
byteBuffer
,
this
.
fileSize
-
currentPos
,
(
MessageExtBrokerInner
)
messageExt
,
putMessageContext
);
}
else
if
(
messageExt
instanceof
MessageExtBatch
)
{
}
else
if
(
messageExt
instanceof
MessageExtBatch
)
{
result
=
cb
.
doAppend
(
this
.
getFileFromOffset
(),
byteBuffer
,
this
.
fileSize
-
currentPos
,
(
MessageExtBatch
)
messageExt
);
result
=
cb
.
doAppend
(
this
.
getFileFromOffset
(),
byteBuffer
,
this
.
fileSize
-
currentPos
,
(
MessageExtBatch
)
messageExt
,
putMessageContext
);
}
else
{
}
else
{
return
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
);
return
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
);
}
}
...
...
store/src/main/java/org/apache/rocketmq/store/MessageExtBrokerInner.java
浏览文件 @
d21266d9
...
@@ -16,6 +16,8 @@
...
@@ -16,6 +16,8 @@
*/
*/
package
org.apache.rocketmq.store
;
package
org.apache.rocketmq.store
;
import
java.nio.ByteBuffer
;
import
org.apache.rocketmq.common.TopicFilterType
;
import
org.apache.rocketmq.common.TopicFilterType
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageExt
;
...
@@ -24,6 +26,16 @@ public class MessageExtBrokerInner extends MessageExt {
...
@@ -24,6 +26,16 @@ public class MessageExtBrokerInner extends MessageExt {
private
String
propertiesString
;
private
String
propertiesString
;
private
long
tagsCode
;
private
long
tagsCode
;
private
ByteBuffer
encodedBuff
;
public
ByteBuffer
getEncodedBuff
()
{
return
encodedBuff
;
}
public
void
setEncodedBuff
(
ByteBuffer
encodedBuff
)
{
this
.
encodedBuff
=
encodedBuff
;
}
public
static
long
tagsString2tagsCode
(
final
TopicFilterType
filter
,
final
String
tags
)
{
public
static
long
tagsString2tagsCode
(
final
TopicFilterType
filter
,
final
String
tags
)
{
if
(
null
==
tags
||
tags
.
length
()
==
0
)
{
return
0
;
}
if
(
null
==
tags
||
tags
.
length
()
==
0
)
{
return
0
;
}
...
...
store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
浏览文件 @
d21266d9
...
@@ -37,7 +37,6 @@ import java.util.HashMap;
...
@@ -37,7 +37,6 @@ import java.util.HashMap;
import
java.util.LinkedList
;
import
java.util.LinkedList
;
import
java.util.List
;
import
java.util.List
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.CompletableFuture
;
import
java.util.concurrent.TimeUnit
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.apache.rocketmq.common.message.MessageAccessor
;
import
org.apache.rocketmq.common.message.MessageAccessor
;
import
org.apache.rocketmq.common.message.MessageConst
;
import
org.apache.rocketmq.common.message.MessageConst
;
...
@@ -413,237 +412,6 @@ public class DLedgerCommitLog extends CommitLog {
...
@@ -413,237 +412,6 @@ public class DLedgerCommitLog extends CommitLog {
}
}
}
}
@Override
public
PutMessageResult
putMessage
(
final
MessageExtBrokerInner
msg
)
{
StoreStatsService
storeStatsService
=
this
.
defaultMessageStore
.
getStoreStatsService
();
final
int
tranType
=
MessageSysFlag
.
getTransactionValue
(
msg
.
getSysFlag
());
String
topic
=
msg
.
getTopic
();
setMessageInfo
(
msg
,
tranType
);
// Back to Results
AppendMessageResult
appendResult
;
AppendFuture
<
AppendEntryResponse
>
dledgerFuture
;
EncodeResult
encodeResult
;
encodeResult
=
this
.
messageSerializer
.
serialize
(
msg
);
if
(
encodeResult
.
status
!=
AppendMessageStatus
.
PUT_OK
)
{
return
new
PutMessageResult
(
PutMessageStatus
.
MESSAGE_ILLEGAL
,
new
AppendMessageResult
(
encodeResult
.
status
));
}
putMessageLock
.
lock
();
//spin or ReentrantLock ,depending on store config
long
elapsedTimeInLock
;
long
queueOffset
;
try
{
beginTimeInDledgerLock
=
this
.
defaultMessageStore
.
getSystemClock
().
now
();
queueOffset
=
getQueueOffsetByKey
(
encodeResult
.
queueOffsetKey
,
tranType
);
encodeResult
.
setQueueOffsetKey
(
queueOffset
,
false
);
AppendEntryRequest
request
=
new
AppendEntryRequest
();
request
.
setGroup
(
dLedgerConfig
.
getGroup
());
request
.
setRemoteId
(
dLedgerServer
.
getMemberState
().
getSelfId
());
request
.
setBody
(
encodeResult
.
getData
());
dledgerFuture
=
(
AppendFuture
<
AppendEntryResponse
>)
dLedgerServer
.
handleAppend
(
request
);
if
(
dledgerFuture
.
getPos
()
==
-
1
)
{
return
new
PutMessageResult
(
PutMessageStatus
.
OS_PAGECACHE_BUSY
,
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
));
}
long
wroteOffset
=
dledgerFuture
.
getPos
()
+
DLedgerEntry
.
BODY_OFFSET
;
int
msgIdLength
=
(
msg
.
getSysFlag
()
&
MessageSysFlag
.
STOREHOSTADDRESS_V6_FLAG
)
==
0
?
4
+
4
+
8
:
16
+
4
+
8
;
ByteBuffer
buffer
=
ByteBuffer
.
allocate
(
msgIdLength
);
String
msgId
=
MessageDecoder
.
createMessageId
(
buffer
,
msg
.
getStoreHostBytes
(),
wroteOffset
);
elapsedTimeInLock
=
this
.
defaultMessageStore
.
getSystemClock
().
now
()
-
beginTimeInDledgerLock
;
appendResult
=
new
AppendMessageResult
(
AppendMessageStatus
.
PUT_OK
,
wroteOffset
,
encodeResult
.
getData
().
length
,
msgId
,
System
.
currentTimeMillis
(),
queueOffset
,
elapsedTimeInLock
);
switch
(
tranType
)
{
case
MessageSysFlag
.
TRANSACTION_PREPARED_TYPE
:
case
MessageSysFlag
.
TRANSACTION_ROLLBACK_TYPE
:
break
;
case
MessageSysFlag
.
TRANSACTION_NOT_TYPE
:
case
MessageSysFlag
.
TRANSACTION_COMMIT_TYPE
:
// The next update ConsumeQueue information
DLedgerCommitLog
.
this
.
topicQueueTable
.
put
(
encodeResult
.
queueOffsetKey
,
queueOffset
+
1
);
break
;
default
:
break
;
}
}
catch
(
Exception
e
)
{
log
.
error
(
"Put message error"
,
e
);
return
new
PutMessageResult
(
PutMessageStatus
.
UNKNOWN_ERROR
,
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
));
}
finally
{
beginTimeInDledgerLock
=
0
;
putMessageLock
.
unlock
();
}
if
(
elapsedTimeInLock
>
500
)
{
log
.
warn
(
"[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}"
,
elapsedTimeInLock
,
msg
.
getBody
().
length
,
appendResult
);
}
PutMessageStatus
putMessageStatus
=
PutMessageStatus
.
UNKNOWN_ERROR
;
try
{
AppendEntryResponse
appendEntryResponse
=
dledgerFuture
.
get
(
3
,
TimeUnit
.
SECONDS
);
switch
(
DLedgerResponseCode
.
valueOf
(
appendEntryResponse
.
getCode
()))
{
case
SUCCESS:
putMessageStatus
=
PutMessageStatus
.
PUT_OK
;
break
;
case
INCONSISTENT_LEADER:
case
NOT_LEADER:
case
LEADER_NOT_READY:
case
DISK_FULL:
putMessageStatus
=
PutMessageStatus
.
SERVICE_NOT_AVAILABLE
;
break
;
case
WAIT_QUORUM_ACK_TIMEOUT:
//Do not return flush_slave_timeout to the client, for the ons client will ignore it.
putMessageStatus
=
PutMessageStatus
.
OS_PAGECACHE_BUSY
;
break
;
case
LEADER_PENDING_FULL:
putMessageStatus
=
PutMessageStatus
.
OS_PAGECACHE_BUSY
;
break
;
}
}
catch
(
Throwable
t
)
{
log
.
error
(
"Failed to get dledger append result"
,
t
);
}
PutMessageResult
putMessageResult
=
new
PutMessageResult
(
putMessageStatus
,
appendResult
);
if
(
putMessageStatus
==
PutMessageStatus
.
PUT_OK
)
{
// Statistics
storeStatsService
.
getSinglePutMessageTopicTimesTotal
(
msg
.
getTopic
()).
incrementAndGet
();
storeStatsService
.
getSinglePutMessageTopicSizeTotal
(
topic
).
addAndGet
(
appendResult
.
getWroteBytes
());
}
return
putMessageResult
;
}
@Override
public
PutMessageResult
putMessages
(
final
MessageExtBatch
messageExtBatch
)
{
final
int
tranType
=
MessageSysFlag
.
getTransactionValue
(
messageExtBatch
.
getSysFlag
());
if
(
tranType
!=
MessageSysFlag
.
TRANSACTION_NOT_TYPE
)
{
return
new
PutMessageResult
(
PutMessageStatus
.
MESSAGE_ILLEGAL
,
null
);
}
if
(
messageExtBatch
.
getDelayTimeLevel
()
>
0
)
{
return
new
PutMessageResult
(
PutMessageStatus
.
MESSAGE_ILLEGAL
,
null
);
}
// Set the storage time
messageExtBatch
.
setStoreTimestamp
(
System
.
currentTimeMillis
());
StoreStatsService
storeStatsService
=
this
.
defaultMessageStore
.
getStoreStatsService
();
InetSocketAddress
bornSocketAddress
=
(
InetSocketAddress
)
messageExtBatch
.
getBornHost
();
if
(
bornSocketAddress
.
getAddress
()
instanceof
Inet6Address
)
{
messageExtBatch
.
setBornHostV6Flag
();
}
InetSocketAddress
storeSocketAddress
=
(
InetSocketAddress
)
messageExtBatch
.
getStoreHost
();
if
(
storeSocketAddress
.
getAddress
()
instanceof
Inet6Address
)
{
messageExtBatch
.
setStoreHostAddressV6Flag
();
}
// Back to Results
AppendMessageResult
appendResult
;
BatchAppendFuture
<
AppendEntryResponse
>
dledgerFuture
;
EncodeResult
encodeResult
;
encodeResult
=
this
.
messageSerializer
.
serialize
(
messageExtBatch
);
if
(
encodeResult
.
status
!=
AppendMessageStatus
.
PUT_OK
)
{
return
new
PutMessageResult
(
PutMessageStatus
.
MESSAGE_ILLEGAL
,
new
AppendMessageResult
(
encodeResult
.
status
));
}
putMessageLock
.
lock
();
//spin or ReentrantLock ,depending on store config
msgIdBuilder
.
setLength
(
0
);
long
elapsedTimeInLock
;
long
queueOffset
;
int
msgNum
=
0
;
try
{
beginTimeInDledgerLock
=
this
.
defaultMessageStore
.
getSystemClock
().
now
();
queueOffset
=
getQueueOffsetByKey
(
encodeResult
.
queueOffsetKey
,
tranType
);
encodeResult
.
setQueueOffsetKey
(
queueOffset
,
true
);
BatchAppendEntryRequest
request
=
new
BatchAppendEntryRequest
();
request
.
setGroup
(
dLedgerConfig
.
getGroup
());
request
.
setRemoteId
(
dLedgerServer
.
getMemberState
().
getSelfId
());
request
.
setBatchMsgs
(
encodeResult
.
batchData
);
AppendFuture
<
AppendEntryResponse
>
appendFuture
=
(
AppendFuture
<
AppendEntryResponse
>)
dLedgerServer
.
handleAppend
(
request
);
if
(
appendFuture
.
getPos
()
==
-
1
)
{
log
.
warn
(
"HandleAppend return false due to error code {}"
,
appendFuture
.
get
().
getCode
());
return
new
PutMessageResult
(
PutMessageStatus
.
OS_PAGECACHE_BUSY
,
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
));
}
dledgerFuture
=
(
BatchAppendFuture
<
AppendEntryResponse
>)
appendFuture
;
long
wroteOffset
=
0
;
int
msgIdLength
=
(
messageExtBatch
.
getSysFlag
()
&
MessageSysFlag
.
STOREHOSTADDRESS_V6_FLAG
)
==
0
?
4
+
4
+
8
:
16
+
4
+
8
;
ByteBuffer
buffer
=
ByteBuffer
.
allocate
(
msgIdLength
);
boolean
isFirstOffset
=
true
;
long
firstWroteOffset
=
0
;
for
(
long
pos
:
dledgerFuture
.
getPositions
())
{
wroteOffset
=
pos
+
DLedgerEntry
.
BODY_OFFSET
;
if
(
isFirstOffset
)
{
firstWroteOffset
=
wroteOffset
;
isFirstOffset
=
false
;
}
String
msgId
=
MessageDecoder
.
createMessageId
(
buffer
,
messageExtBatch
.
getStoreHostBytes
(),
wroteOffset
);
if
(
msgIdBuilder
.
length
()
>
0
)
{
msgIdBuilder
.
append
(
','
).
append
(
msgId
);
}
else
{
msgIdBuilder
.
append
(
msgId
);
}
msgNum
++;
}
elapsedTimeInLock
=
this
.
defaultMessageStore
.
getSystemClock
().
now
()
-
beginTimeInDledgerLock
;
appendResult
=
new
AppendMessageResult
(
AppendMessageStatus
.
PUT_OK
,
firstWroteOffset
,
encodeResult
.
totalMsgLen
,
msgIdBuilder
.
toString
(),
System
.
currentTimeMillis
(),
queueOffset
,
elapsedTimeInLock
);
appendResult
.
setMsgNum
(
msgNum
);
DLedgerCommitLog
.
this
.
topicQueueTable
.
put
(
encodeResult
.
queueOffsetKey
,
queueOffset
+
msgNum
);
}
catch
(
Exception
e
)
{
log
.
error
(
"Put message error"
,
e
);
return
new
PutMessageResult
(
PutMessageStatus
.
UNKNOWN_ERROR
,
new
AppendMessageResult
(
AppendMessageStatus
.
UNKNOWN_ERROR
));
}
finally
{
beginTimeInDledgerLock
=
0
;
putMessageLock
.
unlock
();
}
if
(
elapsedTimeInLock
>
500
)
{
log
.
warn
(
"[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}"
,
elapsedTimeInLock
,
messageExtBatch
.
getBody
().
length
,
appendResult
);
}
PutMessageStatus
putMessageStatus
=
PutMessageStatus
.
UNKNOWN_ERROR
;
try
{
AppendEntryResponse
appendEntryResponse
=
dledgerFuture
.
get
(
3
,
TimeUnit
.
SECONDS
);
switch
(
DLedgerResponseCode
.
valueOf
(
appendEntryResponse
.
getCode
()))
{
case
SUCCESS:
putMessageStatus
=
PutMessageStatus
.
PUT_OK
;
break
;
case
INCONSISTENT_LEADER:
case
NOT_LEADER:
case
LEADER_NOT_READY:
case
DISK_FULL:
putMessageStatus
=
PutMessageStatus
.
SERVICE_NOT_AVAILABLE
;
break
;
case
WAIT_QUORUM_ACK_TIMEOUT:
//Do not return flush_slave_timeout to the client, for the ons client will ignore it.
putMessageStatus
=
PutMessageStatus
.
OS_PAGECACHE_BUSY
;
break
;
case
LEADER_PENDING_FULL:
putMessageStatus
=
PutMessageStatus
.
OS_PAGECACHE_BUSY
;
break
;
}
}
catch
(
Throwable
t
)
{
log
.
error
(
"Failed to get dledger append result"
,
t
);
}
PutMessageResult
putMessageResult
=
new
PutMessageResult
(
putMessageStatus
,
appendResult
);
if
(
putMessageStatus
==
PutMessageStatus
.
PUT_OK
)
{
// Statistics
storeStatsService
.
getSinglePutMessageTopicTimesTotal
(
messageExtBatch
.
getTopic
()).
addAndGet
(
msgNum
);
storeStatsService
.
getSinglePutMessageTopicSizeTotal
(
messageExtBatch
.
getTopic
()).
addAndGet
(
encodeResult
.
totalMsgLen
);
}
return
putMessageResult
;
}
@Override
@Override
public
CompletableFuture
<
PutMessageResult
>
asyncPutMessage
(
MessageExtBrokerInner
msg
)
{
public
CompletableFuture
<
PutMessageResult
>
asyncPutMessage
(
MessageExtBrokerInner
msg
)
{
...
...
store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java
浏览文件 @
d21266d9
...
@@ -30,6 +30,8 @@ import org.apache.rocketmq.common.message.Message;
...
@@ -30,6 +30,8 @@ import org.apache.rocketmq.common.message.Message;
import
org.apache.rocketmq.common.message.MessageDecoder
;
import
org.apache.rocketmq.common.message.MessageDecoder
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.common.message.MessageExtBatch
;
import
org.apache.rocketmq.store.CommitLog.MessageExtEncoder
;
import
org.apache.rocketmq.store.CommitLog.PutMessageContext
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.junit.After
;
import
org.junit.After
;
import
org.junit.Before
;
import
org.junit.Before
;
...
@@ -42,7 +44,7 @@ public class AppendCallbackTest {
...
@@ -42,7 +44,7 @@ public class AppendCallbackTest {
AppendMessageCallback
callback
;
AppendMessageCallback
callback
;
CommitLog
.
MessageExtBatchEncoder
batchEncoder
=
new
CommitLog
.
MessageExtBatch
Encoder
(
10
*
1024
*
1024
);
MessageExtEncoder
batchEncoder
=
new
MessageExt
Encoder
(
10
*
1024
*
1024
);
@Before
@Before
public
void
init
()
throws
Exception
{
public
void
init
()
throws
Exception
{
...
@@ -84,10 +86,12 @@ public class AppendCallbackTest {
...
@@ -84,10 +86,12 @@ public class AppendCallbackTest {
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"127.0.0.1"
,
124
));
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"127.0.0.1"
,
124
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
));
PutMessageContext
putMessageContext
=
new
PutMessageContext
(
topic
+
"-"
+
queue
);
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
,
putMessageContext
));
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
//encounter end of file when append half of the data
//encounter end of file when append half of the data
AppendMessageResult
result
=
callback
.
doAppend
(
0
,
buff
,
1000
,
messageExtBatch
);
AppendMessageResult
result
=
callback
.
doAppend
(
0
,
buff
,
1000
,
messageExtBatch
,
putMessageContext
);
assertEquals
(
AppendMessageStatus
.
END_OF_FILE
,
result
.
getStatus
());
assertEquals
(
AppendMessageStatus
.
END_OF_FILE
,
result
.
getStatus
());
assertEquals
(
0
,
result
.
getWroteOffset
());
assertEquals
(
0
,
result
.
getWroteOffset
());
assertEquals
(
0
,
result
.
getLogicsOffset
());
assertEquals
(
0
,
result
.
getLogicsOffset
());
...
@@ -121,10 +125,12 @@ public class AppendCallbackTest {
...
@@ -121,10 +125,12 @@ public class AppendCallbackTest {
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"::1"
,
124
));
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"::1"
,
124
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
));
PutMessageContext
putMessageContext
=
new
PutMessageContext
(
topic
+
"-"
+
queue
);
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
,
putMessageContext
));
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
//encounter end of file when append half of the data
//encounter end of file when append half of the data
AppendMessageResult
result
=
callback
.
doAppend
(
0
,
buff
,
1000
,
messageExtBatch
);
AppendMessageResult
result
=
callback
.
doAppend
(
0
,
buff
,
1000
,
messageExtBatch
,
putMessageContext
);
assertEquals
(
AppendMessageStatus
.
END_OF_FILE
,
result
.
getStatus
());
assertEquals
(
AppendMessageStatus
.
END_OF_FILE
,
result
.
getStatus
());
assertEquals
(
0
,
result
.
getWroteOffset
());
assertEquals
(
0
,
result
.
getWroteOffset
());
assertEquals
(
0
,
result
.
getLogicsOffset
());
assertEquals
(
0
,
result
.
getLogicsOffset
());
...
@@ -154,9 +160,11 @@ public class AppendCallbackTest {
...
@@ -154,9 +160,11 @@ public class AppendCallbackTest {
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"127.0.0.1"
,
124
));
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"127.0.0.1"
,
124
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
));
PutMessageContext
putMessageContext
=
new
PutMessageContext
(
topic
+
"-"
+
queue
);
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
,
putMessageContext
));
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
AppendMessageResult
allresult
=
callback
.
doAppend
(
0
,
buff
,
1024
*
10
,
messageExtBatch
);
AppendMessageResult
allresult
=
callback
.
doAppend
(
0
,
buff
,
1024
*
10
,
messageExtBatch
,
putMessageContext
);
assertEquals
(
AppendMessageStatus
.
PUT_OK
,
allresult
.
getStatus
());
assertEquals
(
AppendMessageStatus
.
PUT_OK
,
allresult
.
getStatus
());
assertEquals
(
0
,
allresult
.
getWroteOffset
());
assertEquals
(
0
,
allresult
.
getWroteOffset
());
...
@@ -214,9 +222,11 @@ public class AppendCallbackTest {
...
@@ -214,9 +222,11 @@ public class AppendCallbackTest {
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"::1"
,
124
));
messageExtBatch
.
setStoreHost
(
new
InetSocketAddress
(
"::1"
,
124
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setBody
(
MessageDecoder
.
encodeMessages
(
messages
));
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
));
PutMessageContext
putMessageContext
=
new
PutMessageContext
(
topic
+
"-"
+
queue
);
messageExtBatch
.
setEncodedBuff
(
batchEncoder
.
encode
(
messageExtBatch
,
putMessageContext
));
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
ByteBuffer
buff
=
ByteBuffer
.
allocate
(
1024
*
10
);
AppendMessageResult
allresult
=
callback
.
doAppend
(
0
,
buff
,
1024
*
10
,
messageExtBatch
);
AppendMessageResult
allresult
=
callback
.
doAppend
(
0
,
buff
,
1024
*
10
,
messageExtBatch
,
putMessageContext
);
assertEquals
(
AppendMessageStatus
.
PUT_OK
,
allresult
.
getStatus
());
assertEquals
(
AppendMessageStatus
.
PUT_OK
,
allresult
.
getStatus
());
assertEquals
(
0
,
allresult
.
getWroteOffset
());
assertEquals
(
0
,
allresult
.
getWroteOffset
());
...
...
tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
浏览文件 @
d21266d9
...
@@ -164,9 +164,9 @@ public class UpdateAccessConfigSubCommand implements SubCommand {
...
@@ -164,9 +164,9 @@ public class UpdateAccessConfigSubCommand implements SubCommand {
String
clusterName
=
commandLine
.
getOptionValue
(
'c'
).
trim
();
String
clusterName
=
commandLine
.
getOptionValue
(
'c'
).
trim
();
defaultMQAdminExt
.
start
();
defaultMQAdminExt
.
start
();
Set
<
String
>
maste
rSet
=
Set
<
String
>
brokerAdd
rSet
=
CommandUtil
.
fetchMasterAddrByClusterName
(
defaultMQAdminExt
,
clusterName
);
CommandUtil
.
fetchMasterA
ndSlaveA
ddrByClusterName
(
defaultMQAdminExt
,
clusterName
);
for
(
String
addr
:
maste
rSet
)
{
for
(
String
addr
:
brokerAdd
rSet
)
{
defaultMQAdminExt
.
createAndUpdatePlainAccessConfig
(
addr
,
accessConfig
);
defaultMQAdminExt
.
createAndUpdatePlainAccessConfig
(
addr
,
accessConfig
);
System
.
out
.
printf
(
"create or update plain access config to %s success.%n"
,
addr
);
System
.
out
.
printf
(
"create or update plain access config to %s success.%n"
,
addr
);
}
}
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录