Skip to content

Commit 3fe81bf

Browse files
authored
[ISSUE #6545] Remove getTopicConfigs method in interface MessageStore (#6531)
* change map -> lambda * f * fix unit test * remove getTopicConfig function * Update MultiDispatchTest.java * Update CompactionStore.java * update * update test * update test * Update BatchConsumeMessageTest.java * Update BrokerController.java * Update BrokerController.java * check * Update BrokerController.java * Update BatchConsumeMessageTest.java
1 parent d1b14b0 commit 3fe81bf

25 files changed

+101
-113
lines changed

broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -737,8 +737,7 @@ public boolean initialize() throws CloneNotSupportedException {
737737

738738
if (result) {
739739
try {
740-
DefaultMessageStore defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig);
741-
defaultMessageStore.setTopicConfigTable(topicConfigManager.getTopicConfigTable());
740+
DefaultMessageStore defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
742741

743742
if (messageStoreConfig.isEnableDLegerCommitLog()) {
744743
DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, defaultMessageStore);

broker/src/test/java/org/apache/rocketmq/broker/filter/MessageStoreWithFilterTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import java.util.ArrayList;
2727
import java.util.List;
2828
import java.util.Map;
29+
import java.util.concurrent.ConcurrentHashMap;
2930
import java.util.concurrent.TimeUnit;
3031
import org.apache.rocketmq.common.BrokerConfig;
3132
import org.apache.rocketmq.common.MixAll;
@@ -159,7 +160,7 @@ public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
159160
long msgStoreTime, byte[] filterBitMap, Map<String, String> properties) {
160161
}
161162
}
162-
, brokerConfig);
163+
, brokerConfig, new ConcurrentHashMap<>());
163164

164165
master.getDispatcherList().addFirst(new CommitLogDispatcher() {
165166
@Override

broker/src/test/java/org/apache/rocketmq/broker/schedule/ScheduleMessageServiceTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -119,7 +119,7 @@ public void setUp() throws Exception {
119119

120120
brokerConfig = new BrokerConfig();
121121
BrokerStatsManager manager = new BrokerStatsManager(brokerConfig.getBrokerClusterName(), brokerConfig.isEnableDetailStat());
122-
messageStore = new DefaultMessageStore(messageStoreConfig, manager, new MyMessageArrivingListener(), new BrokerConfig());
122+
messageStore = new DefaultMessageStore(messageStoreConfig, manager, new MyMessageArrivingListener(), new BrokerConfig(), new ConcurrentHashMap<>());
123123

124124
assertThat(messageStore.load()).isTrue();
125125

store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -200,16 +200,20 @@ public class DefaultMessageStore implements MessageStore {
200200

201201
private long stateMachineVersion = 0L;
202202

203+
// this is a unmodifiableMap
204+
private ConcurrentMap<String, TopicConfig> topicConfigTable;
205+
203206
private final ScheduledExecutorService scheduledCleanQueueExecutorService =
204207
Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreCleanQueueScheduledThread"));
205208

206209
public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
207-
final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig) throws IOException {
210+
final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig, final ConcurrentMap<String, TopicConfig> topicConfigTable) throws IOException {
208211
this.messageArrivingListener = messageArrivingListener;
209212
this.brokerConfig = brokerConfig;
210213
this.messageStoreConfig = messageStoreConfig;
211214
this.aliveReplicasNum = messageStoreConfig.getTotalReplicas();
212215
this.brokerStatsManager = brokerStatsManager;
216+
this.topicConfigTable = topicConfigTable;
213217
this.allocateMappedFileService = new AllocateMappedFileService(this);
214218
if (messageStoreConfig.isEnableDLegerCommitLog()) {
215219
this.commitLog = new DLedgerCommitLog(this);
@@ -2047,18 +2051,16 @@ public void assignOffset(MessageExtBrokerInner msg, short messageNum) {
20472051
}
20482052
}
20492053

2050-
@Override
20512054
public ConcurrentMap<String, TopicConfig> getTopicConfigs() {
2052-
return this.consumeQueueStore.getTopicConfigs();
2055+
return this.topicConfigTable;
20532056
}
20542057

2055-
@Override
20562058
public Optional<TopicConfig> getTopicConfig(String topic) {
2057-
return this.consumeQueueStore.getTopicConfig(topic);
2058-
}
2059+
if (this.topicConfigTable == null) {
2060+
return Optional.empty();
2061+
}
20592062

2060-
public void setTopicConfigTable(ConcurrentMap<String, TopicConfig> topicConfigTable) {
2061-
this.consumeQueueStore.setTopicConfigTable(topicConfigTable);
2063+
return Optional.ofNullable(this.topicConfigTable.get(topic));
20622064
}
20632065

20642066
public BrokerIdentity getBrokerIdentity() {

store/src/main/java/org/apache/rocketmq/store/MessageStore.java

Lines changed: 0 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -25,15 +25,12 @@
2525
import java.util.HashMap;
2626
import java.util.LinkedList;
2727
import java.util.List;
28-
import java.util.Map;
29-
import java.util.Optional;
3028
import java.util.Set;
3129
import java.util.concurrent.CompletableFuture;
3230
import java.util.function.Supplier;
3331

3432
import org.apache.rocketmq.common.Pair;
3533
import org.apache.rocketmq.common.SystemClock;
36-
import org.apache.rocketmq.common.TopicConfig;
3734
import org.apache.rocketmq.common.message.MessageExt;
3835
import org.apache.rocketmq.common.message.MessageExtBatch;
3936
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
@@ -736,21 +733,6 @@ void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, Ma
736733
*/
737734
void assignOffset(MessageExtBrokerInner msg, short messageNum);
738735

739-
/**
740-
* get all topic config
741-
*
742-
* @return all topic config info
743-
*/
744-
Map<String, TopicConfig> getTopicConfigs();
745-
746-
/**
747-
* get topic config
748-
*
749-
* @param topic topic name
750-
* @return topic config info
751-
*/
752-
Optional<TopicConfig> getTopicConfig(String topic);
753-
754736
/**
755737
* Get master broker message store in process in broker container
756738
*

store/src/main/java/org/apache/rocketmq/store/kv/CompactionStore.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -29,9 +29,9 @@
2929
import org.apache.rocketmq.common.utils.CleanupPolicyUtils;
3030
import org.apache.rocketmq.logging.org.slf4j.Logger;
3131
import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
32+
import org.apache.rocketmq.store.DefaultMessageStore;
3233
import org.apache.rocketmq.store.DispatchRequest;
3334
import org.apache.rocketmq.store.GetMessageResult;
34-
import org.apache.rocketmq.store.MessageStore;
3535
import org.apache.rocketmq.store.SelectMappedBufferResult;
3636
import org.apache.rocketmq.store.config.MessageStoreConfig;
3737

@@ -53,7 +53,7 @@ public class CompactionStore {
5353
private final String compactionPath;
5454
private final String compactionLogPath;
5555
private final String compactionCqPath;
56-
private final MessageStore defaultMessageStore;
56+
private final DefaultMessageStore defaultMessageStore;
5757
private final CompactionPositionMgr positionMgr;
5858
private final ConcurrentHashMap<String, CompactionLog> compactionLogTable;
5959
private final ScheduledExecutorService compactionSchedule;
@@ -65,7 +65,7 @@ public class CompactionStore {
6565

6666
private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
6767

68-
public CompactionStore(MessageStore defaultMessageStore) {
68+
public CompactionStore(DefaultMessageStore defaultMessageStore) {
6969
this.defaultMessageStore = defaultMessageStore;
7070
this.compactionLogTable = new ConcurrentHashMap<>();
7171
MessageStoreConfig config = defaultMessageStore.getMessageStoreConfig();

store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java

Lines changed: 0 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -26,15 +26,12 @@
2626
import java.util.HashMap;
2727
import java.util.LinkedList;
2828
import java.util.List;
29-
import java.util.Map;
30-
import java.util.Optional;
3129
import java.util.Set;
3230
import java.util.concurrent.CompletableFuture;
3331
import java.util.function.Supplier;
3432

3533
import org.apache.rocketmq.common.Pair;
3634
import org.apache.rocketmq.common.SystemClock;
37-
import org.apache.rocketmq.common.TopicConfig;
3835
import org.apache.rocketmq.common.message.MessageExt;
3936
import org.apache.rocketmq.common.message.MessageExtBatch;
4037
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
@@ -595,16 +592,6 @@ public void assignOffset(MessageExtBrokerInner msg, short messageNum) {
595592
next.assignOffset(msg, messageNum);
596593
}
597594

598-
@Override
599-
public Map<String, TopicConfig> getTopicConfigs() {
600-
return next.getTopicConfigs();
601-
}
602-
603-
@Override
604-
public Optional<TopicConfig> getTopicConfig(String topic) {
605-
return next.getTopicConfig(topic);
606-
}
607-
608595
@Override
609596
public List<PutMessageHook> getPutMessageHookList() {
610597
return next.getPutMessageHookList();

store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java

Lines changed: 3 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -64,20 +64,12 @@ public class ConsumeQueueStore {
6464
protected final QueueOffsetAssigner queueOffsetAssigner = new QueueOffsetAssigner();
6565
protected final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
6666

67-
// Should be careful, do not change the topic config
68-
// TopicConfigManager is more suitable here.
69-
private ConcurrentMap<String, TopicConfig> topicConfigTable;
70-
7167
public ConsumeQueueStore(DefaultMessageStore messageStore, MessageStoreConfig messageStoreConfig) {
7268
this.messageStore = messageStore;
7369
this.messageStoreConfig = messageStoreConfig;
7470
this.consumeQueueTable = new ConcurrentHashMap<>(32);
7571
}
7672

77-
public void setTopicConfigTable(ConcurrentMap<String, TopicConfig> topicConfigTable) {
78-
this.topicConfigTable = topicConfigTable;
79-
}
80-
8173
private FileQueueLifeCycle getLifeCycle(String topic, int queueId) {
8274
return findOrCreateConsumeQueue(topic, queueId);
8375
}
@@ -173,9 +165,9 @@ private ConsumeQueueInterface createConsumeQueueByType(CQType cqType, String top
173165
}
174166

175167
private void queueTypeShouldBe(String topic, CQType cqTypeExpected) {
176-
TopicConfig topicConfig = this.topicConfigTable == null ? null : this.topicConfigTable.get(topic);
168+
Optional<TopicConfig> topicConfig = this.messageStore.getTopicConfig(topic);
177169

178-
CQType cqTypeActual = QueueTypeUtils.getCQType(Optional.ofNullable(topicConfig));
170+
CQType cqTypeActual = QueueTypeUtils.getCQType(topicConfig);
179171

180172
if (!Objects.equals(cqTypeExpected, cqTypeActual)) {
181173
throw new RuntimeException(format("The queue type of topic: %s should be %s, but is %s", topic, cqTypeExpected, cqTypeActual));
@@ -341,7 +333,7 @@ private ConsumeQueueInterface doFindOrCreateConsumeQueue(String topic, int queue
341333

342334
ConsumeQueueInterface newLogic;
343335

344-
Optional<TopicConfig> topicConfig = this.getTopicConfig(topic);
336+
Optional<TopicConfig> topicConfig = this.messageStore.getTopicConfig(topic);
345337
// TODO maybe the topic has been deleted.
346338
if (Objects.equals(CQType.BatchCQ, QueueTypeUtils.getCQType(topicConfig))) {
347339
newLogic = new BatchConsumeQueue(
@@ -537,18 +529,6 @@ public void truncateDirty(long phyOffset) {
537529
}
538530
}
539531

540-
public ConcurrentMap<String, TopicConfig> getTopicConfigs() {
541-
return this.topicConfigTable;
542-
}
543-
544-
public Optional<TopicConfig> getTopicConfig(String topic) {
545-
if (this.topicConfigTable == null) {
546-
return Optional.empty();
547-
}
548-
549-
return Optional.ofNullable(this.topicConfigTable.get(topic));
550-
}
551-
552532
public long getTotalSize() {
553533
long totalSize = 0;
554534
for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {

store/src/test/java/org/apache/rocketmq/store/AppendCallbackTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import java.util.List;
2626
import java.util.Set;
2727

28+
import java.util.concurrent.ConcurrentHashMap;
2829
import org.apache.rocketmq.common.BrokerConfig;
2930
import org.apache.rocketmq.common.UtilAll;
3031
import org.apache.rocketmq.common.message.Message;
@@ -55,7 +56,7 @@ public void init() throws Exception {
5556
messageStoreConfig.setStorePathRootDir(System.getProperty("java.io.tmpdir") + File.separator + "unitteststore");
5657
messageStoreConfig.setStorePathCommitLog(System.getProperty("java.io.tmpdir") + File.separator + "unitteststore" + File.separator + "commitlog");
5758
//too much reference
58-
DefaultMessageStore messageStore = new DefaultMessageStore(messageStoreConfig, null, null, new BrokerConfig());
59+
DefaultMessageStore messageStore = new DefaultMessageStore(messageStoreConfig, null, null, new BrokerConfig(), new ConcurrentHashMap<>());
5960
CommitLog commitLog = new CommitLog(messageStore);
6061
callback = commitLog.new DefaultAppendMessageCallback();
6162
}

store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import java.util.HashMap;
2626
import java.util.List;
2727
import java.util.Map;
28+
import java.util.concurrent.ConcurrentHashMap;
2829
import org.apache.rocketmq.common.BrokerConfig;
2930
import org.apache.rocketmq.common.UtilAll;
3031
import org.apache.rocketmq.common.message.Message;
@@ -78,7 +79,7 @@ private MessageStore buildMessageStore() throws Exception {
7879
messageStoreConfig.setStorePathCommitLog(System.getProperty("java.io.tmpdir") + File.separator
7980
+ "putmessagesteststore" + File.separator + "commitlog");
8081
messageStoreConfig.setHaListenPort(0);
81-
return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), new MyMessageArrivingListener(), new BrokerConfig());
82+
return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), new MyMessageArrivingListener(), new BrokerConfig(), new ConcurrentHashMap<>());
8283
}
8384

8485
@Test

store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import java.net.SocketAddress;
2525
import java.net.UnknownHostException;
2626
import java.util.Map;
27+
import java.util.concurrent.ConcurrentHashMap;
2728
import java.util.concurrent.ConcurrentMap;
2829
import java.util.concurrent.TimeUnit;
2930

@@ -151,7 +152,7 @@ public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
151152
long msgStoreTime, byte[] filterBitMap, Map<String, String> properties) {
152153
}
153154
}
154-
, brokerConfig);
155+
, brokerConfig, new ConcurrentHashMap<>());
155156

156157
assertThat(master.load()).isTrue();
157158

@@ -179,7 +180,7 @@ public void arriving(String topic, int queueId, long logicOffset, long tagsCode,
179180
long msgStoreTime, byte[] filterBitMap, Map<String, String> properties) {
180181
}
181182
}
182-
, brokerConfig);
183+
, brokerConfig, new ConcurrentHashMap<>());
183184

184185
assertThat(master.load()).isTrue();
185186

store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.rocketmq.store;
1919

20+
import java.util.concurrent.ConcurrentHashMap;
2021
import org.apache.rocketmq.common.BrokerConfig;
2122
import org.apache.rocketmq.common.MixAll;
2223
import org.apache.rocketmq.common.UtilAll;
@@ -483,7 +484,7 @@ private MessageStoreConfig genMessageStoreConfig(String deleteWhen, int diskMaxU
483484

484485
private void initMessageStore(MessageStoreConfig messageStoreConfig, double diskSpaceCleanForciblyRatio) throws Exception {
485486
messageStore = new DefaultMessageStore(messageStoreConfig,
486-
new BrokerStatsManager("test", true), new MyMessageArrivingListener(), new BrokerConfig());
487+
new BrokerStatsManager("test", true), new MyMessageArrivingListener(), new BrokerConfig(), new ConcurrentHashMap<>());
487488

488489
cleanCommitLogService = getCleanCommitLogService();
489490
cleanConsumeQueueService = getCleanConsumeQueueService();

store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.rocketmq.store;
1919

2020
import java.io.File;
21+
import java.util.concurrent.ConcurrentHashMap;
2122
import org.apache.rocketmq.common.BrokerConfig;
2223
import org.apache.rocketmq.common.UtilAll;
2324
import org.apache.rocketmq.store.config.FlushDiskType;
@@ -74,7 +75,7 @@ public DefaultMessageStore buildMessageStore() throws Exception {
7475
String storeRootPath = System.getProperty("java.io.tmpdir") + File.separator + "store";
7576
messageStoreConfig.setStorePathRootDir(storeRootPath);
7677
messageStoreConfig.setHaListenPort(0);
77-
return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), null, new BrokerConfig());
78+
return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest", true), null, new BrokerConfig(), new ConcurrentHashMap<>());
7879
}
7980

8081
}

store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ public void test_repeat_restart() throws Exception {
9999
messageStoreConfig.setMaxIndexNum(100 * 10);
100100
messageStoreConfig.setStorePathRootDir(System.getProperty("java.io.tmpdir") + File.separator + "store");
101101
messageStoreConfig.setHaListenPort(0);
102-
MessageStore master = new DefaultMessageStore(messageStoreConfig, null, new MyMessageArrivingListener(), new BrokerConfig());
102+
MessageStore master = new DefaultMessageStore(messageStoreConfig, null, new MyMessageArrivingListener(), new BrokerConfig(), new ConcurrentHashMap<>());
103103

104104
boolean load = master.load();
105105
assertTrue(load);
@@ -144,7 +144,7 @@ private MessageStore buildMessageStore(String storePathRootDir) throws Exception
144144
return new DefaultMessageStore(messageStoreConfig,
145145
new BrokerStatsManager("simpleTest", true),
146146
new MyMessageArrivingListener(),
147-
new BrokerConfig());
147+
new BrokerConfig(), new ConcurrentHashMap<>());
148148
}
149149

150150
@Test

store/src/test/java/org/apache/rocketmq/store/HATest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.time.Duration;
2828
import java.util.UUID;
2929
import java.util.concurrent.CompletableFuture;
30+
import java.util.concurrent.ConcurrentHashMap;
3031
import java.util.concurrent.atomic.AtomicInteger;
3132
import org.apache.rocketmq.common.BrokerConfig;
3233
import org.apache.rocketmq.common.MixAll;
@@ -250,7 +251,7 @@ public void destroy() throws Exception {
250251
private MessageStore buildMessageStore(MessageStoreConfig messageStoreConfig, long brokerId) throws Exception {
251252
BrokerConfig brokerConfig = new BrokerConfig();
252253
brokerConfig.setBrokerId(brokerId);
253-
return new DefaultMessageStore(messageStoreConfig, brokerStatsManager, null, brokerConfig);
254+
return new DefaultMessageStore(messageStoreConfig, brokerStatsManager, null, brokerConfig, new ConcurrentHashMap<>());
254255
}
255256

256257
private void buildMessageStoreConfig(MessageStoreConfig messageStoreConfig) {

store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.net.InetSocketAddress;
2222
import java.nio.charset.Charset;
2323

24+
import java.util.concurrent.ConcurrentHashMap;
2425
import org.apache.rocketmq.common.BrokerConfig;
2526
import org.apache.rocketmq.common.UtilAll;
2627
import org.apache.rocketmq.common.message.MessageConst;
@@ -57,7 +58,7 @@ public void init() throws Exception {
5758
messageStoreConfig.setEnableMultiDispatch(true);
5859
BrokerConfig brokerConfig = new BrokerConfig();
5960
//too much reference
60-
messageStore = new DefaultMessageStore(messageStoreConfig, null, null, brokerConfig);
61+
messageStore = new DefaultMessageStore(messageStoreConfig, null, null, brokerConfig, new ConcurrentHashMap<>());
6162
consumeQueue = new ConsumeQueue("xxx", 0,
6263
getStorePathConsumeQueue(messageStoreConfig.getStorePathRootDir()), messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore);
6364
}

0 commit comments

Comments
 (0)