Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
Apache RocketMQ
Rocketmq
提交
515bc353
R
Rocketmq
项目概览
Apache RocketMQ
/
Rocketmq
上一次同步 大约 3 年
通知
268
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看板
提交
515bc353
编写于
11月 05, 2018
作者:
D
dongeforever
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
Add dleger commitlog with tests
上级
0f153b9d
变更
13
展开全部
隐藏空白更改
内联
并排
Showing
13 changed file
with
914 addition
and
16 deletion
+914
-16
common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
...va/org/apache/rocketmq/common/message/MessageDecoder.java
+1
-0
pom.xml
pom.xml
+2
-2
store/pom.xml
store/pom.xml
+15
-0
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
+8
-8
store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
...src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
+1
-1
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
...n/java/org/apache/rocketmq/store/DefaultMessageStore.java
+7
-2
store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java
.../main/java/org/apache/rocketmq/store/DispatchRequest.java
+15
-1
store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java
...a/org/apache/rocketmq/store/SelectMappedBufferResult.java
+2
-2
store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
.../org/apache/rocketmq/store/config/MessageStoreConfig.java
+36
-0
store/src/main/java/org/apache/rocketmq/store/dleger/DLegerCommitLog.java
...ava/org/apache/rocketmq/store/dleger/DLegerCommitLog.java
+629
-0
store/src/main/java/org/apache/rocketmq/store/dleger/DLegerSelectMappedBufferResult.java
...rocketmq/store/dleger/DLegerSelectMappedBufferResult.java
+36
-0
store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java
...rc/test/java/org/apache/rocketmq/store/StoreTestBase.java
+69
-0
store/src/test/java/org/apache/rocketmq/store/dleger/DLegerCommitlogTest.java
...org/apache/rocketmq/store/dleger/DLegerCommitlogTest.java
+93
-0
未找到文件。
common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java
浏览文件 @
515bc353
...
...
@@ -41,6 +41,7 @@ public class MessageDecoder {
public
final
static
int
MESSAGE_MAGIC_CODE
=
-
626843481
;
public
static
final
char
NAME_VALUE_SEPARATOR
=
1
;
public
static
final
char
PROPERTY_SEPARATOR
=
2
;
public
static
final
int
PHY_POS_POSITION
=
4
+
4
+
4
+
4
+
4
+
8
;
public
static
final
int
BODY_SIZE_POSITION
=
4
// 1 TOTALSIZE
+
4
// 2 MAGICCODE
+
4
// 3 BODYCRC
...
...
pom.xml
浏览文件 @
515bc353
...
...
@@ -100,8 +100,8 @@
<maven.test.skip>
false
</maven.test.skip>
<maven.javadoc.skip>
true
</maven.javadoc.skip>
<!-- Compiler settings properties -->
<maven.compiler.source>
1.
7
</maven.compiler.source>
<maven.compiler.target>
1.
7
</maven.compiler.target>
<maven.compiler.source>
1.
8
</maven.compiler.source>
<maven.compiler.target>
1.
8
</maven.compiler.target>
<sonar.java.coveragePlugin>
jacoco
</sonar.java.coveragePlugin>
<!-- Exclude all generated code -->
<sonar.jacoco.itReportPath>
${project.basedir}/../test/target/jacoco-it.exec
</sonar.jacoco.itReportPath>
...
...
store/pom.xml
浏览文件 @
515bc353
...
...
@@ -28,6 +28,21 @@
<name>
rocketmq-store ${project.version}
</name>
<dependencies>
<dependency>
<groupId>
org.apache.rocketmq
</groupId>
<artifactId>
rocketmq-dleger
</artifactId>
<version>
0.1-SNAPSHOT
</version>
<exclusions>
<exclusion>
<groupId>
org.apache.rocketmq
</groupId>
<artifactId>
rocketmq-remoting
</artifactId>
</exclusion>
<exclusion>
<groupId>
org.slf4j
</groupId>
<artifactId>
slf4j-log4j12
</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>
${project.groupId}
</groupId>
<artifactId>
rocketmq-common
</artifactId>
...
...
store/src/main/java/org/apache/rocketmq/store/CommitLog.java
浏览文件 @
515bc353
...
...
@@ -45,11 +45,11 @@ import org.apache.rocketmq.store.schedule.ScheduleMessageService;
public
class
CommitLog
{
// Message's MAGIC CODE daa320a7
public
final
static
int
MESSAGE_MAGIC_CODE
=
-
626843481
;
pr
ivate
static
final
InternalLogger
log
=
InternalLoggerFactory
.
getLogger
(
LoggerName
.
STORE_LOGGER_NAME
);
pr
otected
static
final
InternalLogger
log
=
InternalLoggerFactory
.
getLogger
(
LoggerName
.
STORE_LOGGER_NAME
);
// End of file empty MAGIC CODE cbd43194
pr
ivate
final
static
int
BLANK_MAGIC_CODE
=
-
875286124
;
pr
otected
final
static
int
BLANK_MAGIC_CODE
=
-
875286124
;
private
final
MappedFileQueue
mappedFileQueue
;
pr
ivate
final
DefaultMessageStore
defaultMessageStore
;
pr
otected
final
DefaultMessageStore
defaultMessageStore
;
private
final
FlushCommitLogService
flushCommitLogService
;
//If TransientStorePool enabled, we must flush message to FileChannel at fixed periods
...
...
@@ -57,11 +57,11 @@ public class CommitLog {
private
final
AppendMessageCallback
appendMessageCallback
;
private
final
ThreadLocal
<
MessageExtBatchEncoder
>
batchEncoderThreadLocal
;
pr
ivate
HashMap
<
String
/* topic-queueid */
,
Long
/* offset */
>
topicQueueTable
=
new
HashMap
<
String
,
Long
>(
1024
);
pr
ivate
volatile
long
confirmOffset
=
-
1L
;
pr
otected
HashMap
<
String
/* topic-queueid */
,
Long
/* offset */
>
topicQueueTable
=
new
HashMap
<
String
,
Long
>(
1024
);
pr
otected
volatile
long
confirmOffset
=
-
1L
;
pr
ivate
volatile
long
beginTimeInLock
=
0
;
pr
ivate
final
PutMessageLock
putMessageLock
;
pr
otected
volatile
long
beginTimeInLock
=
0
;
pr
otected
final
PutMessageLock
putMessageLock
;
public
CommitLog
(
final
DefaultMessageStore
defaultMessageStore
)
{
this
.
mappedFileQueue
=
new
MappedFileQueue
(
defaultMessageStore
.
getMessageStoreConfig
().
getStorePathCommitLog
(),
...
...
@@ -366,7 +366,7 @@ public class CommitLog {
return
new
DispatchRequest
(-
1
,
false
/* success */
);
}
pr
ivate
static
int
calMsgLength
(
int
bodyLength
,
int
topicLength
,
int
propertiesLength
)
{
pr
otected
static
int
calMsgLength
(
int
bodyLength
,
int
topicLength
,
int
propertiesLength
)
{
final
int
msgLen
=
4
//TOTALSIZE
+
4
//MAGICCODE
+
4
//BODYCRC
...
...
store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
浏览文件 @
515bc353
...
...
@@ -348,7 +348,7 @@ public class ConsumeQueue {
long
tagsCode
=
result
.
getByteBuffer
().
getLong
();
if
(
offsetPy
>=
phyMinOffset
)
{
this
.
minLogicOffset
=
result
.
getMappedFile
()
.
getFileFromOffset
()
+
i
;
this
.
minLogicOffset
=
mappedFile
.
getFileFromOffset
()
+
i
;
log
.
info
(
"Compute logical min offset: {}, topic: {}, queueId: {}"
,
this
.
getMinOffsetInQueue
(),
this
.
topic
,
this
.
queueId
);
// This maybe not take effect, when not every consume queue has extend file.
...
...
store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
浏览文件 @
515bc353
...
...
@@ -52,6 +52,7 @@ import org.apache.rocketmq.common.sysflag.MessageSysFlag;
import
org.apache.rocketmq.store.config.BrokerRole
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.apache.rocketmq.store.config.StorePathConfigHelper
;
import
org.apache.rocketmq.store.dleger.DLegerCommitLog
;
import
org.apache.rocketmq.store.ha.HAService
;
import
org.apache.rocketmq.store.index.IndexService
;
import
org.apache.rocketmq.store.index.QueryOffsetResult
;
...
...
@@ -119,7 +120,11 @@ public class DefaultMessageStore implements MessageStore {
this
.
messageStoreConfig
=
messageStoreConfig
;
this
.
brokerStatsManager
=
brokerStatsManager
;
this
.
allocateMappedFileService
=
new
AllocateMappedFileService
(
this
);
this
.
commitLog
=
new
CommitLog
(
this
);
if
(
messageStoreConfig
.
isEnableDLegerCommitLog
())
{
this
.
commitLog
=
new
DLegerCommitLog
(
this
);
}
else
{
this
.
commitLog
=
new
CommitLog
(
this
);
}
this
.
consumeQueueTable
=
new
ConcurrentHashMap
<>(
32
);
this
.
flushConsumeQueueService
=
new
FlushConsumeQueueService
();
...
...
@@ -1763,7 +1768,7 @@ public class DefaultMessageStore implements MessageStore {
for
(
int
readSize
=
0
;
readSize
<
result
.
getSize
()
&&
doNext
;
)
{
DispatchRequest
dispatchRequest
=
DefaultMessageStore
.
this
.
commitLog
.
checkMessageAndReturnSize
(
result
.
getByteBuffer
(),
false
,
false
);
int
size
=
dispatchRequest
.
get
Msg
Size
();
int
size
=
dispatchRequest
.
get
BufferSize
()
==
-
1
?
dispatchRequest
.
getMsgSize
()
:
dispatchRequest
.
getBuffer
Size
();
if
(
dispatchRequest
.
isSuccess
())
{
if
(
size
>
0
)
{
...
...
store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java
浏览文件 @
515bc353
...
...
@@ -22,7 +22,7 @@ public class DispatchRequest {
private
final
String
topic
;
private
final
int
queueId
;
private
final
long
commitLogOffset
;
private
final
int
msgSize
;
private
int
msgSize
;
private
final
long
tagsCode
;
private
final
long
storeTimestamp
;
private
final
long
consumeQueueOffset
;
...
...
@@ -35,6 +35,8 @@ public class DispatchRequest {
private
final
Map
<
String
,
String
>
propertiesMap
;
private
byte
[]
bitMap
;
private
int
bufferSize
=
-
1
;
//the buffer size maybe larger than the msg size if the message is wrapped by something
public
DispatchRequest
(
final
String
topic
,
final
int
queueId
,
...
...
@@ -156,4 +158,16 @@ public class DispatchRequest {
public
void
setBitMap
(
byte
[]
bitMap
)
{
this
.
bitMap
=
bitMap
;
}
public
void
setMsgSize
(
int
msgSize
)
{
this
.
msgSize
=
msgSize
;
}
public
int
getBufferSize
()
{
return
bufferSize
;
}
public
void
setBufferSize
(
int
bufferSize
)
{
this
.
bufferSize
=
bufferSize
;
}
}
store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java
浏览文件 @
515bc353
...
...
@@ -48,9 +48,9 @@ public class SelectMappedBufferResult {
this
.
byteBuffer
.
limit
(
this
.
size
);
}
public
MappedFile
getMappedFile
()
{
/*
public MappedFile getMappedFile() {
return mappedFile;
}
}
*/
// @Override
// protected void finalize() {
...
...
store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
浏览文件 @
515bc353
...
...
@@ -143,6 +143,11 @@ public class MessageStoreConfig {
private
int
transientStorePoolSize
=
5
;
private
boolean
fastFailIfNoBufferInStorePool
=
false
;
private
boolean
enableDLegerCommitLog
;
private
String
dLegerGroup
;
private
String
dLegerPeers
;
private
String
dLegerSelfId
;
public
boolean
isDebugLockEnable
()
{
return
debugLockEnable
;
}
...
...
@@ -666,4 +671,35 @@ public class MessageStoreConfig {
this
.
commitCommitLogThoroughInterval
=
commitCommitLogThoroughInterval
;
}
public
String
getdLegerGroup
()
{
return
dLegerGroup
;
}
public
void
setdLegerGroup
(
String
dLegerGroup
)
{
this
.
dLegerGroup
=
dLegerGroup
;
}
public
String
getdLegerPeers
()
{
return
dLegerPeers
;
}
public
void
setdLegerPeers
(
String
dLegerPeers
)
{
this
.
dLegerPeers
=
dLegerPeers
;
}
public
String
getdLegerSelfId
()
{
return
dLegerSelfId
;
}
public
void
setdLegerSelfId
(
String
dLegerSelfId
)
{
this
.
dLegerSelfId
=
dLegerSelfId
;
}
public
boolean
isEnableDLegerCommitLog
()
{
return
enableDLegerCommitLog
;
}
public
void
setEnableDLegerCommitLog
(
boolean
enableDLegerCommitLog
)
{
this
.
enableDLegerCommitLog
=
enableDLegerCommitLog
;
}
}
store/src/main/java/org/apache/rocketmq/store/dleger/DLegerCommitLog.java
0 → 100644
浏览文件 @
515bc353
此差异已折叠。
点击以展开。
store/src/main/java/org/apache/rocketmq/store/dleger/DLegerSelectMappedBufferResult.java
0 → 100644
浏览文件 @
515bc353
/*
* 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.store.dleger
;
import
org.apache.rocketmq.dleger.store.file.SelectMmapBufferResult
;
import
org.apache.rocketmq.store.SelectMappedBufferResult
;
public
class
DLegerSelectMappedBufferResult
extends
SelectMappedBufferResult
{
private
SelectMmapBufferResult
sbr
;
public
DLegerSelectMappedBufferResult
(
SelectMmapBufferResult
sbr
)
{
super
(
sbr
.
getStartOffset
(),
sbr
.
getByteBuffer
(),
sbr
.
getSize
(),
null
);
this
.
sbr
=
sbr
;
}
@Override
public
synchronized
void
release
()
{
super
.
release
();
sbr
.
release
();
}
}
store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java
0 → 100644
浏览文件 @
515bc353
package
org.apache.rocketmq.store
;
import
java.io.File
;
import
java.net.InetAddress
;
import
java.net.InetSocketAddress
;
import
java.net.SocketAddress
;
import
java.util.HashSet
;
import
java.util.Set
;
import
java.util.UUID
;
import
java.util.concurrent.atomic.AtomicInteger
;
import
org.apache.rocketmq.common.UtilAll
;
import
org.junit.After
;
public
class
StoreTestBase
{
private
int
QUEUE_TOTAL
=
100
;
private
AtomicInteger
QueueId
=
new
AtomicInteger
(
0
);
private
SocketAddress
BornHost
=
new
InetSocketAddress
(
"127.0.0.1"
,
8123
);
private
SocketAddress
StoreHost
=
BornHost
;
private
byte
[]
MessageBody
=
new
byte
[
1024
];
protected
Set
<
String
>
baseDirs
=
new
HashSet
<>();
private
AtomicInteger
port
=
new
AtomicInteger
(
30000
);
public
int
nextPort
()
{
return
port
.
incrementAndGet
();
}
protected
MessageExtBrokerInner
buildMessage
()
{
MessageExtBrokerInner
msg
=
new
MessageExtBrokerInner
();
msg
.
setTopic
(
"StoreTest"
);
msg
.
setTags
(
"TAG1"
);
msg
.
setKeys
(
"Hello"
);
msg
.
setBody
(
MessageBody
);
msg
.
setKeys
(
String
.
valueOf
(
System
.
currentTimeMillis
()));
msg
.
setQueueId
(
Math
.
abs
(
QueueId
.
getAndIncrement
())
%
QUEUE_TOTAL
);
msg
.
setSysFlag
(
0
);
msg
.
setBornTimestamp
(
System
.
currentTimeMillis
());
msg
.
setStoreHost
(
StoreHost
);
msg
.
setBornHost
(
BornHost
);
return
msg
;
}
public
static
String
createBaseDir
()
{
String
baseDir
=
System
.
getProperty
(
"user.home"
)
+
File
.
separator
+
"unitteststore"
+
File
.
separator
+
UUID
.
randomUUID
();
final
File
file
=
new
File
(
baseDir
);
if
(
file
.
exists
())
{
System
.
exit
(
1
);
}
return
baseDir
;
}
public
static
void
deleteFile
(
String
fileName
)
{
deleteFile
(
new
File
(
fileName
));
}
public
static
void
deleteFile
(
File
file
)
{
UtilAll
.
deleteFile
(
file
);
}
@After
public
void
clear
()
{
for
(
String
baseDir
:
baseDirs
)
{
deleteFile
(
baseDir
);
}
}
}
store/src/test/java/org/apache/rocketmq/store/dleger/DLegerCommitlogTest.java
0 → 100644
浏览文件 @
515bc353
package
org.apache.rocketmq.store.dleger
;
import
java.io.File
;
import
java.nio.ByteBuffer
;
import
java.util.ArrayList
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.UUID
;
import
org.apache.rocketmq.common.BrokerConfig
;
import
org.apache.rocketmq.common.message.MessageDecoder
;
import
org.apache.rocketmq.common.message.MessageExt
;
import
org.apache.rocketmq.store.DefaultMessageStore
;
import
org.apache.rocketmq.store.GetMessageResult
;
import
org.apache.rocketmq.store.GetMessageStatus
;
import
org.apache.rocketmq.store.MessageArrivingListener
;
import
org.apache.rocketmq.store.MessageExtBrokerInner
;
import
org.apache.rocketmq.store.PutMessageResult
;
import
org.apache.rocketmq.store.PutMessageStatus
;
import
org.apache.rocketmq.store.StoreTestBase
;
import
org.apache.rocketmq.store.config.FlushDiskType
;
import
org.apache.rocketmq.store.config.MessageStoreConfig
;
import
org.apache.rocketmq.store.stats.BrokerStatsManager
;
import
org.junit.Assert
;
import
org.junit.Test
;
public
class
DLegerCommitlogTest
extends
StoreTestBase
{
private
DefaultMessageStore
createMessageStore
(
String
base
)
throws
Exception
{
baseDirs
.
add
(
base
);
MessageStoreConfig
storeConfig
=
new
MessageStoreConfig
();
storeConfig
.
setMapedFileSizeCommitLog
(
1024
*
100
);
storeConfig
.
setMapedFileSizeConsumeQueue
(
1024
);
storeConfig
.
setMaxHashSlotNum
(
100
);
storeConfig
.
setMaxIndexNum
(
100
*
10
);
storeConfig
.
setStorePathRootDir
(
base
);
storeConfig
.
setStorePathCommitLog
(
base
+
File
.
separator
+
"commitlog"
);
storeConfig
.
setFlushDiskType
(
FlushDiskType
.
ASYNC_FLUSH
);
storeConfig
.
setEnableDLegerCommitLog
(
true
);
storeConfig
.
setdLegerGroup
(
UUID
.
randomUUID
().
toString
());
storeConfig
.
setdLegerPeers
(
String
.
format
(
"n0-localhost:%d"
,
nextPort
()));
storeConfig
.
setdLegerSelfId
(
"n0"
);
DefaultMessageStore
defaultMessageStore
=
new
DefaultMessageStore
(
storeConfig
,
new
BrokerStatsManager
(
"DLegerCommitlogTest"
),
new
MessageArrivingListener
()
{
@Override
public
void
arriving
(
String
topic
,
int
queueId
,
long
logicOffset
,
long
tagsCode
,
long
msgStoreTime
,
byte
[]
filterBitMap
,
Map
<
String
,
String
>
properties
)
{
}
},
new
BrokerConfig
());
defaultMessageStore
.
load
();
defaultMessageStore
.
start
();
return
defaultMessageStore
;
}
@Test
public
void
testPutAndGetMessage
()
throws
Exception
{
String
base
=
createBaseDir
();
DefaultMessageStore
messageStore
=
createMessageStore
(
base
);
Thread
.
sleep
(
1000
);
String
topic
=
UUID
.
randomUUID
().
toString
();
List
<
PutMessageResult
>
results
=
new
ArrayList
<>();
for
(
int
i
=
0
;
i
<
10
;
i
++)
{
MessageExtBrokerInner
msgInner
=
buildMessage
();
msgInner
.
setTopic
(
topic
);
msgInner
.
setQueueId
(
0
);
PutMessageResult
putMessageResult
=
messageStore
.
putMessage
(
msgInner
);
results
.
add
(
putMessageResult
);
Assert
.
assertEquals
(
PutMessageStatus
.
PUT_OK
,
putMessageResult
.
getPutMessageStatus
());
Assert
.
assertEquals
(
i
,
putMessageResult
.
getAppendMessageResult
().
getLogicsOffset
());
}
Thread
.
sleep
(
100
);
Assert
.
assertEquals
(
0
,
messageStore
.
getMinOffsetInQueue
(
topic
,
0
));
Assert
.
assertEquals
(
10
,
messageStore
.
getMaxOffsetInQueue
(
topic
,
0
));
Assert
.
assertEquals
(
0
,
messageStore
.
dispatchBehindBytes
());
GetMessageResult
getMessageResult
=
messageStore
.
getMessage
(
"group"
,
topic
,
0
,
0
,
32
,
null
);
Assert
.
assertEquals
(
GetMessageStatus
.
FOUND
,
getMessageResult
.
getStatus
());
Assert
.
assertEquals
(
10
,
getMessageResult
.
getMessageBufferList
().
size
());
Assert
.
assertEquals
(
10
,
getMessageResult
.
getMessageMapedList
().
size
());
for
(
int
i
=
0
;
i
<
results
.
size
();
i
++)
{
ByteBuffer
buffer
=
getMessageResult
.
getMessageBufferList
().
get
(
i
);
MessageExt
messageExt
=
MessageDecoder
.
decode
(
buffer
);
Assert
.
assertEquals
(
i
,
messageExt
.
getQueueOffset
());
Assert
.
assertEquals
(
results
.
get
(
i
).
getAppendMessageResult
().
getMsgId
(),
messageExt
.
getMsgId
());
Assert
.
assertEquals
(
results
.
get
(
i
).
getAppendMessageResult
().
getWroteOffset
(),
messageExt
.
getCommitLogOffset
());
}
}
}
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录