29481 lines
1.5 MiB
Diff
29481 lines
1.5 MiB
Diff
|
|
From 0a046b85611bdc93c529650dc951f4cabbb61db9 Mon Sep 17 00:00:00 2001
|
|||
|
|
From: zhiliatom <zhiliatox@163.com>
|
|||
|
|
Date: Fri, 8 Dec 2023 09:25:41 +0800
|
|||
|
|
Subject: [PATCH] [ISSUE #7064] [RIP-66-2] Support KV(RocksDB) Storage for
|
|||
|
|
ConsumeQueue
|
|||
|
|
|
|||
|
|
---
|
|||
|
|
.../rocketmq/broker/BrokerController.java | 74 +-
|
|||
|
|
.../broker/controller/ReplicasManager.java | 4 +-
|
|||
|
|
.../offset/RocksDBConsumerOffsetManager.java | 8 +-
|
|||
|
|
.../broker/processor/AckMessageProcessor.java | 2 +-
|
|||
|
|
.../processor/AdminBrokerProcessor.java | 28 +-
|
|||
|
|
.../ChangeInvisibleTimeProcessor.java | 4 +-
|
|||
|
|
.../processor/PopBufferMergeService.java | 4 +-
|
|||
|
|
.../broker/processor/PopMessageProcessor.java | 2 +-
|
|||
|
|
.../broker/processor/PopReviveService.java | 6 +-
|
|||
|
|
.../RocksDBSubscriptionGroupManager.java | 10 +-
|
|||
|
|
.../topic/RocksDBTopicConfigManager.java | 8 +-
|
|||
|
|
.../processor/PopReviveServiceTest.java | 4 +-
|
|||
|
|
common/pom.xml | 2 +-
|
|||
|
|
.../org/apache/rocketmq/common/MixAll.java | 1 +
|
|||
|
|
.../rocketmq/common/attribute/CQType.java | 3 +-
|
|||
|
|
.../common/config/AbstractRocksDBStorage.java | 112 +-
|
|||
|
|
.../common/config/ConfigRocksDBStorage.java | 2 +-
|
|||
|
|
.../common/message/MessageExtBrokerInner.java | 13 +
|
|||
|
|
.../rocketmq/common/topic/TopicValidator.java | 2 +
|
|||
|
|
.../rocketmq/common/utils/DataConverter.java | 2 +-
|
|||
|
|
.../rocketmq/example/quickstart/Producer.java | 2 +-
|
|||
|
|
metrics | 241 ++
|
|||
|
|
patch009-modify-CRLF-to-LF.patch | 127 +
|
|||
|
|
patch010-backport-add-some-fixes | 1286 ++++++
|
|||
|
|
patch011-backport-optimize-config | 1390 +++++++
|
|||
|
|
...optimize-opentelemetry-metric-config.patch | 2081 ++++++++++
|
|||
|
|
...-backport-enhance-rockdbconfigtojson.patch | 2920 +++++++++++++
|
|||
|
|
patch013-backport-enhance-admin-output.patch | 892 ++++
|
|||
|
|
...ueue-Selection-Strategy-Optimization.patch | 2023 +++++++++
|
|||
|
|
patch015-backport-fix-some-bugs.patch | 1894 +++++++++
|
|||
|
|
...rt-Optimize-fault-tolerant-mechanism.patch | 520 +++
|
|||
|
|
...port-Convergent-thread-pool-creation.patch | 2243 ++++++++++
|
|||
|
|
...ckport-enhancement-of-tiered-storage.patch | 601 +++
|
|||
|
|
patch019-backport-some-bugfix.patch | 1499 +++++++
|
|||
|
|
patch020-backport-add-goaway-mechanism.patch | 3696 +++++++++++++++++
|
|||
|
|
patch021-backport-some-enhancements.patch | 344 ++
|
|||
|
|
pom.xml | 4 +-
|
|||
|
|
store/pom.xml | 4 +
|
|||
|
|
.../org/apache/rocketmq/store/CommitLog.java | 94 +-
|
|||
|
|
.../rocketmq/store/CommitLogDispatcher.java | 5 +-
|
|||
|
|
.../apache/rocketmq/store/ConsumeQueue.java | 93 +-
|
|||
|
|
.../rocketmq/store/DefaultMessageStore.java | 279 +-
|
|||
|
|
.../apache/rocketmq/store/MessageStore.java | 54 +-
|
|||
|
|
.../rocketmq/store/RocksDBMessageStore.java | 169 +
|
|||
|
|
.../apache/rocketmq/store/RunningFlags.java | 22 +-
|
|||
|
|
.../store/config/MessageStoreConfig.java | 33 +-
|
|||
|
|
.../store/dledger/DLedgerCommitLog.java | 53 +-
|
|||
|
|
.../apache/rocketmq/store/ha/HAService.java | 3 +-
|
|||
|
|
.../ha/autoswitch/AutoSwitchHAClient.java | 2 +-
|
|||
|
|
.../ha/autoswitch/AutoSwitchHAService.java | 7 +-
|
|||
|
|
.../plugin/AbstractPluginMessageStore.java | 42 +-
|
|||
|
|
.../queue/AbstractConsumeQueueStore.java | 105 +
|
|||
|
|
.../store/queue/BatchConsumeQueue.java | 20 +
|
|||
|
|
.../store/queue/ConsumeQueueInterface.java | 27 +-
|
|||
|
|
.../store/queue/ConsumeQueueStore.java | 293 +-
|
|||
|
|
.../queue/ConsumeQueueStoreInterface.java | 289 ++
|
|||
|
|
.../rocketmq/store/queue/MultiDispatch.java | 76 +
|
|||
|
|
.../store/queue/QueueOffsetOperator.java | 8 +
|
|||
|
|
.../store/queue/RocksDBConsumeQueue.java | 437 ++
|
|||
|
|
.../queue/RocksDBConsumeQueueOffsetTable.java | 641 +++
|
|||
|
|
.../store/queue/RocksDBConsumeQueueStore.java | 441 ++
|
|||
|
|
.../store/queue/RocksDBConsumeQueueTable.java | 312 ++
|
|||
|
|
.../ConsumeQueueCompactionFilterFactory.java | 47 +
|
|||
|
|
.../rocksdb/ConsumeQueueRocksDBStorage.java | 133 +
|
|||
|
|
.../store/rocksdb/RocksDBOptionsFactory.java | 161 +
|
|||
|
|
.../store/timer/TimerMessageStore.java | 65 +-
|
|||
|
|
.../store/DefaultMessageStoreTest.java | 5 +-
|
|||
|
|
.../rocketmq/store/MultiDispatchTest.java | 8 +-
|
|||
|
|
.../store/RocksDBMessageStoreTest.java | 1060 +++++
|
|||
|
|
.../apache/rocketmq/store/StoreTestUtil.java | 9 +-
|
|||
|
|
.../rocketmq/store/ha/HAServerTest.java | 16 +-
|
|||
|
|
.../store/ha/autoswitch/AutoSwitchHATest.java | 3 +-
|
|||
|
|
.../tieredstore/TieredMessageStore.java | 15 +-
|
|||
|
|
.../ExportMetadataInRocksDBCommand.java | 8 +-
|
|||
|
|
.../metadata/RocksDBConfigToJsonCommand.java | 118 +
|
|||
|
|
75 files changed, 26543 insertions(+), 668 deletions(-)
|
|||
|
|
create mode 100644 metrics
|
|||
|
|
create mode 100644 patch009-modify-CRLF-to-LF.patch
|
|||
|
|
create mode 100644 patch010-backport-add-some-fixes
|
|||
|
|
create mode 100644 patch011-backport-optimize-config
|
|||
|
|
create mode 100644 patch011-backport-optimize-opentelemetry-metric-config.patch
|
|||
|
|
create mode 100644 patch012-backport-enhance-rockdbconfigtojson.patch
|
|||
|
|
create mode 100644 patch013-backport-enhance-admin-output.patch
|
|||
|
|
create mode 100644 patch014-backport-Queue-Selection-Strategy-Optimization.patch
|
|||
|
|
create mode 100644 patch015-backport-fix-some-bugs.patch
|
|||
|
|
create mode 100644 patch016-backport-Optimize-fault-tolerant-mechanism.patch
|
|||
|
|
create mode 100644 patch017-backport-Convergent-thread-pool-creation.patch
|
|||
|
|
create mode 100644 patch018-backport-enhancement-of-tiered-storage.patch
|
|||
|
|
create mode 100644 patch019-backport-some-bugfix.patch
|
|||
|
|
create mode 100644 patch020-backport-add-goaway-mechanism.patch
|
|||
|
|
create mode 100644 patch021-backport-some-enhancements.patch
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/MultiDispatch.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTable.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueCompactionFilterFactory.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueRocksDBStorage.java
|
|||
|
|
create mode 100644 store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java
|
|||
|
|
create mode 100644 store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java
|
|||
|
|
create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
index d4bded600..9f1fd0ad0 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
@@ -16,7 +16,33 @@
|
|||
|
|
*/
|
|||
|
|
package org.apache.rocketmq.broker;
|
|||
|
|
|
|||
|
|
+import java.io.IOException;
|
|||
|
|
+import java.net.InetSocketAddress;
|
|||
|
|
+import java.util.AbstractMap;
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.Arrays;
|
|||
|
|
+import java.util.Collections;
|
|||
|
|
+import java.util.HashMap;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+import java.util.Objects;
|
|||
|
|
+import java.util.Optional;
|
|||
|
|
+import java.util.concurrent.BlockingQueue;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+import java.util.concurrent.CountDownLatch;
|
|||
|
|
+import java.util.concurrent.ExecutorService;
|
|||
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+import java.util.concurrent.ScheduledFuture;
|
|||
|
|
+import java.util.concurrent.TimeUnit;
|
|||
|
|
+import java.util.concurrent.locks.Lock;
|
|||
|
|
+import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
+import java.util.function.Function;
|
|||
|
|
+import java.util.stream.Collectors;
|
|||
|
|
+
|
|||
|
|
import com.google.common.collect.Lists;
|
|||
|
|
+
|
|||
|
|
import org.apache.rocketmq.acl.AccessValidator;
|
|||
|
|
import org.apache.rocketmq.acl.plain.PlainAccessValidator;
|
|||
|
|
import org.apache.rocketmq.broker.client.ClientHousekeepingService;
|
|||
|
|
@@ -126,7 +152,7 @@ import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.MessageArrivingListener;
|
|||
|
|
import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
import org.apache.rocketmq.store.PutMessageResult;
|
|||
|
|
-import org.apache.rocketmq.store.StoreType;
|
|||
|
|
+import org.apache.rocketmq.store.RocksDBMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
import org.apache.rocketmq.store.dledger.DLedgerCommitLog;
|
|||
|
|
@@ -141,31 +167,6 @@ import org.apache.rocketmq.store.timer.TimerCheckpoint;
|
|||
|
|
import org.apache.rocketmq.store.timer.TimerMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.timer.TimerMetrics;
|
|||
|
|
|
|||
|
|
-import java.io.IOException;
|
|||
|
|
-import java.net.InetSocketAddress;
|
|||
|
|
-import java.util.AbstractMap;
|
|||
|
|
-import java.util.ArrayList;
|
|||
|
|
-import java.util.Arrays;
|
|||
|
|
-import java.util.Collections;
|
|||
|
|
-import java.util.HashMap;
|
|||
|
|
-import java.util.List;
|
|||
|
|
-import java.util.Map;
|
|||
|
|
-import java.util.Objects;
|
|||
|
|
-import java.util.Optional;
|
|||
|
|
-import java.util.concurrent.BlockingQueue;
|
|||
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
-import java.util.concurrent.ConcurrentMap;
|
|||
|
|
-import java.util.concurrent.CountDownLatch;
|
|||
|
|
-import java.util.concurrent.ExecutorService;
|
|||
|
|
-import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
-import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
-import java.util.concurrent.ScheduledFuture;
|
|||
|
|
-import java.util.concurrent.TimeUnit;
|
|||
|
|
-import java.util.concurrent.locks.Lock;
|
|||
|
|
-import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
-import java.util.function.Function;
|
|||
|
|
-import java.util.stream.Collectors;
|
|||
|
|
-
|
|||
|
|
public class BrokerController {
|
|||
|
|
protected static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
|
|||
|
|
private static final Logger LOG_PROTECTION = LoggerFactory.getLogger(LoggerName.PROTECTION_LOGGER_NAME);
|
|||
|
|
@@ -308,7 +309,7 @@ public class BrokerController {
|
|||
|
|
this.setStoreHost(new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), getListenPort()));
|
|||
|
|
this.brokerStatsManager = messageStoreConfig.isEnableLmq() ? new LmqBrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat()) : new BrokerStatsManager(this.brokerConfig.getBrokerClusterName(), this.brokerConfig.isEnableDetailStat());
|
|||
|
|
this.broadcastOffsetManager = new BroadcastOffsetManager(this);
|
|||
|
|
- if (isEnableRocksDBStore()) {
|
|||
|
|
+ if (this.messageStoreConfig.isEnableRocksDBStore()) {
|
|||
|
|
this.topicConfigManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqTopicConfigManager(this) : new RocksDBTopicConfigManager(this);
|
|||
|
|
this.subscriptionGroupManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqSubscriptionGroupManager(this) : new RocksDBSubscriptionGroupManager(this);
|
|||
|
|
this.consumerOffsetManager = messageStoreConfig.isEnableLmq() ? new RocksDBLmqConsumerOffsetManager(this) : new RocksDBConsumerOffsetManager(this);
|
|||
|
|
@@ -747,7 +748,12 @@ public class BrokerController {
|
|||
|
|
public boolean initializeMessageStore() {
|
|||
|
|
boolean result = true;
|
|||
|
|
try {
|
|||
|
|
- DefaultMessageStore defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
|
|||
|
|
+ DefaultMessageStore defaultMessageStore;
|
|||
|
|
+ if (this.messageStoreConfig.isEnableRocksDBStore()) {
|
|||
|
|
+ defaultMessageStore = new RocksDBMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
|
|||
|
|
+ } else {
|
|||
|
|
+ defaultMessageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig, topicConfigManager.getTopicConfigTable());
|
|||
|
|
+ }
|
|||
|
|
|
|||
|
|
if (messageStoreConfig.isEnableDLegerCommitLog()) {
|
|||
|
|
DLedgerRoleChangeHandler roleChangeHandler =
|
|||
|
|
@@ -944,16 +950,16 @@ public class BrokerController {
|
|||
|
|
this.transactionalMessageService = ServiceProvider.loadClass(TransactionalMessageService.class);
|
|||
|
|
if (null == this.transactionalMessageService) {
|
|||
|
|
this.transactionalMessageService = new TransactionalMessageServiceImpl(
|
|||
|
|
- new TransactionalMessageBridge(this, this.getMessageStore()));
|
|||
|
|
+ new TransactionalMessageBridge(this, this.getMessageStore()));
|
|||
|
|
LOG.warn("Load default transaction message hook service: {}",
|
|||
|
|
- TransactionalMessageServiceImpl.class.getSimpleName());
|
|||
|
|
+ TransactionalMessageServiceImpl.class.getSimpleName());
|
|||
|
|
}
|
|||
|
|
this.transactionalMessageCheckListener = ServiceProvider.loadClass(
|
|||
|
|
- AbstractTransactionalMessageCheckListener.class);
|
|||
|
|
+ AbstractTransactionalMessageCheckListener.class);
|
|||
|
|
if (null == this.transactionalMessageCheckListener) {
|
|||
|
|
this.transactionalMessageCheckListener = new DefaultTransactionalMessageCheckListener();
|
|||
|
|
LOG.warn("Load default discard message hook service: {}",
|
|||
|
|
- DefaultTransactionalMessageCheckListener.class.getSimpleName());
|
|||
|
|
+ DefaultTransactionalMessageCheckListener.class.getSimpleName());
|
|||
|
|
}
|
|||
|
|
this.transactionalMessageCheckListener.setBrokerController(this);
|
|||
|
|
this.transactionalMessageCheckService = new TransactionalMessageCheckService(this);
|
|||
|
|
@@ -2412,8 +2418,4 @@ public class BrokerController {
|
|||
|
|
public void setColdDataCgCtrService(ColdDataCgCtrService coldDataCgCtrService) {
|
|||
|
|
this.coldDataCgCtrService = coldDataCgCtrService;
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
- public boolean isEnableRocksDBStore() {
|
|||
|
|
- return StoreType.DEFAULT_ROCKSDB.getStoreType().equalsIgnoreCase(this.messageStoreConfig.getStoreType());
|
|||
|
|
- }
|
|||
|
|
}
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
index a989e6e68..a1d711cb2 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
@@ -224,7 +224,7 @@ public class ReplicasManager {
|
|||
|
|
|
|||
|
|
public synchronized void changeBrokerRole(final Long newMasterBrokerId, final String newMasterAddress,
|
|||
|
|
final Integer newMasterEpoch,
|
|||
|
|
- final Integer syncStateSetEpoch, final Set<Long> syncStateSet) {
|
|||
|
|
+ final Integer syncStateSetEpoch, final Set<Long> syncStateSet) throws Exception {
|
|||
|
|
if (newMasterBrokerId != null && newMasterEpoch > this.masterEpoch) {
|
|||
|
|
if (newMasterBrokerId.equals(this.brokerControllerId)) {
|
|||
|
|
changeToMaster(newMasterEpoch, syncStateSetEpoch, syncStateSet);
|
|||
|
|
@@ -234,7 +234,7 @@ public class ReplicasManager {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void changeToMaster(final int newMasterEpoch, final int syncStateSetEpoch, final Set<Long> syncStateSet) {
|
|||
|
|
+ public void changeToMaster(final int newMasterEpoch, final int syncStateSetEpoch, final Set<Long> syncStateSet) throws Exception {
|
|||
|
|
synchronized (this) {
|
|||
|
|
if (newMasterEpoch > this.masterEpoch) {
|
|||
|
|
LOGGER.info("Begin to change to master, brokerName:{}, replicas:{}, new Epoch:{}", this.brokerConfig.getBrokerName(), this.brokerAddress, newMasterEpoch);
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java
|
|||
|
|
index 5695a3356..05b53b0bc 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/offset/RocksDBConsumerOffsetManager.java
|
|||
|
|
@@ -33,7 +33,7 @@ public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager {
|
|||
|
|
|
|||
|
|
public RocksDBConsumerOffsetManager(BrokerController brokerController) {
|
|||
|
|
super(brokerController);
|
|||
|
|
- this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval());
|
|||
|
|
+ this.rocksDBConfigManager = new RocksDBConfigManager(brokerController.getMessageStoreConfig().getMemTableFlushIntervalMs());
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
@@ -49,7 +49,7 @@ public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager {
|
|||
|
|
@Override
|
|||
|
|
protected void removeConsumerOffset(String topicAtGroup) {
|
|||
|
|
try {
|
|||
|
|
- byte[] keyBytes = topicAtGroup.getBytes(DataConverter.charset);
|
|||
|
|
+ byte[] keyBytes = topicAtGroup.getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
this.rocksDBConfigManager.delete(keyBytes);
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
LOG.error("kv remove consumerOffset Failed, {}", topicAtGroup);
|
|||
|
|
@@ -58,7 +58,7 @@ public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
protected void decode0(final byte[] key, final byte[] body) {
|
|||
|
|
- String topicAtGroup = new String(key, DataConverter.charset);
|
|||
|
|
+ String topicAtGroup = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
RocksDBOffsetSerializeWrapper wrapper = JSON.parseObject(body, RocksDBOffsetSerializeWrapper.class);
|
|||
|
|
|
|||
|
|
this.offsetTable.put(topicAtGroup, wrapper.getOffsetTable());
|
|||
|
|
@@ -93,7 +93,7 @@ public class RocksDBConsumerOffsetManager extends ConsumerOffsetManager {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
private void putWriteBatch(final WriteBatch writeBatch, final String topicGroupName, final ConcurrentMap<Integer, Long> offsetMap) throws Exception {
|
|||
|
|
- byte[] keyBytes = topicGroupName.getBytes(DataConverter.charset);
|
|||
|
|
+ byte[] keyBytes = topicGroupName.getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
RocksDBOffsetSerializeWrapper wrapper = new RocksDBOffsetSerializeWrapper();
|
|||
|
|
wrapper.setOffsetTable(offsetMap);
|
|||
|
|
byte[] valueBytes = JSON.toJSONBytes(wrapper, SerializerFeature.BrowserCompatible);
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
index 244b459d6..59a3e63b2 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
@@ -253,7 +253,7 @@ public class AckMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
|
|||
|
|
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
|
|||
|
|
msgInner.setTopic(reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(rqId);
|
|||
|
|
if (ackMsg instanceof BatchAckMsg) {
|
|||
|
|
msgInner.setTags(PopAckConstants.BATCH_ACK_TAG);
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
index e77120e15..dd4ec960f 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
@@ -539,14 +539,18 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
|
|||
|
|
final Set<String> groups = this.brokerController.getConsumerOffsetManager().whichGroupByTopic(topic);
|
|||
|
|
// delete pop retry topics first
|
|||
|
|
- for (String group : groups) {
|
|||
|
|
- final String popRetryTopic = KeyBuilder.buildPopRetryTopic(topic, group);
|
|||
|
|
- if (brokerController.getTopicConfigManager().selectTopicConfig(popRetryTopic) != null) {
|
|||
|
|
- deleteTopicInBroker(popRetryTopic);
|
|||
|
|
+ try {
|
|||
|
|
+ for (String group : groups) {
|
|||
|
|
+ final String popRetryTopic = KeyBuilder.buildPopRetryTopic(topic, group);
|
|||
|
|
+ if (brokerController.getTopicConfigManager().selectTopicConfig(popRetryTopic) != null) {
|
|||
|
|
+ deleteTopicInBroker(popRetryTopic);
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
+ // delete topic
|
|||
|
|
+ deleteTopicInBroker(topic);
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ return buildErrorResponse(ResponseCode.SYSTEM_ERROR, t.getMessage());
|
|||
|
|
}
|
|||
|
|
- // delete topic
|
|||
|
|
- deleteTopicInBroker(topic);
|
|||
|
|
response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
response.setRemark(null);
|
|||
|
|
return response;
|
|||
|
|
@@ -2081,7 +2085,11 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
public RemotingCommand cleanExpiredConsumeQueue() {
|
|||
|
|
LOGGER.info("AdminBrokerProcessor#cleanExpiredConsumeQueue: start.");
|
|||
|
|
final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
- brokerController.getMessageStore().cleanExpiredConsumerQueue();
|
|||
|
|
+ try {
|
|||
|
|
+ brokerController.getMessageStore().cleanExpiredConsumerQueue();
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ return buildErrorResponse(ResponseCode.SYSTEM_ERROR, t.getMessage());
|
|||
|
|
+ }
|
|||
|
|
LOGGER.info("AdminBrokerProcessor#cleanExpiredConsumeQueue: end.");
|
|||
|
|
response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
response.setRemark(null);
|
|||
|
|
@@ -2781,7 +2789,11 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
|
|||
|
|
final ReplicasManager replicasManager = this.brokerController.getReplicasManager();
|
|||
|
|
if (replicasManager != null) {
|
|||
|
|
- replicasManager.changeBrokerRole(requestHeader.getMasterBrokerId(), requestHeader.getMasterAddress(), requestHeader.getMasterEpoch(), requestHeader.getSyncStateSetEpoch(), syncStateSetInfo.getSyncStateSet());
|
|||
|
|
+ try {
|
|||
|
|
+ replicasManager.changeBrokerRole(requestHeader.getMasterBrokerId(), requestHeader.getMasterAddress(), requestHeader.getMasterEpoch(), requestHeader.getSyncStateSetEpoch(), syncStateSetInfo.getSyncStateSet());
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ throw new RemotingCommandException(e.getMessage());
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
response.setRemark(null);
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
|
|||
|
|
index 2ccdf07f6..bdfffff09 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java
|
|||
|
|
@@ -180,7 +180,7 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
msgInner.setTopic(reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(rqId);
|
|||
|
|
msgInner.setTags(PopAckConstants.ACK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
@@ -216,7 +216,7 @@ public class ChangeInvisibleTimeProcessor implements NettyRequestProcessor {
|
|||
|
|
ck.addDiff(0);
|
|||
|
|
ck.setBrokerName(brokerName);
|
|||
|
|
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(reviveQid);
|
|||
|
|
msgInner.setTags(PopAckConstants.CK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
|
|||
|
|
index b7ba8ad4a..8a85dd8fe 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java
|
|||
|
|
@@ -633,7 +633,7 @@ public class PopBufferMergeService extends ServiceThread {
|
|||
|
|
ackMsg.setQueueId(point.getQueueId());
|
|||
|
|
ackMsg.setPopTime(point.getPopTime());
|
|||
|
|
msgInner.setTopic(popMessageProcessor.reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(pointWrapper.getReviveQueueId());
|
|||
|
|
msgInner.setTags(PopAckConstants.ACK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
@@ -673,7 +673,7 @@ public class PopBufferMergeService extends ServiceThread {
|
|||
|
|
batchAckMsg.setQueueId(point.getQueueId());
|
|||
|
|
batchAckMsg.setPopTime(point.getPopTime());
|
|||
|
|
msgInner.setTopic(popMessageProcessor.reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(batchAckMsg).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(batchAckMsg).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(pointWrapper.getReviveQueueId());
|
|||
|
|
msgInner.setTags(PopAckConstants.BATCH_ACK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
index 0d9bdf143..f5d07c5aa 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
@@ -685,7 +685,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
|
|||
|
|
|
|||
|
|
msgInner.setTopic(reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(reviveQid);
|
|||
|
|
msgInner.setTags(PopAckConstants.CK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
index d5174d3d1..4f80752e1 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
@@ -356,7 +356,7 @@ public class PopReviveService extends ServiceThread {
|
|||
|
|
}
|
|||
|
|
for (MessageExt messageExt : messageExts) {
|
|||
|
|
if (PopAckConstants.CK_TAG.equals(messageExt.getTags())) {
|
|||
|
|
- String raw = new String(messageExt.getBody(), DataConverter.charset);
|
|||
|
|
+ String raw = new String(messageExt.getBody(), DataConverter.CHARSET_UTF8);
|
|||
|
|
if (brokerController.getBrokerConfig().isEnablePopLog()) {
|
|||
|
|
POP_LOGGER.info("reviveQueueId={},find ck, offset:{}, raw : {}", messageExt.getQueueId(), messageExt.getQueueOffset(), raw);
|
|||
|
|
}
|
|||
|
|
@@ -371,7 +371,7 @@ public class PopReviveService extends ServiceThread {
|
|||
|
|
firstRt = point.getReviveTime();
|
|||
|
|
}
|
|||
|
|
} else if (PopAckConstants.ACK_TAG.equals(messageExt.getTags())) {
|
|||
|
|
- String raw = new String(messageExt.getBody(), DataConverter.charset);
|
|||
|
|
+ String raw = new String(messageExt.getBody(), DataConverter.CHARSET_UTF8);
|
|||
|
|
if (brokerController.getBrokerConfig().isEnablePopLog()) {
|
|||
|
|
POP_LOGGER.info("reviveQueueId={},find ack, offset:{}, raw : {}", messageExt.getQueueId(), messageExt.getQueueOffset(), raw);
|
|||
|
|
}
|
|||
|
|
@@ -395,7 +395,7 @@ public class PopReviveService extends ServiceThread {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
} else if (PopAckConstants.BATCH_ACK_TAG.equals(messageExt.getTags())) {
|
|||
|
|
- String raw = new String(messageExt.getBody(), DataConverter.charset);
|
|||
|
|
+ String raw = new String(messageExt.getBody(), DataConverter.CHARSET_UTF8);
|
|||
|
|
if (brokerController.getBrokerConfig().isEnablePopLog()) {
|
|||
|
|
POP_LOGGER.info("reviveQueueId={}, find batch ack, offset:{}, raw : {}", messageExt.getQueueId(), messageExt.getQueueOffset(), raw);
|
|||
|
|
}
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java
|
|||
|
|
index 6503970af..e9a81a8d6 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/RocksDBSubscriptionGroupManager.java
|
|||
|
|
@@ -30,7 +30,7 @@ public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager {
|
|||
|
|
|
|||
|
|
public RocksDBSubscriptionGroupManager(BrokerController brokerController) {
|
|||
|
|
super(brokerController, false);
|
|||
|
|
- this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval());
|
|||
|
|
+ this.rocksDBConfigManager = new RocksDBConfigManager(brokerController.getMessageStoreConfig().getMemTableFlushIntervalMs());
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
@@ -53,7 +53,7 @@ public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager {
|
|||
|
|
SubscriptionGroupConfig oldConfig = this.subscriptionGroupTable.put(groupName, subscriptionGroupConfig);
|
|||
|
|
|
|||
|
|
try {
|
|||
|
|
- byte[] keyBytes = groupName.getBytes(DataConverter.charset);
|
|||
|
|
+ byte[] keyBytes = groupName.getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible);
|
|||
|
|
this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes);
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
@@ -68,7 +68,7 @@ public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager {
|
|||
|
|
SubscriptionGroupConfig oldConfig = this.subscriptionGroupTable.putIfAbsent(groupName, subscriptionGroupConfig);
|
|||
|
|
if (oldConfig == null) {
|
|||
|
|
try {
|
|||
|
|
- byte[] keyBytes = groupName.getBytes(DataConverter.charset);
|
|||
|
|
+ byte[] keyBytes = groupName.getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible);
|
|||
|
|
this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes);
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
@@ -82,7 +82,7 @@ public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager {
|
|||
|
|
protected SubscriptionGroupConfig removeSubscriptionGroupConfig(String groupName) {
|
|||
|
|
SubscriptionGroupConfig subscriptionGroupConfig = this.subscriptionGroupTable.remove(groupName);
|
|||
|
|
try {
|
|||
|
|
- this.rocksDBConfigManager.delete(groupName.getBytes(DataConverter.charset));
|
|||
|
|
+ this.rocksDBConfigManager.delete(groupName.getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
log.error("kv delete sub Failed, {}", subscriptionGroupConfig.toString());
|
|||
|
|
}
|
|||
|
|
@@ -91,7 +91,7 @@ public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
protected void decode0(byte[] key, byte[] body) {
|
|||
|
|
- String groupName = new String(key, DataConverter.charset);
|
|||
|
|
+ String groupName = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
SubscriptionGroupConfig subscriptionGroupConfig = JSON.parseObject(body, SubscriptionGroupConfig.class);
|
|||
|
|
|
|||
|
|
this.subscriptionGroupTable.put(groupName, subscriptionGroupConfig);
|
|||
|
|
diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java
|
|||
|
|
index 7da0d7c8a..fddecf2d9 100644
|
|||
|
|
--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java
|
|||
|
|
+++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/RocksDBTopicConfigManager.java
|
|||
|
|
@@ -30,7 +30,7 @@ public class RocksDBTopicConfigManager extends TopicConfigManager {
|
|||
|
|
|
|||
|
|
public RocksDBTopicConfigManager(BrokerController brokerController) {
|
|||
|
|
super(brokerController, false);
|
|||
|
|
- this.rocksDBConfigManager = new RocksDBConfigManager(this.brokerController.getMessageStoreConfig().getMemTableFlushInterval());
|
|||
|
|
+ this.rocksDBConfigManager = new RocksDBConfigManager(brokerController.getMessageStoreConfig().getMemTableFlushIntervalMs());
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
@@ -49,7 +49,7 @@ public class RocksDBTopicConfigManager extends TopicConfigManager {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
protected void decode0(byte[] key, byte[] body) {
|
|||
|
|
- String topicName = new String(key, DataConverter.charset);
|
|||
|
|
+ String topicName = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
TopicConfig topicConfig = JSON.parseObject(body, TopicConfig.class);
|
|||
|
|
|
|||
|
|
this.topicConfigTable.put(topicName, topicConfig);
|
|||
|
|
@@ -66,7 +66,7 @@ public class RocksDBTopicConfigManager extends TopicConfigManager {
|
|||
|
|
String topicName = topicConfig.getTopicName();
|
|||
|
|
TopicConfig oldTopicConfig = this.topicConfigTable.put(topicName, topicConfig);
|
|||
|
|
try {
|
|||
|
|
- byte[] keyBytes = topicName.getBytes(DataConverter.charset);
|
|||
|
|
+ byte[] keyBytes = topicName.getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
byte[] valueBytes = JSON.toJSONBytes(topicConfig, SerializerFeature.BrowserCompatible);
|
|||
|
|
this.rocksDBConfigManager.put(keyBytes, keyBytes.length, valueBytes);
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
@@ -79,7 +79,7 @@ public class RocksDBTopicConfigManager extends TopicConfigManager {
|
|||
|
|
protected TopicConfig removeTopicConfig(String topicName) {
|
|||
|
|
TopicConfig topicConfig = this.topicConfigTable.remove(topicName);
|
|||
|
|
try {
|
|||
|
|
- this.rocksDBConfigManager.delete(topicName.getBytes(DataConverter.charset));
|
|||
|
|
+ this.rocksDBConfigManager.delete(topicName.getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
log.error("kv remove topic Failed, {}", topicConfig.toString());
|
|||
|
|
}
|
|||
|
|
diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java
|
|||
|
|
index 1c3a0cd45..78b76264f 100644
|
|||
|
|
--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java
|
|||
|
|
+++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java
|
|||
|
|
@@ -234,7 +234,7 @@ public class PopReviveServiceTest {
|
|||
|
|
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
|
|||
|
|
|
|||
|
|
msgInner.setTopic(REVIVE_TOPIC);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ck).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(REVIVE_QUEUE_ID);
|
|||
|
|
msgInner.setTags(PopAckConstants.CK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
@@ -269,7 +269,7 @@ public class PopReviveServiceTest {
|
|||
|
|
SocketAddress host, long deliverMs, String ackUniqueId) {
|
|||
|
|
MessageExtBrokerInner msgInner = new MessageExtBrokerInner();
|
|||
|
|
msgInner.setTopic(reviveTopic);
|
|||
|
|
- msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.charset));
|
|||
|
|
+ msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8));
|
|||
|
|
msgInner.setQueueId(reviveQid);
|
|||
|
|
msgInner.setTags(PopAckConstants.ACK_TAG);
|
|||
|
|
msgInner.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
diff --git a/common/pom.xml b/common/pom.xml
|
|||
|
|
index 6104c3ac6..a28ed228f 100644
|
|||
|
|
--- a/common/pom.xml
|
|||
|
|
+++ b/common/pom.xml
|
|||
|
|
@@ -109,7 +109,7 @@
|
|||
|
|
<artifactId>rocketmq-logback-classic</artifactId>
|
|||
|
|
</dependency>
|
|||
|
|
<dependency>
|
|||
|
|
- <groupId>io.github.aliyunmq</groupId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
<artifactId>rocketmq-rocksdb</artifactId>
|
|||
|
|
</dependency>
|
|||
|
|
</dependencies>
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java
|
|||
|
|
index 1233a5422..407ef2842 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java
|
|||
|
|
@@ -492,6 +492,7 @@ public class MixAll {
|
|||
|
|
public static int compareLong(long x, long y) {
|
|||
|
|
return Long.compare(x, y);
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
public static boolean isLmq(String lmqMetaData) {
|
|||
|
|
return lmqMetaData != null && lmqMetaData.startsWith(LMQ_PREFIX);
|
|||
|
|
}
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/attribute/CQType.java b/common/src/main/java/org/apache/rocketmq/common/attribute/CQType.java
|
|||
|
|
index 73ef21880..9148d5a18 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/attribute/CQType.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/attribute/CQType.java
|
|||
|
|
@@ -19,5 +19,6 @@ package org.apache.rocketmq.common.attribute;
|
|||
|
|
|
|||
|
|
public enum CQType {
|
|||
|
|
SimpleCQ,
|
|||
|
|
- BatchCQ
|
|||
|
|
+ BatchCQ,
|
|||
|
|
+ RocksDBCQ
|
|||
|
|
}
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
index 6f19a9815..20319abba 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
@@ -17,7 +17,6 @@
|
|||
|
|
package org.apache.rocketmq.common.config;
|
|||
|
|
|
|||
|
|
import java.nio.ByteBuffer;
|
|||
|
|
-import java.nio.charset.Charset;
|
|||
|
|
import java.util.ArrayList;
|
|||
|
|
import java.util.List;
|
|||
|
|
import java.util.Map;
|
|||
|
|
@@ -27,11 +26,11 @@ import java.util.concurrent.Semaphore;
|
|||
|
|
import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
import java.util.concurrent.TimeUnit;
|
|||
|
|
|
|||
|
|
-import com.google.common.collect.Lists;
|
|||
|
|
import com.google.common.collect.Maps;
|
|||
|
|
|
|||
|
|
import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
@@ -47,7 +46,6 @@ import org.rocksdb.Priority;
|
|||
|
|
import org.rocksdb.ReadOptions;
|
|||
|
|
import org.rocksdb.RocksDB;
|
|||
|
|
import org.rocksdb.RocksDBException;
|
|||
|
|
-import org.rocksdb.RocksIterator;
|
|||
|
|
import org.rocksdb.Statistics;
|
|||
|
|
import org.rocksdb.Status;
|
|||
|
|
import org.rocksdb.WriteBatch;
|
|||
|
|
@@ -58,7 +56,6 @@ import static org.rocksdb.RocksDB.NOT_FOUND;
|
|||
|
|
public abstract class AbstractRocksDBStorage {
|
|||
|
|
protected static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME);
|
|||
|
|
|
|||
|
|
- private static final Charset CHARSET_UTF8 = Charset.forName("UTF-8");
|
|||
|
|
private static final String SPACE = " | ";
|
|||
|
|
|
|||
|
|
protected String dbPath;
|
|||
|
|
@@ -223,10 +220,6 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- protected WrappedRocksIterator newIterator(ColumnFamilyHandle cfHandle, ReadOptions readOptions) {
|
|||
|
|
- return new WrappedRocksIterator(this.db.newIterator(cfHandle, readOptions));
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
protected void rangeDelete(ColumnFamilyHandle cfHandle, WriteOptions writeOptions,
|
|||
|
|
final byte[] startKey, final byte[] endKey) throws RocksDBException {
|
|||
|
|
if (!hold()) {
|
|||
|
|
@@ -243,46 +236,6 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- protected void manualCompactionDefaultCfMaxLevel(final CompactionOptions compactionOptions) throws Exception {
|
|||
|
|
- final ColumnFamilyHandle defaultCFHandle = this.defaultCFHandle;
|
|||
|
|
- final byte[] defaultCFName = defaultCFHandle.getName();
|
|||
|
|
- List<LiveFileMetaData> fileMetaDataList = this.db.getLiveFilesMetaData();
|
|||
|
|
- if (fileMetaDataList == null || fileMetaDataList.isEmpty()) {
|
|||
|
|
- return;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- List<LiveFileMetaData> defaultLiveFileDataList = Lists.newArrayList();
|
|||
|
|
- List<String> inputFileNames = Lists.newArrayList();
|
|||
|
|
- int maxLevel = 0;
|
|||
|
|
- for (LiveFileMetaData fileMetaData : fileMetaDataList) {
|
|||
|
|
- if (compareTo(fileMetaData.columnFamilyName(), defaultCFName) != 0) {
|
|||
|
|
- continue;
|
|||
|
|
- }
|
|||
|
|
- defaultLiveFileDataList.add(fileMetaData);
|
|||
|
|
- if (fileMetaData.level() > maxLevel) {
|
|||
|
|
- maxLevel = fileMetaData.level();
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- if (maxLevel == 0) {
|
|||
|
|
- LOGGER.info("manualCompactionDefaultCfFiles skip level 0.");
|
|||
|
|
- return;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- for (LiveFileMetaData fileMetaData : defaultLiveFileDataList) {
|
|||
|
|
- if (fileMetaData.level() != maxLevel || fileMetaData.beingCompacted()) {
|
|||
|
|
- continue;
|
|||
|
|
- }
|
|||
|
|
- inputFileNames.add(fileMetaData.path() + fileMetaData.fileName());
|
|||
|
|
- }
|
|||
|
|
- if (!inputFileNames.isEmpty()) {
|
|||
|
|
- List<String> outputLists = this.db.compactFiles(compactionOptions, defaultCFHandle,
|
|||
|
|
- inputFileNames, maxLevel, -1, null);
|
|||
|
|
- LOGGER.info("manualCompactionDefaultCfFiles OK. src: {}, dst: {}", inputFileNames, outputLists);
|
|||
|
|
- } else {
|
|||
|
|
- LOGGER.info("manualCompactionDefaultCfFiles Empty.");
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
protected void manualCompactionDefaultCfRange(CompactRangeOptions compactRangeOptions) {
|
|||
|
|
if (!hold()) {
|
|||
|
|
return;
|
|||
|
|
@@ -494,50 +447,6 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
this.db.flushWal(true);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- protected class WrappedRocksIterator {
|
|||
|
|
- private final RocksIterator iterator;
|
|||
|
|
-
|
|||
|
|
- public WrappedRocksIterator(final RocksIterator iterator) {
|
|||
|
|
- this.iterator = iterator;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public byte[] key() {
|
|||
|
|
- return iterator.key();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public byte[] value() {
|
|||
|
|
- return iterator.value();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void next() {
|
|||
|
|
- iterator.next();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void prev() {
|
|||
|
|
- iterator.prev();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void seek(byte[] target) {
|
|||
|
|
- iterator.seek(target);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void seekForPrev(byte[] target) {
|
|||
|
|
- iterator.seekForPrev(target);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void seekToFirst() {
|
|||
|
|
- iterator.seekToFirst();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public boolean isValid() {
|
|||
|
|
- return iterator.isValid();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void close() {
|
|||
|
|
- iterator.close();
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
private String getStatusError(RocksDBException e) {
|
|||
|
|
if (e == null || e.getStatus() == null) {
|
|||
|
|
return "null";
|
|||
|
|
@@ -574,7 +483,7 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
sb = new StringBuilder(256);
|
|||
|
|
map.put(metaData.level(), sb);
|
|||
|
|
}
|
|||
|
|
- sb.append(new String(metaData.columnFamilyName(), CHARSET_UTF8)).append(SPACE).
|
|||
|
|
+ sb.append(new String(metaData.columnFamilyName(), DataConverter.CHARSET_UTF8)).append(SPACE).
|
|||
|
|
append(metaData.fileName()).append(SPACE).
|
|||
|
|
append("s: ").append(metaData.size()).append(SPACE).
|
|||
|
|
append("a: ").append(metaData.numEntries()).append(SPACE).
|
|||
|
|
@@ -595,21 +504,4 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
} catch (Exception ignored) {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
- public int compareTo(byte[] v1, byte[] v2) {
|
|||
|
|
- int len1 = v1.length;
|
|||
|
|
- int len2 = v2.length;
|
|||
|
|
- int lim = Math.min(len1, len2);
|
|||
|
|
-
|
|||
|
|
- int k = 0;
|
|||
|
|
- while (k < lim) {
|
|||
|
|
- byte c1 = v1[k];
|
|||
|
|
- byte c2 = v2[k];
|
|||
|
|
- if (c1 != c2) {
|
|||
|
|
- return c1 - c2;
|
|||
|
|
- }
|
|||
|
|
- k++;
|
|||
|
|
- }
|
|||
|
|
- return len1 - len2;
|
|||
|
|
- }
|
|||
|
|
}
|
|||
|
|
\ No newline at end of file
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
index 463bd8fed..b40f8046e 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
@@ -203,7 +203,7 @@ public class ConfigRocksDBStorage extends AbstractRocksDBStorage {
|
|||
|
|
setUseDirectReads(true);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private static String getDBLogDir() {
|
|||
|
|
+ public static String getDBLogDir() {
|
|||
|
|
String rootPath = System.getProperty("user.home");
|
|||
|
|
if (StringUtils.isEmpty(rootPath)) {
|
|||
|
|
return "";
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java
|
|||
|
|
index 0c72ebb7b..91599653c 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageExtBrokerInner.java
|
|||
|
|
@@ -70,4 +70,17 @@ public class MessageExtBrokerInner extends MessageExt {
|
|||
|
|
public void setVersion(MessageVersion version) {
|
|||
|
|
this.version = version;
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ public void removeWaitStorePropertyString() {
|
|||
|
|
+ if (this.getProperties().containsKey(MessageConst.PROPERTY_WAIT_STORE_MSG_OK)) {
|
|||
|
|
+ // There is no need to store "WAIT=true", remove it from propertiesString to save 9 bytes for each message.
|
|||
|
|
+ // It works for most case. In some cases msgInner.setPropertiesString invoked later and replace it.
|
|||
|
|
+ String waitStoreMsgOKValue = this.getProperties().remove(MessageConst.PROPERTY_WAIT_STORE_MSG_OK);
|
|||
|
|
+ this.setPropertiesString(MessageDecoder.messageProperties2String(this.getProperties()));
|
|||
|
|
+ // Reput to properties, since msgInner.isWaitStoreMsgOK() will be invoked later
|
|||
|
|
+ this.getProperties().put(MessageConst.PROPERTY_WAIT_STORE_MSG_OK, waitStoreMsgOKValue);
|
|||
|
|
+ } else {
|
|||
|
|
+ this.setPropertiesString(MessageDecoder.messageProperties2String(this.getProperties()));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
|
|||
|
|
index 61265c05d..c19592a44 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java
|
|||
|
|
@@ -31,6 +31,7 @@ public class TopicValidator {
|
|||
|
|
public static final String RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC = "TRANS_CHECK_MAX_TIME_TOPIC";
|
|||
|
|
public static final String RMQ_SYS_SELF_TEST_TOPIC = "SELF_TEST_TOPIC";
|
|||
|
|
public static final String RMQ_SYS_OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT";
|
|||
|
|
+ public static final String RMQ_SYS_ROCKSDB_OFFSET_TOPIC = "CHECKPOINT_TOPIC";
|
|||
|
|
|
|||
|
|
public static final String SYSTEM_TOPIC_PREFIX = "rmq_sys_";
|
|||
|
|
public static final String SYNC_BROKER_MEMBER_GROUP_PREFIX = SYSTEM_TOPIC_PREFIX + "SYNC_BROKER_MEMBER_";
|
|||
|
|
@@ -55,6 +56,7 @@ public class TopicValidator {
|
|||
|
|
SYSTEM_TOPIC_SET.add(RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC);
|
|||
|
|
SYSTEM_TOPIC_SET.add(RMQ_SYS_SELF_TEST_TOPIC);
|
|||
|
|
SYSTEM_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT);
|
|||
|
|
+ SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_OFFSET_TOPIC);
|
|||
|
|
|
|||
|
|
NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SCHEDULE_TOPIC);
|
|||
|
|
NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_TRANS_HALF_TOPIC);
|
|||
|
|
diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/DataConverter.java b/common/src/main/java/org/apache/rocketmq/common/utils/DataConverter.java
|
|||
|
|
index 8b50de12b..cc96770b2 100644
|
|||
|
|
--- a/common/src/main/java/org/apache/rocketmq/common/utils/DataConverter.java
|
|||
|
|
+++ b/common/src/main/java/org/apache/rocketmq/common/utils/DataConverter.java
|
|||
|
|
@@ -20,7 +20,7 @@ import java.nio.ByteBuffer;
|
|||
|
|
import java.nio.charset.Charset;
|
|||
|
|
|
|||
|
|
public class DataConverter {
|
|||
|
|
- public static Charset charset = Charset.forName("UTF-8");
|
|||
|
|
+ public static final Charset CHARSET_UTF8 = Charset.forName("UTF-8");
|
|||
|
|
|
|||
|
|
public static byte[] Long2Byte(Long v) {
|
|||
|
|
ByteBuffer tmp = ByteBuffer.allocate(8);
|
|||
|
|
diff --git a/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java b/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java
|
|||
|
|
index aac295030..8662328ea 100644
|
|||
|
|
--- a/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java
|
|||
|
|
+++ b/example/src/main/java/org/apache/rocketmq/example/quickstart/Producer.java
|
|||
|
|
@@ -71,7 +71,7 @@ public class Producer {
|
|||
|
|
TAG /* Tag */,
|
|||
|
|
("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */
|
|||
|
|
);
|
|||
|
|
-
|
|||
|
|
+ msg.setDelayTimeLevel(1);
|
|||
|
|
/*
|
|||
|
|
* Call send message to deliver message to one of brokers.
|
|||
|
|
*/
|
|||
|
|
diff --git a/metrics b/metrics
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..146e90fae
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/metrics
|
|||
|
|
@@ -0,0 +1,241 @@
|
|||
|
|
+# TYPE target info
|
|||
|
|
+# HELP target Target metadata
|
|||
|
|
+target_info{} 1
|
|||
|
|
+# TYPE otel_scope_info info
|
|||
|
|
+# HELP otel_scope_info Scope metadata
|
|||
|
|
+otel_scope_info{otel_scope_name="broker-meter"} 1
|
|||
|
|
+# TYPE rocketmq_storage_message_reserve_time gauge
|
|||
|
|
+# HELP rocketmq_storage_message_reserve_time default view for gauge.
|
|||
|
|
+rocketmq_storage_message_reserve_time_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 1.9491132515E10 1696905287361
|
|||
|
|
+# TYPE rocketmq_timer_enqueue_latency gauge
|
|||
|
|
+# HELP rocketmq_timer_enqueue_latency default view for gauge.
|
|||
|
|
+rocketmq_timer_enqueue_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_timer_enqueue_lag gauge
|
|||
|
|
+# HELP rocketmq_timer_enqueue_lag default view for gauge.
|
|||
|
|
+rocketmq_timer_enqueue_lag{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_timing_messages gauge
|
|||
|
|
+# HELP rocketmq_timing_messages default view for gauge.
|
|||
|
|
+rocketmq_timing_messages{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local",topic="BenchmarkTest"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_storage_flush_behind_bytes gauge
|
|||
|
|
+# HELP rocketmq_storage_flush_behind_bytes default view for gauge.
|
|||
|
|
+rocketmq_storage_flush_behind_bytes{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 1230.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_processor_watermark gauge
|
|||
|
|
+# HELP rocketmq_processor_watermark default view for gauge.
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="reply"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="transaction"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="send"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="async_put"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="lite_pull"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="pull"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="heartbeat"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="consumer_manager"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="admin"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="client_manager"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="query_message"} 0.0 1696905287361
|
|||
|
|
+rocketmq_processor_watermark{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",processor="ack"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_pop_buffer_scan_time_consume histogram
|
|||
|
|
+# HELP rocketmq_pop_buffer_scan_time_consume Time consuming of pop buffer scan
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker"} 5267005.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker"} 39651.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="1.0"} 5266935.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="10.0"} 5266998.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="100.0"} 5267001.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="1000.0"} 5267005.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="2000.0"} 5267005.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="3000.0"} 5267005.0 1696905287361
|
|||
|
|
+rocketmq_pop_buffer_scan_time_consume_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",le="+Inf"} 5267005.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_pop_revive_lag gauge
|
|||
|
|
+# HELP rocketmq_pop_revive_lag default view for gauge.
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="1"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="2"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="3"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="4"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="5"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="6"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="7"} 1.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_lag_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="0"} 1.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_pop_revive_latency gauge
|
|||
|
|
+# HELP rocketmq_pop_revive_latency default view for gauge.
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="1"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="2"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="3"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="4"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="5"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="6"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="7"} 0.0 1696905287361
|
|||
|
|
+rocketmq_pop_revive_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",queue_id="0"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_storage_size gauge
|
|||
|
|
+# HELP rocketmq_storage_size default view for gauge.
|
|||
|
|
+rocketmq_storage_size_bytes{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 1.5905103872E10 1696905287361
|
|||
|
|
+# TYPE rocketmq_pop_offset_buffer_size gauge
|
|||
|
|
+# HELP rocketmq_pop_offset_buffer_size default view for gauge.
|
|||
|
|
+rocketmq_pop_offset_buffer_size{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_timer_dequeue_latency gauge
|
|||
|
|
+# HELP rocketmq_timer_dequeue_latency default view for gauge.
|
|||
|
|
+rocketmq_timer_dequeue_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_broker_permission gauge
|
|||
|
|
+# HELP rocketmq_broker_permission default view for gauge.
|
|||
|
|
+rocketmq_broker_permission{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker"} 6.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_lag_messages gauge
|
|||
|
|
+# HELP rocketmq_consumer_lag_messages default view for gauge.
|
|||
|
|
+rocketmq_consumer_lag_messages{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_retry="false",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_throughput_out_total counter
|
|||
|
|
+# HELP rocketmq_throughput_out_total default view for counter.
|
|||
|
|
+rocketmq_throughput_out_total{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 1230.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_timer_dequeue_lag gauge
|
|||
|
|
+# HELP rocketmq_timer_dequeue_lag default view for gauge.
|
|||
|
|
+rocketmq_timer_dequeue_lag{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_messages_out_total counter
|
|||
|
|
+# HELP rocketmq_messages_out_total default view for counter.
|
|||
|
|
+rocketmq_messages_out_total{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 5.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_messages_in_total counter
|
|||
|
|
+# HELP rocketmq_messages_in_total default view for counter.
|
|||
|
|
+rocketmq_messages_in_total{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 5.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_queueing_latency gauge
|
|||
|
|
+# HELP rocketmq_consumer_queueing_latency default view for gauge.
|
|||
|
|
+rocketmq_consumer_queueing_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_retry="false",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_connections gauge
|
|||
|
|
+# HELP rocketmq_consumer_connections default view for gauge.
|
|||
|
|
+rocketmq_consumer_connections{otel_scope_name="broker-meter",cluster="DefaultCluster",consume_mode="push",consumer_group="MQ_INST_test%GID_222",is_system="false",language="java",node_id="broker-a",node_type="broker",protocol_type="remoting",version="v4_8_0"} 1.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_rpc_latency histogram
|
|||
|
|
+# HELP rocketmq_rpc_latency Rpc latency
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success"} 128.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="1.0"} 17.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="3.0"} 18.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="5.0"} 19.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="7.0"} 19.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="10.0"} 20.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="100.0"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="1000.0"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="2000.0"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="3000.0"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="heart_beat",response_code="success",result="success",le="+Inf"} 24.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success"} 8.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="1.0"} 3.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="3.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="5.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="7.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="10.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="100.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="1000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="2000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="3000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="query_consumer_offset",response_code="success",result="success",le="+Inf"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success"} 54.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="1.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="3.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="5.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="7.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="10.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="100.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="1000.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="2000.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="3000.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="subscription_not_latest",result="success",le="+Inf"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success"} 108.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="1.0"} 54.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="3.0"} 54.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="5.0"} 54.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="7.0"} 54.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="10.0"} 55.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="100.0"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="1000.0"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="2000.0"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="3000.0"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="get_consumer_list_by_group",response_code="success",result="success",le="+Inf"} 60.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success"} 210.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="1.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="3.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="5.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="7.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="10.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="100.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="1000.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="2000.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="3000.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="success",result="success",le="+Inf"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success"} 155.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success"} 2811756.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="1.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="3.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="5.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="7.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="10.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="100.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="1000.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="2000.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="3000.0"} 0.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="true",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="pull_message",response_code="pull_not_found",result="success",le="+Inf"} 155.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success"} 225.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="1.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="3.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="5.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="7.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="10.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="100.0"} 4.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="1000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="2000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="3000.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="send_message_v2",response_code="success",result="success",le="+Inf"} 5.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway"} 84.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="1.0"} 551.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="3.0"} 557.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="5.0"} 558.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="7.0"} 560.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="10.0"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="100.0"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="1000.0"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="2000.0"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="3000.0"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="update_consumer_offset",response_code="success",result="oneway",le="+Inf"} 565.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success"} 6.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="1.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="3.0"} 1.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="5.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="7.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="10.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="100.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="1000.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="2000.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="3000.0"} 2.0 1696905287361
|
|||
|
|
+rocketmq_rpc_latency_milliseconds_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_long_polling="false",node_id="broker-a",node_type="broker",protocol_type="remoting",request_code="unregister_client",response_code="success",result="success",le="+Inf"} 2.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_storage_dispatch_behind_bytes gauge
|
|||
|
|
+# HELP rocketmq_storage_dispatch_behind_bytes default view for gauge.
|
|||
|
|
+rocketmq_storage_dispatch_behind_bytes{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker",storage_medium="disk",storage_type="local"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_ready_messages gauge
|
|||
|
|
+# HELP rocketmq_consumer_ready_messages default view for gauge.
|
|||
|
|
+rocketmq_consumer_ready_messages{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_retry="false",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_lag_latency gauge
|
|||
|
|
+# HELP rocketmq_consumer_lag_latency default view for gauge.
|
|||
|
|
+rocketmq_consumer_lag_latency_milliseconds{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_retry="false",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_throughput_in_total counter
|
|||
|
|
+# HELP rocketmq_throughput_in_total default view for counter.
|
|||
|
|
+rocketmq_throughput_in_total{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 1230.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_pop_checkpoint_buffer_size gauge
|
|||
|
|
+# HELP rocketmq_pop_checkpoint_buffer_size default view for gauge.
|
|||
|
|
+rocketmq_pop_checkpoint_buffer_size{otel_scope_name="broker-meter",cluster="DefaultCluster",node_id="broker-a",node_type="broker"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_message_size histogram
|
|||
|
|
+# HELP rocketmq_message_size Incoming messages size
|
|||
|
|
+rocketmq_message_size_count{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_sum{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 1230.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="1024.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="4096.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="524288.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="1048576.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="2097152.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="4194304.0"} 5.0 1696905287361
|
|||
|
|
+rocketmq_message_size_bucket{otel_scope_name="broker-meter",cluster="DefaultCluster",is_system="false",message_type="normal",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl",le="+Inf"} 5.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_consumer_inflight_messages gauge
|
|||
|
|
+# HELP rocketmq_consumer_inflight_messages default view for gauge.
|
|||
|
|
+rocketmq_consumer_inflight_messages{otel_scope_name="broker-meter",cluster="DefaultCluster",consumer_group="MQ_INST_test%GID_222",is_retry="false",is_system="false",node_id="broker-a",node_type="broker",topic="MQ_INST_test%zzl"} 0.0 1696905287361
|
|||
|
|
+# TYPE rocketmq_producer_connections gauge
|
|||
|
|
+# HELP rocketmq_producer_connections default view for gauge.
|
|||
|
|
+rocketmq_producer_connections{otel_scope_name="broker-meter",cluster="DefaultCluster",language="java",node_id="broker-a",node_type="broker",protocol_type="remoting",version="v4_8_0"} 1.0 1696905287361
|
|||
|
|
diff --git a/patch009-modify-CRLF-to-LF.patch b/patch009-modify-CRLF-to-LF.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..c9f1f1843
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch009-modify-CRLF-to-LF.patch
|
|||
|
|
@@ -0,0 +1,127 @@
|
|||
|
|
+From 81cd734ee33586544694675bd1081f2c81024d22 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: zhiliatom <zhiliatox@163.com>
|
|||
|
|
+Date: Thu, 28 Sep 2023 09:58:13 +0800
|
|||
|
|
+Subject: [PATCH] modify line separater from CRLF to LF
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/common/constant/LoggerName.java | 110 +++++++++---------
|
|||
|
|
+ 1 file changed, 55 insertions(+), 55 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
+index cb04b00b3..5c7afab9f 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
+@@ -1,55 +1,55 @@
|
|||
|
|
+-/*
|
|||
|
|
+- * 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.common.constant;
|
|||
|
|
+-
|
|||
|
|
+-public class LoggerName {
|
|||
|
|
+- public static final String FILTERSRV_LOGGER_NAME = "RocketmqFiltersrv";
|
|||
|
|
+- public static final String NAMESRV_LOGGER_NAME = "RocketmqNamesrv";
|
|||
|
|
+- public static final String NAMESRV_CONSOLE_LOGGER_NAME = "RocketmqNamesrvConsole";
|
|||
|
|
+- public static final String CONTROLLER_LOGGER_NAME = "RocketmqController";
|
|||
|
|
+- public static final String NAMESRV_WATER_MARK_LOGGER_NAME = "RocketmqNamesrvWaterMark";
|
|||
|
|
+- public static final String BROKER_LOGGER_NAME = "RocketmqBroker";
|
|||
|
|
+- public static final String BROKER_CONSOLE_NAME = "RocketmqConsole";
|
|||
|
|
+- public static final String CLIENT_LOGGER_NAME = "RocketmqClient";
|
|||
|
|
+- public static final String ROCKETMQ_TRAFFIC_NAME = "RocketmqTraffic";
|
|||
|
|
+- public static final String ROCKETMQ_REMOTING_NAME = "RocketmqRemoting";
|
|||
|
|
+- public static final String TOOLS_LOGGER_NAME = "RocketmqTools";
|
|||
|
|
+- public static final String COMMON_LOGGER_NAME = "RocketmqCommon";
|
|||
|
|
+- public static final String STORE_LOGGER_NAME = "RocketmqStore";
|
|||
|
|
+- public static final String STORE_ERROR_LOGGER_NAME = "RocketmqStoreError";
|
|||
|
|
+- public static final String TRANSACTION_LOGGER_NAME = "RocketmqTransaction";
|
|||
|
|
+- public static final String REBALANCE_LOCK_LOGGER_NAME = "RocketmqRebalanceLock";
|
|||
|
|
+- public static final String ROCKETMQ_STATS_LOGGER_NAME = "RocketmqStats";
|
|||
|
|
+- public static final String DLQ_STATS_LOGGER_NAME = "RocketmqDLQStats";
|
|||
|
|
+- public static final String DLQ_LOGGER_NAME = "RocketmqDLQ";
|
|||
|
|
+- public static final String CONSUMER_STATS_LOGGER_NAME = "RocketmqConsumerStats";
|
|||
|
|
+- public static final String COMMERCIAL_LOGGER_NAME = "RocketmqCommercial";
|
|||
|
|
+- public static final String ACCOUNT_LOGGER_NAME = "RocketmqAccount";
|
|||
|
|
+- public static final String FLOW_CONTROL_LOGGER_NAME = "RocketmqFlowControl";
|
|||
|
|
+- public static final String ROCKETMQ_AUTHORIZE_LOGGER_NAME = "RocketmqAuthorize";
|
|||
|
|
+- public static final String DUPLICATION_LOGGER_NAME = "RocketmqDuplication";
|
|||
|
|
+- public static final String PROTECTION_LOGGER_NAME = "RocketmqProtection";
|
|||
|
|
+- public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark";
|
|||
|
|
+- public static final String FILTER_LOGGER_NAME = "RocketmqFilter";
|
|||
|
|
+- public static final String ROCKETMQ_POP_LOGGER_NAME = "RocketmqPop";
|
|||
|
|
+- public static final String FAILOVER_LOGGER_NAME = "RocketmqFailover";
|
|||
|
|
+- public static final String STDOUT_LOGGER_NAME = "STDOUT";
|
|||
|
|
+- public static final String PROXY_LOGGER_NAME = "RocketmqProxy";
|
|||
|
|
+- public static final String PROXY_WATER_MARK_LOGGER_NAME = "RocketmqProxyWatermark";
|
|||
|
|
+- public static final String ROCKETMQ_COLDCTR_LOGGER_NAME = "RocketmqColdCtr";
|
|||
|
|
+- public static final String ROCKSDB_LOGGER_NAME = "RocketmqRocksDB";
|
|||
|
|
+-}
|
|||
|
|
++/*
|
|||
|
|
++ * 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.common.constant;
|
|||
|
|
++
|
|||
|
|
++public class LoggerName {
|
|||
|
|
++ public static final String FILTERSRV_LOGGER_NAME = "RocketmqFiltersrv";
|
|||
|
|
++ public static final String NAMESRV_LOGGER_NAME = "RocketmqNamesrv";
|
|||
|
|
++ public static final String NAMESRV_CONSOLE_LOGGER_NAME = "RocketmqNamesrvConsole";
|
|||
|
|
++ public static final String CONTROLLER_LOGGER_NAME = "RocketmqController";
|
|||
|
|
++ public static final String NAMESRV_WATER_MARK_LOGGER_NAME = "RocketmqNamesrvWaterMark";
|
|||
|
|
++ public static final String BROKER_LOGGER_NAME = "RocketmqBroker";
|
|||
|
|
++ public static final String BROKER_CONSOLE_NAME = "RocketmqConsole";
|
|||
|
|
++ public static final String CLIENT_LOGGER_NAME = "RocketmqClient";
|
|||
|
|
++ public static final String ROCKETMQ_TRAFFIC_NAME = "RocketmqTraffic";
|
|||
|
|
++ public static final String ROCKETMQ_REMOTING_NAME = "RocketmqRemoting";
|
|||
|
|
++ public static final String TOOLS_LOGGER_NAME = "RocketmqTools";
|
|||
|
|
++ public static final String COMMON_LOGGER_NAME = "RocketmqCommon";
|
|||
|
|
++ public static final String STORE_LOGGER_NAME = "RocketmqStore";
|
|||
|
|
++ public static final String STORE_ERROR_LOGGER_NAME = "RocketmqStoreError";
|
|||
|
|
++ public static final String TRANSACTION_LOGGER_NAME = "RocketmqTransaction";
|
|||
|
|
++ public static final String REBALANCE_LOCK_LOGGER_NAME = "RocketmqRebalanceLock";
|
|||
|
|
++ public static final String ROCKETMQ_STATS_LOGGER_NAME = "RocketmqStats";
|
|||
|
|
++ public static final String DLQ_STATS_LOGGER_NAME = "RocketmqDLQStats";
|
|||
|
|
++ public static final String DLQ_LOGGER_NAME = "RocketmqDLQ";
|
|||
|
|
++ public static final String CONSUMER_STATS_LOGGER_NAME = "RocketmqConsumerStats";
|
|||
|
|
++ public static final String COMMERCIAL_LOGGER_NAME = "RocketmqCommercial";
|
|||
|
|
++ public static final String ACCOUNT_LOGGER_NAME = "RocketmqAccount";
|
|||
|
|
++ public static final String FLOW_CONTROL_LOGGER_NAME = "RocketmqFlowControl";
|
|||
|
|
++ public static final String ROCKETMQ_AUTHORIZE_LOGGER_NAME = "RocketmqAuthorize";
|
|||
|
|
++ public static final String DUPLICATION_LOGGER_NAME = "RocketmqDuplication";
|
|||
|
|
++ public static final String PROTECTION_LOGGER_NAME = "RocketmqProtection";
|
|||
|
|
++ public static final String WATER_MARK_LOGGER_NAME = "RocketmqWaterMark";
|
|||
|
|
++ public static final String FILTER_LOGGER_NAME = "RocketmqFilter";
|
|||
|
|
++ public static final String ROCKETMQ_POP_LOGGER_NAME = "RocketmqPop";
|
|||
|
|
++ public static final String FAILOVER_LOGGER_NAME = "RocketmqFailover";
|
|||
|
|
++ public static final String STDOUT_LOGGER_NAME = "STDOUT";
|
|||
|
|
++ public static final String PROXY_LOGGER_NAME = "RocketmqProxy";
|
|||
|
|
++ public static final String PROXY_WATER_MARK_LOGGER_NAME = "RocketmqProxyWatermark";
|
|||
|
|
++ public static final String ROCKETMQ_COLDCTR_LOGGER_NAME = "RocketmqColdCtr";
|
|||
|
|
++ public static final String ROCKSDB_LOGGER_NAME = "RocketmqRocksDB";
|
|||
|
|
++}
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch010-backport-add-some-fixes b/patch010-backport-add-some-fixes
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..36d5e4a1f
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch010-backport-add-some-fixes
|
|||
|
|
@@ -0,0 +1,1286 @@
|
|||
|
|
+From b2deef179dbc6a9eb1a2b6dd7b652d95cb768295 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Thu, 10 Aug 2023 10:38:47 +0800
|
|||
|
|
+Subject: [PATCH 01/12] [ISSUE #7144] Accelerate the recovery speed of the
|
|||
|
|
+ tiered storage module (#7145)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/TieredDispatcher.java | 3 +
|
|||
|
|
+ .../tieredstore/TieredMessageStore.java | 2 +-
|
|||
|
|
+ .../common/TieredStoreExecutor.java | 25 ++--
|
|||
|
|
+ .../tieredstore/file/CompositeFlatFile.java | 15 +-
|
|||
|
|
+ .../file/CompositeQueueFlatFile.java | 20 ++-
|
|||
|
|
+ .../tieredstore/file/TieredCommitLog.java | 24 +++-
|
|||
|
|
+ .../tieredstore/file/TieredFlatFile.java | 42 +++---
|
|||
|
|
+ .../file/TieredFlatFileManager.java | 135 ++++++++++--------
|
|||
|
|
+ .../metadata/FileSegmentMetadata.java | 26 +++-
|
|||
|
|
+ .../tieredstore/TieredDispatcherTest.java | 15 +-
|
|||
|
|
+ .../tieredstore/TieredMessageFetcherTest.java | 2 +-
|
|||
|
|
+ .../file/CompositeQueueFlatFileTest.java | 2 +-
|
|||
|
|
+ .../file/TieredFlatFileManagerTest.java | 7 +-
|
|||
|
|
+ 13 files changed, 194 insertions(+), 124 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+index bb58ea7dd..1746190cd 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+@@ -279,6 +279,9 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ long upperBound = Math.min(dispatchOffset + maxCount, maxOffsetInQueue);
|
|||
|
|
+ ConsumeQueue consumeQueue = (ConsumeQueue) defaultStore.getConsumeQueue(topic, queueId);
|
|||
|
|
+
|
|||
|
|
++ logger.debug("DispatchFlatFile race, topic={}, queueId={}, cq range={}-{}, dispatch offset={}-{}",
|
|||
|
|
++ topic, queueId, minOffsetInQueue, maxOffsetInQueue, dispatchOffset, upperBound - 1);
|
|||
|
|
++
|
|||
|
|
+ for (; dispatchOffset < upperBound; dispatchOffset++) {
|
|||
|
|
+ // get consume queue
|
|||
|
|
+ SelectMappedBufferResult cqItem = consumeQueue.getIndexBuffer(dispatchOffset);
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+index 1f12410f2..ced1fb818 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+@@ -147,7 +147,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ int queueId, long offset, int maxMsgNums, MessageFilter messageFilter) {
|
|||
|
|
+
|
|||
|
|
+ if (!viaTieredStorage(topic, queueId, offset, maxMsgNums)) {
|
|||
|
|
+- logger.debug("GetMessageAsync from next store topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
++ logger.trace("GetMessageAsync from next store topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
+ return next.getMessageAsync(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
+index 6eb3478b3..6dd0e8846 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
+@@ -43,18 +43,9 @@ public class TieredStoreExecutor {
|
|||
|
|
+ public static ExecutorService compactIndexFileExecutor;
|
|||
|
|
+
|
|||
|
|
+ public static void init() {
|
|||
|
|
+- dispatchThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+- dispatchExecutor = new ThreadPoolExecutor(
|
|||
|
|
+- Math.max(2, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+- Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
+- 1000 * 60,
|
|||
|
|
+- TimeUnit.MILLISECONDS,
|
|||
|
|
+- dispatchThreadPoolQueue,
|
|||
|
|
+- new ThreadFactoryImpl("TieredCommonExecutor_"));
|
|||
|
|
+-
|
|||
|
|
+ commonScheduledExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
+ Math.max(4, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+- new ThreadFactoryImpl("TieredCommonScheduledExecutor_"));
|
|||
|
|
++ new ThreadFactoryImpl("TieredCommonExecutor_"));
|
|||
|
|
+
|
|||
|
|
+ commitExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
+ Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
+@@ -62,7 +53,17 @@ public class TieredStoreExecutor {
|
|||
|
|
+
|
|||
|
|
+ cleanExpiredFileExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
+ Math.max(4, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+- new ThreadFactoryImpl("TieredCleanExpiredFileExecutor_"));
|
|||
|
|
++ new ThreadFactoryImpl("TieredCleanFileExecutor_"));
|
|||
|
|
++
|
|||
|
|
++ dispatchThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
++ dispatchExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ Math.max(2, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
++ Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
++ 1000 * 60,
|
|||
|
|
++ TimeUnit.MILLISECONDS,
|
|||
|
|
++ dispatchThreadPoolQueue,
|
|||
|
|
++ new ThreadFactoryImpl("TieredDispatchExecutor_"),
|
|||
|
|
++ new ThreadPoolExecutor.DiscardOldestPolicy());
|
|||
|
|
+
|
|||
|
|
+ fetchDataThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+ fetchDataExecutor = new ThreadPoolExecutor(
|
|||
|
|
+@@ -71,7 +72,7 @@ public class TieredStoreExecutor {
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+ TimeUnit.MILLISECONDS,
|
|||
|
|
+ fetchDataThreadPoolQueue,
|
|||
|
|
+- new ThreadFactoryImpl("TieredFetchDataExecutor_"));
|
|||
|
|
++ new ThreadFactoryImpl("TieredFetchExecutor_"));
|
|||
|
|
+
|
|||
|
|
+ compactIndexFileThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+ compactIndexFileExecutor = new ThreadPoolExecutor(
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeFlatFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeFlatFile.java
|
|||
|
|
+index df4baf33f..5ad3a6ff3 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeFlatFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeFlatFile.java
|
|||
|
|
+@@ -76,20 +76,15 @@ public class CompositeFlatFile implements CompositeAccess {
|
|||
|
|
+ this.storeConfig = fileQueueFactory.getStoreConfig();
|
|||
|
|
+ this.readAheadFactor = this.storeConfig.getReadAheadMinFactor();
|
|||
|
|
+ this.metadataStore = TieredStoreUtil.getMetadataStore(this.storeConfig);
|
|||
|
|
+- this.dispatchOffset = new AtomicLong();
|
|||
|
|
+ this.compositeFlatFileLock = new ReentrantLock();
|
|||
|
|
+ this.inFlightRequestMap = new ConcurrentHashMap<>();
|
|||
|
|
+ this.commitLog = new TieredCommitLog(fileQueueFactory, filePath);
|
|||
|
|
+ this.consumeQueue = new TieredConsumeQueue(fileQueueFactory, filePath);
|
|||
|
|
++ this.dispatchOffset = new AtomicLong(
|
|||
|
|
++ this.consumeQueue.isInitialized() ? this.getConsumeQueueCommitOffset() : -1L);
|
|||
|
|
+ this.groupOffsetCache = this.initOffsetCache();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- protected void recoverMetadata() {
|
|||
|
|
+- if (!consumeQueue.isInitialized() && this.dispatchOffset.get() != -1) {
|
|||
|
|
+- consumeQueue.setBaseOffset(this.dispatchOffset.get() * TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE);
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ private Cache<String, Long> initOffsetCache() {
|
|||
|
|
+ return Caffeine.newBuilder()
|
|||
|
|
+ .expireAfterWrite(2, TimeUnit.MINUTES)
|
|||
|
|
+@@ -310,10 +305,12 @@ public class CompositeFlatFile implements CompositeAccess {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void initOffset(long offset) {
|
|||
|
|
+- if (!consumeQueue.isInitialized()) {
|
|||
|
|
++ if (consumeQueue.isInitialized()) {
|
|||
|
|
++ dispatchOffset.set(this.getConsumeQueueCommitOffset());
|
|||
|
|
++ } else {
|
|||
|
|
+ consumeQueue.setBaseOffset(offset * TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE);
|
|||
|
|
++ dispatchOffset.set(offset);
|
|||
|
|
+ }
|
|||
|
|
+- dispatchOffset.set(offset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFile.java
|
|||
|
|
+index f6c0afed0..0a797f465 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFile.java
|
|||
|
|
+@@ -36,8 +36,7 @@ public class CompositeQueueFlatFile extends CompositeFlatFile {
|
|||
|
|
+ public CompositeQueueFlatFile(TieredFileAllocator fileQueueFactory, MessageQueue messageQueue) {
|
|||
|
|
+ super(fileQueueFactory, TieredStoreUtil.toPath(messageQueue));
|
|||
|
|
+ this.messageQueue = messageQueue;
|
|||
|
|
+- this.recoverTopicMetadata();
|
|||
|
|
+- super.recoverMetadata();
|
|||
|
|
++ this.recoverQueueMetadata();
|
|||
|
|
+ this.indexFile = TieredFlatFileManager.getIndexFile(storeConfig);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -46,11 +45,12 @@ public class CompositeQueueFlatFile extends CompositeFlatFile {
|
|||
|
|
+ if (!consumeQueue.isInitialized()) {
|
|||
|
|
+ queueMetadata.setMinOffset(offset);
|
|||
|
|
+ queueMetadata.setMaxOffset(offset);
|
|||
|
|
++ metadataStore.updateQueue(queueMetadata);
|
|||
|
|
+ }
|
|||
|
|
+ super.initOffset(offset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void recoverTopicMetadata() {
|
|||
|
|
++ public void recoverQueueMetadata() {
|
|||
|
|
+ TopicMetadata topicMetadata = this.metadataStore.getTopic(messageQueue.getTopic());
|
|||
|
|
+ if (topicMetadata == null) {
|
|||
|
|
+ topicMetadata = this.metadataStore.addTopic(messageQueue.getTopic(), -1L);
|
|||
|
|
+@@ -64,18 +64,16 @@ public class CompositeQueueFlatFile extends CompositeFlatFile {
|
|||
|
|
+ if (queueMetadata.getMaxOffset() < queueMetadata.getMinOffset()) {
|
|||
|
|
+ queueMetadata.setMaxOffset(queueMetadata.getMinOffset());
|
|||
|
|
+ }
|
|||
|
|
+- this.dispatchOffset.set(queueMetadata.getMaxOffset());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void persistMetadata() {
|
|||
|
|
++ public void flushMetadata() {
|
|||
|
|
+ try {
|
|||
|
|
+- if (consumeQueue.getCommitOffset() < queueMetadata.getMinOffset()) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- queueMetadata.setMaxOffset(consumeQueue.getCommitOffset() / TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE);
|
|||
|
|
++ queueMetadata.setMinOffset(super.getConsumeQueueMinOffset());
|
|||
|
|
++ queueMetadata.setMaxOffset(super.getConsumeQueueMaxOffset());
|
|||
|
|
+ metadataStore.updateQueue(queueMetadata);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- LOGGER.error("CompositeFlatFile#flushMetadata: update queue metadata failed: topic: {}, queue: {}", messageQueue.getTopic(), messageQueue.getQueueId(), e);
|
|||
|
|
++ LOGGER.error("CompositeFlatFile#flushMetadata error, topic: {}, queue: {}",
|
|||
|
|
++ messageQueue.getTopic(), messageQueue.getQueueId(), e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -114,7 +112,7 @@ public class CompositeQueueFlatFile extends CompositeFlatFile {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
+ super.shutdown();
|
|||
|
|
+- metadataStore.updateQueue(queueMetadata);
|
|||
|
|
++ this.flushMetadata();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredCommitLog.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredCommitLog.java
|
|||
|
|
+index 80e1bce50..0e5f79132 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredCommitLog.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredCommitLog.java
|
|||
|
|
+@@ -50,7 +50,7 @@ public class TieredCommitLog {
|
|||
|
|
+ this.storeConfig = fileQueueFactory.getStoreConfig();
|
|||
|
|
+ this.flatFile = fileQueueFactory.createFlatFileForCommitLog(filePath);
|
|||
|
|
+ this.minConsumeQueueOffset = new AtomicLong(NOT_EXIST_MIN_OFFSET);
|
|||
|
|
+- this.correctMinOffset();
|
|||
|
|
++ this.correctMinOffsetAsync();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @VisibleForTesting
|
|||
|
|
+@@ -91,17 +91,26 @@ public class TieredCommitLog {
|
|||
|
|
+ return flatFile.getFileToWrite().getMaxTimestamp();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public synchronized long correctMinOffset() {
|
|||
|
|
++ public long correctMinOffset() {
|
|||
|
|
++ try {
|
|||
|
|
++ return correctMinOffsetAsync().get();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ log.error("Correct min offset failed in clean expired file", e);
|
|||
|
|
++ }
|
|||
|
|
++ return NOT_EXIST_MIN_OFFSET;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public synchronized CompletableFuture<Long> correctMinOffsetAsync() {
|
|||
|
|
+ if (flatFile.getFileSegmentCount() == 0) {
|
|||
|
|
+ this.minConsumeQueueOffset.set(NOT_EXIST_MIN_OFFSET);
|
|||
|
|
+- return NOT_EXIST_MIN_OFFSET;
|
|||
|
|
++ return CompletableFuture.completedFuture(NOT_EXIST_MIN_OFFSET);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // queue offset field length is 8
|
|||
|
|
+ int length = MessageBufferUtil.QUEUE_OFFSET_POSITION + 8;
|
|||
|
|
+ if (flatFile.getCommitOffset() - flatFile.getMinOffset() < length) {
|
|||
|
|
+ this.minConsumeQueueOffset.set(NOT_EXIST_MIN_OFFSET);
|
|||
|
|
+- return NOT_EXIST_MIN_OFFSET;
|
|||
|
|
++ return CompletableFuture.completedFuture(NOT_EXIST_MIN_OFFSET);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+@@ -109,7 +118,8 @@ public class TieredCommitLog {
|
|||
|
|
+ .thenApply(buffer -> {
|
|||
|
|
+ long offset = MessageBufferUtil.getQueueOffset(buffer);
|
|||
|
|
+ minConsumeQueueOffset.set(offset);
|
|||
|
|
+- log.info("Correct commitlog min cq offset success, filePath={}, min cq offset={}, range={}-{}",
|
|||
|
|
++ log.debug("Correct commitlog min cq offset success, " +
|
|||
|
|
++ "filePath={}, min cq offset={}, commitlog range={}-{}",
|
|||
|
|
+ flatFile.getFilePath(), offset, flatFile.getMinOffset(), flatFile.getCommitOffset());
|
|||
|
|
+ return offset;
|
|||
|
|
+ })
|
|||
|
|
+@@ -117,11 +127,11 @@ public class TieredCommitLog {
|
|||
|
|
+ log.warn("Correct commitlog min cq offset error, filePath={}, range={}-{}",
|
|||
|
|
+ flatFile.getFilePath(), flatFile.getMinOffset(), flatFile.getCommitOffset(), throwable);
|
|||
|
|
+ return minConsumeQueueOffset.get();
|
|||
|
|
+- }).get();
|
|||
|
|
++ });
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.error("Correct commitlog min cq offset error, filePath={}", flatFile.getFilePath(), e);
|
|||
|
|
+ }
|
|||
|
|
+- return minConsumeQueueOffset.get();
|
|||
|
|
++ return CompletableFuture.completedFuture(minConsumeQueueOffset.get());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public AppendResult append(ByteBuffer byteBuf) {
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+index 75ce8d89f..426c4e09d 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+@@ -16,6 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
++import com.alibaba.fastjson.JSON;
|
|||
|
|
+ import com.google.common.annotations.VisibleForTesting;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+@@ -24,6 +25,7 @@ import java.util.Comparator;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Objects;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
|
|
+@@ -178,32 +180,26 @@ public class TieredFlatFile {
|
|||
|
|
+ private FileSegmentMetadata getOrCreateFileSegmentMetadata(TieredFileSegment fileSegment) {
|
|||
|
|
+
|
|||
|
|
+ FileSegmentMetadata metadata = tieredMetadataStore.getFileSegment(
|
|||
|
|
+- fileSegment.getPath(), fileSegment.getFileType(), fileSegment.getBaseOffset());
|
|||
|
|
+-
|
|||
|
|
+- if (metadata != null) {
|
|||
|
|
+- return metadata;
|
|||
|
|
+- }
|
|||
|
|
++ this.filePath, fileSegment.getFileType(), fileSegment.getBaseOffset());
|
|||
|
|
+
|
|||
|
|
+ // Note: file segment path may not the same as file base path, use base path here.
|
|||
|
|
+- metadata = new FileSegmentMetadata(
|
|||
|
|
+- this.filePath, fileSegment.getBaseOffset(), fileSegment.getFileType().getType());
|
|||
|
|
+-
|
|||
|
|
+- if (fileSegment.isClosed()) {
|
|||
|
|
+- metadata.setStatus(FileSegmentMetadata.STATUS_DELETED);
|
|||
|
|
++ if (metadata == null) {
|
|||
|
|
++ metadata = new FileSegmentMetadata(
|
|||
|
|
++ this.filePath, fileSegment.getBaseOffset(), fileSegment.getFileType().getType());
|
|||
|
|
++ metadata.setCreateTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
++ metadata.setBeginTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
++ metadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
++ if (fileSegment.isClosed()) {
|
|||
|
|
++ metadata.setStatus(FileSegmentMetadata.STATUS_DELETED);
|
|||
|
|
++ }
|
|||
|
|
++ this.tieredMetadataStore.updateFileSegment(metadata);
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+- metadata.setBeginTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
+- metadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
+-
|
|||
|
|
+- // Submit to persist
|
|||
|
|
+- this.tieredMetadataStore.updateFileSegment(metadata);
|
|||
|
|
+ return metadata;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * FileQueue Status: Sealed | Sealed | Sealed | Not sealed, Allow appended && Not Full
|
|||
|
|
+ */
|
|||
|
|
+- @VisibleForTesting
|
|||
|
|
+ public void updateFileSegment(TieredFileSegment fileSegment) {
|
|||
|
|
+ FileSegmentMetadata segmentMetadata = getOrCreateFileSegmentMetadata(fileSegment);
|
|||
|
|
+
|
|||
|
|
+@@ -219,9 +215,16 @@ public class TieredFlatFile {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ segmentMetadata.setSize(fileSegment.getCommitPosition());
|
|||
|
|
+- segmentMetadata.setBeginTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
+ segmentMetadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
+- this.tieredMetadataStore.updateFileSegment(segmentMetadata);
|
|||
|
|
++
|
|||
|
|
++ FileSegmentMetadata metadata = tieredMetadataStore.getFileSegment(
|
|||
|
|
++ this.filePath, fileSegment.getFileType(), fileSegment.getBaseOffset());
|
|||
|
|
++
|
|||
|
|
++ if (!Objects.equals(metadata, segmentMetadata)) {
|
|||
|
|
++ this.tieredMetadataStore.updateFileSegment(segmentMetadata);
|
|||
|
|
++ logger.debug("TieredFlatFile#UpdateSegmentMetadata, filePath: {}, content: {}",
|
|||
|
|
++ segmentMetadata.getPath(), JSON.toJSONString(segmentMetadata));
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void checkAndFixFileSize() {
|
|||
|
|
+@@ -257,6 +260,7 @@ public class TieredFlatFile {
|
|||
|
|
+ logger.warn("TieredFlatFile#checkAndFixFileSize: fix last file {} size: origin: {}, actual: {}",
|
|||
|
|
+ lastFile.getPath(), lastFile.getCommitOffset() - lastFile.getBaseOffset(), lastFileSize);
|
|||
|
|
+ lastFile.initPosition(lastFileSize);
|
|||
|
|
++ this.updateFileSegment(lastFile);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+index aeca44b8c..e9ae4a5a5 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+@@ -16,16 +16,19 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
++import com.google.common.base.Stopwatch;
|
|||
|
|
+ import com.google.common.collect.ImmutableList;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Random;
|
|||
|
|
++import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+-import java.util.concurrent.Future;
|
|||
|
|
++import java.util.concurrent.Semaphore;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
+ import javax.annotation.Nullable;
|
|||
|
|
++import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+@@ -36,6 +39,7 @@ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+
|
|||
|
|
+ public class TieredFlatFileManager {
|
|||
|
|
+
|
|||
|
|
++ private static final Logger BROKER_LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
|
|||
|
|
+ private static final Logger logger = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ private static volatile TieredFlatFileManager instance;
|
|||
|
|
+@@ -44,7 +48,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ private final TieredMetadataStore metadataStore;
|
|||
|
|
+ private final TieredMessageStoreConfig storeConfig;
|
|||
|
|
+ private final TieredFileAllocator tieredFileAllocator;
|
|||
|
|
+- private final ConcurrentMap<MessageQueue, CompositeQueueFlatFile> queueFlatFileMap;
|
|||
|
|
++ private final ConcurrentMap<MessageQueue, CompositeQueueFlatFile> flatFileConcurrentMap;
|
|||
|
|
+
|
|||
|
|
+ public TieredFlatFileManager(TieredMessageStoreConfig storeConfig)
|
|||
|
|
+ throws ClassNotFoundException, NoSuchMethodException {
|
|||
|
|
+@@ -52,23 +56,20 @@ public class TieredFlatFileManager {
|
|||
|
|
+ this.storeConfig = storeConfig;
|
|||
|
|
+ this.metadataStore = TieredStoreUtil.getMetadataStore(storeConfig);
|
|||
|
|
+ this.tieredFileAllocator = new TieredFileAllocator(storeConfig);
|
|||
|
|
+- this.queueFlatFileMap = new ConcurrentHashMap<>();
|
|||
|
|
++ this.flatFileConcurrentMap = new ConcurrentHashMap<>();
|
|||
|
|
+ this.doScheduleTask();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static TieredFlatFileManager getInstance(TieredMessageStoreConfig storeConfig) {
|
|||
|
|
+- if (storeConfig == null) {
|
|||
|
|
++ if (storeConfig == null || instance != null) {
|
|||
|
|
+ return instance;
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+- if (instance == null) {
|
|||
|
|
+- synchronized (TieredFlatFileManager.class) {
|
|||
|
|
+- if (instance == null) {
|
|||
|
|
+- try {
|
|||
|
|
+- instance = new TieredFlatFileManager(storeConfig);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- logger.error("TieredFlatFileManager#getInstance: create flat file manager failed", e);
|
|||
|
|
+- }
|
|||
|
|
++ synchronized (TieredFlatFileManager.class) {
|
|||
|
|
++ if (instance == null) {
|
|||
|
|
++ try {
|
|||
|
|
++ instance = new TieredFlatFileManager(storeConfig);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ logger.error("Construct FlatFileManager instance error", e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -88,7 +89,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ TieredStoreUtil.RMQ_SYS_TIERED_STORE_INDEX_TOPIC, storeConfig.getBrokerName(), 0));
|
|||
|
|
+ indexFile = new TieredIndexFile(new TieredFileAllocator(storeConfig), filePath);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- logger.error("TieredFlatFileManager#getIndexFile: create index file failed", e);
|
|||
|
|
++ logger.error("Construct FlatFileManager indexFile error", e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -105,7 +106,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ flatFile.commitCommitLog();
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ MessageQueue mq = flatFile.getMessageQueue();
|
|||
|
|
+- logger.error("commit commitLog periodically failed: topic: {}, queue: {}",
|
|||
|
|
++ logger.error("Commit commitLog periodically failed: topic: {}, queue: {}",
|
|||
|
|
+ mq.getTopic(), mq.getQueueId(), e);
|
|||
|
|
+ }
|
|||
|
|
+ }, delay, TimeUnit.MILLISECONDS);
|
|||
|
|
+@@ -114,7 +115,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ flatFile.commitConsumeQueue();
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ MessageQueue mq = flatFile.getMessageQueue();
|
|||
|
|
+- logger.error("commit consumeQueue periodically failed: topic: {}, queue: {}",
|
|||
|
|
++ logger.error("Commit consumeQueue periodically failed: topic: {}, queue: {}",
|
|||
|
|
+ mq.getTopic(), mq.getQueueId(), e);
|
|||
|
|
+ }
|
|||
|
|
+ }, delay, TimeUnit.MILLISECONDS);
|
|||
|
|
+@@ -125,7 +126,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ indexFile.commit(true);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+- logger.error("commit indexFile periodically failed", e);
|
|||
|
|
++ logger.error("Commit indexFile periodically failed", e);
|
|||
|
|
+ }
|
|||
|
|
+ }, 0, TimeUnit.MILLISECONDS);
|
|||
|
|
+ }
|
|||
|
|
+@@ -160,7 +161,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ try {
|
|||
|
|
+ doCommit();
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+- logger.error("commit flat file periodically failed: ", e);
|
|||
|
|
++ logger.error("Commit flat file periodically failed: ", e);
|
|||
|
|
+ }
|
|||
|
|
+ }, 60, 60, TimeUnit.SECONDS);
|
|||
|
|
+
|
|||
|
|
+@@ -168,49 +169,73 @@ public class TieredFlatFileManager {
|
|||
|
|
+ try {
|
|||
|
|
+ doCleanExpiredFile();
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+- logger.error("clean expired flat file failed: ", e);
|
|||
|
|
++ logger.error("Clean expired flat file failed: ", e);
|
|||
|
|
+ }
|
|||
|
|
+ }, 30, 30, TimeUnit.SECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean load() {
|
|||
|
|
++ Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
+ try {
|
|||
|
|
+- AtomicLong topicSequenceNumber = new AtomicLong();
|
|||
|
|
+- List<Future<?>> futureList = new ArrayList<>();
|
|||
|
|
+- queueFlatFileMap.clear();
|
|||
|
|
+- metadataStore.iterateTopic(topicMetadata -> {
|
|||
|
|
++ flatFileConcurrentMap.clear();
|
|||
|
|
++ this.recoverSequenceNumber();
|
|||
|
|
++ this.recoverTieredFlatFile();
|
|||
|
|
++ logger.info("Message store recover end, total cost={}ms", stopwatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ long costTime = stopwatch.elapsed(TimeUnit.MILLISECONDS);
|
|||
|
|
++ logger.info("Message store recover error, total cost={}ms", costTime);
|
|||
|
|
++ BROKER_LOG.error("Message store recover error, total cost={}ms", costTime, e);
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void recoverSequenceNumber() {
|
|||
|
|
++ AtomicLong topicSequenceNumber = new AtomicLong();
|
|||
|
|
++ metadataStore.iterateTopic(topicMetadata -> {
|
|||
|
|
++ if (topicMetadata != null && topicMetadata.getTopicId() > 0) {
|
|||
|
|
+ topicSequenceNumber.set(Math.max(topicSequenceNumber.get(), topicMetadata.getTopicId()));
|
|||
|
|
+- Future<?> future = TieredStoreExecutor.dispatchExecutor.submit(() -> {
|
|||
|
|
+- if (topicMetadata.getStatus() != 0) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
++ metadataStore.setTopicSequenceNumber(topicSequenceNumber.incrementAndGet());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void recoverTieredFlatFile() {
|
|||
|
|
++ Semaphore semaphore = new Semaphore((int) (TieredStoreExecutor.QUEUE_CAPACITY * 0.75));
|
|||
|
|
++ List<CompletableFuture<Void>> futures = new ArrayList<>();
|
|||
|
|
++ metadataStore.iterateTopic(topicMetadata -> {
|
|||
|
|
++ try {
|
|||
|
|
++ semaphore.acquire();
|
|||
|
|
++ CompletableFuture<Void> future = CompletableFuture.runAsync(() -> {
|
|||
|
|
+ try {
|
|||
|
|
+- metadataStore.iterateQueue(topicMetadata.getTopic(),
|
|||
|
|
+- queueMetadata -> getOrCreateFlatFileIfAbsent(
|
|||
|
|
+- new MessageQueue(topicMetadata.getTopic(),
|
|||
|
|
+- storeConfig.getBrokerName(),
|
|||
|
|
+- queueMetadata.getQueue().getQueueId())));
|
|||
|
|
++ Stopwatch subWatch = Stopwatch.createStarted();
|
|||
|
|
++ if (topicMetadata.getStatus() != 0) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ AtomicLong queueCount = new AtomicLong();
|
|||
|
|
++ metadataStore.iterateQueue(topicMetadata.getTopic(), queueMetadata -> {
|
|||
|
|
++ this.getOrCreateFlatFileIfAbsent(new MessageQueue(topicMetadata.getTopic(),
|
|||
|
|
++ storeConfig.getBrokerName(), queueMetadata.getQueue().getQueueId()));
|
|||
|
|
++ queueCount.incrementAndGet();
|
|||
|
|
++ });
|
|||
|
|
++ logger.info("Recover TopicFlatFile, topic: {}, queueCount: {}, cost: {}ms",
|
|||
|
|
++ topicMetadata.getTopic(), queueCount.get(), subWatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- logger.error("load mq composite flat file from metadata failed", e);
|
|||
|
|
++ logger.error("Recover TopicFlatFile error, topic: {}", topicMetadata.getTopic(), e);
|
|||
|
|
++ } finally {
|
|||
|
|
++ semaphore.release();
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- futureList.add(future);
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- // Wait for load all metadata done
|
|||
|
|
+- for (Future<?> future : futureList) {
|
|||
|
|
+- future.get();
|
|||
|
|
++ }, TieredStoreExecutor.commitExecutor);
|
|||
|
|
++ futures.add(future);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ throw new RuntimeException(e);
|
|||
|
|
+ }
|
|||
|
|
+- metadataStore.setTopicSequenceNumber(topicSequenceNumber.incrementAndGet());
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- logger.error("load mq composite flat file from metadata failed", e);
|
|||
|
|
+- return false;
|
|||
|
|
+- }
|
|||
|
|
+- return true;
|
|||
|
|
++ });
|
|||
|
|
++ CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).join();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void cleanup() {
|
|||
|
|
+- queueFlatFileMap.clear();
|
|||
|
|
++ flatFileConcurrentMap.clear();
|
|||
|
|
+ cleanStaticReference();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -221,27 +246,25 @@ public class TieredFlatFileManager {
|
|||
|
|
+
|
|||
|
|
+ @Nullable
|
|||
|
|
+ public CompositeQueueFlatFile getOrCreateFlatFileIfAbsent(MessageQueue messageQueue) {
|
|||
|
|
+- return queueFlatFileMap.computeIfAbsent(messageQueue, mq -> {
|
|||
|
|
++ return flatFileConcurrentMap.computeIfAbsent(messageQueue, mq -> {
|
|||
|
|
+ try {
|
|||
|
|
+- logger.debug("TieredFlatFileManager#getOrCreateFlatFileIfAbsent: " +
|
|||
|
|
+- "try to create new flat file: topic: {}, queueId: {}",
|
|||
|
|
++ logger.debug("Create new TopicFlatFile, topic: {}, queueId: {}",
|
|||
|
|
+ messageQueue.getTopic(), messageQueue.getQueueId());
|
|||
|
|
+ return new CompositeQueueFlatFile(tieredFileAllocator, mq);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- logger.error("TieredFlatFileManager#getOrCreateFlatFileIfAbsent: " +
|
|||
|
|
+- "create new flat file: topic: {}, queueId: {}",
|
|||
|
|
++ logger.debug("Create new TopicFlatFile failed, topic: {}, queueId: {}",
|
|||
|
|
+ messageQueue.getTopic(), messageQueue.getQueueId(), e);
|
|||
|
|
+- return null;
|
|||
|
|
+ }
|
|||
|
|
++ return null;
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompositeQueueFlatFile getFlatFile(MessageQueue messageQueue) {
|
|||
|
|
+- return queueFlatFileMap.get(messageQueue);
|
|||
|
|
++ return flatFileConcurrentMap.get(messageQueue);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ImmutableList<CompositeQueueFlatFile> deepCopyFlatFileToList() {
|
|||
|
|
+- return ImmutableList.copyOf(queueFlatFileMap.values());
|
|||
|
|
++ return ImmutableList.copyOf(flatFileConcurrentMap.values());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
+@@ -270,7 +293,7 @@ public class TieredFlatFileManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // delete memory reference
|
|||
|
|
+- CompositeQueueFlatFile flatFile = queueFlatFileMap.remove(mq);
|
|||
|
|
++ CompositeQueueFlatFile flatFile = flatFileConcurrentMap.remove(mq);
|
|||
|
|
+ if (flatFile != null) {
|
|||
|
|
+ MessageQueue messageQueue = flatFile.getMessageQueue();
|
|||
|
|
+ logger.info("TieredFlatFileManager#destroyCompositeFile: " +
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/FileSegmentMetadata.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/FileSegmentMetadata.java
|
|||
|
|
+index 1b232fc75..2f0fd71de 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/FileSegmentMetadata.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/FileSegmentMetadata.java
|
|||
|
|
+@@ -16,6 +16,8 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.metadata;
|
|||
|
|
+
|
|||
|
|
++import java.util.Objects;
|
|||
|
|
++
|
|||
|
|
+ public class FileSegmentMetadata {
|
|||
|
|
+
|
|||
|
|
+ public static final int STATUS_NEW = 0;
|
|||
|
|
+@@ -43,7 +45,6 @@ public class FileSegmentMetadata {
|
|||
|
|
+ this.baseOffset = baseOffset;
|
|||
|
|
+ this.type = type;
|
|||
|
|
+ this.status = STATUS_NEW;
|
|||
|
|
+- this.createTimestamp = System.currentTimeMillis();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void markSealed() {
|
|||
|
|
+@@ -122,4 +123,27 @@ public class FileSegmentMetadata {
|
|||
|
|
+ public void setSealTimestamp(long sealTimestamp) {
|
|||
|
|
+ this.sealTimestamp = sealTimestamp;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean equals(Object o) {
|
|||
|
|
++ if (this == o)
|
|||
|
|
++ return true;
|
|||
|
|
++ if (o == null || getClass() != o.getClass())
|
|||
|
|
++ return false;
|
|||
|
|
++ FileSegmentMetadata metadata = (FileSegmentMetadata) o;
|
|||
|
|
++ return size == metadata.size
|
|||
|
|
++ && baseOffset == metadata.baseOffset
|
|||
|
|
++ && status == metadata.status
|
|||
|
|
++ && path.equals(metadata.path)
|
|||
|
|
++ && type == metadata.type
|
|||
|
|
++ && createTimestamp == metadata.createTimestamp
|
|||
|
|
++ && beginTimestamp == metadata.beginTimestamp
|
|||
|
|
++ && endTimestamp == metadata.endTimestamp
|
|||
|
|
++ && sealTimestamp == metadata.sealTimestamp;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public int hashCode() {
|
|||
|
|
++ return Objects.hash(type, path, baseOffset, status, size, createTimestamp, beginTimestamp, endTimestamp, sealTimestamp);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredDispatcherTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredDispatcherTest.java
|
|||
|
|
+index e6adef1d1..5791dc9a4 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredDispatcherTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredDispatcherTest.java
|
|||
|
|
+@@ -116,19 +116,20 @@ public class TieredDispatcherTest {
|
|||
|
|
+ buffer3.putLong(MessageBufferUtil.QUEUE_OFFSET_POSITION, 9);
|
|||
|
|
+ flatFile.appendCommitLog(buffer3);
|
|||
|
|
+ flatFile.commitCommitLog();
|
|||
|
|
+- Assert.assertEquals(10, flatFile.getDispatchOffset());
|
|||
|
|
++ Assert.assertEquals(9 + 1, flatFile.getDispatchOffset());
|
|||
|
|
++ Assert.assertEquals(9, flatFile.getCommitLogDispatchCommitOffset());
|
|||
|
|
+
|
|||
|
|
+ dispatcher.doRedispatchRequestToWriteMap(AppendResult.SUCCESS, flatFile, 8, 8, 0, 0, buffer1);
|
|||
|
|
+ dispatcher.doRedispatchRequestToWriteMap(AppendResult.SUCCESS, flatFile, 9, 9, 0, 0, buffer2);
|
|||
|
|
+ dispatcher.buildConsumeQueueAndIndexFile();
|
|||
|
|
+ Assert.assertEquals(7, flatFile.getConsumeQueueMaxOffset());
|
|||
|
|
+- Assert.assertEquals(7, flatFile.getDispatchOffset());
|
|||
|
|
+
|
|||
|
|
+ dispatcher.doRedispatchRequestToWriteMap(AppendResult.SUCCESS, flatFile, 7, 7, 0, 0, buffer1);
|
|||
|
|
+ dispatcher.doRedispatchRequestToWriteMap(AppendResult.SUCCESS, flatFile, 8, 8, 0, 0, buffer2);
|
|||
|
|
+ dispatcher.doRedispatchRequestToWriteMap(AppendResult.SUCCESS, flatFile, 9, 9, 0, 0, buffer3);
|
|||
|
|
+ dispatcher.buildConsumeQueueAndIndexFile();
|
|||
|
|
+- Assert.assertEquals(10, flatFile.getConsumeQueueMaxOffset());
|
|||
|
|
++ Assert.assertEquals(6, flatFile.getConsumeQueueMinOffset());
|
|||
|
|
++ Assert.assertEquals(9 + 1, flatFile.getConsumeQueueMaxOffset());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+@@ -142,6 +143,7 @@ public class TieredDispatcherTest {
|
|||
|
|
+ Mockito.when(defaultStore.getMinOffsetInQueue(mq.getTopic(), mq.getQueueId())).thenReturn(0L);
|
|||
|
|
+ Mockito.when(defaultStore.getMaxOffsetInQueue(mq.getTopic(), mq.getQueueId())).thenReturn(9L);
|
|||
|
|
+
|
|||
|
|
++ // mock cq item, position = 7
|
|||
|
|
+ ByteBuffer cqItem = ByteBuffer.allocate(ConsumeQueue.CQ_STORE_UNIT_SIZE);
|
|||
|
|
+ cqItem.putLong(7);
|
|||
|
|
+ cqItem.putInt(MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
+@@ -150,13 +152,13 @@ public class TieredDispatcherTest {
|
|||
|
|
+ SelectMappedBufferResult mockResult = new SelectMappedBufferResult(0, cqItem, ConsumeQueue.CQ_STORE_UNIT_SIZE, null);
|
|||
|
|
+ Mockito.when(((ConsumeQueue) defaultStore.getConsumeQueue(mq.getTopic(), mq.getQueueId())).getIndexBuffer(6)).thenReturn(mockResult);
|
|||
|
|
+
|
|||
|
|
++ // mock cq item, position = 8
|
|||
|
|
+ cqItem = ByteBuffer.allocate(ConsumeQueue.CQ_STORE_UNIT_SIZE);
|
|||
|
|
+ cqItem.putLong(8);
|
|||
|
|
+ cqItem.putInt(MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
+ cqItem.putLong(1);
|
|||
|
|
+ cqItem.flip();
|
|||
|
|
+ mockResult = new SelectMappedBufferResult(0, cqItem, ConsumeQueue.CQ_STORE_UNIT_SIZE, null);
|
|||
|
|
+-
|
|||
|
|
+ Mockito.when(((ConsumeQueue) defaultStore.getConsumeQueue(mq.getTopic(), mq.getQueueId())).getIndexBuffer(7)).thenReturn(mockResult);
|
|||
|
|
+
|
|||
|
|
+ mockResult = new SelectMappedBufferResult(0, MessageBufferUtilTest.buildMockedMessageBuffer(), MessageBufferUtilTest.MSG_LEN, null);
|
|||
|
|
+@@ -167,7 +169,10 @@ public class TieredDispatcherTest {
|
|||
|
|
+ mockResult = new SelectMappedBufferResult(0, msg, MessageBufferUtilTest.MSG_LEN, null);
|
|||
|
|
+ Mockito.when(defaultStore.selectOneMessageByOffset(8, MessageBufferUtilTest.MSG_LEN)).thenReturn(mockResult);
|
|||
|
|
+
|
|||
|
|
+- dispatcher.dispatchFlatFile(flatFileManager.getOrCreateFlatFileIfAbsent(mq));
|
|||
|
|
++ CompositeQueueFlatFile flatFile = flatFileManager.getOrCreateFlatFileIfAbsent(mq);
|
|||
|
|
++ Assert.assertNotNull(flatFile);
|
|||
|
|
++ flatFile.initOffset(7);
|
|||
|
|
++ dispatcher.dispatchFlatFile(flatFile);
|
|||
|
|
+ Assert.assertEquals(8, flatFileManager.getFlatFile(mq).getDispatchOffset());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageFetcherTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageFetcherTest.java
|
|||
|
|
+index d75b2f916..774c6cf64 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageFetcherTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageFetcherTest.java
|
|||
|
|
+@@ -23,6 +23,7 @@ import java.util.Objects;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.lang3.SystemUtils;
|
|||
|
|
+ import org.apache.commons.lang3.tuple.Triple;
|
|||
|
|
++import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+ import org.apache.rocketmq.store.GetMessageResult;
|
|||
|
|
+@@ -40,7 +41,6 @@ import org.apache.rocketmq.tieredstore.file.TieredIndexFile;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtilTest;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+-import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.awaitility.Awaitility;
|
|||
|
|
+ import org.junit.After;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFileTest.java
|
|||
|
|
+index 27efe111e..2e028ada3 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/CompositeQueueFlatFileTest.java
|
|||
|
|
+@@ -119,7 +119,7 @@ public class CompositeQueueFlatFileTest {
|
|||
|
|
+ Assert.assertEquals(AppendResult.SUCCESS, result);
|
|||
|
|
+
|
|||
|
|
+ file.commit(true);
|
|||
|
|
+- file.persistMetadata();
|
|||
|
|
++ file.flushMetadata();
|
|||
|
|
+
|
|||
|
|
+ QueueMetadata queueMetadata = metadataStore.getQueue(mq);
|
|||
|
|
+ Assert.assertEquals(53, queueMetadata.getMaxOffset());
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManagerTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManagerTest.java
|
|||
|
|
+index b7528c5e4..20fe4dd70 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManagerTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManagerTest.java
|
|||
|
|
+@@ -72,10 +72,15 @@ public class TieredFlatFileManagerTest {
|
|||
|
|
+
|
|||
|
|
+ CompositeFlatFile flatFile = flatFileManager.getFlatFile(mq);
|
|||
|
|
+ Assert.assertNotNull(flatFile);
|
|||
|
|
+- Assert.assertEquals(100, flatFile.getDispatchOffset());
|
|||
|
|
++ Assert.assertEquals(-1L, flatFile.getDispatchOffset());
|
|||
|
|
++ flatFile.initOffset(100L);
|
|||
|
|
++ Assert.assertEquals(100L, flatFile.getDispatchOffset());
|
|||
|
|
++ flatFile.initOffset(200L);
|
|||
|
|
++ Assert.assertEquals(100L, flatFile.getDispatchOffset());
|
|||
|
|
+
|
|||
|
|
+ CompositeFlatFile flatFile1 = flatFileManager.getFlatFile(mq1);
|
|||
|
|
+ Assert.assertNotNull(flatFile1);
|
|||
|
|
++ flatFile1.initOffset(200L);
|
|||
|
|
+ Assert.assertEquals(200, flatFile1.getDispatchOffset());
|
|||
|
|
+
|
|||
|
|
+ flatFileManager.destroyCompositeFile(mq);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 99b39a35f29e491862296d56b7938a995d153974 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: ShuangxiDing <dingshuangxi888@gmail.com>
|
|||
|
|
+Date: Thu, 10 Aug 2023 11:28:39 +0800
|
|||
|
|
+Subject: [PATCH 02/12] [ISSUE #7115] Fix grpc response message NPE (#7116)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java
|
|||
|
|
+index 0b3c85ea6..efa879a9c 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/common/ResponseBuilder.java
|
|||
|
|
+@@ -92,7 +92,7 @@ public class ResponseBuilder {
|
|||
|
|
+ public Status buildStatus(Code code, String message) {
|
|||
|
|
+ return Status.newBuilder()
|
|||
|
|
+ .setCode(code)
|
|||
|
|
+- .setMessage(message)
|
|||
|
|
++ .setMessage(message != null ? message : code.name())
|
|||
|
|
+ .build();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From c0ba453f38183266cf9a69be754e620311e1923b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: caigy <csgytsai@163.com>
|
|||
|
|
+Date: Thu, 10 Aug 2023 14:08:17 +0800
|
|||
|
|
+Subject: [PATCH 03/12] [ISSUE #7129] Fix resource collisions in acl tests
|
|||
|
|
+ (#7130)
|
|||
|
|
+
|
|||
|
|
+* run acl tests sequencially to avoid collision
|
|||
|
|
+
|
|||
|
|
+* disable reuseForks for acl like broker
|
|||
|
|
+
|
|||
|
|
+* Revert "[ISSUE #7135] Temporarily ignoring plainAccessValidator test (#7135)"
|
|||
|
|
+
|
|||
|
|
+This reverts commit 6bc2c8474a0ce1e2833c82dffea7b1d8f718fcd7.
|
|||
|
|
+---
|
|||
|
|
+ acl/pom.xml | 13 +++++++++++++
|
|||
|
|
+ .../acl/plain/PlainAccessControlFlowTest.java | 5 -----
|
|||
|
|
+ .../acl/plain/PlainAccessValidatorTest.java | 3 ---
|
|||
|
|
+ .../acl/plain/PlainPermissionManagerTest.java | 3 ---
|
|||
|
|
+ 4 files changed, 13 insertions(+), 11 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/acl/pom.xml b/acl/pom.xml
|
|||
|
|
+index 67bfcb8d2..989c0cf77 100644
|
|||
|
|
+--- a/acl/pom.xml
|
|||
|
|
++++ b/acl/pom.xml
|
|||
|
|
+@@ -74,4 +74,17 @@
|
|||
|
|
+ <scope>test</scope>
|
|||
|
|
+ </dependency>
|
|||
|
|
+ </dependencies>
|
|||
|
|
++
|
|||
|
|
++ <build>
|
|||
|
|
++ <plugins>
|
|||
|
|
++ <plugin>
|
|||
|
|
++ <artifactId>maven-surefire-plugin</artifactId>
|
|||
|
|
++ <version>${maven-surefire-plugin.version}</version>
|
|||
|
|
++ <configuration>
|
|||
|
|
++ <forkCount>1</forkCount>
|
|||
|
|
++ <reuseForks>false</reuseForks>
|
|||
|
|
++ </configuration>
|
|||
|
|
++ </plugin>
|
|||
|
|
++ </plugins>
|
|||
|
|
++ </build>
|
|||
|
|
+ </project>
|
|||
|
|
+diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java
|
|||
|
|
+index e7fd0932f..519345714 100644
|
|||
|
|
+--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java
|
|||
|
|
++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessControlFlowTest.java
|
|||
|
|
+@@ -31,7 +31,6 @@ import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeaderV2;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+-import org.junit.Ignore;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+@@ -44,7 +43,6 @@ import java.util.Collections;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ /**
|
|||
|
|
+ * <p> In this class, we'll test the following scenarios, each containing several consecutive operations on ACL,
|
|||
|
|
+ * <p> like updating and deleting ACL, changing config files and checking validations.
|
|||
|
|
+@@ -52,9 +50,6 @@ import java.util.List;
|
|||
|
|
+ * <p> Case 2: Only conf/acl/plain_acl.yml exists;
|
|||
|
|
+ * <p> Case 3: Both conf/plain_acl.yml and conf/acl/plain_acl.yml exists.
|
|||
|
|
+ */
|
|||
|
|
+-
|
|||
|
|
+-// Ignore this test case as it is currently unable to pass on ubuntu workflow
|
|||
|
|
+-@Ignore
|
|||
|
|
+ public class PlainAccessControlFlowTest {
|
|||
|
|
+ public static final String DEFAULT_TOPIC = "topic-acl";
|
|||
|
|
+
|
|||
|
|
+diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java
|
|||
|
|
+index a3a925758..ef0cffbdc 100644
|
|||
|
|
+--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java
|
|||
|
|
++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainAccessValidatorTest.java
|
|||
|
|
+@@ -56,11 +56,8 @@ import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData;
|
|||
|
|
+ import org.junit.After;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+-import org.junit.Ignore;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+-// Ignore this test case as it is currently unable to pass on ubuntu workflow
|
|||
|
|
+-@Ignore
|
|||
|
|
+ public class PlainAccessValidatorTest {
|
|||
|
|
+
|
|||
|
|
+ private PlainAccessValidator plainAccessValidator;
|
|||
|
|
+diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java
|
|||
|
|
+index aa7539f3a..941d8c779 100644
|
|||
|
|
+--- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java
|
|||
|
|
++++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionManagerTest.java
|
|||
|
|
+@@ -29,7 +29,6 @@ import org.assertj.core.api.Assertions;
|
|||
|
|
+ import org.assertj.core.util.Lists;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+-import org.junit.Ignore;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+@@ -42,8 +41,6 @@ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+
|
|||
|
|
+-// Ignore this test case as it is currently unable to pass on ubuntu workflow
|
|||
|
|
+-@Ignore
|
|||
|
|
+ public class PlainPermissionManagerTest {
|
|||
|
|
+
|
|||
|
|
+ PlainPermissionManager plainPermissionManager;
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 8741ff8c9b3bdbfc97976285affa7ea35c81243c Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: ShuangxiDing <dingshuangxi888@gmail.com>
|
|||
|
|
+Date: Thu, 10 Aug 2023 17:41:15 +0800
|
|||
|
|
+Subject: [PATCH 04/12] [ISSUE #7153] Add switch for MIXED message type (#7154)
|
|||
|
|
+
|
|||
|
|
+Add a switch for MIXED message type when creating a Topic in the Broker.
|
|||
|
|
+---
|
|||
|
|
+ .../broker/processor/AdminBrokerProcessor.java | 8 ++++++++
|
|||
|
|
+ .../java/org/apache/rocketmq/common/BrokerConfig.java | 10 ++++++++++
|
|||
|
|
+ 2 files changed, 18 insertions(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+index a6ce03dc2..bbddcec2d 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+@@ -59,6 +59,7 @@ import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.UnlockCallback;
|
|||
|
|
+ import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+ import org.apache.rocketmq.common.attribute.AttributeParser;
|
|||
|
|
++import org.apache.rocketmq.common.attribute.TopicMessageType;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.ConsumeInitMode;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.FIleReadaheadMode;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+@@ -439,6 +440,13 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ String attributesModification = requestHeader.getAttributes();
|
|||
|
|
+ topicConfig.setAttributes(AttributeParser.parseToMap(attributesModification));
|
|||
|
|
+
|
|||
|
|
++ if (topicConfig.getTopicMessageType() == TopicMessageType.MIXED
|
|||
|
|
++ && !brokerController.getBrokerConfig().isEnableMixedMessageType()) {
|
|||
|
|
++ response.setCode(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
++ response.setRemark("MIXED message type is not supported.");
|
|||
|
|
++ return response;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ try {
|
|||
|
|
+ this.brokerController.getTopicConfigManager().updateTopicConfig(topicConfig);
|
|||
|
|
+ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+index a815636b1..99a5db5ad 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+@@ -393,6 +393,8 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+ */
|
|||
|
|
+ private boolean enableSingleTopicRegister = false;
|
|||
|
|
+
|
|||
|
|
++ private boolean enableMixedMessageType = false;
|
|||
|
|
++
|
|||
|
|
+ public long getMaxPopPollingSize() {
|
|||
|
|
+ return maxPopPollingSize;
|
|||
|
|
+ }
|
|||
|
|
+@@ -1712,4 +1714,12 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+ public void setEnableSingleTopicRegister(boolean enableSingleTopicRegister) {
|
|||
|
|
+ this.enableSingleTopicRegister = enableSingleTopicRegister;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableMixedMessageType() {
|
|||
|
|
++ return enableMixedMessageType;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableMixedMessageType(boolean enableMixedMessageType) {
|
|||
|
|
++ this.enableMixedMessageType = enableMixedMessageType;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From f534501855f8edbcb58f5b856973bf1027b5cf3a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Steven <shirenchuang@users.noreply.github.com>
|
|||
|
|
+Date: Fri, 11 Aug 2023 10:25:48 +0800
|
|||
|
|
+Subject: [PATCH 05/12] [Feature 7155] add errlog when cmd err (#7157)
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 十真 <shirenchuang.src@cainiao.com>
|
|||
|
|
+---
|
|||
|
|
+ .../src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java | 1 +
|
|||
|
|
+ 1 file changed, 1 insertion(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java
|
|||
|
|
+index b00bad3c5..5a8a7cd54 100644
|
|||
|
|
+--- a/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java
|
|||
|
|
++++ b/srvutil/src/main/java/org/apache/rocketmq/srvutil/ServerUtil.java
|
|||
|
|
+@@ -52,6 +52,7 @@ public class ServerUtil {
|
|||
|
|
+ System.exit(0);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (ParseException e) {
|
|||
|
|
++ System.err.println(e.getMessage());
|
|||
|
|
+ hf.printHelp(appName, options, true);
|
|||
|
|
+ System.exit(1);
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From db58f00c0fe0f129611d654291f2177de55dc9ff Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Fri, 11 Aug 2023 19:18:30 +0800
|
|||
|
|
+Subject: [PATCH 06/12] [ISSUE #7169] Change metadataThreadPoolQueueCapacity to
|
|||
|
|
+ 100000 (#7170)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index 4f57a7052..39caaa0d9 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -165,7 +165,7 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ private int subscriptionGroupConfigCacheExpiredInSeconds = 20;
|
|||
|
|
+ private int subscriptionGroupConfigCacheMaxNum = 20000;
|
|||
|
|
+ private int metadataThreadPoolNums = 3;
|
|||
|
|
+- private int metadataThreadPoolQueueCapacity = 1000;
|
|||
|
|
++ private int metadataThreadPoolQueueCapacity = 100000;
|
|||
|
|
+
|
|||
|
|
+ private int transactionHeartbeatThreadPoolNums = 20;
|
|||
|
|
+ private int transactionHeartbeatThreadPoolQueueCapacity = 200;
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 1f04e68a2e331ab035b791280c5a91b60fe0c85f Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yx9o <yangx_soft@163.com>
|
|||
|
|
+Date: Sat, 12 Aug 2023 21:12:22 +0800
|
|||
|
|
+Subject: [PATCH 07/12] [ISSUE #7172] Unified Chinese for Name Server (#7173)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ docs/cn/concept.md | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/docs/cn/concept.md b/docs/cn/concept.md
|
|||
|
|
+index cb2c863bd..3d67e9371 100644
|
|||
|
|
+--- a/docs/cn/concept.md
|
|||
|
|
++++ b/docs/cn/concept.md
|
|||
|
|
+@@ -17,7 +17,7 @@ RocketMQ主要由 Producer、Broker、Consumer 三部分组成,其中Producer
|
|||
|
|
+ 消息中转角色,负责存储消息、转发消息。代理服务器在RocketMQ系统中负责接收从生产者发送来的消息并存储、同时为消费者的拉取请求作准备。代理服务器也存储消息相关的元数据,包括消费者组、消费进度偏移和主题和队列消息等。
|
|||
|
|
+
|
|||
|
|
+ ## 6 名字服务(Name Server)
|
|||
|
|
+- 名称服务充当路由消息的提供者。生产者或消费者能够通过名字服务查找各主题相应的Broker IP列表。多个Namesrv实例组成集群,但相互独立,没有信息交换。
|
|||
|
|
++名字服务充当路由消息的提供者。生产者或消费者能够通过名字服务查找各主题相应的Broker IP列表。多个Namesrv实例组成集群,但相互独立,没有信息交换。
|
|||
|
|
+
|
|||
|
|
+ ## 7 拉取式消费(Pull Consumer)
|
|||
|
|
+ Consumer消费的一种类型,应用通常主动调用Consumer的拉消息方法从Broker服务器拉消息、主动权由应用控制。一旦获取了批量消息,应用就会启动消费过程。
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 25005060bbace477eeaaf4c0142cece5213efbbf Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yx9o <yangx_soft@163.com>
|
|||
|
|
+Date: Sun, 13 Aug 2023 20:52:17 +0800
|
|||
|
|
+Subject: [PATCH 08/12] [ISSUE #7176] Correct mismatched logs (#7177)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
|
|||
|
|
+index 0055a1cc8..f7a95f0a6 100644
|
|||
|
|
+--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
|
|||
|
|
++++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/routeinfo/RouteInfoManager.java
|
|||
|
|
+@@ -522,7 +522,7 @@ public class RouteInfoManager {
|
|||
|
|
+ this.lock.writeLock().unlock();
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- log.error("wipeWritePermOfBrokerByLock Exception", e);
|
|||
|
|
++ log.error("addWritePermOfBrokerByLock Exception", e);
|
|||
|
|
+ }
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From ac411daa27117e9115a8fc5e2d5753085f009ed9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yx9o <yangx_soft@163.com>
|
|||
|
|
+Date: Tue, 15 Aug 2023 08:31:00 +0800
|
|||
|
|
+Subject: [PATCH 09/12] [ISSUE #7183] Correct mismatched commandDesc (#7184)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tools/command/topic/RemappingStaticTopicSubCommand.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/RemappingStaticTopicSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/RemappingStaticTopicSubCommand.java
|
|||
|
|
+index 849f680d0..2a08fdb5b 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/RemappingStaticTopicSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/RemappingStaticTopicSubCommand.java
|
|||
|
|
+@@ -47,7 +47,7 @@ public class RemappingStaticTopicSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update or create static topic, which has fixed number of queues";
|
|||
|
|
++ return "Remapping static topic.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 55e0cdb2af3ab75a6d892f919d60797f17a99fda Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: redlsz <szliu0927@gmail.com>
|
|||
|
|
+Date: Tue, 15 Aug 2023 19:19:45 +0800
|
|||
|
|
+Subject: [PATCH 10/12] fix: IndexOutOfBoundsException when process pop
|
|||
|
|
+ response (#7003)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../org/apache/rocketmq/client/impl/MQClientAPIImpl.java | 5 ++++-
|
|||
|
|
+ .../rocketmq/proxy/service/message/LocalMessageService.java | 5 ++++-
|
|||
|
|
+ .../rocketmq/remoting/protocol/header/ExtraInfoUtil.java | 4 ++++
|
|||
|
|
+ 3 files changed, 12 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+index 708a6acd1..5101ffc8e 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+@@ -1174,7 +1174,10 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ Long.parseLong(messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET)));
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+- key = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(), messageExt.getQueueId());
|
|||
|
|
++ // Value of POP_CK is used to determine whether it is a pop retry,
|
|||
|
|
++ // cause topic could be rewritten by broker.
|
|||
|
|
++ key = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(),
|
|||
|
|
++ messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getQueueId());
|
|||
|
|
+ if (!sortMap.containsKey(key)) {
|
|||
|
|
+ sortMap.put(key, new ArrayList<>(4));
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+index 115c140ff..eb2c4d9ee 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+@@ -249,7 +249,10 @@ public class LocalMessageService implements MessageService {
|
|||
|
|
+ // <topicMark@queueId, msg queueOffset>
|
|||
|
|
+ Map<String, List<Long>> sortMap = new HashMap<>(16);
|
|||
|
|
+ for (MessageExt messageExt : messageExtList) {
|
|||
|
|
+- String key = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(), messageExt.getQueueId());
|
|||
|
|
++ // Value of POP_CK is used to determine whether it is a pop retry,
|
|||
|
|
++ // cause topic could be rewritten by broker.
|
|||
|
|
++ String key = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(),
|
|||
|
|
++ messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getQueueId());
|
|||
|
|
+ if (!sortMap.containsKey(key)) {
|
|||
|
|
+ sortMap.put(key, new ArrayList<>(4));
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java
|
|||
|
|
+index 9a5fa89ab..13094331e 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/ExtraInfoUtil.java
|
|||
|
|
+@@ -282,6 +282,10 @@ public class ExtraInfoUtil {
|
|||
|
|
+ return (topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) ? RETRY_TOPIC : NORMAL_TOPIC) + "@" + key;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public static String getStartOffsetInfoMapKey(String topic, String popCk, long key) {
|
|||
|
|
++ return ((topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX) || popCk != null) ? RETRY_TOPIC : NORMAL_TOPIC) + "@" + key;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public static String getQueueOffsetKeyValueKey(long queueId, long queueOffset) {
|
|||
|
|
+ return QUEUE_OFFSET + queueId + "%" + queueOffset;
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From a9c0b43f7f6ce5acfc4f2f3069553071fa93dfee Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yx9o <yangx_soft@163.com>
|
|||
|
|
+Date: Wed, 16 Aug 2023 18:45:00 +0800
|
|||
|
|
+Subject: [PATCH 11/12] [ISSUE #7192] Correct typos (#7193)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tools/command/consumer/ConsumerProgressSubCommand.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
|
|||
|
|
+index f51a24673..97125b854 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerProgressSubCommand.java
|
|||
|
|
+@@ -54,7 +54,7 @@ public class ConsumerProgressSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query consumers's progress, speed";
|
|||
|
|
++ return "Query consumer's progress, speed.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 5a3de926b816db5a121c1d788430072a3bc942ae Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Wed, 16 Aug 2023 20:52:53 +0800
|
|||
|
|
+Subject: [PATCH 12/12] Optimize updateSubscription check exist loop (#7190)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/client/ConsumerGroupInfo.java | 17 ++++++-----------
|
|||
|
|
+ 1 file changed, 6 insertions(+), 11 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
|
|||
|
|
+index 867b9c720..1ea58c125 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ConsumerGroupInfo.java
|
|||
|
|
+@@ -18,6 +18,7 @@ package org.apache.rocketmq.broker.client;
|
|||
|
|
+
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
++import java.util.HashSet;
|
|||
|
|
+ import java.util.Iterator;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Map.Entry;
|
|||
|
|
+@@ -172,7 +173,7 @@ public class ConsumerGroupInfo {
|
|||
|
|
+ */
|
|||
|
|
+ public boolean updateSubscription(final Set<SubscriptionData> subList) {
|
|||
|
|
+ boolean updated = false;
|
|||
|
|
+-
|
|||
|
|
++ Set<String> topicSet = new HashSet<>();
|
|||
|
|
+ for (SubscriptionData sub : subList) {
|
|||
|
|
+ SubscriptionData old = this.subscriptionTable.get(sub.getTopic());
|
|||
|
|
+ if (old == null) {
|
|||
|
|
+@@ -194,22 +195,16 @@ public class ConsumerGroupInfo {
|
|||
|
|
+
|
|||
|
|
+ this.subscriptionTable.put(sub.getTopic(), sub);
|
|||
|
|
+ }
|
|||
|
|
++ // Add all new topics to the HashSet
|
|||
|
|
++ topicSet.add(sub.getTopic());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ Iterator<Entry<String, SubscriptionData>> it = this.subscriptionTable.entrySet().iterator();
|
|||
|
|
+ while (it.hasNext()) {
|
|||
|
|
+ Entry<String, SubscriptionData> next = it.next();
|
|||
|
|
+ String oldTopic = next.getKey();
|
|||
|
|
+-
|
|||
|
|
+- boolean exist = false;
|
|||
|
|
+- for (SubscriptionData sub : subList) {
|
|||
|
|
+- if (sub.getTopic().equals(oldTopic)) {
|
|||
|
|
+- exist = true;
|
|||
|
|
+- break;
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- if (!exist) {
|
|||
|
|
++ // Check HashSet with O(1) time complexity
|
|||
|
|
++ if (!topicSet.contains(oldTopic)) {
|
|||
|
|
+ log.warn("subscription changed, group: {} remove topic {} {}",
|
|||
|
|
+ this.groupName,
|
|||
|
|
+ oldTopic,
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch011-backport-optimize-config b/patch011-backport-optimize-config
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..fc8e2eceb
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch011-backport-optimize-config
|
|||
|
|
@@ -0,0 +1,1390 @@
|
|||
|
|
+From 50d1050437ed8748f86ee50261b50a1e1f63162e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Jixiang Jin <lollipop@apache.org>
|
|||
|
|
+Date: Wed, 16 Aug 2023 21:15:00 +0800
|
|||
|
|
+Subject: [PATCH 1/7] To config the cardinalityLimit for openTelemetry metrics
|
|||
|
|
+ exporting and fix logging config for metrics (#7196)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ WORKSPACE | 14 +++---
|
|||
|
|
+ .../broker/metrics/BrokerMetricsManager.java | 47 ++++++++++++++-----
|
|||
|
|
+ .../broker/metrics/PopMetricsManager.java | 11 +++--
|
|||
|
|
+ .../src/main/resources/rmq.broker.logback.xml | 17 ++++---
|
|||
|
|
+ .../apache/rocketmq/common/BrokerConfig.java | 9 ++++
|
|||
|
|
+ .../metrics/ControllerMetricsManager.java | 6 +--
|
|||
|
|
+ pom.xml | 4 +-
|
|||
|
|
+ .../metrics/RemotingMetricsManager.java | 10 ++--
|
|||
|
|
+ .../rocketmq/store/DefaultMessageStore.java | 24 +++++-----
|
|||
|
|
+ .../apache/rocketmq/store/MessageStore.java | 6 +--
|
|||
|
|
+ .../metrics/DefaultStoreMetricsManager.java | 4 +-
|
|||
|
|
+ .../plugin/AbstractPluginMessageStore.java | 6 +--
|
|||
|
|
+ .../tieredstore/TieredMessageStore.java | 6 +--
|
|||
|
|
+ .../metrics/TieredStoreMetricsManager.java | 23 +++++----
|
|||
|
|
+ 14 files changed, 110 insertions(+), 77 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/WORKSPACE b/WORKSPACE
|
|||
|
|
+index a8a0aafe9..3126f2d1d 100644
|
|||
|
|
+--- a/WORKSPACE
|
|||
|
|
++++ b/WORKSPACE
|
|||
|
|
+@@ -88,14 +88,14 @@ maven_install(
|
|||
|
|
+ "io.grpc:grpc-api:1.47.0",
|
|||
|
|
+ "io.grpc:grpc-testing:1.47.0",
|
|||
|
|
+ "org.springframework:spring-core:5.3.26",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-exporter-otlp:1.19.0",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-exporter-prometheus:1.19.0-alpha",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-exporter-logging:1.19.0",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-sdk:1.19.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-exporter-otlp:1.29.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-exporter-prometheus:1.29.0-alpha",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-exporter-logging:1.29.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-sdk:1.29.0",
|
|||
|
|
+ "com.squareup.okio:okio-jvm:3.0.0",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-api:1.19.0",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-sdk-metrics:1.19.0",
|
|||
|
|
+- "io.opentelemetry:opentelemetry-sdk-common:1.19.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-api:1.29.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-sdk-metrics:1.29.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-sdk-common:1.29.0",
|
|||
|
|
+ "io.github.aliyunmq:rocketmq-slf4j-api:1.0.0",
|
|||
|
|
+ "io.github.aliyunmq:rocketmq-logback-classic:1.0.0",
|
|||
|
|
+ "org.slf4j:jul-to-slf4j:2.0.6",
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
+index f0b76107e..6af5afc14 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
+@@ -34,8 +34,10 @@ import io.opentelemetry.sdk.metrics.InstrumentType;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.SdkMeterProvider;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.SdkMeterProviderBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil;
|
|||
|
|
+ import io.opentelemetry.sdk.resources.Resource;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+@@ -361,22 +363,45 @@ public class BrokerMetricsManager {
|
|||
|
|
+ .setType(InstrumentType.HISTOGRAM)
|
|||
|
|
+ .setName(HISTOGRAM_MESSAGE_SIZE)
|
|||
|
|
+ .build();
|
|||
|
|
+- View messageSizeView = View.builder()
|
|||
|
|
+- .setAggregation(Aggregation.explicitBucketHistogram(messageSizeBuckets))
|
|||
|
|
+- .build();
|
|||
|
|
+- providerBuilder.registerView(messageSizeSelector, messageSizeView);
|
|||
|
|
+-
|
|||
|
|
+- for (Pair<InstrumentSelector, View> selectorViewPair : RemotingMetricsManager.getMetricsView()) {
|
|||
|
|
+- providerBuilder.registerView(selectorViewPair.getObject1(), selectorViewPair.getObject2());
|
|||
|
|
++ ViewBuilder messageSizeViewBuilder = View.builder()
|
|||
|
|
++ .setAggregation(Aggregation.explicitBucketHistogram(messageSizeBuckets));
|
|||
|
|
++ // To config the cardinalityLimit for openTelemetry metrics exporting.
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(messageSizeViewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(messageSizeSelector, messageSizeViewBuilder.build());
|
|||
|
|
++
|
|||
|
|
++ for (Pair<InstrumentSelector, ViewBuilder> selectorViewPair : RemotingMetricsManager.getMetricsView()) {
|
|||
|
|
++ ViewBuilder viewBuilder = selectorViewPair.getObject2();
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(viewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(selectorViewPair.getObject1(), viewBuilder.build());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- for (Pair<InstrumentSelector, View> selectorViewPair : messageStore.getMetricsView()) {
|
|||
|
|
+- providerBuilder.registerView(selectorViewPair.getObject1(), selectorViewPair.getObject2());
|
|||
|
|
++ for (Pair<InstrumentSelector, ViewBuilder> selectorViewPair : messageStore.getMetricsView()) {
|
|||
|
|
++ ViewBuilder viewBuilder = selectorViewPair.getObject2();
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(viewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(selectorViewPair.getObject1(), viewBuilder.build());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- for (Pair<InstrumentSelector, View> selectorViewPair : PopMetricsManager.getMetricsView()) {
|
|||
|
|
+- providerBuilder.registerView(selectorViewPair.getObject1(), selectorViewPair.getObject2());
|
|||
|
|
++ for (Pair<InstrumentSelector, ViewBuilder> selectorViewPair : PopMetricsManager.getMetricsView()) {
|
|||
|
|
++ ViewBuilder viewBuilder = selectorViewPair.getObject2();
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(viewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(selectorViewPair.getObject1(), viewBuilder.build());
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ // default view builder for all counter.
|
|||
|
|
++ InstrumentSelector defaultCounterSelector = InstrumentSelector.builder()
|
|||
|
|
++ .setType(InstrumentType.COUNTER)
|
|||
|
|
++ .build();
|
|||
|
|
++ ViewBuilder defaultCounterViewBuilder = View.builder().setDescription("default view for counter.");
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(defaultCounterViewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(defaultCounterSelector, defaultCounterViewBuilder.build());
|
|||
|
|
++
|
|||
|
|
++ //default view builder for all observable gauge.
|
|||
|
|
++ InstrumentSelector defaultGaugeSelector = InstrumentSelector.builder()
|
|||
|
|
++ .setType(InstrumentType.OBSERVABLE_GAUGE)
|
|||
|
|
++ .build();
|
|||
|
|
++ ViewBuilder defaultGaugeViewBuilder = View.builder().setDescription("default view for gauge.");
|
|||
|
|
++ SdkMeterProviderUtil.setCardinalityLimit(defaultGaugeViewBuilder, brokerConfig.getMetricsOtelCardinalityLimit());
|
|||
|
|
++ providerBuilder.registerView(defaultGaugeSelector, defaultGaugeViewBuilder.build());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void initStatsMetrics() {
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/PopMetricsManager.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/PopMetricsManager.java
|
|||
|
|
+index 463371d7e..2de220da1 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/PopMetricsManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/PopMetricsManager.java
|
|||
|
|
+@@ -27,6 +27,7 @@ import io.opentelemetry.sdk.metrics.Aggregation;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentType;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+@@ -63,7 +64,7 @@ public class PopMetricsManager {
|
|||
|
|
+ private static LongCounter popReviveGetTotal = new NopLongCounter();
|
|||
|
|
+ private static LongCounter popReviveRetryMessageTotal = new NopLongCounter();
|
|||
|
|
+
|
|||
|
|
+- public static List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
++ public static List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
+ List<Double> rpcCostTimeBuckets = Arrays.asList(
|
|||
|
|
+ (double) Duration.ofMillis(1).toMillis(),
|
|||
|
|
+ (double) Duration.ofMillis(10).toMillis(),
|
|||
|
|
+@@ -76,10 +77,10 @@ public class PopMetricsManager {
|
|||
|
|
+ .setType(InstrumentType.HISTOGRAM)
|
|||
|
|
+ .setName(HISTOGRAM_POP_BUFFER_SCAN_TIME_CONSUME)
|
|||
|
|
+ .build();
|
|||
|
|
+- View popBufferScanTimeConsumeView = View.builder()
|
|||
|
|
+- .setAggregation(Aggregation.explicitBucketHistogram(rpcCostTimeBuckets))
|
|||
|
|
+- .build();
|
|||
|
|
+- return Lists.newArrayList(new Pair<>(popBufferScanTimeConsumeSelector, popBufferScanTimeConsumeView));
|
|||
|
|
++ ViewBuilder popBufferScanTimeConsumeViewBuilder = View.builder()
|
|||
|
|
++ .setAggregation(Aggregation.explicitBucketHistogram(rpcCostTimeBuckets));
|
|||
|
|
++
|
|||
|
|
++ return Lists.newArrayList(new Pair<>(popBufferScanTimeConsumeSelector, popBufferScanTimeConsumeViewBuilder));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static void initMetrics(Meter meter, BrokerController brokerController,
|
|||
|
|
+diff --git a/broker/src/main/resources/rmq.broker.logback.xml b/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
+index 7d49f6664..3c51e59d4 100644
|
|||
|
|
+--- a/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
++++ b/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
+@@ -559,27 +559,27 @@
|
|||
|
|
+ </sift>
|
|||
|
|
+ </appender>
|
|||
|
|
+
|
|||
|
|
+- <appender name="RocketmqBrokerMetricsAppender" class="ch.qos.logback.classic.sift.SiftingAppender">
|
|||
|
|
++ <appender name="RocketmqBrokerMetricsSiftingAppender_inner" class="ch.qos.logback.classic.sift.SiftingAppender">
|
|||
|
|
+ <discriminator>
|
|||
|
|
+ <key>brokerContainerLogDir</key>
|
|||
|
|
+ <defaultValue>${file.separator}</defaultValue>
|
|||
|
|
+ </discriminator>
|
|||
|
|
+ <sift>
|
|||
|
|
+- <appender name="RocketmqCommercialAppender"
|
|||
|
|
++ <appender name="RocketmqBrokerMetricsAppender"
|
|||
|
|
+ class="ch.qos.logback.core.rolling.RollingFileAppender">
|
|||
|
|
+ <file>
|
|||
|
|
+- ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}broker_metric.log
|
|||
|
|
++ ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}broker_metrics.log
|
|||
|
|
+ </file>
|
|||
|
|
+ <append>true</append>
|
|||
|
|
+ <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
|
|||
|
|
+ <fileNamePattern>
|
|||
|
|
+- ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}otherdays${file.separator}broker_metric.%i.log.gz
|
|||
|
|
++ ${user.home}${file.separator}logs${file.separator}rocketmqlogs${brokerLogDir:-${file.separator}}${brokerContainerLogDir}${file.separator}otherdays${file.separator}broker_metrics.%i.log.gz
|
|||
|
|
+ </fileNamePattern>
|
|||
|
|
+ <minIndex>1</minIndex>
|
|||
|
|
+- <maxIndex>10</maxIndex>
|
|||
|
|
++ <maxIndex>3</maxIndex>
|
|||
|
|
+ </rollingPolicy>
|
|||
|
|
+ <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
|
|||
|
|
+- <maxFileSize>500MB</maxFileSize>
|
|||
|
|
++ <maxFileSize>512MB</maxFileSize>
|
|||
|
|
+ </triggeringPolicy>
|
|||
|
|
+ <encoder>
|
|||
|
|
+ <pattern>%d{yyy-MM-dd HH:mm:ss,GMT+8} %p %t - %m%n</pattern>
|
|||
|
|
+@@ -588,6 +588,9 @@
|
|||
|
|
+ </appender>
|
|||
|
|
+ </sift>
|
|||
|
|
+ </appender>
|
|||
|
|
++ <appender name="RocketmqBrokerMetricsSiftingAppender" class="ch.qos.logback.classic.AsyncAppender">
|
|||
|
|
++ <appender-ref ref="RocketmqBrokerMetricsSiftingAppender_inner"/>
|
|||
|
|
++ </appender>
|
|||
|
|
+
|
|||
|
|
+ <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
|
|||
|
|
+ <encoder>
|
|||
|
|
+@@ -670,7 +673,7 @@
|
|||
|
|
+ </logger>
|
|||
|
|
+
|
|||
|
|
+ <logger name="io.opentelemetry.exporter.logging.LoggingMetricExporter" additivity="false" level="INFO">
|
|||
|
|
+- <appender-ref ref="RocketmqBrokerMetricsAppender"/>
|
|||
|
|
++ <appender-ref ref="RocketmqBrokerMetricsSiftingAppender"/>
|
|||
|
|
+ </logger>
|
|||
|
|
+
|
|||
|
|
+ <root level="INFO">
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+index 99a5db5ad..45d26b29c 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+@@ -350,6 +350,7 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+
|
|||
|
|
+ private MetricsExporterType metricsExporterType = MetricsExporterType.DISABLE;
|
|||
|
|
+
|
|||
|
|
++ private int metricsOtelCardinalityLimit = 50 * 1000;
|
|||
|
|
+ private String metricsGrpcExporterTarget = "";
|
|||
|
|
+ private String metricsGrpcExporterHeader = "";
|
|||
|
|
+ private long metricGrpcExporterTimeOutInMills = 3 * 1000;
|
|||
|
|
+@@ -1531,6 +1532,14 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+ this.metricsExporterType = MetricsExporterType.valueOf(metricsExporterType);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public int getMetricsOtelCardinalityLimit() {
|
|||
|
|
++ return metricsOtelCardinalityLimit;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setMetricsOtelCardinalityLimit(int metricsOtelCardinalityLimit) {
|
|||
|
|
++ this.metricsOtelCardinalityLimit = metricsOtelCardinalityLimit;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public String getMetricsGrpcExporterTarget() {
|
|||
|
|
+ return metricsGrpcExporterTarget;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java b/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
+index 9b30a3b43..650740bcc 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
+@@ -203,7 +203,7 @@ public class ControllerMetricsManager {
|
|||
|
|
+ 10 * s
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
+- View latecyView = View.builder()
|
|||
|
|
++ View latencyView = View.builder()
|
|||
|
|
+ .setAggregation(Aggregation.explicitBucketHistogram(latencyBuckets))
|
|||
|
|
+ .build();
|
|||
|
|
+
|
|||
|
|
+@@ -217,8 +217,8 @@ public class ControllerMetricsManager {
|
|||
|
|
+ .setName(HISTOGRAM_DLEDGER_OP_LATENCY)
|
|||
|
|
+ .build();
|
|||
|
|
+
|
|||
|
|
+- providerBuilder.registerView(requestLatencySelector, latecyView);
|
|||
|
|
+- providerBuilder.registerView(dLedgerOpLatencySelector, latecyView);
|
|||
|
|
++ providerBuilder.registerView(requestLatencySelector, latencyView);
|
|||
|
|
++ providerBuilder.registerView(dLedgerOpLatencySelector, latencyView);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void initMetric(Meter meter) {
|
|||
|
|
+diff --git a/pom.xml b/pom.xml
|
|||
|
|
+index 3a08d75f2..9f0b3eb96 100644
|
|||
|
|
+--- a/pom.xml
|
|||
|
|
++++ b/pom.xml
|
|||
|
|
+@@ -133,8 +133,8 @@
|
|||
|
|
+ <caffeine.version>2.9.3</caffeine.version>
|
|||
|
|
+ <spring.version>5.3.27</spring.version>
|
|||
|
|
+ <okio-jvm.version>3.0.0</okio-jvm.version>
|
|||
|
|
+- <opentelemetry.version>1.26.0</opentelemetry.version>
|
|||
|
|
+- <opentelemetry-exporter-prometheus.version>1.26.0-alpha</opentelemetry-exporter-prometheus.version>
|
|||
|
|
++ <opentelemetry.version>1.29.0</opentelemetry.version>
|
|||
|
|
++ <opentelemetry-exporter-prometheus.version>1.29.0-alpha</opentelemetry-exporter-prometheus.version>
|
|||
|
|
+ <jul-to-slf4j.version>2.0.6</jul-to-slf4j.version>
|
|||
|
|
+ <s3.version>2.20.29</s3.version>
|
|||
|
|
+ <rocksdb.version>1.0.3</rocksdb.version>
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/metrics/RemotingMetricsManager.java b/remoting/src/main/java/org/apache/rocketmq/remoting/metrics/RemotingMetricsManager.java
|
|||
|
|
+index 34136f94f..2e0d70856 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/metrics/RemotingMetricsManager.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/metrics/RemotingMetricsManager.java
|
|||
|
|
+@@ -26,6 +26,7 @@ import io.opentelemetry.sdk.metrics.Aggregation;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentType;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+@@ -61,7 +62,7 @@ public class RemotingMetricsManager {
|
|||
|
|
+ .build();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public static List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
++ public static List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
+ List<Double> rpcCostTimeBuckets = Arrays.asList(
|
|||
|
|
+ (double) Duration.ofMillis(1).toMillis(),
|
|||
|
|
+ (double) Duration.ofMillis(3).toMillis(),
|
|||
|
|
+@@ -77,10 +78,9 @@ public class RemotingMetricsManager {
|
|||
|
|
+ .setType(InstrumentType.HISTOGRAM)
|
|||
|
|
+ .setName(HISTOGRAM_RPC_LATENCY)
|
|||
|
|
+ .build();
|
|||
|
|
+- View view = View.builder()
|
|||
|
|
+- .setAggregation(Aggregation.explicitBucketHistogram(rpcCostTimeBuckets))
|
|||
|
|
+- .build();
|
|||
|
|
+- return Lists.newArrayList(new Pair<>(selector, view));
|
|||
|
|
++ ViewBuilder viewBuilder = View.builder()
|
|||
|
|
++ .setAggregation(Aggregation.explicitBucketHistogram(rpcCostTimeBuckets));
|
|||
|
|
++ return Lists.newArrayList(new Pair<>(selector, viewBuilder));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static String getWriteAndFlushResult(Future<?> future) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+index 25e4a166f..6115ead59 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+@@ -22,7 +22,7 @@ import io.openmessaging.storage.dledger.entry.DLedgerEntry;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+-import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.io.RandomAccessFile;
|
|||
|
|
+@@ -42,23 +42,24 @@ import java.util.Map;
|
|||
|
|
+ import java.util.Objects;
|
|||
|
|
+ import java.util.Optional;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+-import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-import java.util.concurrent.TimeoutException;
|
|||
|
|
+-import java.util.concurrent.ExecutionException;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+-import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+-import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+-import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentLinkedQueue;
|
|||
|
|
++import java.util.concurrent.ConcurrentMap;
|
|||
|
|
++import java.util.concurrent.ExecutionException;
|
|||
|
|
++import java.util.concurrent.ExecutorService;
|
|||
|
|
++import java.util.concurrent.Executors;
|
|||
|
|
++import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
++import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
++import java.util.concurrent.TimeoutException;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
+ import java.util.function.Supplier;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.AbstractBrokerRunnable;
|
|||
|
|
++import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerIdentity;
|
|||
|
|
+ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+@@ -82,7 +83,6 @@ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.CleanupPolicyUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.QueueTypeUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.ServiceProvider;
|
|||
|
|
+-import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo;
|
|||
|
|
+@@ -3268,7 +3268,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
++ public List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
+ return DefaultStoreMetricsManager.getMetricsView();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
+index 31bbb907f..989cbbe31 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
+@@ -19,8 +19,7 @@ package org.apache.rocketmq.store;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+-import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
+-
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+@@ -28,7 +27,6 @@ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.function.Supplier;
|
|||
|
|
+-
|
|||
|
|
+ import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.SystemClock;
|
|||
|
|
+@@ -964,7 +962,7 @@ public interface MessageStore {
|
|||
|
|
+ *
|
|||
|
|
+ * @return List of metrics selector and view pair
|
|||
|
|
+ */
|
|||
|
|
+- List<Pair<InstrumentSelector, View>> getMetricsView();
|
|||
|
|
++ List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Init store metrics
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java b/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java
|
|||
|
|
+index ff87f6369..45a6bbc68 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/metrics/DefaultStoreMetricsManager.java
|
|||
|
|
+@@ -23,7 +23,7 @@ import io.opentelemetry.api.metrics.LongCounter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.ObservableLongGauge;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+-import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.function.Supplier;
|
|||
|
|
+@@ -69,7 +69,7 @@ public class DefaultStoreMetricsManager {
|
|||
|
|
+ public static LongCounter timerDequeueTotal = new NopLongCounter();
|
|||
|
|
+ public static LongCounter timerEnqueueTotal = new NopLongCounter();
|
|||
|
|
+
|
|||
|
|
+- public static List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
++ public static List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
+ return Lists.newArrayList();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
+index 25e947512..ab9fc6da7 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
+@@ -20,8 +20,7 @@ package org.apache.rocketmq.store.plugin;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+-import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
+-
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+@@ -29,7 +28,6 @@ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.function.Supplier;
|
|||
|
|
+-
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.SystemClock;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+@@ -643,7 +641,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
++ public List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
+ return next.getMetricsView();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+index ced1fb818..5240ac8e9 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+@@ -21,7 +21,7 @@ import io.opentelemetry.api.common.Attributes;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+-import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+@@ -352,8 +352,8 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
+- List<Pair<InstrumentSelector, View>> res = super.getMetricsView();
|
|||
|
|
++ public List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
++ List<Pair<InstrumentSelector, ViewBuilder>> res = super.getMetricsView();
|
|||
|
|
+ res.addAll(TieredStoreMetricsManager.getMetricsView());
|
|||
|
|
+ return res;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java
|
|||
|
|
+index 3ca0fb614..d8a07f0a7 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java
|
|||
|
|
+@@ -27,6 +27,7 @@ import io.opentelemetry.sdk.metrics.Aggregation;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentType;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+@@ -101,8 +102,8 @@ public class TieredStoreMetricsManager {
|
|||
|
|
+ public static ObservableLongGauge storageSize = new NopObservableLongGauge();
|
|||
|
|
+ public static ObservableLongGauge storageMessageReserveTime = new NopObservableLongGauge();
|
|||
|
|
+
|
|||
|
|
+- public static List<Pair<InstrumentSelector, View>> getMetricsView() {
|
|||
|
|
+- ArrayList<Pair<InstrumentSelector, View>> res = new ArrayList<>();
|
|||
|
|
++ public static List<Pair<InstrumentSelector, ViewBuilder>> getMetricsView() {
|
|||
|
|
++ ArrayList<Pair<InstrumentSelector, ViewBuilder>> res = new ArrayList<>();
|
|||
|
|
+
|
|||
|
|
+ InstrumentSelector providerRpcLatencySelector = InstrumentSelector.builder()
|
|||
|
|
+ .setType(InstrumentType.HISTOGRAM)
|
|||
|
|
+@@ -114,10 +115,9 @@ public class TieredStoreMetricsManager {
|
|||
|
|
+ .setName(HISTOGRAM_API_LATENCY)
|
|||
|
|
+ .build();
|
|||
|
|
+
|
|||
|
|
+- View rpcLatencyView = View.builder()
|
|||
|
|
++ ViewBuilder rpcLatencyViewBuilder = View.builder()
|
|||
|
|
+ .setAggregation(Aggregation.explicitBucketHistogram(Arrays.asList(1d, 3d, 5d, 7d, 10d, 100d, 200d, 400d, 600d, 800d, 1d * 1000, 1d * 1500, 1d * 3000)))
|
|||
|
|
+- .setDescription("tiered_store_rpc_latency_view")
|
|||
|
|
+- .build();
|
|||
|
|
++ .setDescription("tiered_store_rpc_latency_view");
|
|||
|
|
+
|
|||
|
|
+ InstrumentSelector uploadBufferSizeSelector = InstrumentSelector.builder()
|
|||
|
|
+ .setType(InstrumentType.HISTOGRAM)
|
|||
|
|
+@@ -129,15 +129,14 @@ public class TieredStoreMetricsManager {
|
|||
|
|
+ .setName(HISTOGRAM_DOWNLOAD_BYTES)
|
|||
|
|
+ .build();
|
|||
|
|
+
|
|||
|
|
+- View bufferSizeView = View.builder()
|
|||
|
|
++ ViewBuilder bufferSizeViewBuilder = View.builder()
|
|||
|
|
+ .setAggregation(Aggregation.explicitBucketHistogram(Arrays.asList(1d * TieredStoreUtil.KB, 10d * TieredStoreUtil.KB, 100d * TieredStoreUtil.KB, 1d * TieredStoreUtil.MB, 10d * TieredStoreUtil.MB, 32d * TieredStoreUtil.MB, 50d * TieredStoreUtil.MB, 100d * TieredStoreUtil.MB)))
|
|||
|
|
+- .setDescription("tiered_store_buffer_size_view")
|
|||
|
|
+- .build();
|
|||
|
|
++ .setDescription("tiered_store_buffer_size_view");
|
|||
|
|
+
|
|||
|
|
+- res.add(new Pair<>(rpcLatencySelector, rpcLatencyView));
|
|||
|
|
+- res.add(new Pair<>(providerRpcLatencySelector, rpcLatencyView));
|
|||
|
|
+- res.add(new Pair<>(uploadBufferSizeSelector, bufferSizeView));
|
|||
|
|
+- res.add(new Pair<>(downloadBufferSizeSelector, bufferSizeView));
|
|||
|
|
++ res.add(new Pair<>(rpcLatencySelector, rpcLatencyViewBuilder));
|
|||
|
|
++ res.add(new Pair<>(providerRpcLatencySelector, rpcLatencyViewBuilder));
|
|||
|
|
++ res.add(new Pair<>(uploadBufferSizeSelector, bufferSizeViewBuilder));
|
|||
|
|
++ res.add(new Pair<>(downloadBufferSizeSelector, bufferSizeViewBuilder));
|
|||
|
|
+ return res;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From a4bcc2a74d8bec9c9d34565536e87df06e0b11c1 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <tanziyi0925@gmail.com>
|
|||
|
|
+Date: Thu, 17 Aug 2023 13:53:48 +0800
|
|||
|
|
+Subject: [PATCH 2/7] [ISSUE #7178] refresh metadata after broker startup
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb459684460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/broker/BrokerController.java | 24 +++++++++----------
|
|||
|
|
+ 1 file changed, 12 insertions(+), 12 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 30b1d2299..13f9d002b 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -663,7 +663,7 @@ public class BrokerController {
|
|||
|
|
+ BrokerController.this.getSlaveSynchronize().syncAll();
|
|||
|
|
+ lastSyncTimeMs = System.currentTimeMillis();
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
++
|
|||
|
|
+ //timer checkpoint, latency-sensitive, so sync it more frequently
|
|||
|
|
+ if (messageStoreConfig.isTimerWheelEnable()) {
|
|||
|
|
+ BrokerController.this.getSlaveSynchronize().syncTimerCheckPoint();
|
|||
|
|
+@@ -698,17 +698,6 @@ public class BrokerController {
|
|||
|
|
+
|
|||
|
|
+ initializeBrokerScheduledTasks();
|
|||
|
|
+
|
|||
|
|
+- this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
|
|||
|
|
+- @Override
|
|||
|
|
+- public void run() {
|
|||
|
|
+- try {
|
|||
|
|
+- BrokerController.this.brokerOuterAPI.refreshMetadata();
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- LOG.error("ScheduledTask refresh metadata exception", e);
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+- }, 10, 5, TimeUnit.SECONDS);
|
|||
|
|
+-
|
|||
|
|
+ if (this.brokerConfig.getNamesrvAddr() != null) {
|
|||
|
|
+ this.updateNamesrvAddr();
|
|||
|
|
+ LOG.info("Set user specified name server address: {}", this.brokerConfig.getNamesrvAddr());
|
|||
|
|
+@@ -1682,6 +1671,17 @@ public class BrokerController {
|
|||
|
|
+ if (brokerConfig.isSkipPreOnline()) {
|
|||
|
|
+ startServiceWithoutCondition();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void run() {
|
|||
|
|
++ try {
|
|||
|
|
++ BrokerController.this.brokerOuterAPI.refreshMetadata();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ LOG.error("ScheduledTask refresh metadata exception", e);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }, 10, 5, TimeUnit.SECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ protected void scheduleSendHeartbeat() {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3df1b9232af99944cb3d4d4d2d00c5a85cd3b57d Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Thu, 17 Aug 2023 13:59:04 +0800
|
|||
|
|
+Subject: [PATCH 3/7] [ISSUE #7201] Remove the DefaultMessageStore.class
|
|||
|
|
+ dependency in TransientStorePool
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: guyinyou <guyinyou.gyy@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/store/AllocateMappedFileService.java | 6 +++---
|
|||
|
|
+ .../apache/rocketmq/store/DefaultMessageStore.java | 7 +++++--
|
|||
|
|
+ .../apache/rocketmq/store/TransientStorePool.java | 13 ++++---------
|
|||
|
|
+ 3 files changed, 12 insertions(+), 14 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
|
|||
|
|
+index dca7d5325..c8420fea1 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java
|
|||
|
|
+@@ -55,7 +55,7 @@ public class AllocateMappedFileService extends ServiceThread {
|
|||
|
|
+ if (this.messageStore.isTransientStorePoolEnable()) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isFastFailIfNoBufferInStorePool()
|
|||
|
|
+ && BrokerRole.SLAVE != this.messageStore.getMessageStoreConfig().getBrokerRole()) { //if broker is slave, don't fast fail even no buffer in pool
|
|||
|
|
+- canSubmitRequests = this.messageStore.getTransientStorePool().availableBufferNums() - this.requestQueue.size();
|
|||
|
|
++ canSubmitRequests = this.messageStore.remainTransientStoreBufferNumbs() - this.requestQueue.size();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -65,7 +65,7 @@ public class AllocateMappedFileService extends ServiceThread {
|
|||
|
|
+ if (nextPutOK) {
|
|||
|
|
+ if (canSubmitRequests <= 0) {
|
|||
|
|
+ log.warn("[NOTIFYME]TransientStorePool is not enough, so create mapped file error, " +
|
|||
|
|
+- "RequestQueueSize : {}, StorePoolSize: {}", this.requestQueue.size(), this.messageStore.getTransientStorePool().availableBufferNums());
|
|||
|
|
++ "RequestQueueSize : {}, StorePoolSize: {}", this.requestQueue.size(), this.messageStore.remainTransientStoreBufferNumbs());
|
|||
|
|
+ this.requestTable.remove(nextFilePath);
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+@@ -81,7 +81,7 @@ public class AllocateMappedFileService extends ServiceThread {
|
|||
|
|
+ if (nextNextPutOK) {
|
|||
|
|
+ if (canSubmitRequests <= 0) {
|
|||
|
|
+ log.warn("[NOTIFYME]TransientStorePool is not enough, so skip preallocate mapped file, " +
|
|||
|
|
+- "RequestQueueSize : {}, StorePoolSize: {}", this.requestQueue.size(), this.messageStore.getTransientStorePool().availableBufferNums());
|
|||
|
|
++ "RequestQueueSize : {}, StorePoolSize: {}", this.requestQueue.size(), this.messageStore.remainTransientStoreBufferNumbs());
|
|||
|
|
+ this.requestTable.remove(nextNextFilePath);
|
|||
|
|
+ } else {
|
|||
|
|
+ boolean offerOK = this.requestQueue.offer(nextNextReq);
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+index 6115ead59..f2a54ddf6 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+@@ -250,7 +250,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ this.reputMessageService = new ConcurrentReputMessageService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- this.transientStorePool = new TransientStorePool(this);
|
|||
|
|
++ this.transientStorePool = new TransientStorePool(messageStoreConfig.getTransientStorePoolSize(), messageStoreConfig.getMappedFileSizeCommitLog());
|
|||
|
|
+
|
|||
|
|
+ this.scheduledExecutorService =
|
|||
|
|
+ Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread", getBrokerIdentity()));
|
|||
|
|
+@@ -1983,7 +1983,10 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public int remainTransientStoreBufferNumbs() {
|
|||
|
|
+- return this.transientStorePool.availableBufferNums();
|
|||
|
|
++ if (this.isTransientStorePoolEnable()) {
|
|||
|
|
++ return this.transientStorePool.availableBufferNums();
|
|||
|
|
++ }
|
|||
|
|
++ return Integer.MAX_VALUE;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java b/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java
|
|||
|
|
+index 8c1a5338b..0d42ee69e 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java
|
|||
|
|
+@@ -33,13 +33,11 @@ public class TransientStorePool {
|
|||
|
|
+ private final int poolSize;
|
|||
|
|
+ private final int fileSize;
|
|||
|
|
+ private final Deque<ByteBuffer> availableBuffers;
|
|||
|
|
+- private final DefaultMessageStore messageStore;
|
|||
|
|
+ private volatile boolean isRealCommit = true;
|
|||
|
|
+
|
|||
|
|
+- public TransientStorePool(final DefaultMessageStore messageStore) {
|
|||
|
|
+- this.messageStore = messageStore;
|
|||
|
|
+- this.poolSize = messageStore.getMessageStoreConfig().getTransientStorePoolSize();
|
|||
|
|
+- this.fileSize = messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
|
|||
|
|
++ public TransientStorePool(final int poolSize, final int fileSize) {
|
|||
|
|
++ this.poolSize = poolSize;
|
|||
|
|
++ this.fileSize = fileSize;
|
|||
|
|
+ this.availableBuffers = new ConcurrentLinkedDeque<>();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -81,10 +79,7 @@ public class TransientStorePool {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public int availableBufferNums() {
|
|||
|
|
+- if (messageStore.isTransientStorePoolEnable()) {
|
|||
|
|
+- return availableBuffers.size();
|
|||
|
|
+- }
|
|||
|
|
+- return Integer.MAX_VALUE;
|
|||
|
|
++ return availableBuffers.size();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean isRealCommit() {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 2b93e1e32fd458d9df2091e89ea259ddd4d54061 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: iamgd67 <iamgd67@sina.com>
|
|||
|
|
+Date: Thu, 17 Aug 2023 15:31:14 +0800
|
|||
|
|
+Subject: [PATCH 4/7] Update mqbroker to use runbroker.sh instead of
|
|||
|
|
+ runserver.sh when use --enable-proxy (#7150)
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+Update mqbroker to use runbroker.sh instead of runserver.sh when enabling `--enable-proxy`
|
|||
|
|
+this allow JVM `heap` and `gc` configuration using broker's settings instead of other common serverices'(proxy,namenode, etc).
|
|||
|
|
+our main purpose, like the filename `mqbroker` suggest, is to start broker (which embeds a proxy), so use broker's config is reasonable
|
|||
|
|
+
|
|||
|
|
+chinese version
|
|||
|
|
+mqbroker的--enable-proxy选项是启动内嵌了proxy的broker,而不是内嵌broker的proxy,而且broker的工作量和重要程度大于proxy,所以使用broker的gc和heap配置更合适
|
|||
|
|
+---
|
|||
|
|
+ distribution/bin/mqbroker | 4 ++--
|
|||
|
|
+ 1 file changed, 2 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/distribution/bin/mqbroker b/distribution/bin/mqbroker
|
|||
|
|
+index 3758ed597..35eb93c44 100644
|
|||
|
|
+--- a/distribution/bin/mqbroker
|
|||
|
|
++++ b/distribution/bin/mqbroker
|
|||
|
|
+@@ -68,11 +68,11 @@ if [ "$enable_proxy" = true ]; then
|
|||
|
|
+ if [ "$broker_config" != "" ]; then
|
|||
|
|
+ args_for_proxy=${args_for_proxy}" -bc "${broker_config}
|
|||
|
|
+ fi
|
|||
|
|
+- sh ${ROCKETMQ_HOME}/bin/runserver.sh -Drmq.logback.configurationFile=$ROCKETMQ_HOME/conf/rmq.proxy.logback.xml org.apache.rocketmq.proxy.ProxyStartup ${args_for_proxy}
|
|||
|
|
++ sh ${ROCKETMQ_HOME}/bin/runbroker.sh -Drmq.logback.configurationFile=$ROCKETMQ_HOME/conf/rmq.proxy.logback.xml org.apache.rocketmq.proxy.ProxyStartup ${args_for_proxy}
|
|||
|
|
+ else
|
|||
|
|
+ args_for_broker=$other_args
|
|||
|
|
+ if [ "$broker_config" != "" ]; then
|
|||
|
|
+ args_for_broker=${args_for_broker}" -c "${broker_config}
|
|||
|
|
+ fi
|
|||
|
|
+ sh ${ROCKETMQ_HOME}/bin/runbroker.sh -Drmq.logback.configurationFile=$ROCKETMQ_HOME/conf/rmq.broker.logback.xml org.apache.rocketmq.broker.BrokerStartup ${args_for_broker}
|
|||
|
|
+-fi
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
++fi
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 05e7cde610255ed9410fffb0f153efe7c2c8a326 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yao-wenbin <ywb992134@163.com>
|
|||
|
|
+Date: Fri, 18 Aug 2023 09:49:59 +0800
|
|||
|
|
+Subject: [PATCH 5/7] [ISSUE #7042] maven-compile job failed, Because TlsTest's
|
|||
|
|
+ serverRejectsSSLClient test case will throw TooLongFrameException (#7179)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../remoting/netty/NettyRemotingServer.java | 2 +-
|
|||
|
|
+ .../java/org/apache/rocketmq/remoting/TlsTest.java | 14 ++++++++++++--
|
|||
|
|
+ 2 files changed, 13 insertions(+), 3 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+index 90e358ce3..17f138f86 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+@@ -502,7 +502,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ case DISABLED:
|
|||
|
|
+ ctx.close();
|
|||
|
|
+ log.warn("Clients intend to establish an SSL connection while this server is running in SSL disabled mode");
|
|||
|
|
+- break;
|
|||
|
|
++ throw new UnsupportedOperationException("The NettyRemotingServer in SSL disabled mode doesn't support ssl client");
|
|||
|
|
+ case PERMISSIVE:
|
|||
|
|
+ case ENFORCING:
|
|||
|
|
+ if (null != sslContext) {
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/TlsTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/TlsTest.java
|
|||
|
|
+index de7edbbfb..a4890d73d 100644
|
|||
|
|
+--- a/remoting/src/test/java/org/apache/rocketmq/remoting/TlsTest.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/TlsTest.java
|
|||
|
|
+@@ -144,8 +144,13 @@ public class TlsTest {
|
|||
|
|
+ tlsClientKeyPath = "";
|
|||
|
|
+ tlsClientCertPath = "";
|
|||
|
|
+ clientConfig.setUseTLS(false);
|
|||
|
|
+- } else if ("serverRejectsSSLClient".equals(name.getMethodName())) {
|
|||
|
|
++ } else if ("disabledServerRejectsSSLClient".equals(name.getMethodName())) {
|
|||
|
|
+ tlsMode = TlsMode.DISABLED;
|
|||
|
|
++ } else if ("disabledServerAcceptUnAuthClient".equals(name.getMethodName())) {
|
|||
|
|
++ tlsMode = TlsMode.DISABLED;
|
|||
|
|
++ tlsClientKeyPath = "";
|
|||
|
|
++ tlsClientCertPath = "";
|
|||
|
|
++ clientConfig.setUseTLS(false);
|
|||
|
|
+ } else if ("reloadSslContextForServer".equals(name.getMethodName())) {
|
|||
|
|
+ tlsClientAuthServer = false;
|
|||
|
|
+ tlsServerNeedClientAuth = "none";
|
|||
|
|
+@@ -211,7 +216,7 @@ public class TlsTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+- public void serverRejectsSSLClient() throws Exception {
|
|||
|
|
++ public void disabledServerRejectsSSLClient() throws Exception {
|
|||
|
|
+ try {
|
|||
|
|
+ RemotingCommand response = remotingClient.invokeSync(getServerAddress(), createRequest(), 1000 * 5);
|
|||
|
|
+ failBecauseExceptionWasNotThrown(RemotingSendRequestException.class);
|
|||
|
|
+@@ -219,6 +224,11 @@ public class TlsTest {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void disabledServerAcceptUnAuthClient() throws Exception {
|
|||
|
|
++ requestThenAssertResponse();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ /**
|
|||
|
|
+ * Tests that a server configured to require client authentication refuses to accept connections
|
|||
|
|
+ * from a client that has an untrusted certificate.
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 72d796f2b20b3ec6aebca8c004d9275d7c749a95 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Fri, 18 Aug 2023 11:55:39 +0800
|
|||
|
|
+Subject: [PATCH 6/7] [ISSUE #7205] support batch ack for pop orderly (#7206)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/processor/AckMessageProcessor.java | 99 ++++++-----
|
|||
|
|
+ .../rocketmq/client/impl/MQClientAPIImpl.java | 91 ++++++++--
|
|||
|
|
+ .../test/client/rmq/RMQPopClient.java | 22 +++
|
|||
|
|
+ .../client/consumer/pop/BasePopNormally.java | 6 +
|
|||
|
|
+ .../test/client/consumer/pop/BatchAckIT.java | 159 ++++++++++++++++++
|
|||
|
|
+ 5 files changed, 322 insertions(+), 55 deletions(-)
|
|||
|
|
+ create mode 100644 test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BatchAckIT.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
+index 687811409..244b459d6 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AckMessageProcessor.java
|
|||
|
|
+@@ -19,6 +19,7 @@ package org.apache.rocketmq.broker.processor;
|
|||
|
|
+ import com.alibaba.fastjson.JSON;
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import io.netty.channel.ChannelHandlerContext;
|
|||
|
|
++import java.util.BitSet;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.broker.metrics.PopMetricsManager;
|
|||
|
|
+ import org.apache.rocketmq.common.KeyBuilder;
|
|||
|
|
+@@ -186,46 +187,7 @@ public class AckMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
+ invisibleTime = ExtraInfoUtil.getInvisibleTime(extraInfo);
|
|||
|
|
+
|
|||
|
|
+ if (rqId == KeyBuilder.POP_ORDER_REVIVE_QUEUE) {
|
|||
|
|
+- // order
|
|||
|
|
+- String lockKey = topic + PopAckConstants.SPLIT + consumeGroup + PopAckConstants.SPLIT + qId;
|
|||
|
|
+- long oldOffset = this.brokerController.getConsumerOffsetManager().queryOffset(consumeGroup, topic, qId);
|
|||
|
|
+- if (ackOffset < oldOffset) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- while (!this.brokerController.getPopMessageProcessor().getQueueLockManager().tryLock(lockKey)) {
|
|||
|
|
+- }
|
|||
|
|
+- try {
|
|||
|
|
+- oldOffset = this.brokerController.getConsumerOffsetManager().queryOffset(consumeGroup, topic, qId);
|
|||
|
|
+- if (ackOffset < oldOffset) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- long nextOffset = brokerController.getConsumerOrderInfoManager().commitAndNext(
|
|||
|
|
+- topic, consumeGroup,
|
|||
|
|
+- qId, ackOffset,
|
|||
|
|
+- popTime);
|
|||
|
|
+- if (nextOffset > -1) {
|
|||
|
|
+- if (!this.brokerController.getConsumerOffsetManager().hasOffsetReset(
|
|||
|
|
+- topic, consumeGroup, qId)) {
|
|||
|
|
+- this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(),
|
|||
|
|
+- consumeGroup, topic, qId, nextOffset);
|
|||
|
|
+- }
|
|||
|
|
+- if (!this.brokerController.getConsumerOrderInfoManager().checkBlock(null, topic,
|
|||
|
|
+- consumeGroup, qId, invisibleTime)) {
|
|||
|
|
+- this.brokerController.getPopMessageProcessor().notifyMessageArriving(
|
|||
|
|
+- topic, consumeGroup, qId);
|
|||
|
|
+- }
|
|||
|
|
+- } else if (nextOffset == -1) {
|
|||
|
|
+- String errorInfo = String.format("offset is illegal, key:%s, old:%d, commit:%d, next:%d, %s",
|
|||
|
|
+- lockKey, oldOffset, ackOffset, nextOffset, channel.remoteAddress());
|
|||
|
|
+- POP_LOGGER.warn(errorInfo);
|
|||
|
|
+- response.setCode(ResponseCode.MESSAGE_ILLEGAL);
|
|||
|
|
+- response.setRemark(errorInfo);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- } finally {
|
|||
|
|
+- this.brokerController.getPopMessageProcessor().getQueueLockManager().unLock(lockKey);
|
|||
|
|
+- }
|
|||
|
|
+- brokerController.getPopInflightMessageCounter().decrementInFlightMessageNum(topic, consumeGroup, popTime, qId, ackCount);
|
|||
|
|
++ ackOrderly(topic, consumeGroup, qId, ackOffset, popTime, invisibleTime, channel, response);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -250,17 +212,22 @@ public class AckMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ BatchAckMsg batchAckMsg = new BatchAckMsg();
|
|||
|
|
+- for (int i = 0; batchAck.getBitSet() != null && i < batchAck.getBitSet().length(); i++) {
|
|||
|
|
+- if (!batchAck.getBitSet().get(i)) {
|
|||
|
|
+- continue;
|
|||
|
|
++ BitSet bitSet = batchAck.getBitSet();
|
|||
|
|
++ for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
|
|||
|
|
++ if (i == Integer.MAX_VALUE) {
|
|||
|
|
++ break;
|
|||
|
|
+ }
|
|||
|
|
+ long offset = startOffset + i;
|
|||
|
|
+ if (offset < minOffset || offset > maxOffset) {
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+- batchAckMsg.getAckOffsetList().add(offset);
|
|||
|
|
++ if (rqId == KeyBuilder.POP_ORDER_REVIVE_QUEUE) {
|
|||
|
|
++ ackOrderly(topic, consumeGroup, qId, offset, popTime, invisibleTime, channel, response);
|
|||
|
|
++ } else {
|
|||
|
|
++ batchAckMsg.getAckOffsetList().add(offset);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+- if (batchAckMsg.getAckOffsetList().isEmpty()) {
|
|||
|
|
++ if (rqId == KeyBuilder.POP_ORDER_REVIVE_QUEUE || batchAckMsg.getAckOffsetList().isEmpty()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -311,4 +278,46 @@ public class AckMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
+ PopMetricsManager.incPopReviveAckPutCount(ackMsg, putMessageResult.getPutMessageStatus());
|
|||
|
|
+ brokerController.getPopInflightMessageCounter().decrementInFlightMessageNum(topic, consumeGroup, popTime, qId, ackCount);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ protected void ackOrderly(String topic, String consumeGroup, int qId, long ackOffset, long popTime, long invisibleTime, Channel channel, RemotingCommand response) {
|
|||
|
|
++ String lockKey = topic + PopAckConstants.SPLIT + consumeGroup + PopAckConstants.SPLIT + qId;
|
|||
|
|
++ long oldOffset = this.brokerController.getConsumerOffsetManager().queryOffset(consumeGroup, topic, qId);
|
|||
|
|
++ if (ackOffset < oldOffset) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ while (!this.brokerController.getPopMessageProcessor().getQueueLockManager().tryLock(lockKey)) {
|
|||
|
|
++ }
|
|||
|
|
++ try {
|
|||
|
|
++ oldOffset = this.brokerController.getConsumerOffsetManager().queryOffset(consumeGroup, topic, qId);
|
|||
|
|
++ if (ackOffset < oldOffset) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ long nextOffset = brokerController.getConsumerOrderInfoManager().commitAndNext(
|
|||
|
|
++ topic, consumeGroup,
|
|||
|
|
++ qId, ackOffset,
|
|||
|
|
++ popTime);
|
|||
|
|
++ if (nextOffset > -1) {
|
|||
|
|
++ if (!this.brokerController.getConsumerOffsetManager().hasOffsetReset(
|
|||
|
|
++ topic, consumeGroup, qId)) {
|
|||
|
|
++ this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(),
|
|||
|
|
++ consumeGroup, topic, qId, nextOffset);
|
|||
|
|
++ }
|
|||
|
|
++ if (!this.brokerController.getConsumerOrderInfoManager().checkBlock(null, topic,
|
|||
|
|
++ consumeGroup, qId, invisibleTime)) {
|
|||
|
|
++ this.brokerController.getPopMessageProcessor().notifyMessageArriving(
|
|||
|
|
++ topic, consumeGroup, qId);
|
|||
|
|
++ }
|
|||
|
|
++ } else if (nextOffset == -1) {
|
|||
|
|
++ String errorInfo = String.format("offset is illegal, key:%s, old:%d, commit:%d, next:%d, %s",
|
|||
|
|
++ lockKey, oldOffset, ackOffset, nextOffset, channel.remoteAddress());
|
|||
|
|
++ POP_LOGGER.warn(errorInfo);
|
|||
|
|
++ response.setCode(ResponseCode.MESSAGE_ILLEGAL);
|
|||
|
|
++ response.setRemark(errorInfo);
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ } finally {
|
|||
|
|
++ this.brokerController.getPopMessageProcessor().getQueueLockManager().unLock(lockKey);
|
|||
|
|
++ }
|
|||
|
|
++ brokerController.getPopInflightMessageCounter().decrementInFlightMessageNum(topic, consumeGroup, popTime, qId, 1);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+index 5101ffc8e..213c26fd6 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+@@ -21,6 +21,7 @@ import java.io.UnsupportedEncodingException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
++import java.util.BitSet;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.Iterator;
|
|||
|
|
+@@ -54,6 +55,7 @@ import org.apache.rocketmq.client.producer.SendCallback;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendResult;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendStatus;
|
|||
|
|
+ import org.apache.rocketmq.common.AclConfig;
|
|||
|
|
++import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.MQVersion;
|
|||
|
|
+ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+@@ -76,7 +78,8 @@ import org.apache.rocketmq.common.namesrv.NameServerUpdateCallback;
|
|||
|
|
+ import org.apache.rocketmq.common.namesrv.TopAddressing;
|
|||
|
|
+ import org.apache.rocketmq.common.sysflag.PullSysFlag;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+-import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.CommandCustomHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+@@ -101,7 +104,10 @@ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.admin.ConsumeStats;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.admin.TopicStatsTable;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAck;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAckMessageRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.BrokerMemberGroup;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BrokerReplicasInfo;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.BrokerStatsData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.CheckClientRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.ClusterAclVersionInfo;
|
|||
|
|
+@@ -114,7 +120,6 @@ import org.apache.rocketmq.remoting.protocol.body.EpochEntryCache;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.GetConsumerStatusBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.GroupList;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.body.BrokerReplicasInfo;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.KVTable;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchResponseBody;
|
|||
|
|
+@@ -196,6 +201,10 @@ import org.apache.rocketmq.remoting.protocol.header.UpdateGlobalWhiteAddrsConfig
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.UpdateGroupForbiddenRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.ViewBrokerStatsDataRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.ViewMessageRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.admin.CleanControllerBrokerDataRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.AddWritePermOfBrokerRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.AddWritePermOfBrokerResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.DeleteKVConfigRequestHeader;
|
|||
|
|
+@@ -207,10 +216,6 @@ import org.apache.rocketmq.remoting.protocol.header.namesrv.GetRouteInfoRequestH
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.PutKVConfigRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.WipeWritePermOfBrokerRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.WipeWritePermOfBrokerResponseHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.admin.CleanControllerBrokerDataRequestHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterRequestHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.heartbeat.HeartbeatData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData;
|
|||
|
|
+@@ -221,8 +226,6 @@ import org.apache.rocketmq.remoting.protocol.subscription.GroupForbidden;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.rpchook.DynamicalExtFieldRPCHook;
|
|||
|
|
+ import org.apache.rocketmq.remoting.rpchook.StreamTypeRPCHook;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+ import static org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode.SUCCESS;
|
|||
|
|
+
|
|||
|
|
+@@ -885,9 +888,77 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ final String addr,
|
|||
|
|
+ final long timeOut,
|
|||
|
|
+ final AckCallback ackCallback,
|
|||
|
|
+- final AckMessageRequestHeader requestHeader //
|
|||
|
|
++ final AckMessageRequestHeader requestHeader
|
|||
|
|
++ ) throws RemotingException, MQBrokerException, InterruptedException {
|
|||
|
|
++ ackMessageAsync(addr, timeOut, ackCallback, requestHeader, null);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void batchAckMessageAsync(
|
|||
|
|
++ final String addr,
|
|||
|
|
++ final long timeOut,
|
|||
|
|
++ final AckCallback ackCallback,
|
|||
|
|
++ final String topic,
|
|||
|
|
++ final String consumerGroup,
|
|||
|
|
++ final List<String> extraInfoList
|
|||
|
|
++ ) throws RemotingException, MQBrokerException, InterruptedException {
|
|||
|
|
++ String brokerName = null;
|
|||
|
|
++ Map<String, BatchAck> batchAckMap = new HashMap<>();
|
|||
|
|
++ for (String extraInfo : extraInfoList) {
|
|||
|
|
++ String[] extraInfoData = ExtraInfoUtil.split(extraInfo);
|
|||
|
|
++ if (brokerName == null) {
|
|||
|
|
++ brokerName = ExtraInfoUtil.getBrokerName(extraInfoData);
|
|||
|
|
++ }
|
|||
|
|
++ String mergeKey = ExtraInfoUtil.getRetry(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getQueueId(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getCkQueueOffset(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getPopTime(extraInfoData);
|
|||
|
|
++ BatchAck bAck = batchAckMap.computeIfAbsent(mergeKey, k -> {
|
|||
|
|
++ BatchAck newBatchAck = new BatchAck();
|
|||
|
|
++ newBatchAck.setConsumerGroup(consumerGroup);
|
|||
|
|
++ newBatchAck.setTopic(topic);
|
|||
|
|
++ newBatchAck.setRetry(ExtraInfoUtil.getRetry(extraInfoData));
|
|||
|
|
++ newBatchAck.setStartOffset(ExtraInfoUtil.getCkQueueOffset(extraInfoData));
|
|||
|
|
++ newBatchAck.setQueueId(ExtraInfoUtil.getQueueId(extraInfoData));
|
|||
|
|
++ newBatchAck.setReviveQueueId(ExtraInfoUtil.getReviveQid(extraInfoData));
|
|||
|
|
++ newBatchAck.setPopTime(ExtraInfoUtil.getPopTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setInvisibleTime(ExtraInfoUtil.getInvisibleTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setBitSet(new BitSet());
|
|||
|
|
++ return newBatchAck;
|
|||
|
|
++ });
|
|||
|
|
++ bAck.getBitSet().set((int) (ExtraInfoUtil.getQueueOffset(extraInfoData) - ExtraInfoUtil.getCkQueueOffset(extraInfoData)));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ BatchAckMessageRequestBody requestBody = new BatchAckMessageRequestBody();
|
|||
|
|
++ requestBody.setBrokerName(brokerName);
|
|||
|
|
++ requestBody.setAcks(new ArrayList<>(batchAckMap.values()));
|
|||
|
|
++ batchAckMessageAsync(addr, timeOut, ackCallback, requestBody);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void batchAckMessageAsync(
|
|||
|
|
++ final String addr,
|
|||
|
|
++ final long timeOut,
|
|||
|
|
++ final AckCallback ackCallback,
|
|||
|
|
++ final BatchAckMessageRequestBody requestBody
|
|||
|
|
+ ) throws RemotingException, MQBrokerException, InterruptedException {
|
|||
|
|
+- final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.ACK_MESSAGE, requestHeader);
|
|||
|
|
++ ackMessageAsync(addr, timeOut, ackCallback, null, requestBody);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected void ackMessageAsync(
|
|||
|
|
++ final String addr,
|
|||
|
|
++ final long timeOut,
|
|||
|
|
++ final AckCallback ackCallback,
|
|||
|
|
++ final AckMessageRequestHeader requestHeader,
|
|||
|
|
++ final BatchAckMessageRequestBody requestBody
|
|||
|
|
++ ) throws RemotingException, MQBrokerException, InterruptedException {
|
|||
|
|
++ RemotingCommand request;
|
|||
|
|
++ if (requestHeader != null) {
|
|||
|
|
++ request = RemotingCommand.createRequestCommand(RequestCode.ACK_MESSAGE, requestHeader);
|
|||
|
|
++ } else {
|
|||
|
|
++ request = RemotingCommand.createRequestCommand(RequestCode.BATCH_ACK_MESSAGE, null);
|
|||
|
|
++ if (requestBody != null) {
|
|||
|
|
++ request.setBody(requestBody.encode());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+ this.remotingClient.invokeAsync(addr, request, timeOut, new BaseInvokeCallback(MQClientAPIImpl.this) {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/test/src/main/java/org/apache/rocketmq/test/client/rmq/RMQPopClient.java b/test/src/main/java/org/apache/rocketmq/test/client/rmq/RMQPopClient.java
|
|||
|
|
+index 496bd6da4..09c60c0b4 100644
|
|||
|
|
+--- a/test/src/main/java/org/apache/rocketmq/test/client/rmq/RMQPopClient.java
|
|||
|
|
++++ b/test/src/main/java/org/apache/rocketmq/test/client/rmq/RMQPopClient.java
|
|||
|
|
+@@ -17,6 +17,7 @@
|
|||
|
|
+
|
|||
|
|
+ package org.apache.rocketmq.test.client.rmq;
|
|||
|
|
+
|
|||
|
|
++import java.util.List;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckCallback;
|
|||
|
|
+@@ -140,6 +141,27 @@ public class RMQPopClient implements MQConsumer {
|
|||
|
|
+ return future;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessageAsync(String brokerAddr, String topic, String consumerGroup,
|
|||
|
|
++ List<String> extraInfoList) {
|
|||
|
|
++ CompletableFuture<AckResult> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ this.mqClientAPI.batchAckMessageAsync(brokerAddr, DEFAULT_TIMEOUT, new AckCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onSuccess(AckResult ackResult) {
|
|||
|
|
++ future.complete(ackResult);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onException(Throwable e) {
|
|||
|
|
++ future.completeExceptionally(e);
|
|||
|
|
++ }
|
|||
|
|
++ }, topic, consumerGroup, extraInfoList);
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTimeAsync(String brokerAddr, String brokerName, String topic,
|
|||
|
|
+ String consumerGroup, String extraInfo, long invisibleTime) {
|
|||
|
|
+ String[] extraInfoStrs = ExtraInfoUtil.split(extraInfo);
|
|||
|
|
+diff --git a/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BasePopNormally.java b/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BasePopNormally.java
|
|||
|
|
+index 952fbe3f5..2e29b95a5 100644
|
|||
|
|
+--- a/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BasePopNormally.java
|
|||
|
|
++++ b/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BasePopNormally.java
|
|||
|
|
+@@ -63,4 +63,10 @@ public class BasePopNormally extends BasePop {
|
|||
|
|
+ brokerAddr, messageQueue, invisibleTime, maxNums, group, timeout, true,
|
|||
|
|
+ ConsumeInitMode.MIN, false, ExpressionType.TAG, "*");
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<PopResult> popMessageAsync(long invisibleTime, int maxNums) {
|
|||
|
|
++ return client.popMessageAsync(
|
|||
|
|
++ brokerAddr, messageQueue, invisibleTime, maxNums, group, 3000, false,
|
|||
|
|
++ ConsumeInitMode.MIN, false, ExpressionType.TAG, "*");
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BatchAckIT.java b/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BatchAckIT.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..ec9153ccc
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/test/src/test/java/org/apache/rocketmq/test/client/consumer/pop/BatchAckIT.java
|
|||
|
|
+@@ -0,0 +1,159 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.test.client.consumer.pop;
|
|||
|
|
++
|
|||
|
|
++import java.time.Duration;
|
|||
|
|
++import java.util.ArrayList;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++import java.util.concurrent.CompletableFuture;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
++import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
++import java.util.function.Supplier;
|
|||
|
|
++import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
++import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
++import org.apache.rocketmq.client.consumer.PopResult;
|
|||
|
|
++import org.apache.rocketmq.client.consumer.PopStatus;
|
|||
|
|
++import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
++import org.apache.rocketmq.common.attribute.TopicMessageType;
|
|||
|
|
++import org.apache.rocketmq.common.constant.ConsumeInitMode;
|
|||
|
|
++import org.apache.rocketmq.common.filter.ExpressionType;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
++import org.apache.rocketmq.test.base.IntegrationTestBase;
|
|||
|
|
++import org.apache.rocketmq.test.client.rmq.RMQNormalProducer;
|
|||
|
|
++import org.apache.rocketmq.test.client.rmq.RMQPopClient;
|
|||
|
|
++import org.apache.rocketmq.test.util.MQRandomUtils;
|
|||
|
|
++import org.junit.After;
|
|||
|
|
++import org.junit.Before;
|
|||
|
|
++import org.junit.Test;
|
|||
|
|
++
|
|||
|
|
++import static org.awaitility.Awaitility.await;
|
|||
|
|
++import static org.junit.Assert.assertEquals;
|
|||
|
|
++
|
|||
|
|
++public class BatchAckIT extends BasePop {
|
|||
|
|
++
|
|||
|
|
++ protected String topic;
|
|||
|
|
++ protected String group;
|
|||
|
|
++ protected RMQNormalProducer producer = null;
|
|||
|
|
++ protected RMQPopClient client = null;
|
|||
|
|
++ protected String brokerAddr;
|
|||
|
|
++ protected MessageQueue messageQueue;
|
|||
|
|
++
|
|||
|
|
++ @Before
|
|||
|
|
++ public void setUp() {
|
|||
|
|
++ brokerAddr = brokerController1.getBrokerAddr();
|
|||
|
|
++ topic = MQRandomUtils.getRandomTopic();
|
|||
|
|
++ group = initConsumerGroup();
|
|||
|
|
++ IntegrationTestBase.initTopic(topic, NAMESRV_ADDR, BROKER1_NAME, 8, CQType.SimpleCQ, TopicMessageType.NORMAL);
|
|||
|
|
++ producer = getProducer(NAMESRV_ADDR, topic);
|
|||
|
|
++ client = getRMQPopClient();
|
|||
|
|
++ messageQueue = new MessageQueue(topic, BROKER1_NAME, -1);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @After
|
|||
|
|
++ public void tearDown() {
|
|||
|
|
++ shutdown();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckNormallyWithPopBuffer() throws Throwable {
|
|||
|
|
++ brokerController1.getBrokerConfig().setEnablePopBufferMerge(true);
|
|||
|
|
++ brokerController2.getBrokerConfig().setEnablePopBufferMerge(true);
|
|||
|
|
++
|
|||
|
|
++ testBatchAck(() -> {
|
|||
|
|
++ try {
|
|||
|
|
++ return popMessageAsync().get();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ throw new RuntimeException(e);
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckNormallyWithOutPopBuffer() throws Throwable {
|
|||
|
|
++ brokerController1.getBrokerConfig().setEnablePopBufferMerge(false);
|
|||
|
|
++ brokerController2.getBrokerConfig().setEnablePopBufferMerge(false);
|
|||
|
|
++
|
|||
|
|
++ testBatchAck(() -> {
|
|||
|
|
++ try {
|
|||
|
|
++ return popMessageAsync().get();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ throw new RuntimeException(e);
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckOrderly() throws Throwable {
|
|||
|
|
++ testBatchAck(() -> {
|
|||
|
|
++ try {
|
|||
|
|
++ return popMessageOrderlyAsync().get();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ throw new RuntimeException(e);
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void testBatchAck(Supplier<PopResult> popResultSupplier) throws Throwable {
|
|||
|
|
++ // Send 10 messages but do not ack, let them enter the retry topic
|
|||
|
|
++ producer.send(10);
|
|||
|
|
++ AtomicInteger firstMsgRcvNum = new AtomicInteger();
|
|||
|
|
++ await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> {
|
|||
|
|
++ PopResult popResult = popResultSupplier.get();
|
|||
|
|
++ if (popResult.getPopStatus().equals(PopStatus.FOUND)) {
|
|||
|
|
++ firstMsgRcvNum.addAndGet(popResult.getMsgFoundList().size());
|
|||
|
|
++ }
|
|||
|
|
++ assertEquals(10, firstMsgRcvNum.get());
|
|||
|
|
++ });
|
|||
|
|
++ // sleep 6s, expect messages to enter the retry topic
|
|||
|
|
++ TimeUnit.SECONDS.sleep(6);
|
|||
|
|
++
|
|||
|
|
++ producer.send(20);
|
|||
|
|
++ List<String> extraInfoList = new ArrayList<>();
|
|||
|
|
++ await().atMost(Duration.ofSeconds(3)).untilAsserted(() -> {
|
|||
|
|
++ PopResult popResult = popResultSupplier.get();
|
|||
|
|
++ if (popResult.getPopStatus().equals(PopStatus.FOUND)) {
|
|||
|
|
++ for (MessageExt messageExt : popResult.getMsgFoundList()) {
|
|||
|
|
++ extraInfoList.add(messageExt.getProperty(MessageConst.PROPERTY_POP_CK));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ assertEquals(30, extraInfoList.size());
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ AckResult ackResult = client.batchAckMessageAsync(brokerAddr, topic, group, extraInfoList).get();
|
|||
|
|
++ assertEquals(AckStatus.OK, ackResult.getStatus());
|
|||
|
|
++
|
|||
|
|
++ // sleep 6s, expected that messages that have been acked will not be re-consumed
|
|||
|
|
++ TimeUnit.SECONDS.sleep(6);
|
|||
|
|
++ PopResult popResult = popResultSupplier.get();
|
|||
|
|
++ assertEquals(PopStatus.POLLING_NOT_FOUND, popResult.getPopStatus());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private CompletableFuture<PopResult> popMessageAsync() {
|
|||
|
|
++ return client.popMessageAsync(
|
|||
|
|
++ brokerAddr, messageQueue, Duration.ofSeconds(3).toMillis(), 30, group, 3000, false,
|
|||
|
|
++ ConsumeInitMode.MIN, false, ExpressionType.TAG, "*");
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private CompletableFuture<PopResult> popMessageOrderlyAsync() {
|
|||
|
|
++ return client.popMessageAsync(
|
|||
|
|
++ brokerAddr, messageQueue, Duration.ofSeconds(3).toMillis(), 30, group, 3000, false,
|
|||
|
|
++ ConsumeInitMode.MIN, true, ExpressionType.TAG, "*", null);
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From cc16a1b51216e1e80c22011b8b01e060bb4af8b3 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Tue, 22 Aug 2023 10:42:25 +0800
|
|||
|
|
+Subject: [PATCH 7/7] Set table reference the same object for
|
|||
|
|
+ setSubscriptionGroupTable method (#7204)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/subscription/SubscriptionGroupManager.java | 5 +----
|
|||
|
|
+ 1 file changed, 1 insertion(+), 4 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java
|
|||
|
|
+index 74e39c0fe..e63b93058 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManager.java
|
|||
|
|
+@@ -341,10 +341,7 @@ public class SubscriptionGroupManager extends ConfigManager {
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+ public void setSubscriptionGroupTable(ConcurrentMap<String, SubscriptionGroupConfig> subscriptionGroupTable) {
|
|||
|
|
+- this.subscriptionGroupTable.clear();
|
|||
|
|
+- for (String key : subscriptionGroupTable.keySet()) {
|
|||
|
|
+- putSubscriptionGroupConfig(subscriptionGroupTable.get(key));
|
|||
|
|
+- }
|
|||
|
|
++ this.subscriptionGroupTable = subscriptionGroupTable;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean containsSubscriptionGroup(String group) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch011-backport-optimize-opentelemetry-metric-config.patch b/patch011-backport-optimize-opentelemetry-metric-config.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..b59d7e2c7
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch011-backport-optimize-opentelemetry-metric-config.patch
|
|||
|
|
@@ -0,0 +1,2081 @@
|
|||
|
|
+From 744167bd01fab6821b4d5ae1794dc845153d5156 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <tanziyi0925@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 08:32:17 +0800
|
|||
|
|
+Subject: [PATCH 1/7] [ISSUE #7142] Add command `RocksDBConfigToJson` to
|
|||
|
|
+ inspect rocksdb content (#7180)
|
|||
|
|
+
|
|||
|
|
+* feat: add command `RocksDBConfigToJson` to inspect rocksdb content
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+
|
|||
|
|
+* refactor: fix style
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+Co-authored-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../tools/command/MQAdminStartup.java | 2 +
|
|||
|
|
+ .../metadata/RocksDBConfigToJsonCommand.java | 118 ++++++++++++++++++
|
|||
|
|
+ .../metadata/KvConfigToJsonCommandTest.java | 65 ++++++++++
|
|||
|
|
+ 3 files changed, 185 insertions(+)
|
|||
|
|
+ create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+ create mode 100644 tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+index 890125ca0..324aa1856 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+@@ -80,6 +80,7 @@ import org.apache.rocketmq.tools.command.message.QueryMsgByOffsetSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgByUniqueKeySubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgTraceByIdSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.SendMessageCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.metadata.RocksDBConfigToJsonCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.AddWritePermSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.DeleteKvConfigCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.GetNamesrvConfigCommand;
|
|||
|
|
+@@ -211,6 +212,7 @@ public class MQAdminStartup {
|
|||
|
|
+
|
|||
|
|
+ initCommand(new ClusterListSubCommand());
|
|||
|
|
+ initCommand(new TopicListSubCommand());
|
|||
|
|
++ initCommand(new RocksDBConfigToJsonCommand());
|
|||
|
|
+
|
|||
|
|
+ initCommand(new UpdateKvConfigCommand());
|
|||
|
|
+ initCommand(new DeleteKvConfigCommand());
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..3053f4684
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+@@ -0,0 +1,118 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tools.command.metadata;
|
|||
|
|
++
|
|||
|
|
++import com.alibaba.fastjson.JSONObject;
|
|||
|
|
++import org.apache.commons.cli.CommandLine;
|
|||
|
|
++import org.apache.commons.cli.Option;
|
|||
|
|
++import org.apache.commons.cli.Options;
|
|||
|
|
++import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.common.config.RocksDBConfigManager;
|
|||
|
|
++import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
++import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++
|
|||
|
|
++import java.io.File;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
++
|
|||
|
|
++public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
++ private static final String TOPICS_JSON_CONFIG = "topics";
|
|||
|
|
++ private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups";
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandName() {
|
|||
|
|
++ return "rocksDBConfigToJson";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandDesc() {
|
|||
|
|
++ return "Convert RocksDB kv config (topics/subscriptionGroups) to json";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Options buildCommandlineOptions(Options options) {
|
|||
|
|
++ Option pathOption = new Option("p", "path", true,
|
|||
|
|
++ "Absolute path to the metadata directory");
|
|||
|
|
++ pathOption.setRequired(true);
|
|||
|
|
++ options.addOption(pathOption);
|
|||
|
|
++
|
|||
|
|
++ Option configTypeOption = new Option("t", "configType", true, "Name of kv config, e.g. " +
|
|||
|
|
++ "topics/subscriptionGroups");
|
|||
|
|
++ configTypeOption.setRequired(true);
|
|||
|
|
++ options.addOption(configTypeOption);
|
|||
|
|
++
|
|||
|
|
++ return options;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
++ String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
++ if (StringUtils.isEmpty(path) || !new File(path).exists()) {
|
|||
|
|
++ System.out.print("Rocksdb path is invalid.\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
++
|
|||
|
|
++ final long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
++ RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(memTableFlushInterval);
|
|||
|
|
++ try {
|
|||
|
|
++ if (TOPICS_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
++ // for topics.json
|
|||
|
|
++ final Map<String, JSONObject> topicsJsonConfig = new HashMap<>();
|
|||
|
|
++ final Map<String, JSONObject> topicConfigTable = new HashMap<>();
|
|||
|
|
++ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
++ final String topic = new String(key, DataConverter.charset);
|
|||
|
|
++ final String topicConfig = new String(value, DataConverter.charset);
|
|||
|
|
++ final JSONObject jsonObject = JSONObject.parseObject(topicConfig);
|
|||
|
|
++ topicConfigTable.put(topic, jsonObject);
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ if (isLoad) {
|
|||
|
|
++ topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
++ final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
++ System.out.print(topicsJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ if (SUBSCRIPTION_GROUP_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
++ // for subscriptionGroup.json
|
|||
|
|
++ final Map<String, JSONObject> subscriptionGroupJsonConfig = new HashMap<>();
|
|||
|
|
++ final Map<String, JSONObject> subscriptionGroupTable = new HashMap<>();
|
|||
|
|
++ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
++ final String subscriptionGroup = new String(key, DataConverter.charset);
|
|||
|
|
++ final String subscriptionGroupConfig = new String(value, DataConverter.charset);
|
|||
|
|
++ final JSONObject jsonObject = JSONObject.parseObject(subscriptionGroupConfig);
|
|||
|
|
++ subscriptionGroupTable.put(subscriptionGroup, jsonObject);
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ if (isLoad) {
|
|||
|
|
++ subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
++ (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
++ final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
++ System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ System.out.print("Config type was not recognized, configType=" + configType + "\n");
|
|||
|
|
++ } finally {
|
|||
|
|
++ kvConfigManager.stop();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..b2f66c7b0
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+@@ -0,0 +1,65 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tools.command.metadata;
|
|||
|
|
++
|
|||
|
|
++import org.apache.commons.cli.CommandLine;
|
|||
|
|
++import org.apache.commons.cli.DefaultParser;
|
|||
|
|
++import org.apache.commons.cli.Options;
|
|||
|
|
++import org.apache.rocketmq.srvutil.ServerUtil;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++import org.junit.Test;
|
|||
|
|
++
|
|||
|
|
++import java.io.File;
|
|||
|
|
++
|
|||
|
|
++import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
++
|
|||
|
|
++public class KvConfigToJsonCommandTest {
|
|||
|
|
++ private static final String BASE_PATH = System.getProperty("user.home") + File.separator + "store/config/";
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testExecute() throws SubCommandException {
|
|||
|
|
++ {
|
|||
|
|
++ String[] cases = new String[]{"topics", "subscriptionGroups"};
|
|||
|
|
++ for (String c : cases) {
|
|||
|
|
++ RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
++ String[] subargs = new String[]{"-p " + BASE_PATH + c, "-t " + c};
|
|||
|
|
++ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs,
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.execute(commandLine, options, null);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("p").trim()).isEqualTo(BASE_PATH + c);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("t").trim()).isEqualTo(c);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ // invalid cases
|
|||
|
|
++ {
|
|||
|
|
++ String[][] cases = new String[][]{
|
|||
|
|
++ {"-p " + BASE_PATH + "tmpPath", "-t topics"},
|
|||
|
|
++ {"-p ", "-t topics"},
|
|||
|
|
++ {"-p " + BASE_PATH + "topics", "-t invalid_type"}
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ for (String[] c : cases) {
|
|||
|
|
++ RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
++ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), c,
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.execute(commandLine, options, null);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From bdede35db365a49b211cdc249c68b0f60a3df46d Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: mxsm <ljbmxsm@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 08:34:56 +0800
|
|||
|
|
+Subject: [PATCH 2/7] [ISSUE #7124] Fix the typos in the code comments (#7125)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/broker/processor/ReplyMessageProcessor.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
+index b2db356c8..d3bb048f7 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
+@@ -234,7 +234,7 @@ public class ReplyMessageProcessor extends AbstractSendMessageProcessor {
|
|||
|
|
+ } else {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ response.setRemark(null);
|
|||
|
|
+- //set to zore to avoid client decoding exception
|
|||
|
|
++ //set to zero to avoid client decoding exception
|
|||
|
|
+ responseHeader.setMsgId("0");
|
|||
|
|
+ responseHeader.setQueueId(queueIdInt);
|
|||
|
|
+ responseHeader.setQueueOffset(0L);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 9bb73b9a38548b99ac5126c40380c3c2e7fc586e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 09:46:27 +0800
|
|||
|
|
+Subject: [PATCH 3/7] [#ISSUE 7222] Bug fix and refactoring of the Indexfile in
|
|||
|
|
+ tiered storage (#7224)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/file/TieredIndexFile.java | 38 +++++++--
|
|||
|
|
+ .../tieredstore/file/TieredIndexFileTest.java | 84 +++++--------------
|
|||
|
|
+ 2 files changed, 52 insertions(+), 70 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
+index 50beb01ae..eda5e0106 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
+@@ -16,6 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
++import com.google.common.annotations.VisibleForTesting;
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+@@ -99,7 +100,7 @@ public class TieredIndexFile {
|
|||
|
|
+ this::doScheduleTask, 10, 10, TimeUnit.SECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private void doScheduleTask() {
|
|||
|
|
++ protected void doScheduleTask() {
|
|||
|
|
+ try {
|
|||
|
|
+ curFileLock.lock();
|
|||
|
|
+ try {
|
|||
|
|
+@@ -145,6 +146,11 @@ public class TieredIndexFile {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @VisibleForTesting
|
|||
|
|
++ public MappedFile getPreMappedFile() {
|
|||
|
|
++ return preMappedFile;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private void initFile() throws IOException {
|
|||
|
|
+ curMappedFile = new DefaultMappedFile(curFilePath, fileMaxSize);
|
|||
|
|
+ initIndexFileHeader(curMappedFile);
|
|||
|
|
+@@ -156,19 +162,26 @@ public class TieredIndexFile {
|
|||
|
|
+
|
|||
|
|
+ if (isFileSealed(curMappedFile)) {
|
|||
|
|
+ if (preFileExists) {
|
|||
|
|
+- preFile.delete();
|
|||
|
|
++ if (preFile.delete()) {
|
|||
|
|
++ logger.info("Pre IndexFile deleted success", preFilepath);
|
|||
|
|
++ } else {
|
|||
|
|
++ logger.error("Pre IndexFile deleted failed", preFilepath);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ boolean rename = curMappedFile.renameTo(preFilepath);
|
|||
|
|
+ if (rename) {
|
|||
|
|
+ preMappedFile = curMappedFile;
|
|||
|
|
+ curMappedFile = new DefaultMappedFile(curFilePath, fileMaxSize);
|
|||
|
|
++ initIndexFileHeader(curMappedFile);
|
|||
|
|
+ preFileExists = true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ if (preFileExists) {
|
|||
|
|
+ synchronized (TieredIndexFile.class) {
|
|||
|
|
+ if (inflightCompactFuture.isDone()) {
|
|||
|
|
+- inflightCompactFuture = TieredStoreExecutor.compactIndexFileExecutor.submit(new CompactTask(storeConfig, preMappedFile, flatFile), null);
|
|||
|
|
++ inflightCompactFuture = TieredStoreExecutor.compactIndexFileExecutor.submit(
|
|||
|
|
++ new CompactTask(storeConfig, preMappedFile, flatFile), null);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -261,7 +274,8 @@ public class TieredIndexFile {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public CompletableFuture<List<Pair<Long, ByteBuffer>>> queryAsync(String topic, String key, long beginTime, long endTime) {
|
|||
|
|
++ public CompletableFuture<List<Pair<Long, ByteBuffer>>> queryAsync(String topic, String key, long beginTime,
|
|||
|
|
++ long endTime) {
|
|||
|
|
+ int hashCode = indexKeyHashMethod(buildKey(topic, key));
|
|||
|
|
+ int slotPosition = hashCode % maxHashSlotNum;
|
|||
|
|
+ List<TieredFileSegment> fileSegmentList = flatFile.getFileListByTime(beginTime, endTime);
|
|||
|
|
+@@ -355,7 +369,7 @@ public class TieredIndexFile {
|
|||
|
|
+ private final int fileMaxSize;
|
|||
|
|
+ private MappedFile originFile;
|
|||
|
|
+ private TieredFlatFile fileQueue;
|
|||
|
|
+- private final MappedFile compactFile;
|
|||
|
|
++ private MappedFile compactFile;
|
|||
|
|
+
|
|||
|
|
+ public CompactTask(TieredMessageStoreConfig storeConfig, MappedFile originFile,
|
|||
|
|
+ TieredFlatFile fileQueue) throws IOException {
|
|||
|
|
+@@ -381,6 +395,17 @@ public class TieredIndexFile {
|
|||
|
|
+ } catch (Throwable throwable) {
|
|||
|
|
+ logger.error("TieredIndexFile#compactTask: compact index file failed:", throwable);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ try {
|
|||
|
|
++ if (originFile != null) {
|
|||
|
|
++ originFile.destroy(-1);
|
|||
|
|
++ }
|
|||
|
|
++ if (compactFile != null) {
|
|||
|
|
++ compactFile.destroy(-1);
|
|||
|
|
++ }
|
|||
|
|
++ } catch (Throwable throwable) {
|
|||
|
|
++ logger.error("TieredIndexFile#compactTask: destroy index file failed:", throwable);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void compact() {
|
|||
|
|
+@@ -396,6 +421,8 @@ public class TieredIndexFile {
|
|||
|
|
+ fileQueue.commit(true);
|
|||
|
|
+ compactFile.destroy(-1);
|
|||
|
|
+ originFile.destroy(-1);
|
|||
|
|
++ compactFile = null;
|
|||
|
|
++ originFile = null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCompactFile() {
|
|||
|
|
+@@ -414,6 +441,7 @@ public class TieredIndexFile {
|
|||
|
|
+ if (slotValue != -1) {
|
|||
|
|
+ int indexTotalSize = 0;
|
|||
|
|
+ int indexPosition = slotValue;
|
|||
|
|
++
|
|||
|
|
+ while (indexPosition >= 0 && indexPosition < maxIndexNum) {
|
|||
|
|
+ int indexOffset = INDEX_FILE_HEADER_SIZE + maxHashSlotNum * INDEX_FILE_HASH_SLOT_SIZE
|
|||
|
|
+ + indexPosition * INDEX_FILE_HASH_ORIGIN_INDEX_SIZE;
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+index 7ef49578d..262d6645b 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+@@ -19,9 +19,8 @@ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+ import com.sun.jna.Platform;
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
++import java.time.Duration;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-import org.apache.commons.lang3.SystemUtils;
|
|||
|
|
+ import org.apache.commons.lang3.tuple.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.TieredStoreTestUtil;
|
|||
|
|
+@@ -31,9 +30,7 @@ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+ import org.awaitility.Awaitility;
|
|||
|
|
+ import org.junit.After;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+-import org.junit.Assume;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+-import org.junit.Ignore;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ public class TieredIndexFileTest {
|
|||
|
|
+@@ -45,11 +42,12 @@ public class TieredIndexFileTest {
|
|||
|
|
+ @Before
|
|||
|
|
+ public void setUp() {
|
|||
|
|
+ storeConfig = new TieredMessageStoreConfig();
|
|||
|
|
++ storeConfig.setBrokerName("IndexFileBroker");
|
|||
|
|
+ storeConfig.setStorePathRootDir(storePath);
|
|||
|
|
+- storeConfig.setTieredBackendServiceProvider("org.apache.rocketmq.tieredstore.provider.memory.MemoryFileSegment");
|
|||
|
|
+- storeConfig.setTieredStoreIndexFileMaxHashSlotNum(2);
|
|||
|
|
+- storeConfig.setTieredStoreIndexFileMaxIndexNum(3);
|
|||
|
|
+- mq = new MessageQueue("TieredIndexFileTest", storeConfig.getBrokerName(), 1);
|
|||
|
|
++ storeConfig.setTieredBackendServiceProvider("org.apache.rocketmq.tieredstore.provider.posix.PosixFileSegment");
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileMaxHashSlotNum(5);
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileMaxIndexNum(20);
|
|||
|
|
++ mq = new MessageQueue("IndexFileTest", storeConfig.getBrokerName(), 1);
|
|||
|
|
+ TieredStoreUtil.getMetadataStore(storeConfig);
|
|||
|
|
+ TieredStoreExecutor.init();
|
|||
|
|
+ }
|
|||
|
|
+@@ -61,77 +59,33 @@ public class TieredIndexFileTest {
|
|||
|
|
+ TieredStoreExecutor.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Ignore
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testAppendAndQuery() throws IOException, ClassNotFoundException, NoSuchMethodException {
|
|||
|
|
+ if (Platform.isWindows()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // skip this test on windows
|
|||
|
|
+- Assume.assumeFalse(SystemUtils.IS_OS_WINDOWS);
|
|||
|
|
+-
|
|||
|
|
+ TieredFileAllocator fileQueueFactory = new TieredFileAllocator(storeConfig);
|
|||
|
|
+ TieredIndexFile indexFile = new TieredIndexFile(fileQueueFactory, storePath);
|
|||
|
|
++
|
|||
|
|
+ indexFile.append(mq, 0, "key3", 3, 300, 1000);
|
|||
|
|
+ indexFile.append(mq, 0, "key2", 2, 200, 1100);
|
|||
|
|
+ indexFile.append(mq, 0, "key1", 1, 100, 1200);
|
|||
|
|
+
|
|||
|
|
+- Awaitility.waitAtMost(5, TimeUnit.SECONDS)
|
|||
|
|
+- .until(() -> {
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
+- if (indexList.size() != 1) {
|
|||
|
|
+- return false;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 2, indexBuffer.remaining());
|
|||
|
|
+-
|
|||
|
|
+- Assert.assertEquals(1, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(100, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(200, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
+-
|
|||
|
|
+- Assert.assertEquals(3, indexBuffer.getLong(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(300, indexBuffer.getInt(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4 + 8 + 4));
|
|||
|
|
+- return true;
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+-
|
|||
|
|
+- Awaitility.waitAtMost(5, TimeUnit.SECONDS)
|
|||
|
|
+- .until(() -> {
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key4", 1300, 1300).join();
|
|||
|
|
+- if (indexList.size() != 1) {
|
|||
|
|
+- return false;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 3, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(4, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(400, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
+- return true;
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1300, 1300).join();
|
|||
|
|
++ // do not do schedule task here
|
|||
|
|
++ TieredStoreExecutor.shutdown();
|
|||
|
|
++ List<Pair<Long, ByteBuffer>> indexList =
|
|||
|
|
++ indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
+ Assert.assertEquals(0, indexList.size());
|
|||
|
|
+
|
|||
|
|
+- indexList = indexFile.queryAsync(mq.getTopic(), "key4", 1200, 1300).join();
|
|||
|
|
+- Assert.assertEquals(2, indexList.size());
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 3, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(4, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(400, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
++ // do compaction once
|
|||
|
|
++ TieredStoreExecutor.init();
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileRollingIdleInterval(0);
|
|||
|
|
++ indexFile.doScheduleTask();
|
|||
|
|
++ Awaitility.await().atMost(Duration.ofSeconds(10))
|
|||
|
|
++ .until(() -> !indexFile.getPreMappedFile().getFile().exists());
|
|||
|
|
+
|
|||
|
|
+- indexBuffer = indexList.get(1).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(2, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(200, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(100, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
++ indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
++ Assert.assertEquals(1, indexList.size());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 69c26d3d29cde7b4484ecd112ab9224f9f42bf45 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 10:27:52 +0800
|
|||
|
|
+Subject: [PATCH 4/7] [ISSUE #7228] Converge the use of some important
|
|||
|
|
+ variables for some class
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/store/ConsumeQueue.java | 16 ++++++------
|
|||
|
|
+ .../rocketmq/store/MappedFileQueue.java | 26 +++++++++++--------
|
|||
|
|
+ .../store/MultiPathMappedFileQueue.java | 4 +--
|
|||
|
|
+ 3 files changed, 24 insertions(+), 22 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
+index a0b886eb0..56bee2af3 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
+@@ -145,7 +145,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ if (offset >= 0 && size > 0) {
|
|||
|
|
+ mappedFileOffset = i + CQ_STORE_UNIT_SIZE;
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+ }
|
|||
|
|
+@@ -409,7 +409,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ int logicFileSize = this.mappedFileSize;
|
|||
|
|
+
|
|||
|
|
+- this.maxPhysicOffset = phyOffset;
|
|||
|
|
++ this.setMaxPhysicOffset(phyOffset);
|
|||
|
|
+ long maxExtAddr = 1;
|
|||
|
|
+ boolean shouldDeleteFile = false;
|
|||
|
|
+ while (true) {
|
|||
|
|
+@@ -435,7 +435,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ mappedFile.setWrotePosition(pos);
|
|||
|
|
+ mappedFile.setCommittedPosition(pos);
|
|||
|
|
+ mappedFile.setFlushedPosition(pos);
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ // This maybe not take effect, when not every consume queue has extend file.
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+@@ -453,7 +453,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ mappedFile.setWrotePosition(pos);
|
|||
|
|
+ mappedFile.setCommittedPosition(pos);
|
|||
|
|
+ mappedFile.setFlushedPosition(pos);
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+ }
|
|||
|
|
+@@ -881,8 +881,8 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode,
|
|||
|
|
+ final long cqOffset) {
|
|||
|
|
+
|
|||
|
|
+- if (offset + size <= this.maxPhysicOffset) {
|
|||
|
|
+- log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", maxPhysicOffset, offset);
|
|||
|
|
++ if (offset + size <= this.getMaxPhysicOffset()) {
|
|||
|
|
++ log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", this.getMaxPhysicOffset(), offset);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -926,7 +926,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ );
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ return mappedFile.appendMessage(this.byteBufferIndex.array());
|
|||
|
|
+ }
|
|||
|
|
+ return false;
|
|||
|
|
+@@ -1130,7 +1130,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void destroy() {
|
|||
|
|
+- this.maxPhysicOffset = -1;
|
|||
|
|
++ this.setMaxPhysicOffset(-1);
|
|||
|
|
+ this.minLogicOffset = 0;
|
|||
|
|
+ this.mappedFileQueue.destroy();
|
|||
|
|
+ if (isExtReadEnable()) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+index 0bc70642f..32b90d14f 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+@@ -285,7 +285,7 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ if (this.mappedFiles.isEmpty())
|
|||
|
|
+ return 0;
|
|||
|
|
+
|
|||
|
|
+- long committed = this.flushedWhere;
|
|||
|
|
++ long committed = this.getFlushedWhere();
|
|||
|
|
+ if (committed != 0) {
|
|||
|
|
+ MappedFile mappedFile = this.getLastMappedFile(0, false);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+@@ -442,11 +442,11 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long remainHowManyDataToCommit() {
|
|||
|
|
+- return getMaxWrotePosition() - committedWhere;
|
|||
|
|
++ return getMaxWrotePosition() - getCommittedWhere();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long remainHowManyDataToFlush() {
|
|||
|
|
+- return getMaxOffset() - flushedWhere;
|
|||
|
|
++ return getMaxOffset() - this.getFlushedWhere();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void deleteLastMappedFile() {
|
|||
|
|
+@@ -616,15 +616,15 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+
|
|||
|
|
+ public boolean flush(final int flushLeastPages) {
|
|||
|
|
+ boolean result = true;
|
|||
|
|
+- MappedFile mappedFile = this.findMappedFileByOffset(this.flushedWhere, this.flushedWhere == 0);
|
|||
|
|
++ MappedFile mappedFile = this.findMappedFileByOffset(this.getFlushedWhere(), this.getFlushedWhere() == 0);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+ long tmpTimeStamp = mappedFile.getStoreTimestamp();
|
|||
|
|
+ int offset = mappedFile.flush(flushLeastPages);
|
|||
|
|
+ long where = mappedFile.getFileFromOffset() + offset;
|
|||
|
|
+- result = where == this.flushedWhere;
|
|||
|
|
+- this.flushedWhere = where;
|
|||
|
|
++ result = where == this.getFlushedWhere();
|
|||
|
|
++ this.setFlushedWhere(where);
|
|||
|
|
+ if (0 == flushLeastPages) {
|
|||
|
|
+- this.storeTimestamp = tmpTimeStamp;
|
|||
|
|
++ this.setStoreTimestamp(tmpTimeStamp);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -633,12 +633,12 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+
|
|||
|
|
+ public synchronized boolean commit(final int commitLeastPages) {
|
|||
|
|
+ boolean result = true;
|
|||
|
|
+- MappedFile mappedFile = this.findMappedFileByOffset(this.committedWhere, this.committedWhere == 0);
|
|||
|
|
++ MappedFile mappedFile = this.findMappedFileByOffset(this.getCommittedWhere(), this.getCommittedWhere() == 0);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+ int offset = mappedFile.commit(commitLeastPages);
|
|||
|
|
+ long where = mappedFile.getFileFromOffset() + offset;
|
|||
|
|
+- result = where == this.committedWhere;
|
|||
|
|
+- this.committedWhere = where;
|
|||
|
|
++ result = where == this.getCommittedWhere();
|
|||
|
|
++ this.setCommittedWhere(where);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return result;
|
|||
|
|
+@@ -763,7 +763,7 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ mf.destroy(1000 * 3);
|
|||
|
|
+ }
|
|||
|
|
+ this.mappedFiles.clear();
|
|||
|
|
+- this.flushedWhere = 0;
|
|||
|
|
++ this.setFlushedWhere(0);
|
|||
|
|
+
|
|||
|
|
+ // delete parent directory
|
|||
|
|
+ File file = new File(storePath);
|
|||
|
|
+@@ -848,6 +848,10 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ return storeTimestamp;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void setStoreTimestamp(long storeTimestamp) {
|
|||
|
|
++ this.storeTimestamp = storeTimestamp;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public List<MappedFile> getMappedFiles() {
|
|||
|
|
+ return mappedFiles;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
+index 8f5af9438..8ff050dfe 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
+@@ -16,7 +16,6 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.store;
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+@@ -113,8 +112,7 @@ public class MultiPathMappedFileQueue extends MappedFileQueue {
|
|||
|
|
+ mf.destroy(1000 * 3);
|
|||
|
|
+ }
|
|||
|
|
+ this.mappedFiles.clear();
|
|||
|
|
+- this.flushedWhere = 0;
|
|||
|
|
+-
|
|||
|
|
++ this.setFlushedWhere(0);
|
|||
|
|
+
|
|||
|
|
+ Set<String> storePathSet = getPaths();
|
|||
|
|
+ storePathSet.addAll(getReadonlyPaths());
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3884f595949462044c5cb3c236199bc1d7ad2341 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: =?UTF-8?q?=E7=9F=B3=E8=87=BB=E8=87=BB=28Steven=20shi=29?=
|
|||
|
|
+ <shirenchuang@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 11:10:30 +0800
|
|||
|
|
+Subject: [PATCH 5/7] [ISSUE #7149] When creating and updating Topic, there
|
|||
|
|
+ will be problems with permission settings (#7151)
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+* [ISSUE #7149] fix bug : When creating and updating Topic, there will be problems with permission settings
|
|||
|
|
+
|
|||
|
|
+* [ISSUE #7149] fix bug : When creating and updating Topic, there will be problems with permission settings
|
|||
|
|
+
|
|||
|
|
+* [issue#7249]
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 十真 <shirenchuang.src@cainiao.com>
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/broker/BrokerController.java | 3 ++-
|
|||
|
|
+ 1 file changed, 2 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 13f9d002b..e8f943702 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1733,7 +1733,8 @@ public class BrokerController {
|
|||
|
|
+ new TopicConfig(topicConfig.getTopicName(),
|
|||
|
|
+ topicConfig.getReadQueueNums(),
|
|||
|
|
+ topicConfig.getWriteQueueNums(),
|
|||
|
|
+- this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
|
|||
|
|
++ topicConfig.getPerm()
|
|||
|
|
++ & this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
|
|||
|
|
+ } else {
|
|||
|
|
+ registerTopicConfig = new TopicConfig(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 017ad110475e8024585327b44f47e5e97aabc63b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: echooymxq <echooy.mxq@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 11:11:42 +0800
|
|||
|
|
+Subject: [PATCH 6/7] [ISSUE #7219] Fix Concurrent modify syncStateSet and Mark
|
|||
|
|
+ synchronizing frequently when shrink. (#7220)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/controller/ReplicasManager.java | 29 ++++++++++---------
|
|||
|
|
+ .../ha/autoswitch/AutoSwitchHAService.java | 21 ++++++++------
|
|||
|
|
+ 2 files changed, 28 insertions(+), 22 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+index abae7cdb0..37c82e434 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+@@ -542,7 +542,7 @@ public class ReplicasManager {
|
|||
|
|
+ this.brokerMetadata.updateAndPersist(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName(), tempBrokerMetadata.getBrokerId());
|
|||
|
|
+ this.tempBrokerMetadata.clear();
|
|||
|
|
+ this.brokerControllerId = this.brokerMetadata.getBrokerId();
|
|||
|
|
+- this.haService.setBrokerControllerId(this.brokerControllerId);
|
|||
|
|
++ this.haService.setLocalBrokerId(this.brokerControllerId);
|
|||
|
|
+ return true;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ LOGGER.error("fail to create metadata file", e);
|
|||
|
|
+@@ -594,7 +594,7 @@ public class ReplicasManager {
|
|||
|
|
+ if (this.brokerMetadata.isLoaded()) {
|
|||
|
|
+ this.registerState = RegisterState.CREATE_METADATA_FILE_DONE;
|
|||
|
|
+ this.brokerControllerId = brokerMetadata.getBrokerId();
|
|||
|
|
+- this.haService.setBrokerControllerId(this.brokerControllerId);
|
|||
|
|
++ this.haService.setLocalBrokerId(this.brokerControllerId);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ // 2. check if temp metadata exist
|
|||
|
|
+@@ -735,23 +735,26 @@ public class ReplicasManager {
|
|||
|
|
+ if (this.checkSyncStateSetTaskFuture != null) {
|
|||
|
|
+ this.checkSyncStateSetTaskFuture.cancel(false);
|
|||
|
|
+ }
|
|||
|
|
+- this.checkSyncStateSetTaskFuture = this.scheduledService.scheduleAtFixedRate(() -> {
|
|||
|
|
+- checkSyncStateSetAndDoReport();
|
|||
|
|
+- }, 3 * 1000, this.brokerConfig.getCheckSyncStateSetPeriod(), TimeUnit.MILLISECONDS);
|
|||
|
|
++ this.checkSyncStateSetTaskFuture = this.scheduledService.scheduleAtFixedRate(this::checkSyncStateSetAndDoReport, 3 * 1000,
|
|||
|
|
++ this.brokerConfig.getCheckSyncStateSetPeriod(), TimeUnit.MILLISECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void checkSyncStateSetAndDoReport() {
|
|||
|
|
+- final Set<Long> newSyncStateSet = this.haService.maybeShrinkSyncStateSet();
|
|||
|
|
+- newSyncStateSet.add(this.brokerControllerId);
|
|||
|
|
+- synchronized (this) {
|
|||
|
|
+- if (this.syncStateSet != null) {
|
|||
|
|
+- // Check if syncStateSet changed
|
|||
|
|
+- if (this.syncStateSet.size() == newSyncStateSet.size() && this.syncStateSet.containsAll(newSyncStateSet)) {
|
|||
|
|
+- return;
|
|||
|
|
++ try {
|
|||
|
|
++ final Set<Long> newSyncStateSet = this.haService.maybeShrinkSyncStateSet();
|
|||
|
|
++ newSyncStateSet.add(this.brokerControllerId);
|
|||
|
|
++ synchronized (this) {
|
|||
|
|
++ if (this.syncStateSet != null) {
|
|||
|
|
++ // Check if syncStateSet changed
|
|||
|
|
++ if (this.syncStateSet.size() == newSyncStateSet.size() && this.syncStateSet.containsAll(newSyncStateSet)) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++ doReportSyncStateSetChanged(newSyncStateSet);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ LOGGER.error("Check syncStateSet error", e);
|
|||
|
|
+ }
|
|||
|
|
+- doReportSyncStateSetChanged(newSyncStateSet);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void doReportSyncStateSetChanged(Set<Long> newSyncStateSet) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+index 6dc734e0c..d5393fdca 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+@@ -41,6 +41,7 @@ import java.nio.channels.SocketChannel;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Iterator;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Objects;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+@@ -73,7 +74,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ private EpochFileCache epochCache;
|
|||
|
|
+ private AutoSwitchHAClient haClient;
|
|||
|
|
+
|
|||
|
|
+- private Long brokerControllerId = null;
|
|||
|
|
++ private Long localBrokerId = null;
|
|||
|
|
+
|
|||
|
|
+ public AutoSwitchHAService() {
|
|||
|
|
+ }
|
|||
|
|
+@@ -287,9 +288,11 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+
|
|||
|
|
+ // If the slaveBrokerId is in syncStateSet but not in connectionCaughtUpTimeTable,
|
|||
|
|
+ // it means that the broker has not connected.
|
|||
|
|
+- for (Long slaveBrokerId : newSyncStateSet) {
|
|||
|
|
+- if (!this.connectionCaughtUpTimeTable.containsKey(slaveBrokerId)) {
|
|||
|
|
+- newSyncStateSet.remove(slaveBrokerId);
|
|||
|
|
++ Iterator<Long> iterator = newSyncStateSet.iterator();
|
|||
|
|
++ while (iterator.hasNext()) {
|
|||
|
|
++ Long slaveBrokerId = iterator.next();
|
|||
|
|
++ if (!Objects.equals(slaveBrokerId, this.localBrokerId) && !this.connectionCaughtUpTimeTable.containsKey(slaveBrokerId)) {
|
|||
|
|
++ iterator.remove();
|
|||
|
|
+ isSyncStateSetChanged = true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -419,7 +422,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ // To avoid the syncStateSet is not consistent with connectionList.
|
|||
|
|
+ // Fix issue: https://github.com/apache/rocketmq/issues/6662
|
|||
|
|
+ for (Long syncId : currentSyncStateSet) {
|
|||
|
|
+- if (!idList.contains(syncId) && this.brokerControllerId != null && !Objects.equals(syncId, this.brokerControllerId)) {
|
|||
|
|
++ if (!idList.contains(syncId) && this.localBrokerId != null && !Objects.equals(syncId, this.localBrokerId)) {
|
|||
|
|
+ LOGGER.warn("Slave {} is still in syncStateSet, but has lost its connection. So new offset can't be compute.", syncId);
|
|||
|
|
+ // Without check and re-compute, return the confirmOffset's value directly.
|
|||
|
|
+ return this.defaultMessageStore.getConfirmOffsetDirectly();
|
|||
|
|
+@@ -545,12 +548,12 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ return this.epochCache.getAllEntries();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public Long getBrokerControllerId() {
|
|||
|
|
+- return brokerControllerId;
|
|||
|
|
++ public Long getLocalBrokerId() {
|
|||
|
|
++ return localBrokerId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void setBrokerControllerId(Long brokerControllerId) {
|
|||
|
|
+- this.brokerControllerId = brokerControllerId;
|
|||
|
|
++ public void setLocalBrokerId(Long localBrokerId) {
|
|||
|
|
++ this.localBrokerId = localBrokerId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class AutoSwitchAcceptSocketService extends AcceptSocketService {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 77e8e54b37c3fc3ea0beffc1ace6f5bf20af10d9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 15:56:39 +0800
|
|||
|
|
+Subject: [PATCH 7/7] [ISSUE #7223] Support batch ack for grpc client in proxy
|
|||
|
|
+ (#7225)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../client/impl/mqclient/MQClientAPIExt.java | 26 +++
|
|||
|
|
+ .../rocketmq/proxy/config/ProxyConfig.java | 10 +
|
|||
|
|
+ .../grpc/v2/consumer/AckMessageActivity.java | 136 ++++++++---
|
|||
|
|
+ .../proxy/processor/AbstractProcessor.java | 4 +-
|
|||
|
|
+ .../proxy/processor/BatchAckResult.java | 53 +++++
|
|||
|
|
+ .../proxy/processor/ConsumerProcessor.java | 64 +++++
|
|||
|
|
+ .../processor/DefaultMessagingProcessor.java | 7 +
|
|||
|
|
+ .../proxy/processor/MessagingProcessor.java | 18 ++
|
|||
|
|
+ .../message/ClusterMessageService.java | 16 +-
|
|||
|
|
+ .../service/message/LocalMessageService.java | 58 +++++
|
|||
|
|
+ .../proxy/service/message/MessageService.java | 8 +
|
|||
|
|
+ .../service/message/ReceiptHandleMessage.java | 39 ++++
|
|||
|
|
+ .../v2/consumer/AckMessageActivityTest.java | 221 +++++++++++++++---
|
|||
|
|
+ .../proxy/processor/BaseProcessorTest.java | 18 +-
|
|||
|
|
+ .../processor/ConsumerProcessorTest.java | 115 +++++++++
|
|||
|
|
+ .../service/mqclient/MQClientAPIExtTest.java | 12 +
|
|||
|
|
+ 16 files changed, 728 insertions(+), 77 deletions(-)
|
|||
|
|
+ create mode 100644 proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+ create mode 100644 proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+index fb8f8d11f..d7c8ef8d9 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+@@ -306,6 +306,32 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ return future;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessageAsync(
|
|||
|
|
++ String brokerAddr,
|
|||
|
|
++ String topic,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ List<String> extraInfoList,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ ) {
|
|||
|
|
++ CompletableFuture<AckResult> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ this.batchAckMessageAsync(brokerAddr, timeoutMillis, new AckCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onSuccess(AckResult ackResult) {
|
|||
|
|
++ future.complete(ackResult);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onException(Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ }, topic, consumerGroup, extraInfoList);
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTimeAsync(
|
|||
|
|
+ String brokerAddr,
|
|||
|
|
+ String brokerName,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index 39caaa0d9..76a243919 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -250,6 +250,8 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ private long remotingWaitTimeMillsInTopicRouteQueue = 3 * 1000;
|
|||
|
|
+ private long remotingWaitTimeMillsInDefaultQueue = 3 * 1000;
|
|||
|
|
+
|
|||
|
|
++ private boolean enableBatchAck = false;
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public void initData() {
|
|||
|
|
+ parseDelayLevel();
|
|||
|
|
+@@ -1379,4 +1381,12 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ public void setRemotingWaitTimeMillsInDefaultQueue(long remotingWaitTimeMillsInDefaultQueue) {
|
|||
|
|
+ this.remotingWaitTimeMillsInDefaultQueue = remotingWaitTimeMillsInDefaultQueue;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableBatchAck() {
|
|||
|
|
++ return enableBatchAck;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableBatchAck(boolean enableBatchAck) {
|
|||
|
|
++ this.enableBatchAck = enableBatchAck;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
+index 9a3a77201..97c716c8f 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
+@@ -31,12 +31,15 @@ import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
+ import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.MessageReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
++import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.AbstractMessingActivity;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.channel.GrpcChannelManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcClientSettingsManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcConverter;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.ResponseBuilder;
|
|||
|
|
++import org.apache.rocketmq.proxy.processor.BatchAckResult;
|
|||
|
|
+ import org.apache.rocketmq.proxy.processor.MessagingProcessor;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+
|
|||
|
|
+ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+
|
|||
|
|
+@@ -50,60 +53,98 @@ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ validateTopicAndConsumerGroup(request.getTopic(), request.getGroup());
|
|||
|
|
+-
|
|||
|
|
+- CompletableFuture<AckMessageResultEntry>[] futures = new CompletableFuture[request.getEntriesCount()];
|
|||
|
|
+- for (int i = 0; i < request.getEntriesCount(); i++) {
|
|||
|
|
+- futures[i] = processAckMessage(ctx, request, request.getEntries(i));
|
|||
|
|
++ String group = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getGroup());
|
|||
|
|
++ String topic = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getTopic());
|
|||
|
|
++ if (ConfigurationManager.getProxyConfig().isEnableBatchAck()) {
|
|||
|
|
++ future = ackMessageInBatch(ctx, group, topic, request);
|
|||
|
|
++ } else {
|
|||
|
|
++ future = ackMessageOneByOne(ctx, group, topic, request);
|
|||
|
|
+ }
|
|||
|
|
+- CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
+- if (throwable != null) {
|
|||
|
|
+- future.completeExceptionally(throwable);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<AckMessageResponse> ackMessageInBatch(ProxyContext ctx, String group, String topic, AckMessageRequest request) {
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList = new ArrayList<>(request.getEntriesCount());
|
|||
|
|
+
|
|||
|
|
++ for (AckMessageEntry ackMessageEntry : request.getEntriesList()) {
|
|||
|
|
++ String handleString = getHandleString(ctx, group, request, ackMessageEntry);
|
|||
|
|
++ handleMessageList.add(new ReceiptHandleMessage(ReceiptHandle.decode(handleString), ackMessageEntry.getMessageId()));
|
|||
|
|
++ }
|
|||
|
|
++ return this.messagingProcessor.batchAckMessage(ctx, handleMessageList, group, topic)
|
|||
|
|
++ .thenApply(batchAckResultList -> {
|
|||
|
|
++ AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder();
|
|||
|
|
+ Set<Code> responseCodes = new HashSet<>();
|
|||
|
|
+- List<AckMessageResultEntry> entryList = new ArrayList<>();
|
|||
|
|
+- for (CompletableFuture<AckMessageResultEntry> entryFuture : futures) {
|
|||
|
|
+- AckMessageResultEntry entryResult = entryFuture.join();
|
|||
|
|
+- responseCodes.add(entryResult.getStatus().getCode());
|
|||
|
|
+- entryList.add(entryResult);
|
|||
|
|
+- }
|
|||
|
|
+- AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder()
|
|||
|
|
+- .addAllEntries(entryList);
|
|||
|
|
+- if (responseCodes.size() > 1) {
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.MULTIPLE_RESULTS, Code.MULTIPLE_RESULTS.name()));
|
|||
|
|
+- } else if (responseCodes.size() == 1) {
|
|||
|
|
+- Code code = responseCodes.stream().findAny().get();
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(code, code.name()));
|
|||
|
|
+- } else {
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack message result is empty"));
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ AckMessageResultEntry entry = convertToAckMessageResultEntry(batchAckResult);
|
|||
|
|
++ responseBuilder.addEntries(entry);
|
|||
|
|
++ responseCodes.add(entry.getStatus().getCode());
|
|||
|
|
+ }
|
|||
|
|
+- future.complete(responseBuilder.build());
|
|||
|
|
++ setAckResponseStatus(responseBuilder, responseCodes);
|
|||
|
|
++ return responseBuilder.build();
|
|||
|
|
+ });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected AckMessageResultEntry convertToAckMessageResultEntry(BatchAckResult batchAckResult) {
|
|||
|
|
++ ReceiptHandleMessage handleMessage = batchAckResult.getReceiptHandleMessage();
|
|||
|
|
++ AckMessageResultEntry.Builder resultBuilder = AckMessageResultEntry.newBuilder()
|
|||
|
|
++ .setMessageId(handleMessage.getMessageId())
|
|||
|
|
++ .setReceiptHandle(handleMessage.getReceiptHandle().getReceiptHandle());
|
|||
|
|
++ if (batchAckResult.getProxyException() != null) {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(batchAckResult.getProxyException()));
|
|||
|
|
++ } else {
|
|||
|
|
++ AckResult ackResult = batchAckResult.getAckResult();
|
|||
|
|
++ if (AckStatus.OK.equals(ackResult.getStatus())) {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.OK, Code.OK.name()));
|
|||
|
|
++ } else {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack failed: status is abnormal"));
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+- return future;
|
|||
|
|
++ return resultBuilder.build();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- protected CompletableFuture<AckMessageResultEntry> processAckMessage(ProxyContext ctx, AckMessageRequest request,
|
|||
|
|
++ protected CompletableFuture<AckMessageResponse> ackMessageOneByOne(ProxyContext ctx, String group, String topic, AckMessageRequest request) {
|
|||
|
|
++ CompletableFuture<AckMessageResponse> resultFuture = new CompletableFuture<>();
|
|||
|
|
++ CompletableFuture<AckMessageResultEntry>[] futures = new CompletableFuture[request.getEntriesCount()];
|
|||
|
|
++ for (int i = 0; i < request.getEntriesCount(); i++) {
|
|||
|
|
++ futures[i] = processAckMessage(ctx, group, topic, request, request.getEntries(i));
|
|||
|
|
++ }
|
|||
|
|
++ CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
++ if (throwable != null) {
|
|||
|
|
++ resultFuture.completeExceptionally(throwable);
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ Set<Code> responseCodes = new HashSet<>();
|
|||
|
|
++ List<AckMessageResultEntry> entryList = new ArrayList<>();
|
|||
|
|
++ for (CompletableFuture<AckMessageResultEntry> entryFuture : futures) {
|
|||
|
|
++ AckMessageResultEntry entryResult = entryFuture.join();
|
|||
|
|
++ responseCodes.add(entryResult.getStatus().getCode());
|
|||
|
|
++ entryList.add(entryResult);
|
|||
|
|
++ }
|
|||
|
|
++ AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder()
|
|||
|
|
++ .addAllEntries(entryList);
|
|||
|
|
++ setAckResponseStatus(responseBuilder, responseCodes);
|
|||
|
|
++ resultFuture.complete(responseBuilder.build());
|
|||
|
|
++ });
|
|||
|
|
++ return resultFuture;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<AckMessageResultEntry> processAckMessage(ProxyContext ctx, String group, String topic, AckMessageRequest request,
|
|||
|
|
+ AckMessageEntry ackMessageEntry) {
|
|||
|
|
+ CompletableFuture<AckMessageResultEntry> future = new CompletableFuture<>();
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+- String handleString = ackMessageEntry.getReceiptHandle();
|
|||
|
|
+-
|
|||
|
|
+- String group = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getGroup());
|
|||
|
|
+- MessageReceiptHandle messageReceiptHandle = messagingProcessor.removeReceiptHandle(ctx, grpcChannelManager.getChannel(ctx.getClientID()), group, ackMessageEntry.getMessageId(), ackMessageEntry.getReceiptHandle());
|
|||
|
|
+- if (messageReceiptHandle != null) {
|
|||
|
|
+- handleString = messageReceiptHandle.getReceiptHandleStr();
|
|||
|
|
+- }
|
|||
|
|
++ String handleString = this.getHandleString(ctx, group, request, ackMessageEntry);
|
|||
|
|
+ CompletableFuture<AckResult> ackResultFuture = this.messagingProcessor.ackMessage(
|
|||
|
|
+ ctx,
|
|||
|
|
+ ReceiptHandle.decode(handleString),
|
|||
|
|
+ ackMessageEntry.getMessageId(),
|
|||
|
|
+ group,
|
|||
|
|
+- GrpcConverter.getInstance().wrapResourceWithNamespace(request.getTopic()));
|
|||
|
|
++ topic
|
|||
|
|
++ );
|
|||
|
|
+ ackResultFuture.thenAccept(result -> {
|
|||
|
|
+ future.complete(convertToAckMessageResultEntry(ctx, ackMessageEntry, result));
|
|||
|
|
+ }).exceptionally(t -> {
|
|||
|
|
+@@ -139,4 +180,25 @@ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+ .setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack failed: status is abnormal"))
|
|||
|
|
+ .build();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ protected void setAckResponseStatus(AckMessageResponse.Builder responseBuilder, Set<Code> responseCodes) {
|
|||
|
|
++ if (responseCodes.size() > 1) {
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.MULTIPLE_RESULTS, Code.MULTIPLE_RESULTS.name()));
|
|||
|
|
++ } else if (responseCodes.size() == 1) {
|
|||
|
|
++ Code code = responseCodes.stream().findAny().get();
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(code, code.name()));
|
|||
|
|
++ } else {
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack message result is empty"));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected String getHandleString(ProxyContext ctx, String group, AckMessageRequest request, AckMessageEntry ackMessageEntry) {
|
|||
|
|
++ String handleString = ackMessageEntry.getReceiptHandle();
|
|||
|
|
++
|
|||
|
|
++ MessageReceiptHandle messageReceiptHandle = messagingProcessor.removeReceiptHandle(ctx, grpcChannelManager.getChannel(ctx.getClientID()), group, ackMessageEntry.getMessageId(), ackMessageEntry.getReceiptHandle());
|
|||
|
|
++ if (messageReceiptHandle != null) {
|
|||
|
|
++ handleString = messageReceiptHandle.getReceiptHandleStr();
|
|||
|
|
++ }
|
|||
|
|
++ return handleString;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
+index b61c3df9e..c63212c23 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
+@@ -27,6 +27,8 @@ public abstract class AbstractProcessor extends AbstractStartAndShutdown {
|
|||
|
|
+ protected MessagingProcessor messagingProcessor;
|
|||
|
|
+ protected ServiceManager serviceManager;
|
|||
|
|
+
|
|||
|
|
++ protected static final ProxyException EXPIRED_HANDLE_PROXY_EXCEPTION = new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired");
|
|||
|
|
++
|
|||
|
|
+ public AbstractProcessor(MessagingProcessor messagingProcessor,
|
|||
|
|
+ ServiceManager serviceManager) {
|
|||
|
|
+ this.messagingProcessor = messagingProcessor;
|
|||
|
|
+@@ -35,7 +37,7 @@ public abstract class AbstractProcessor extends AbstractStartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ protected void validateReceiptHandle(ReceiptHandle handle) {
|
|||
|
|
+ if (handle.isExpired()) {
|
|||
|
|
+- throw new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired");
|
|||
|
|
++ throw EXPIRED_HANDLE_PROXY_EXCEPTION;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..dfb9c9b9e
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+@@ -0,0 +1,53 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.proxy.processor;
|
|||
|
|
++
|
|||
|
|
++import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
++
|
|||
|
|
++public class BatchAckResult {
|
|||
|
|
++
|
|||
|
|
++ private final ReceiptHandleMessage receiptHandleMessage;
|
|||
|
|
++ private AckResult ackResult;
|
|||
|
|
++ private ProxyException proxyException;
|
|||
|
|
++
|
|||
|
|
++ public BatchAckResult(ReceiptHandleMessage receiptHandleMessage,
|
|||
|
|
++ AckResult ackResult) {
|
|||
|
|
++ this.receiptHandleMessage = receiptHandleMessage;
|
|||
|
|
++ this.ackResult = ackResult;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public BatchAckResult(ReceiptHandleMessage receiptHandleMessage,
|
|||
|
|
++ ProxyException proxyException) {
|
|||
|
|
++ this.receiptHandleMessage = receiptHandleMessage;
|
|||
|
|
++ this.proxyException = proxyException;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandleMessage getReceiptHandleMessage() {
|
|||
|
|
++ return receiptHandleMessage;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public AckResult getAckResult() {
|
|||
|
|
++ return ackResult;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ProxyException getProxyException() {
|
|||
|
|
++ return proxyException;
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
+index 656a6339d..f3522b374 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
+@@ -48,6 +48,7 @@ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.ProxyUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManager;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody;
|
|||
|
|
+@@ -241,6 +242,69 @@ public class ConsumerProcessor extends AbstractProcessor {
|
|||
|
|
+ return FutureUtils.addExecutor(future, this.executor);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ ) {
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = new ArrayList<>(handleMessageList.size());
|
|||
|
|
++ Map<String, List<ReceiptHandleMessage>> brokerHandleListMap = new HashMap<>();
|
|||
|
|
++
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ if (handleMessage.getReceiptHandle().isExpired()) {
|
|||
|
|
++ batchAckResultList.add(new BatchAckResult(handleMessage, EXPIRED_HANDLE_PROXY_EXCEPTION));
|
|||
|
|
++ continue;
|
|||
|
|
++ }
|
|||
|
|
++ List<ReceiptHandleMessage> brokerHandleList = brokerHandleListMap.computeIfAbsent(handleMessage.getReceiptHandle().getBrokerName(), key -> new ArrayList<>());
|
|||
|
|
++ brokerHandleList.add(handleMessage);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (brokerHandleListMap.isEmpty()) {
|
|||
|
|
++ return FutureUtils.addExecutor(CompletableFuture.completedFuture(batchAckResultList), this.executor);
|
|||
|
|
++ }
|
|||
|
|
++ Set<Map.Entry<String, List<ReceiptHandleMessage>>> brokerHandleListMapEntrySet = brokerHandleListMap.entrySet();
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>>[] futures = new CompletableFuture[brokerHandleListMapEntrySet.size()];
|
|||
|
|
++ int futureIndex = 0;
|
|||
|
|
++ for (Map.Entry<String, List<ReceiptHandleMessage>> entry : brokerHandleListMapEntrySet) {
|
|||
|
|
++ futures[futureIndex++] = processBrokerHandle(ctx, consumerGroup, topic, entry.getValue(), timeoutMillis);
|
|||
|
|
++ }
|
|||
|
|
++ CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
++ if (throwable != null) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
++ }
|
|||
|
|
++ for (CompletableFuture<List<BatchAckResult>> resultFuture : futures) {
|
|||
|
|
++ batchAckResultList.addAll(resultFuture.join());
|
|||
|
|
++ }
|
|||
|
|
++ future.complete(batchAckResultList);
|
|||
|
|
++ });
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return FutureUtils.addExecutor(future, this.executor);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<List<BatchAckResult>> processBrokerHandle(ProxyContext ctx, String consumerGroup, String topic, List<ReceiptHandleMessage> handleMessageList, long timeoutMillis) {
|
|||
|
|
++ return this.serviceManager.getMessageService().batchAckMessage(ctx, handleMessageList, consumerGroup, topic, timeoutMillis)
|
|||
|
|
++ .thenApply(result -> {
|
|||
|
|
++ List<BatchAckResult> results = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ results.add(new BatchAckResult(handleMessage, result));
|
|||
|
|
++ }
|
|||
|
|
++ return results;
|
|||
|
|
++ })
|
|||
|
|
++ .exceptionally(throwable -> {
|
|||
|
|
++ List<BatchAckResult> results = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ results.add(new BatchAckResult(handleMessage, new ProxyException(ProxyExceptionCode.INTERNAL_SERVER_ERROR, throwable.getMessage(), throwable)));
|
|||
|
|
++ }
|
|||
|
|
++ return results;
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle,
|
|||
|
|
+ String messageId, String groupName, String topicName, long invisibleTime, long timeoutMillis) {
|
|||
|
|
+ CompletableFuture<AckResult> future = new CompletableFuture<>();
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
+index 188cb7b9b..ba150051b 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
+@@ -46,6 +46,7 @@ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManagerFactory;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData;
|
|||
|
|
+@@ -183,6 +184,12 @@ public class DefaultMessagingProcessor extends AbstractStartAndShutdown implemen
|
|||
|
|
+ return this.consumerProcessor.ackMessage(ctx, handle, messageId, consumerGroup, topic, timeoutMillis);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<List<BatchAckResult>> batchAckMessage(ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList, String consumerGroup, String topic, long timeoutMillis) {
|
|||
|
|
++ return this.consumerProcessor.batchAckMessage(ctx, handleMessageList, consumerGroup, topic, timeoutMillis);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, String messageId,
|
|||
|
|
+ String groupName, String topicName, long invisibleTime, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
+index d86be0bd8..2ae7418ba 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
+@@ -37,6 +37,7 @@ import org.apache.rocketmq.proxy.common.Address;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.MessageReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData;
|
|||
|
|
+@@ -155,6 +156,23 @@ public interface MessagingProcessor extends StartAndShutdown {
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
++ default CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic
|
|||
|
|
++ ) {
|
|||
|
|
++ return batchAckMessage(ctx, handleMessageList, consumerGroup, topic, DEFAULT_TIMEOUT_MILLS);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
+ default CompletableFuture<AckResult> changeInvisibleTime(
|
|||
|
|
+ ProxyContext ctx,
|
|||
|
|
+ ReceiptHandle handle,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
+index 9f163f1b9..70b72deae 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
+@@ -20,9 +20,11 @@ import com.google.common.collect.Lists;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
++import java.util.stream.Collectors;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PopResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullResult;
|
|||
|
|
++import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendResult;
|
|||
|
|
+ import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.common.message.Message;
|
|||
|
|
+@@ -31,7 +33,6 @@ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+-import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.TopicRouteService;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+@@ -137,6 +138,19 @@ public class ClusterMessageService implements MessageService {
|
|||
|
|
+ );
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessage(ProxyContext ctx, List<ReceiptHandleMessage> handleList, String consumerGroup,
|
|||
|
|
++ String topic, long timeoutMillis) {
|
|||
|
|
++ List<String> extraInfoList = handleList.stream().map(message -> message.getReceiptHandle().getReceiptHandle()).collect(Collectors.toList());
|
|||
|
|
++ return this.mqClientAPIFactory.getClient().batchAckMessageAsync(
|
|||
|
|
++ this.resolveBrokerAddrInReceiptHandle(ctx, handleList.get(0).getReceiptHandle()),
|
|||
|
|
++ topic,
|
|||
|
|
++ consumerGroup,
|
|||
|
|
++ extraInfoList,
|
|||
|
|
++ timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<PullResult> pullMessage(ProxyContext ctx, AddressableMessageQueue messageQueue,
|
|||
|
|
+ PullMessageRequestHeader requestHeader, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+index eb2c4d9ee..ca7dcc9eb 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+@@ -19,6 +19,7 @@ package org.apache.rocketmq.proxy.service.message;
|
|||
|
|
+ import io.netty.channel.ChannelHandlerContext;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
++import java.util.BitSet;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+@@ -54,6 +55,8 @@ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAck;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAckMessageRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.AckMessageRequestHeader;
|
|||
|
|
+@@ -364,6 +367,61 @@ public class LocalMessageService implements MessageService {
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessage(ProxyContext ctx, List<ReceiptHandleMessage> handleList,
|
|||
|
|
++ String consumerGroup, String topic, long timeoutMillis) {
|
|||
|
|
++ SimpleChannel channel = channelManager.createChannel(ctx);
|
|||
|
|
++ ChannelHandlerContext channelHandlerContext = channel.getChannelHandlerContext();
|
|||
|
|
++ RemotingCommand command = LocalRemotingCommand.createRequestCommand(RequestCode.BATCH_ACK_MESSAGE, null);
|
|||
|
|
++
|
|||
|
|
++ Map<String, BatchAck> batchAckMap = new HashMap<>();
|
|||
|
|
++ for (ReceiptHandleMessage receiptHandleMessage : handleList) {
|
|||
|
|
++ String extraInfo = receiptHandleMessage.getReceiptHandle().getReceiptHandle();
|
|||
|
|
++ String[] extraInfoData = ExtraInfoUtil.split(extraInfo);
|
|||
|
|
++ String mergeKey = ExtraInfoUtil.getRetry(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getQueueId(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getCkQueueOffset(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getPopTime(extraInfoData);
|
|||
|
|
++ BatchAck bAck = batchAckMap.computeIfAbsent(mergeKey, k -> {
|
|||
|
|
++ BatchAck newBatchAck = new BatchAck();
|
|||
|
|
++ newBatchAck.setConsumerGroup(consumerGroup);
|
|||
|
|
++ newBatchAck.setTopic(topic);
|
|||
|
|
++ newBatchAck.setRetry(ExtraInfoUtil.getRetry(extraInfoData));
|
|||
|
|
++ newBatchAck.setStartOffset(ExtraInfoUtil.getCkQueueOffset(extraInfoData));
|
|||
|
|
++ newBatchAck.setQueueId(ExtraInfoUtil.getQueueId(extraInfoData));
|
|||
|
|
++ newBatchAck.setReviveQueueId(ExtraInfoUtil.getReviveQid(extraInfoData));
|
|||
|
|
++ newBatchAck.setPopTime(ExtraInfoUtil.getPopTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setInvisibleTime(ExtraInfoUtil.getInvisibleTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setBitSet(new BitSet());
|
|||
|
|
++ return newBatchAck;
|
|||
|
|
++ });
|
|||
|
|
++ bAck.getBitSet().set((int) (ExtraInfoUtil.getQueueOffset(extraInfoData) - ExtraInfoUtil.getCkQueueOffset(extraInfoData)));
|
|||
|
|
++ }
|
|||
|
|
++ BatchAckMessageRequestBody requestBody = new BatchAckMessageRequestBody();
|
|||
|
|
++ requestBody.setBrokerName(brokerController.getBrokerConfig().getBrokerName());
|
|||
|
|
++ requestBody.setAcks(new ArrayList<>(batchAckMap.values()));
|
|||
|
|
++
|
|||
|
|
++ command.setBody(requestBody.encode());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ RemotingCommand response = brokerController.getAckMessageProcessor()
|
|||
|
|
++ .processRequest(channelHandlerContext, command);
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ log.error("Fail to process batchAckMessage command", e);
|
|||
|
|
++ future.completeExceptionally(e);
|
|||
|
|
++ }
|
|||
|
|
++ return future.thenApply(r -> {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ if (ResponseCode.SUCCESS == r.getCode()) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ } else {
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ }
|
|||
|
|
++ return ackResult;
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<PullResult> pullMessage(ProxyContext ctx, AddressableMessageQueue messageQueue,
|
|||
|
|
+ PullMessageRequestHeader requestHeader, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
+index 15da17154..58a835adb 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
+@@ -91,6 +91,14 @@ public interface MessageService {
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
++ CompletableFuture<AckResult> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
+ CompletableFuture<PullResult> pullMessage(
|
|||
|
|
+ ProxyContext ctx,
|
|||
|
|
+ AddressableMessageQueue messageQueue,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..ae63fed49
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+@@ -0,0 +1,39 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.proxy.service.message;
|
|||
|
|
++
|
|||
|
|
++import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
++
|
|||
|
|
++public class ReceiptHandleMessage {
|
|||
|
|
++
|
|||
|
|
++ private final ReceiptHandle receiptHandle;
|
|||
|
|
++ private final String messageId;
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandleMessage(ReceiptHandle receiptHandle, String messageId) {
|
|||
|
|
++ this.receiptHandle = receiptHandle;
|
|||
|
|
++ this.messageId = messageId;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandle getReceiptHandle() {
|
|||
|
|
++ return receiptHandle;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public String getMessageId() {
|
|||
|
|
++ return messageId;
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
+index 49fdfc6a8..3c4746105 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
+@@ -20,21 +20,32 @@ package org.apache.rocketmq.proxy.grpc.v2.consumer;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageEntry;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageRequest;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageResponse;
|
|||
|
|
++import apache.rocketmq.v2.AckMessageResultEntry;
|
|||
|
|
+ import apache.rocketmq.v2.Code;
|
|||
|
|
+ import apache.rocketmq.v2.Resource;
|
|||
|
|
++import java.util.ArrayList;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
++import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.BaseActivityTest;
|
|||
|
|
++import org.apache.rocketmq.proxy.processor.BatchAckResult;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
++import org.mockito.stubbing.Answer;
|
|||
|
|
+
|
|||
|
|
+ import static org.junit.Assert.assertEquals;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.anyList;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
++import static org.mockito.Mockito.doAnswer;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ public class AckMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+@@ -52,43 +63,197 @@ public class AckMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testAckMessage() throws Throwable {
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg1"), anyString(), anyString()))
|
|||
|
|
++ ConfigurationManager.getProxyConfig().setEnableBatchAck(false);
|
|||
|
|
++
|
|||
|
|
++ String msg1 = "msg1";
|
|||
|
|
++ String msg2 = "msg2";
|
|||
|
|
++ String msg3 = "msg3";
|
|||
|
|
++
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg1), anyString(), anyString()))
|
|||
|
|
+ .thenThrow(new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired"));
|
|||
|
|
+
|
|||
|
|
+ AckResult msg2AckResult = new AckResult();
|
|||
|
|
+ msg2AckResult.setStatus(AckStatus.OK);
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg2"), anyString(), anyString()))
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg2), anyString(), anyString()))
|
|||
|
|
+ .thenReturn(CompletableFuture.completedFuture(msg2AckResult));
|
|||
|
|
+
|
|||
|
|
+ AckResult msg3AckResult = new AckResult();
|
|||
|
|
+ msg3AckResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg3"), anyString(), anyString()))
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg3), anyString(), anyString()))
|
|||
|
|
+ .thenReturn(CompletableFuture.completedFuture(msg3AckResult));
|
|||
|
|
+
|
|||
|
|
+- AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
+- createContext(),
|
|||
|
|
+- AckMessageRequest.newBuilder()
|
|||
|
|
+- .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
+- .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg1")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg2")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg3")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .build()
|
|||
|
|
+- ).get();
|
|||
|
|
+-
|
|||
|
|
+- assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
+- assertEquals(3, response.getEntriesCount());
|
|||
|
|
+- assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getEntries(0).getStatus().getCode());
|
|||
|
|
+- assertEquals(Code.OK, response.getEntries(1).getStatus().getCode());
|
|||
|
|
+- assertEquals(Code.INTERNAL_SERVER_ERROR, response.getEntries(2).getStatus().getCode());
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg1)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg2)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.OK, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg3)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg1)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg2)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg3)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++
|
|||
|
|
++ assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
++ assertEquals(3, response.getEntriesCount());
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getEntries(0).getStatus().getCode());
|
|||
|
|
++ assertEquals(Code.OK, response.getEntries(1).getStatus().getCode());
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getEntries(2).getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testAckMessageInBatch() throws Throwable {
|
|||
|
|
++ ConfigurationManager.getProxyConfig().setEnableBatchAck(true);
|
|||
|
|
++
|
|||
|
|
++ String successMessageId = "msg1";
|
|||
|
|
++ String notOkMessageId = "msg2";
|
|||
|
|
++ String exceptionMessageId = "msg3";
|
|||
|
|
++
|
|||
|
|
++ doAnswer((Answer<CompletableFuture<List<BatchAckResult>>>) invocation -> {
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = invocation.getArgument(1, List.class);
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage receiptHandleMessage : receiptHandleMessageList) {
|
|||
|
|
++ BatchAckResult batchAckResult;
|
|||
|
|
++ if (receiptHandleMessage.getMessageId().equals(successMessageId)) {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, ackResult);
|
|||
|
|
++ } else if (receiptHandleMessage.getMessageId().equals(notOkMessageId)) {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, ackResult);
|
|||
|
|
++ } else {
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, ""));
|
|||
|
|
++ }
|
|||
|
|
++ batchAckResultList.add(batchAckResult);
|
|||
|
|
++ }
|
|||
|
|
++ return CompletableFuture.completedFuture(batchAckResultList);
|
|||
|
|
++ }).when(this.messagingProcessor).batchAckMessage(any(), anyList(), anyString(), anyString());
|
|||
|
|
++
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(successMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.OK, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(notOkMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(exceptionMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(successMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(notOkMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(exceptionMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++
|
|||
|
|
++ assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
++ assertEquals(3, response.getEntriesCount());
|
|||
|
|
++ Map<String, Code> msgCode = new HashMap<>();
|
|||
|
|
++ for (AckMessageResultEntry entry : response.getEntriesList()) {
|
|||
|
|
++ msgCode.put(entry.getMessageId(), entry.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ assertEquals(Code.OK, msgCode.get(successMessageId));
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, msgCode.get(notOkMessageId));
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, msgCode.get(exceptionMessageId));
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
+index 5c1ea9627..072630e39 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
+@@ -66,14 +66,6 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ protected ProxyRelayService proxyRelayService;
|
|||
|
|
+ @Mock
|
|||
|
|
+ protected MetadataService metadataService;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ProducerProcessor producerProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ConsumerProcessor consumerProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected TransactionProcessor transactionProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ClientProcessor clientProcessor;
|
|||
|
|
+
|
|||
|
|
+ public void before() throws Throwable {
|
|||
|
|
+ super.before();
|
|||
|
|
+@@ -92,6 +84,13 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ protected static MessageExt createMessageExt(String topic, String tags, int reconsumeTimes, long invisibleTime) {
|
|||
|
|
++ return createMessageExt(topic, tags, reconsumeTimes, invisibleTime, System.currentTimeMillis(),
|
|||
|
|
++ RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE),
|
|||
|
|
++ RANDOM.nextInt(Integer.MAX_VALUE), "mockBroker");
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected static MessageExt createMessageExt(String topic, String tags, int reconsumeTimes, long invisibleTime, long popTime,
|
|||
|
|
++ long startOffset, int reviveQid, int queueId, long queueOffset, String brokerName) {
|
|||
|
|
+ MessageExt messageExt = new MessageExt();
|
|||
|
|
+ messageExt.setTopic(topic);
|
|||
|
|
+ messageExt.setTags(tags);
|
|||
|
|
+@@ -100,8 +99,7 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ messageExt.setMsgId(MessageClientIDSetter.createUniqID());
|
|||
|
|
+ messageExt.setCommitLogOffset(RANDOM.nextInt(Integer.MAX_VALUE));
|
|||
|
|
+ MessageAccessor.putProperty(messageExt, MessageConst.PROPERTY_POP_CK,
|
|||
|
|
+- ExtraInfoUtil.buildExtraInfo(RANDOM.nextInt(Integer.MAX_VALUE), System.currentTimeMillis(), invisibleTime,
|
|||
|
|
+- RANDOM.nextInt(Integer.MAX_VALUE), topic, "mockBroker", RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE)));
|
|||
|
|
++ ExtraInfoUtil.buildExtraInfo(startOffset, popTime, invisibleTime, reviveQid, topic, brokerName, queueId, queueOffset));
|
|||
|
|
+ return messageExt;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
+index 717e86fc0..db268a06e 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
+@@ -20,8 +20,11 @@ package org.apache.rocketmq.proxy.processor;
|
|||
|
|
+ import com.google.common.collect.Sets;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
++import java.util.Collections;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+@@ -39,7 +42,10 @@ import org.apache.rocketmq.common.message.MessageClientIDSetter;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.ProxyUtils;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.MessageQueueView;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+@@ -50,16 +56,22 @@ import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+ import org.mockito.ArgumentCaptor;
|
|||
|
|
++import org.mockito.stubbing.Answer;
|
|||
|
|
+
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+ import static org.junit.Assert.assertEquals;
|
|||
|
|
+ import static org.junit.Assert.assertNotNull;
|
|||
|
|
++import static org.junit.Assert.assertNull;
|
|||
|
|
+ import static org.junit.Assert.assertSame;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.anyList;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
++import static org.mockito.Mockito.doAnswer;
|
|||
|
|
+ import static org.mockito.Mockito.mock;
|
|||
|
|
++import static org.mockito.Mockito.never;
|
|||
|
|
++import static org.mockito.Mockito.verify;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ public class ConsumerProcessorTest extends BaseProcessorTest {
|
|||
|
|
+@@ -162,6 +174,109 @@ public class ConsumerProcessorTest extends BaseProcessorTest {
|
|||
|
|
+ assertEquals(handle.getReceiptHandle(), requestHeaderArgumentCaptor.getValue().getExtraInfo());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckExpireMessage() throws Throwable {
|
|||
|
|
++ String brokerName1 = "brokerName1";
|
|||
|
|
++
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = new ArrayList<>();
|
|||
|
|
++ for (int i = 0; i < 3; i++) {
|
|||
|
|
++ MessageExt expireMessage = createMessageExt(TOPIC, "", 0, 3000, System.currentTimeMillis() - 10000,
|
|||
|
|
++ 0, 0, 0, i, brokerName1);
|
|||
|
|
++ ReceiptHandle expireHandle = create(expireMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(expireHandle, expireMessage.getMsgId()));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = this.consumerProcessor.batchAckMessage(createContext(), receiptHandleMessageList, CONSUMER_GROUP, TOPIC, 3000).get();
|
|||
|
|
++
|
|||
|
|
++ verify(this.messageService, never()).batchAckMessage(any(), anyList(), anyString(), anyString(), anyLong());
|
|||
|
|
++ assertEquals(receiptHandleMessageList.size(), batchAckResultList.size());
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ assertNull(batchAckResult.getAckResult());
|
|||
|
|
++ assertNotNull(batchAckResult.getProxyException());
|
|||
|
|
++ assertNotNull(batchAckResult.getReceiptHandleMessage());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckMessage() throws Throwable {
|
|||
|
|
++ String brokerName1 = "brokerName1";
|
|||
|
|
++ String brokerName2 = "brokerName2";
|
|||
|
|
++ String errThrowBrokerName = "errThrowBrokerName";
|
|||
|
|
++ MessageExt expireMessage = createMessageExt(TOPIC, "", 0, 3000, System.currentTimeMillis() - 10000,
|
|||
|
|
++ 0, 0, 0, 0, brokerName1);
|
|||
|
|
++ ReceiptHandle expireHandle = create(expireMessage);
|
|||
|
|
++
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = new ArrayList<>();
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(expireHandle, expireMessage.getMsgId()));
|
|||
|
|
++ List<String> broker1Msg = new ArrayList<>();
|
|||
|
|
++ List<String> broker2Msg = new ArrayList<>();
|
|||
|
|
++
|
|||
|
|
++ long now = System.currentTimeMillis();
|
|||
|
|
++ int msgNum = 3;
|
|||
|
|
++ for (int i = 0; i < msgNum; i++) {
|
|||
|
|
++ MessageExt brokerMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, i + 1, brokerName1);
|
|||
|
|
++ ReceiptHandle brokerHandle = create(brokerMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(brokerHandle, brokerMessage.getMsgId()));
|
|||
|
|
++ broker1Msg.add(brokerMessage.getMsgId());
|
|||
|
|
++ }
|
|||
|
|
++ for (int i = 0; i < msgNum; i++) {
|
|||
|
|
++ MessageExt brokerMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, i + 1, brokerName2);
|
|||
|
|
++ ReceiptHandle brokerHandle = create(brokerMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(brokerHandle, brokerMessage.getMsgId()));
|
|||
|
|
++ broker2Msg.add(brokerMessage.getMsgId());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // for this message, will throw exception in batchAckMessage
|
|||
|
|
++ MessageExt errThrowMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, 0, errThrowBrokerName);
|
|||
|
|
++ ReceiptHandle errThrowHandle = create(errThrowMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(errThrowHandle, errThrowMessage.getMsgId()));
|
|||
|
|
++
|
|||
|
|
++ Collections.shuffle(receiptHandleMessageList);
|
|||
|
|
++
|
|||
|
|
++ doAnswer((Answer<CompletableFuture<AckResult>>) invocation -> {
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList = invocation.getArgument(1, List.class);
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ String brokerName = handleMessageList.get(0).getReceiptHandle().getBrokerName();
|
|||
|
|
++ if (brokerName.equals(brokerName1)) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ } else if (brokerName.equals(brokerName2)) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ } else {
|
|||
|
|
++ return FutureUtils.completeExceptionally(new RuntimeException());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ return CompletableFuture.completedFuture(ackResult);
|
|||
|
|
++ }).when(this.messageService).batchAckMessage(any(), anyList(), anyString(), anyString(), anyLong());
|
|||
|
|
++
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = this.consumerProcessor.batchAckMessage(createContext(), receiptHandleMessageList, CONSUMER_GROUP, TOPIC, 3000).get();
|
|||
|
|
++ assertEquals(receiptHandleMessageList.size(), batchAckResultList.size());
|
|||
|
|
++
|
|||
|
|
++ // check ackResult for each msg
|
|||
|
|
++ Map<String, BatchAckResult> msgBatchAckResult = new HashMap<>();
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ msgBatchAckResult.put(batchAckResult.getReceiptHandleMessage().getMessageId(), batchAckResult);
|
|||
|
|
++ }
|
|||
|
|
++ for (String msgId : broker1Msg) {
|
|||
|
|
++ assertEquals(AckStatus.OK, msgBatchAckResult.get(msgId).getAckResult().getStatus());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(msgId).getProxyException());
|
|||
|
|
++ }
|
|||
|
|
++ for (String msgId : broker2Msg) {
|
|||
|
|
++ assertEquals(AckStatus.NO_EXIST, msgBatchAckResult.get(msgId).getAckResult().getStatus());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(msgId).getProxyException());
|
|||
|
|
++ }
|
|||
|
|
++ assertNotNull(msgBatchAckResult.get(expireMessage.getMsgId()).getProxyException());
|
|||
|
|
++ assertEquals(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, msgBatchAckResult.get(expireMessage.getMsgId()).getProxyException().getCode());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(expireMessage.getMsgId()).getAckResult());
|
|||
|
|
++
|
|||
|
|
++ assertNotNull(msgBatchAckResult.get(errThrowMessage.getMsgId()).getProxyException());
|
|||
|
|
++ assertEquals(ProxyExceptionCode.INTERNAL_SERVER_ERROR, msgBatchAckResult.get(errThrowMessage.getMsgId()).getProxyException().getCode());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(errThrowMessage.getMsgId()).getAckResult());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testChangeInvisibleTime() throws Throwable {
|
|||
|
|
+ ReceiptHandle handle = create(createMessageExt(MixAll.RETRY_GROUP_TOPIC_PREFIX + TOPIC, "", 0, 3000));
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+index 77a119a29..3f3a4ae40 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+@@ -220,6 +220,18 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ assertSame(ackResult, mqClientAPI.ackMessageAsync(BROKER_ADDR, new AckMessageRequestHeader(), TIMEOUT).get());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckMessageAsync() throws Exception {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ doAnswer((Answer<Void>) mock -> {
|
|||
|
|
++ AckCallback ackCallback = mock.getArgument(2);
|
|||
|
|
++ ackCallback.onSuccess(ackResult);
|
|||
|
|
++ return null;
|
|||
|
|
++ }).when(mqClientAPI).batchAckMessageAsync(anyString(), anyLong(), any(AckCallback.class), any());
|
|||
|
|
++
|
|||
|
|
++ assertSame(ackResult, mqClientAPI.batchAckMessageAsync(BROKER_ADDR, TOPIC, CONSUMER_GROUP, new ArrayList<>(), TIMEOUT).get());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testChangeInvisibleTimeAsync() throws Exception {
|
|||
|
|
+ AckResult ackResult = new AckResult();
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch012-backport-enhance-rockdbconfigtojson.patch b/patch012-backport-enhance-rockdbconfigtojson.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..cf6128b24
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch012-backport-enhance-rockdbconfigtojson.patch
|
|||
|
|
@@ -0,0 +1,2920 @@
|
|||
|
|
+From fec141481496c53a0db398367006c34264662d18 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yx9o <yangx_soft@163.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 08:22:34 +0800
|
|||
|
|
+Subject: [PATCH 1/8] [ISSUE #7166] Optimize the display format of admin
|
|||
|
|
+ (#7210)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../java/org/apache/rocketmq/tools/command/MQAdminStartup.java | 2 +-
|
|||
|
|
+ .../command/acl/ClusterAclConfigVersionListSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/acl/DeleteAccessConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/acl/GetAccessConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/acl/UpdateAccessConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/acl/UpdateGlobalWhiteAddrSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/BrokerConsumeStatsSubCommad.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/broker/BrokerStatusSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/CommitLogSetReadAheadSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/DeleteExpiredCommitLogSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/broker/GetBrokerConfigCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/broker/GetBrokerEpochSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java | 2 +-
|
|||
|
|
+ .../broker/RemoveColdDataFlowCtrGroupConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/ResetMasterFlushOffsetSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/broker/UpdateBrokerConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../broker/UpdateColdDataFlowCtrGroupConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/cluster/ClusterListSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/connection/ConsumerConnectionSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/connection/ProducerConnectionSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/consumer/ConsumerStatusSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/consumer/GetConsumerConfigSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/consumer/StartMonitoringSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/consumer/UpdateSubGroupSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/container/AddBrokerSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/container/RemoveBrokerSubCommand.java | 2 +-
|
|||
|
|
+ .../command/controller/CleanControllerBrokerMetaSubCommand.java | 2 +-
|
|||
|
|
+ .../command/controller/GetControllerMetaDataSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/controller/ReElectMasterSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/export/ExportConfigsCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/export/ExportMetadataCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/export/ExportMetricsCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/ha/GetSyncStateSetSubCommand.java | 2 +-
|
|||
|
|
+ .../apache/rocketmq/tools/command/ha/HAStatusSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/CheckMsgSendRTCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/ConsumeMessageCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/message/DumpCompactionLogCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/message/PrintMessageByQueueCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/PrintMessageSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/QueryMsgByIdSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/QueryMsgByKeySubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/message/QueryMsgByOffsetSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/message/QueryMsgByUniqueKeySubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/message/QueryMsgTraceByIdSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/message/SendMessageCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/namesrv/AddWritePermSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/namesrv/WipeWritePermSubCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/offset/SkipAccumulationSubCommand.java | 2 +-
|
|||
|
|
+ .../apache/rocketmq/tools/command/stats/StatsAllSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/AllocateMQSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/TopicClusterSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/TopicListSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/TopicRouteSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/TopicStatusSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/UpdateOrderConfCommand.java | 2 +-
|
|||
|
|
+ .../tools/command/topic/UpdateStaticTopicSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/UpdateTopicPermSubCommand.java | 2 +-
|
|||
|
|
+ .../rocketmq/tools/command/topic/UpdateTopicSubCommand.java | 2 +-
|
|||
|
|
+ 59 files changed, 59 insertions(+), 59 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+index 0c2618e91..890125ca0 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+@@ -278,7 +278,7 @@ public class MQAdminStartup {
|
|||
|
|
+ System.out.printf("The most commonly used mqadmin commands are:%n");
|
|||
|
|
+
|
|||
|
|
+ for (SubCommand cmd : SUB_COMMANDS) {
|
|||
|
|
+- System.out.printf(" %-25s %s%n", cmd.commandName(), cmd.commandDesc());
|
|||
|
|
++ System.out.printf(" %-35s %s%n", cmd.commandName(), cmd.commandDesc());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ System.out.printf("%nSee 'mqadmin help <command>' for more information on a specific command.%n");
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java
|
|||
|
|
+index f8a00b1e0..26ed028fb 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/ClusterAclConfigVersionListSubCommand.java
|
|||
|
|
+@@ -47,7 +47,7 @@ public class ClusterAclConfigVersionListSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "List all of acl config version information in cluster";
|
|||
|
|
++ return "List all of acl config version information in cluster.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/DeleteAccessConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/DeleteAccessConfigSubCommand.java
|
|||
|
|
+index fd3a92fff..a7f3d295a 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/DeleteAccessConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/DeleteAccessConfigSubCommand.java
|
|||
|
|
+@@ -42,7 +42,7 @@ public class DeleteAccessConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Delete Acl Config Account in broker";
|
|||
|
|
++ return "Delete Acl Config Account in broker.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/GetAccessConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/GetAccessConfigSubCommand.java
|
|||
|
|
+index 25844d6a1..f1c9a1496 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/GetAccessConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/GetAccessConfigSubCommand.java
|
|||
|
|
+@@ -49,7 +49,7 @@ public class GetAccessConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "List all of acl config information in cluster";
|
|||
|
|
++ return "List all of acl config information in cluster.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
|
|||
|
|
+index 3be40daa1..d8a06f92d 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateAccessConfigSubCommand.java
|
|||
|
|
+@@ -40,7 +40,7 @@ public class UpdateAccessConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update acl config yaml file in broker";
|
|||
|
|
++ return "Update acl config yaml file in broker.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateGlobalWhiteAddrSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateGlobalWhiteAddrSubCommand.java
|
|||
|
|
+index ff662b506..9dacf1fae 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateGlobalWhiteAddrSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/acl/UpdateGlobalWhiteAddrSubCommand.java
|
|||
|
|
+@@ -37,7 +37,7 @@ public class UpdateGlobalWhiteAddrSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update global white address for acl Config File in broker";
|
|||
|
|
++ return "Update global white address for acl Config File in broker.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java
|
|||
|
|
+index 3f2f90673..7658a2139 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerConsumeStatsSubCommad.java
|
|||
|
|
+@@ -61,7 +61,7 @@ public class BrokerConsumeStatsSubCommad implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch broker consume stats data";
|
|||
|
|
++ return "Fetch broker consume stats data.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommand.java
|
|||
|
|
+index 830ff3425..ce934f547 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/BrokerStatusSubCommand.java
|
|||
|
|
+@@ -44,7 +44,7 @@ public class BrokerStatusSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch broker runtime status data";
|
|||
|
|
++ return "Fetch broker runtime status data.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CommitLogSetReadAheadSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CommitLogSetReadAheadSubCommand.java
|
|||
|
|
+index b00c7f5f5..4fdabfdf8 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CommitLogSetReadAheadSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/CommitLogSetReadAheadSubCommand.java
|
|||
|
|
+@@ -44,7 +44,7 @@ public class CommitLogSetReadAheadSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "set read ahead mode for all commitlog files";
|
|||
|
|
++ return "Set read ahead mode for all commitlog files.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/DeleteExpiredCommitLogSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/DeleteExpiredCommitLogSubCommand.java
|
|||
|
|
+index a4b2a51ad..142bb7b3c 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/DeleteExpiredCommitLogSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/DeleteExpiredCommitLogSubCommand.java
|
|||
|
|
+@@ -37,7 +37,7 @@ public class DeleteExpiredCommitLogSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Delete expired CommitLog files";
|
|||
|
|
++ return "Delete expired CommitLog files.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
|
|||
|
|
+index 5d86c10e4..c4762a296 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java
|
|||
|
|
+@@ -45,7 +45,7 @@ public class GetBrokerConfigCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Get broker config by cluster or special broker";
|
|||
|
|
++ return "Get broker config by cluster or special broker.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerEpochSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerEpochSubCommand.java
|
|||
|
|
+index abe8fc622..1a8961e04 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerEpochSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerEpochSubCommand.java
|
|||
|
|
+@@ -38,7 +38,7 @@ public class GetBrokerEpochSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch broker epoch entries";
|
|||
|
|
++ return "Fetch broker epoch entries.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java
|
|||
|
|
+index 7c54e650c..34b3ba7d3 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java
|
|||
|
|
+@@ -47,7 +47,7 @@ public class GetColdDataFlowCtrInfoSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "get cold data flow ctr info";
|
|||
|
|
++ return "Get cold data flow ctr info.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/RemoveColdDataFlowCtrGroupConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/RemoveColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
+index b0477924f..f20407480 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/RemoveColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/RemoveColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
+@@ -36,7 +36,7 @@ public class RemoveColdDataFlowCtrGroupConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "remove consumer from cold ctr config";
|
|||
|
|
++ return "Remove consumer from cold ctr config.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java
|
|||
|
|
+index b2ac48c84..90451b51f 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/ResetMasterFlushOffsetSubCommand.java
|
|||
|
|
+@@ -33,7 +33,7 @@ public class ResetMasterFlushOffsetSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Reset master flush offset in slave";
|
|||
|
|
++ return "Reset master flush offset in slave.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
|
|||
|
|
+index 98abeb6ae..62816ef03 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateBrokerConfigSubCommand.java
|
|||
|
|
+@@ -37,7 +37,7 @@ public class UpdateBrokerConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update broker's config";
|
|||
|
|
++ return "Update broker's config.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateColdDataFlowCtrGroupConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
+index d06a24b57..8d1a00077 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/UpdateColdDataFlowCtrGroupConfigSubCommand.java
|
|||
|
|
+@@ -39,7 +39,7 @@ public class UpdateColdDataFlowCtrGroupConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "addOrUpdate cold data flow ctr group config";
|
|||
|
|
++ return "Add or update cold data flow ctr group config.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
|
|||
|
|
+index 7253970bd..d755e9e5d 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/CLusterSendMsgRTCommand.java
|
|||
|
|
+@@ -48,7 +48,7 @@ public class CLusterSendMsgRTCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "List All clusters Message Send RT";
|
|||
|
|
++ return "List All clusters Message Send RT.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
|
|||
|
|
+index a7a840a44..ede0fa5cf 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/cluster/ClusterListSubCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class ClusterListSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "List cluster infos";
|
|||
|
|
++ return "List cluster infos.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java
|
|||
|
|
+index 630961e31..35f73d8a0 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommand.java
|
|||
|
|
+@@ -39,7 +39,7 @@ public class ConsumerConnectionSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query consumer's socket connection, client version and subscription";
|
|||
|
|
++ return "Query consumer's socket connection, client version and subscription.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommand.java
|
|||
|
|
+index 2533982c8..bde674ab2 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommand.java
|
|||
|
|
+@@ -36,7 +36,7 @@ public class ProducerConnectionSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query producer's socket connection and client version";
|
|||
|
|
++ return "Query producer's socket connection and client version.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommand.java
|
|||
|
|
+index 72b9c975e..d8f6f9aa9 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommand.java
|
|||
|
|
+@@ -47,7 +47,7 @@ public class ConsumerStatusSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query consumer's internal data structure";
|
|||
|
|
++ return "Query consumer's internal data structure.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
|
|||
|
|
+index 6095e7668..4a8253a02 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommand.java
|
|||
|
|
+@@ -43,7 +43,7 @@ public class GetConsumerConfigSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Get consumer config by subscription group name";
|
|||
|
|
++ return "Get consumer config by subscription group name.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/StartMonitoringSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/StartMonitoringSubCommand.java
|
|||
|
|
+index 2d08d0bd0..f5e140433 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/StartMonitoringSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/StartMonitoringSubCommand.java
|
|||
|
|
+@@ -34,7 +34,7 @@ public class StartMonitoringSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Start Monitoring";
|
|||
|
|
++ return "Start Monitoring.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java
|
|||
|
|
+index f87bafc93..b17da4de4 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class UpdateSubGroupSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update or create subscription group";
|
|||
|
|
++ return "Update or create subscription group.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java
|
|||
|
|
+index e9e5be4a5..007d42ae6 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/container/AddBrokerSubCommand.java
|
|||
|
|
+@@ -33,7 +33,7 @@ public class AddBrokerSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Add a broker to specified container";
|
|||
|
|
++ return "Add a broker to specified container.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java
|
|||
|
|
+index 7c455f858..ab25d8ebe 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/container/RemoveBrokerSubCommand.java
|
|||
|
|
+@@ -33,7 +33,7 @@ public class RemoveBrokerSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Remove a broker from specified container";
|
|||
|
|
++ return "Remove a broker from specified container.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/CleanControllerBrokerMetaSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/CleanControllerBrokerMetaSubCommand.java
|
|||
|
|
+index 856e4b426..24ed02566 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/CleanControllerBrokerMetaSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/CleanControllerBrokerMetaSubCommand.java
|
|||
|
|
+@@ -37,7 +37,7 @@ public class CleanControllerBrokerMetaSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Clean metadata of broker on controller";
|
|||
|
|
++ return "Clean metadata of broker on controller.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/GetControllerMetaDataSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/GetControllerMetaDataSubCommand.java
|
|||
|
|
+index 70bd7f8e9..966443127 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/GetControllerMetaDataSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/GetControllerMetaDataSubCommand.java
|
|||
|
|
+@@ -34,7 +34,7 @@ public class GetControllerMetaDataSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Get controller cluster's metadata";
|
|||
|
|
++ return "Get controller cluster's metadata.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/ReElectMasterSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/ReElectMasterSubCommand.java
|
|||
|
|
+index 1affe81f9..a522a903d 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/controller/ReElectMasterSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/controller/ReElectMasterSubCommand.java
|
|||
|
|
+@@ -37,7 +37,7 @@ public class ReElectMasterSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Re-elect the specified broker as master";
|
|||
|
|
++ return "Re-elect the specified broker as master.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
+index b8191296d..03613b29c 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
+@@ -42,7 +42,7 @@ public class ExportConfigsCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Export configs";
|
|||
|
|
++ return "Export configs.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java
|
|||
|
|
+index 1f9cf7d96..748f7b16e 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java
|
|||
|
|
+@@ -46,7 +46,7 @@ public class ExportMetadataCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Export metadata";
|
|||
|
|
++ return "Export metadata.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java
|
|||
|
|
+index a793b4b84..5d8bb37ba 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java
|
|||
|
|
+@@ -56,7 +56,7 @@ public class ExportMetricsCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Export metrics";
|
|||
|
|
++ return "Export metrics.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/ha/GetSyncStateSetSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/GetSyncStateSetSubCommand.java
|
|||
|
|
+index 44b3ec3e1..b6231e4f9 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/ha/GetSyncStateSetSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/GetSyncStateSetSubCommand.java
|
|||
|
|
+@@ -40,7 +40,7 @@ public class GetSyncStateSetSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch syncStateSet for target brokers";
|
|||
|
|
++ return "Fetch syncStateSet for target brokers.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java
|
|||
|
|
+index b1795e046..931658a08 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/ha/HAStatusSubCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class HAStatusSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch ha runtime status data";
|
|||
|
|
++ return "Fetch ha runtime status data.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/CheckMsgSendRTCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/CheckMsgSendRTCommand.java
|
|||
|
|
+index 4c6d5ffb6..b15b59d50 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/CheckMsgSendRTCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/CheckMsgSendRTCommand.java
|
|||
|
|
+@@ -40,7 +40,7 @@ public class CheckMsgSendRTCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Check message send response time";
|
|||
|
|
++ return "Check message send response time.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommand.java
|
|||
|
|
+index 8aed59ea4..02ff53269 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/ConsumeMessageCommand.java
|
|||
|
|
+@@ -70,7 +70,7 @@ public class ConsumeMessageCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Consume message";
|
|||
|
|
++ return "Consume message.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/DumpCompactionLogCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/DumpCompactionLogCommand.java
|
|||
|
|
+index ae6d9bdcf..eee8f3d4b 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/DumpCompactionLogCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/DumpCompactionLogCommand.java
|
|||
|
|
+@@ -38,7 +38,7 @@ import java.nio.file.Paths;
|
|||
|
|
+ public class DumpCompactionLogCommand implements SubCommand {
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "parse compaction log to message";
|
|||
|
|
++ return "Parse compaction log to message.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageByQueueCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageByQueueCommand.java
|
|||
|
|
+index 654560167..0418e88a7 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageByQueueCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageByQueueCommand.java
|
|||
|
|
+@@ -108,7 +108,7 @@ public class PrintMessageByQueueCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Print Message Detail";
|
|||
|
|
++ return "Print Message Detail by queueId.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageSubCommand.java
|
|||
|
|
+index d01c36d42..bb82f5079 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/PrintMessageSubCommand.java
|
|||
|
|
+@@ -62,7 +62,7 @@ public class PrintMessageSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Print Message Detail";
|
|||
|
|
++ return "Print Message Detail.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByIdSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByIdSubCommand.java
|
|||
|
|
+index 2880477f1..b42612150 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByIdSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByIdSubCommand.java
|
|||
|
|
+@@ -186,7 +186,7 @@ public class QueryMsgByIdSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query Message by Id";
|
|||
|
|
++ return "Query Message by Id.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java
|
|||
|
|
+index ba7b00c3b..64627fd19 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java
|
|||
|
|
+@@ -36,7 +36,7 @@ public class QueryMsgByKeySubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query Message by Key";
|
|||
|
|
++ return "Query Message by Key.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java
|
|||
|
|
+index d27313af1..14d0625fd 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByOffsetSubCommand.java
|
|||
|
|
+@@ -39,7 +39,7 @@ public class QueryMsgByOffsetSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query Message by offset";
|
|||
|
|
++ return "Query Message by offset.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java
|
|||
|
|
+index 1b28f8be1..b71cee901 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java
|
|||
|
|
+@@ -141,7 +141,7 @@ public class QueryMsgByUniqueKeySubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query Message by Unique key";
|
|||
|
|
++ return "Query Message by Unique key.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommand.java
|
|||
|
|
+index 2b982efef..2c546ec56 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgTraceByIdSubCommand.java
|
|||
|
|
+@@ -65,7 +65,7 @@ public class QueryMsgTraceByIdSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Query a message trace";
|
|||
|
|
++ return "Query a message trace.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/SendMessageCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/SendMessageCommand.java
|
|||
|
|
+index 836ee192b..970da6b16 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/SendMessageCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/SendMessageCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class SendMessageCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Send a message";
|
|||
|
|
++ return "Send a message.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommand.java
|
|||
|
|
+index 98542d065..0b0a075bd 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/AddWritePermSubCommand.java
|
|||
|
|
+@@ -34,7 +34,7 @@ public class AddWritePermSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Add write perm of broker in all name server you defined in the -n param";
|
|||
|
|
++ return "Add write perm of broker in all name server you defined in the -n param.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommand.java
|
|||
|
|
+index 213931ed8..637dd52c8 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/namesrv/WipeWritePermSubCommand.java
|
|||
|
|
+@@ -34,7 +34,7 @@ public class WipeWritePermSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Wipe write perm of broker in all name server you defined in the -n param";
|
|||
|
|
++ return "Wipe write perm of broker in all name server you defined in the -n param.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java
|
|||
|
|
+index 139821f9c..b22491a59 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/offset/SkipAccumulationSubCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class SkipAccumulationSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Skip all messages that are accumulated (not consumed) currently";
|
|||
|
|
++ return "Skip all messages that are accumulated (not consumed) currently.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java
|
|||
|
|
+index 1d49bbe11..96097a93e 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/StatsAllSubCommand.java
|
|||
|
|
+@@ -144,7 +144,7 @@ public class StatsAllSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Topic and Consumer tps stats";
|
|||
|
|
++ return "Topic and Consumer tps stats.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/AllocateMQSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/AllocateMQSubCommand.java
|
|||
|
|
+index 3fa42f297..6a9b81eb8 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/AllocateMQSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/AllocateMQSubCommand.java
|
|||
|
|
+@@ -41,7 +41,7 @@ public class AllocateMQSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Allocate MQ";
|
|||
|
|
++ return "Allocate MQ.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicClusterSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicClusterSubCommand.java
|
|||
|
|
+index 1dab693d9..098f34ff0 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicClusterSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicClusterSubCommand.java
|
|||
|
|
+@@ -34,7 +34,7 @@ public class TopicClusterSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Get cluster info for topic";
|
|||
|
|
++ return "Get cluster info for topic.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java
|
|||
|
|
+index 346bac704..d9a279f80 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicListSubCommand.java
|
|||
|
|
+@@ -45,7 +45,7 @@ public class TopicListSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Fetch all topic list from name server";
|
|||
|
|
++ return "Fetch all topic list from name server.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicRouteSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicRouteSubCommand.java
|
|||
|
|
+index f2dabec4e..70949d388 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicRouteSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicRouteSubCommand.java
|
|||
|
|
+@@ -42,7 +42,7 @@ public class TopicRouteSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Examine topic route info";
|
|||
|
|
++ return "Examine topic route info.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicStatusSubCommand.java
|
|||
|
|
+index fdb249fab..a1619eced 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicStatusSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/TopicStatusSubCommand.java
|
|||
|
|
+@@ -40,7 +40,7 @@ public class TopicStatusSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Examine topic Status info";
|
|||
|
|
++ return "Examine topic Status info.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateOrderConfCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateOrderConfCommand.java
|
|||
|
|
+index bebc646b4..3040d04c2 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateOrderConfCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateOrderConfCommand.java
|
|||
|
|
+@@ -36,7 +36,7 @@ public class UpdateOrderConfCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Create or update or delete order conf";
|
|||
|
|
++ return "Create or update or delete order conf.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateStaticTopicSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateStaticTopicSubCommand.java
|
|||
|
|
+index 85a18c654..3daeee86c 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateStaticTopicSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateStaticTopicSubCommand.java
|
|||
|
|
+@@ -48,7 +48,7 @@ public class UpdateStaticTopicSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update or create static topic, which has fixed number of queues";
|
|||
|
|
++ return "Update or create static topic, which has fixed number of queues.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicPermSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicPermSubCommand.java
|
|||
|
|
+index aaa881538..d27cd1861 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicPermSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicPermSubCommand.java
|
|||
|
|
+@@ -44,7 +44,7 @@ public class UpdateTopicPermSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update topic perm";
|
|||
|
|
++ return "Update topic perm.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java
|
|||
|
|
+index b68463396..298914175 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/topic/UpdateTopicSubCommand.java
|
|||
|
|
+@@ -42,7 +42,7 @@ public class UpdateTopicSubCommand implements SubCommand {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+- return "Update or create topic";
|
|||
|
|
++ return "Update or create topic.";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 744167bd01fab6821b4d5ae1794dc845153d5156 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <tanziyi0925@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 08:32:17 +0800
|
|||
|
|
+Subject: [PATCH 2/8] [ISSUE #7142] Add command `RocksDBConfigToJson` to
|
|||
|
|
+ inspect rocksdb content (#7180)
|
|||
|
|
+
|
|||
|
|
+* feat: add command `RocksDBConfigToJson` to inspect rocksdb content
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+
|
|||
|
|
+* refactor: fix style
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+Co-authored-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../tools/command/MQAdminStartup.java | 2 +
|
|||
|
|
+ .../metadata/RocksDBConfigToJsonCommand.java | 118 ++++++++++++++++++
|
|||
|
|
+ .../metadata/KvConfigToJsonCommandTest.java | 65 ++++++++++
|
|||
|
|
+ 3 files changed, 185 insertions(+)
|
|||
|
|
+ create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+ create mode 100644 tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+index 890125ca0..324aa1856 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+@@ -80,6 +80,7 @@ import org.apache.rocketmq.tools.command.message.QueryMsgByOffsetSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgByUniqueKeySubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgTraceByIdSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.SendMessageCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.metadata.RocksDBConfigToJsonCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.AddWritePermSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.DeleteKvConfigCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.GetNamesrvConfigCommand;
|
|||
|
|
+@@ -211,6 +212,7 @@ public class MQAdminStartup {
|
|||
|
|
+
|
|||
|
|
+ initCommand(new ClusterListSubCommand());
|
|||
|
|
+ initCommand(new TopicListSubCommand());
|
|||
|
|
++ initCommand(new RocksDBConfigToJsonCommand());
|
|||
|
|
+
|
|||
|
|
+ initCommand(new UpdateKvConfigCommand());
|
|||
|
|
+ initCommand(new DeleteKvConfigCommand());
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..3053f4684
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+@@ -0,0 +1,118 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tools.command.metadata;
|
|||
|
|
++
|
|||
|
|
++import com.alibaba.fastjson.JSONObject;
|
|||
|
|
++import org.apache.commons.cli.CommandLine;
|
|||
|
|
++import org.apache.commons.cli.Option;
|
|||
|
|
++import org.apache.commons.cli.Options;
|
|||
|
|
++import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.common.config.RocksDBConfigManager;
|
|||
|
|
++import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
++import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++
|
|||
|
|
++import java.io.File;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
++
|
|||
|
|
++public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
++ private static final String TOPICS_JSON_CONFIG = "topics";
|
|||
|
|
++ private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups";
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandName() {
|
|||
|
|
++ return "rocksDBConfigToJson";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandDesc() {
|
|||
|
|
++ return "Convert RocksDB kv config (topics/subscriptionGroups) to json";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Options buildCommandlineOptions(Options options) {
|
|||
|
|
++ Option pathOption = new Option("p", "path", true,
|
|||
|
|
++ "Absolute path to the metadata directory");
|
|||
|
|
++ pathOption.setRequired(true);
|
|||
|
|
++ options.addOption(pathOption);
|
|||
|
|
++
|
|||
|
|
++ Option configTypeOption = new Option("t", "configType", true, "Name of kv config, e.g. " +
|
|||
|
|
++ "topics/subscriptionGroups");
|
|||
|
|
++ configTypeOption.setRequired(true);
|
|||
|
|
++ options.addOption(configTypeOption);
|
|||
|
|
++
|
|||
|
|
++ return options;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
++ String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
++ if (StringUtils.isEmpty(path) || !new File(path).exists()) {
|
|||
|
|
++ System.out.print("Rocksdb path is invalid.\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
++
|
|||
|
|
++ final long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
++ RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(memTableFlushInterval);
|
|||
|
|
++ try {
|
|||
|
|
++ if (TOPICS_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
++ // for topics.json
|
|||
|
|
++ final Map<String, JSONObject> topicsJsonConfig = new HashMap<>();
|
|||
|
|
++ final Map<String, JSONObject> topicConfigTable = new HashMap<>();
|
|||
|
|
++ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
++ final String topic = new String(key, DataConverter.charset);
|
|||
|
|
++ final String topicConfig = new String(value, DataConverter.charset);
|
|||
|
|
++ final JSONObject jsonObject = JSONObject.parseObject(topicConfig);
|
|||
|
|
++ topicConfigTable.put(topic, jsonObject);
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ if (isLoad) {
|
|||
|
|
++ topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
++ final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
++ System.out.print(topicsJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ if (SUBSCRIPTION_GROUP_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
++ // for subscriptionGroup.json
|
|||
|
|
++ final Map<String, JSONObject> subscriptionGroupJsonConfig = new HashMap<>();
|
|||
|
|
++ final Map<String, JSONObject> subscriptionGroupTable = new HashMap<>();
|
|||
|
|
++ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
++ final String subscriptionGroup = new String(key, DataConverter.charset);
|
|||
|
|
++ final String subscriptionGroupConfig = new String(value, DataConverter.charset);
|
|||
|
|
++ final JSONObject jsonObject = JSONObject.parseObject(subscriptionGroupConfig);
|
|||
|
|
++ subscriptionGroupTable.put(subscriptionGroup, jsonObject);
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ if (isLoad) {
|
|||
|
|
++ subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
++ (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
++ final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
++ System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ System.out.print("Config type was not recognized, configType=" + configType + "\n");
|
|||
|
|
++ } finally {
|
|||
|
|
++ kvConfigManager.stop();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..b2f66c7b0
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+@@ -0,0 +1,65 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tools.command.metadata;
|
|||
|
|
++
|
|||
|
|
++import org.apache.commons.cli.CommandLine;
|
|||
|
|
++import org.apache.commons.cli.DefaultParser;
|
|||
|
|
++import org.apache.commons.cli.Options;
|
|||
|
|
++import org.apache.rocketmq.srvutil.ServerUtil;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++import org.junit.Test;
|
|||
|
|
++
|
|||
|
|
++import java.io.File;
|
|||
|
|
++
|
|||
|
|
++import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
++
|
|||
|
|
++public class KvConfigToJsonCommandTest {
|
|||
|
|
++ private static final String BASE_PATH = System.getProperty("user.home") + File.separator + "store/config/";
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testExecute() throws SubCommandException {
|
|||
|
|
++ {
|
|||
|
|
++ String[] cases = new String[]{"topics", "subscriptionGroups"};
|
|||
|
|
++ for (String c : cases) {
|
|||
|
|
++ RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
++ String[] subargs = new String[]{"-p " + BASE_PATH + c, "-t " + c};
|
|||
|
|
++ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs,
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.execute(commandLine, options, null);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("p").trim()).isEqualTo(BASE_PATH + c);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("t").trim()).isEqualTo(c);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ // invalid cases
|
|||
|
|
++ {
|
|||
|
|
++ String[][] cases = new String[][]{
|
|||
|
|
++ {"-p " + BASE_PATH + "tmpPath", "-t topics"},
|
|||
|
|
++ {"-p ", "-t topics"},
|
|||
|
|
++ {"-p " + BASE_PATH + "topics", "-t invalid_type"}
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ for (String[] c : cases) {
|
|||
|
|
++ RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
++ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), c,
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.execute(commandLine, options, null);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From bdede35db365a49b211cdc249c68b0f60a3df46d Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: mxsm <ljbmxsm@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 08:34:56 +0800
|
|||
|
|
+Subject: [PATCH 3/8] [ISSUE #7124] Fix the typos in the code comments (#7125)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/broker/processor/ReplyMessageProcessor.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
+index b2db356c8..d3bb048f7 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ReplyMessageProcessor.java
|
|||
|
|
+@@ -234,7 +234,7 @@ public class ReplyMessageProcessor extends AbstractSendMessageProcessor {
|
|||
|
|
+ } else {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ response.setRemark(null);
|
|||
|
|
+- //set to zore to avoid client decoding exception
|
|||
|
|
++ //set to zero to avoid client decoding exception
|
|||
|
|
+ responseHeader.setMsgId("0");
|
|||
|
|
+ responseHeader.setQueueId(queueIdInt);
|
|||
|
|
+ responseHeader.setQueueOffset(0L);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 9bb73b9a38548b99ac5126c40380c3c2e7fc586e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 09:46:27 +0800
|
|||
|
|
+Subject: [PATCH 4/8] [#ISSUE 7222] Bug fix and refactoring of the Indexfile in
|
|||
|
|
+ tiered storage (#7224)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/file/TieredIndexFile.java | 38 +++++++--
|
|||
|
|
+ .../tieredstore/file/TieredIndexFileTest.java | 84 +++++--------------
|
|||
|
|
+ 2 files changed, 52 insertions(+), 70 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
+index 50beb01ae..eda5e0106 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredIndexFile.java
|
|||
|
|
+@@ -16,6 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
++import com.google.common.annotations.VisibleForTesting;
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+@@ -99,7 +100,7 @@ public class TieredIndexFile {
|
|||
|
|
+ this::doScheduleTask, 10, 10, TimeUnit.SECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private void doScheduleTask() {
|
|||
|
|
++ protected void doScheduleTask() {
|
|||
|
|
+ try {
|
|||
|
|
+ curFileLock.lock();
|
|||
|
|
+ try {
|
|||
|
|
+@@ -145,6 +146,11 @@ public class TieredIndexFile {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @VisibleForTesting
|
|||
|
|
++ public MappedFile getPreMappedFile() {
|
|||
|
|
++ return preMappedFile;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private void initFile() throws IOException {
|
|||
|
|
+ curMappedFile = new DefaultMappedFile(curFilePath, fileMaxSize);
|
|||
|
|
+ initIndexFileHeader(curMappedFile);
|
|||
|
|
+@@ -156,19 +162,26 @@ public class TieredIndexFile {
|
|||
|
|
+
|
|||
|
|
+ if (isFileSealed(curMappedFile)) {
|
|||
|
|
+ if (preFileExists) {
|
|||
|
|
+- preFile.delete();
|
|||
|
|
++ if (preFile.delete()) {
|
|||
|
|
++ logger.info("Pre IndexFile deleted success", preFilepath);
|
|||
|
|
++ } else {
|
|||
|
|
++ logger.error("Pre IndexFile deleted failed", preFilepath);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ boolean rename = curMappedFile.renameTo(preFilepath);
|
|||
|
|
+ if (rename) {
|
|||
|
|
+ preMappedFile = curMappedFile;
|
|||
|
|
+ curMappedFile = new DefaultMappedFile(curFilePath, fileMaxSize);
|
|||
|
|
++ initIndexFileHeader(curMappedFile);
|
|||
|
|
+ preFileExists = true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ if (preFileExists) {
|
|||
|
|
+ synchronized (TieredIndexFile.class) {
|
|||
|
|
+ if (inflightCompactFuture.isDone()) {
|
|||
|
|
+- inflightCompactFuture = TieredStoreExecutor.compactIndexFileExecutor.submit(new CompactTask(storeConfig, preMappedFile, flatFile), null);
|
|||
|
|
++ inflightCompactFuture = TieredStoreExecutor.compactIndexFileExecutor.submit(
|
|||
|
|
++ new CompactTask(storeConfig, preMappedFile, flatFile), null);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -261,7 +274,8 @@ public class TieredIndexFile {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public CompletableFuture<List<Pair<Long, ByteBuffer>>> queryAsync(String topic, String key, long beginTime, long endTime) {
|
|||
|
|
++ public CompletableFuture<List<Pair<Long, ByteBuffer>>> queryAsync(String topic, String key, long beginTime,
|
|||
|
|
++ long endTime) {
|
|||
|
|
+ int hashCode = indexKeyHashMethod(buildKey(topic, key));
|
|||
|
|
+ int slotPosition = hashCode % maxHashSlotNum;
|
|||
|
|
+ List<TieredFileSegment> fileSegmentList = flatFile.getFileListByTime(beginTime, endTime);
|
|||
|
|
+@@ -355,7 +369,7 @@ public class TieredIndexFile {
|
|||
|
|
+ private final int fileMaxSize;
|
|||
|
|
+ private MappedFile originFile;
|
|||
|
|
+ private TieredFlatFile fileQueue;
|
|||
|
|
+- private final MappedFile compactFile;
|
|||
|
|
++ private MappedFile compactFile;
|
|||
|
|
+
|
|||
|
|
+ public CompactTask(TieredMessageStoreConfig storeConfig, MappedFile originFile,
|
|||
|
|
+ TieredFlatFile fileQueue) throws IOException {
|
|||
|
|
+@@ -381,6 +395,17 @@ public class TieredIndexFile {
|
|||
|
|
+ } catch (Throwable throwable) {
|
|||
|
|
+ logger.error("TieredIndexFile#compactTask: compact index file failed:", throwable);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ try {
|
|||
|
|
++ if (originFile != null) {
|
|||
|
|
++ originFile.destroy(-1);
|
|||
|
|
++ }
|
|||
|
|
++ if (compactFile != null) {
|
|||
|
|
++ compactFile.destroy(-1);
|
|||
|
|
++ }
|
|||
|
|
++ } catch (Throwable throwable) {
|
|||
|
|
++ logger.error("TieredIndexFile#compactTask: destroy index file failed:", throwable);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void compact() {
|
|||
|
|
+@@ -396,6 +421,8 @@ public class TieredIndexFile {
|
|||
|
|
+ fileQueue.commit(true);
|
|||
|
|
+ compactFile.destroy(-1);
|
|||
|
|
+ originFile.destroy(-1);
|
|||
|
|
++ compactFile = null;
|
|||
|
|
++ originFile = null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCompactFile() {
|
|||
|
|
+@@ -414,6 +441,7 @@ public class TieredIndexFile {
|
|||
|
|
+ if (slotValue != -1) {
|
|||
|
|
+ int indexTotalSize = 0;
|
|||
|
|
+ int indexPosition = slotValue;
|
|||
|
|
++
|
|||
|
|
+ while (indexPosition >= 0 && indexPosition < maxIndexNum) {
|
|||
|
|
+ int indexOffset = INDEX_FILE_HEADER_SIZE + maxHashSlotNum * INDEX_FILE_HASH_SLOT_SIZE
|
|||
|
|
+ + indexPosition * INDEX_FILE_HASH_ORIGIN_INDEX_SIZE;
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+index 7ef49578d..262d6645b 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+@@ -19,9 +19,8 @@ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+ import com.sun.jna.Platform;
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
++import java.time.Duration;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-import org.apache.commons.lang3.SystemUtils;
|
|||
|
|
+ import org.apache.commons.lang3.tuple.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.TieredStoreTestUtil;
|
|||
|
|
+@@ -31,9 +30,7 @@ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+ import org.awaitility.Awaitility;
|
|||
|
|
+ import org.junit.After;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+-import org.junit.Assume;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+-import org.junit.Ignore;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ public class TieredIndexFileTest {
|
|||
|
|
+@@ -45,11 +42,12 @@ public class TieredIndexFileTest {
|
|||
|
|
+ @Before
|
|||
|
|
+ public void setUp() {
|
|||
|
|
+ storeConfig = new TieredMessageStoreConfig();
|
|||
|
|
++ storeConfig.setBrokerName("IndexFileBroker");
|
|||
|
|
+ storeConfig.setStorePathRootDir(storePath);
|
|||
|
|
+- storeConfig.setTieredBackendServiceProvider("org.apache.rocketmq.tieredstore.provider.memory.MemoryFileSegment");
|
|||
|
|
+- storeConfig.setTieredStoreIndexFileMaxHashSlotNum(2);
|
|||
|
|
+- storeConfig.setTieredStoreIndexFileMaxIndexNum(3);
|
|||
|
|
+- mq = new MessageQueue("TieredIndexFileTest", storeConfig.getBrokerName(), 1);
|
|||
|
|
++ storeConfig.setTieredBackendServiceProvider("org.apache.rocketmq.tieredstore.provider.posix.PosixFileSegment");
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileMaxHashSlotNum(5);
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileMaxIndexNum(20);
|
|||
|
|
++ mq = new MessageQueue("IndexFileTest", storeConfig.getBrokerName(), 1);
|
|||
|
|
+ TieredStoreUtil.getMetadataStore(storeConfig);
|
|||
|
|
+ TieredStoreExecutor.init();
|
|||
|
|
+ }
|
|||
|
|
+@@ -61,77 +59,33 @@ public class TieredIndexFileTest {
|
|||
|
|
+ TieredStoreExecutor.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Ignore
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testAppendAndQuery() throws IOException, ClassNotFoundException, NoSuchMethodException {
|
|||
|
|
+ if (Platform.isWindows()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // skip this test on windows
|
|||
|
|
+- Assume.assumeFalse(SystemUtils.IS_OS_WINDOWS);
|
|||
|
|
+-
|
|||
|
|
+ TieredFileAllocator fileQueueFactory = new TieredFileAllocator(storeConfig);
|
|||
|
|
+ TieredIndexFile indexFile = new TieredIndexFile(fileQueueFactory, storePath);
|
|||
|
|
++
|
|||
|
|
+ indexFile.append(mq, 0, "key3", 3, 300, 1000);
|
|||
|
|
+ indexFile.append(mq, 0, "key2", 2, 200, 1100);
|
|||
|
|
+ indexFile.append(mq, 0, "key1", 1, 100, 1200);
|
|||
|
|
+
|
|||
|
|
+- Awaitility.waitAtMost(5, TimeUnit.SECONDS)
|
|||
|
|
+- .until(() -> {
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
+- if (indexList.size() != 1) {
|
|||
|
|
+- return false;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 2, indexBuffer.remaining());
|
|||
|
|
+-
|
|||
|
|
+- Assert.assertEquals(1, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(100, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(200, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
+-
|
|||
|
|
+- Assert.assertEquals(3, indexBuffer.getLong(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(300, indexBuffer.getInt(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE + 4 + 4 + 4 + 8 + 4));
|
|||
|
|
+- return true;
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+- indexFile.append(mq, 0, "key4", 4, 400, 1300);
|
|||
|
|
+-
|
|||
|
|
+- Awaitility.waitAtMost(5, TimeUnit.SECONDS)
|
|||
|
|
+- .until(() -> {
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key4", 1300, 1300).join();
|
|||
|
|
+- if (indexList.size() != 1) {
|
|||
|
|
+- return false;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 3, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(4, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(400, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
+- return true;
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- List<Pair<Long, ByteBuffer>> indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1300, 1300).join();
|
|||
|
|
++ // do not do schedule task here
|
|||
|
|
++ TieredStoreExecutor.shutdown();
|
|||
|
|
++ List<Pair<Long, ByteBuffer>> indexList =
|
|||
|
|
++ indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
+ Assert.assertEquals(0, indexList.size());
|
|||
|
|
+
|
|||
|
|
+- indexList = indexFile.queryAsync(mq.getTopic(), "key4", 1200, 1300).join();
|
|||
|
|
+- Assert.assertEquals(2, indexList.size());
|
|||
|
|
+-
|
|||
|
|
+- ByteBuffer indexBuffer = indexList.get(0).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE * 3, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(4, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(400, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(0, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
++ // do compaction once
|
|||
|
|
++ TieredStoreExecutor.init();
|
|||
|
|
++ storeConfig.setTieredStoreIndexFileRollingIdleInterval(0);
|
|||
|
|
++ indexFile.doScheduleTask();
|
|||
|
|
++ Awaitility.await().atMost(Duration.ofSeconds(10))
|
|||
|
|
++ .until(() -> !indexFile.getPreMappedFile().getFile().exists());
|
|||
|
|
+
|
|||
|
|
+- indexBuffer = indexList.get(1).getValue();
|
|||
|
|
+- Assert.assertEquals(TieredIndexFile.INDEX_FILE_HASH_COMPACT_INDEX_SIZE, indexBuffer.remaining());
|
|||
|
|
+- Assert.assertEquals(2, indexBuffer.getLong(4 + 4 + 4));
|
|||
|
|
+- Assert.assertEquals(200, indexBuffer.getInt(4 + 4 + 4 + 8));
|
|||
|
|
+- Assert.assertEquals(100, indexBuffer.getInt(4 + 4 + 4 + 8 + 4));
|
|||
|
|
++ indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
++ Assert.assertEquals(1, indexList.size());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 69c26d3d29cde7b4484ecd112ab9224f9f42bf45 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 10:27:52 +0800
|
|||
|
|
+Subject: [PATCH 5/8] [ISSUE #7228] Converge the use of some important
|
|||
|
|
+ variables for some class
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/store/ConsumeQueue.java | 16 ++++++------
|
|||
|
|
+ .../rocketmq/store/MappedFileQueue.java | 26 +++++++++++--------
|
|||
|
|
+ .../store/MultiPathMappedFileQueue.java | 4 +--
|
|||
|
|
+ 3 files changed, 24 insertions(+), 22 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
+index a0b886eb0..56bee2af3 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
+@@ -145,7 +145,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ if (offset >= 0 && size > 0) {
|
|||
|
|
+ mappedFileOffset = i + CQ_STORE_UNIT_SIZE;
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+ }
|
|||
|
|
+@@ -409,7 +409,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ int logicFileSize = this.mappedFileSize;
|
|||
|
|
+
|
|||
|
|
+- this.maxPhysicOffset = phyOffset;
|
|||
|
|
++ this.setMaxPhysicOffset(phyOffset);
|
|||
|
|
+ long maxExtAddr = 1;
|
|||
|
|
+ boolean shouldDeleteFile = false;
|
|||
|
|
+ while (true) {
|
|||
|
|
+@@ -435,7 +435,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ mappedFile.setWrotePosition(pos);
|
|||
|
|
+ mappedFile.setCommittedPosition(pos);
|
|||
|
|
+ mappedFile.setFlushedPosition(pos);
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ // This maybe not take effect, when not every consume queue has extend file.
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+@@ -453,7 +453,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ mappedFile.setWrotePosition(pos);
|
|||
|
|
+ mappedFile.setCommittedPosition(pos);
|
|||
|
|
+ mappedFile.setFlushedPosition(pos);
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ if (isExtAddr(tagsCode)) {
|
|||
|
|
+ maxExtAddr = tagsCode;
|
|||
|
|
+ }
|
|||
|
|
+@@ -881,8 +881,8 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode,
|
|||
|
|
+ final long cqOffset) {
|
|||
|
|
+
|
|||
|
|
+- if (offset + size <= this.maxPhysicOffset) {
|
|||
|
|
+- log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", maxPhysicOffset, offset);
|
|||
|
|
++ if (offset + size <= this.getMaxPhysicOffset()) {
|
|||
|
|
++ log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", this.getMaxPhysicOffset(), offset);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -926,7 +926,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+ );
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+- this.maxPhysicOffset = offset + size;
|
|||
|
|
++ this.setMaxPhysicOffset(offset + size);
|
|||
|
|
+ return mappedFile.appendMessage(this.byteBufferIndex.array());
|
|||
|
|
+ }
|
|||
|
|
+ return false;
|
|||
|
|
+@@ -1130,7 +1130,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void destroy() {
|
|||
|
|
+- this.maxPhysicOffset = -1;
|
|||
|
|
++ this.setMaxPhysicOffset(-1);
|
|||
|
|
+ this.minLogicOffset = 0;
|
|||
|
|
+ this.mappedFileQueue.destroy();
|
|||
|
|
+ if (isExtReadEnable()) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+index 0bc70642f..32b90d14f 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+@@ -285,7 +285,7 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ if (this.mappedFiles.isEmpty())
|
|||
|
|
+ return 0;
|
|||
|
|
+
|
|||
|
|
+- long committed = this.flushedWhere;
|
|||
|
|
++ long committed = this.getFlushedWhere();
|
|||
|
|
+ if (committed != 0) {
|
|||
|
|
+ MappedFile mappedFile = this.getLastMappedFile(0, false);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+@@ -442,11 +442,11 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long remainHowManyDataToCommit() {
|
|||
|
|
+- return getMaxWrotePosition() - committedWhere;
|
|||
|
|
++ return getMaxWrotePosition() - getCommittedWhere();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long remainHowManyDataToFlush() {
|
|||
|
|
+- return getMaxOffset() - flushedWhere;
|
|||
|
|
++ return getMaxOffset() - this.getFlushedWhere();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void deleteLastMappedFile() {
|
|||
|
|
+@@ -616,15 +616,15 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+
|
|||
|
|
+ public boolean flush(final int flushLeastPages) {
|
|||
|
|
+ boolean result = true;
|
|||
|
|
+- MappedFile mappedFile = this.findMappedFileByOffset(this.flushedWhere, this.flushedWhere == 0);
|
|||
|
|
++ MappedFile mappedFile = this.findMappedFileByOffset(this.getFlushedWhere(), this.getFlushedWhere() == 0);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+ long tmpTimeStamp = mappedFile.getStoreTimestamp();
|
|||
|
|
+ int offset = mappedFile.flush(flushLeastPages);
|
|||
|
|
+ long where = mappedFile.getFileFromOffset() + offset;
|
|||
|
|
+- result = where == this.flushedWhere;
|
|||
|
|
+- this.flushedWhere = where;
|
|||
|
|
++ result = where == this.getFlushedWhere();
|
|||
|
|
++ this.setFlushedWhere(where);
|
|||
|
|
+ if (0 == flushLeastPages) {
|
|||
|
|
+- this.storeTimestamp = tmpTimeStamp;
|
|||
|
|
++ this.setStoreTimestamp(tmpTimeStamp);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -633,12 +633,12 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+
|
|||
|
|
+ public synchronized boolean commit(final int commitLeastPages) {
|
|||
|
|
+ boolean result = true;
|
|||
|
|
+- MappedFile mappedFile = this.findMappedFileByOffset(this.committedWhere, this.committedWhere == 0);
|
|||
|
|
++ MappedFile mappedFile = this.findMappedFileByOffset(this.getCommittedWhere(), this.getCommittedWhere() == 0);
|
|||
|
|
+ if (mappedFile != null) {
|
|||
|
|
+ int offset = mappedFile.commit(commitLeastPages);
|
|||
|
|
+ long where = mappedFile.getFileFromOffset() + offset;
|
|||
|
|
+- result = where == this.committedWhere;
|
|||
|
|
+- this.committedWhere = where;
|
|||
|
|
++ result = where == this.getCommittedWhere();
|
|||
|
|
++ this.setCommittedWhere(where);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return result;
|
|||
|
|
+@@ -763,7 +763,7 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ mf.destroy(1000 * 3);
|
|||
|
|
+ }
|
|||
|
|
+ this.mappedFiles.clear();
|
|||
|
|
+- this.flushedWhere = 0;
|
|||
|
|
++ this.setFlushedWhere(0);
|
|||
|
|
+
|
|||
|
|
+ // delete parent directory
|
|||
|
|
+ File file = new File(storePath);
|
|||
|
|
+@@ -848,6 +848,10 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ return storeTimestamp;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void setStoreTimestamp(long storeTimestamp) {
|
|||
|
|
++ this.storeTimestamp = storeTimestamp;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public List<MappedFile> getMappedFiles() {
|
|||
|
|
+ return mappedFiles;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
+index 8f5af9438..8ff050dfe 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
|
|||
|
|
+@@ -16,7 +16,6 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.store;
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ import java.util.Arrays;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+@@ -113,8 +112,7 @@ public class MultiPathMappedFileQueue extends MappedFileQueue {
|
|||
|
|
+ mf.destroy(1000 * 3);
|
|||
|
|
+ }
|
|||
|
|
+ this.mappedFiles.clear();
|
|||
|
|
+- this.flushedWhere = 0;
|
|||
|
|
+-
|
|||
|
|
++ this.setFlushedWhere(0);
|
|||
|
|
+
|
|||
|
|
+ Set<String> storePathSet = getPaths();
|
|||
|
|
+ storePathSet.addAll(getReadonlyPaths());
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3884f595949462044c5cb3c236199bc1d7ad2341 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: =?UTF-8?q?=E7=9F=B3=E8=87=BB=E8=87=BB=28Steven=20shi=29?=
|
|||
|
|
+ <shirenchuang@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 11:10:30 +0800
|
|||
|
|
+Subject: [PATCH 6/8] [ISSUE #7149] When creating and updating Topic, there
|
|||
|
|
+ will be problems with permission settings (#7151)
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+* [ISSUE #7149] fix bug : When creating and updating Topic, there will be problems with permission settings
|
|||
|
|
+
|
|||
|
|
+* [ISSUE #7149] fix bug : When creating and updating Topic, there will be problems with permission settings
|
|||
|
|
+
|
|||
|
|
+* [issue#7249]
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 十真 <shirenchuang.src@cainiao.com>
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/broker/BrokerController.java | 3 ++-
|
|||
|
|
+ 1 file changed, 2 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 13f9d002b..e8f943702 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1733,7 +1733,8 @@ public class BrokerController {
|
|||
|
|
+ new TopicConfig(topicConfig.getTopicName(),
|
|||
|
|
+ topicConfig.getReadQueueNums(),
|
|||
|
|
+ topicConfig.getWriteQueueNums(),
|
|||
|
|
+- this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
|
|||
|
|
++ topicConfig.getPerm()
|
|||
|
|
++ & this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
|
|||
|
|
+ } else {
|
|||
|
|
+ registerTopicConfig = new TopicConfig(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 017ad110475e8024585327b44f47e5e97aabc63b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: echooymxq <echooy.mxq@gmail.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 11:11:42 +0800
|
|||
|
|
+Subject: [PATCH 7/8] [ISSUE #7219] Fix Concurrent modify syncStateSet and Mark
|
|||
|
|
+ synchronizing frequently when shrink. (#7220)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/controller/ReplicasManager.java | 29 ++++++++++---------
|
|||
|
|
+ .../ha/autoswitch/AutoSwitchHAService.java | 21 ++++++++------
|
|||
|
|
+ 2 files changed, 28 insertions(+), 22 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+index abae7cdb0..37c82e434 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+@@ -542,7 +542,7 @@ public class ReplicasManager {
|
|||
|
|
+ this.brokerMetadata.updateAndPersist(brokerConfig.getBrokerClusterName(), brokerConfig.getBrokerName(), tempBrokerMetadata.getBrokerId());
|
|||
|
|
+ this.tempBrokerMetadata.clear();
|
|||
|
|
+ this.brokerControllerId = this.brokerMetadata.getBrokerId();
|
|||
|
|
+- this.haService.setBrokerControllerId(this.brokerControllerId);
|
|||
|
|
++ this.haService.setLocalBrokerId(this.brokerControllerId);
|
|||
|
|
+ return true;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ LOGGER.error("fail to create metadata file", e);
|
|||
|
|
+@@ -594,7 +594,7 @@ public class ReplicasManager {
|
|||
|
|
+ if (this.brokerMetadata.isLoaded()) {
|
|||
|
|
+ this.registerState = RegisterState.CREATE_METADATA_FILE_DONE;
|
|||
|
|
+ this.brokerControllerId = brokerMetadata.getBrokerId();
|
|||
|
|
+- this.haService.setBrokerControllerId(this.brokerControllerId);
|
|||
|
|
++ this.haService.setLocalBrokerId(this.brokerControllerId);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ // 2. check if temp metadata exist
|
|||
|
|
+@@ -735,23 +735,26 @@ public class ReplicasManager {
|
|||
|
|
+ if (this.checkSyncStateSetTaskFuture != null) {
|
|||
|
|
+ this.checkSyncStateSetTaskFuture.cancel(false);
|
|||
|
|
+ }
|
|||
|
|
+- this.checkSyncStateSetTaskFuture = this.scheduledService.scheduleAtFixedRate(() -> {
|
|||
|
|
+- checkSyncStateSetAndDoReport();
|
|||
|
|
+- }, 3 * 1000, this.brokerConfig.getCheckSyncStateSetPeriod(), TimeUnit.MILLISECONDS);
|
|||
|
|
++ this.checkSyncStateSetTaskFuture = this.scheduledService.scheduleAtFixedRate(this::checkSyncStateSetAndDoReport, 3 * 1000,
|
|||
|
|
++ this.brokerConfig.getCheckSyncStateSetPeriod(), TimeUnit.MILLISECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void checkSyncStateSetAndDoReport() {
|
|||
|
|
+- final Set<Long> newSyncStateSet = this.haService.maybeShrinkSyncStateSet();
|
|||
|
|
+- newSyncStateSet.add(this.brokerControllerId);
|
|||
|
|
+- synchronized (this) {
|
|||
|
|
+- if (this.syncStateSet != null) {
|
|||
|
|
+- // Check if syncStateSet changed
|
|||
|
|
+- if (this.syncStateSet.size() == newSyncStateSet.size() && this.syncStateSet.containsAll(newSyncStateSet)) {
|
|||
|
|
+- return;
|
|||
|
|
++ try {
|
|||
|
|
++ final Set<Long> newSyncStateSet = this.haService.maybeShrinkSyncStateSet();
|
|||
|
|
++ newSyncStateSet.add(this.brokerControllerId);
|
|||
|
|
++ synchronized (this) {
|
|||
|
|
++ if (this.syncStateSet != null) {
|
|||
|
|
++ // Check if syncStateSet changed
|
|||
|
|
++ if (this.syncStateSet.size() == newSyncStateSet.size() && this.syncStateSet.containsAll(newSyncStateSet)) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++ doReportSyncStateSetChanged(newSyncStateSet);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ LOGGER.error("Check syncStateSet error", e);
|
|||
|
|
+ }
|
|||
|
|
+- doReportSyncStateSetChanged(newSyncStateSet);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void doReportSyncStateSetChanged(Set<Long> newSyncStateSet) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+index 6dc734e0c..d5393fdca 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+@@ -41,6 +41,7 @@ import java.nio.channels.SocketChannel;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Iterator;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Objects;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+@@ -73,7 +74,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ private EpochFileCache epochCache;
|
|||
|
|
+ private AutoSwitchHAClient haClient;
|
|||
|
|
+
|
|||
|
|
+- private Long brokerControllerId = null;
|
|||
|
|
++ private Long localBrokerId = null;
|
|||
|
|
+
|
|||
|
|
+ public AutoSwitchHAService() {
|
|||
|
|
+ }
|
|||
|
|
+@@ -287,9 +288,11 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+
|
|||
|
|
+ // If the slaveBrokerId is in syncStateSet but not in connectionCaughtUpTimeTable,
|
|||
|
|
+ // it means that the broker has not connected.
|
|||
|
|
+- for (Long slaveBrokerId : newSyncStateSet) {
|
|||
|
|
+- if (!this.connectionCaughtUpTimeTable.containsKey(slaveBrokerId)) {
|
|||
|
|
+- newSyncStateSet.remove(slaveBrokerId);
|
|||
|
|
++ Iterator<Long> iterator = newSyncStateSet.iterator();
|
|||
|
|
++ while (iterator.hasNext()) {
|
|||
|
|
++ Long slaveBrokerId = iterator.next();
|
|||
|
|
++ if (!Objects.equals(slaveBrokerId, this.localBrokerId) && !this.connectionCaughtUpTimeTable.containsKey(slaveBrokerId)) {
|
|||
|
|
++ iterator.remove();
|
|||
|
|
+ isSyncStateSetChanged = true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -419,7 +422,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ // To avoid the syncStateSet is not consistent with connectionList.
|
|||
|
|
+ // Fix issue: https://github.com/apache/rocketmq/issues/6662
|
|||
|
|
+ for (Long syncId : currentSyncStateSet) {
|
|||
|
|
+- if (!idList.contains(syncId) && this.brokerControllerId != null && !Objects.equals(syncId, this.brokerControllerId)) {
|
|||
|
|
++ if (!idList.contains(syncId) && this.localBrokerId != null && !Objects.equals(syncId, this.localBrokerId)) {
|
|||
|
|
+ LOGGER.warn("Slave {} is still in syncStateSet, but has lost its connection. So new offset can't be compute.", syncId);
|
|||
|
|
+ // Without check and re-compute, return the confirmOffset's value directly.
|
|||
|
|
+ return this.defaultMessageStore.getConfirmOffsetDirectly();
|
|||
|
|
+@@ -545,12 +548,12 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ return this.epochCache.getAllEntries();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public Long getBrokerControllerId() {
|
|||
|
|
+- return brokerControllerId;
|
|||
|
|
++ public Long getLocalBrokerId() {
|
|||
|
|
++ return localBrokerId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void setBrokerControllerId(Long brokerControllerId) {
|
|||
|
|
+- this.brokerControllerId = brokerControllerId;
|
|||
|
|
++ public void setLocalBrokerId(Long localBrokerId) {
|
|||
|
|
++ this.localBrokerId = localBrokerId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class AutoSwitchAcceptSocketService extends AcceptSocketService {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 77e8e54b37c3fc3ea0beffc1ace6f5bf20af10d9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Wed, 23 Aug 2023 15:56:39 +0800
|
|||
|
|
+Subject: [PATCH 8/8] [ISSUE #7223] Support batch ack for grpc client in proxy
|
|||
|
|
+ (#7225)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../client/impl/mqclient/MQClientAPIExt.java | 26 +++
|
|||
|
|
+ .../rocketmq/proxy/config/ProxyConfig.java | 10 +
|
|||
|
|
+ .../grpc/v2/consumer/AckMessageActivity.java | 136 ++++++++---
|
|||
|
|
+ .../proxy/processor/AbstractProcessor.java | 4 +-
|
|||
|
|
+ .../proxy/processor/BatchAckResult.java | 53 +++++
|
|||
|
|
+ .../proxy/processor/ConsumerProcessor.java | 64 +++++
|
|||
|
|
+ .../processor/DefaultMessagingProcessor.java | 7 +
|
|||
|
|
+ .../proxy/processor/MessagingProcessor.java | 18 ++
|
|||
|
|
+ .../message/ClusterMessageService.java | 16 +-
|
|||
|
|
+ .../service/message/LocalMessageService.java | 58 +++++
|
|||
|
|
+ .../proxy/service/message/MessageService.java | 8 +
|
|||
|
|
+ .../service/message/ReceiptHandleMessage.java | 39 ++++
|
|||
|
|
+ .../v2/consumer/AckMessageActivityTest.java | 221 +++++++++++++++---
|
|||
|
|
+ .../proxy/processor/BaseProcessorTest.java | 18 +-
|
|||
|
|
+ .../processor/ConsumerProcessorTest.java | 115 +++++++++
|
|||
|
|
+ .../service/mqclient/MQClientAPIExtTest.java | 12 +
|
|||
|
|
+ 16 files changed, 728 insertions(+), 77 deletions(-)
|
|||
|
|
+ create mode 100644 proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+ create mode 100644 proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+index fb8f8d11f..d7c8ef8d9 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+@@ -306,6 +306,32 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ return future;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessageAsync(
|
|||
|
|
++ String brokerAddr,
|
|||
|
|
++ String topic,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ List<String> extraInfoList,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ ) {
|
|||
|
|
++ CompletableFuture<AckResult> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ this.batchAckMessageAsync(brokerAddr, timeoutMillis, new AckCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onSuccess(AckResult ackResult) {
|
|||
|
|
++ future.complete(ackResult);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onException(Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ }, topic, consumerGroup, extraInfoList);
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTimeAsync(
|
|||
|
|
+ String brokerAddr,
|
|||
|
|
+ String brokerName,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index 39caaa0d9..76a243919 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -250,6 +250,8 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ private long remotingWaitTimeMillsInTopicRouteQueue = 3 * 1000;
|
|||
|
|
+ private long remotingWaitTimeMillsInDefaultQueue = 3 * 1000;
|
|||
|
|
+
|
|||
|
|
++ private boolean enableBatchAck = false;
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public void initData() {
|
|||
|
|
+ parseDelayLevel();
|
|||
|
|
+@@ -1379,4 +1381,12 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ public void setRemotingWaitTimeMillsInDefaultQueue(long remotingWaitTimeMillsInDefaultQueue) {
|
|||
|
|
+ this.remotingWaitTimeMillsInDefaultQueue = remotingWaitTimeMillsInDefaultQueue;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableBatchAck() {
|
|||
|
|
++ return enableBatchAck;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableBatchAck(boolean enableBatchAck) {
|
|||
|
|
++ this.enableBatchAck = enableBatchAck;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
+index 9a3a77201..97c716c8f 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivity.java
|
|||
|
|
+@@ -31,12 +31,15 @@ import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
+ import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.MessageReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
++import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.AbstractMessingActivity;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.channel.GrpcChannelManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcClientSettingsManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcConverter;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.ResponseBuilder;
|
|||
|
|
++import org.apache.rocketmq.proxy.processor.BatchAckResult;
|
|||
|
|
+ import org.apache.rocketmq.proxy.processor.MessagingProcessor;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+
|
|||
|
|
+ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+
|
|||
|
|
+@@ -50,60 +53,98 @@ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ validateTopicAndConsumerGroup(request.getTopic(), request.getGroup());
|
|||
|
|
+-
|
|||
|
|
+- CompletableFuture<AckMessageResultEntry>[] futures = new CompletableFuture[request.getEntriesCount()];
|
|||
|
|
+- for (int i = 0; i < request.getEntriesCount(); i++) {
|
|||
|
|
+- futures[i] = processAckMessage(ctx, request, request.getEntries(i));
|
|||
|
|
++ String group = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getGroup());
|
|||
|
|
++ String topic = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getTopic());
|
|||
|
|
++ if (ConfigurationManager.getProxyConfig().isEnableBatchAck()) {
|
|||
|
|
++ future = ackMessageInBatch(ctx, group, topic, request);
|
|||
|
|
++ } else {
|
|||
|
|
++ future = ackMessageOneByOne(ctx, group, topic, request);
|
|||
|
|
+ }
|
|||
|
|
+- CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
+- if (throwable != null) {
|
|||
|
|
+- future.completeExceptionally(throwable);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<AckMessageResponse> ackMessageInBatch(ProxyContext ctx, String group, String topic, AckMessageRequest request) {
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList = new ArrayList<>(request.getEntriesCount());
|
|||
|
|
+
|
|||
|
|
++ for (AckMessageEntry ackMessageEntry : request.getEntriesList()) {
|
|||
|
|
++ String handleString = getHandleString(ctx, group, request, ackMessageEntry);
|
|||
|
|
++ handleMessageList.add(new ReceiptHandleMessage(ReceiptHandle.decode(handleString), ackMessageEntry.getMessageId()));
|
|||
|
|
++ }
|
|||
|
|
++ return this.messagingProcessor.batchAckMessage(ctx, handleMessageList, group, topic)
|
|||
|
|
++ .thenApply(batchAckResultList -> {
|
|||
|
|
++ AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder();
|
|||
|
|
+ Set<Code> responseCodes = new HashSet<>();
|
|||
|
|
+- List<AckMessageResultEntry> entryList = new ArrayList<>();
|
|||
|
|
+- for (CompletableFuture<AckMessageResultEntry> entryFuture : futures) {
|
|||
|
|
+- AckMessageResultEntry entryResult = entryFuture.join();
|
|||
|
|
+- responseCodes.add(entryResult.getStatus().getCode());
|
|||
|
|
+- entryList.add(entryResult);
|
|||
|
|
+- }
|
|||
|
|
+- AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder()
|
|||
|
|
+- .addAllEntries(entryList);
|
|||
|
|
+- if (responseCodes.size() > 1) {
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.MULTIPLE_RESULTS, Code.MULTIPLE_RESULTS.name()));
|
|||
|
|
+- } else if (responseCodes.size() == 1) {
|
|||
|
|
+- Code code = responseCodes.stream().findAny().get();
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(code, code.name()));
|
|||
|
|
+- } else {
|
|||
|
|
+- responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack message result is empty"));
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ AckMessageResultEntry entry = convertToAckMessageResultEntry(batchAckResult);
|
|||
|
|
++ responseBuilder.addEntries(entry);
|
|||
|
|
++ responseCodes.add(entry.getStatus().getCode());
|
|||
|
|
+ }
|
|||
|
|
+- future.complete(responseBuilder.build());
|
|||
|
|
++ setAckResponseStatus(responseBuilder, responseCodes);
|
|||
|
|
++ return responseBuilder.build();
|
|||
|
|
+ });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected AckMessageResultEntry convertToAckMessageResultEntry(BatchAckResult batchAckResult) {
|
|||
|
|
++ ReceiptHandleMessage handleMessage = batchAckResult.getReceiptHandleMessage();
|
|||
|
|
++ AckMessageResultEntry.Builder resultBuilder = AckMessageResultEntry.newBuilder()
|
|||
|
|
++ .setMessageId(handleMessage.getMessageId())
|
|||
|
|
++ .setReceiptHandle(handleMessage.getReceiptHandle().getReceiptHandle());
|
|||
|
|
++ if (batchAckResult.getProxyException() != null) {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(batchAckResult.getProxyException()));
|
|||
|
|
++ } else {
|
|||
|
|
++ AckResult ackResult = batchAckResult.getAckResult();
|
|||
|
|
++ if (AckStatus.OK.equals(ackResult.getStatus())) {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.OK, Code.OK.name()));
|
|||
|
|
++ } else {
|
|||
|
|
++ resultBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack failed: status is abnormal"));
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+- return future;
|
|||
|
|
++ return resultBuilder.build();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- protected CompletableFuture<AckMessageResultEntry> processAckMessage(ProxyContext ctx, AckMessageRequest request,
|
|||
|
|
++ protected CompletableFuture<AckMessageResponse> ackMessageOneByOne(ProxyContext ctx, String group, String topic, AckMessageRequest request) {
|
|||
|
|
++ CompletableFuture<AckMessageResponse> resultFuture = new CompletableFuture<>();
|
|||
|
|
++ CompletableFuture<AckMessageResultEntry>[] futures = new CompletableFuture[request.getEntriesCount()];
|
|||
|
|
++ for (int i = 0; i < request.getEntriesCount(); i++) {
|
|||
|
|
++ futures[i] = processAckMessage(ctx, group, topic, request, request.getEntries(i));
|
|||
|
|
++ }
|
|||
|
|
++ CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
++ if (throwable != null) {
|
|||
|
|
++ resultFuture.completeExceptionally(throwable);
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ Set<Code> responseCodes = new HashSet<>();
|
|||
|
|
++ List<AckMessageResultEntry> entryList = new ArrayList<>();
|
|||
|
|
++ for (CompletableFuture<AckMessageResultEntry> entryFuture : futures) {
|
|||
|
|
++ AckMessageResultEntry entryResult = entryFuture.join();
|
|||
|
|
++ responseCodes.add(entryResult.getStatus().getCode());
|
|||
|
|
++ entryList.add(entryResult);
|
|||
|
|
++ }
|
|||
|
|
++ AckMessageResponse.Builder responseBuilder = AckMessageResponse.newBuilder()
|
|||
|
|
++ .addAllEntries(entryList);
|
|||
|
|
++ setAckResponseStatus(responseBuilder, responseCodes);
|
|||
|
|
++ resultFuture.complete(responseBuilder.build());
|
|||
|
|
++ });
|
|||
|
|
++ return resultFuture;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<AckMessageResultEntry> processAckMessage(ProxyContext ctx, String group, String topic, AckMessageRequest request,
|
|||
|
|
+ AckMessageEntry ackMessageEntry) {
|
|||
|
|
+ CompletableFuture<AckMessageResultEntry> future = new CompletableFuture<>();
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+- String handleString = ackMessageEntry.getReceiptHandle();
|
|||
|
|
+-
|
|||
|
|
+- String group = GrpcConverter.getInstance().wrapResourceWithNamespace(request.getGroup());
|
|||
|
|
+- MessageReceiptHandle messageReceiptHandle = messagingProcessor.removeReceiptHandle(ctx, grpcChannelManager.getChannel(ctx.getClientID()), group, ackMessageEntry.getMessageId(), ackMessageEntry.getReceiptHandle());
|
|||
|
|
+- if (messageReceiptHandle != null) {
|
|||
|
|
+- handleString = messageReceiptHandle.getReceiptHandleStr();
|
|||
|
|
+- }
|
|||
|
|
++ String handleString = this.getHandleString(ctx, group, request, ackMessageEntry);
|
|||
|
|
+ CompletableFuture<AckResult> ackResultFuture = this.messagingProcessor.ackMessage(
|
|||
|
|
+ ctx,
|
|||
|
|
+ ReceiptHandle.decode(handleString),
|
|||
|
|
+ ackMessageEntry.getMessageId(),
|
|||
|
|
+ group,
|
|||
|
|
+- GrpcConverter.getInstance().wrapResourceWithNamespace(request.getTopic()));
|
|||
|
|
++ topic
|
|||
|
|
++ );
|
|||
|
|
+ ackResultFuture.thenAccept(result -> {
|
|||
|
|
+ future.complete(convertToAckMessageResultEntry(ctx, ackMessageEntry, result));
|
|||
|
|
+ }).exceptionally(t -> {
|
|||
|
|
+@@ -139,4 +180,25 @@ public class AckMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+ .setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack failed: status is abnormal"))
|
|||
|
|
+ .build();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ protected void setAckResponseStatus(AckMessageResponse.Builder responseBuilder, Set<Code> responseCodes) {
|
|||
|
|
++ if (responseCodes.size() > 1) {
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.MULTIPLE_RESULTS, Code.MULTIPLE_RESULTS.name()));
|
|||
|
|
++ } else if (responseCodes.size() == 1) {
|
|||
|
|
++ Code code = responseCodes.stream().findAny().get();
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(code, code.name()));
|
|||
|
|
++ } else {
|
|||
|
|
++ responseBuilder.setStatus(ResponseBuilder.getInstance().buildStatus(Code.INTERNAL_SERVER_ERROR, "ack message result is empty"));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected String getHandleString(ProxyContext ctx, String group, AckMessageRequest request, AckMessageEntry ackMessageEntry) {
|
|||
|
|
++ String handleString = ackMessageEntry.getReceiptHandle();
|
|||
|
|
++
|
|||
|
|
++ MessageReceiptHandle messageReceiptHandle = messagingProcessor.removeReceiptHandle(ctx, grpcChannelManager.getChannel(ctx.getClientID()), group, ackMessageEntry.getMessageId(), ackMessageEntry.getReceiptHandle());
|
|||
|
|
++ if (messageReceiptHandle != null) {
|
|||
|
|
++ handleString = messageReceiptHandle.getReceiptHandleStr();
|
|||
|
|
++ }
|
|||
|
|
++ return handleString;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
+index b61c3df9e..c63212c23 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/AbstractProcessor.java
|
|||
|
|
+@@ -27,6 +27,8 @@ public abstract class AbstractProcessor extends AbstractStartAndShutdown {
|
|||
|
|
+ protected MessagingProcessor messagingProcessor;
|
|||
|
|
+ protected ServiceManager serviceManager;
|
|||
|
|
+
|
|||
|
|
++ protected static final ProxyException EXPIRED_HANDLE_PROXY_EXCEPTION = new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired");
|
|||
|
|
++
|
|||
|
|
+ public AbstractProcessor(MessagingProcessor messagingProcessor,
|
|||
|
|
+ ServiceManager serviceManager) {
|
|||
|
|
+ this.messagingProcessor = messagingProcessor;
|
|||
|
|
+@@ -35,7 +37,7 @@ public abstract class AbstractProcessor extends AbstractStartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ protected void validateReceiptHandle(ReceiptHandle handle) {
|
|||
|
|
+ if (handle.isExpired()) {
|
|||
|
|
+- throw new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired");
|
|||
|
|
++ throw EXPIRED_HANDLE_PROXY_EXCEPTION;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..dfb9c9b9e
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/BatchAckResult.java
|
|||
|
|
+@@ -0,0 +1,53 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.proxy.processor;
|
|||
|
|
++
|
|||
|
|
++import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
++
|
|||
|
|
++public class BatchAckResult {
|
|||
|
|
++
|
|||
|
|
++ private final ReceiptHandleMessage receiptHandleMessage;
|
|||
|
|
++ private AckResult ackResult;
|
|||
|
|
++ private ProxyException proxyException;
|
|||
|
|
++
|
|||
|
|
++ public BatchAckResult(ReceiptHandleMessage receiptHandleMessage,
|
|||
|
|
++ AckResult ackResult) {
|
|||
|
|
++ this.receiptHandleMessage = receiptHandleMessage;
|
|||
|
|
++ this.ackResult = ackResult;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public BatchAckResult(ReceiptHandleMessage receiptHandleMessage,
|
|||
|
|
++ ProxyException proxyException) {
|
|||
|
|
++ this.receiptHandleMessage = receiptHandleMessage;
|
|||
|
|
++ this.proxyException = proxyException;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandleMessage getReceiptHandleMessage() {
|
|||
|
|
++ return receiptHandleMessage;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public AckResult getAckResult() {
|
|||
|
|
++ return ackResult;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ProxyException getProxyException() {
|
|||
|
|
++ return proxyException;
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
+index 656a6339d..f3522b374 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ConsumerProcessor.java
|
|||
|
|
+@@ -48,6 +48,7 @@ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.ProxyUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManager;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody;
|
|||
|
|
+@@ -241,6 +242,69 @@ public class ConsumerProcessor extends AbstractProcessor {
|
|||
|
|
+ return FutureUtils.addExecutor(future, this.executor);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ ) {
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = new ArrayList<>(handleMessageList.size());
|
|||
|
|
++ Map<String, List<ReceiptHandleMessage>> brokerHandleListMap = new HashMap<>();
|
|||
|
|
++
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ if (handleMessage.getReceiptHandle().isExpired()) {
|
|||
|
|
++ batchAckResultList.add(new BatchAckResult(handleMessage, EXPIRED_HANDLE_PROXY_EXCEPTION));
|
|||
|
|
++ continue;
|
|||
|
|
++ }
|
|||
|
|
++ List<ReceiptHandleMessage> brokerHandleList = brokerHandleListMap.computeIfAbsent(handleMessage.getReceiptHandle().getBrokerName(), key -> new ArrayList<>());
|
|||
|
|
++ brokerHandleList.add(handleMessage);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (brokerHandleListMap.isEmpty()) {
|
|||
|
|
++ return FutureUtils.addExecutor(CompletableFuture.completedFuture(batchAckResultList), this.executor);
|
|||
|
|
++ }
|
|||
|
|
++ Set<Map.Entry<String, List<ReceiptHandleMessage>>> brokerHandleListMapEntrySet = brokerHandleListMap.entrySet();
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>>[] futures = new CompletableFuture[brokerHandleListMapEntrySet.size()];
|
|||
|
|
++ int futureIndex = 0;
|
|||
|
|
++ for (Map.Entry<String, List<ReceiptHandleMessage>> entry : brokerHandleListMapEntrySet) {
|
|||
|
|
++ futures[futureIndex++] = processBrokerHandle(ctx, consumerGroup, topic, entry.getValue(), timeoutMillis);
|
|||
|
|
++ }
|
|||
|
|
++ CompletableFuture.allOf(futures).whenComplete((val, throwable) -> {
|
|||
|
|
++ if (throwable != null) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
++ }
|
|||
|
|
++ for (CompletableFuture<List<BatchAckResult>> resultFuture : futures) {
|
|||
|
|
++ batchAckResultList.addAll(resultFuture.join());
|
|||
|
|
++ }
|
|||
|
|
++ future.complete(batchAckResultList);
|
|||
|
|
++ });
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return FutureUtils.addExecutor(future, this.executor);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected CompletableFuture<List<BatchAckResult>> processBrokerHandle(ProxyContext ctx, String consumerGroup, String topic, List<ReceiptHandleMessage> handleMessageList, long timeoutMillis) {
|
|||
|
|
++ return this.serviceManager.getMessageService().batchAckMessage(ctx, handleMessageList, consumerGroup, topic, timeoutMillis)
|
|||
|
|
++ .thenApply(result -> {
|
|||
|
|
++ List<BatchAckResult> results = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ results.add(new BatchAckResult(handleMessage, result));
|
|||
|
|
++ }
|
|||
|
|
++ return results;
|
|||
|
|
++ })
|
|||
|
|
++ .exceptionally(throwable -> {
|
|||
|
|
++ List<BatchAckResult> results = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage handleMessage : handleMessageList) {
|
|||
|
|
++ results.add(new BatchAckResult(handleMessage, new ProxyException(ProxyExceptionCode.INTERNAL_SERVER_ERROR, throwable.getMessage(), throwable)));
|
|||
|
|
++ }
|
|||
|
|
++ return results;
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle,
|
|||
|
|
+ String messageId, String groupName, String topicName, long invisibleTime, long timeoutMillis) {
|
|||
|
|
+ CompletableFuture<AckResult> future = new CompletableFuture<>();
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
+index 188cb7b9b..ba150051b 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/DefaultMessagingProcessor.java
|
|||
|
|
+@@ -46,6 +46,7 @@ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.ServiceManagerFactory;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData;
|
|||
|
|
+@@ -183,6 +184,12 @@ public class DefaultMessagingProcessor extends AbstractStartAndShutdown implemen
|
|||
|
|
+ return this.consumerProcessor.ackMessage(ctx, handle, messageId, consumerGroup, topic, timeoutMillis);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<List<BatchAckResult>> batchAckMessage(ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList, String consumerGroup, String topic, long timeoutMillis) {
|
|||
|
|
++ return this.consumerProcessor.batchAckMessage(ctx, handleMessageList, consumerGroup, topic, timeoutMillis);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<AckResult> changeInvisibleTime(ProxyContext ctx, ReceiptHandle handle, String messageId,
|
|||
|
|
+ String groupName, String topicName, long invisibleTime, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
+index d86be0bd8..2ae7418ba 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/MessagingProcessor.java
|
|||
|
|
+@@ -37,6 +37,7 @@ import org.apache.rocketmq.proxy.common.Address;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.MessageReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData;
|
|||
|
|
+@@ -155,6 +156,23 @@ public interface MessagingProcessor extends StartAndShutdown {
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
++ default CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic
|
|||
|
|
++ ) {
|
|||
|
|
++ return batchAckMessage(ctx, handleMessageList, consumerGroup, topic, DEFAULT_TIMEOUT_MILLS);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ CompletableFuture<List<BatchAckResult>> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
+ default CompletableFuture<AckResult> changeInvisibleTime(
|
|||
|
|
+ ProxyContext ctx,
|
|||
|
|
+ ReceiptHandle handle,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
+index 9f163f1b9..70b72deae 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ClusterMessageService.java
|
|||
|
|
+@@ -20,9 +20,11 @@ import com.google.common.collect.Lists;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
++import java.util.stream.Collectors;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PopResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullResult;
|
|||
|
|
++import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendResult;
|
|||
|
|
+ import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.common.message.Message;
|
|||
|
|
+@@ -31,7 +33,6 @@ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+-import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.TopicRouteService;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+@@ -137,6 +138,19 @@ public class ClusterMessageService implements MessageService {
|
|||
|
|
+ );
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessage(ProxyContext ctx, List<ReceiptHandleMessage> handleList, String consumerGroup,
|
|||
|
|
++ String topic, long timeoutMillis) {
|
|||
|
|
++ List<String> extraInfoList = handleList.stream().map(message -> message.getReceiptHandle().getReceiptHandle()).collect(Collectors.toList());
|
|||
|
|
++ return this.mqClientAPIFactory.getClient().batchAckMessageAsync(
|
|||
|
|
++ this.resolveBrokerAddrInReceiptHandle(ctx, handleList.get(0).getReceiptHandle()),
|
|||
|
|
++ topic,
|
|||
|
|
++ consumerGroup,
|
|||
|
|
++ extraInfoList,
|
|||
|
|
++ timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<PullResult> pullMessage(ProxyContext ctx, AddressableMessageQueue messageQueue,
|
|||
|
|
+ PullMessageRequestHeader requestHeader, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+index eb2c4d9ee..ca7dcc9eb 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/LocalMessageService.java
|
|||
|
|
+@@ -19,6 +19,7 @@ package org.apache.rocketmq.proxy.service.message;
|
|||
|
|
+ import io.netty.channel.ChannelHandlerContext;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
++import java.util.BitSet;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+@@ -54,6 +55,8 @@ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAck;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.BatchAckMessageRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.LockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.AckMessageRequestHeader;
|
|||
|
|
+@@ -364,6 +367,61 @@ public class LocalMessageService implements MessageService {
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<AckResult> batchAckMessage(ProxyContext ctx, List<ReceiptHandleMessage> handleList,
|
|||
|
|
++ String consumerGroup, String topic, long timeoutMillis) {
|
|||
|
|
++ SimpleChannel channel = channelManager.createChannel(ctx);
|
|||
|
|
++ ChannelHandlerContext channelHandlerContext = channel.getChannelHandlerContext();
|
|||
|
|
++ RemotingCommand command = LocalRemotingCommand.createRequestCommand(RequestCode.BATCH_ACK_MESSAGE, null);
|
|||
|
|
++
|
|||
|
|
++ Map<String, BatchAck> batchAckMap = new HashMap<>();
|
|||
|
|
++ for (ReceiptHandleMessage receiptHandleMessage : handleList) {
|
|||
|
|
++ String extraInfo = receiptHandleMessage.getReceiptHandle().getReceiptHandle();
|
|||
|
|
++ String[] extraInfoData = ExtraInfoUtil.split(extraInfo);
|
|||
|
|
++ String mergeKey = ExtraInfoUtil.getRetry(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getQueueId(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getCkQueueOffset(extraInfoData) + "@" +
|
|||
|
|
++ ExtraInfoUtil.getPopTime(extraInfoData);
|
|||
|
|
++ BatchAck bAck = batchAckMap.computeIfAbsent(mergeKey, k -> {
|
|||
|
|
++ BatchAck newBatchAck = new BatchAck();
|
|||
|
|
++ newBatchAck.setConsumerGroup(consumerGroup);
|
|||
|
|
++ newBatchAck.setTopic(topic);
|
|||
|
|
++ newBatchAck.setRetry(ExtraInfoUtil.getRetry(extraInfoData));
|
|||
|
|
++ newBatchAck.setStartOffset(ExtraInfoUtil.getCkQueueOffset(extraInfoData));
|
|||
|
|
++ newBatchAck.setQueueId(ExtraInfoUtil.getQueueId(extraInfoData));
|
|||
|
|
++ newBatchAck.setReviveQueueId(ExtraInfoUtil.getReviveQid(extraInfoData));
|
|||
|
|
++ newBatchAck.setPopTime(ExtraInfoUtil.getPopTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setInvisibleTime(ExtraInfoUtil.getInvisibleTime(extraInfoData));
|
|||
|
|
++ newBatchAck.setBitSet(new BitSet());
|
|||
|
|
++ return newBatchAck;
|
|||
|
|
++ });
|
|||
|
|
++ bAck.getBitSet().set((int) (ExtraInfoUtil.getQueueOffset(extraInfoData) - ExtraInfoUtil.getCkQueueOffset(extraInfoData)));
|
|||
|
|
++ }
|
|||
|
|
++ BatchAckMessageRequestBody requestBody = new BatchAckMessageRequestBody();
|
|||
|
|
++ requestBody.setBrokerName(brokerController.getBrokerConfig().getBrokerName());
|
|||
|
|
++ requestBody.setAcks(new ArrayList<>(batchAckMap.values()));
|
|||
|
|
++
|
|||
|
|
++ command.setBody(requestBody.encode());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ RemotingCommand response = brokerController.getAckMessageProcessor()
|
|||
|
|
++ .processRequest(channelHandlerContext, command);
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ log.error("Fail to process batchAckMessage command", e);
|
|||
|
|
++ future.completeExceptionally(e);
|
|||
|
|
++ }
|
|||
|
|
++ return future.thenApply(r -> {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ if (ResponseCode.SUCCESS == r.getCode()) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ } else {
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ }
|
|||
|
|
++ return ackResult;
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<PullResult> pullMessage(ProxyContext ctx, AddressableMessageQueue messageQueue,
|
|||
|
|
+ PullMessageRequestHeader requestHeader, long timeoutMillis) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
+index 15da17154..58a835adb 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/MessageService.java
|
|||
|
|
+@@ -91,6 +91,14 @@ public interface MessageService {
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
++ CompletableFuture<AckResult> batchAckMessage(
|
|||
|
|
++ ProxyContext ctx,
|
|||
|
|
++ List<ReceiptHandleMessage> handleList,
|
|||
|
|
++ String consumerGroup,
|
|||
|
|
++ String topic,
|
|||
|
|
++ long timeoutMillis
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
+ CompletableFuture<PullResult> pullMessage(
|
|||
|
|
+ ProxyContext ctx,
|
|||
|
|
+ AddressableMessageQueue messageQueue,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..ae63fed49
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/message/ReceiptHandleMessage.java
|
|||
|
|
+@@ -0,0 +1,39 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.proxy.service.message;
|
|||
|
|
++
|
|||
|
|
++import org.apache.rocketmq.common.consumer.ReceiptHandle;
|
|||
|
|
++
|
|||
|
|
++public class ReceiptHandleMessage {
|
|||
|
|
++
|
|||
|
|
++ private final ReceiptHandle receiptHandle;
|
|||
|
|
++ private final String messageId;
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandleMessage(ReceiptHandle receiptHandle, String messageId) {
|
|||
|
|
++ this.receiptHandle = receiptHandle;
|
|||
|
|
++ this.messageId = messageId;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ReceiptHandle getReceiptHandle() {
|
|||
|
|
++ return receiptHandle;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public String getMessageId() {
|
|||
|
|
++ return messageId;
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
+index 49fdfc6a8..3c4746105 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/AckMessageActivityTest.java
|
|||
|
|
+@@ -20,21 +20,32 @@ package org.apache.rocketmq.proxy.grpc.v2.consumer;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageEntry;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageRequest;
|
|||
|
|
+ import apache.rocketmq.v2.AckMessageResponse;
|
|||
|
|
++import apache.rocketmq.v2.AckMessageResultEntry;
|
|||
|
|
+ import apache.rocketmq.v2.Code;
|
|||
|
|
+ import apache.rocketmq.v2.Resource;
|
|||
|
|
++import java.util.ArrayList;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
++import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.BaseActivityTest;
|
|||
|
|
++import org.apache.rocketmq.proxy.processor.BatchAckResult;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
++import org.mockito.stubbing.Answer;
|
|||
|
|
+
|
|||
|
|
+ import static org.junit.Assert.assertEquals;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.anyList;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
++import static org.mockito.Mockito.doAnswer;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ public class AckMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+@@ -52,43 +63,197 @@ public class AckMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testAckMessage() throws Throwable {
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg1"), anyString(), anyString()))
|
|||
|
|
++ ConfigurationManager.getProxyConfig().setEnableBatchAck(false);
|
|||
|
|
++
|
|||
|
|
++ String msg1 = "msg1";
|
|||
|
|
++ String msg2 = "msg2";
|
|||
|
|
++ String msg3 = "msg3";
|
|||
|
|
++
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg1), anyString(), anyString()))
|
|||
|
|
+ .thenThrow(new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, "receipt handle is expired"));
|
|||
|
|
+
|
|||
|
|
+ AckResult msg2AckResult = new AckResult();
|
|||
|
|
+ msg2AckResult.setStatus(AckStatus.OK);
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg2"), anyString(), anyString()))
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg2), anyString(), anyString()))
|
|||
|
|
+ .thenReturn(CompletableFuture.completedFuture(msg2AckResult));
|
|||
|
|
+
|
|||
|
|
+ AckResult msg3AckResult = new AckResult();
|
|||
|
|
+ msg3AckResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+- when(this.messagingProcessor.ackMessage(any(), any(), eq("msg3"), anyString(), anyString()))
|
|||
|
|
++ when(this.messagingProcessor.ackMessage(any(), any(), eq(msg3), anyString(), anyString()))
|
|||
|
|
+ .thenReturn(CompletableFuture.completedFuture(msg3AckResult));
|
|||
|
|
+
|
|||
|
|
+- AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
+- createContext(),
|
|||
|
|
+- AckMessageRequest.newBuilder()
|
|||
|
|
+- .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
+- .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg1")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg2")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
+- .setMessageId("msg3")
|
|||
|
|
+- .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
+- .build())
|
|||
|
|
+- .build()
|
|||
|
|
+- ).get();
|
|||
|
|
+-
|
|||
|
|
+- assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
+- assertEquals(3, response.getEntriesCount());
|
|||
|
|
+- assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getEntries(0).getStatus().getCode());
|
|||
|
|
+- assertEquals(Code.OK, response.getEntries(1).getStatus().getCode());
|
|||
|
|
+- assertEquals(Code.INTERNAL_SERVER_ERROR, response.getEntries(2).getStatus().getCode());
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg1)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg2)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.OK, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg3)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg1)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis() - 10000, 1000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg2)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(msg3)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++
|
|||
|
|
++ assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
++ assertEquals(3, response.getEntriesCount());
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getEntries(0).getStatus().getCode());
|
|||
|
|
++ assertEquals(Code.OK, response.getEntries(1).getStatus().getCode());
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getEntries(2).getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testAckMessageInBatch() throws Throwable {
|
|||
|
|
++ ConfigurationManager.getProxyConfig().setEnableBatchAck(true);
|
|||
|
|
++
|
|||
|
|
++ String successMessageId = "msg1";
|
|||
|
|
++ String notOkMessageId = "msg2";
|
|||
|
|
++ String exceptionMessageId = "msg3";
|
|||
|
|
++
|
|||
|
|
++ doAnswer((Answer<CompletableFuture<List<BatchAckResult>>>) invocation -> {
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = invocation.getArgument(1, List.class);
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = new ArrayList<>();
|
|||
|
|
++ for (ReceiptHandleMessage receiptHandleMessage : receiptHandleMessageList) {
|
|||
|
|
++ BatchAckResult batchAckResult;
|
|||
|
|
++ if (receiptHandleMessage.getMessageId().equals(successMessageId)) {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, ackResult);
|
|||
|
|
++ } else if (receiptHandleMessage.getMessageId().equals(notOkMessageId)) {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, ackResult);
|
|||
|
|
++ } else {
|
|||
|
|
++ batchAckResult = new BatchAckResult(receiptHandleMessage, new ProxyException(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, ""));
|
|||
|
|
++ }
|
|||
|
|
++ batchAckResultList.add(batchAckResult);
|
|||
|
|
++ }
|
|||
|
|
++ return CompletableFuture.completedFuture(batchAckResultList);
|
|||
|
|
++ }).when(this.messagingProcessor).batchAckMessage(any(), anyList(), anyString(), anyString());
|
|||
|
|
++
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(successMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.OK, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(notOkMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(exceptionMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, response.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ {
|
|||
|
|
++ AckMessageResponse response = this.ackMessageActivity.ackMessage(
|
|||
|
|
++ createContext(),
|
|||
|
|
++ AckMessageRequest.newBuilder()
|
|||
|
|
++ .setTopic(Resource.newBuilder().setName(TOPIC).build())
|
|||
|
|
++ .setGroup(Resource.newBuilder().setName(GROUP).build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(successMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(notOkMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .addEntries(AckMessageEntry.newBuilder()
|
|||
|
|
++ .setMessageId(exceptionMessageId)
|
|||
|
|
++ .setReceiptHandle(buildReceiptHandle(TOPIC, System.currentTimeMillis(), 3000))
|
|||
|
|
++ .build())
|
|||
|
|
++ .build()
|
|||
|
|
++ ).get();
|
|||
|
|
++
|
|||
|
|
++ assertEquals(Code.MULTIPLE_RESULTS, response.getStatus().getCode());
|
|||
|
|
++ assertEquals(3, response.getEntriesCount());
|
|||
|
|
++ Map<String, Code> msgCode = new HashMap<>();
|
|||
|
|
++ for (AckMessageResultEntry entry : response.getEntriesList()) {
|
|||
|
|
++ msgCode.put(entry.getMessageId(), entry.getStatus().getCode());
|
|||
|
|
++ }
|
|||
|
|
++ assertEquals(Code.OK, msgCode.get(successMessageId));
|
|||
|
|
++ assertEquals(Code.INTERNAL_SERVER_ERROR, msgCode.get(notOkMessageId));
|
|||
|
|
++ assertEquals(Code.INVALID_RECEIPT_HANDLE, msgCode.get(exceptionMessageId));
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
+index 5c1ea9627..072630e39 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/BaseProcessorTest.java
|
|||
|
|
+@@ -66,14 +66,6 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ protected ProxyRelayService proxyRelayService;
|
|||
|
|
+ @Mock
|
|||
|
|
+ protected MetadataService metadataService;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ProducerProcessor producerProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ConsumerProcessor consumerProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected TransactionProcessor transactionProcessor;
|
|||
|
|
+- @Mock
|
|||
|
|
+- protected ClientProcessor clientProcessor;
|
|||
|
|
+
|
|||
|
|
+ public void before() throws Throwable {
|
|||
|
|
+ super.before();
|
|||
|
|
+@@ -92,6 +84,13 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ protected static MessageExt createMessageExt(String topic, String tags, int reconsumeTimes, long invisibleTime) {
|
|||
|
|
++ return createMessageExt(topic, tags, reconsumeTimes, invisibleTime, System.currentTimeMillis(),
|
|||
|
|
++ RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE),
|
|||
|
|
++ RANDOM.nextInt(Integer.MAX_VALUE), "mockBroker");
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ protected static MessageExt createMessageExt(String topic, String tags, int reconsumeTimes, long invisibleTime, long popTime,
|
|||
|
|
++ long startOffset, int reviveQid, int queueId, long queueOffset, String brokerName) {
|
|||
|
|
+ MessageExt messageExt = new MessageExt();
|
|||
|
|
+ messageExt.setTopic(topic);
|
|||
|
|
+ messageExt.setTags(tags);
|
|||
|
|
+@@ -100,8 +99,7 @@ public class BaseProcessorTest extends InitConfigTest {
|
|||
|
|
+ messageExt.setMsgId(MessageClientIDSetter.createUniqID());
|
|||
|
|
+ messageExt.setCommitLogOffset(RANDOM.nextInt(Integer.MAX_VALUE));
|
|||
|
|
+ MessageAccessor.putProperty(messageExt, MessageConst.PROPERTY_POP_CK,
|
|||
|
|
+- ExtraInfoUtil.buildExtraInfo(RANDOM.nextInt(Integer.MAX_VALUE), System.currentTimeMillis(), invisibleTime,
|
|||
|
|
+- RANDOM.nextInt(Integer.MAX_VALUE), topic, "mockBroker", RANDOM.nextInt(Integer.MAX_VALUE), RANDOM.nextInt(Integer.MAX_VALUE)));
|
|||
|
|
++ ExtraInfoUtil.buildExtraInfo(startOffset, popTime, invisibleTime, reviveQid, topic, brokerName, queueId, queueOffset));
|
|||
|
|
+ return messageExt;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
+index 717e86fc0..db268a06e 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/processor/ConsumerProcessorTest.java
|
|||
|
|
+@@ -20,8 +20,11 @@ package org.apache.rocketmq.proxy.processor;
|
|||
|
|
+ import com.google.common.collect.Sets;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
++import java.util.Collections;
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+@@ -39,7 +42,10 @@ import org.apache.rocketmq.common.message.MessageClientIDSetter;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.ProxyUtils;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.message.ReceiptHandleMessage;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.MessageQueueView;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+@@ -50,16 +56,22 @@ import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+ import org.mockito.ArgumentCaptor;
|
|||
|
|
++import org.mockito.stubbing.Answer;
|
|||
|
|
+
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+ import static org.junit.Assert.assertEquals;
|
|||
|
|
+ import static org.junit.Assert.assertNotNull;
|
|||
|
|
++import static org.junit.Assert.assertNull;
|
|||
|
|
+ import static org.junit.Assert.assertSame;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.anyList;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
++import static org.mockito.Mockito.doAnswer;
|
|||
|
|
+ import static org.mockito.Mockito.mock;
|
|||
|
|
++import static org.mockito.Mockito.never;
|
|||
|
|
++import static org.mockito.Mockito.verify;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ public class ConsumerProcessorTest extends BaseProcessorTest {
|
|||
|
|
+@@ -162,6 +174,109 @@ public class ConsumerProcessorTest extends BaseProcessorTest {
|
|||
|
|
+ assertEquals(handle.getReceiptHandle(), requestHeaderArgumentCaptor.getValue().getExtraInfo());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckExpireMessage() throws Throwable {
|
|||
|
|
++ String brokerName1 = "brokerName1";
|
|||
|
|
++
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = new ArrayList<>();
|
|||
|
|
++ for (int i = 0; i < 3; i++) {
|
|||
|
|
++ MessageExt expireMessage = createMessageExt(TOPIC, "", 0, 3000, System.currentTimeMillis() - 10000,
|
|||
|
|
++ 0, 0, 0, i, brokerName1);
|
|||
|
|
++ ReceiptHandle expireHandle = create(expireMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(expireHandle, expireMessage.getMsgId()));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = this.consumerProcessor.batchAckMessage(createContext(), receiptHandleMessageList, CONSUMER_GROUP, TOPIC, 3000).get();
|
|||
|
|
++
|
|||
|
|
++ verify(this.messageService, never()).batchAckMessage(any(), anyList(), anyString(), anyString(), anyLong());
|
|||
|
|
++ assertEquals(receiptHandleMessageList.size(), batchAckResultList.size());
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ assertNull(batchAckResult.getAckResult());
|
|||
|
|
++ assertNotNull(batchAckResult.getProxyException());
|
|||
|
|
++ assertNotNull(batchAckResult.getReceiptHandleMessage());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckMessage() throws Throwable {
|
|||
|
|
++ String brokerName1 = "brokerName1";
|
|||
|
|
++ String brokerName2 = "brokerName2";
|
|||
|
|
++ String errThrowBrokerName = "errThrowBrokerName";
|
|||
|
|
++ MessageExt expireMessage = createMessageExt(TOPIC, "", 0, 3000, System.currentTimeMillis() - 10000,
|
|||
|
|
++ 0, 0, 0, 0, brokerName1);
|
|||
|
|
++ ReceiptHandle expireHandle = create(expireMessage);
|
|||
|
|
++
|
|||
|
|
++ List<ReceiptHandleMessage> receiptHandleMessageList = new ArrayList<>();
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(expireHandle, expireMessage.getMsgId()));
|
|||
|
|
++ List<String> broker1Msg = new ArrayList<>();
|
|||
|
|
++ List<String> broker2Msg = new ArrayList<>();
|
|||
|
|
++
|
|||
|
|
++ long now = System.currentTimeMillis();
|
|||
|
|
++ int msgNum = 3;
|
|||
|
|
++ for (int i = 0; i < msgNum; i++) {
|
|||
|
|
++ MessageExt brokerMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, i + 1, brokerName1);
|
|||
|
|
++ ReceiptHandle brokerHandle = create(brokerMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(brokerHandle, brokerMessage.getMsgId()));
|
|||
|
|
++ broker1Msg.add(brokerMessage.getMsgId());
|
|||
|
|
++ }
|
|||
|
|
++ for (int i = 0; i < msgNum; i++) {
|
|||
|
|
++ MessageExt brokerMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, i + 1, brokerName2);
|
|||
|
|
++ ReceiptHandle brokerHandle = create(brokerMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(brokerHandle, brokerMessage.getMsgId()));
|
|||
|
|
++ broker2Msg.add(brokerMessage.getMsgId());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // for this message, will throw exception in batchAckMessage
|
|||
|
|
++ MessageExt errThrowMessage = createMessageExt(TOPIC, "", 0, 3000, now,
|
|||
|
|
++ 0, 0, 0, 0, errThrowBrokerName);
|
|||
|
|
++ ReceiptHandle errThrowHandle = create(errThrowMessage);
|
|||
|
|
++ receiptHandleMessageList.add(new ReceiptHandleMessage(errThrowHandle, errThrowMessage.getMsgId()));
|
|||
|
|
++
|
|||
|
|
++ Collections.shuffle(receiptHandleMessageList);
|
|||
|
|
++
|
|||
|
|
++ doAnswer((Answer<CompletableFuture<AckResult>>) invocation -> {
|
|||
|
|
++ List<ReceiptHandleMessage> handleMessageList = invocation.getArgument(1, List.class);
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ String brokerName = handleMessageList.get(0).getReceiptHandle().getBrokerName();
|
|||
|
|
++ if (brokerName.equals(brokerName1)) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ } else if (brokerName.equals(brokerName2)) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
++ } else {
|
|||
|
|
++ return FutureUtils.completeExceptionally(new RuntimeException());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ return CompletableFuture.completedFuture(ackResult);
|
|||
|
|
++ }).when(this.messageService).batchAckMessage(any(), anyList(), anyString(), anyString(), anyLong());
|
|||
|
|
++
|
|||
|
|
++ List<BatchAckResult> batchAckResultList = this.consumerProcessor.batchAckMessage(createContext(), receiptHandleMessageList, CONSUMER_GROUP, TOPIC, 3000).get();
|
|||
|
|
++ assertEquals(receiptHandleMessageList.size(), batchAckResultList.size());
|
|||
|
|
++
|
|||
|
|
++ // check ackResult for each msg
|
|||
|
|
++ Map<String, BatchAckResult> msgBatchAckResult = new HashMap<>();
|
|||
|
|
++ for (BatchAckResult batchAckResult : batchAckResultList) {
|
|||
|
|
++ msgBatchAckResult.put(batchAckResult.getReceiptHandleMessage().getMessageId(), batchAckResult);
|
|||
|
|
++ }
|
|||
|
|
++ for (String msgId : broker1Msg) {
|
|||
|
|
++ assertEquals(AckStatus.OK, msgBatchAckResult.get(msgId).getAckResult().getStatus());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(msgId).getProxyException());
|
|||
|
|
++ }
|
|||
|
|
++ for (String msgId : broker2Msg) {
|
|||
|
|
++ assertEquals(AckStatus.NO_EXIST, msgBatchAckResult.get(msgId).getAckResult().getStatus());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(msgId).getProxyException());
|
|||
|
|
++ }
|
|||
|
|
++ assertNotNull(msgBatchAckResult.get(expireMessage.getMsgId()).getProxyException());
|
|||
|
|
++ assertEquals(ProxyExceptionCode.INVALID_RECEIPT_HANDLE, msgBatchAckResult.get(expireMessage.getMsgId()).getProxyException().getCode());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(expireMessage.getMsgId()).getAckResult());
|
|||
|
|
++
|
|||
|
|
++ assertNotNull(msgBatchAckResult.get(errThrowMessage.getMsgId()).getProxyException());
|
|||
|
|
++ assertEquals(ProxyExceptionCode.INTERNAL_SERVER_ERROR, msgBatchAckResult.get(errThrowMessage.getMsgId()).getProxyException().getCode());
|
|||
|
|
++ assertNull(msgBatchAckResult.get(errThrowMessage.getMsgId()).getAckResult());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testChangeInvisibleTime() throws Throwable {
|
|||
|
|
+ ReceiptHandle handle = create(createMessageExt(MixAll.RETRY_GROUP_TOPIC_PREFIX + TOPIC, "", 0, 3000));
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+index 77a119a29..3f3a4ae40 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+@@ -220,6 +220,18 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ assertSame(ackResult, mqClientAPI.ackMessageAsync(BROKER_ADDR, new AckMessageRequestHeader(), TIMEOUT).get());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testBatchAckMessageAsync() throws Exception {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ doAnswer((Answer<Void>) mock -> {
|
|||
|
|
++ AckCallback ackCallback = mock.getArgument(2);
|
|||
|
|
++ ackCallback.onSuccess(ackResult);
|
|||
|
|
++ return null;
|
|||
|
|
++ }).when(mqClientAPI).batchAckMessageAsync(anyString(), anyLong(), any(AckCallback.class), any());
|
|||
|
|
++
|
|||
|
|
++ assertSame(ackResult, mqClientAPI.batchAckMessageAsync(BROKER_ADDR, TOPIC, CONSUMER_GROUP, new ArrayList<>(), TIMEOUT).get());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testChangeInvisibleTimeAsync() throws Exception {
|
|||
|
|
+ AckResult ackResult = new AckResult();
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch013-backport-enhance-admin-output.patch b/patch013-backport-enhance-admin-output.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..3fa60916f
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch013-backport-enhance-admin-output.patch
|
|||
|
|
@@ -0,0 +1,892 @@
|
|||
|
|
+From 7e018520ef707a841c66c55d621f6560d03b631b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Fri, 25 Aug 2023 09:49:22 +0800
|
|||
|
|
+Subject: [PATCH 1/6] Add expireAfterAccess for cache (#7247)
|
|||
|
|
+
|
|||
|
|
+Add expireAfterAccess for cache
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/proxy/config/ProxyConfig.java | 59 ++++++++++++++-----
|
|||
|
|
+ .../metadata/ClusterMetadataService.java | 6 +-
|
|||
|
|
+ .../service/route/TopicRouteService.java | 14 +++--
|
|||
|
|
+ 3 files changed, 56 insertions(+), 23 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index 76a243919..2994893d7 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -155,14 +155,17 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ private int consumerProcessorThreadPoolQueueCapacity = 10000;
|
|||
|
|
+
|
|||
|
|
+ private boolean useEndpointPortFromRequest = false;
|
|||
|
|
+- private int topicRouteServiceCacheExpiredInSeconds = 20;
|
|||
|
|
++
|
|||
|
|
++ private int topicRouteServiceCacheExpiredSeconds = 300;
|
|||
|
|
++ private int topicRouteServiceCacheRefreshSeconds = 20;
|
|||
|
|
+ private int topicRouteServiceCacheMaxNum = 20000;
|
|||
|
|
+ private int topicRouteServiceThreadPoolNums = PROCESSOR_NUMBER;
|
|||
|
|
+ private int topicRouteServiceThreadPoolQueueCapacity = 5000;
|
|||
|
|
+-
|
|||
|
|
+- private int topicConfigCacheExpiredInSeconds = 20;
|
|||
|
|
++ private int topicConfigCacheExpiredSeconds = 300;
|
|||
|
|
++ private int topicConfigCacheRefreshSeconds = 20;
|
|||
|
|
+ private int topicConfigCacheMaxNum = 20000;
|
|||
|
|
+- private int subscriptionGroupConfigCacheExpiredInSeconds = 20;
|
|||
|
|
++ private int subscriptionGroupConfigCacheExpiredSeconds = 300;
|
|||
|
|
++ private int subscriptionGroupConfigCacheRefreshSeconds = 20;
|
|||
|
|
+ private int subscriptionGroupConfigCacheMaxNum = 20000;
|
|||
|
|
+ private int metadataThreadPoolNums = 3;
|
|||
|
|
+ private int metadataThreadPoolQueueCapacity = 100000;
|
|||
|
|
+@@ -794,12 +797,20 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ this.consumerProcessorThreadPoolQueueCapacity = consumerProcessorThreadPoolQueueCapacity;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public int getTopicRouteServiceCacheExpiredInSeconds() {
|
|||
|
|
+- return topicRouteServiceCacheExpiredInSeconds;
|
|||
|
|
++ public int getTopicRouteServiceCacheExpiredSeconds() {
|
|||
|
|
++ return topicRouteServiceCacheExpiredSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setTopicRouteServiceCacheExpiredSeconds(int topicRouteServiceCacheExpiredSeconds) {
|
|||
|
|
++ this.topicRouteServiceCacheExpiredSeconds = topicRouteServiceCacheExpiredSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void setTopicRouteServiceCacheExpiredInSeconds(int topicRouteServiceCacheExpiredInSeconds) {
|
|||
|
|
+- this.topicRouteServiceCacheExpiredInSeconds = topicRouteServiceCacheExpiredInSeconds;
|
|||
|
|
++ public int getTopicRouteServiceCacheRefreshSeconds() {
|
|||
|
|
++ return topicRouteServiceCacheRefreshSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setTopicRouteServiceCacheRefreshSeconds(int topicRouteServiceCacheRefreshSeconds) {
|
|||
|
|
++ this.topicRouteServiceCacheRefreshSeconds = topicRouteServiceCacheRefreshSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public int getTopicRouteServiceCacheMaxNum() {
|
|||
|
|
+@@ -826,12 +837,20 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ this.topicRouteServiceThreadPoolQueueCapacity = topicRouteServiceThreadPoolQueueCapacity;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public int getTopicConfigCacheExpiredInSeconds() {
|
|||
|
|
+- return topicConfigCacheExpiredInSeconds;
|
|||
|
|
++ public int getTopicConfigCacheRefreshSeconds() {
|
|||
|
|
++ return topicConfigCacheRefreshSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setTopicConfigCacheRefreshSeconds(int topicConfigCacheRefreshSeconds) {
|
|||
|
|
++ this.topicConfigCacheRefreshSeconds = topicConfigCacheRefreshSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getTopicConfigCacheExpiredSeconds() {
|
|||
|
|
++ return topicConfigCacheExpiredSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void setTopicConfigCacheExpiredInSeconds(int topicConfigCacheExpiredInSeconds) {
|
|||
|
|
+- this.topicConfigCacheExpiredInSeconds = topicConfigCacheExpiredInSeconds;
|
|||
|
|
++ public void setTopicConfigCacheExpiredSeconds(int topicConfigCacheExpiredSeconds) {
|
|||
|
|
++ this.topicConfigCacheExpiredSeconds = topicConfigCacheExpiredSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public int getTopicConfigCacheMaxNum() {
|
|||
|
|
+@@ -842,12 +861,20 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ this.topicConfigCacheMaxNum = topicConfigCacheMaxNum;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public int getSubscriptionGroupConfigCacheExpiredInSeconds() {
|
|||
|
|
+- return subscriptionGroupConfigCacheExpiredInSeconds;
|
|||
|
|
++ public int getSubscriptionGroupConfigCacheRefreshSeconds() {
|
|||
|
|
++ return subscriptionGroupConfigCacheRefreshSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setSubscriptionGroupConfigCacheRefreshSeconds(int subscriptionGroupConfigCacheRefreshSeconds) {
|
|||
|
|
++ this.subscriptionGroupConfigCacheRefreshSeconds = subscriptionGroupConfigCacheRefreshSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getSubscriptionGroupConfigCacheExpiredSeconds() {
|
|||
|
|
++ return subscriptionGroupConfigCacheExpiredSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void setSubscriptionGroupConfigCacheExpiredInSeconds(int subscriptionGroupConfigCacheExpiredInSeconds) {
|
|||
|
|
+- this.subscriptionGroupConfigCacheExpiredInSeconds = subscriptionGroupConfigCacheExpiredInSeconds;
|
|||
|
|
++ public void setSubscriptionGroupConfigCacheExpiredSeconds(int subscriptionGroupConfigCacheExpiredSeconds) {
|
|||
|
|
++ this.subscriptionGroupConfigCacheExpiredSeconds = subscriptionGroupConfigCacheExpiredSeconds;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public int getSubscriptionGroupConfigCacheMaxNum() {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java
|
|||
|
|
+index bc9582ad8..d34a0efd9 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java
|
|||
|
|
+@@ -69,11 +69,13 @@ public class ClusterMetadataService extends AbstractStartAndShutdown implements
|
|||
|
|
+ );
|
|||
|
|
+ this.topicConfigCache = CacheBuilder.newBuilder()
|
|||
|
|
+ .maximumSize(config.getTopicConfigCacheMaxNum())
|
|||
|
|
+- .refreshAfterWrite(config.getTopicConfigCacheExpiredInSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .expireAfterAccess(config.getTopicConfigCacheExpiredSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .refreshAfterWrite(config.getTopicConfigCacheRefreshSeconds(), TimeUnit.SECONDS)
|
|||
|
|
+ .build(new ClusterTopicConfigCacheLoader());
|
|||
|
|
+ this.subscriptionGroupConfigCache = CacheBuilder.newBuilder()
|
|||
|
|
+ .maximumSize(config.getSubscriptionGroupConfigCacheMaxNum())
|
|||
|
|
+- .refreshAfterWrite(config.getSubscriptionGroupConfigCacheExpiredInSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .expireAfterAccess(config.getSubscriptionGroupConfigCacheExpiredSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .refreshAfterWrite(config.getSubscriptionGroupConfigCacheRefreshSeconds(), TimeUnit.SECONDS)
|
|||
|
|
+ .build(new ClusterSubscriptionGroupConfigCacheLoader());
|
|||
|
|
+
|
|||
|
|
+ this.init();
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+index e012a5465..84348adc3 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+@@ -68,10 +68,13 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ );
|
|||
|
|
+ this.mqClientAPIFactory = mqClientAPIFactory;
|
|||
|
|
+
|
|||
|
|
+- this.topicCache = Caffeine.newBuilder().maximumSize(config.getTopicRouteServiceCacheMaxNum()).
|
|||
|
|
+- refreshAfterWrite(config.getTopicRouteServiceCacheExpiredInSeconds(), TimeUnit.SECONDS).
|
|||
|
|
+- executor(cacheRefreshExecutor).build(new CacheLoader<String, MessageQueueView>() {
|
|||
|
|
+- @Override public @Nullable MessageQueueView load(String topic) throws Exception {
|
|||
|
|
++ this.topicCache = Caffeine.newBuilder().maximumSize(config.getTopicRouteServiceCacheMaxNum())
|
|||
|
|
++ .expireAfterAccess(config.getTopicRouteServiceCacheExpiredSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .refreshAfterWrite(config.getTopicRouteServiceCacheRefreshSeconds(), TimeUnit.SECONDS)
|
|||
|
|
++ .executor(cacheRefreshExecutor)
|
|||
|
|
++ .build(new CacheLoader<String, MessageQueueView>() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public @Nullable MessageQueueView load(String topic) throws Exception {
|
|||
|
|
+ try {
|
|||
|
|
+ TopicRouteData topicRouteData = mqClientAPIFactory.getClient().getTopicRouteInfoFromNameServer(topic, Duration.ofSeconds(3).toMillis());
|
|||
|
|
+ return buildMessageQueueView(topic, topicRouteData);
|
|||
|
|
+@@ -83,7 +86,8 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Override public @Nullable MessageQueueView reload(@NonNull String key,
|
|||
|
|
++ @Override
|
|||
|
|
++ public @Nullable MessageQueueView reload(@NonNull String key,
|
|||
|
|
+ @NonNull MessageQueueView oldValue) throws Exception {
|
|||
|
|
+ try {
|
|||
|
|
+ return load(key);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 5f6dc90f9dab35809fcb0407d4d5cc2737d2335e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <tanziyi0925@gmail.com>
|
|||
|
|
+Date: Fri, 25 Aug 2023 11:17:23 +0800
|
|||
|
|
+Subject: [PATCH 2/6] [ISSUE #7250] Beautify command rocksDBConfigToJson output
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: Ziy1-Tan <ajb4596984460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../metadata/RocksDBConfigToJsonCommand.java | 32 +++++++++++--------
|
|||
|
|
+ 1 file changed, 18 insertions(+), 14 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+index 3053f4684..3fc63e4dd 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+@@ -21,13 +21,13 @@ import org.apache.commons.cli.CommandLine;
|
|||
|
|
+ import org.apache.commons.cli.Option;
|
|||
|
|
+ import org.apache.commons.cli.Options;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+ import org.apache.rocketmq.common.config.RocksDBConfigManager;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
+
|
|||
|
|
+-import java.io.File;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+
|
|||
|
|
+@@ -48,7 +48,7 @@ public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+ @Override
|
|||
|
|
+ public Options buildCommandlineOptions(Options options) {
|
|||
|
|
+ Option pathOption = new Option("p", "path", true,
|
|||
|
|
+- "Absolute path to the metadata directory");
|
|||
|
|
++ "Absolute path for the metadata directory");
|
|||
|
|
+ pathOption.setRequired(true);
|
|||
|
|
+ options.addOption(pathOption);
|
|||
|
|
+
|
|||
|
|
+@@ -63,15 +63,14 @@ public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
+ String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
+- if (StringUtils.isEmpty(path) || !new File(path).exists()) {
|
|||
|
|
++ if (StringUtils.isEmpty(path) || !UtilAll.isPathExists(path)) {
|
|||
|
|
+ System.out.print("Rocksdb path is invalid.\n");
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
+
|
|||
|
|
+- final long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
+- RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(memTableFlushInterval);
|
|||
|
|
++ RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(60 * 60 * 1000L);
|
|||
|
|
+ try {
|
|||
|
|
+ if (TOPICS_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+ // for topics.json
|
|||
|
|
+@@ -84,13 +83,16 @@ public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+ topicConfigTable.put(topic, jsonObject);
|
|||
|
|
+ });
|
|||
|
|
+
|
|||
|
|
+- if (isLoad) {
|
|||
|
|
+- topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
+- final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
+- System.out.print(topicsJsonStr + "\n");
|
|||
|
|
++ if (!isLoad) {
|
|||
|
|
++ System.out.print("RocksDB load error, path=" + path);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
++ topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
++ final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
++ System.out.print(topicsJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ if (SUBSCRIPTION_GROUP_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+ // for subscriptionGroup.json
|
|||
|
|
+ final Map<String, JSONObject> subscriptionGroupJsonConfig = new HashMap<>();
|
|||
|
|
+@@ -102,13 +104,15 @@ public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+ subscriptionGroupTable.put(subscriptionGroup, jsonObject);
|
|||
|
|
+ });
|
|||
|
|
+
|
|||
|
|
+- if (isLoad) {
|
|||
|
|
+- subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
+- (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
+- final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
+- System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
++ if (!isLoad) {
|
|||
|
|
++ System.out.print("RocksDB load error, path=" + path);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
++ subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
++ (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
++ final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
++ System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
++ return;
|
|||
|
|
+ }
|
|||
|
|
+ System.out.print("Config type was not recognized, configType=" + configType + "\n");
|
|||
|
|
+ } finally {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From b4f73e2aabc1b141cec98431899e4090340adf0f Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: mxsm <ljbmxsm@gmail.com>
|
|||
|
|
+Date: Sun, 27 Aug 2023 20:58:58 +0800
|
|||
|
|
+Subject: [PATCH 3/6] [ISSUE #7271] Optimize the configuration for setting the
|
|||
|
|
+ quantity of TimerDequeuePutMessageService (#7272)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../java/org/apache/rocketmq/store/timer/TimerMessageStore.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+index 690f4863e..181f7087a 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+@@ -222,7 +222,7 @@ public class TimerMessageStore {
|
|||
|
|
+ dequeueGetMessageServices[i] = new TimerDequeueGetMessageService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- int putThreadNum = Math.max(storeConfig.getTimerGetMessageThreadNum(), 1);
|
|||
|
|
++ int putThreadNum = Math.max(storeConfig.getTimerPutMessageThreadNum(), 1);
|
|||
|
|
+ dequeuePutMessageServices = new TimerDequeuePutMessageService[putThreadNum];
|
|||
|
|
+ for (int i = 0; i < dequeuePutMessageServices.length; i++) {
|
|||
|
|
+ dequeuePutMessageServices[i] = new TimerDequeuePutMessageService();
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3e100103af68588528bf32f3752a85e8023f46f8 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <tanziyi0925@gmail.com>
|
|||
|
|
+Date: Tue, 29 Aug 2023 13:48:51 +0800
|
|||
|
|
+Subject: [PATCH 4/6] [ISSUE #7277] Enhance rocksDBConfigToJson to support
|
|||
|
|
+ metadata counting (#7276)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../common/config/AbstractRocksDBStorage.java | 4 +-
|
|||
|
|
+ .../common/config/ConfigRocksDBStorage.java | 6 +
|
|||
|
|
+ .../tools/command/MQAdminStartup.java | 4 +-
|
|||
|
|
+ .../ExportMetadataInRocksDBCommand.java | 138 ++++++++++++++++++
|
|||
|
|
+ .../metadata/RocksDBConfigToJsonCommand.java | 122 ----------------
|
|||
|
|
+ ...> ExportMetadataInRocksDBCommandTest.java} | 38 +++--
|
|||
|
|
+ 6 files changed, 173 insertions(+), 139 deletions(-)
|
|||
|
|
+ create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
+ delete mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+ rename tools/src/test/java/org/apache/rocketmq/tools/command/metadata/{KvConfigToJsonCommandTest.java => ExportMetadataInRocksDBCommandTest.java} (62%)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
+index e3673baad..a720a5be3 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
+@@ -385,8 +385,10 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
+ this.options.close();
|
|||
|
|
+ }
|
|||
|
|
+ //4. close db.
|
|||
|
|
+- if (db != null) {
|
|||
|
|
++ if (db != null && !this.readOnly) {
|
|||
|
|
+ this.db.syncWal();
|
|||
|
|
++ }
|
|||
|
|
++ if (db != null) {
|
|||
|
|
+ this.db.closeE();
|
|||
|
|
+ }
|
|||
|
|
+ //5. help gc.
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
+index 9d05ed282..463bd8fed 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/config/ConfigRocksDBStorage.java
|
|||
|
|
+@@ -60,6 +60,12 @@ public class ConfigRocksDBStorage extends AbstractRocksDBStorage {
|
|||
|
|
+ this.readOnly = false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public ConfigRocksDBStorage(final String dbPath, boolean readOnly) {
|
|||
|
|
++ super();
|
|||
|
|
++ this.dbPath = dbPath;
|
|||
|
|
++ this.readOnly = readOnly;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private void initOptions() {
|
|||
|
|
+ this.options = createConfigDBOptions();
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+index 324aa1856..788fa83c2 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java
|
|||
|
|
+@@ -80,7 +80,7 @@ import org.apache.rocketmq.tools.command.message.QueryMsgByOffsetSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgByUniqueKeySubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.QueryMsgTraceByIdSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.message.SendMessageCommand;
|
|||
|
|
+-import org.apache.rocketmq.tools.command.metadata.RocksDBConfigToJsonCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.export.ExportMetadataInRocksDBCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.AddWritePermSubCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.DeleteKvConfigCommand;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.namesrv.GetNamesrvConfigCommand;
|
|||
|
|
+@@ -212,7 +212,6 @@ public class MQAdminStartup {
|
|||
|
|
+
|
|||
|
|
+ initCommand(new ClusterListSubCommand());
|
|||
|
|
+ initCommand(new TopicListSubCommand());
|
|||
|
|
+- initCommand(new RocksDBConfigToJsonCommand());
|
|||
|
|
+
|
|||
|
|
+ initCommand(new UpdateKvConfigCommand());
|
|||
|
|
+ initCommand(new DeleteKvConfigCommand());
|
|||
|
|
+@@ -257,6 +256,7 @@ public class MQAdminStartup {
|
|||
|
|
+ initCommand(new ExportMetadataCommand());
|
|||
|
|
+ initCommand(new ExportConfigsCommand());
|
|||
|
|
+ initCommand(new ExportMetricsCommand());
|
|||
|
|
++ initCommand(new ExportMetadataInRocksDBCommand());
|
|||
|
|
+
|
|||
|
|
+ initCommand(new HAStatusSubCommand());
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..2a7d3fba4
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
+@@ -0,0 +1,138 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tools.command.export;
|
|||
|
|
++
|
|||
|
|
++import com.alibaba.fastjson.JSONObject;
|
|||
|
|
++import org.apache.commons.cli.CommandLine;
|
|||
|
|
++import org.apache.commons.cli.Option;
|
|||
|
|
++import org.apache.commons.cli.Options;
|
|||
|
|
++import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
++import org.apache.rocketmq.common.config.ConfigRocksDBStorage;
|
|||
|
|
++import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
++import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
++import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++import org.rocksdb.RocksIterator;
|
|||
|
|
++
|
|||
|
|
++import java.util.HashMap;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
++import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
++import java.util.function.BiConsumer;
|
|||
|
|
++
|
|||
|
|
++public class ExportMetadataInRocksDBCommand implements SubCommand {
|
|||
|
|
++ private static final String TOPICS_JSON_CONFIG = "topics";
|
|||
|
|
++ private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups";
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandName() {
|
|||
|
|
++ return "exportMetadataInRocksDB";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String commandDesc() {
|
|||
|
|
++ return "export RocksDB kv config (topics/subscriptionGroups)";
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Options buildCommandlineOptions(Options options) {
|
|||
|
|
++ Option pathOption = new Option("p", "path", true,
|
|||
|
|
++ "Absolute path for the metadata directory");
|
|||
|
|
++ pathOption.setRequired(true);
|
|||
|
|
++ options.addOption(pathOption);
|
|||
|
|
++
|
|||
|
|
++ Option configTypeOption = new Option("t", "configType", true, "Name of kv config, e.g. " +
|
|||
|
|
++ "topics/subscriptionGroups");
|
|||
|
|
++ configTypeOption.setRequired(true);
|
|||
|
|
++ options.addOption(configTypeOption);
|
|||
|
|
++
|
|||
|
|
++ Option jsonEnableOption = new Option("j", "jsonEnable", true,
|
|||
|
|
++ "Json format enable, Default: false");
|
|||
|
|
++ jsonEnableOption.setRequired(false);
|
|||
|
|
++ options.addOption(jsonEnableOption);
|
|||
|
|
++
|
|||
|
|
++ return options;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
++ String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
++ if (StringUtils.isEmpty(path) || !UtilAll.isPathExists(path)) {
|
|||
|
|
++ System.out.print("RocksDB path is invalid.\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
++
|
|||
|
|
++ boolean jsonEnable = false;
|
|||
|
|
++ if (commandLine.hasOption("jsonEnable")) {
|
|||
|
|
++ jsonEnable = Boolean.parseBoolean(commandLine.getOptionValue("jsonEnable").trim());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ ConfigRocksDBStorage kvStore = new ConfigRocksDBStorage(path, true /* readOnly */);
|
|||
|
|
++ if (!kvStore.start()) {
|
|||
|
|
++ System.out.print("RocksDB load error, path=" + path + "\n");
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ try {
|
|||
|
|
++ if (TOPICS_JSON_CONFIG.equalsIgnoreCase(configType) || SUBSCRIPTION_GROUP_JSON_CONFIG.equalsIgnoreCase(configType)) {
|
|||
|
|
++ handleExportMetadata(kvStore, configType, jsonEnable);
|
|||
|
|
++ } else {
|
|||
|
|
++ System.out.printf("Invalid config type=%s, Options: topics,subscriptionGroups\n", configType);
|
|||
|
|
++ }
|
|||
|
|
++ } finally {
|
|||
|
|
++ kvStore.shutdown();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private static void handleExportMetadata(ConfigRocksDBStorage kvStore, String configType, boolean jsonEnable) {
|
|||
|
|
++ if (jsonEnable) {
|
|||
|
|
++ final Map<String, JSONObject> jsonConfig = new HashMap<>();
|
|||
|
|
++ final Map<String, JSONObject> configTable = new HashMap<>();
|
|||
|
|
++ iterateKvStore(kvStore, (key, value) -> {
|
|||
|
|
++ final String configKey = new String(key, DataConverter.charset);
|
|||
|
|
++ final String configValue = new String(value, DataConverter.charset);
|
|||
|
|
++ final JSONObject jsonObject = JSONObject.parseObject(configValue);
|
|||
|
|
++ configTable.put(configKey, jsonObject);
|
|||
|
|
++ }
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
++ jsonConfig.put(configType.equalsIgnoreCase(TOPICS_JSON_CONFIG) ? "topicConfigTable" : "subscriptionGroupTable",
|
|||
|
|
++ (JSONObject) JSONObject.toJSON(configTable));
|
|||
|
|
++ final String jsonConfigStr = JSONObject.toJSONString(jsonConfig, true);
|
|||
|
|
++ System.out.print(jsonConfigStr + "\n");
|
|||
|
|
++ } else {
|
|||
|
|
++ AtomicLong count = new AtomicLong(0);
|
|||
|
|
++ iterateKvStore(kvStore, (key, value) -> {
|
|||
|
|
++ final String configKey = new String(key, DataConverter.charset);
|
|||
|
|
++ final String configValue = new String(value, DataConverter.charset);
|
|||
|
|
++ System.out.printf("%d, Key: %s, Value: %s%n", count.incrementAndGet(), configKey, configValue);
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private static void iterateKvStore(ConfigRocksDBStorage kvStore, BiConsumer<byte[], byte[]> biConsumer) {
|
|||
|
|
++ try (RocksIterator iterator = kvStore.iterator()) {
|
|||
|
|
++ iterator.seekToFirst();
|
|||
|
|
++ for (iterator.seekToFirst(); iterator.isValid(); iterator.next()) {
|
|||
|
|
++ biConsumer.accept(iterator.key(), iterator.value());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
+deleted file mode 100644
|
|||
|
|
+index 3fc63e4dd..000000000
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
++++ /dev/null
|
|||
|
|
+@@ -1,122 +0,0 @@
|
|||
|
|
+-/*
|
|||
|
|
+- * 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.tools.command.metadata;
|
|||
|
|
+-
|
|||
|
|
+-import com.alibaba.fastjson.JSONObject;
|
|||
|
|
+-import org.apache.commons.cli.CommandLine;
|
|||
|
|
+-import org.apache.commons.cli.Option;
|
|||
|
|
+-import org.apache.commons.cli.Options;
|
|||
|
|
+-import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+-import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+-import org.apache.rocketmq.common.config.RocksDBConfigManager;
|
|||
|
|
+-import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
+-import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+-import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
+-import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
+-
|
|||
|
|
+-import java.util.HashMap;
|
|||
|
|
+-import java.util.Map;
|
|||
|
|
+-
|
|||
|
|
+-public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+- private static final String TOPICS_JSON_CONFIG = "topics";
|
|||
|
|
+- private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups";
|
|||
|
|
+-
|
|||
|
|
+- @Override
|
|||
|
|
+- public String commandName() {
|
|||
|
|
+- return "rocksDBConfigToJson";
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Override
|
|||
|
|
+- public String commandDesc() {
|
|||
|
|
+- return "Convert RocksDB kv config (topics/subscriptionGroups) to json";
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Override
|
|||
|
|
+- public Options buildCommandlineOptions(Options options) {
|
|||
|
|
+- Option pathOption = new Option("p", "path", true,
|
|||
|
|
+- "Absolute path for the metadata directory");
|
|||
|
|
+- pathOption.setRequired(true);
|
|||
|
|
+- options.addOption(pathOption);
|
|||
|
|
+-
|
|||
|
|
+- Option configTypeOption = new Option("t", "configType", true, "Name of kv config, e.g. " +
|
|||
|
|
+- "topics/subscriptionGroups");
|
|||
|
|
+- configTypeOption.setRequired(true);
|
|||
|
|
+- options.addOption(configTypeOption);
|
|||
|
|
+-
|
|||
|
|
+- return options;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Override
|
|||
|
|
+- public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
+- String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
+- if (StringUtils.isEmpty(path) || !UtilAll.isPathExists(path)) {
|
|||
|
|
+- System.out.print("Rocksdb path is invalid.\n");
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
+-
|
|||
|
|
+- RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(60 * 60 * 1000L);
|
|||
|
|
+- try {
|
|||
|
|
+- if (TOPICS_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+- // for topics.json
|
|||
|
|
+- final Map<String, JSONObject> topicsJsonConfig = new HashMap<>();
|
|||
|
|
+- final Map<String, JSONObject> topicConfigTable = new HashMap<>();
|
|||
|
|
+- boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
+- final String topic = new String(key, DataConverter.charset);
|
|||
|
|
+- final String topicConfig = new String(value, DataConverter.charset);
|
|||
|
|
+- final JSONObject jsonObject = JSONObject.parseObject(topicConfig);
|
|||
|
|
+- topicConfigTable.put(topic, jsonObject);
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- if (!isLoad) {
|
|||
|
|
+- System.out.print("RocksDB load error, path=" + path);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
+- final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
+- System.out.print(topicsJsonStr + "\n");
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- if (SUBSCRIPTION_GROUP_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+- // for subscriptionGroup.json
|
|||
|
|
+- final Map<String, JSONObject> subscriptionGroupJsonConfig = new HashMap<>();
|
|||
|
|
+- final Map<String, JSONObject> subscriptionGroupTable = new HashMap<>();
|
|||
|
|
+- boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
+- final String subscriptionGroup = new String(key, DataConverter.charset);
|
|||
|
|
+- final String subscriptionGroupConfig = new String(value, DataConverter.charset);
|
|||
|
|
+- final JSONObject jsonObject = JSONObject.parseObject(subscriptionGroupConfig);
|
|||
|
|
+- subscriptionGroupTable.put(subscriptionGroup, jsonObject);
|
|||
|
|
+- });
|
|||
|
|
+-
|
|||
|
|
+- if (!isLoad) {
|
|||
|
|
+- System.out.print("RocksDB load error, path=" + path);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
+- (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
+- final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
+- System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- System.out.print("Config type was not recognized, configType=" + configType + "\n");
|
|||
|
|
+- } finally {
|
|||
|
|
+- kvConfigManager.stop();
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-}
|
|||
|
|
+diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/ExportMetadataInRocksDBCommandTest.java
|
|||
|
|
+similarity index 62%
|
|||
|
|
+rename from tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
+rename to tools/src/test/java/org/apache/rocketmq/tools/command/metadata/ExportMetadataInRocksDBCommandTest.java
|
|||
|
|
+index b2f66c7b0..2b938c90f 100644
|
|||
|
|
+--- a/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/KvConfigToJsonCommandTest.java
|
|||
|
|
++++ b/tools/src/test/java/org/apache/rocketmq/tools/command/metadata/ExportMetadataInRocksDBCommandTest.java
|
|||
|
|
+@@ -21,43 +21,53 @@ import org.apache.commons.cli.DefaultParser;
|
|||
|
|
+ import org.apache.commons.cli.Options;
|
|||
|
|
+ import org.apache.rocketmq.srvutil.ServerUtil;
|
|||
|
|
+ import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
++import org.apache.rocketmq.tools.command.export.ExportMetadataInRocksDBCommand;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+
|
|||
|
|
+-public class KvConfigToJsonCommandTest {
|
|||
|
|
++public class ExportMetadataInRocksDBCommandTest {
|
|||
|
|
+ private static final String BASE_PATH = System.getProperty("user.home") + File.separator + "store/config/";
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testExecute() throws SubCommandException {
|
|||
|
|
+ {
|
|||
|
|
+- String[] cases = new String[]{"topics", "subscriptionGroups"};
|
|||
|
|
+- for (String c : cases) {
|
|||
|
|
+- RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ String[][] cases = new String[][] {
|
|||
|
|
++ {"topics", "false"},
|
|||
|
|
++ {"topics", "false1"},
|
|||
|
|
++ {"topics", "true"},
|
|||
|
|
++ {"subscriptionGroups", "false"},
|
|||
|
|
++ {"subscriptionGroups", "false2"},
|
|||
|
|
++ {"subscriptionGroups", "true"}
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ for (String[] c : cases) {
|
|||
|
|
++ ExportMetadataInRocksDBCommand cmd = new ExportMetadataInRocksDBCommand();
|
|||
|
|
+ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
+- String[] subargs = new String[]{"-p " + BASE_PATH + c, "-t " + c};
|
|||
|
|
++ String[] subargs = new String[] {"-p " + BASE_PATH + c[0], "-t " + c[0], "-j " + c[1]};
|
|||
|
|
+ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs,
|
|||
|
|
+- cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
+ cmd.execute(commandLine, options, null);
|
|||
|
|
+- assertThat(commandLine.getOptionValue("p").trim()).isEqualTo(BASE_PATH + c);
|
|||
|
|
+- assertThat(commandLine.getOptionValue("t").trim()).isEqualTo(c);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("p").trim()).isEqualTo(BASE_PATH + c[0]);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("t").trim()).isEqualTo(c[0]);
|
|||
|
|
++ assertThat(commandLine.getOptionValue("j").trim()).isEqualTo(c[1]);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ // invalid cases
|
|||
|
|
+ {
|
|||
|
|
+- String[][] cases = new String[][]{
|
|||
|
|
+- {"-p " + BASE_PATH + "tmpPath", "-t topics"},
|
|||
|
|
+- {"-p ", "-t topics"},
|
|||
|
|
+- {"-p " + BASE_PATH + "topics", "-t invalid_type"}
|
|||
|
|
++ String[][] cases = new String[][] {
|
|||
|
|
++ {"-p " + BASE_PATH + "tmpPath", "-t topics", "-j true"},
|
|||
|
|
++ {"-p ", "-t topics", "-j true"},
|
|||
|
|
++ {"-p " + BASE_PATH + "topics", "-t invalid_type", "-j true"}
|
|||
|
|
+ };
|
|||
|
|
+
|
|||
|
|
+ for (String[] c : cases) {
|
|||
|
|
+- RocksDBConfigToJsonCommand cmd = new RocksDBConfigToJsonCommand();
|
|||
|
|
++ ExportMetadataInRocksDBCommand cmd = new ExportMetadataInRocksDBCommand();
|
|||
|
|
+ Options options = ServerUtil.buildCommandlineOptions(new Options());
|
|||
|
|
+ final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), c,
|
|||
|
|
+- cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
++ cmd.buildCommandlineOptions(options), new DefaultParser());
|
|||
|
|
+ cmd.execute(commandLine, options, null);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From fa549154370cb866a90e37c13a90d2c598d6b1f6 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yuz10 <845238369@qq.com>
|
|||
|
|
+Date: Tue, 29 Aug 2023 15:22:09 +0800
|
|||
|
|
+Subject: [PATCH 5/6] [ISSUE #7261] Slave high CPU usage when
|
|||
|
|
+ enableScheduleAsyncDeliver=true (#7262)
|
|||
|
|
+
|
|||
|
|
+* [ISSUE #6390] Add break to the exception of WHEEL_TIMER_NOT_ENABLE.
|
|||
|
|
+
|
|||
|
|
+* fix broker start fail if mapped file size is 0
|
|||
|
|
+
|
|||
|
|
+* log
|
|||
|
|
+
|
|||
|
|
+* only delete the last empty file
|
|||
|
|
+
|
|||
|
|
+* change dataReadAheadEnable default to true
|
|||
|
|
+
|
|||
|
|
+* fix endless loop when master change to slave.
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/broker/schedule/ScheduleMessageService.java | 7 ++++++-
|
|||
|
|
+ 1 file changed, 6 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
+index aed0ee19f..297b14207 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
+@@ -566,7 +566,8 @@ public class ScheduleMessageService extends ConfigManager {
|
|||
|
|
+ pendingQueue.remove();
|
|||
|
|
+ break;
|
|||
|
|
+ case RUNNING:
|
|||
|
|
+- break;
|
|||
|
|
++ scheduleNextTask();
|
|||
|
|
++ return;
|
|||
|
|
+ case EXCEPTION:
|
|||
|
|
+ if (!isStarted()) {
|
|||
|
|
+ log.warn("HandlePutResultTask shutdown, info={}", putResultProcess.toString());
|
|||
|
|
+@@ -586,6 +587,10 @@ public class ScheduleMessageService extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ scheduleNextTask();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private void scheduleNextTask() {
|
|||
|
|
+ if (isStarted()) {
|
|||
|
|
+ ScheduleMessageService.this.handleExecutorService
|
|||
|
|
+ .schedule(new HandlePutResultTask(this.delayLevel), DELAY_FOR_A_SLEEP, TimeUnit.MILLISECONDS);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 9f34f55e1dac495730c9cd5469f2ab3225b8f0b9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: ShuangxiDing <dingshuangxi888@gmail.com>
|
|||
|
|
+Date: Tue, 29 Aug 2023 15:48:46 +0800
|
|||
|
|
+Subject: [PATCH 6/6] [ISSUE #7226] Filter tlvs in ppv2 which contents not are
|
|||
|
|
+ spec-compliant ASCII characters and space (#7227)
|
|||
|
|
+
|
|||
|
|
+Filter tlvs in ppv2 which not are spec-compliant ASCII characters and space
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/common/utils/BinaryUtil.java | 17 +++++++++++++++++
|
|||
|
|
+ .../grpc/ProxyAndTlsProtocolNegotiator.java | 8 +++++++-
|
|||
|
|
+ .../remoting/netty/NettyRemotingServer.java | 8 +++++++-
|
|||
|
|
+ 3 files changed, 31 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/BinaryUtil.java b/common/src/main/java/org/apache/rocketmq/common/utils/BinaryUtil.java
|
|||
|
|
+index 421adaca4..7b4b24819 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/utils/BinaryUtil.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/utils/BinaryUtil.java
|
|||
|
|
+@@ -43,4 +43,21 @@ public class BinaryUtil {
|
|||
|
|
+ byte[] bytes = calculateMd5(content);
|
|||
|
|
+ return Hex.encodeHexString(bytes, false);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Returns true if subject contains only bytes that are spec-compliant ASCII characters.
|
|||
|
|
++ * @param subject
|
|||
|
|
++ * @return
|
|||
|
|
++ */
|
|||
|
|
++ public static boolean isAscii(byte[] subject) {
|
|||
|
|
++ if (subject == null) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ for (byte b : subject) {
|
|||
|
|
++ if ((b & 0x80) != 0) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/ProxyAndTlsProtocolNegotiator.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/ProxyAndTlsProtocolNegotiator.java
|
|||
|
|
+index ee167bd7b..b584ddfbd 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/ProxyAndTlsProtocolNegotiator.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/ProxyAndTlsProtocolNegotiator.java
|
|||
|
|
+@@ -24,6 +24,7 @@ import io.grpc.netty.shaded.io.grpc.netty.InternalProtocolNegotiator;
|
|||
|
|
+ import io.grpc.netty.shaded.io.grpc.netty.InternalProtocolNegotiators;
|
|||
|
|
+ import io.grpc.netty.shaded.io.grpc.netty.ProtocolNegotiationEvent;
|
|||
|
|
+ import io.grpc.netty.shaded.io.netty.buffer.ByteBuf;
|
|||
|
|
++import io.grpc.netty.shaded.io.netty.buffer.ByteBufUtil;
|
|||
|
|
+ import io.grpc.netty.shaded.io.netty.channel.ChannelHandler;
|
|||
|
|
+ import io.grpc.netty.shaded.io.netty.channel.ChannelHandlerContext;
|
|||
|
|
+ import io.grpc.netty.shaded.io.netty.channel.ChannelInboundHandlerAdapter;
|
|||
|
|
+@@ -44,6 +45,7 @@ import org.apache.commons.collections.CollectionUtils;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.HAProxyConstants;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.BinaryUtil;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+@@ -191,9 +193,13 @@ public class ProxyAndTlsProtocolNegotiator implements InternalProtocolNegotiator
|
|||
|
|
+ }
|
|||
|
|
+ if (CollectionUtils.isNotEmpty(msg.tlvs())) {
|
|||
|
|
+ msg.tlvs().forEach(tlv -> {
|
|||
|
|
++ byte[] valueBytes = ByteBufUtil.getBytes(tlv.content());
|
|||
|
|
++ if (!BinaryUtil.isAscii(valueBytes)) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
+ Attributes.Key<String> key = AttributeKeys.valueOf(
|
|||
|
|
+ HAProxyConstants.PROXY_PROTOCOL_TLV_PREFIX + String.format("%02x", tlv.typeByteValue()));
|
|||
|
|
+- String value = StringUtils.trim(tlv.content().toString(CharsetUtil.UTF_8));
|
|||
|
|
++ String value = StringUtils.trim(new String(valueBytes, CharsetUtil.UTF_8));
|
|||
|
|
+ builder.set(key, value);
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+index 17f138f86..e626260c9 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+@@ -18,6 +18,7 @@ package org.apache.rocketmq.remoting.netty;
|
|||
|
|
+
|
|||
|
|
+ import io.netty.bootstrap.ServerBootstrap;
|
|||
|
|
+ import io.netty.buffer.ByteBuf;
|
|||
|
|
++import io.netty.buffer.ByteBufUtil;
|
|||
|
|
+ import io.netty.buffer.PooledByteBufAllocator;
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import io.netty.channel.ChannelDuplexHandler;
|
|||
|
|
+@@ -58,6 +59,7 @@ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.HAProxyConstants;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.BinaryUtil;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+@@ -787,9 +789,13 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+ if (CollectionUtils.isNotEmpty(msg.tlvs())) {
|
|||
|
|
+ msg.tlvs().forEach(tlv -> {
|
|||
|
|
++ byte[] valueBytes = ByteBufUtil.getBytes(tlv.content());
|
|||
|
|
++ if (!BinaryUtil.isAscii(valueBytes)) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
+ AttributeKey<String> key = AttributeKeys.valueOf(
|
|||
|
|
+ HAProxyConstants.PROXY_PROTOCOL_TLV_PREFIX + String.format("%02x", tlv.typeByteValue()));
|
|||
|
|
+- String value = StringUtils.trim(tlv.content().toString(CharsetUtil.UTF_8));
|
|||
|
|
++ String value = StringUtils.trim(new String(valueBytes, CharsetUtil.UTF_8));
|
|||
|
|
+ channel.attr(key).set(value);
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch014-backport-Queue-Selection-Strategy-Optimization.patch b/patch014-backport-Queue-Selection-Strategy-Optimization.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..90d3553aa
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch014-backport-Queue-Selection-Strategy-Optimization.patch
|
|||
|
|
@@ -0,0 +1,2023 @@
|
|||
|
|
+From b028277018946868838a82a08211071bc231a175 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ji Juntao <juntao.jjt@alibaba-inc.com>
|
|||
|
|
+Date: Tue, 29 Aug 2023 16:13:38 +0800
|
|||
|
|
+Subject: [PATCH] [ISSUE #6567] [RIP-63] Queue Selection Strategy Optimization
|
|||
|
|
+ (#6568)
|
|||
|
|
+
|
|||
|
|
+Optimize the proxy's and client's selection strategy for brokers when sending messages, and use multiple selection strategies as a pipeline to filter suitable queues.
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/client/ClientConfig.java | 54 +++++
|
|||
|
|
+ .../client/common/ThreadLocalIndex.java | 8 +
|
|||
|
|
+ .../rocketmq/client/impl/MQClientAPIImpl.java | 12 +-
|
|||
|
|
+ .../client/impl/factory/MQClientInstance.java | 7 +
|
|||
|
|
+ .../impl/producer/DefaultMQProducerImpl.java | 87 ++++++--
|
|||
|
|
+ .../impl/producer/TopicPublishInfo.java | 40 ++++
|
|||
|
|
+ .../client/latency/LatencyFaultTolerance.java | 66 +++++-
|
|||
|
|
+ .../latency/LatencyFaultToleranceImpl.java | 189 ++++++++++++++----
|
|||
|
|
+ .../client/latency/MQFaultStrategy.java | 155 ++++++++++----
|
|||
|
|
+ .../rocketmq/client/latency/Resolver.java | 17 +-
|
|||
|
|
+ .../client/latency/ServiceDetector.java | 30 +++
|
|||
|
|
+ .../LatencyFaultToleranceImplTest.java | 36 +++-
|
|||
|
|
+ .../processor/DefaultRequestProcessor.java | 24 ---
|
|||
|
|
+ .../rocketmq/proxy/config/ProxyConfig.java | 46 +++++
|
|||
|
|
+ .../grpc/v2/producer/SendMessageActivity.java | 2 +-
|
|||
|
|
+ .../proxy/processor/ProducerProcessor.java | 18 +-
|
|||
|
|
+ .../service/route/LocalTopicRouteService.java | 2 +-
|
|||
|
|
+ .../service/route/MessageQueueSelector.java | 95 ++++++++-
|
|||
|
|
+ .../proxy/service/route/MessageQueueView.java | 18 +-
|
|||
|
|
+ .../service/route/TopicRouteService.java | 80 +++++++-
|
|||
|
|
+ .../consumer/ReceiveMessageActivityTest.java | 5 +-
|
|||
|
|
+ .../v2/producer/SendMessageActivityTest.java | 82 +++++++-
|
|||
|
|
+ .../proxy/service/BaseServiceTest.java | 4 +-
|
|||
|
|
+ .../route/MessageQueueSelectorTest.java | 8 +-
|
|||
|
|
+ .../sysmessage/HeartbeatSyncerTest.java | 2 +-
|
|||
|
|
+ .../ClusterTransactionServiceTest.java | 8 +-
|
|||
|
|
+ 26 files changed, 919 insertions(+), 176 deletions(-)
|
|||
|
|
+ rename remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetRemoteClientConfigBody.java => client/src/main/java/org/apache/rocketmq/client/latency/Resolver.java (65%)
|
|||
|
|
+ create mode 100644 client/src/main/java/org/apache/rocketmq/client/latency/ServiceDetector.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
+index f87450f66..bb0fe3522 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
+@@ -38,6 +38,8 @@ public class ClientConfig {
|
|||
|
|
+ public static final String SOCKS_PROXY_CONFIG = "com.rocketmq.socks.proxy.config";
|
|||
|
|
+ public static final String DECODE_READ_BODY = "com.rocketmq.read.body";
|
|||
|
|
+ public static final String DECODE_DECOMPRESS_BODY = "com.rocketmq.decompress.body";
|
|||
|
|
++ public static final String SEND_LATENCY_ENABLE = "com.rocketmq.sendLatencyEnable";
|
|||
|
|
++ public static final String START_DETECTOR_ENABLE = "com.rocketmq.startDetectorEnable";
|
|||
|
|
+ public static final String HEART_BEAT_V2 = "com.rocketmq.heartbeat.v2";
|
|||
|
|
+ private String namesrvAddr = NameServerAddressUtils.getNameServerAddresses();
|
|||
|
|
+ private String clientIP = NetworkUtil.getLocalAddress();
|
|||
|
|
+@@ -72,6 +74,8 @@ public class ClientConfig {
|
|||
|
|
+ private String socksProxyConfig = System.getProperty(SOCKS_PROXY_CONFIG, "{}");
|
|||
|
|
+
|
|||
|
|
+ private int mqClientApiTimeout = 3 * 1000;
|
|||
|
|
++ private int detectTimeout = 200;
|
|||
|
|
++ private int detectInterval = 2 * 1000;
|
|||
|
|
+
|
|||
|
|
+ private LanguageCode language = LanguageCode.JAVA;
|
|||
|
|
+
|
|||
|
|
+@@ -81,6 +85,15 @@ public class ClientConfig {
|
|||
|
|
+ */
|
|||
|
|
+ protected boolean enableStreamRequestType = false;
|
|||
|
|
+
|
|||
|
|
++ /**
|
|||
|
|
++ * Enable the fault tolerance mechanism of the client sending process.
|
|||
|
|
++ * DO NOT OPEN when ORDER messages are required.
|
|||
|
|
++ * Turning on will interfere with the queue selection functionality,
|
|||
|
|
++ * possibly conflicting with the order message.
|
|||
|
|
++ */
|
|||
|
|
++ private boolean sendLatencyEnable = Boolean.parseBoolean(System.getProperty(SEND_LATENCY_ENABLE, "false"));
|
|||
|
|
++ private boolean startDetectorEnable = Boolean.parseBoolean(System.getProperty(START_DETECTOR_ENABLE, "false"));
|
|||
|
|
++
|
|||
|
|
+ public String buildMQClientId() {
|
|||
|
|
+ StringBuilder sb = new StringBuilder();
|
|||
|
|
+ sb.append(this.getClientIP());
|
|||
|
|
+@@ -186,6 +199,10 @@ public class ClientConfig {
|
|||
|
|
+ this.decodeDecompressBody = cc.decodeDecompressBody;
|
|||
|
|
+ this.enableStreamRequestType = cc.enableStreamRequestType;
|
|||
|
|
+ this.useHeartbeatV2 = cc.useHeartbeatV2;
|
|||
|
|
++ this.startDetectorEnable = cc.startDetectorEnable;
|
|||
|
|
++ this.sendLatencyEnable = cc.sendLatencyEnable;
|
|||
|
|
++ this.detectInterval = cc.detectInterval;
|
|||
|
|
++ this.detectTimeout = cc.detectTimeout;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ClientConfig cloneClientConfig() {
|
|||
|
|
+@@ -210,6 +227,10 @@ public class ClientConfig {
|
|||
|
|
+ cc.decodeDecompressBody = decodeDecompressBody;
|
|||
|
|
+ cc.enableStreamRequestType = enableStreamRequestType;
|
|||
|
|
+ cc.useHeartbeatV2 = useHeartbeatV2;
|
|||
|
|
++ cc.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ cc.sendLatencyEnable = sendLatencyEnable;
|
|||
|
|
++ cc.detectInterval = detectInterval;
|
|||
|
|
++ cc.detectTimeout = detectTimeout;
|
|||
|
|
+ return cc;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -381,6 +402,38 @@ public class ClientConfig {
|
|||
|
|
+ this.enableStreamRequestType = enableStreamRequestType;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isSendLatencyEnable() {
|
|||
|
|
++ return sendLatencyEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setSendLatencyEnable(boolean sendLatencyEnable) {
|
|||
|
|
++ this.sendLatencyEnable = sendLatencyEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isStartDetectorEnable() {
|
|||
|
|
++ return startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setStartDetectorEnable(boolean startDetectorEnable) {
|
|||
|
|
++ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getDetectTimeout() {
|
|||
|
|
++ return this.detectTimeout;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setDetectTimeout(int detectTimeout) {
|
|||
|
|
++ this.detectTimeout = detectTimeout;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getDetectInterval() {
|
|||
|
|
++ return this.detectInterval;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setDetectInterval(int detectInterval) {
|
|||
|
|
++ this.detectInterval = detectInterval;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean isUseHeartbeatV2() {
|
|||
|
|
+ return useHeartbeatV2;
|
|||
|
|
+ }
|
|||
|
|
+@@ -403,6 +456,7 @@ public class ClientConfig {
|
|||
|
|
+ + ", socksProxyConfig=" + socksProxyConfig + ", language=" + language.name()
|
|||
|
|
+ + ", namespace=" + namespace + ", mqClientApiTimeout=" + mqClientApiTimeout
|
|||
|
|
+ + ", decodeReadBody=" + decodeReadBody + ", decodeDecompressBody=" + decodeDecompressBody
|
|||
|
|
++ + ", sendLatencyEnable=" + sendLatencyEnable + ", startDetectorEnable=" + startDetectorEnable
|
|||
|
|
+ + ", enableStreamRequestType=" + enableStreamRequestType + ", useHeartbeatV2=" + useHeartbeatV2 + "]";
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/common/ThreadLocalIndex.java b/client/src/main/java/org/apache/rocketmq/client/common/ThreadLocalIndex.java
|
|||
|
|
+index 4a3d90135..3a086c13d 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/common/ThreadLocalIndex.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/common/ThreadLocalIndex.java
|
|||
|
|
+@@ -33,6 +33,14 @@ public class ThreadLocalIndex {
|
|||
|
|
+ return index & POSITIVE_MASK;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void reset() {
|
|||
|
|
++ int index = Math.abs(random.nextInt(Integer.MAX_VALUE));
|
|||
|
|
++ if (index < 0) {
|
|||
|
|
++ index = 0;
|
|||
|
|
++ }
|
|||
|
|
++ this.threadLocalIndex.set(index);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public String toString() {
|
|||
|
|
+ return "ThreadLocalIndex{" +
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+index 213c26fd6..3201a493f 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+@@ -666,7 +666,7 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false);
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false, true);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -684,14 +684,14 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false);
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false, true);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true);
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true, true);
|
|||
|
|
+ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+ retryTimesWhenSendFailed, times, e, context, false, producer);
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true);
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true, true);
|
|||
|
|
+ if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+ MQClientException ex = new MQClientException("send request failed", responseFuture.getCause());
|
|||
|
|
+ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+@@ -711,7 +711,7 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Exception ex) {
|
|||
|
|
+ long cost = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+- producer.updateFaultItem(brokerName, cost, true);
|
|||
|
|
++ producer.updateFaultItem(brokerName, cost, true, false);
|
|||
|
|
+ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+ retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+ }
|
|||
|
|
+@@ -735,7 +735,7 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ if (needRetry && tmp <= timesTotal) {
|
|||
|
|
+ String retryBrokerName = brokerName;//by default, it will send to the same broker
|
|||
|
|
+ if (topicPublishInfo != null) { //select one message queue accordingly, in order to determine which broker to send
|
|||
|
|
+- MessageQueue mqChosen = producer.selectOneMessageQueue(topicPublishInfo, brokerName);
|
|||
|
|
++ MessageQueue mqChosen = producer.selectOneMessageQueue(topicPublishInfo, brokerName, false);
|
|||
|
|
+ retryBrokerName = instance.getBrokerNameFromMessageQueue(mqChosen);
|
|||
|
|
+ }
|
|||
|
|
+ String addr = instance.findBrokerAddressInPublish(retryBrokerName);
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
+index 8851bc815..9484b26f8 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
+@@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.ThreadFactory;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
+ import java.util.concurrent.locks.Lock;
|
|||
|
|
+@@ -125,6 +126,12 @@ public class MQClientInstance {
|
|||
|
|
+ private final Set<String/* Broker address */> brokerSupportV2HeartbeatSet = new HashSet();
|
|||
|
|
+ private final ConcurrentMap<String, Integer> brokerAddrHeartbeatFingerprintTable = new ConcurrentHashMap();
|
|||
|
|
+ private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "MQClientFactoryScheduledThread"));
|
|||
|
|
++ private final ScheduledExecutorService fetchRemoteConfigExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public Thread newThread(Runnable r) {
|
|||
|
|
++ return new Thread(r, "MQClientFactoryFetchRemoteConfigScheduledThread");
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
+ private final PullMessageService pullMessageService;
|
|||
|
|
+ private final RebalanceService rebalanceService;
|
|||
|
|
+ private final DefaultMQProducer defaultMQProducer;
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+index 3f4c6e5f7..bbbb17b07 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+@@ -33,6 +33,8 @@ import java.util.concurrent.RejectedExecutionException;
|
|||
|
|
+ import java.util.concurrent.Semaphore;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
++
|
|||
|
|
++import com.google.common.base.Optional;
|
|||
|
|
+ import org.apache.rocketmq.client.QueryResult;
|
|||
|
|
+ import org.apache.rocketmq.client.Validators;
|
|||
|
|
+ import org.apache.rocketmq.client.common.ClientErrorCode;
|
|||
|
|
+@@ -49,6 +51,8 @@ import org.apache.rocketmq.client.impl.CommunicationMode;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.MQClientManager;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.factory.MQClientInstance;
|
|||
|
|
+ import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
++import org.apache.rocketmq.client.latency.Resolver;
|
|||
|
|
++import org.apache.rocketmq.client.latency.ServiceDetector;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.DefaultMQProducer;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.LocalTransactionExecuter;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.LocalTransactionState;
|
|||
|
|
+@@ -112,7 +116,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ private ServiceState serviceState = ServiceState.CREATE_JUST;
|
|||
|
|
+ private MQClientInstance mQClientFactory;
|
|||
|
|
+ private ArrayList<CheckForbiddenHook> checkForbiddenHookList = new ArrayList<>();
|
|||
|
|
+- private MQFaultStrategy mqFaultStrategy = new MQFaultStrategy();
|
|||
|
|
++ private MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+ private ExecutorService asyncSenderExecutor;
|
|||
|
|
+
|
|||
|
|
+ // compression related
|
|||
|
|
+@@ -153,8 +157,38 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ semaphoreAsyncSendSize = new Semaphore(1024 * 1024, true);
|
|||
|
|
+ log.info("semaphoreAsyncSendSize can not be smaller than 1M.");
|
|||
|
|
+ }
|
|||
|
|
+- }
|
|||
|
|
+
|
|||
|
|
++ ServiceDetector serviceDetector = new ServiceDetector() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean detect(String endpoint, long timeoutMillis) {
|
|||
|
|
++ Optional<String> candidateTopic = pickTopic();
|
|||
|
|
++ if (!candidateTopic.isPresent()) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ try {
|
|||
|
|
++ MessageQueue mq = new MessageQueue(candidateTopic.get(), null, 0);
|
|||
|
|
++ mQClientFactory.getMQClientAPIImpl()
|
|||
|
|
++ .getMaxOffset(endpoint, mq, timeoutMillis);
|
|||
|
|
++ return true;
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ this.mqFaultStrategy = new MQFaultStrategy(defaultMQProducer.cloneClientConfig(), new Resolver() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public String resolve(String name) {
|
|||
|
|
++ return DefaultMQProducerImpl.this.mQClientFactory.findBrokerAddressInPublish(name);
|
|||
|
|
++ }
|
|||
|
|
++ }, serviceDetector);
|
|||
|
|
++ }
|
|||
|
|
++ private Optional<String> pickTopic() {
|
|||
|
|
++ if (topicPublishInfoTable.isEmpty()) {
|
|||
|
|
++ return Optional.absent();
|
|||
|
|
++ }
|
|||
|
|
++ return Optional.of(topicPublishInfoTable.keySet().iterator().next());
|
|||
|
|
++ }
|
|||
|
|
+ public void registerCheckForbiddenHook(CheckForbiddenHook checkForbiddenHook) {
|
|||
|
|
+ this.checkForbiddenHookList.add(checkForbiddenHook);
|
|||
|
|
+ log.info("register a new checkForbiddenHook. hookName={}, allHookSize={}", checkForbiddenHook.hookName(),
|
|||
|
|
+@@ -229,6 +263,10 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ mQClientFactory.start();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
++ this.mqFaultStrategy.startDetector();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ log.info("the producer [{}] start OK. sendMessageWithVIPChannel={}", this.defaultMQProducer.getProducerGroup(),
|
|||
|
|
+ this.defaultMQProducer.isSendMessageWithVIPChannel());
|
|||
|
|
+ this.serviceState = ServiceState.RUNNING;
|
|||
|
|
+@@ -273,6 +311,9 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ if (shutdownFactory) {
|
|||
|
|
+ this.mQClientFactory.shutdown();
|
|||
|
|
+ }
|
|||
|
|
++ if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
++ this.mqFaultStrategy.shutdown();
|
|||
|
|
++ }
|
|||
|
|
+ RequestFutureHolder.getInstance().shutdown(this);
|
|||
|
|
+ log.info("the producer [{}] shutdown OK", this.defaultMQProducer.getProducerGroup());
|
|||
|
|
+ this.serviceState = ServiceState.SHUTDOWN_ALREADY;
|
|||
|
|
+@@ -574,7 +615,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MessageQueue invokeMessageQueueSelector(Message msg, MessageQueueSelector selector, Object arg,
|
|||
|
|
+- final long timeout) throws MQClientException, RemotingTooMuchRequestException {
|
|||
|
|
++ final long timeout) throws MQClientException, RemotingTooMuchRequestException {
|
|||
|
|
+ long beginStartTime = System.currentTimeMillis();
|
|||
|
|
+ this.makeSureStateOK();
|
|||
|
|
+ Validators.checkMessage(msg, this.defaultMQProducer);
|
|||
|
|
+@@ -584,7 +625,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ MessageQueue mq = null;
|
|||
|
|
+ try {
|
|||
|
|
+ List<MessageQueue> messageQueueList =
|
|||
|
|
+- mQClientFactory.getMQAdminImpl().parsePublishMessageQueues(topicPublishInfo.getMessageQueueList());
|
|||
|
|
++ mQClientFactory.getMQAdminImpl().parsePublishMessageQueues(topicPublishInfo.getMessageQueueList());
|
|||
|
|
+ Message userMessage = MessageAccessor.cloneMessage(msg);
|
|||
|
|
+ String userTopic = NamespaceUtil.withoutNamespace(userMessage.getTopic(), mQClientFactory.getClientConfig().getNamespace());
|
|||
|
|
+ userMessage.setTopic(userTopic);
|
|||
|
|
+@@ -609,12 +650,13 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ throw new MQClientException("No route info for this topic, " + msg.getTopic(), null);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName) {
|
|||
|
|
+- return this.mqFaultStrategy.selectOneMessageQueue(tpInfo, lastBrokerName);
|
|||
|
|
++ public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName, final boolean resetIndex) {
|
|||
|
|
++ return this.mqFaultStrategy.selectOneMessageQueue(tpInfo, lastBrokerName, resetIndex);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation) {
|
|||
|
|
+- this.mqFaultStrategy.updateFaultItem(brokerName, currentLatency, isolation);
|
|||
|
|
++ public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation,
|
|||
|
|
++ boolean reachable) {
|
|||
|
|
++ this.mqFaultStrategy.updateFaultItem(brokerName, currentLatency, isolation, reachable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void validateNameServerSetting() throws MQClientException {
|
|||
|
|
+@@ -647,9 +689,13 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ int timesTotal = communicationMode == CommunicationMode.SYNC ? 1 + this.defaultMQProducer.getRetryTimesWhenSendFailed() : 1;
|
|||
|
|
+ int times = 0;
|
|||
|
|
+ String[] brokersSent = new String[timesTotal];
|
|||
|
|
++ boolean resetIndex = false;
|
|||
|
|
+ for (; times < timesTotal; times++) {
|
|||
|
|
+ String lastBrokerName = null == mq ? null : mq.getBrokerName();
|
|||
|
|
+- MessageQueue mqSelected = this.selectOneMessageQueue(topicPublishInfo, lastBrokerName);
|
|||
|
|
++ if (times > 0) {
|
|||
|
|
++ resetIndex = true;
|
|||
|
|
++ }
|
|||
|
|
++ MessageQueue mqSelected = this.selectOneMessageQueue(topicPublishInfo, lastBrokerName, resetIndex);
|
|||
|
|
+ if (mqSelected != null) {
|
|||
|
|
+ mq = mqSelected;
|
|||
|
|
+ brokersSent[times] = mq.getBrokerName();
|
|||
|
|
+@@ -667,7 +713,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+
|
|||
|
|
+ sendResult = this.sendKernelImpl(msg, mq, communicationMode, sendCallback, topicPublishInfo, timeout - costTime);
|
|||
|
|
+ endTimestamp = System.currentTimeMillis();
|
|||
|
|
+- this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false);
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false, true);
|
|||
|
|
+ switch (communicationMode) {
|
|||
|
|
+ case ASYNC:
|
|||
|
|
+ return null;
|
|||
|
|
+@@ -684,9 +730,22 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ default:
|
|||
|
|
+ break;
|
|||
|
|
+ }
|
|||
|
|
+- } catch (RemotingException | MQClientException e) {
|
|||
|
|
++ } catch (MQClientException e) {
|
|||
|
|
+ endTimestamp = System.currentTimeMillis();
|
|||
|
|
+- this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true);
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false, true);
|
|||
|
|
++ log.warn("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq, e);
|
|||
|
|
++ log.warn(msg.toString());
|
|||
|
|
++ exception = e;
|
|||
|
|
++ continue;
|
|||
|
|
++ } catch (RemotingException e) {
|
|||
|
|
++ endTimestamp = System.currentTimeMillis();
|
|||
|
|
++ if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
++ // Set this broker unreachable when detecting schedule task is running for RemotingException.
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true, false);
|
|||
|
|
++ } else {
|
|||
|
|
++ // Otherwise, isolate this broker.
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true, true);
|
|||
|
|
++ }
|
|||
|
|
+ log.warn("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq, e);
|
|||
|
|
+ if (log.isDebugEnabled()) {
|
|||
|
|
+ log.debug(msg.toString());
|
|||
|
|
+@@ -695,7 +754,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ continue;
|
|||
|
|
+ } catch (MQBrokerException e) {
|
|||
|
|
+ endTimestamp = System.currentTimeMillis();
|
|||
|
|
+- this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true);
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, true, false);
|
|||
|
|
+ log.warn("sendKernelImpl exception, resend at once, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq, e);
|
|||
|
|
+ if (log.isDebugEnabled()) {
|
|||
|
|
+ log.debug(msg.toString());
|
|||
|
|
+@@ -712,7 +771,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ }
|
|||
|
|
+ } catch (InterruptedException e) {
|
|||
|
|
+ endTimestamp = System.currentTimeMillis();
|
|||
|
|
+- this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false);
|
|||
|
|
++ this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false, true);
|
|||
|
|
+ log.warn("sendKernelImpl exception, throw exception, InvokeID: %s, RT: %sms, Broker: %s", invokeID, endTimestamp - beginTimestampPrev, mq, e);
|
|||
|
|
+ if (log.isDebugEnabled()) {
|
|||
|
|
+ log.debug(msg.toString());
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/TopicPublishInfo.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/TopicPublishInfo.java
|
|||
|
|
+index 275ada7ac..37b1f3252 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/TopicPublishInfo.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/TopicPublishInfo.java
|
|||
|
|
+@@ -18,6 +18,8 @@ package org.apache.rocketmq.client.impl.producer;
|
|||
|
|
+
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++
|
|||
|
|
++import com.google.common.base.Preconditions;
|
|||
|
|
+ import org.apache.rocketmq.client.common.ThreadLocalIndex;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.QueueData;
|
|||
|
|
+@@ -30,6 +32,10 @@ public class TopicPublishInfo {
|
|||
|
|
+ private volatile ThreadLocalIndex sendWhichQueue = new ThreadLocalIndex();
|
|||
|
|
+ private TopicRouteData topicRouteData;
|
|||
|
|
+
|
|||
|
|
++ public interface QueueFilter {
|
|||
|
|
++ boolean filter(MessageQueue mq);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean isOrderTopic() {
|
|||
|
|
+ return orderTopic;
|
|||
|
|
+ }
|
|||
|
|
+@@ -66,6 +72,40 @@ public class TopicPublishInfo {
|
|||
|
|
+ this.haveTopicRouterInfo = haveTopicRouterInfo;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public MessageQueue selectOneMessageQueue(QueueFilter ...filter) {
|
|||
|
|
++ return selectOneMessageQueue(this.messageQueueList, this.sendWhichQueue, filter);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private MessageQueue selectOneMessageQueue(List<MessageQueue> messageQueueList, ThreadLocalIndex sendQueue, QueueFilter ...filter) {
|
|||
|
|
++ if (messageQueueList == null || messageQueueList.isEmpty()) {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (filter != null && filter.length != 0) {
|
|||
|
|
++ for (int i = 0; i < messageQueueList.size(); i++) {
|
|||
|
|
++ int index = Math.abs(sendQueue.incrementAndGet() % messageQueueList.size());
|
|||
|
|
++ MessageQueue mq = messageQueueList.get(index);
|
|||
|
|
++ boolean filterResult = true;
|
|||
|
|
++ for (QueueFilter f: filter) {
|
|||
|
|
++ Preconditions.checkNotNull(f);
|
|||
|
|
++ filterResult &= f.filter(mq);
|
|||
|
|
++ }
|
|||
|
|
++ if (filterResult) {
|
|||
|
|
++ return mq;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ int index = Math.abs(sendQueue.incrementAndGet() % messageQueueList.size());
|
|||
|
|
++ return messageQueueList.get(index);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void resetIndex() {
|
|||
|
|
++ this.sendWhichQueue.reset();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public MessageQueue selectOneMessageQueue(final String lastBrokerName) {
|
|||
|
|
+ if (lastBrokerName == null) {
|
|||
|
|
+ return selectOneMessageQueue();
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
+index 09a8aa461..72d2f3450 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
+@@ -18,11 +18,75 @@
|
|||
|
|
+ package org.apache.rocketmq.client.latency;
|
|||
|
|
+
|
|||
|
|
+ public interface LatencyFaultTolerance<T> {
|
|||
|
|
+- void updateFaultItem(final T name, final long currentLatency, final long notAvailableDuration);
|
|||
|
|
++ /**
|
|||
|
|
++ * Update brokers' states, to decide if they are good or not.
|
|||
|
|
++ *
|
|||
|
|
++ * @param name Broker's name.
|
|||
|
|
++ * @param currentLatency Current message sending process's latency.
|
|||
|
|
++ * @param notAvailableDuration Corresponding not available time, ms. The broker will be not available until it
|
|||
|
|
++ * spends such time.
|
|||
|
|
++ * @param reachable To decide if this broker is reachable or not.
|
|||
|
|
++ */
|
|||
|
|
++ void updateFaultItem(final T name, final long currentLatency, final long notAvailableDuration,
|
|||
|
|
++ final boolean reachable);
|
|||
|
|
+
|
|||
|
|
++ /**
|
|||
|
|
++ * To check if this broker is available.
|
|||
|
|
++ *
|
|||
|
|
++ * @param name Broker's name.
|
|||
|
|
++ * @return boolean variable, if this is true, then the broker is available.
|
|||
|
|
++ */
|
|||
|
|
+ boolean isAvailable(final T name);
|
|||
|
|
+
|
|||
|
|
++ /**
|
|||
|
|
++ * To check if this broker is reachable.
|
|||
|
|
++ *
|
|||
|
|
++ * @param name Broker's name.
|
|||
|
|
++ * @return boolean variable, if this is true, then the broker is reachable.
|
|||
|
|
++ */
|
|||
|
|
++ boolean isReachable(final T name);
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Remove the broker in this fault item table.
|
|||
|
|
++ *
|
|||
|
|
++ * @param name broker's name.
|
|||
|
|
++ */
|
|||
|
|
+ void remove(final T name);
|
|||
|
|
+
|
|||
|
|
++ /**
|
|||
|
|
++ * The worst situation, no broker can be available. Then choose random one.
|
|||
|
|
++ *
|
|||
|
|
++ * @return A random mq will be returned.
|
|||
|
|
++ */
|
|||
|
|
+ T pickOneAtLeast();
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Start a new thread, to detect the broker's reachable tag.
|
|||
|
|
++ */
|
|||
|
|
++ void startDetector();
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Shutdown threads that started by LatencyFaultTolerance.
|
|||
|
|
++ */
|
|||
|
|
++ void shutdown();
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * A function reserved, just detect by once, won't create a new thread.
|
|||
|
|
++ */
|
|||
|
|
++ void detectByOneRound();
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Use it to set the detect timeout bound.
|
|||
|
|
++ *
|
|||
|
|
++ * @param detectTimeout timeout bound
|
|||
|
|
++ */
|
|||
|
|
++ void setDetectTimeout(final int detectTimeout);
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Use it to set the detector's detector interval for each broker (each broker will be detected once during this
|
|||
|
|
++ * time)
|
|||
|
|
++ *
|
|||
|
|
++ * @param detectInterval each broker's detecting interval
|
|||
|
|
++ */
|
|||
|
|
++ void setDetectInterval(final int detectInterval);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
+index 93795d957..8af629574 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
+@@ -21,30 +21,97 @@ import java.util.Collections;
|
|||
|
|
+ import java.util.Enumeration;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
++import java.util.concurrent.Executors;
|
|||
|
|
++import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.ThreadFactory;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.client.common.ThreadLocalIndex;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String> {
|
|||
|
|
+- private final ConcurrentHashMap<String, FaultItem> faultItemTable = new ConcurrentHashMap<>(16);
|
|||
|
|
++ private final static Logger log = LoggerFactory.getLogger(MQFaultStrategy.class);
|
|||
|
|
++ private final ConcurrentHashMap<String, FaultItem> faultItemTable = new ConcurrentHashMap<String, FaultItem>(16);
|
|||
|
|
++ private int detectTimeout = 200;
|
|||
|
|
++ private int detectInterval = 2000;
|
|||
|
|
++ private final ThreadLocalIndex whichItemWorst = new ThreadLocalIndex();
|
|||
|
|
++ private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public Thread newThread(Runnable r) {
|
|||
|
|
++ return new Thread(r, "LatencyFaultToleranceScheduledThread");
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
+
|
|||
|
|
+- private final ThreadLocalIndex randomItem = new ThreadLocalIndex();
|
|||
|
|
++ private final Resolver resolver;
|
|||
|
|
++
|
|||
|
|
++ private final ServiceDetector serviceDetector;
|
|||
|
|
++
|
|||
|
|
++ public LatencyFaultToleranceImpl(Resolver resolver, ServiceDetector serviceDetector) {
|
|||
|
|
++ this.resolver = resolver;
|
|||
|
|
++ this.serviceDetector = serviceDetector;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void detectByOneRound() {
|
|||
|
|
++ for (Map.Entry<String, FaultItem> item : this.faultItemTable.entrySet()) {
|
|||
|
|
++ FaultItem brokerItem = item.getValue();
|
|||
|
|
++ if (System.currentTimeMillis() - brokerItem.checkStamp >= 0) {
|
|||
|
|
++ brokerItem.checkStamp = System.currentTimeMillis() + this.detectInterval;
|
|||
|
|
++ String brokerAddr = resolver.resolve(brokerItem.getName());
|
|||
|
|
++ if (brokerAddr == null) {
|
|||
|
|
++ faultItemTable.remove(item.getKey());
|
|||
|
|
++ continue;
|
|||
|
|
++ }
|
|||
|
|
++ if (null == serviceDetector) {
|
|||
|
|
++ continue;
|
|||
|
|
++ }
|
|||
|
|
++ boolean serviceOK = serviceDetector.detect(brokerAddr, detectTimeout);
|
|||
|
|
++ if (serviceOK && !brokerItem.reachableFlag) {
|
|||
|
|
++ log.info(brokerItem.name + " is reachable now, then it can be used.");
|
|||
|
|
++ brokerItem.reachableFlag = true;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void startDetector() {
|
|||
|
|
++ this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void run() {
|
|||
|
|
++ try {
|
|||
|
|
++ detectByOneRound();
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ log.warn("Unexpected exception raised while detecting service reachability", e);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }, 3, 3, TimeUnit.SECONDS);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void shutdown() {
|
|||
|
|
++ this.scheduledExecutorService.shutdown();
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public void updateFaultItem(final String name, final long currentLatency, final long notAvailableDuration) {
|
|||
|
|
++ public void updateFaultItem(final String name, final long currentLatency, final long notAvailableDuration,
|
|||
|
|
++ final boolean reachable) {
|
|||
|
|
+ FaultItem old = this.faultItemTable.get(name);
|
|||
|
|
+ if (null == old) {
|
|||
|
|
+ final FaultItem faultItem = new FaultItem(name);
|
|||
|
|
+ faultItem.setCurrentLatency(currentLatency);
|
|||
|
|
+- faultItem.setStartTimestamp(System.currentTimeMillis() + notAvailableDuration);
|
|||
|
|
+-
|
|||
|
|
++ faultItem.updateNotAvailableDuration(notAvailableDuration);
|
|||
|
|
++ faultItem.setReachable(reachable);
|
|||
|
|
+ old = this.faultItemTable.putIfAbsent(name, faultItem);
|
|||
|
|
+- if (old != null) {
|
|||
|
|
+- old.setCurrentLatency(currentLatency);
|
|||
|
|
+- old.setStartTimestamp(System.currentTimeMillis() + notAvailableDuration);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (null != old) {
|
|||
|
|
+ old.setCurrentLatency(currentLatency);
|
|||
|
|
+- old.setStartTimestamp(System.currentTimeMillis() + notAvailableDuration);
|
|||
|
|
++ old.updateNotAvailableDuration(notAvailableDuration);
|
|||
|
|
++ old.setReachable(reachable);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (!reachable) {
|
|||
|
|
++ log.info(name + " is unreachable, it will not be used until it's reachable");
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -57,6 +124,14 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isReachable(final String name) {
|
|||
|
|
++ final FaultItem faultItem = this.faultItemTable.get(name);
|
|||
|
|
++ if (faultItem != null) {
|
|||
|
|
++ return faultItem.isReachable();
|
|||
|
|
++ }
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public void remove(final String name) {
|
|||
|
|
+ this.faultItemTable.remove(name);
|
|||
|
|
+@@ -65,68 +140,98 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+ @Override
|
|||
|
|
+ public String pickOneAtLeast() {
|
|||
|
|
+ final Enumeration<FaultItem> elements = this.faultItemTable.elements();
|
|||
|
|
+- List<FaultItem> tmpList = new LinkedList<>();
|
|||
|
|
++ List<FaultItem> tmpList = new LinkedList<FaultItem>();
|
|||
|
|
+ while (elements.hasMoreElements()) {
|
|||
|
|
+ final FaultItem faultItem = elements.nextElement();
|
|||
|
|
+ tmpList.add(faultItem);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ if (!tmpList.isEmpty()) {
|
|||
|
|
+- Collections.sort(tmpList);
|
|||
|
|
+- final int half = tmpList.size() / 2;
|
|||
|
|
+- if (half <= 0) {
|
|||
|
|
+- return tmpList.get(0).getName();
|
|||
|
|
+- } else {
|
|||
|
|
+- final int i = this.randomItem.incrementAndGet() % half;
|
|||
|
|
+- return tmpList.get(i).getName();
|
|||
|
|
++ Collections.shuffle(tmpList);
|
|||
|
|
++ for (FaultItem faultItem : tmpList) {
|
|||
|
|
++ if (faultItem.reachableFlag) {
|
|||
|
|
++ return faultItem.name;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String toString() {
|
|||
|
|
+ return "LatencyFaultToleranceImpl{" +
|
|||
|
|
+- "faultItemTable=" + faultItemTable +
|
|||
|
|
+- ", whichItemWorst=" + randomItem +
|
|||
|
|
+- '}';
|
|||
|
|
++ "faultItemTable=" + faultItemTable +
|
|||
|
|
++ ", whichItemWorst=" + whichItemWorst +
|
|||
|
|
++ '}';
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setDetectTimeout(final int detectTimeout) {
|
|||
|
|
++ this.detectTimeout = detectTimeout;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- class FaultItem implements Comparable<FaultItem> {
|
|||
|
|
++ public void setDetectInterval(final int detectInterval) {
|
|||
|
|
++ this.detectInterval = detectInterval;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public class FaultItem implements Comparable<FaultItem> {
|
|||
|
|
+ private final String name;
|
|||
|
|
+ private volatile long currentLatency;
|
|||
|
|
+ private volatile long startTimestamp;
|
|||
|
|
++ private volatile long checkStamp;
|
|||
|
|
++ private volatile boolean reachableFlag;
|
|||
|
|
+
|
|||
|
|
+ public FaultItem(final String name) {
|
|||
|
|
+ this.name = name;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void updateNotAvailableDuration(long notAvailableDuration) {
|
|||
|
|
++ if (notAvailableDuration > 0 && System.currentTimeMillis() + notAvailableDuration > this.startTimestamp) {
|
|||
|
|
++ this.startTimestamp = System.currentTimeMillis() + notAvailableDuration;
|
|||
|
|
++ log.info(name + " will be isolated for " + notAvailableDuration + " ms.");
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public int compareTo(final FaultItem other) {
|
|||
|
|
+ if (this.isAvailable() != other.isAvailable()) {
|
|||
|
|
+- if (this.isAvailable())
|
|||
|
|
++ if (this.isAvailable()) {
|
|||
|
|
+ return -1;
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+- if (other.isAvailable())
|
|||
|
|
++ if (other.isAvailable()) {
|
|||
|
|
+ return 1;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- if (this.currentLatency < other.currentLatency)
|
|||
|
|
++ if (this.currentLatency < other.currentLatency) {
|
|||
|
|
+ return -1;
|
|||
|
|
+- else if (this.currentLatency > other.currentLatency) {
|
|||
|
|
++ } else if (this.currentLatency > other.currentLatency) {
|
|||
|
|
+ return 1;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- if (this.startTimestamp < other.startTimestamp)
|
|||
|
|
++ if (this.startTimestamp < other.startTimestamp) {
|
|||
|
|
+ return -1;
|
|||
|
|
+- else if (this.startTimestamp > other.startTimestamp) {
|
|||
|
|
++ } else if (this.startTimestamp > other.startTimestamp) {
|
|||
|
|
+ return 1;
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void setReachable(boolean reachableFlag) {
|
|||
|
|
++ this.reachableFlag = reachableFlag;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setCheckStamp(long checkStamp) {
|
|||
|
|
++ this.checkStamp = checkStamp;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean isAvailable() {
|
|||
|
|
+- return (System.currentTimeMillis() - startTimestamp) >= 0;
|
|||
|
|
++ return reachableFlag && System.currentTimeMillis() >= startTimestamp;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isReachable() {
|
|||
|
|
++ return reachableFlag;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -139,28 +244,32 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean equals(final Object o) {
|
|||
|
|
+- if (this == o)
|
|||
|
|
++ if (this == o) {
|
|||
|
|
+ return true;
|
|||
|
|
+- if (!(o instanceof FaultItem))
|
|||
|
|
++ }
|
|||
|
|
++ if (!(o instanceof FaultItem)) {
|
|||
|
|
+ return false;
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+ final FaultItem faultItem = (FaultItem) o;
|
|||
|
|
+
|
|||
|
|
+- if (getCurrentLatency() != faultItem.getCurrentLatency())
|
|||
|
|
++ if (getCurrentLatency() != faultItem.getCurrentLatency()) {
|
|||
|
|
+ return false;
|
|||
|
|
+- if (getStartTimestamp() != faultItem.getStartTimestamp())
|
|||
|
|
++ }
|
|||
|
|
++ if (getStartTimestamp() != faultItem.getStartTimestamp()) {
|
|||
|
|
+ return false;
|
|||
|
|
++ }
|
|||
|
|
+ return getName() != null ? getName().equals(faultItem.getName()) : faultItem.getName() == null;
|
|||
|
|
+-
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String toString() {
|
|||
|
|
+ return "FaultItem{" +
|
|||
|
|
+- "name='" + name + '\'' +
|
|||
|
|
+- ", currentLatency=" + currentLatency +
|
|||
|
|
+- ", startTimestamp=" + startTimestamp +
|
|||
|
|
+- '}';
|
|||
|
|
++ "name='" + name + '\'' +
|
|||
|
|
++ ", currentLatency=" + currentLatency +
|
|||
|
|
++ ", startTimestamp=" + startTimestamp +
|
|||
|
|
++ ", reachableFlag=" + reachableFlag +
|
|||
|
|
++ '}';
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public String getName() {
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java b/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
+index 1e1953fad..c01490784 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
+@@ -17,25 +17,86 @@
|
|||
|
|
+
|
|||
|
|
+ package org.apache.rocketmq.client.latency;
|
|||
|
|
+
|
|||
|
|
+-import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
|
|||
|
|
++import org.apache.rocketmq.client.impl.producer.TopicPublishInfo.QueueFilter;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+ public class MQFaultStrategy {
|
|||
|
|
+- private final static Logger log = LoggerFactory.getLogger(MQFaultStrategy.class);
|
|||
|
|
+- private final LatencyFaultTolerance<String> latencyFaultTolerance = new LatencyFaultToleranceImpl();
|
|||
|
|
++ private LatencyFaultTolerance<String> latencyFaultTolerance;
|
|||
|
|
++ private boolean sendLatencyFaultEnable;
|
|||
|
|
++ private boolean startDetectorEnable;
|
|||
|
|
++ private long[] latencyMax = {50L, 100L, 550L, 1800L, 3000L, 5000L, 15000L};
|
|||
|
|
++ private long[] notAvailableDuration = {0L, 0L, 2000L, 5000L, 6000L, 10000L, 30000L};
|
|||
|
|
+
|
|||
|
|
+- private boolean sendLatencyFaultEnable = false;
|
|||
|
|
++ public static class BrokerFilter implements QueueFilter {
|
|||
|
|
++ private String lastBrokerName;
|
|||
|
|
++
|
|||
|
|
++ public void setLastBrokerName(String lastBrokerName) {
|
|||
|
|
++ this.lastBrokerName = lastBrokerName;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override public boolean filter(MessageQueue mq) {
|
|||
|
|
++ if (lastBrokerName != null) {
|
|||
|
|
++ return !mq.getBrokerName().equals(lastBrokerName);
|
|||
|
|
++ }
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private ThreadLocal<BrokerFilter> threadBrokerFilter = new ThreadLocal<BrokerFilter>() {
|
|||
|
|
++ @Override protected BrokerFilter initialValue() {
|
|||
|
|
++ return new BrokerFilter();
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ private QueueFilter reachableFilter = new QueueFilter() {
|
|||
|
|
++ @Override public boolean filter(MessageQueue mq) {
|
|||
|
|
++ return latencyFaultTolerance.isReachable(mq.getBrokerName());
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++ private QueueFilter availableFilter = new QueueFilter() {
|
|||
|
|
++ @Override public boolean filter(MessageQueue mq) {
|
|||
|
|
++ return latencyFaultTolerance.isAvailable(mq.getBrokerName());
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ public MQFaultStrategy(ClientConfig cc, Resolver fetcher, ServiceDetector serviceDetector) {
|
|||
|
|
++ this.setStartDetectorEnable(cc.isStartDetectorEnable());
|
|||
|
|
++ this.setSendLatencyFaultEnable(cc.isSendLatencyEnable());
|
|||
|
|
++ this.latencyFaultTolerance = new LatencyFaultToleranceImpl(fetcher, serviceDetector);
|
|||
|
|
++ this.latencyFaultTolerance.setDetectInterval(cc.getDetectInterval());
|
|||
|
|
++ this.latencyFaultTolerance.setDetectTimeout(cc.getDetectTimeout());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // For unit test.
|
|||
|
|
++ public MQFaultStrategy(ClientConfig cc, LatencyFaultTolerance<String> tolerance) {
|
|||
|
|
++ this.setStartDetectorEnable(cc.isStartDetectorEnable());
|
|||
|
|
++ this.setSendLatencyFaultEnable(cc.isSendLatencyEnable());
|
|||
|
|
++ this.latencyFaultTolerance = tolerance;
|
|||
|
|
++ this.latencyFaultTolerance.setDetectInterval(cc.getDetectInterval());
|
|||
|
|
++ this.latencyFaultTolerance.setDetectTimeout(cc.getDetectTimeout());
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+- private long[] latencyMax = {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L};
|
|||
|
|
+- private long[] notAvailableDuration = {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L};
|
|||
|
|
+
|
|||
|
|
+ public long[] getNotAvailableDuration() {
|
|||
|
|
+ return notAvailableDuration;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public QueueFilter getAvailableFilter() {
|
|||
|
|
++ return availableFilter;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public QueueFilter getReachableFilter() {
|
|||
|
|
++ return reachableFilter;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ThreadLocal<BrokerFilter> getThreadBrokerFilter() {
|
|||
|
|
++ return threadBrokerFilter;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public void setNotAvailableDuration(final long[] notAvailableDuration) {
|
|||
|
|
+ this.notAvailableDuration = notAvailableDuration;
|
|||
|
|
+ }
|
|||
|
|
+@@ -56,51 +117,69 @@ public class MQFaultStrategy {
|
|||
|
|
+ this.sendLatencyFaultEnable = sendLatencyFaultEnable;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName) {
|
|||
|
|
++ public boolean isStartDetectorEnable() {
|
|||
|
|
++ return startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setStartDetectorEnable(boolean startDetectorEnable) {
|
|||
|
|
++ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void startDetector() {
|
|||
|
|
++ // user should start the detector
|
|||
|
|
++ // and the thread should not be in running state.
|
|||
|
|
++ if (this.sendLatencyFaultEnable && this.startDetectorEnable) {
|
|||
|
|
++ // start the detector.
|
|||
|
|
++ this.latencyFaultTolerance.startDetector();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void shutdown() {
|
|||
|
|
++ if (this.sendLatencyFaultEnable && this.startDetectorEnable) {
|
|||
|
|
++ this.latencyFaultTolerance.shutdown();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName, final boolean resetIndex) {
|
|||
|
|
++ BrokerFilter brokerFilter = threadBrokerFilter.get();
|
|||
|
|
++ brokerFilter.setLastBrokerName(lastBrokerName);
|
|||
|
|
+ if (this.sendLatencyFaultEnable) {
|
|||
|
|
+- try {
|
|||
|
|
+- int index = tpInfo.getSendWhichQueue().incrementAndGet();
|
|||
|
|
+- for (int i = 0; i < tpInfo.getMessageQueueList().size(); i++) {
|
|||
|
|
+- int pos = index++ % tpInfo.getMessageQueueList().size();
|
|||
|
|
+- MessageQueue mq = tpInfo.getMessageQueueList().get(pos);
|
|||
|
|
+- if (!StringUtils.equals(lastBrokerName, mq.getBrokerName()) && latencyFaultTolerance.isAvailable(mq.getBrokerName())) {
|
|||
|
|
+- return mq;
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- final String notBestBroker = latencyFaultTolerance.pickOneAtLeast();
|
|||
|
|
+- int writeQueueNums = tpInfo.getWriteQueueNumsByBroker(notBestBroker);
|
|||
|
|
+- if (writeQueueNums > 0) {
|
|||
|
|
+- final MessageQueue mq = tpInfo.selectOneMessageQueue();
|
|||
|
|
+- if (notBestBroker != null) {
|
|||
|
|
+- mq.setBrokerName(notBestBroker);
|
|||
|
|
+- mq.setQueueId(tpInfo.getSendWhichQueue().incrementAndGet() % writeQueueNums);
|
|||
|
|
+- }
|
|||
|
|
+- return mq;
|
|||
|
|
+- } else {
|
|||
|
|
+- latencyFaultTolerance.remove(notBestBroker);
|
|||
|
|
+- }
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- log.error("Error occurred when selecting message queue", e);
|
|||
|
|
++ if (resetIndex) {
|
|||
|
|
++ tpInfo.resetIndex();
|
|||
|
|
++ }
|
|||
|
|
++ MessageQueue mq = tpInfo.selectOneMessageQueue(availableFilter, brokerFilter);
|
|||
|
|
++ if (mq != null) {
|
|||
|
|
++ return mq;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ mq = tpInfo.selectOneMessageQueue(reachableFilter, brokerFilter);
|
|||
|
|
++ if (mq != null) {
|
|||
|
|
++ return mq;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return tpInfo.selectOneMessageQueue();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- return tpInfo.selectOneMessageQueue(lastBrokerName);
|
|||
|
|
++ MessageQueue mq = tpInfo.selectOneMessageQueue(brokerFilter);
|
|||
|
|
++ if (mq != null) {
|
|||
|
|
++ return mq;
|
|||
|
|
++ }
|
|||
|
|
++ return tpInfo.selectOneMessageQueue();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation) {
|
|||
|
|
++ public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation,
|
|||
|
|
++ final boolean reachable) {
|
|||
|
|
+ if (this.sendLatencyFaultEnable) {
|
|||
|
|
+- long duration = computeNotAvailableDuration(isolation ? 30000 : currentLatency);
|
|||
|
|
+- this.latencyFaultTolerance.updateFaultItem(brokerName, currentLatency, duration);
|
|||
|
|
++ long duration = computeNotAvailableDuration(isolation ? 10000 : currentLatency);
|
|||
|
|
++ this.latencyFaultTolerance.updateFaultItem(brokerName, currentLatency, duration, reachable);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private long computeNotAvailableDuration(final long currentLatency) {
|
|||
|
|
+ for (int i = latencyMax.length - 1; i >= 0; i--) {
|
|||
|
|
+- if (currentLatency >= latencyMax[i])
|
|||
|
|
++ if (currentLatency >= latencyMax[i]) {
|
|||
|
|
+ return this.notAvailableDuration[i];
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return 0;
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetRemoteClientConfigBody.java b/client/src/main/java/org/apache/rocketmq/client/latency/Resolver.java
|
|||
|
|
+similarity index 65%
|
|||
|
|
+rename from remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetRemoteClientConfigBody.java
|
|||
|
|
+rename to client/src/main/java/org/apache/rocketmq/client/latency/Resolver.java
|
|||
|
|
+index 6aa547047..1c29ba334 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/GetRemoteClientConfigBody.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/Resolver.java
|
|||
|
|
+@@ -14,20 +14,9 @@
|
|||
|
|
+ * See the License for the specific language governing permissions and
|
|||
|
|
+ * limitations under the License.
|
|||
|
|
+ */
|
|||
|
|
+-package org.apache.rocketmq.remoting.protocol.body;
|
|||
|
|
++package org.apache.rocketmq.client.latency;
|
|||
|
|
+
|
|||
|
|
+-import java.util.ArrayList;
|
|||
|
|
+-import java.util.List;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.RemotingSerializable;
|
|||
|
|
++public interface Resolver {
|
|||
|
|
+
|
|||
|
|
+-public class GetRemoteClientConfigBody extends RemotingSerializable {
|
|||
|
|
+- private List<String> keys = new ArrayList<>();
|
|||
|
|
+-
|
|||
|
|
+- public List<String> getKeys() {
|
|||
|
|
+- return keys;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public void setKeys(List<String> keys) {
|
|||
|
|
+- this.keys = keys;
|
|||
|
|
+- }
|
|||
|
|
++ String resolve(String name);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/ServiceDetector.java b/client/src/main/java/org/apache/rocketmq/client/latency/ServiceDetector.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..c6ffbad1c
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/ServiceDetector.java
|
|||
|
|
+@@ -0,0 +1,30 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.client.latency;
|
|||
|
|
++
|
|||
|
|
++/**
|
|||
|
|
++ * Detect whether the remote service state is normal.
|
|||
|
|
++ */
|
|||
|
|
++public interface ServiceDetector {
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Check if the remote service is normal.
|
|||
|
|
++ * @param endpoint Service endpoint to check against
|
|||
|
|
++ * @return true if the service is back to normal; false otherwise.
|
|||
|
|
++ */
|
|||
|
|
++ boolean detect(String endpoint, long timeoutMillis);
|
|||
|
|
++}
|
|||
|
|
+diff --git a/client/src/test/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImplTest.java b/client/src/test/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImplTest.java
|
|||
|
|
+index 86690e40b..42ccdae5a 100644
|
|||
|
|
+--- a/client/src/test/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImplTest.java
|
|||
|
|
++++ b/client/src/test/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImplTest.java
|
|||
|
|
+@@ -16,11 +16,14 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.client.latency;
|
|||
|
|
+
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
++import org.awaitility.core.ThrowingRunnable;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
++
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
++import static org.awaitility.Awaitility.await;
|
|||
|
|
+
|
|||
|
|
+ public class LatencyFaultToleranceImplTest {
|
|||
|
|
+ private LatencyFaultTolerance<String> latencyFaultTolerance;
|
|||
|
|
+@@ -29,28 +32,31 @@ public class LatencyFaultToleranceImplTest {
|
|||
|
|
+
|
|||
|
|
+ @Before
|
|||
|
|
+ public void init() {
|
|||
|
|
+- latencyFaultTolerance = new LatencyFaultToleranceImpl();
|
|||
|
|
++ latencyFaultTolerance = new LatencyFaultToleranceImpl(null, null);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testUpdateFaultItem() throws Exception {
|
|||
|
|
+- latencyFaultTolerance.updateFaultItem(brokerName, 3000, 3000);
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(brokerName, 3000, 3000, true);
|
|||
|
|
+ assertThat(latencyFaultTolerance.isAvailable(brokerName)).isFalse();
|
|||
|
|
+ assertThat(latencyFaultTolerance.isAvailable(anotherBrokerName)).isTrue();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testIsAvailable() throws Exception {
|
|||
|
|
+- latencyFaultTolerance.updateFaultItem(brokerName, 3000, 50);
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(brokerName, 3000, 50, true);
|
|||
|
|
+ assertThat(latencyFaultTolerance.isAvailable(brokerName)).isFalse();
|
|||
|
|
+
|
|||
|
|
+- TimeUnit.MILLISECONDS.sleep(70);
|
|||
|
|
+- assertThat(latencyFaultTolerance.isAvailable(brokerName)).isTrue();
|
|||
|
|
++ await().atMost(500, TimeUnit.MILLISECONDS).untilAsserted(new ThrowingRunnable() {
|
|||
|
|
++ @Override public void run() throws Throwable {
|
|||
|
|
++ assertThat(latencyFaultTolerance.isAvailable(brokerName)).isTrue();
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testRemove() throws Exception {
|
|||
|
|
+- latencyFaultTolerance.updateFaultItem(brokerName, 3000, 3000);
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(brokerName, 3000, 3000, true);
|
|||
|
|
+ assertThat(latencyFaultTolerance.isAvailable(brokerName)).isFalse();
|
|||
|
|
+ latencyFaultTolerance.remove(brokerName);
|
|||
|
|
+ assertThat(latencyFaultTolerance.isAvailable(brokerName)).isTrue();
|
|||
|
|
+@@ -58,10 +64,20 @@ public class LatencyFaultToleranceImplTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPickOneAtLeast() throws Exception {
|
|||
|
|
+- latencyFaultTolerance.updateFaultItem(brokerName, 1000, 3000);
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(brokerName, 1000, 3000, true);
|
|||
|
|
+ assertThat(latencyFaultTolerance.pickOneAtLeast()).isEqualTo(brokerName);
|
|||
|
|
+
|
|||
|
|
+- latencyFaultTolerance.updateFaultItem(anotherBrokerName, 1001, 3000);
|
|||
|
|
+- assertThat(latencyFaultTolerance.pickOneAtLeast()).isEqualTo(brokerName);
|
|||
|
|
++ // Bad case, since pickOneAtLeast's behavior becomes random
|
|||
|
|
++ // latencyFaultTolerance.updateFaultItem(anotherBrokerName, 1001, 3000, "127.0.0.1:12011", true);
|
|||
|
|
++ // assertThat(latencyFaultTolerance.pickOneAtLeast()).isEqualTo(brokerName);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testIsReachable() throws Exception {
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(brokerName, 1000, 3000, true);
|
|||
|
|
++ assertThat(latencyFaultTolerance.isReachable(brokerName)).isEqualTo(true);
|
|||
|
|
++
|
|||
|
|
++ latencyFaultTolerance.updateFaultItem(anotherBrokerName, 1001, 3000, false);
|
|||
|
|
++ assertThat(latencyFaultTolerance.isReachable(anotherBrokerName)).isEqualTo(false);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
|
|||
|
|
+index fada0efd7..485b95c42 100644
|
|||
|
|
+--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
|
|||
|
|
++++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/DefaultRequestProcessor.java
|
|||
|
|
+@@ -41,7 +41,6 @@ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.BrokerMemberGroup;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.GetBrokerMemberGroupResponseBody;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.body.GetRemoteClientConfigBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.RegisterBrokerBody;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.TopicList;
|
|||
|
|
+@@ -132,8 +131,6 @@ public class DefaultRequestProcessor implements NettyRequestProcessor {
|
|||
|
|
+ return this.updateConfig(ctx, request);
|
|||
|
|
+ case RequestCode.GET_NAMESRV_CONFIG:
|
|||
|
|
+ return this.getConfig(ctx, request);
|
|||
|
|
+- case RequestCode.GET_CLIENT_CONFIG:
|
|||
|
|
+- return this.getClientConfigs(ctx, request);
|
|||
|
|
+ default:
|
|||
|
|
+ String error = " request type " + request.getCode() + " not supported";
|
|||
|
|
+ return RemotingCommand.createResponseCommand(RemotingSysResponseCode.REQUEST_CODE_NOT_SUPPORTED, error);
|
|||
|
|
+@@ -661,25 +658,4 @@ public class DefaultRequestProcessor implements NettyRequestProcessor {
|
|||
|
|
+ return response;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private RemotingCommand getClientConfigs(ChannelHandlerContext ctx, RemotingCommand request) {
|
|||
|
|
+- final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- final GetRemoteClientConfigBody body = GetRemoteClientConfigBody.decode(request.getBody(), GetRemoteClientConfigBody.class);
|
|||
|
|
+-
|
|||
|
|
+- String content = this.namesrvController.getConfiguration().getClientConfigsFormatString(body.getKeys());
|
|||
|
|
+- if (StringUtils.isNotBlank(content)) {
|
|||
|
|
+- try {
|
|||
|
|
+- response.setBody(content.getBytes(MixAll.DEFAULT_CHARSET));
|
|||
|
|
+- } catch (UnsupportedEncodingException e) {
|
|||
|
|
+- log.error("getConfig error, ", e);
|
|||
|
|
+- response.setCode(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+- response.setRemark("UnsupportedEncodingException " + e);
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- response.setRemark(null);
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index 2994893d7..b2478fec3 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -232,6 +232,12 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ private String remotingAccessAddr = "";
|
|||
|
|
+ private int remotingListenPort = 8080;
|
|||
|
|
+
|
|||
|
|
++ // related to proxy's send strategy in cluster mode.
|
|||
|
|
++ private boolean sendLatencyEnable = false;
|
|||
|
|
++ private boolean startDetectorEnable = false;
|
|||
|
|
++ private int detectTimeout = 200;
|
|||
|
|
++ private int detectInterval = 2 * 1000;
|
|||
|
|
++
|
|||
|
|
+ private int remotingHeartbeatThreadPoolNums = 2 * PROCESSOR_NUMBER;
|
|||
|
|
+ private int remotingTopicRouteThreadPoolNums = 2 * PROCESSOR_NUMBER;
|
|||
|
|
+ private int remotingSendMessageThreadPoolNums = 4 * PROCESSOR_NUMBER;
|
|||
|
|
+@@ -1409,6 +1415,46 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ this.remotingWaitTimeMillsInDefaultQueue = remotingWaitTimeMillsInDefaultQueue;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isSendLatencyEnable() {
|
|||
|
|
++ return sendLatencyEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isStartDetectorEnable() {
|
|||
|
|
++ return startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setStartDetectorEnable(boolean startDetectorEnable) {
|
|||
|
|
++ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setSendLatencyEnable(boolean sendLatencyEnable) {
|
|||
|
|
++ this.sendLatencyEnable = sendLatencyEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean getStartDetectorEnable() {
|
|||
|
|
++ return this.startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean getSendLatencyEnable() {
|
|||
|
|
++ return this.sendLatencyEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getDetectTimeout() {
|
|||
|
|
++ return detectTimeout;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setDetectTimeout(int detectTimeout) {
|
|||
|
|
++ this.detectTimeout = detectTimeout;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getDetectInterval() {
|
|||
|
|
++ return detectInterval;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setDetectInterval(int detectInterval) {
|
|||
|
|
++ this.detectInterval = detectInterval;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean isEnableBatchAck() {
|
|||
|
|
+ return enableBatchAck;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java
|
|||
|
|
+index 6146c80cd..f670df205 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivity.java
|
|||
|
|
+@@ -382,7 +382,7 @@ public class SendMessageActivity extends AbstractMessingActivity {
|
|||
|
|
+ int bucket = Hashing.consistentHash(shardingKey.hashCode(), writeQueues.size());
|
|||
|
|
+ targetMessageQueue = writeQueues.get(bucket);
|
|||
|
|
+ } else {
|
|||
|
|
+- targetMessageQueue = messageQueueView.getWriteSelector().selectOne(false);
|
|||
|
|
++ targetMessageQueue = messageQueueView.getWriteSelector().selectOneByPipeline(false);
|
|||
|
|
+ }
|
|||
|
|
+ return targetMessageQueue;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java
|
|||
|
|
+index 0d0c62168..a80f6df0b 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/ProducerProcessor.java
|
|||
|
|
+@@ -19,6 +19,7 @@ package org.apache.rocketmq.proxy.processor;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
++
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendResult;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendStatus;
|
|||
|
|
+@@ -66,6 +67,8 @@ public class ProducerProcessor extends AbstractProcessor {
|
|||
|
|
+ public CompletableFuture<List<SendResult>> sendMessage(ProxyContext ctx, QueueSelector queueSelector,
|
|||
|
|
+ String producerGroup, int sysFlag, List<Message> messageList, long timeoutMillis) {
|
|||
|
|
+ CompletableFuture<List<SendResult>> future = new CompletableFuture<>();
|
|||
|
|
++ long beginTimestampFirst = System.currentTimeMillis();
|
|||
|
|
++ AddressableMessageQueue messageQueue = null;
|
|||
|
|
+ try {
|
|||
|
|
+ Message message = messageList.get(0);
|
|||
|
|
+ String topic = message.getTopic();
|
|||
|
|
+@@ -79,7 +82,7 @@ public class ProducerProcessor extends AbstractProcessor {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+- AddressableMessageQueue messageQueue = queueSelector.select(ctx,
|
|||
|
|
++ messageQueue = queueSelector.select(ctx,
|
|||
|
|
+ this.serviceManager.getTopicRouteService().getCurrentMessageQueueView(ctx, topic));
|
|||
|
|
+ if (messageQueue == null) {
|
|||
|
|
+ throw new ProxyException(ProxyExceptionCode.FORBIDDEN, "no writable queue");
|
|||
|
|
+@@ -90,6 +93,7 @@ public class ProducerProcessor extends AbstractProcessor {
|
|||
|
|
+ }
|
|||
|
|
+ SendMessageRequestHeader requestHeader = buildSendMessageRequestHeader(messageList, producerGroup, sysFlag, messageQueue.getQueueId());
|
|||
|
|
+
|
|||
|
|
++ AddressableMessageQueue finalMessageQueue = messageQueue;
|
|||
|
|
+ future = this.serviceManager.getMessageService().sendMessage(
|
|||
|
|
+ ctx,
|
|||
|
|
+ messageQueue,
|
|||
|
|
+@@ -102,11 +106,19 @@ public class ProducerProcessor extends AbstractProcessor {
|
|||
|
|
+ if (SendStatus.SEND_OK.equals(sendResult.getSendStatus()) &&
|
|||
|
|
+ tranType == MessageSysFlag.TRANSACTION_PREPARED_TYPE &&
|
|||
|
|
+ StringUtils.isNotBlank(sendResult.getTransactionId())) {
|
|||
|
|
+- fillTransactionData(ctx, producerGroup, messageQueue, sendResult, messageList);
|
|||
|
|
++ fillTransactionData(ctx, producerGroup, finalMessageQueue, sendResult, messageList);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return sendResultList;
|
|||
|
|
+- }, this.executor);
|
|||
|
|
++ }, this.executor)
|
|||
|
|
++ .whenComplete((result, exception) -> {
|
|||
|
|
++ long endTimestamp = System.currentTimeMillis();
|
|||
|
|
++ if (exception != null) {
|
|||
|
|
++ this.serviceManager.getTopicRouteService().updateFaultItem(finalMessageQueue.getBrokerName(), endTimestamp - beginTimestampFirst, true, false);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.serviceManager.getTopicRouteService().updateFaultItem(finalMessageQueue.getBrokerName(),endTimestamp - beginTimestampFirst, false, true);
|
|||
|
|
++ }
|
|||
|
|
++ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ future.completeExceptionally(t);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/LocalTopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/LocalTopicRouteService.java
|
|||
|
|
+index d67b68f38..aced15cee 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/LocalTopicRouteService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/LocalTopicRouteService.java
|
|||
|
|
+@@ -54,7 +54,7 @@ public class LocalTopicRouteService extends TopicRouteService {
|
|||
|
|
+ @Override
|
|||
|
|
+ public MessageQueueView getCurrentMessageQueueView(ProxyContext ctx, String topic) throws Exception {
|
|||
|
|
+ TopicConfig topicConfig = this.brokerController.getTopicConfigManager().getTopicConfigTable().get(topic);
|
|||
|
|
+- return new MessageQueueView(topic, toTopicRouteData(topicConfig));
|
|||
|
|
++ return new MessageQueueView(topic, toTopicRouteData(topicConfig), null);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelector.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelector.java
|
|||
|
|
+index 85cd18d45..f25fb907e 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelector.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelector.java
|
|||
|
|
+@@ -17,6 +17,7 @@
|
|||
|
|
+ package org.apache.rocketmq.proxy.service.route;
|
|||
|
|
+
|
|||
|
|
+ import com.google.common.base.MoreObjects;
|
|||
|
|
++import com.google.common.base.Preconditions;
|
|||
|
|
+ import com.google.common.math.IntMath;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+@@ -30,6 +31,8 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.stream.Collectors;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.client.impl.producer.TopicPublishInfo;
|
|||
|
|
++import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.PermName;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.QueueData;
|
|||
|
|
+@@ -44,8 +47,9 @@ public class MessageQueueSelector {
|
|||
|
|
+ private final Map<String, AddressableMessageQueue> brokerNameQueueMap = new ConcurrentHashMap<>();
|
|||
|
|
+ private final AtomicInteger queueIndex;
|
|||
|
|
+ private final AtomicInteger brokerIndex;
|
|||
|
|
++ private MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+
|
|||
|
|
+- public MessageQueueSelector(TopicRouteWrapper topicRouteWrapper, boolean read) {
|
|||
|
|
++ public MessageQueueSelector(TopicRouteWrapper topicRouteWrapper, MQFaultStrategy mqFaultStrategy, boolean read) {
|
|||
|
|
+ if (read) {
|
|||
|
|
+ this.queues.addAll(buildRead(topicRouteWrapper));
|
|||
|
|
+ } else {
|
|||
|
|
+@@ -55,6 +59,7 @@ public class MessageQueueSelector {
|
|||
|
|
+ Random random = new Random();
|
|||
|
|
+ this.queueIndex = new AtomicInteger(random.nextInt());
|
|||
|
|
+ this.brokerIndex = new AtomicInteger(random.nextInt());
|
|||
|
|
++ this.mqFaultStrategy = mqFaultStrategy;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private static List<AddressableMessageQueue> buildRead(TopicRouteWrapper topicRoute) {
|
|||
|
|
+@@ -154,6 +159,86 @@ public class MessageQueueSelector {
|
|||
|
|
+ return selectOneByIndex(nextIndex, onlyBroker);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public AddressableMessageQueue selectOneByPipeline(boolean onlyBroker) {
|
|||
|
|
++ if (mqFaultStrategy != null && mqFaultStrategy.isSendLatencyFaultEnable()) {
|
|||
|
|
++ List<MessageQueue> messageQueueList = null;
|
|||
|
|
++ MessageQueue messageQueue = null;
|
|||
|
|
++ if (onlyBroker) {
|
|||
|
|
++ messageQueueList = transferAddressableQueues(brokerActingQueues);
|
|||
|
|
++ } else {
|
|||
|
|
++ messageQueueList = transferAddressableQueues(queues);
|
|||
|
|
++ }
|
|||
|
|
++ AddressableMessageQueue addressableMessageQueue = null;
|
|||
|
|
++
|
|||
|
|
++ // use both available filter.
|
|||
|
|
++ messageQueue = selectOneMessageQueue(messageQueueList, onlyBroker ? brokerIndex : queueIndex,
|
|||
|
|
++ mqFaultStrategy.getAvailableFilter(), mqFaultStrategy.getReachableFilter());
|
|||
|
|
++ addressableMessageQueue = transferQueue2Addressable(messageQueue);
|
|||
|
|
++ if (addressableMessageQueue != null) {
|
|||
|
|
++ return addressableMessageQueue;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // use available filter.
|
|||
|
|
++ messageQueue = selectOneMessageQueue(messageQueueList, onlyBroker ? brokerIndex : queueIndex,
|
|||
|
|
++ mqFaultStrategy.getAvailableFilter());
|
|||
|
|
++ addressableMessageQueue = transferQueue2Addressable(messageQueue);
|
|||
|
|
++ if (addressableMessageQueue != null) {
|
|||
|
|
++ return addressableMessageQueue;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // no available filter, then use reachable filter.
|
|||
|
|
++ messageQueue = selectOneMessageQueue(messageQueueList, onlyBroker ? brokerIndex : queueIndex,
|
|||
|
|
++ mqFaultStrategy.getReachableFilter());
|
|||
|
|
++ addressableMessageQueue = transferQueue2Addressable(messageQueue);
|
|||
|
|
++ if (addressableMessageQueue != null) {
|
|||
|
|
++ return addressableMessageQueue;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ // SendLatency is not enabled, or no queue is selected, then select by index.
|
|||
|
|
++ return selectOne(onlyBroker);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private MessageQueue selectOneMessageQueue(List<MessageQueue> messageQueueList, AtomicInteger sendQueue, TopicPublishInfo.QueueFilter...filter) {
|
|||
|
|
++ if (messageQueueList == null || messageQueueList.isEmpty()) {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++ if (filter != null && filter.length != 0) {
|
|||
|
|
++ for (int i = 0; i < messageQueueList.size(); i++) {
|
|||
|
|
++ int index = Math.abs(sendQueue.incrementAndGet() % messageQueueList.size());
|
|||
|
|
++ MessageQueue mq = messageQueueList.get(index);
|
|||
|
|
++ boolean filterResult = true;
|
|||
|
|
++ for (TopicPublishInfo.QueueFilter f: filter) {
|
|||
|
|
++ Preconditions.checkNotNull(f);
|
|||
|
|
++ filterResult &= f.filter(mq);
|
|||
|
|
++ }
|
|||
|
|
++ if (filterResult) {
|
|||
|
|
++ return mq;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public List<MessageQueue> transferAddressableQueues(List<AddressableMessageQueue> addressableMessageQueueList) {
|
|||
|
|
++ if (addressableMessageQueueList == null) {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ return addressableMessageQueueList.stream()
|
|||
|
|
++ .map(AddressableMessageQueue::getMessageQueue)
|
|||
|
|
++ .collect(Collectors.toList());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private AddressableMessageQueue transferQueue2Addressable(MessageQueue messageQueue) {
|
|||
|
|
++ for (AddressableMessageQueue amq: queues) {
|
|||
|
|
++ if (amq.getMessageQueue().equals(messageQueue)) {
|
|||
|
|
++ return amq;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public AddressableMessageQueue selectNextOne(AddressableMessageQueue last) {
|
|||
|
|
+ boolean onlyBroker = last.getQueueId() < 0;
|
|||
|
|
+ AddressableMessageQueue newOne = last;
|
|||
|
|
+@@ -190,6 +275,14 @@ public class MessageQueueSelector {
|
|||
|
|
+ return brokerActingQueues;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public MQFaultStrategy getMQFaultStrategy() {
|
|||
|
|
++ return mqFaultStrategy;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setMQFaultStrategy(MQFaultStrategy mqFaultStrategy) {
|
|||
|
|
++ this.mqFaultStrategy = mqFaultStrategy;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean equals(Object o) {
|
|||
|
|
+ if (this == o) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
+index fe5387cfd..8b3c2f7c8 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
+@@ -17,20 +17,22 @@
|
|||
|
|
+ package org.apache.rocketmq.proxy.service.route;
|
|||
|
|
+
|
|||
|
|
+ import com.google.common.base.MoreObjects;
|
|||
|
|
++import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.TopicRouteData;
|
|||
|
|
+
|
|||
|
|
+ public class MessageQueueView {
|
|||
|
|
+- public static final MessageQueueView WRAPPED_EMPTY_QUEUE = new MessageQueueView("", new TopicRouteData());
|
|||
|
|
++ public static final MessageQueueView WRAPPED_EMPTY_QUEUE = new MessageQueueView("", new TopicRouteData(), null);
|
|||
|
|
+
|
|||
|
|
+ private final MessageQueueSelector readSelector;
|
|||
|
|
+ private final MessageQueueSelector writeSelector;
|
|||
|
|
+ private final TopicRouteWrapper topicRouteWrapper;
|
|||
|
|
++ private MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+
|
|||
|
|
+- public MessageQueueView(String topic, TopicRouteData topicRouteData) {
|
|||
|
|
++ public MessageQueueView(String topic, TopicRouteData topicRouteData, MQFaultStrategy mqFaultStrategy) {
|
|||
|
|
+ this.topicRouteWrapper = new TopicRouteWrapper(topicRouteData, topic);
|
|||
|
|
+
|
|||
|
|
+- this.readSelector = new MessageQueueSelector(topicRouteWrapper, true);
|
|||
|
|
+- this.writeSelector = new MessageQueueSelector(topicRouteWrapper, false);
|
|||
|
|
++ this.readSelector = new MessageQueueSelector(topicRouteWrapper, mqFaultStrategy, true);
|
|||
|
|
++ this.writeSelector = new MessageQueueSelector(topicRouteWrapper, mqFaultStrategy, false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public TopicRouteData getTopicRouteData() {
|
|||
|
|
+@@ -65,4 +67,12 @@ public class MessageQueueView {
|
|||
|
|
+ .add("topicRouteWrapper", topicRouteWrapper)
|
|||
|
|
+ .toString();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public MQFaultStrategy getMQFaultStrategy() {
|
|||
|
|
++ return mqFaultStrategy;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setMQFaultStrategy(MQFaultStrategy mqFaultStrategy) {
|
|||
|
|
++ this.mqFaultStrategy = mqFaultStrategy;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+index 84348adc3..74769a423 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+@@ -25,7 +25,13 @@ import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
++
|
|||
|
|
++import com.google.common.base.Optional;
|
|||
|
|
++import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.client.exception.MQClientException;
|
|||
|
|
++import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
++import org.apache.rocketmq.client.latency.Resolver;
|
|||
|
|
++import org.apache.rocketmq.client.latency.ServiceDetector;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+@@ -39,6 +45,7 @@ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.TopicRouteData;
|
|||
|
|
+ import org.checkerframework.checker.nullness.qual.NonNull;
|
|||
|
|
+ import org.checkerframework.checker.nullness.qual.Nullable;
|
|||
|
|
+@@ -47,6 +54,7 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ private final MQClientAPIFactory mqClientAPIFactory;
|
|||
|
|
++ private MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+
|
|||
|
|
+ protected final LoadingCache<String /* topicName */, MessageQueueView> topicCache;
|
|||
|
|
+ protected final ScheduledExecutorService scheduledExecutorService;
|
|||
|
|
+@@ -97,15 +105,83 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+-
|
|||
|
|
++ ServiceDetector serviceDetector = new ServiceDetector() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean detect(String endpoint, long timeoutMillis) {
|
|||
|
|
++ Optional<String> candidateTopic = pickTopic();
|
|||
|
|
++ if (!candidateTopic.isPresent()) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ try {
|
|||
|
|
++ GetMaxOffsetRequestHeader requestHeader = new GetMaxOffsetRequestHeader();
|
|||
|
|
++ requestHeader.setTopic(candidateTopic.get());
|
|||
|
|
++ requestHeader.setQueueId(0);
|
|||
|
|
++ Long maxOffset = mqClientAPIFactory.getClient().getMaxOffset(endpoint, requestHeader, timeoutMillis).get();
|
|||
|
|
++ return true;
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++ mqFaultStrategy = new MQFaultStrategy(extractClientConfigFromProxyConfig(config), new Resolver() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public String resolve(String name) {
|
|||
|
|
++ try {
|
|||
|
|
++ String brokerAddr = getBrokerAddr(null, name);
|
|||
|
|
++ return brokerAddr;
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }, serviceDetector);
|
|||
|
|
+ this.init();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ // pickup one topic in the topic cache
|
|||
|
|
++ private Optional<String> pickTopic() {
|
|||
|
|
++ if (topicCache.asMap().isEmpty()) {
|
|||
|
|
++ return Optional.absent();
|
|||
|
|
++ }
|
|||
|
|
++ return Optional.of(topicCache.asMap().keySet().iterator().next());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ protected void init() {
|
|||
|
|
+ this.appendShutdown(this.scheduledExecutorService::shutdown);
|
|||
|
|
+ this.appendStartAndShutdown(this.mqClientAPIFactory);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public void shutdown() throws Exception {
|
|||
|
|
++ if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
++ mqFaultStrategy.shutdown();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void start() throws Exception {
|
|||
|
|
++ if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
++ this.mqFaultStrategy.startDetector();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ClientConfig extractClientConfigFromProxyConfig(ProxyConfig proxyConfig) {
|
|||
|
|
++ ClientConfig tempClientConfig = new ClientConfig();
|
|||
|
|
++ tempClientConfig.setSendLatencyEnable(proxyConfig.getSendLatencyEnable());
|
|||
|
|
++ tempClientConfig.setStartDetectorEnable(proxyConfig.getStartDetectorEnable());
|
|||
|
|
++ tempClientConfig.setDetectTimeout(proxyConfig.getDetectTimeout());
|
|||
|
|
++ tempClientConfig.setDetectInterval(proxyConfig.getDetectInterval());
|
|||
|
|
++ return tempClientConfig;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation,
|
|||
|
|
++ boolean reachable) {
|
|||
|
|
++ this.mqFaultStrategy.updateFaultItem(brokerName, currentLatency, isolation, reachable);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public MQFaultStrategy getMqFaultStrategy() {
|
|||
|
|
++ return this.mqFaultStrategy;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public MessageQueueView getAllMessageQueueView(ProxyContext ctx, String topicName) throws Exception {
|
|||
|
|
+ return getCacheMessageQueueWrapper(this.topicCache, topicName);
|
|||
|
|
+ }
|
|||
|
|
+@@ -136,7 +212,7 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ protected MessageQueueView buildMessageQueueView(String topic, TopicRouteData topicRouteData) {
|
|||
|
|
+ if (isTopicRouteValid(topicRouteData)) {
|
|||
|
|
+- MessageQueueView tmp = new MessageQueueView(topic, topicRouteData);
|
|||
|
|
++ MessageQueueView tmp = new MessageQueueView(topic, topicRouteData, TopicRouteService.this.getMqFaultStrategy());
|
|||
|
|
+ log.debug("load topic route from namesrv. topic: {}, queue: {}", topic, tmp);
|
|||
|
|
+ return tmp;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivityTest.java
|
|||
|
|
+index 7fd9a9ffd..77ae5e4d1 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivityTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/consumer/ReceiveMessageActivityTest.java
|
|||
|
|
+@@ -93,7 +93,6 @@ public class ReceiveMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ pollTimeCaptor.capture(), anyInt(), any(), anyBoolean(), any(), isNull(), anyLong()))
|
|||
|
|
+ .thenReturn(CompletableFuture.completedFuture(new PopResult(PopStatus.NO_NEW_MSG, Collections.emptyList())));
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ ProxyContext context = createContext();
|
|||
|
|
+ context.setRemainingMs(1L);
|
|||
|
|
+ this.receiveMessageActivity.receiveMessage(
|
|||
|
|
+@@ -274,7 +273,7 @@ public class ReceiveMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+- public void testReceiveMessageQueueSelector() {
|
|||
|
|
++ public void testReceiveMessageQueueSelector() throws Exception {
|
|||
|
|
+ TopicRouteData topicRouteData = new TopicRouteData();
|
|||
|
|
+ List<QueueData> queueDatas = new ArrayList<>();
|
|||
|
|
+ for (int i = 0; i < 2; i++) {
|
|||
|
|
+@@ -298,7 +297,7 @@ public class ReceiveMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ }
|
|||
|
|
+ topicRouteData.setBrokerDatas(brokerDatas);
|
|||
|
|
+
|
|||
|
|
+- MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData, null);
|
|||
|
|
+ ReceiveMessageActivity.ReceiveMessageQueueSelector selector = new ReceiveMessageActivity.ReceiveMessageQueueSelector("");
|
|||
|
|
+
|
|||
|
|
+ AddressableMessageQueue firstSelect = selector.select(ProxyContext.create(), messageQueueView);
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java
|
|||
|
|
+index 588423bb9..4882a5ed8 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/grpc/v2/producer/SendMessageActivityTest.java
|
|||
|
|
+@@ -35,6 +35,8 @@ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
++import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendResult;
|
|||
|
|
+ import org.apache.rocketmq.client.producer.SendStatus;
|
|||
|
|
+ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+@@ -49,6 +51,7 @@ import org.apache.rocketmq.proxy.grpc.v2.BaseActivityTest;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.MessageQueueView;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.route.TopicRouteService;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.BrokerData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.QueueData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.TopicRouteData;
|
|||
|
|
+@@ -62,15 +65,19 @@ import static org.junit.Assert.assertThrows;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyInt;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
++import static org.mockito.Mockito.mock;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+
|
|||
|
|
+ protected static final String BROKER_NAME = "broker";
|
|||
|
|
++ protected static final String BROKER_NAME2 = "broker2";
|
|||
|
|
+ protected static final String CLUSTER_NAME = "cluster";
|
|||
|
|
+ protected static final String BROKER_ADDR = "127.0.0.1:10911";
|
|||
|
|
++ protected static final String BROKER_ADDR2 = "127.0.0.1:10912";
|
|||
|
|
+ private static final String TOPIC = "topic";
|
|||
|
|
+ private static final String CONSUMER_GROUP = "consumerGroup";
|
|||
|
|
++ MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+
|
|||
|
|
+ private SendMessageActivity sendMessageActivity;
|
|||
|
|
+
|
|||
|
|
+@@ -262,7 +269,7 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+- public void testSendOrderMessageQueueSelector() {
|
|||
|
|
++ public void testSendOrderMessageQueueSelector() throws Exception {
|
|||
|
|
+ TopicRouteData topicRouteData = new TopicRouteData();
|
|||
|
|
+ QueueData queueData = new QueueData();
|
|||
|
|
+ BrokerData brokerData = new BrokerData();
|
|||
|
|
+@@ -277,7 +284,7 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ brokerData.setBrokerAddrs(brokerAddrs);
|
|||
|
|
+ topicRouteData.setBrokerDatas(Lists.newArrayList(brokerData));
|
|||
|
|
+
|
|||
|
|
+- MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData, null);
|
|||
|
|
+ SendMessageActivity.SendMessageQueueSelector selector1 = new SendMessageActivity.SendMessageQueueSelector(
|
|||
|
|
+ SendMessageRequest.newBuilder()
|
|||
|
|
+ .addMessages(Message.newBuilder()
|
|||
|
|
+@@ -288,6 +295,12 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ .build()
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
++ TopicRouteService topicRouteService = mock(TopicRouteService.class);
|
|||
|
|
++ MQFaultStrategy mqFaultStrategy = mock(MQFaultStrategy.class);
|
|||
|
|
++ when(topicRouteService.getAllMessageQueueView(any(), any())).thenReturn(messageQueueView);
|
|||
|
|
++ when(topicRouteService.getMqFaultStrategy()).thenReturn(mqFaultStrategy);
|
|||
|
|
++ when(mqFaultStrategy.isSendLatencyFaultEnable()).thenReturn(false);
|
|||
|
|
++
|
|||
|
|
+ SendMessageActivity.SendMessageQueueSelector selector2 = new SendMessageActivity.SendMessageQueueSelector(
|
|||
|
|
+ SendMessageRequest.newBuilder()
|
|||
|
|
+ .addMessages(Message.newBuilder()
|
|||
|
|
+@@ -328,12 +341,17 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ brokerData.setBrokerAddrs(brokerAddrs);
|
|||
|
|
+ topicRouteData.setBrokerDatas(Lists.newArrayList(brokerData));
|
|||
|
|
+
|
|||
|
|
+- MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData);
|
|||
|
|
++
|
|||
|
|
+ SendMessageActivity.SendMessageQueueSelector selector = new SendMessageActivity.SendMessageQueueSelector(
|
|||
|
|
+ SendMessageRequest.newBuilder()
|
|||
|
|
+ .addMessages(Message.newBuilder().build())
|
|||
|
|
+ .build()
|
|||
|
|
+ );
|
|||
|
|
++ TopicRouteService topicRouteService = mock(TopicRouteService.class);
|
|||
|
|
++ MQFaultStrategy mqFaultStrategy = mock(MQFaultStrategy.class);
|
|||
|
|
++ when(topicRouteService.getMqFaultStrategy()).thenReturn(mqFaultStrategy);
|
|||
|
|
++ when(mqFaultStrategy.isSendLatencyFaultEnable()).thenReturn(false);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData, topicRouteService.getMqFaultStrategy());
|
|||
|
|
+
|
|||
|
|
+ AddressableMessageQueue firstSelect = selector.select(ProxyContext.create(), messageQueueView);
|
|||
|
|
+ AddressableMessageQueue secondSelect = selector.select(ProxyContext.create(), messageQueueView);
|
|||
|
|
+@@ -343,6 +361,45 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ assertNotEquals(firstSelect, secondSelect);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testSendNormalMessageQueueSelectorPipeLine() throws Exception {
|
|||
|
|
++ TopicRouteData topicRouteData = new TopicRouteData();
|
|||
|
|
++ int queueNums = 2;
|
|||
|
|
++
|
|||
|
|
++ QueueData queueData = createQueueData(BROKER_NAME, queueNums);
|
|||
|
|
++ QueueData queueData2 = createQueueData(BROKER_NAME2, queueNums);
|
|||
|
|
++ topicRouteData.setQueueDatas(Lists.newArrayList(queueData,queueData2));
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ BrokerData brokerData = createBrokerData(CLUSTER_NAME, BROKER_NAME, BROKER_ADDR);
|
|||
|
|
++ BrokerData brokerData2 = createBrokerData(CLUSTER_NAME, BROKER_NAME2, BROKER_ADDR2);
|
|||
|
|
++ topicRouteData.setBrokerDatas(Lists.newArrayList(brokerData, brokerData2));
|
|||
|
|
++
|
|||
|
|
++ SendMessageActivity.SendMessageQueueSelector selector = new SendMessageActivity.SendMessageQueueSelector(
|
|||
|
|
++ SendMessageRequest.newBuilder()
|
|||
|
|
++ .addMessages(Message.newBuilder().build())
|
|||
|
|
++ .build()
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
++ ClientConfig cc = new ClientConfig();
|
|||
|
|
++ this.mqFaultStrategy = new MQFaultStrategy(cc, null, null);
|
|||
|
|
++ mqFaultStrategy.setSendLatencyFaultEnable(true);
|
|||
|
|
++ mqFaultStrategy.updateFaultItem(BROKER_NAME2, 1000, true, true);
|
|||
|
|
++ mqFaultStrategy.updateFaultItem(BROKER_NAME, 1000, true, false);
|
|||
|
|
++
|
|||
|
|
++ TopicRouteService topicRouteService = mock(TopicRouteService.class);
|
|||
|
|
++ when(topicRouteService.getMqFaultStrategy()).thenReturn(mqFaultStrategy);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData, topicRouteService.getMqFaultStrategy());
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ AddressableMessageQueue firstSelect = selector.select(ProxyContext.create(), messageQueueView);
|
|||
|
|
++ assertEquals(firstSelect.getBrokerName(), BROKER_NAME2);
|
|||
|
|
++
|
|||
|
|
++ mqFaultStrategy.updateFaultItem(BROKER_NAME2, 1000, true, false);
|
|||
|
|
++ mqFaultStrategy.updateFaultItem(BROKER_NAME, 1000, true, true);
|
|||
|
|
++ AddressableMessageQueue secondSelect = selector.select(ProxyContext.create(), messageQueueView);
|
|||
|
|
++ assertEquals(secondSelect.getBrokerName(), BROKER_NAME);
|
|||
|
|
++ }
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testParameterValidate() {
|
|||
|
|
+ // too large message body
|
|||
|
|
+@@ -850,4 +907,23 @@ public class SendMessageActivityTest extends BaseActivityTest {
|
|||
|
|
+ }
|
|||
|
|
+ return sb.toString();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ private static QueueData createQueueData(String brokerName, int writeQueueNums) {
|
|||
|
|
++ QueueData queueData = new QueueData();
|
|||
|
|
++ queueData.setBrokerName(brokerName);
|
|||
|
|
++ queueData.setWriteQueueNums(writeQueueNums);
|
|||
|
|
++ queueData.setPerm(PermName.PERM_WRITE);
|
|||
|
|
++ return queueData;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private static BrokerData createBrokerData(String clusterName, String brokerName, String brokerAddrs) {
|
|||
|
|
++ BrokerData brokerData = new BrokerData();
|
|||
|
|
++ brokerData.setCluster(clusterName);
|
|||
|
|
++ brokerData.setBrokerName(brokerName);
|
|||
|
|
++ HashMap<Long, String> brokerAddrsMap = new HashMap<>();
|
|||
|
|
++ brokerAddrsMap.put(MixAll.MASTER_ID, brokerAddrs);
|
|||
|
|
++ brokerData.setBrokerAddrs(brokerAddrsMap);
|
|||
|
|
++
|
|||
|
|
++ return brokerData;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/BaseServiceTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/BaseServiceTest.java
|
|||
|
|
+index c97bd5a72..ca6fe909e 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/BaseServiceTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/BaseServiceTest.java
|
|||
|
|
+@@ -78,7 +78,7 @@ public class BaseServiceTest extends InitConfigTest {
|
|||
|
|
+ topicRouteData.setBrokerDatas(Lists.newArrayList(brokerData));
|
|||
|
|
+
|
|||
|
|
+ when(this.topicRouteService.getAllMessageQueueView(any(), eq(ERR_TOPIC))).thenThrow(new MQClientException(ResponseCode.TOPIC_NOT_EXIST, ""));
|
|||
|
|
+- when(this.topicRouteService.getAllMessageQueueView(any(), eq(TOPIC))).thenReturn(new MessageQueueView(TOPIC, topicRouteData));
|
|||
|
|
+- when(this.topicRouteService.getAllMessageQueueView(any(), eq(CLUSTER_NAME))).thenReturn(new MessageQueueView(CLUSTER_NAME, topicRouteData));
|
|||
|
|
++ when(this.topicRouteService.getAllMessageQueueView(any(), eq(TOPIC))).thenReturn(new MessageQueueView(TOPIC, topicRouteData, null));
|
|||
|
|
++ when(this.topicRouteService.getAllMessageQueueView(any(), eq(CLUSTER_NAME))).thenReturn(new MessageQueueView(CLUSTER_NAME, topicRouteData, null));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelectorTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelectorTest.java
|
|||
|
|
+index e44ed28f4..d150f87c4 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelectorTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/route/MessageQueueSelectorTest.java
|
|||
|
|
+@@ -30,12 +30,12 @@ public class MessageQueueSelectorTest extends BaseServiceTest {
|
|||
|
|
+ public void testReadMessageQueue() {
|
|||
|
|
+ queueData.setPerm(PermName.PERM_READ);
|
|||
|
|
+ queueData.setReadQueueNums(0);
|
|||
|
|
+- MessageQueueSelector messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), true);
|
|||
|
|
++ MessageQueueSelector messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), null, true);
|
|||
|
|
+ assertTrue(messageQueueSelector.getQueues().isEmpty());
|
|||
|
|
+
|
|||
|
|
+ queueData.setPerm(PermName.PERM_READ);
|
|||
|
|
+ queueData.setReadQueueNums(3);
|
|||
|
|
+- messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), true);
|
|||
|
|
++ messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), null, true);
|
|||
|
|
+ assertEquals(3, messageQueueSelector.getQueues().size());
|
|||
|
|
+ assertEquals(1, messageQueueSelector.getBrokerActingQueues().size());
|
|||
|
|
+ for (int i = 0; i < messageQueueSelector.getQueues().size(); i++) {
|
|||
|
|
+@@ -58,12 +58,12 @@ public class MessageQueueSelectorTest extends BaseServiceTest {
|
|||
|
|
+ public void testWriteMessageQueue() {
|
|||
|
|
+ queueData.setPerm(PermName.PERM_WRITE);
|
|||
|
|
+ queueData.setReadQueueNums(0);
|
|||
|
|
+- MessageQueueSelector messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), false);
|
|||
|
|
++ MessageQueueSelector messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), null, false);
|
|||
|
|
+ assertTrue(messageQueueSelector.getQueues().isEmpty());
|
|||
|
|
+
|
|||
|
|
+ queueData.setPerm(PermName.PERM_WRITE);
|
|||
|
|
+ queueData.setWriteQueueNums(3);
|
|||
|
|
+- messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), false);
|
|||
|
|
++ messageQueueSelector = new MessageQueueSelector(new TopicRouteWrapper(topicRouteData, TOPIC), null, false);
|
|||
|
|
+ assertEquals(3, messageQueueSelector.getQueues().size());
|
|||
|
|
+ assertEquals(1, messageQueueSelector.getBrokerActingQueues().size());
|
|||
|
|
+ for (int i = 0; i < messageQueueSelector.getQueues().size(); i++) {
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
+index c67f4953d..43fba3d03 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
+@@ -132,7 +132,7 @@ public class HeartbeatSyncerTest extends InitConfigTest {
|
|||
|
|
+ brokerAddr.put(0L, "127.0.0.1:10911");
|
|||
|
|
+ brokerData.setBrokerAddrs(brokerAddr);
|
|||
|
|
+ topicRouteData.getBrokerDatas().add(brokerData);
|
|||
|
|
+- MessageQueueView messageQueueView = new MessageQueueView("foo", topicRouteData);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView("foo", topicRouteData, null);
|
|||
|
|
+ when(this.topicRouteService.getAllMessageQueueView(any(), anyString())).thenReturn(messageQueueView);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/transaction/ClusterTransactionServiceTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/transaction/ClusterTransactionServiceTest.java
|
|||
|
|
+index a0063544e..91af74cbe 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/transaction/ClusterTransactionServiceTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/transaction/ClusterTransactionServiceTest.java
|
|||
|
|
+@@ -64,7 +64,7 @@ public class ClusterTransactionServiceTest extends BaseServiceTest {
|
|||
|
|
+ this.clusterTransactionService = new ClusterTransactionService(this.topicRouteService, this.producerManager,
|
|||
|
|
+ this.mqClientAPIFactory);
|
|||
|
|
+
|
|||
|
|
+- MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData);
|
|||
|
|
++ MessageQueueView messageQueueView = new MessageQueueView(TOPIC, topicRouteData, null);
|
|||
|
|
+ when(this.topicRouteService.getAllMessageQueueView(any(), anyString()))
|
|||
|
|
+ .thenReturn(messageQueueView);
|
|||
|
|
+
|
|||
|
|
+@@ -127,7 +127,7 @@ public class ClusterTransactionServiceTest extends BaseServiceTest {
|
|||
|
|
+ brokerData.setBrokerAddrs(brokerAddrs);
|
|||
|
|
+ topicRouteData.getQueueDatas().add(queueData);
|
|||
|
|
+ topicRouteData.getBrokerDatas().add(brokerData);
|
|||
|
|
+- when(this.topicRouteService.getAllMessageQueueView(any(), eq(TOPIC))).thenReturn(new MessageQueueView(TOPIC, topicRouteData));
|
|||
|
|
++ when(this.topicRouteService.getAllMessageQueueView(any(), eq(TOPIC))).thenReturn(new MessageQueueView(TOPIC, topicRouteData, null));
|
|||
|
|
+
|
|||
|
|
+ TopicRouteData clusterTopicRouteData = new TopicRouteData();
|
|||
|
|
+ QueueData clusterQueueData = new QueueData();
|
|||
|
|
+@@ -141,7 +141,7 @@ public class ClusterTransactionServiceTest extends BaseServiceTest {
|
|||
|
|
+ brokerAddrs.put(MixAll.MASTER_ID, BROKER_ADDR);
|
|||
|
|
+ clusterBrokerData.setBrokerAddrs(brokerAddrs);
|
|||
|
|
+ clusterTopicRouteData.setBrokerDatas(Lists.newArrayList(clusterBrokerData));
|
|||
|
|
+- when(this.topicRouteService.getAllMessageQueueView(any(), eq(CLUSTER_NAME))).thenReturn(new MessageQueueView(CLUSTER_NAME, clusterTopicRouteData));
|
|||
|
|
++ when(this.topicRouteService.getAllMessageQueueView(any(), eq(CLUSTER_NAME))).thenReturn(new MessageQueueView(CLUSTER_NAME, clusterTopicRouteData, null));
|
|||
|
|
+
|
|||
|
|
+ TopicRouteData clusterTopicRouteData2 = new TopicRouteData();
|
|||
|
|
+ QueueData clusterQueueData2 = new QueueData();
|
|||
|
|
+@@ -155,7 +155,7 @@ public class ClusterTransactionServiceTest extends BaseServiceTest {
|
|||
|
|
+ brokerAddrs.put(MixAll.MASTER_ID, brokerAddr2);
|
|||
|
|
+ clusterBrokerData2.setBrokerAddrs(brokerAddrs);
|
|||
|
|
+ clusterTopicRouteData2.setBrokerDatas(Lists.newArrayList(clusterBrokerData2));
|
|||
|
|
+- when(this.topicRouteService.getAllMessageQueueView(any(), eq(clusterName2))).thenReturn(new MessageQueueView(clusterName2, clusterTopicRouteData2));
|
|||
|
|
++ when(this.topicRouteService.getAllMessageQueueView(any(), eq(clusterName2))).thenReturn(new MessageQueueView(clusterName2, clusterTopicRouteData2, null));
|
|||
|
|
+
|
|||
|
|
+ ConfigurationManager.getProxyConfig().setTransactionHeartbeatBatchNum(2);
|
|||
|
|
+ this.clusterTransactionService.start();
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch015-backport-fix-some-bugs.patch b/patch015-backport-fix-some-bugs.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..11c10a6bd
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch015-backport-fix-some-bugs.patch
|
|||
|
|
@@ -0,0 +1,1894 @@
|
|||
|
|
+From bd0e9c09db9748f7f74a0c707579142dccf30afc Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: PiteXChen <44110731+RapperCL@users.noreply.github.com>
|
|||
|
|
+Date: Tue, 29 Aug 2023 19:39:27 +0800
|
|||
|
|
+Subject: [PATCH 1/7] [ISSUE #7111] Remove responseFuture from the
|
|||
|
|
+ responseTable when exception occurs (#7112)
|
|||
|
|
+
|
|||
|
|
+* remove responseFuture when exception
|
|||
|
|
+* Empty-Commit
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+Co-authored-by: chenyong152 <chenyong152@midea.com>
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/remoting/netty/NettyRemotingAbstract.java | 1 +
|
|||
|
|
+ 1 file changed, 1 insertion(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+index 44d6a3df4..fce2de267 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+@@ -529,6 +529,7 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ log.warn("send a request command to channel <{}> failed.", RemotingHelper.parseChannelRemoteAddr(channel));
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
++ responseTable.remove(opaque);
|
|||
|
|
+ responseFuture.release();
|
|||
|
|
+ log.warn("send a request command to channel <" + RemotingHelper.parseChannelRemoteAddr(channel) + "> Exception", e);
|
|||
|
|
+ throw new RemotingSendRequestException(RemotingHelper.parseChannelRemoteAddr(channel), e);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From c78061bf6ca5f35452510ec4107c46735c51c316 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Wed, 30 Aug 2023 22:29:51 +0800
|
|||
|
|
+Subject: [PATCH 2/7] [ISSUE#7280] Fix and refactor handle commit exception in
|
|||
|
|
+ tiered storage (#7281)
|
|||
|
|
+
|
|||
|
|
+* refactor handle commit exception
|
|||
|
|
+
|
|||
|
|
+* refactor handle commit exception
|
|||
|
|
+
|
|||
|
|
+* fix handle commit exception
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/TieredDispatcher.java | 3 +-
|
|||
|
|
+ .../tieredstore/TieredMessageFetcher.java | 57 ++--
|
|||
|
|
+ .../tieredstore/TieredMessageStore.java | 26 +-
|
|||
|
|
+ .../provider/TieredFileSegment.java | 291 ++++++++++--------
|
|||
|
|
+ .../provider/TieredStoreProvider.java | 8 +-
|
|||
|
|
+ .../provider/posix/PosixFileSegment.java | 4 +-
|
|||
|
|
+ .../CommitLogInputStream.java} | 30 +-
|
|||
|
|
+ .../FileSegmentInputStream.java} | 49 ++-
|
|||
|
|
+ .../FileSegmentInputStreamFactory.java} | 26 +-
|
|||
|
|
+ .../tieredstore/TieredMessageStoreTest.java | 14 +-
|
|||
|
|
+ .../tieredstore/file/TieredFlatFileTest.java | 2 +
|
|||
|
|
+ .../tieredstore/file/TieredIndexFileTest.java | 2 +
|
|||
|
|
+ ...m.java => MockFileSegmentInputStream.java} | 8 +-
|
|||
|
|
+ .../TieredFileSegmentInputStreamTest.java | 24 +-
|
|||
|
|
+ .../provider/TieredFileSegmentTest.java | 89 +++++-
|
|||
|
|
+ .../provider/memory/MemoryFileSegment.java | 27 +-
|
|||
|
|
+ .../memory/MemoryFileSegmentWithoutCheck.java | 4 +-
|
|||
|
|
+ 17 files changed, 427 insertions(+), 237 deletions(-)
|
|||
|
|
+ rename tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/{inputstream/TieredCommitLogInputStream.java => stream/CommitLogInputStream.java} (88%)
|
|||
|
|
+ rename tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/{inputstream/TieredFileSegmentInputStream.java => stream/FileSegmentInputStream.java} (77%)
|
|||
|
|
+ rename tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/{inputstream/TieredFileSegmentInputStreamFactory.java => stream/FileSegmentInputStreamFactory.java} (54%)
|
|||
|
|
+ rename tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/{MockTieredFileSegmentInputStream.java => MockFileSegmentInputStream.java} (82%)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+index 1746190cd..430c2b62e 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+@@ -318,8 +318,7 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ continue;
|
|||
|
|
+ case FILE_CLOSED:
|
|||
|
|
+ tieredFlatFileManager.destroyCompositeFile(flatFile.getMessageQueue());
|
|||
|
|
+- logger.info("TieredDispatcher#dispatchFlatFile: file has been close and destroy, " +
|
|||
|
|
+- "topic: {}, queueId: {}", topic, queueId);
|
|||
|
|
++ logger.info("File has been closed and destroy, topic: {}, queueId: {}", topic, queueId);
|
|||
|
|
+ return;
|
|||
|
|
+ default:
|
|||
|
|
+ dispatchOffset--;
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
+index 9a9a3e5a5..766ff64f6 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
+@@ -273,15 +273,17 @@ public class TieredMessageFetcher implements MessageStoreFetcher {
|
|||
|
|
+ TieredStoreMetricsManager.cacheHit.add(resultWrapperList.size(), attributes);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // if no cached message found and there is currently an inflight request, wait for the request to end before continuing
|
|||
|
|
++ // If there are no messages in the cache and there are currently requests being pulled.
|
|||
|
|
++ // We need to wait for the request to return before continuing.
|
|||
|
|
+ if (resultWrapperList.isEmpty() && waitInflightRequest) {
|
|||
|
|
+- CompletableFuture<Long> future = flatFile.getInflightRequest(group, queueOffset, maxCount)
|
|||
|
|
+- .getFuture(queueOffset);
|
|||
|
|
++ CompletableFuture<Long> future =
|
|||
|
|
++ flatFile.getInflightRequest(group, queueOffset, maxCount).getFuture(queueOffset);
|
|||
|
|
+ if (!future.isDone()) {
|
|||
|
|
+ Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
+ // to prevent starvation issues, only allow waiting for inflight request once
|
|||
|
|
+ return future.thenCompose(v -> {
|
|||
|
|
+- LOGGER.debug("TieredMessageFetcher#getMessageFromCacheAsync: wait for inflight request cost: {}ms", stopwatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
++ LOGGER.debug("MessageFetcher#getMessageFromCacheAsync: wait for response cost: {}ms",
|
|||
|
|
++ stopwatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
+ return getMessageFromCacheAsync(flatFile, group, queueOffset, maxCount, false);
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+@@ -302,7 +304,8 @@ public class TieredMessageFetcher implements MessageStoreFetcher {
|
|||
|
|
+
|
|||
|
|
+ // if cache hit, result will be returned immediately and asynchronously prefetch messages for later requests
|
|||
|
|
+ if (!resultWrapperList.isEmpty()) {
|
|||
|
|
+- LOGGER.debug("TieredMessageFetcher#getMessageFromCacheAsync: cache hit: topic: {}, queue: {}, queue offset: {}, max message num: {}, cache hit num: {}",
|
|||
|
|
++ LOGGER.debug("MessageFetcher#getMessageFromCacheAsync: cache hit: " +
|
|||
|
|
++ "topic: {}, queue: {}, queue offset: {}, max message num: {}, cache hit num: {}",
|
|||
|
|
+ mq.getTopic(), mq.getQueueId(), queueOffset, maxCount, resultWrapperList.size());
|
|||
|
|
+ prefetchMessage(flatFile, group, maxCount, lastGetOffset + 1);
|
|||
|
|
+
|
|||
|
|
+@@ -316,8 +319,10 @@ public class TieredMessageFetcher implements MessageStoreFetcher {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // if cache is miss, immediately pull messages
|
|||
|
|
+- LOGGER.warn("TieredMessageFetcher#getMessageFromCacheAsync: cache miss: topic: {}, queue: {}, queue offset: {}, max message num: {}",
|
|||
|
|
++ LOGGER.warn("TieredMessageFetcher#getMessageFromCacheAsync: cache miss: " +
|
|||
|
|
++ "topic: {}, queue: {}, queue offset: {}, max message num: {}",
|
|||
|
|
+ mq.getTopic(), mq.getQueueId(), queueOffset, maxCount);
|
|||
|
|
++
|
|||
|
|
+ CompletableFuture<GetMessageResult> resultFuture;
|
|||
|
|
+ synchronized (flatFile) {
|
|||
|
|
+ int batchSize = maxCount * storeConfig.getReadAheadMinFactor();
|
|||
|
|
+@@ -453,42 +458,42 @@ public class TieredMessageFetcher implements MessageStoreFetcher {
|
|||
|
|
+ public CompletableFuture<GetMessageResult> getMessageAsync(
|
|||
|
|
+ String group, String topic, int queueId, long queueOffset, int maxCount, final MessageFilter messageFilter) {
|
|||
|
|
+
|
|||
|
|
++ GetMessageResult result = new GetMessageResult();
|
|||
|
|
+ CompositeQueueFlatFile flatFile = flatFileManager.getFlatFile(new MessageQueue(topic, brokerName, queueId));
|
|||
|
|
++
|
|||
|
|
+ if (flatFile == null) {
|
|||
|
|
+- GetMessageResult result = new GetMessageResult();
|
|||
|
|
+ result.setNextBeginOffset(queueOffset);
|
|||
|
|
+ result.setStatus(GetMessageStatus.NO_MATCHED_LOGIC_QUEUE);
|
|||
|
|
+ return CompletableFuture.completedFuture(result);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- GetMessageResult result = new GetMessageResult();
|
|||
|
|
+- long minQueueOffset = flatFile.getConsumeQueueMinOffset();
|
|||
|
|
+- long maxQueueOffset = flatFile.getConsumeQueueCommitOffset();
|
|||
|
|
+- result.setMinOffset(minQueueOffset);
|
|||
|
|
+- result.setMaxOffset(maxQueueOffset);
|
|||
|
|
++ // Max queue offset means next message put position
|
|||
|
|
++ result.setMinOffset(flatFile.getConsumeQueueMinOffset());
|
|||
|
|
++ result.setMaxOffset(flatFile.getConsumeQueueCommitOffset());
|
|||
|
|
++
|
|||
|
|
++ // Fill result according file offset.
|
|||
|
|
++ // Offset range | Result | Fix to
|
|||
|
|
++ // (-oo, 0] | no message | current offset
|
|||
|
|
++ // (0, min) | too small | min offset
|
|||
|
|
++ // [min, max) | correct |
|
|||
|
|
++ // [max, max] | overflow one | max offset
|
|||
|
|
++ // (max, +oo) | overflow badly | max offset
|
|||
|
|
+
|
|||
|
|
+- if (flatFile.getConsumeQueueCommitOffset() <= 0) {
|
|||
|
|
++ if (result.getMaxOffset() <= 0) {
|
|||
|
|
+ result.setStatus(GetMessageStatus.NO_MESSAGE_IN_QUEUE);
|
|||
|
|
+ result.setNextBeginOffset(queueOffset);
|
|||
|
|
+ return CompletableFuture.completedFuture(result);
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- // request range | result
|
|||
|
|
+- // (0, min) | too small
|
|||
|
|
+- // [min, max) | correct
|
|||
|
|
+- // [max, max] | overflow one
|
|||
|
|
+- // (max, +oo) | overflow badly
|
|||
|
|
+- if (queueOffset < minQueueOffset) {
|
|||
|
|
++ } else if (queueOffset < result.getMinOffset()) {
|
|||
|
|
+ result.setStatus(GetMessageStatus.OFFSET_TOO_SMALL);
|
|||
|
|
+- result.setNextBeginOffset(flatFile.getConsumeQueueMinOffset());
|
|||
|
|
++ result.setNextBeginOffset(result.getMinOffset());
|
|||
|
|
+ return CompletableFuture.completedFuture(result);
|
|||
|
|
+- } else if (queueOffset == maxQueueOffset) {
|
|||
|
|
++ } else if (queueOffset == result.getMaxOffset()) {
|
|||
|
|
+ result.setStatus(GetMessageStatus.OFFSET_OVERFLOW_ONE);
|
|||
|
|
+- result.setNextBeginOffset(flatFile.getConsumeQueueCommitOffset());
|
|||
|
|
++ result.setNextBeginOffset(result.getMaxOffset());
|
|||
|
|
+ return CompletableFuture.completedFuture(result);
|
|||
|
|
+- } else if (queueOffset > maxQueueOffset) {
|
|||
|
|
++ } else if (queueOffset > result.getMaxOffset()) {
|
|||
|
|
+ result.setStatus(GetMessageStatus.OFFSET_OVERFLOW_BADLY);
|
|||
|
|
+- result.setNextBeginOffset(flatFile.getConsumeQueueCommitOffset());
|
|||
|
|
++ result.setNextBeginOffset(result.getMaxOffset());
|
|||
|
|
+ return CompletableFuture.completedFuture(result);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+index 5240ac8e9..78e855f36 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+@@ -99,11 +99,11 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ return storeConfig;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public boolean viaTieredStorage(String topic, int queueId, long offset) {
|
|||
|
|
+- return viaTieredStorage(topic, queueId, offset, 1);
|
|||
|
|
++ public boolean fetchFromCurrentStore(String topic, int queueId, long offset) {
|
|||
|
|
++ return fetchFromCurrentStore(topic, queueId, offset, 1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public boolean viaTieredStorage(String topic, int queueId, long offset, int batchSize) {
|
|||
|
|
++ public boolean fetchFromCurrentStore(String topic, int queueId, long offset, int batchSize) {
|
|||
|
|
+ TieredMessageStoreConfig.TieredStorageLevel deepStorageLevel = storeConfig.getTieredStorageLevel();
|
|||
|
|
+
|
|||
|
|
+ if (deepStorageLevel.check(TieredMessageStoreConfig.TieredStorageLevel.FORCE)) {
|
|||
|
|
+@@ -146,8 +146,10 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ public CompletableFuture<GetMessageResult> getMessageAsync(String group, String topic,
|
|||
|
|
+ int queueId, long offset, int maxMsgNums, MessageFilter messageFilter) {
|
|||
|
|
+
|
|||
|
|
+- if (!viaTieredStorage(topic, queueId, offset, maxMsgNums)) {
|
|||
|
|
+- logger.trace("GetMessageAsync from next store topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
++ if (fetchFromCurrentStore(topic, queueId, offset, maxMsgNums)) {
|
|||
|
|
++ logger.trace("GetMessageAsync from current store, topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
++ } else {
|
|||
|
|
++ logger.trace("GetMessageAsync from next store, topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
+ return next.getMessageAsync(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -168,14 +170,14 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+
|
|||
|
|
+ if (next.checkInStoreByConsumeOffset(topic, queueId, offset)) {
|
|||
|
|
+ TieredStoreMetricsManager.fallbackTotal.add(1, latencyAttributes);
|
|||
|
|
+- logger.debug("GetMessageAsync not found then try back to next store, result: {}, " +
|
|||
|
|
++ logger.debug("GetMessageAsync not found, then back to next store, result: {}, " +
|
|||
|
|
+ "topic: {}, queue: {}, queue offset: {}, offset range: {}-{}",
|
|||
|
|
+ result.getStatus(), topic, queueId, offset, result.getMinOffset(), result.getMaxOffset());
|
|||
|
|
+ return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // system topic
|
|||
|
|
++ // Fetch system topic data from the broker when using the force level.
|
|||
|
|
+ if (result.getStatus() == GetMessageStatus.NO_MATCHED_LOGIC_QUEUE) {
|
|||
|
|
+ if (TieredStoreUtil.isSystemTopic(topic) || PopAckConstants.isStartWithRevivePrefix(topic)) {
|
|||
|
|
+ return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+@@ -198,7 +200,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ TieredStoreMetricsManager.messagesOutTotal.add(result.getMessageCount(), messagesOutAttributes);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // fix min or max offset according next store
|
|||
|
|
++ // Fix min or max offset according next store at last
|
|||
|
|
+ long minOffsetInQueue = next.getMinOffsetInQueue(topic, queueId);
|
|||
|
|
+ if (minOffsetInQueue >= 0 && minOffsetInQueue < result.getMinOffset()) {
|
|||
|
|
+ result.setMinOffset(minOffsetInQueue);
|
|||
|
|
+@@ -209,7 +211,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ }
|
|||
|
|
+ return result;
|
|||
|
|
+ }).exceptionally(e -> {
|
|||
|
|
+- logger.error("GetMessageAsync from tiered store failed: ", e);
|
|||
|
|
++ logger.error("GetMessageAsync from tiered store failed", e);
|
|||
|
|
+ return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+@@ -251,7 +253,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ .build();
|
|||
|
|
+ TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes);
|
|||
|
|
+ if (time < 0) {
|
|||
|
|
+- logger.debug("TieredMessageStore#getEarliestMessageTimeAsync: get earliest message time failed, try to get earliest message time from next store: topic: {}, queue: {}",
|
|||
|
|
++ logger.debug("GetEarliestMessageTimeAsync failed, try to get earliest message time from next store: topic: {}, queue: {}",
|
|||
|
|
+ topic, queueId);
|
|||
|
|
+ return finalNextEarliestMessageTime != Long.MAX_VALUE ? finalNextEarliestMessageTime : -1;
|
|||
|
|
+ }
|
|||
|
|
+@@ -262,7 +264,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<Long> getMessageStoreTimeStampAsync(String topic, int queueId,
|
|||
|
|
+ long consumeQueueOffset) {
|
|||
|
|
+- if (viaTieredStorage(topic, queueId, consumeQueueOffset)) {
|
|||
|
|
++ if (fetchFromCurrentStore(topic, queueId, consumeQueueOffset)) {
|
|||
|
|
+ Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
+ return fetcher.getMessageStoreTimeStampAsync(topic, queueId, consumeQueueOffset)
|
|||
|
|
+ .thenApply(time -> {
|
|||
|
|
+@@ -272,7 +274,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ .build();
|
|||
|
|
+ TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes);
|
|||
|
|
+ if (time == -1) {
|
|||
|
|
+- logger.debug("TieredMessageStore#getMessageStoreTimeStampAsync: get message time failed, try to get message time from next store: topic: {}, queue: {}, queue offset: {}",
|
|||
|
|
++ logger.debug("GetEarliestMessageTimeAsync failed, try to get message time from next store, topic: {}, queue: {}, queue offset: {}",
|
|||
|
|
+ topic, queueId, consumeQueueOffset);
|
|||
|
|
+ return next.getMessageStoreTimeStamp(topic, queueId, consumeQueueOffset);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegment.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegment.java
|
|||
|
|
+index 5062c7d9e..32911a6e8 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegment.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegment.java
|
|||
|
|
+@@ -16,14 +16,11 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.provider;
|
|||
|
|
+
|
|||
|
|
+-import com.google.common.annotations.VisibleForTesting;
|
|||
|
|
+-import com.google.common.base.Stopwatch;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.Semaphore;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+@@ -35,8 +32,8 @@ import org.apache.rocketmq.tieredstore.exception.TieredStoreException;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredCommitLog;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredConsumeQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredIndexFile;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStreamFactory;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStreamFactory;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+
|
|||
|
|
+@@ -50,22 +47,23 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ protected final TieredMessageStoreConfig storeConfig;
|
|||
|
|
+
|
|||
|
|
+ private final long maxSize;
|
|||
|
|
+- private final ReentrantLock bufferLock;
|
|||
|
|
+- private final Semaphore commitLock;
|
|||
|
|
++ private final ReentrantLock bufferLock = new ReentrantLock();
|
|||
|
|
++ private final Semaphore commitLock = new Semaphore(1);
|
|||
|
|
+
|
|||
|
|
+- private volatile boolean full;
|
|||
|
|
+- private volatile boolean closed;
|
|||
|
|
++ private volatile boolean full = false;
|
|||
|
|
++ private volatile boolean closed = false;
|
|||
|
|
+
|
|||
|
|
+- private volatile long minTimestamp;
|
|||
|
|
+- private volatile long maxTimestamp;
|
|||
|
|
+- private volatile long commitPosition;
|
|||
|
|
+- private volatile long appendPosition;
|
|||
|
|
++ private volatile long minTimestamp = Long.MAX_VALUE;
|
|||
|
|
++ private volatile long maxTimestamp = Long.MAX_VALUE;
|
|||
|
|
++ private volatile long commitPosition = 0L;
|
|||
|
|
++ private volatile long appendPosition = 0L;
|
|||
|
|
+
|
|||
|
|
+ // only used in commitLog
|
|||
|
|
+- private volatile long dispatchCommitOffset = 0;
|
|||
|
|
++ private volatile long dispatchCommitOffset = 0L;
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer codaBuffer;
|
|||
|
|
+- private List<ByteBuffer> uploadBufferList = new ArrayList<>();
|
|||
|
|
++ private List<ByteBuffer> bufferList = new ArrayList<>();
|
|||
|
|
++ private FileSegmentInputStream fileSegmentInputStream;
|
|||
|
|
+ private CompletableFuture<Boolean> flightCommitRequest = CompletableFuture.completedFuture(false);
|
|||
|
|
+
|
|||
|
|
+ public TieredFileSegment(TieredMessageStoreConfig storeConfig,
|
|||
|
|
+@@ -75,21 +73,13 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ this.fileType = fileType;
|
|||
|
|
+ this.filePath = filePath;
|
|||
|
|
+ this.baseOffset = baseOffset;
|
|||
|
|
+-
|
|||
|
|
+- this.closed = false;
|
|||
|
|
+- this.bufferLock = new ReentrantLock();
|
|||
|
|
+- this.commitLock = new Semaphore(1);
|
|||
|
|
+-
|
|||
|
|
+- this.commitPosition = 0L;
|
|||
|
|
+- this.appendPosition = 0L;
|
|||
|
|
+- this.minTimestamp = Long.MAX_VALUE;
|
|||
|
|
+- this.maxTimestamp = Long.MAX_VALUE;
|
|||
|
|
+-
|
|||
|
|
+- // The max segment size of a file is determined by the file type
|
|||
|
|
+- this.maxSize = getMaxSizeAccordingFileType(storeConfig);
|
|||
|
|
++ this.maxSize = getMaxSizeByFileType();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private long getMaxSizeAccordingFileType(TieredMessageStoreConfig storeConfig) {
|
|||
|
|
++ /**
|
|||
|
|
++ * The max segment size of a file is determined by the file type
|
|||
|
|
++ */
|
|||
|
|
++ protected long getMaxSizeByFileType() {
|
|||
|
|
+ switch (fileType) {
|
|||
|
|
+ case COMMIT_LOG:
|
|||
|
|
+ return storeConfig.getTieredStoreCommitLogMaxSize();
|
|||
|
|
+@@ -184,39 +174,23 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ this.appendPosition = pos;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private List<ByteBuffer> rollingUploadBuffer() {
|
|||
|
|
++ private List<ByteBuffer> borrowBuffer() {
|
|||
|
|
+ bufferLock.lock();
|
|||
|
|
+ try {
|
|||
|
|
+- List<ByteBuffer> tmp = uploadBufferList;
|
|||
|
|
+- uploadBufferList = new ArrayList<>();
|
|||
|
|
++ List<ByteBuffer> tmp = bufferList;
|
|||
|
|
++ bufferList = new ArrayList<>();
|
|||
|
|
+ return tmp;
|
|||
|
|
+ } finally {
|
|||
|
|
+ bufferLock.unlock();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private void sendBackBuffer(TieredFileSegmentInputStream inputStream) {
|
|||
|
|
+- bufferLock.lock();
|
|||
|
|
+- try {
|
|||
|
|
+- List<ByteBuffer> tmpBufferList = inputStream.getUploadBufferList();
|
|||
|
|
+- for (ByteBuffer buffer : tmpBufferList) {
|
|||
|
|
+- buffer.rewind();
|
|||
|
|
+- }
|
|||
|
|
+- tmpBufferList.addAll(uploadBufferList);
|
|||
|
|
+- uploadBufferList = tmpBufferList;
|
|||
|
|
+- if (inputStream.getCodaBuffer() != null) {
|
|||
|
|
+- codaBuffer.rewind();
|
|||
|
|
+- }
|
|||
|
|
+- } finally {
|
|||
|
|
+- bufferLock.unlock();
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ @SuppressWarnings("NonAtomicOperationOnVolatileField")
|
|||
|
|
+- public AppendResult append(ByteBuffer byteBuf, long timeStamp) {
|
|||
|
|
++ public AppendResult append(ByteBuffer byteBuf, long timestamp) {
|
|||
|
|
+ if (closed) {
|
|||
|
|
+ return AppendResult.FILE_CLOSED;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ bufferLock.lock();
|
|||
|
|
+ try {
|
|||
|
|
+ if (full || codaBuffer != null) {
|
|||
|
|
+@@ -227,7 +201,8 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ minTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_BEGIN_TIME_STAMP_POSITION);
|
|||
|
|
+ maxTimestamp = byteBuf.getLong(TieredIndexFile.INDEX_FILE_HEADER_END_TIME_STAMP_POSITION);
|
|||
|
|
+ appendPosition += byteBuf.remaining();
|
|||
|
|
+- uploadBufferList.add(byteBuf);
|
|||
|
|
++ // IndexFile is large and not change after compaction, no need deep copy
|
|||
|
|
++ bufferList.add(byteBuf);
|
|||
|
|
+ setFull();
|
|||
|
|
+ return AppendResult.SUCCESS;
|
|||
|
|
+ }
|
|||
|
|
+@@ -236,23 +211,34 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ setFull();
|
|||
|
|
+ return AppendResult.FILE_FULL;
|
|||
|
|
+ }
|
|||
|
|
+- if (uploadBufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
|
|||
|
|
++
|
|||
|
|
++ if (bufferList.size() > storeConfig.getTieredStoreGroupCommitCount()
|
|||
|
|
+ || appendPosition - commitPosition > storeConfig.getTieredStoreGroupCommitSize()) {
|
|||
|
|
+ commitAsync();
|
|||
|
|
+ }
|
|||
|
|
+- if (uploadBufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
|
|||
|
|
+- logger.debug("TieredFileSegment#append: buffer full: file: {}, upload buffer size: {}",
|
|||
|
|
+- getPath(), uploadBufferList.size());
|
|||
|
|
++
|
|||
|
|
++ if (bufferList.size() > storeConfig.getTieredStoreMaxGroupCommitCount()) {
|
|||
|
|
++ logger.debug("File segment append buffer full, file: {}, buffer size: {}, pending bytes: {}",
|
|||
|
|
++ getPath(), bufferList.size(), appendPosition - commitPosition);
|
|||
|
|
+ return AppendResult.BUFFER_FULL;
|
|||
|
|
+ }
|
|||
|
|
+- if (timeStamp != Long.MAX_VALUE) {
|
|||
|
|
+- maxTimestamp = timeStamp;
|
|||
|
|
++
|
|||
|
|
++ if (timestamp != Long.MAX_VALUE) {
|
|||
|
|
++ maxTimestamp = timestamp;
|
|||
|
|
+ if (minTimestamp == Long.MAX_VALUE) {
|
|||
|
|
+- minTimestamp = timeStamp;
|
|||
|
|
++ minTimestamp = timestamp;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ appendPosition += byteBuf.remaining();
|
|||
|
|
+- uploadBufferList.add(byteBuf);
|
|||
|
|
++
|
|||
|
|
++ // deep copy buffer
|
|||
|
|
++ ByteBuffer byteBuffer = ByteBuffer.allocateDirect(byteBuf.remaining());
|
|||
|
|
++ byteBuffer.put(byteBuf);
|
|||
|
|
++ byteBuffer.flip();
|
|||
|
|
++ byteBuf.rewind();
|
|||
|
|
++
|
|||
|
|
++ bufferList.add(byteBuffer);
|
|||
|
|
+ return AppendResult.SUCCESS;
|
|||
|
|
+ } finally {
|
|||
|
|
+ bufferLock.unlock();
|
|||
|
|
+@@ -267,7 +253,6 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ return appendPosition;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @VisibleForTesting
|
|||
|
|
+ public void setAppendPosition(long appendPosition) {
|
|||
|
|
+ this.appendPosition = appendPosition;
|
|||
|
|
+ }
|
|||
|
|
+@@ -333,6 +318,8 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ if (closed) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
++ // result is false when we send real commit request
|
|||
|
|
++ // use join for wait flight request done
|
|||
|
|
+ Boolean result = commitAsync().join();
|
|||
|
|
+ if (!result) {
|
|||
|
|
+ result = flightCommitRequest.join();
|
|||
|
|
+@@ -340,92 +327,156 @@ public abstract class TieredFileSegment implements Comparable<TieredFileSegment>
|
|||
|
|
+ return result;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ private void releaseCommitLock() {
|
|||
|
|
++ if (commitLock.availablePermits() == 0) {
|
|||
|
|
++ commitLock.release();
|
|||
|
|
++ } else {
|
|||
|
|
++ logger.error("[Bug] FileSegmentCommitAsync, lock is already released: available permits: {}",
|
|||
|
|
++ commitLock.availablePermits());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private void updateDispatchCommitOffset(List<ByteBuffer> bufferList) {
|
|||
|
|
++ if (fileType == FileSegmentType.COMMIT_LOG && bufferList.size() > 0) {
|
|||
|
|
++ dispatchCommitOffset =
|
|||
|
|
++ MessageBufferUtil.getQueueOffset(bufferList.get(bufferList.size() - 1));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * @return false: commit, true: no commit operation
|
|||
|
|
++ */
|
|||
|
|
+ @SuppressWarnings("NonAtomicOperationOnVolatileField")
|
|||
|
|
+ public CompletableFuture<Boolean> commitAsync() {
|
|||
|
|
+ if (closed) {
|
|||
|
|
+ return CompletableFuture.completedFuture(false);
|
|||
|
|
+ }
|
|||
|
|
+- Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
++
|
|||
|
|
+ if (!needCommit()) {
|
|||
|
|
+ return CompletableFuture.completedFuture(true);
|
|||
|
|
+ }
|
|||
|
|
+- try {
|
|||
|
|
+- int permits = commitLock.drainPermits();
|
|||
|
|
+- if (permits <= 0) {
|
|||
|
|
+- return CompletableFuture.completedFuture(false);
|
|||
|
|
+- }
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
++
|
|||
|
|
++ if (commitLock.drainPermits() <= 0) {
|
|||
|
|
+ return CompletableFuture.completedFuture(false);
|
|||
|
|
+ }
|
|||
|
|
+- List<ByteBuffer> bufferList = rollingUploadBuffer();
|
|||
|
|
+- int bufferSize = 0;
|
|||
|
|
+- for (ByteBuffer buffer : bufferList) {
|
|||
|
|
+- bufferSize += buffer.remaining();
|
|||
|
|
+- }
|
|||
|
|
+- if (codaBuffer != null) {
|
|||
|
|
+- bufferSize += codaBuffer.remaining();
|
|||
|
|
+- }
|
|||
|
|
+- if (bufferSize == 0) {
|
|||
|
|
+- return CompletableFuture.completedFuture(true);
|
|||
|
|
+- }
|
|||
|
|
+- TieredFileSegmentInputStream inputStream = TieredFileSegmentInputStreamFactory.build(
|
|||
|
|
+- fileType, baseOffset + commitPosition, bufferList, codaBuffer, bufferSize);
|
|||
|
|
+- int finalBufferSize = bufferSize;
|
|||
|
|
++
|
|||
|
|
+ try {
|
|||
|
|
+- flightCommitRequest = commit0(inputStream, commitPosition, bufferSize, fileType != FileSegmentType.INDEX)
|
|||
|
|
++ if (fileSegmentInputStream != null) {
|
|||
|
|
++ long fileSize = this.getSize();
|
|||
|
|
++ if (fileSize == -1L) {
|
|||
|
|
++ logger.error("Get commit position error before commit, Commit: %d, Expect: %d, Current Max: %d, FileName: %s",
|
|||
|
|
++ commitPosition, commitPosition + fileSegmentInputStream.getContentLength(), appendPosition, getPath());
|
|||
|
|
++ releaseCommitLock();
|
|||
|
|
++ return CompletableFuture.completedFuture(false);
|
|||
|
|
++ } else {
|
|||
|
|
++ if (correctPosition(fileSize, null)) {
|
|||
|
|
++ updateDispatchCommitOffset(fileSegmentInputStream.getBufferList());
|
|||
|
|
++ fileSegmentInputStream = null;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ int bufferSize;
|
|||
|
|
++ if (fileSegmentInputStream != null) {
|
|||
|
|
++ bufferSize = fileSegmentInputStream.available();
|
|||
|
|
++ } else {
|
|||
|
|
++ List<ByteBuffer> bufferList = borrowBuffer();
|
|||
|
|
++ bufferSize = bufferList.stream().mapToInt(ByteBuffer::remaining).sum()
|
|||
|
|
++ + (codaBuffer != null ? codaBuffer.remaining() : 0);
|
|||
|
|
++ if (bufferSize == 0) {
|
|||
|
|
++ releaseCommitLock();
|
|||
|
|
++ return CompletableFuture.completedFuture(true);
|
|||
|
|
++ }
|
|||
|
|
++ fileSegmentInputStream = FileSegmentInputStreamFactory.build(
|
|||
|
|
++ fileType, baseOffset + commitPosition, bufferList, codaBuffer, bufferSize);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ return flightCommitRequest = this
|
|||
|
|
++ .commit0(fileSegmentInputStream, commitPosition, bufferSize, fileType != FileSegmentType.INDEX)
|
|||
|
|
+ .thenApply(result -> {
|
|||
|
|
+ if (result) {
|
|||
|
|
+- if (fileType == FileSegmentType.COMMIT_LOG && bufferList.size() > 0) {
|
|||
|
|
+- dispatchCommitOffset = MessageBufferUtil.getQueueOffset(bufferList.get(bufferList.size() - 1));
|
|||
|
|
+- }
|
|||
|
|
+- commitPosition += finalBufferSize;
|
|||
|
|
++ updateDispatchCommitOffset(fileSegmentInputStream.getBufferList());
|
|||
|
|
++ commitPosition += bufferSize;
|
|||
|
|
++ fileSegmentInputStream = null;
|
|||
|
|
+ return true;
|
|||
|
|
+- }
|
|||
|
|
+- sendBackBuffer(inputStream);
|
|||
|
|
+- return false;
|
|||
|
|
+- })
|
|||
|
|
+- .exceptionally(e -> handleCommitException(inputStream, e))
|
|||
|
|
+- .whenComplete((result, e) -> {
|
|||
|
|
+- if (commitLock.availablePermits() == 0) {
|
|||
|
|
+- logger.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
|
|||
|
|
+- commitLock.release();
|
|||
|
|
+ } else {
|
|||
|
|
+- logger.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
|
|||
|
|
++ fileSegmentInputStream.rewind();
|
|||
|
|
++ return false;
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- return flightCommitRequest;
|
|||
|
|
++ })
|
|||
|
|
++ .exceptionally(this::handleCommitException)
|
|||
|
|
++ .whenComplete((result, e) -> releaseCommitLock());
|
|||
|
|
++
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- handleCommitException(inputStream, e);
|
|||
|
|
+- if (commitLock.availablePermits() == 0) {
|
|||
|
|
+- logger.debug("TieredFileSegment#commitAsync: commit cost: {}ms, file: {}, item count: {}, buffer size: {}", stopwatch.elapsed(TimeUnit.MILLISECONDS), getPath(), bufferList.size(), finalBufferSize);
|
|||
|
|
+- commitLock.release();
|
|||
|
|
+- } else {
|
|||
|
|
+- logger.error("[Bug]TieredFileSegment#commitAsync: commit lock is already released: available permits: {}", commitLock.availablePermits());
|
|||
|
|
+- }
|
|||
|
|
++ handleCommitException(e);
|
|||
|
|
++ releaseCommitLock();
|
|||
|
|
+ }
|
|||
|
|
+ return CompletableFuture.completedFuture(false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private boolean handleCommitException(TieredFileSegmentInputStream inputStream, Throwable e) {
|
|||
|
|
++ private long getCorrectFileSize(Throwable throwable) {
|
|||
|
|
++ if (throwable instanceof TieredStoreException) {
|
|||
|
|
++ long fileSize = ((TieredStoreException) throwable).getPosition();
|
|||
|
|
++ if (fileSize > 0) {
|
|||
|
|
++ return fileSize;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return getSize();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private boolean handleCommitException(Throwable e) {
|
|||
|
|
++ // Get root cause here
|
|||
|
|
+ Throwable cause = e.getCause() != null ? e.getCause() : e;
|
|||
|
|
+- sendBackBuffer(inputStream);
|
|||
|
|
+- long realSize = 0;
|
|||
|
|
+- if (cause instanceof TieredStoreException && ((TieredStoreException) cause).getPosition() > 0) {
|
|||
|
|
+- realSize = ((TieredStoreException) cause).getPosition();
|
|||
|
|
++ long fileSize = this.getCorrectFileSize(cause);
|
|||
|
|
++
|
|||
|
|
++ if (fileSize == -1L) {
|
|||
|
|
++ logger.error("Get commit position error, Commit: %d, Expect: %d, Current Max: %d, FileName: %s",
|
|||
|
|
++ commitPosition, commitPosition + fileSegmentInputStream.getContentLength(), appendPosition, getPath());
|
|||
|
|
++ fileSegmentInputStream.rewind();
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (correctPosition(fileSize, cause)) {
|
|||
|
|
++ updateDispatchCommitOffset(fileSegmentInputStream.getBufferList());
|
|||
|
|
++ fileSegmentInputStream = null;
|
|||
|
|
++ return true;
|
|||
|
|
++ } else {
|
|||
|
|
++ fileSegmentInputStream.rewind();
|
|||
|
|
++ return false;
|
|||
|
|
+ }
|
|||
|
|
+- if (realSize <= 0) {
|
|||
|
|
+- realSize = getSize();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * return true to clear buffer
|
|||
|
|
++ */
|
|||
|
|
++ private boolean correctPosition(long fileSize, Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
++ // Current we have three offsets here: commit offset, expect offset, file size.
|
|||
|
|
++ // We guarantee that the commit offset is less than or equal to the expect offset.
|
|||
|
|
++ // Max offset will increase because we can continuously put in new buffers
|
|||
|
|
++ String handleInfo = throwable == null ? "before commit" : "after commit";
|
|||
|
|
++ long expectPosition = commitPosition + fileSegmentInputStream.getContentLength();
|
|||
|
|
++
|
|||
|
|
++ String offsetInfo = String.format("Correct Commit Position, %s, result=[{}], " +
|
|||
|
|
++ "Commit: %d, Expect: %d, Current Max: %d, FileSize: %d, FileName: %s",
|
|||
|
|
++ handleInfo, commitPosition, expectPosition, appendPosition, fileSize, this.getPath());
|
|||
|
|
++
|
|||
|
|
++ // We are believing that the file size returned by the server is correct,
|
|||
|
|
++ // can reset the commit offset to the file size reported by the storage system.
|
|||
|
|
++ if (fileSize == expectPosition) {
|
|||
|
|
++ logger.info(offsetInfo, "Success", throwable);
|
|||
|
|
++ commitPosition = fileSize;
|
|||
|
|
++ return true;
|
|||
|
|
+ }
|
|||
|
|
+- if (realSize > 0 && realSize > commitPosition) {
|
|||
|
|
+- logger.error("TieredFileSegment#handleCommitException: commit failed: file: {}, try to fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
|
|||
|
|
+- // TODO check if this diff part is uploaded to backend storage
|
|||
|
|
+- long diff = appendPosition - commitPosition;
|
|||
|
|
+- commitPosition = realSize;
|
|||
|
|
+- appendPosition = realSize + diff;
|
|||
|
|
+- // TODO check if appendPosition is large than maxOffset
|
|||
|
|
+- } else if (realSize < commitPosition) {
|
|||
|
|
+- logger.error("[Bug]TieredFileSegment#handleCommitException: commit failed: file: {}, can not fix position: origin: {}, real: {}", getPath(), commitPosition, realSize, cause);
|
|||
|
|
++
|
|||
|
|
++ if (fileSize < commitPosition) {
|
|||
|
|
++ logger.error(offsetInfo, "FileSizeIncorrect", throwable);
|
|||
|
|
++ } else if (fileSize == commitPosition) {
|
|||
|
|
++ logger.warn(offsetInfo, "CommitFailed", throwable);
|
|||
|
|
++ } else if (fileSize > commitPosition) {
|
|||
|
|
++ logger.warn(offsetInfo, "PartialSuccess", throwable);
|
|||
|
|
+ }
|
|||
|
|
++ commitPosition = fileSize;
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreProvider.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreProvider.java
|
|||
|
|
+index 5a0ca25f5..0db3eaf8f 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreProvider.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreProvider.java
|
|||
|
|
+@@ -18,7 +18,7 @@ package org.apache.rocketmq.tieredstore.provider;
|
|||
|
|
+
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
+
|
|||
|
|
+ public interface TieredStoreProvider {
|
|||
|
|
+
|
|||
|
|
+@@ -30,7 +30,9 @@ public interface TieredStoreProvider {
|
|||
|
|
+ String getPath();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+- * Get file size in backend file system
|
|||
|
|
++ * Get the real length of the file.
|
|||
|
|
++ * Return 0 if the file does not exist,
|
|||
|
|
++ * Return -1 if system get size failed.
|
|||
|
|
+ *
|
|||
|
|
+ * @return file real size
|
|||
|
|
+ */
|
|||
|
|
+@@ -71,5 +73,5 @@ public interface TieredStoreProvider {
|
|||
|
|
+ * @param append try to append or create a new file
|
|||
|
|
+ * @return put result, <code>true</code> if data successfully write; <code>false</code> otherwise
|
|||
|
|
+ */
|
|||
|
|
+- CompletableFuture<Boolean> commit0(TieredFileSegmentInputStream inputStream,long position, int length, boolean append);
|
|||
|
|
++ CompletableFuture<Boolean> commit0(FileSegmentInputStream inputStream,long position, int length, boolean append);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/posix/PosixFileSegment.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/posix/PosixFileSegment.java
|
|||
|
|
+index 52be90b1d..7e949cb28 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/posix/PosixFileSegment.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/posix/PosixFileSegment.java
|
|||
|
|
+@@ -36,7 +36,7 @@ import org.apache.rocketmq.tieredstore.common.TieredMessageStoreConfig;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.TieredStoreExecutor;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsManager;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.provider.TieredFileSegment;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+
|
|||
|
|
+ import static org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsConstant.LABEL_FILE_TYPE;
|
|||
|
|
+@@ -184,7 +184,7 @@ public class PosixFileSegment extends TieredFileSegment {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<Boolean> commit0(
|
|||
|
|
+- TieredFileSegmentInputStream inputStream, long position, int length, boolean append) {
|
|||
|
|
++ FileSegmentInputStream inputStream, long position, int length, boolean append) {
|
|||
|
|
+
|
|||
|
|
+ Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
+ AttributesBuilder attributesBuilder = newAttributesBuilder()
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredCommitLogInputStream.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/CommitLogInputStream.java
|
|||
|
|
+similarity index 88%
|
|||
|
|
+rename from tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredCommitLogInputStream.java
|
|||
|
|
+rename to tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/CommitLogInputStream.java
|
|||
|
|
+index c70bb7656..13b6e0ef9 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredCommitLogInputStream.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/CommitLogInputStream.java
|
|||
|
|
+@@ -15,7 +15,7 @@
|
|||
|
|
+ * limitations under the License.
|
|||
|
|
+ */
|
|||
|
|
+
|
|||
|
|
+-package org.apache.rocketmq.tieredstore.provider.inputstream;
|
|||
|
|
++package org.apache.rocketmq.tieredstore.provider.stream;
|
|||
|
|
+
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+@@ -23,20 +23,23 @@ import java.util.List;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtil;
|
|||
|
|
+
|
|||
|
|
+-public class TieredCommitLogInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
++public class CommitLogInputStream extends FileSegmentInputStream {
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * commitLogOffset is the real physical offset of the commitLog buffer which is being read
|
|||
|
|
+ */
|
|||
|
|
++ private final long startCommitLogOffset;
|
|||
|
|
++
|
|||
|
|
+ private long commitLogOffset;
|
|||
|
|
+
|
|||
|
|
+ private final ByteBuffer codaBuffer;
|
|||
|
|
+
|
|||
|
|
+ private long markCommitLogOffset = -1;
|
|||
|
|
+
|
|||
|
|
+- public TieredCommitLogInputStream(FileSegmentType fileType, long startOffset,
|
|||
|
|
++ public CommitLogInputStream(FileSegmentType fileType, long startOffset,
|
|||
|
|
+ List<ByteBuffer> uploadBufferList, ByteBuffer codaBuffer, int contentLength) {
|
|||
|
|
+ super(fileType, uploadBufferList, contentLength);
|
|||
|
|
++ this.startCommitLogOffset = startOffset;
|
|||
|
|
+ this.commitLogOffset = startOffset;
|
|||
|
|
+ this.codaBuffer = codaBuffer;
|
|||
|
|
+ }
|
|||
|
|
+@@ -53,6 +56,15 @@ public class TieredCommitLogInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
+ this.commitLogOffset = markCommitLogOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public synchronized void rewind() {
|
|||
|
|
++ super.rewind();
|
|||
|
|
++ this.commitLogOffset = this.startCommitLogOffset;
|
|||
|
|
++ if (this.codaBuffer != null) {
|
|||
|
|
++ this.codaBuffer.rewind();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public ByteBuffer getCodaBuffer() {
|
|||
|
|
+ return this.codaBuffer;
|
|||
|
|
+@@ -64,17 +76,17 @@ public class TieredCommitLogInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
+ return -1;
|
|||
|
|
+ }
|
|||
|
|
+ readPosition++;
|
|||
|
|
+- if (curReadBufferIndex >= uploadBufferList.size()) {
|
|||
|
|
++ if (curReadBufferIndex >= bufferList.size()) {
|
|||
|
|
+ return readCoda();
|
|||
|
|
+ }
|
|||
|
|
+ int res;
|
|||
|
|
+ if (readPosInCurBuffer >= curBuffer.remaining()) {
|
|||
|
|
+ curReadBufferIndex++;
|
|||
|
|
+- if (curReadBufferIndex >= uploadBufferList.size()) {
|
|||
|
|
++ if (curReadBufferIndex >= bufferList.size()) {
|
|||
|
|
+ readPosInCurBuffer = 0;
|
|||
|
|
+ return readCoda();
|
|||
|
|
+ }
|
|||
|
|
+- curBuffer = uploadBufferList.get(curReadBufferIndex);
|
|||
|
|
++ curBuffer = bufferList.get(curReadBufferIndex);
|
|||
|
|
+ commitLogOffset += readPosInCurBuffer;
|
|||
|
|
+ readPosInCurBuffer = 0;
|
|||
|
|
+ }
|
|||
|
|
+@@ -119,9 +131,9 @@ public class TieredCommitLogInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
+ int posInCurBuffer = readPosInCurBuffer;
|
|||
|
|
+ long curCommitLogOffset = commitLogOffset;
|
|||
|
|
+ ByteBuffer curBuf = curBuffer;
|
|||
|
|
+- while (needRead > 0 && bufIndex <= uploadBufferList.size()) {
|
|||
|
|
++ while (needRead > 0 && bufIndex <= bufferList.size()) {
|
|||
|
|
+ int readLen, remaining, realReadLen = 0;
|
|||
|
|
+- if (bufIndex == uploadBufferList.size()) {
|
|||
|
|
++ if (bufIndex == bufferList.size()) {
|
|||
|
|
+ // read from coda buffer
|
|||
|
|
+ remaining = codaBuffer.remaining() - posInCurBuffer;
|
|||
|
|
+ readLen = Math.min(remaining, needRead);
|
|||
|
|
+@@ -137,7 +149,7 @@ public class TieredCommitLogInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
+ }
|
|||
|
|
+ remaining = curBuf.remaining() - posInCurBuffer;
|
|||
|
|
+ readLen = Math.min(remaining, needRead);
|
|||
|
|
+- curBuf = uploadBufferList.get(bufIndex);
|
|||
|
|
++ curBuf = bufferList.get(bufIndex);
|
|||
|
|
+ if (posInCurBuffer < MessageBufferUtil.PHYSICAL_OFFSET_POSITION) {
|
|||
|
|
+ realReadLen = Math.min(MessageBufferUtil.PHYSICAL_OFFSET_POSITION - posInCurBuffer, readLen);
|
|||
|
|
+ // read from commitLog buffer
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStream.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStream.java
|
|||
|
|
+similarity index 77%
|
|||
|
|
+rename from tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStream.java
|
|||
|
|
+rename to tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStream.java
|
|||
|
|
+index e1758ca93..9e9d5135c 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStream.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStream.java
|
|||
|
|
+@@ -15,15 +15,16 @@
|
|||
|
|
+ * limitations under the License.
|
|||
|
|
+ */
|
|||
|
|
+
|
|||
|
|
+-package org.apache.rocketmq.tieredstore.provider.inputstream;
|
|||
|
|
++package org.apache.rocketmq.tieredstore.provider.stream;
|
|||
|
|
+
|
|||
|
|
+ import java.io.IOException;
|
|||
|
|
+ import java.io.InputStream;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
++import org.apache.commons.collections.CollectionUtils;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+
|
|||
|
|
+-public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
++public class FileSegmentInputStream extends InputStream {
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * file type, can be commitlog, consume queue or indexfile now
|
|||
|
|
+@@ -33,7 +34,7 @@ public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
+ /**
|
|||
|
|
+ * hold bytebuffer
|
|||
|
|
+ */
|
|||
|
|
+- protected final List<ByteBuffer> uploadBufferList;
|
|||
|
|
++ protected final List<ByteBuffer> bufferList;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * total remaining of bytebuffer list
|
|||
|
|
+@@ -65,13 +66,13 @@ public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
+
|
|||
|
|
+ private int markReadPosInCurBuffer = -1;
|
|||
|
|
+
|
|||
|
|
+- public TieredFileSegmentInputStream(FileSegmentType fileType, List<ByteBuffer> uploadBufferList,
|
|||
|
|
+- int contentLength) {
|
|||
|
|
++ public FileSegmentInputStream(
|
|||
|
|
++ FileSegmentType fileType, List<ByteBuffer> bufferList, int contentLength) {
|
|||
|
|
+ this.fileType = fileType;
|
|||
|
|
+ this.contentLength = contentLength;
|
|||
|
|
+- this.uploadBufferList = uploadBufferList;
|
|||
|
|
+- if (uploadBufferList != null && uploadBufferList.size() > 0) {
|
|||
|
|
+- this.curBuffer = uploadBufferList.get(curReadBufferIndex);
|
|||
|
|
++ this.bufferList = bufferList;
|
|||
|
|
++ if (bufferList != null && bufferList.size() > 0) {
|
|||
|
|
++ this.curBuffer = bufferList.get(curReadBufferIndex);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -95,18 +96,34 @@ public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
+ this.readPosition = markReadPosition;
|
|||
|
|
+ this.curReadBufferIndex = markCurReadBufferIndex;
|
|||
|
|
+ this.readPosInCurBuffer = markReadPosInCurBuffer;
|
|||
|
|
+- if (this.curReadBufferIndex < uploadBufferList.size()) {
|
|||
|
|
+- this.curBuffer = uploadBufferList.get(curReadBufferIndex);
|
|||
|
|
++ if (this.curReadBufferIndex < bufferList.size()) {
|
|||
|
|
++ this.curBuffer = bufferList.get(curReadBufferIndex);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public synchronized void rewind() {
|
|||
|
|
++ this.readPosition = 0;
|
|||
|
|
++ this.curReadBufferIndex = 0;
|
|||
|
|
++ this.readPosInCurBuffer = 0;
|
|||
|
|
++ if (CollectionUtils.isNotEmpty(bufferList)) {
|
|||
|
|
++ this.curBuffer = bufferList.get(0);
|
|||
|
|
++ for (ByteBuffer buffer : bufferList) {
|
|||
|
|
++ buffer.rewind();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getContentLength() {
|
|||
|
|
++ return contentLength;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public int available() {
|
|||
|
|
+ return contentLength - readPosition;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public List<ByteBuffer> getUploadBufferList() {
|
|||
|
|
+- return uploadBufferList;
|
|||
|
|
++ public List<ByteBuffer> getBufferList() {
|
|||
|
|
++ return bufferList;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ByteBuffer getCodaBuffer() {
|
|||
|
|
+@@ -121,10 +138,10 @@ public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
+ readPosition++;
|
|||
|
|
+ if (readPosInCurBuffer >= curBuffer.remaining()) {
|
|||
|
|
+ curReadBufferIndex++;
|
|||
|
|
+- if (curReadBufferIndex >= uploadBufferList.size()) {
|
|||
|
|
++ if (curReadBufferIndex >= bufferList.size()) {
|
|||
|
|
+ return -1;
|
|||
|
|
+ }
|
|||
|
|
+- curBuffer = uploadBufferList.get(curReadBufferIndex);
|
|||
|
|
++ curBuffer = bufferList.get(curReadBufferIndex);
|
|||
|
|
+ readPosInCurBuffer = 0;
|
|||
|
|
+ }
|
|||
|
|
+ return curBuffer.get(readPosInCurBuffer++) & 0xff;
|
|||
|
|
+@@ -153,8 +170,8 @@ public class TieredFileSegmentInputStream extends InputStream {
|
|||
|
|
+ int bufIndex = curReadBufferIndex;
|
|||
|
|
+ int posInCurBuffer = readPosInCurBuffer;
|
|||
|
|
+ ByteBuffer curBuf = curBuffer;
|
|||
|
|
+- while (needRead > 0 && bufIndex < uploadBufferList.size()) {
|
|||
|
|
+- curBuf = uploadBufferList.get(bufIndex);
|
|||
|
|
++ while (needRead > 0 && bufIndex < bufferList.size()) {
|
|||
|
|
++ curBuf = bufferList.get(bufIndex);
|
|||
|
|
+ int remaining = curBuf.remaining() - posInCurBuffer;
|
|||
|
|
+ int readLen = Math.min(remaining, needRead);
|
|||
|
|
+ // read from curBuf
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStreamFactory.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStreamFactory.java
|
|||
|
|
+similarity index 54%
|
|||
|
|
+rename from tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStreamFactory.java
|
|||
|
|
+rename to tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStreamFactory.java
|
|||
|
|
+index d0c983fd4..a90baff3a 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/inputstream/TieredFileSegmentInputStreamFactory.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/stream/FileSegmentInputStreamFactory.java
|
|||
|
|
+@@ -15,30 +15,34 @@
|
|||
|
|
+ * limitations under the License.
|
|||
|
|
+ */
|
|||
|
|
+
|
|||
|
|
+-package org.apache.rocketmq.tieredstore.provider.inputstream;
|
|||
|
|
++package org.apache.rocketmq.tieredstore.provider.stream;
|
|||
|
|
+
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+
|
|||
|
|
+-public class TieredFileSegmentInputStreamFactory {
|
|||
|
|
++public class FileSegmentInputStreamFactory {
|
|||
|
|
+
|
|||
|
|
+- public static TieredFileSegmentInputStream build(FileSegmentType fileType,
|
|||
|
|
+- long startOffset, List<ByteBuffer> uploadBufferList, ByteBuffer codaBuffer, int contentLength) {
|
|||
|
|
++ public static FileSegmentInputStream build(
|
|||
|
|
++ FileSegmentType fileType, long offset, List<ByteBuffer> bufferList, ByteBuffer byteBuffer, int length) {
|
|||
|
|
++
|
|||
|
|
++ if (bufferList == null) {
|
|||
|
|
++ throw new IllegalArgumentException("bufferList is null");
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+ switch (fileType) {
|
|||
|
|
+ case COMMIT_LOG:
|
|||
|
|
+- return new TieredCommitLogInputStream(
|
|||
|
|
+- fileType, startOffset, uploadBufferList, codaBuffer, contentLength);
|
|||
|
|
++ return new CommitLogInputStream(
|
|||
|
|
++ fileType, offset, bufferList, byteBuffer, length);
|
|||
|
|
+ case CONSUME_QUEUE:
|
|||
|
|
+- return new TieredFileSegmentInputStream(fileType, uploadBufferList, contentLength);
|
|||
|
|
++ return new FileSegmentInputStream(fileType, bufferList, length);
|
|||
|
|
+ case INDEX:
|
|||
|
|
+- if (uploadBufferList.size() != 1) {
|
|||
|
|
+- throw new IllegalArgumentException("uploadBufferList size in INDEX type input stream must be 1");
|
|||
|
|
++ if (bufferList.size() != 1) {
|
|||
|
|
++ throw new IllegalArgumentException("buffer block size must be 1 when file type is IndexFile");
|
|||
|
|
+ }
|
|||
|
|
+- return new TieredFileSegmentInputStream(fileType, uploadBufferList, contentLength);
|
|||
|
|
++ return new FileSegmentInputStream(fileType, bufferList, length);
|
|||
|
|
+ default:
|
|||
|
|
+- throw new IllegalArgumentException("fileType is not supported");
|
|||
|
|
++ throw new IllegalArgumentException("file type is not supported");
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
+index 8601392e7..2451199c2 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
+@@ -130,36 +130,36 @@ public class TieredMessageStoreTest {
|
|||
|
|
+ // TieredStorageLevel.DISABLE
|
|||
|
|
+ properties.setProperty("tieredStorageLevel", "0");
|
|||
|
|
+ configuration.update(properties);
|
|||
|
|
+- Assert.assertFalse(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertFalse(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ // TieredStorageLevel.NOT_IN_DISK
|
|||
|
|
+ properties.setProperty("tieredStorageLevel", "1");
|
|||
|
|
+ configuration.update(properties);
|
|||
|
|
+ when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(false);
|
|||
|
|
+- Assert.assertTrue(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertTrue(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(true);
|
|||
|
|
+- Assert.assertFalse(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertFalse(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ // TieredStorageLevel.NOT_IN_MEM
|
|||
|
|
+ properties.setProperty("tieredStorageLevel", "2");
|
|||
|
|
+ configuration.update(properties);
|
|||
|
|
+ Mockito.when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(false);
|
|||
|
|
+ Mockito.when(nextStore.checkInMemByConsumeOffset(anyString(), anyInt(), anyLong(), anyInt())).thenReturn(true);
|
|||
|
|
+- Assert.assertTrue(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertTrue(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ Mockito.when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(true);
|
|||
|
|
+ Mockito.when(nextStore.checkInMemByConsumeOffset(anyString(), anyInt(), anyLong(), anyInt())).thenReturn(false);
|
|||
|
|
+- Assert.assertTrue(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertTrue(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ Mockito.when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(true);
|
|||
|
|
+ Mockito.when(nextStore.checkInMemByConsumeOffset(anyString(), anyInt(), anyLong(), anyInt())).thenReturn(true);
|
|||
|
|
+- Assert.assertFalse(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertFalse(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+
|
|||
|
|
+ // TieredStorageLevel.FORCE
|
|||
|
|
+ properties.setProperty("tieredStorageLevel", "3");
|
|||
|
|
+ configuration.update(properties);
|
|||
|
|
+- Assert.assertTrue(store.viaTieredStorage(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
++ Assert.assertTrue(store.fetchFromCurrentStore(mq.getTopic(), mq.getQueueId(), 0));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
+index cc39cfbfc..7a4d05969 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
+@@ -24,6 +24,7 @@ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.TieredStoreTestUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.TieredMessageStoreConfig;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.common.TieredStoreExecutor;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.metadata.FileSegmentMetadata;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.metadata.TieredMetadataStore;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.provider.TieredFileSegment;
|
|||
|
|
+@@ -55,6 +56,7 @@ public class TieredFlatFileTest {
|
|||
|
|
+ public void tearDown() throws IOException {
|
|||
|
|
+ TieredStoreTestUtil.destroyMetadataStore();
|
|||
|
|
+ TieredStoreTestUtil.destroyTempDir(storePath);
|
|||
|
|
++ TieredStoreExecutor.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private List<FileSegmentMetadata> getSegmentMetadataList(TieredMetadataStore metadataStore) {
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+index 262d6645b..2da72bc7a 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredIndexFileTest.java
|
|||
|
|
+@@ -87,5 +87,7 @@ public class TieredIndexFileTest {
|
|||
|
|
+
|
|||
|
|
+ indexList = indexFile.queryAsync(mq.getTopic(), "key1", 1000, 1200).join();
|
|||
|
|
+ Assert.assertEquals(1, indexList.size());
|
|||
|
|
++
|
|||
|
|
++ indexFile.destroy();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockTieredFileSegmentInputStream.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockFileSegmentInputStream.java
|
|||
|
|
+similarity index 82%
|
|||
|
|
+rename from tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockTieredFileSegmentInputStream.java
|
|||
|
|
+rename to tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockFileSegmentInputStream.java
|
|||
|
|
+index a6566b7de..3bbe41dd4 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockTieredFileSegmentInputStream.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/MockFileSegmentInputStream.java
|
|||
|
|
+@@ -20,13 +20,13 @@ package org.apache.rocketmq.tieredstore.provider;
|
|||
|
|
+ import java.io.InputStream;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
+
|
|||
|
|
+-public class MockTieredFileSegmentInputStream extends TieredFileSegmentInputStream {
|
|||
|
|
++public class MockFileSegmentInputStream extends FileSegmentInputStream {
|
|||
|
|
+
|
|||
|
|
+ private final InputStream inputStream;
|
|||
|
|
+
|
|||
|
|
+- public MockTieredFileSegmentInputStream(InputStream inputStream) {
|
|||
|
|
++ public MockFileSegmentInputStream(InputStream inputStream) {
|
|||
|
|
+ super(null, null, Integer.MAX_VALUE);
|
|||
|
|
+ this.inputStream = inputStream;
|
|||
|
|
+ }
|
|||
|
|
+@@ -43,7 +43,7 @@ public class MockTieredFileSegmentInputStream extends TieredFileSegmentInputStre
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public List<ByteBuffer> getUploadBufferList() {
|
|||
|
|
++ public List<ByteBuffer> getBufferList() {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentInputStreamTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentInputStreamTest.java
|
|||
|
|
+index a2554ba3d..743d9182c 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentInputStreamTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentInputStreamTest.java
|
|||
|
|
+@@ -28,8 +28,8 @@ import java.util.Random;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredCommitLog;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredConsumeQueue;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStreamFactory;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStreamFactory;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtilTest;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+@@ -57,7 +57,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ bufferSize += byteBuffer.remaining();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // build expected byte buffer for verifying the TieredFileSegmentInputStream
|
|||
|
|
++ // build expected byte buffer for verifying the FileSegmentInputStream
|
|||
|
|
+ ByteBuffer expectedByteBuffer = ByteBuffer.allocate(bufferSize);
|
|||
|
|
+ for (ByteBuffer byteBuffer : uploadBufferList) {
|
|||
|
|
+ expectedByteBuffer.put(byteBuffer);
|
|||
|
|
+@@ -74,7 +74,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ int[] batchReadSizeTestSet = {
|
|||
|
|
+ MessageBufferUtil.PHYSICAL_OFFSET_POSITION - 1, MessageBufferUtil.PHYSICAL_OFFSET_POSITION, MessageBufferUtil.PHYSICAL_OFFSET_POSITION + 1, MSG_LEN - 1, MSG_LEN, MSG_LEN + 1
|
|||
|
|
+ };
|
|||
|
|
+- verifyReadAndReset(expectedByteBuffer, () -> TieredFileSegmentInputStreamFactory.build(
|
|||
|
|
++ verifyReadAndReset(expectedByteBuffer, () -> FileSegmentInputStreamFactory.build(
|
|||
|
|
+ FileSegmentType.COMMIT_LOG, COMMIT_LOG_START_OFFSET, uploadBufferList, null, finalBufferSize), finalBufferSize, batchReadSizeTestSet);
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+@@ -98,7 +98,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ int codaBufferSize = codaBuffer.remaining();
|
|||
|
|
+ bufferSize += codaBufferSize;
|
|||
|
|
+
|
|||
|
|
+- // build expected byte buffer for verifying the TieredFileSegmentInputStream
|
|||
|
|
++ // build expected byte buffer for verifying the FileSegmentInputStream
|
|||
|
|
+ ByteBuffer expectedByteBuffer = ByteBuffer.allocate(bufferSize);
|
|||
|
|
+ for (ByteBuffer byteBuffer : uploadBufferList) {
|
|||
|
|
+ expectedByteBuffer.put(byteBuffer);
|
|||
|
|
+@@ -119,7 +119,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ MSG_LEN - 1, MSG_LEN, MSG_LEN + 1,
|
|||
|
|
+ bufferSize - 1, bufferSize, bufferSize + 1
|
|||
|
|
+ };
|
|||
|
|
+- verifyReadAndReset(expectedByteBuffer, () -> TieredFileSegmentInputStreamFactory.build(
|
|||
|
|
++ verifyReadAndReset(expectedByteBuffer, () -> FileSegmentInputStreamFactory.build(
|
|||
|
|
+ FileSegmentType.COMMIT_LOG, COMMIT_LOG_START_OFFSET, uploadBufferList, codaBuffer, finalBufferSize), finalBufferSize, batchReadSizeTestSet);
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+@@ -134,7 +134,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ bufferSize += byteBuffer.remaining();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // build expected byte buffer for verifying the TieredFileSegmentInputStream
|
|||
|
|
++ // build expected byte buffer for verifying the FileSegmentInputStream
|
|||
|
|
+ ByteBuffer expectedByteBuffer = ByteBuffer.allocate(bufferSize);
|
|||
|
|
+ for (ByteBuffer byteBuffer : uploadBufferList) {
|
|||
|
|
+ expectedByteBuffer.put(byteBuffer);
|
|||
|
|
+@@ -143,7 +143,7 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+
|
|||
|
|
+ int finalBufferSize = bufferSize;
|
|||
|
|
+ int[] batchReadSizeTestSet = {TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE - 1, TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE, TieredConsumeQueue.CONSUME_QUEUE_STORE_UNIT_SIZE + 1};
|
|||
|
|
+- verifyReadAndReset(expectedByteBuffer, () -> TieredFileSegmentInputStreamFactory.build(
|
|||
|
|
++ verifyReadAndReset(expectedByteBuffer, () -> FileSegmentInputStreamFactory.build(
|
|||
|
|
+ FileSegmentType.CONSUME_QUEUE, COMMIT_LOG_START_OFFSET, uploadBufferList, null, finalBufferSize), bufferSize, batchReadSizeTestSet);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -156,16 +156,16 @@ public class TieredFileSegmentInputStreamTest {
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ List<ByteBuffer> uploadBufferList = Arrays.asList(byteBuffer);
|
|||
|
|
+
|
|||
|
|
+- // build expected byte buffer for verifying the TieredFileSegmentInputStream
|
|||
|
|
++ // build expected byte buffer for verifying the FileSegmentInputStream
|
|||
|
|
+ ByteBuffer expectedByteBuffer = byteBuffer.slice();
|
|||
|
|
+
|
|||
|
|
+- verifyReadAndReset(expectedByteBuffer, () -> TieredFileSegmentInputStreamFactory.build(
|
|||
|
|
++ verifyReadAndReset(expectedByteBuffer, () -> FileSegmentInputStreamFactory.build(
|
|||
|
|
+ FileSegmentType.INDEX, COMMIT_LOG_START_OFFSET, uploadBufferList, null, byteBuffer.limit()), byteBuffer.limit(), new int[] {23, 24, 25});
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private void verifyReadAndReset(ByteBuffer expectedByteBuffer, Supplier<TieredFileSegmentInputStream> constructor,
|
|||
|
|
++ private void verifyReadAndReset(ByteBuffer expectedByteBuffer, Supplier<FileSegmentInputStream> constructor,
|
|||
|
|
+ int bufferSize, int[] readBatchSizeTestSet) {
|
|||
|
|
+- TieredFileSegmentInputStream inputStream = constructor.get();
|
|||
|
|
++ FileSegmentInputStream inputStream = constructor.get();
|
|||
|
|
+
|
|||
|
|
+ // verify
|
|||
|
|
+ verifyInputStream(inputStream, expectedByteBuffer);
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentTest.java
|
|||
|
|
+index 4cd83e0d2..a655710a5 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredFileSegmentTest.java
|
|||
|
|
+@@ -116,13 +116,22 @@ public class TieredFileSegmentTest {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+- public void testCommitFailed() {
|
|||
|
|
++ public void testCommitFailedThenSuccess() {
|
|||
|
|
+ long startTime = System.currentTimeMillis();
|
|||
|
|
+ MemoryFileSegment segment = (MemoryFileSegment) createFileSegment(FileSegmentType.COMMIT_LOG);
|
|||
|
|
+ long lastSize = segment.getSize();
|
|||
|
|
+- segment.append(MessageBufferUtilTest.buildMockedMessageBuffer(), 0);
|
|||
|
|
+- segment.append(MessageBufferUtilTest.buildMockedMessageBuffer(), 0);
|
|||
|
|
+-
|
|||
|
|
++ segment.setCheckSize(false);
|
|||
|
|
++ segment.initPosition(lastSize);
|
|||
|
|
++ segment.setSize((int) lastSize);
|
|||
|
|
++
|
|||
|
|
++ ByteBuffer buffer1 = MessageBufferUtilTest.buildMockedMessageBuffer().putLong(
|
|||
|
|
++ MessageBufferUtil.PHYSICAL_OFFSET_POSITION, baseOffset + lastSize);
|
|||
|
|
++ ByteBuffer buffer2 = MessageBufferUtilTest.buildMockedMessageBuffer().putLong(
|
|||
|
|
++ MessageBufferUtil.PHYSICAL_OFFSET_POSITION, baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
++ segment.append(buffer1, 0);
|
|||
|
|
++ segment.append(buffer2, 0);
|
|||
|
|
++
|
|||
|
|
++ // Mock new message arrive
|
|||
|
|
+ segment.blocker = new CompletableFuture<>();
|
|||
|
|
+ new Thread(() -> {
|
|||
|
|
+ try {
|
|||
|
|
+@@ -131,20 +140,88 @@ public class TieredFileSegmentTest {
|
|||
|
|
+ Assert.fail(e.getMessage());
|
|||
|
|
+ }
|
|||
|
|
+ ByteBuffer buffer = MessageBufferUtilTest.buildMockedMessageBuffer();
|
|||
|
|
++ buffer.putLong(MessageBufferUtil.PHYSICAL_OFFSET_POSITION, MessageBufferUtilTest.MSG_LEN * 2);
|
|||
|
|
+ buffer.putLong(MessageBufferUtil.STORE_TIMESTAMP_POSITION, startTime);
|
|||
|
|
+ segment.append(buffer, 0);
|
|||
|
|
+ segment.blocker.complete(false);
|
|||
|
|
+ }).start();
|
|||
|
|
+
|
|||
|
|
++ // Commit failed
|
|||
|
|
+ segment.commit();
|
|||
|
|
+ segment.blocker.join();
|
|||
|
|
++ segment.blocker = null;
|
|||
|
|
++
|
|||
|
|
++ // Copy data and assume commit success
|
|||
|
|
++ segment.getMemStore().put(buffer1);
|
|||
|
|
++ segment.getMemStore().put(buffer2);
|
|||
|
|
++ segment.setSize((int) (lastSize + MessageBufferUtilTest.MSG_LEN * 2));
|
|||
|
|
+
|
|||
|
|
+- segment.blocker = new CompletableFuture<>();
|
|||
|
|
+- segment.blocker.complete(true);
|
|||
|
|
+ segment.commit();
|
|||
|
|
++ Assert.assertEquals(lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getCommitPosition());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getCommitOffset());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getMaxOffset());
|
|||
|
|
++
|
|||
|
|
++ ByteBuffer msg1 = segment.read(lastSize, MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize, MessageBufferUtil.getCommitLogOffset(msg1));
|
|||
|
|
++
|
|||
|
|
++ ByteBuffer msg2 = segment.read(lastSize + MessageBufferUtilTest.MSG_LEN, MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN, MessageBufferUtil.getCommitLogOffset(msg2));
|
|||
|
|
++
|
|||
|
|
++ ByteBuffer msg3 = segment.read(lastSize + MessageBufferUtilTest.MSG_LEN * 2, MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 2, MessageBufferUtil.getCommitLogOffset(msg3));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testCommitFailed3Times() {
|
|||
|
|
++ long startTime = System.currentTimeMillis();
|
|||
|
|
++ MemoryFileSegment segment = (MemoryFileSegment) createFileSegment(FileSegmentType.COMMIT_LOG);
|
|||
|
|
++ long lastSize = segment.getSize();
|
|||
|
|
++ segment.setCheckSize(false);
|
|||
|
|
++ segment.initPosition(lastSize);
|
|||
|
|
++ segment.setSize((int) lastSize);
|
|||
|
|
++
|
|||
|
|
++ ByteBuffer buffer1 = MessageBufferUtilTest.buildMockedMessageBuffer().putLong(
|
|||
|
|
++ MessageBufferUtil.PHYSICAL_OFFSET_POSITION, baseOffset + lastSize);
|
|||
|
|
++ ByteBuffer buffer2 = MessageBufferUtilTest.buildMockedMessageBuffer().putLong(
|
|||
|
|
++ MessageBufferUtil.PHYSICAL_OFFSET_POSITION, baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
++ segment.append(buffer1, 0);
|
|||
|
|
++ segment.append(buffer2, 0);
|
|||
|
|
++
|
|||
|
|
++ // Mock new message arrive
|
|||
|
|
++ segment.blocker = new CompletableFuture<>();
|
|||
|
|
++ new Thread(() -> {
|
|||
|
|
++ try {
|
|||
|
|
++ Thread.sleep(3000);
|
|||
|
|
++ } catch (InterruptedException e) {
|
|||
|
|
++ Assert.fail(e.getMessage());
|
|||
|
|
++ }
|
|||
|
|
++ ByteBuffer buffer = MessageBufferUtilTest.buildMockedMessageBuffer();
|
|||
|
|
++ buffer.putLong(MessageBufferUtil.PHYSICAL_OFFSET_POSITION, MessageBufferUtilTest.MSG_LEN * 2);
|
|||
|
|
++ buffer.putLong(MessageBufferUtil.STORE_TIMESTAMP_POSITION, startTime);
|
|||
|
|
++ segment.append(buffer, 0);
|
|||
|
|
++ segment.blocker.complete(false);
|
|||
|
|
++ }).start();
|
|||
|
|
++
|
|||
|
|
++ for (int i = 0; i < 3; i++) {
|
|||
|
|
++ segment.commit();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ Assert.assertEquals(lastSize, segment.getCommitPosition());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize, segment.getCommitOffset());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getMaxOffset());
|
|||
|
|
++
|
|||
|
|
++ segment.blocker.join();
|
|||
|
|
++ segment.blocker = null;
|
|||
|
|
+
|
|||
|
|
++ segment.commit();
|
|||
|
|
++ Assert.assertEquals(lastSize + MessageBufferUtilTest.MSG_LEN * 2, segment.getCommitPosition());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 2, segment.getCommitOffset());
|
|||
|
|
+ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getMaxOffset());
|
|||
|
|
++
|
|||
|
|
++ segment.commit();
|
|||
|
|
++ Assert.assertEquals(lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getCommitPosition());
|
|||
|
|
+ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getCommitOffset());
|
|||
|
|
++ Assert.assertEquals(baseOffset + lastSize + MessageBufferUtilTest.MSG_LEN * 3, segment.getMaxOffset());
|
|||
|
|
+
|
|||
|
|
+ ByteBuffer msg1 = segment.read(lastSize, MessageBufferUtilTest.MSG_LEN);
|
|||
|
|
+ Assert.assertEquals(baseOffset + lastSize, MessageBufferUtil.getCommitLogOffset(msg1));
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegment.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegment.java
|
|||
|
|
+index cb155cf8f..80ad41f68 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegment.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegment.java
|
|||
|
|
+@@ -23,7 +23,7 @@ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.TieredMessageStoreConfig;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.provider.TieredFileSegment;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+
|
|||
|
|
+@@ -33,6 +33,8 @@ public class MemoryFileSegment extends TieredFileSegment {
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<Boolean> blocker;
|
|||
|
|
+
|
|||
|
|
++ protected int size = 0;
|
|||
|
|
++
|
|||
|
|
+ protected boolean checkSize = true;
|
|||
|
|
+
|
|||
|
|
+ public MemoryFileSegment(FileSegmentType fileType, MessageQueue messageQueue, long baseOffset,
|
|||
|
|
+@@ -56,6 +58,18 @@ public class MemoryFileSegment extends TieredFileSegment {
|
|||
|
|
+ memStore.position((int) getSize());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isCheckSize() {
|
|||
|
|
++ return checkSize;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setCheckSize(boolean checkSize) {
|
|||
|
|
++ this.checkSize = checkSize;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public ByteBuffer getMemStore() {
|
|||
|
|
++ return memStore;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public String getPath() {
|
|||
|
|
+ return filePath;
|
|||
|
|
+@@ -66,7 +80,11 @@ public class MemoryFileSegment extends TieredFileSegment {
|
|||
|
|
+ if (checkSize) {
|
|||
|
|
+ return 1000;
|
|||
|
|
+ }
|
|||
|
|
+- return 0;
|
|||
|
|
++ return size;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setSize(int size) {
|
|||
|
|
++ this.size = size;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -85,11 +103,11 @@ public class MemoryFileSegment extends TieredFileSegment {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<Boolean> commit0(
|
|||
|
|
+- TieredFileSegmentInputStream inputStream, long position, int length, boolean append) {
|
|||
|
|
++ FileSegmentInputStream inputStream, long position, int length, boolean append) {
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ if (blocker != null && !blocker.get()) {
|
|||
|
|
+- throw new IllegalStateException();
|
|||
|
|
++ throw new IllegalStateException("Commit Exception for Memory Test");
|
|||
|
|
+ }
|
|||
|
|
+ } catch (InterruptedException | ExecutionException e) {
|
|||
|
|
+ Assert.fail(e.getMessage());
|
|||
|
|
+@@ -98,7 +116,6 @@ public class MemoryFileSegment extends TieredFileSegment {
|
|||
|
|
+ Assert.assertTrue(!checkSize || position >= getSize());
|
|||
|
|
+
|
|||
|
|
+ byte[] buffer = new byte[1024];
|
|||
|
|
+-
|
|||
|
|
+ int startPos = memStore.position();
|
|||
|
|
+ try {
|
|||
|
|
+ int len;
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegmentWithoutCheck.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegmentWithoutCheck.java
|
|||
|
|
+index 8ac330b37..630fd2223 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegmentWithoutCheck.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/memory/MemoryFileSegmentWithoutCheck.java
|
|||
|
|
+@@ -22,7 +22,7 @@ import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.TieredMessageStoreConfig;
|
|||
|
|
+-import org.apache.rocketmq.tieredstore.provider.inputstream.TieredFileSegmentInputStream;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.stream.FileSegmentInputStream;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+
|
|||
|
|
+@@ -46,7 +46,7 @@ public class MemoryFileSegmentWithoutCheck extends MemoryFileSegment {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public CompletableFuture<Boolean> commit0(TieredFileSegmentInputStream inputStream, long position, int length,
|
|||
|
|
++ public CompletableFuture<Boolean> commit0(FileSegmentInputStream inputStream, long position, int length,
|
|||
|
|
+ boolean append) {
|
|||
|
|
+ try {
|
|||
|
|
+ if (blocker != null && !blocker.get()) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From d000ef947d7c99918ceba0fa451c1e29fd84ba07 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yuz10 <845238369@qq.com>
|
|||
|
|
+Date: Thu, 31 Aug 2023 09:41:33 +0800
|
|||
|
|
+Subject: [PATCH 3/7] [ISSUE #7283] Incorrect dledger commitlog min offset
|
|||
|
|
+ after mappedFile re delete failed (#7284)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/store/dledger/DLedgerCommitLog.java | 7 ++++++-
|
|||
|
|
+ 1 file changed, 6 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
+index ec5e86d70..d5f6acdc0 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
+@@ -162,7 +162,12 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
+ if (!mappedFileQueue.getMappedFiles().isEmpty()) {
|
|||
|
|
+ return mappedFileQueue.getMinOffset();
|
|||
|
|
+ }
|
|||
|
|
+- return dLedgerFileList.getMinOffset();
|
|||
|
|
++ for (MmapFile file : dLedgerFileList.getMappedFiles()) {
|
|||
|
|
++ if (file.isAvailable()) {
|
|||
|
|
++ return file.getFileFromOffset() + file.getStartPosition();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From f82718ae3b77a16b553c03f672dc971a2d5d48fa Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: cnScarb <jjhfen00@163.com>
|
|||
|
|
+Date: Thu, 31 Aug 2023 15:50:10 +0800
|
|||
|
|
+Subject: [PATCH 4/7] [ISSUE #7208] fix: when deleting topic also delete its
|
|||
|
|
+ pop retry topic (#7209)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../processor/AdminBrokerProcessor.java | 24 ++++++++++---
|
|||
|
|
+ .../processor/AdminBrokerProcessorTest.java | 36 +++++++++++++++++++
|
|||
|
|
+ 2 files changed, 55 insertions(+), 5 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+index bbddcec2d..8fbcd3c94 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+@@ -51,6 +51,7 @@ import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager;
|
|||
|
|
+ import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageUtil;
|
|||
|
|
+ import org.apache.rocketmq.common.AclConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
++import org.apache.rocketmq.common.KeyBuilder;
|
|||
|
|
+ import org.apache.rocketmq.common.LockCallback;
|
|||
|
|
+ import org.apache.rocketmq.common.MQVersion;
|
|||
|
|
+ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+@@ -542,16 +543,29 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- this.brokerController.getTopicConfigManager().deleteTopicConfig(requestHeader.getTopic());
|
|||
|
|
+- this.brokerController.getTopicQueueMappingManager().delete(requestHeader.getTopic());
|
|||
|
|
+- this.brokerController.getConsumerOffsetManager().cleanOffsetByTopic(requestHeader.getTopic());
|
|||
|
|
+- this.brokerController.getPopInflightMessageCounter().clearInFlightMessageNumByTopicName(requestHeader.getTopic());
|
|||
|
|
+- this.brokerController.getMessageStore().deleteTopics(Sets.newHashSet(requestHeader.getTopic()));
|
|||
|
|
++ final Set<String> groups = this.brokerController.getConsumerOffsetManager().whichGroupByTopic(topic);
|
|||
|
|
++ // delete pop retry topics first
|
|||
|
|
++ for (String group : groups) {
|
|||
|
|
++ final String popRetryTopic = KeyBuilder.buildPopRetryTopic(topic, group);
|
|||
|
|
++ if (brokerController.getTopicConfigManager().selectTopicConfig(popRetryTopic) != null) {
|
|||
|
|
++ deleteTopicInBroker(popRetryTopic);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ // delete topic
|
|||
|
|
++ deleteTopicInBroker(topic);
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ response.setRemark(null);
|
|||
|
|
+ return response;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ private void deleteTopicInBroker(String topic) {
|
|||
|
|
++ this.brokerController.getTopicConfigManager().deleteTopicConfig(topic);
|
|||
|
|
++ this.brokerController.getTopicQueueMappingManager().delete(topic);
|
|||
|
|
++ this.brokerController.getConsumerOffsetManager().cleanOffsetByTopic(topic);
|
|||
|
|
++ this.brokerController.getPopInflightMessageCounter().clearInFlightMessageNumByTopicName(topic);
|
|||
|
|
++ this.brokerController.getMessageStore().deleteTopics(Sets.newHashSet(topic));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private synchronized RemotingCommand updateAndCreateAccessConfig(ChannelHandlerContext ctx,
|
|||
|
|
+ RemotingCommand request) throws RemotingCommandException {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
+index d33a217f7..9d17011b6 100644
|
|||
|
|
+--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
++++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
+@@ -29,6 +29,7 @@ import java.util.HashMap;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Properties;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
++import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.atomic.LongAdder;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+@@ -41,6 +42,7 @@ import org.apache.rocketmq.broker.topic.RocksDBTopicConfigManager;
|
|||
|
|
+ import org.apache.rocketmq.broker.topic.TopicConfigManager;
|
|||
|
|
+ import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
++import org.apache.rocketmq.common.KeyBuilder;
|
|||
|
|
+ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicFilterType;
|
|||
|
|
+@@ -90,8 +92,11 @@ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyInt;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.anySet;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.Mockito.mock;
|
|||
|
|
++import static org.mockito.Mockito.times;
|
|||
|
|
++import static org.mockito.Mockito.verify;
|
|||
|
|
+ import static org.mockito.Mockito.when;
|
|||
|
|
+
|
|||
|
|
+ @RunWith(MockitoJUnitRunner.class)
|
|||
|
|
+@@ -321,6 +326,37 @@ public class AdminBrokerProcessorTest {
|
|||
|
|
+ "please execute it from master broker.");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testDeleteWithPopRetryTopic() throws Exception {
|
|||
|
|
++ String topic = "topicA";
|
|||
|
|
++ String anotherTopic = "another_topicA";
|
|||
|
|
++
|
|||
|
|
++ topicConfigManager = mock(TopicConfigManager.class);
|
|||
|
|
++ when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager);
|
|||
|
|
++ final ConcurrentHashMap<String, TopicConfig> topicConfigTable = new ConcurrentHashMap<>();
|
|||
|
|
++ topicConfigTable.put(topic, new TopicConfig());
|
|||
|
|
++ topicConfigTable.put(KeyBuilder.buildPopRetryTopic(topic, "cid1"), new TopicConfig());
|
|||
|
|
++
|
|||
|
|
++ topicConfigTable.put(anotherTopic, new TopicConfig());
|
|||
|
|
++ topicConfigTable.put(KeyBuilder.buildPopRetryTopic(anotherTopic, "cid2"), new TopicConfig());
|
|||
|
|
++ when(topicConfigManager.getTopicConfigTable()).thenReturn(topicConfigTable);
|
|||
|
|
++ when(topicConfigManager.selectTopicConfig(anyString())).thenAnswer(invocation -> {
|
|||
|
|
++ final String selectTopic = invocation.getArgument(0);
|
|||
|
|
++ return topicConfigManager.getTopicConfigTable().get(selectTopic);
|
|||
|
|
++ });
|
|||
|
|
++
|
|||
|
|
++ when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager);
|
|||
|
|
++ when(consumerOffsetManager.whichGroupByTopic(topic)).thenReturn(Sets.newHashSet("cid1"));
|
|||
|
|
++
|
|||
|
|
++ RemotingCommand request = buildDeleteTopicRequest(topic);
|
|||
|
|
++ RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request);
|
|||
|
|
++ assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
|
|||
|
|
++
|
|||
|
|
++ verify(topicConfigManager).deleteTopicConfig(topic);
|
|||
|
|
++ verify(topicConfigManager).deleteTopicConfig(KeyBuilder.buildPopRetryTopic(topic, "cid1"));
|
|||
|
|
++ verify(messageStore, times(2)).deleteTopics(anySet());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetAllTopicConfigInRocksdb() throws Exception {
|
|||
|
|
+ if (notToBeExecuted()) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 31d10385d1616445478104ce9ef463a8c4852ba2 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Mon, 4 Sep 2023 14:09:32 +0800
|
|||
|
|
+Subject: [PATCH 5/7] [ISSUE #7289] Fixed asynchronous send backpressure
|
|||
|
|
+ capability
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: guyinyou <guyinyou.gyy@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../impl/producer/DefaultMQProducerImpl.java | 77 +++++++++++++------
|
|||
|
|
+ 1 file changed, 53 insertions(+), 24 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+index bbbb17b07..2d6b83ac2 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+@@ -547,6 +547,8 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ @Deprecated
|
|||
|
|
+ public void send(final Message msg, final SendCallback sendCallback, final long timeout)
|
|||
|
|
+ throws MQClientException, RemotingException, InterruptedException {
|
|||
|
|
++ BackpressureSendCallBack newCallBack = new BackpressureSendCallBack(sendCallback);
|
|||
|
|
++
|
|||
|
|
+ final long beginStartTime = System.currentTimeMillis();
|
|||
|
|
+ Runnable runnable = new Runnable() {
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -554,20 +556,53 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+ if (timeout > costTime) {
|
|||
|
|
+ try {
|
|||
|
|
+- sendDefaultImpl(msg, CommunicationMode.ASYNC, sendCallback, timeout - costTime);
|
|||
|
|
++ sendDefaultImpl(msg, CommunicationMode.ASYNC, newCallBack, timeout - costTime);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- sendCallback.onException(e);
|
|||
|
|
++ newCallBack.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+- sendCallback.onException(
|
|||
|
|
++ newCallBack.onException(
|
|||
|
|
+ new RemotingTooMuchRequestException("DEFAULT ASYNC send call timeout"));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ };
|
|||
|
|
+- executeAsyncMessageSend(runnable, msg, sendCallback, timeout, beginStartTime);
|
|||
|
|
++ executeAsyncMessageSend(runnable, msg, newCallBack, timeout, beginStartTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void executeAsyncMessageSend(Runnable runnable, final Message msg, final SendCallback sendCallback,
|
|||
|
|
++ class BackpressureSendCallBack implements SendCallback {
|
|||
|
|
++ public boolean isSemaphoreAsyncSizeAquired = false;
|
|||
|
|
++ public boolean isSemaphoreAsyncNumAquired = false;
|
|||
|
|
++ public int msgLen;
|
|||
|
|
++ private final SendCallback sendCallback;
|
|||
|
|
++
|
|||
|
|
++ public BackpressureSendCallBack(final SendCallback sendCallback) {
|
|||
|
|
++ this.sendCallback = sendCallback;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onSuccess(SendResult sendResult) {
|
|||
|
|
++ if (isSemaphoreAsyncSizeAquired) {
|
|||
|
|
++ semaphoreAsyncSendSize.release(msgLen);
|
|||
|
|
++ }
|
|||
|
|
++ if (isSemaphoreAsyncNumAquired) {
|
|||
|
|
++ semaphoreAsyncSendNum.release();
|
|||
|
|
++ }
|
|||
|
|
++ sendCallback.onSuccess(sendResult);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onException(Throwable e) {
|
|||
|
|
++ if (isSemaphoreAsyncSizeAquired) {
|
|||
|
|
++ semaphoreAsyncSendSize.release(msgLen);
|
|||
|
|
++ }
|
|||
|
|
++ if (isSemaphoreAsyncNumAquired) {
|
|||
|
|
++ semaphoreAsyncSendNum.release();
|
|||
|
|
++ }
|
|||
|
|
++ sendCallback.onException(e);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void executeAsyncMessageSend(Runnable runnable, final Message msg, final BackpressureSendCallBack sendCallback,
|
|||
|
|
+ final long timeout, final long beginStartTime)
|
|||
|
|
+ throws MQClientException, InterruptedException {
|
|||
|
|
+ ExecutorService executor = this.getAsyncSenderExecutor();
|
|||
|
|
+@@ -595,7 +630,9 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
++ sendCallback.isSemaphoreAsyncSizeAquired = isSemaphoreAsyncSizeAquired;
|
|||
|
|
++ sendCallback.isSemaphoreAsyncNumAquired = isSemaphoreAsyncNumAquired;
|
|||
|
|
++ sendCallback.msgLen = msgLen;
|
|||
|
|
+ executor.submit(runnable);
|
|||
|
|
+ } catch (RejectedExecutionException e) {
|
|||
|
|
+ if (isEnableBackpressureForAsyncMode) {
|
|||
|
|
+@@ -603,15 +640,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ } else {
|
|||
|
|
+ throw new MQClientException("executor rejected ", e);
|
|||
|
|
+ }
|
|||
|
|
+- } finally {
|
|||
|
|
+- if (isSemaphoreAsyncSizeAquired) {
|
|||
|
|
+- semaphoreAsyncSendSize.release(msgLen);
|
|||
|
|
+- }
|
|||
|
|
+- if (isSemaphoreAsyncNumAquired) {
|
|||
|
|
+- semaphoreAsyncSendNum.release();
|
|||
|
|
+- }
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MessageQueue invokeMessageQueueSelector(Message msg, MessageQueueSelector selector, Object arg,
|
|||
|
|
+@@ -1188,7 +1217,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ @Deprecated
|
|||
|
|
+ public void send(final Message msg, final MessageQueue mq, final SendCallback sendCallback, final long timeout)
|
|||
|
|
+ throws MQClientException, RemotingException, InterruptedException {
|
|||
|
|
+-
|
|||
|
|
++ BackpressureSendCallBack newCallBack = new BackpressureSendCallBack(sendCallback);
|
|||
|
|
+ final long beginStartTime = System.currentTimeMillis();
|
|||
|
|
+ Runnable runnable = new Runnable() {
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -1203,22 +1232,22 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+ if (timeout > costTime) {
|
|||
|
|
+ try {
|
|||
|
|
+- sendKernelImpl(msg, mq, CommunicationMode.ASYNC, sendCallback, null,
|
|||
|
|
++ sendKernelImpl(msg, mq, CommunicationMode.ASYNC, newCallBack, null,
|
|||
|
|
+ timeout - costTime);
|
|||
|
|
+ } catch (MQBrokerException e) {
|
|||
|
|
+ throw new MQClientException("unknown exception", e);
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+- sendCallback.onException(new RemotingTooMuchRequestException("call timeout"));
|
|||
|
|
++ newCallBack.onException(new RemotingTooMuchRequestException("call timeout"));
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- sendCallback.onException(e);
|
|||
|
|
++ newCallBack.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ };
|
|||
|
|
+
|
|||
|
|
+- executeAsyncMessageSend(runnable, msg, sendCallback, timeout, beginStartTime);
|
|||
|
|
++ executeAsyncMessageSend(runnable, msg, newCallBack, timeout, beginStartTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+@@ -1315,7 +1344,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ public void send(final Message msg, final MessageQueueSelector selector, final Object arg,
|
|||
|
|
+ final SendCallback sendCallback, final long timeout)
|
|||
|
|
+ throws MQClientException, RemotingException, InterruptedException {
|
|||
|
|
+-
|
|||
|
|
++ BackpressureSendCallBack newCallBack = new BackpressureSendCallBack(sendCallback);
|
|||
|
|
+ final long beginStartTime = System.currentTimeMillis();
|
|||
|
|
+ Runnable runnable = new Runnable() {
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -1324,21 +1353,21 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ if (timeout > costTime) {
|
|||
|
|
+ try {
|
|||
|
|
+ try {
|
|||
|
|
+- sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, sendCallback,
|
|||
|
|
++ sendSelectImpl(msg, selector, arg, CommunicationMode.ASYNC, newCallBack,
|
|||
|
|
+ timeout - costTime);
|
|||
|
|
+ } catch (MQBrokerException e) {
|
|||
|
|
+ throw new MQClientException("unknown exception", e);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- sendCallback.onException(e);
|
|||
|
|
++ newCallBack.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+- sendCallback.onException(new RemotingTooMuchRequestException("call timeout"));
|
|||
|
|
++ newCallBack.onException(new RemotingTooMuchRequestException("call timeout"));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ };
|
|||
|
|
+- executeAsyncMessageSend(runnable, msg, sendCallback, timeout, beginStartTime);
|
|||
|
|
++ executeAsyncMessageSend(runnable, msg, newCallBack, timeout, beginStartTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From d67b9d64cbd53824798af57ba18770e0fcefa37a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yuz10 <845238369@qq.com>
|
|||
|
|
+Date: Wed, 6 Sep 2023 14:07:23 +0800
|
|||
|
|
+Subject: [PATCH 6/7] [ISSUE #7302] Fix singleTopicRegister code deleted in
|
|||
|
|
+ merge
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/broker/topic/TopicConfigManager.java | 6 +++++-
|
|||
|
|
+ 1 file changed, 5 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+index 1c3b9711f..4e3c1736c 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+@@ -330,7 +330,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ log.error("createTopicIfAbsent ", e);
|
|||
|
|
+ }
|
|||
|
|
+ if (createNew && register) {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ return getTopicConfig(topicConfig.getTopicName());
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 37017dbaec5c521fd529ef4aecf3658092884f84 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Wed, 6 Sep 2023 15:23:15 +0800
|
|||
|
|
+Subject: [PATCH 7/7] [ISSUE #7305] Fix metrics and transactional module not
|
|||
|
|
+ shutdown while broker offline cause coredump(#7307)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../java/org/apache/rocketmq/broker/BrokerController.java | 8 ++++++++
|
|||
|
|
+ .../queue/TransactionalMessageServiceImpl.java | 4 +++-
|
|||
|
|
+ 2 files changed, 11 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index e8f943702..6aba70cb2 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1302,6 +1302,10 @@ public class BrokerController {
|
|||
|
|
+ this.fastRemotingServer.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ if (this.brokerMetricsManager != null) {
|
|||
|
|
++ this.brokerMetricsManager.shutdown();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ if (this.brokerStatsManager != null) {
|
|||
|
|
+ this.brokerStatsManager.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+@@ -1324,6 +1328,10 @@ public class BrokerController {
|
|||
|
|
+ this.ackMessageProcessor.shutdownPopReviveService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ if (this.transactionalMessageService != null) {
|
|||
|
|
++ this.transactionalMessageService.close();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ if (this.notificationProcessor != null) {
|
|||
|
|
+ this.notificationProcessor.getPopLongPollingService().shutdown();
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java
|
|||
|
|
+index 93fa725a9..48db828e0 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageServiceImpl.java
|
|||
|
|
+@@ -629,7 +629,9 @@ public class TransactionalMessageServiceImpl implements TransactionalMessageServ
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void close() {
|
|||
|
|
+-
|
|||
|
|
++ if (this.transactionalOpBatchService != null) {
|
|||
|
|
++ this.transactionalOpBatchService.shutdown();
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public Message getOpMessage(int queueId, String moreData) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch016-backport-Optimize-fault-tolerant-mechanism.patch b/patch016-backport-Optimize-fault-tolerant-mechanism.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..9d8ef20a1
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch016-backport-Optimize-fault-tolerant-mechanism.patch
|
|||
|
|
@@ -0,0 +1,520 @@
|
|||
|
|
+From e11e29419f6e2d1d9673d0329e57b824ebf3da47 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Wed, 6 Sep 2023 20:42:24 +0800
|
|||
|
|
+Subject: [PATCH 1/3] [ISSUE #7308] Adding topic blacklist and filter in tiered
|
|||
|
|
+ storage module (#7310)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/TieredDispatcher.java | 21 +++++++--
|
|||
|
|
+ .../tieredstore/TieredMessageStore.java | 1 +
|
|||
|
|
+ .../file/TieredFlatFileManager.java | 17 ++++---
|
|||
|
|
+ .../TieredStoreTopicBlackListFilter.java | 45 +++++++++++++++++++
|
|||
|
|
+ .../provider/TieredStoreTopicFilter.java | 25 +++++++++++
|
|||
|
|
+ .../TieredStoreTopicBlackListFilterTest.java | 36 +++++++++++++++
|
|||
|
|
+ 6 files changed, 136 insertions(+), 9 deletions(-)
|
|||
|
|
+ create mode 100644 tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
+ create mode 100644 tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
+ create mode 100644 tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+index 430c2b62e..766c559e9 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredDispatcher.java
|
|||
|
|
+@@ -48,6 +48,8 @@ import org.apache.rocketmq.tieredstore.file.CompositeQueueFlatFile;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.file.TieredFlatFileManager;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsConstant;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsManager;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.TieredStoreTopicBlackListFilter;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.provider.TieredStoreTopicFilter;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.CQItemBufferUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.MessageBufferUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+@@ -56,6 +58,7 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+
|
|||
|
|
+ private static final Logger logger = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
++ private TieredStoreTopicFilter topicFilter;
|
|||
|
|
+ private final String brokerName;
|
|||
|
|
+ private final MessageStore defaultStore;
|
|||
|
|
+ private final TieredMessageStoreConfig storeConfig;
|
|||
|
|
+@@ -70,15 +73,15 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ this.defaultStore = defaultStore;
|
|||
|
|
+ this.storeConfig = storeConfig;
|
|||
|
|
+ this.brokerName = storeConfig.getBrokerName();
|
|||
|
|
++ this.topicFilter = new TieredStoreTopicBlackListFilter();
|
|||
|
|
+ this.tieredFlatFileManager = TieredFlatFileManager.getInstance(storeConfig);
|
|||
|
|
+ this.dispatchRequestReadMap = new ConcurrentHashMap<>();
|
|||
|
|
+ this.dispatchRequestWriteMap = new ConcurrentHashMap<>();
|
|||
|
|
+ this.dispatchTaskLock = new ReentrantLock();
|
|||
|
|
+ this.dispatchWriteLock = new ReentrantLock();
|
|||
|
|
+- this.initScheduleTask();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private void initScheduleTask() {
|
|||
|
|
++ protected void initScheduleTask() {
|
|||
|
|
+ TieredStoreExecutor.commonScheduledExecutor.scheduleWithFixedDelay(() ->
|
|||
|
|
+ tieredFlatFileManager.deepCopyFlatFileToList().forEach(flatFile -> {
|
|||
|
|
+ if (!flatFile.getCompositeFlatFileLock().isLocked()) {
|
|||
|
|
+@@ -87,6 +90,14 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ }), 30, 10, TimeUnit.SECONDS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public TieredStoreTopicFilter getTopicFilter() {
|
|||
|
|
++ return topicFilter;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setTopicFilter(TieredStoreTopicFilter topicFilter) {
|
|||
|
|
++ this.topicFilter = topicFilter;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public void dispatch(DispatchRequest request) {
|
|||
|
|
+ if (stopped) {
|
|||
|
|
+@@ -94,7 +105,7 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ String topic = request.getTopic();
|
|||
|
|
+- if (TieredStoreUtil.isSystemTopic(topic)) {
|
|||
|
|
++ if (topicFilter != null && topicFilter.filterTopic(topic)) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -219,6 +230,10 @@ public class TieredDispatcher extends ServiceThread implements CommitLogDispatch
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ if (topicFilter != null && topicFilter.filterTopic(flatFile.getMessageQueue().getTopic())) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ if (flatFile.getDispatchOffset() == -1L) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+index 78e855f36..9fb1b2f01 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+@@ -90,6 +90,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ boolean loadNextStore = next.load();
|
|||
|
|
+ boolean result = loadFlatFile && loadNextStore;
|
|||
|
|
+ if (result) {
|
|||
|
|
++ dispatcher.initScheduleTask();
|
|||
|
|
+ dispatcher.start();
|
|||
|
|
+ }
|
|||
|
|
+ return result;
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+index e9ae4a5a5..7c744af3b 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+@@ -134,21 +134,21 @@ public class TieredFlatFileManager {
|
|||
|
|
+ public void doCleanExpiredFile() {
|
|||
|
|
+ long expiredTimeStamp = System.currentTimeMillis() -
|
|||
|
|
+ TimeUnit.HOURS.toMillis(storeConfig.getTieredStoreFileReservedTime());
|
|||
|
|
+- Random random = new Random();
|
|||
|
|
+ for (CompositeQueueFlatFile flatFile : deepCopyFlatFileToList()) {
|
|||
|
|
+- int delay = random.nextInt(storeConfig.getMaxCommitJitter());
|
|||
|
|
+- TieredStoreExecutor.cleanExpiredFileExecutor.schedule(() -> {
|
|||
|
|
++ TieredStoreExecutor.cleanExpiredFileExecutor.submit(() -> {
|
|||
|
|
+ flatFile.getCompositeFlatFileLock().lock();
|
|||
|
|
+ try {
|
|||
|
|
+ flatFile.cleanExpiredFile(expiredTimeStamp);
|
|||
|
|
+ flatFile.destroyExpiredFile();
|
|||
|
|
+ if (flatFile.getConsumeQueueBaseOffset() == -1) {
|
|||
|
|
++ logger.info("Clean flatFile because file not initialized, topic={}, queueId={}",
|
|||
|
|
++ flatFile.getMessageQueue().getTopic(), flatFile.getMessageQueue().getQueueId());
|
|||
|
|
+ destroyCompositeFile(flatFile.getMessageQueue());
|
|||
|
|
+ }
|
|||
|
|
+ } finally {
|
|||
|
|
+ flatFile.getCompositeFlatFileLock().unlock();
|
|||
|
|
+ }
|
|||
|
|
+- }, delay, TimeUnit.MILLISECONDS);
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+ if (indexFile != null) {
|
|||
|
|
+ indexFile.cleanExpiredFile(expiredTimeStamp);
|
|||
|
|
+@@ -218,8 +218,13 @@ public class TieredFlatFileManager {
|
|||
|
|
+ storeConfig.getBrokerName(), queueMetadata.getQueue().getQueueId()));
|
|||
|
|
+ queueCount.incrementAndGet();
|
|||
|
|
+ });
|
|||
|
|
+- logger.info("Recover TopicFlatFile, topic: {}, queueCount: {}, cost: {}ms",
|
|||
|
|
+- topicMetadata.getTopic(), queueCount.get(), subWatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
++
|
|||
|
|
++ if (queueCount.get() == 0L) {
|
|||
|
|
++ metadataStore.deleteTopic(topicMetadata.getTopic());
|
|||
|
|
++ } else {
|
|||
|
|
++ logger.info("Recover TopicFlatFile, topic: {}, queueCount: {}, cost: {}ms",
|
|||
|
|
++ topicMetadata.getTopic(), queueCount.get(), subWatch.elapsed(TimeUnit.MILLISECONDS));
|
|||
|
|
++ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ logger.error("Recover TopicFlatFile error, topic: {}", topicMetadata.getTopic(), e);
|
|||
|
|
+ } finally {
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..50adbb713
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
+@@ -0,0 +1,45 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tieredstore.provider;
|
|||
|
|
++
|
|||
|
|
++import java.util.HashSet;
|
|||
|
|
++import java.util.Set;
|
|||
|
|
++import org.apache.commons.lang3.StringUtils;
|
|||
|
|
++import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
++
|
|||
|
|
++public class TieredStoreTopicBlackListFilter implements TieredStoreTopicFilter {
|
|||
|
|
++
|
|||
|
|
++ private final Set<String> topicBlackSet;
|
|||
|
|
++
|
|||
|
|
++ public TieredStoreTopicBlackListFilter() {
|
|||
|
|
++ this.topicBlackSet = new HashSet<>();
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean filterTopic(String topicName) {
|
|||
|
|
++ if (StringUtils.isBlank(topicName)) {
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++ return TieredStoreUtil.isSystemTopic(topicName) || topicBlackSet.contains(topicName);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void addTopicToWhiteList(String topicName) {
|
|||
|
|
++ this.topicBlackSet.add(topicName);
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..3f26b8b02
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
+@@ -0,0 +1,25 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tieredstore.provider;
|
|||
|
|
++
|
|||
|
|
++public interface TieredStoreTopicFilter {
|
|||
|
|
++
|
|||
|
|
++ boolean filterTopic(String topicName);
|
|||
|
|
++
|
|||
|
|
++ void addTopicToWhiteList(String topicName);
|
|||
|
|
++}
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..2bf48173c
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
+@@ -0,0 +1,36 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.tieredstore.provider;
|
|||
|
|
++
|
|||
|
|
++import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
++import org.junit.Assert;
|
|||
|
|
++import org.junit.Test;
|
|||
|
|
++
|
|||
|
|
++public class TieredStoreTopicBlackListFilterTest {
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void filterTopicTest() {
|
|||
|
|
++ TieredStoreTopicFilter topicFilter = new TieredStoreTopicBlackListFilter();
|
|||
|
|
++ Assert.assertTrue(topicFilter.filterTopic(""));
|
|||
|
|
++ Assert.assertTrue(topicFilter.filterTopic(TopicValidator.SYSTEM_TOPIC_PREFIX + "_Topic"));
|
|||
|
|
++
|
|||
|
|
++ String topicName = "WhiteTopic";
|
|||
|
|
++ Assert.assertFalse(topicFilter.filterTopic(topicName));
|
|||
|
|
++ topicFilter.addTopicToWhiteList(topicName);
|
|||
|
|
++ Assert.assertTrue(topicFilter.filterTopic(topicName));
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 628020537fa7035226bc8dcde9fa33d9d5df30ff Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Thu, 7 Sep 2023 16:17:47 +0800
|
|||
|
|
+Subject: [PATCH 2/3] [ISSUE #7293] Fix NPE when alter sync state set
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/controller/impl/manager/ReplicasInfoManager.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/ReplicasInfoManager.java b/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/ReplicasInfoManager.java
|
|||
|
|
+index b0a67531d..d83a690f9 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/ReplicasInfoManager.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/ReplicasInfoManager.java
|
|||
|
|
+@@ -104,7 +104,7 @@ public class ReplicasInfoManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // Check master
|
|||
|
|
+- if (!syncStateInfo.getMasterBrokerId().equals(request.getMasterBrokerId())) {
|
|||
|
|
++ if (syncStateInfo.getMasterBrokerId() == null || !syncStateInfo.getMasterBrokerId().equals(request.getMasterBrokerId())) {
|
|||
|
|
+ String err = String.format("Rejecting alter syncStateSet request because the current leader is:{%s}, not {%s}",
|
|||
|
|
+ syncStateInfo.getMasterBrokerId(), request.getMasterBrokerId());
|
|||
|
|
+ LOGGER.error("{}", err);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 6fd0073d6475c539e8f4c30dc4f104a56a21d724 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ji Juntao <juntao.jjt@alibaba-inc.com>
|
|||
|
|
+Date: Thu, 7 Sep 2023 20:21:16 +0800
|
|||
|
|
+Subject: [PATCH 3/3] [ISSUE #7319] Optimize fault-tolerant mechanism for
|
|||
|
|
+ sending messages and hot update switch (#7320)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../impl/producer/DefaultMQProducerImpl.java | 8 ++------
|
|||
|
|
+ .../client/latency/LatencyFaultTolerance.java | 14 +++++++++++++
|
|||
|
|
+ .../latency/LatencyFaultToleranceImpl.java | 13 +++++++++++-
|
|||
|
|
+ .../client/latency/MQFaultStrategy.java | 20 +++++++------------
|
|||
|
|
+ .../proxy/service/route/MessageQueueView.java | 9 ---------
|
|||
|
|
+ .../service/route/TopicRouteService.java | 10 +++++++++-
|
|||
|
|
+ 6 files changed, 44 insertions(+), 30 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+index 2d6b83ac2..b0c212e46 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java
|
|||
|
|
+@@ -263,9 +263,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ mQClientFactory.start();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
+- this.mqFaultStrategy.startDetector();
|
|||
|
|
+- }
|
|||
|
|
++ this.mqFaultStrategy.startDetector();
|
|||
|
|
+
|
|||
|
|
+ log.info("the producer [{}] start OK. sendMessageWithVIPChannel={}", this.defaultMQProducer.getProducerGroup(),
|
|||
|
|
+ this.defaultMQProducer.isSendMessageWithVIPChannel());
|
|||
|
|
+@@ -311,9 +309,7 @@ public class DefaultMQProducerImpl implements MQProducerInner {
|
|||
|
|
+ if (shutdownFactory) {
|
|||
|
|
+ this.mQClientFactory.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+- if (this.mqFaultStrategy.isStartDetectorEnable()) {
|
|||
|
|
+- this.mqFaultStrategy.shutdown();
|
|||
|
|
+- }
|
|||
|
|
++ this.mqFaultStrategy.shutdown();
|
|||
|
|
+ RequestFutureHolder.getInstance().shutdown(this);
|
|||
|
|
+ log.info("the producer [{}] shutdown OK", this.defaultMQProducer.getProducerGroup());
|
|||
|
|
+ this.serviceState = ServiceState.SHUTDOWN_ALREADY;
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
+index 72d2f3450..17aaa266a 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultTolerance.java
|
|||
|
|
+@@ -89,4 +89,18 @@ public interface LatencyFaultTolerance<T> {
|
|||
|
|
+ * @param detectInterval each broker's detecting interval
|
|||
|
|
+ */
|
|||
|
|
+ void setDetectInterval(final int detectInterval);
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Use it to set the detector work or not.
|
|||
|
|
++ *
|
|||
|
|
++ * @param startDetectorEnable set the detector's work status
|
|||
|
|
++ */
|
|||
|
|
++ void setStartDetectorEnable(final boolean startDetectorEnable);
|
|||
|
|
++
|
|||
|
|
++ /**
|
|||
|
|
++ * Use it to judge if the detector enabled.
|
|||
|
|
++ *
|
|||
|
|
++ * @return is the detector should be started.
|
|||
|
|
++ */
|
|||
|
|
++ boolean isStartDetectorEnable();
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
+index 8af629574..d3ff7eb45 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/LatencyFaultToleranceImpl.java
|
|||
|
|
+@@ -37,6 +37,8 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+ private int detectTimeout = 200;
|
|||
|
|
+ private int detectInterval = 2000;
|
|||
|
|
+ private final ThreadLocalIndex whichItemWorst = new ThreadLocalIndex();
|
|||
|
|
++
|
|||
|
|
++ private volatile boolean startDetectorEnable = false;
|
|||
|
|
+ private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public Thread newThread(Runnable r) {
|
|||
|
|
+@@ -80,7 +82,9 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+ @Override
|
|||
|
|
+ public void run() {
|
|||
|
|
+ try {
|
|||
|
|
+- detectByOneRound();
|
|||
|
|
++ if (startDetectorEnable) {
|
|||
|
|
++ detectByOneRound();
|
|||
|
|
++ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.warn("Unexpected exception raised while detecting service reachability", e);
|
|||
|
|
+ }
|
|||
|
|
+@@ -137,6 +141,13 @@ public class LatencyFaultToleranceImpl implements LatencyFaultTolerance<String>
|
|||
|
|
+ this.faultItemTable.remove(name);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isStartDetectorEnable() {
|
|||
|
|
++ return startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setStartDetectorEnable(boolean startDetectorEnable) {
|
|||
|
|
++ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ }
|
|||
|
|
+ @Override
|
|||
|
|
+ public String pickOneAtLeast() {
|
|||
|
|
+ final Enumeration<FaultItem> elements = this.faultItemTable.elements();
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java b/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
+index c01490784..69fb533e5 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/latency/MQFaultStrategy.java
|
|||
|
|
+@@ -24,8 +24,8 @@ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+
|
|||
|
|
+ public class MQFaultStrategy {
|
|||
|
|
+ private LatencyFaultTolerance<String> latencyFaultTolerance;
|
|||
|
|
+- private boolean sendLatencyFaultEnable;
|
|||
|
|
+- private boolean startDetectorEnable;
|
|||
|
|
++ private volatile boolean sendLatencyFaultEnable;
|
|||
|
|
++ private volatile boolean startDetectorEnable;
|
|||
|
|
+ private long[] latencyMax = {50L, 100L, 550L, 1800L, 3000L, 5000L, 15000L};
|
|||
|
|
+ private long[] notAvailableDuration = {0L, 0L, 2000L, 5000L, 6000L, 10000L, 30000L};
|
|||
|
|
+
|
|||
|
|
+@@ -64,11 +64,11 @@ public class MQFaultStrategy {
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+ public MQFaultStrategy(ClientConfig cc, Resolver fetcher, ServiceDetector serviceDetector) {
|
|||
|
|
+- this.setStartDetectorEnable(cc.isStartDetectorEnable());
|
|||
|
|
+- this.setSendLatencyFaultEnable(cc.isSendLatencyEnable());
|
|||
|
|
+ this.latencyFaultTolerance = new LatencyFaultToleranceImpl(fetcher, serviceDetector);
|
|||
|
|
+ this.latencyFaultTolerance.setDetectInterval(cc.getDetectInterval());
|
|||
|
|
+ this.latencyFaultTolerance.setDetectTimeout(cc.getDetectTimeout());
|
|||
|
|
++ this.setStartDetectorEnable(cc.isStartDetectorEnable());
|
|||
|
|
++ this.setSendLatencyFaultEnable(cc.isSendLatencyEnable());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // For unit test.
|
|||
|
|
+@@ -123,21 +123,15 @@ public class MQFaultStrategy {
|
|||
|
|
+
|
|||
|
|
+ public void setStartDetectorEnable(boolean startDetectorEnable) {
|
|||
|
|
+ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ this.latencyFaultTolerance.setStartDetectorEnable(startDetectorEnable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void startDetector() {
|
|||
|
|
+- // user should start the detector
|
|||
|
|
+- // and the thread should not be in running state.
|
|||
|
|
+- if (this.sendLatencyFaultEnable && this.startDetectorEnable) {
|
|||
|
|
+- // start the detector.
|
|||
|
|
+- this.latencyFaultTolerance.startDetector();
|
|||
|
|
+- }
|
|||
|
|
++ this.latencyFaultTolerance.startDetector();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
+- if (this.sendLatencyFaultEnable && this.startDetectorEnable) {
|
|||
|
|
+- this.latencyFaultTolerance.shutdown();
|
|||
|
|
+- }
|
|||
|
|
++ this.latencyFaultTolerance.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MessageQueue selectOneMessageQueue(final TopicPublishInfo tpInfo, final String lastBrokerName, final boolean resetIndex) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
+index 8b3c2f7c8..898e529f8 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/MessageQueueView.java
|
|||
|
|
+@@ -26,7 +26,6 @@ public class MessageQueueView {
|
|||
|
|
+ private final MessageQueueSelector readSelector;
|
|||
|
|
+ private final MessageQueueSelector writeSelector;
|
|||
|
|
+ private final TopicRouteWrapper topicRouteWrapper;
|
|||
|
|
+- private MQFaultStrategy mqFaultStrategy;
|
|||
|
|
+
|
|||
|
|
+ public MessageQueueView(String topic, TopicRouteData topicRouteData, MQFaultStrategy mqFaultStrategy) {
|
|||
|
|
+ this.topicRouteWrapper = new TopicRouteWrapper(topicRouteData, topic);
|
|||
|
|
+@@ -67,12 +66,4 @@ public class MessageQueueView {
|
|||
|
|
+ .add("topicRouteWrapper", topicRouteWrapper)
|
|||
|
|
+ .toString();
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+- public MQFaultStrategy getMQFaultStrategy() {
|
|||
|
|
+- return mqFaultStrategy;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public void setMQFaultStrategy(MQFaultStrategy mqFaultStrategy) {
|
|||
|
|
+- this.mqFaultStrategy = mqFaultStrategy;
|
|||
|
|
+- }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+index 74769a423..caf62a1e0 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+@@ -127,7 +127,7 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ @Override
|
|||
|
|
+ public String resolve(String name) {
|
|||
|
|
+ try {
|
|||
|
|
+- String brokerAddr = getBrokerAddr(null, name);
|
|||
|
|
++ String brokerAddr = getBrokerAddr(ProxyContext.createForInner("MQFaultStrategy"), name);
|
|||
|
|
+ return brokerAddr;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ return null;
|
|||
|
|
+@@ -175,9 +175,17 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ public void updateFaultItem(final String brokerName, final long currentLatency, boolean isolation,
|
|||
|
|
+ boolean reachable) {
|
|||
|
|
++ checkSendFaultToleranceEnable();
|
|||
|
|
+ this.mqFaultStrategy.updateFaultItem(brokerName, currentLatency, isolation, reachable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void checkSendFaultToleranceEnable() {
|
|||
|
|
++ boolean hotLatencySwitch = ConfigurationManager.getProxyConfig().isSendLatencyEnable();
|
|||
|
|
++ boolean hotDetectorSwitch = ConfigurationManager.getProxyConfig().isStartDetectorEnable();
|
|||
|
|
++ this.mqFaultStrategy.setSendLatencyFaultEnable(hotLatencySwitch);
|
|||
|
|
++ this.mqFaultStrategy.setStartDetectorEnable(hotDetectorSwitch);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public MQFaultStrategy getMqFaultStrategy() {
|
|||
|
|
+ return this.mqFaultStrategy;
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch017-backport-Convergent-thread-pool-creation.patch b/patch017-backport-Convergent-thread-pool-creation.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..92d0bd2e9
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch017-backport-Convergent-thread-pool-creation.patch
|
|||
|
|
@@ -0,0 +1,2243 @@
|
|||
|
|
+From c100d815d754d7cb330bc63e145bafd2d9b59cb1 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Mon, 11 Sep 2023 10:13:56 +0800
|
|||
|
|
+Subject: [PATCH 1/6] [ISSUE #7328] Convergent thread pool creation (#7329)
|
|||
|
|
+
|
|||
|
|
+* Convergence thread pool creation to facilitate subsequent iteration management
|
|||
|
|
+
|
|||
|
|
+* Convergence thread pool creation in ThreadPoolMonitor.java
|
|||
|
|
+
|
|||
|
|
+* fix unit test
|
|||
|
|
+
|
|||
|
|
+* Convergence ThreadPool constructor
|
|||
|
|
+
|
|||
|
|
+* Convergence ScheduledThreadPool constructor
|
|||
|
|
+
|
|||
|
|
+* remove unused import
|
|||
|
|
+
|
|||
|
|
+* Convergence ScheduledThreadPool constructor
|
|||
|
|
+
|
|||
|
|
+* remove unused import
|
|||
|
|
+
|
|||
|
|
+---------
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/broker/BrokerController.java | 39 +++++-----
|
|||
|
|
+ .../client/ClientHousekeepingService.java | 4 +-
|
|||
|
|
+ .../DefaultConsumerIdsChangeListener.java | 3 +-
|
|||
|
|
+ .../broker/controller/ReplicasManager.java | 9 +--
|
|||
|
|
+ .../dledger/DLedgerRoleChangeHandler.java | 4 +-
|
|||
|
|
+ .../broker/failover/EscapeBridge.java | 4 +-
|
|||
|
|
+ .../broker/latency/BrokerFastFailure.java | 5 +-
|
|||
|
|
+ .../BrokerFixedThreadPoolExecutor.java | 57 --------------
|
|||
|
|
+ .../broker/latency/FutureTaskExt.java | 39 ----------
|
|||
|
|
+ .../rocketmq/broker/out/BrokerOuterAPI.java | 7 +-
|
|||
|
|
+ .../schedule/ScheduleMessageService.java | 7 +-
|
|||
|
|
+ .../broker/topic/TopicRouteInfoManager.java | 4 +-
|
|||
|
|
+ ...ractTransactionalMessageCheckListener.java | 4 +-
|
|||
|
|
+ .../rocketmq/broker/BrokerControllerTest.java | 2 +-
|
|||
|
|
+ .../broker/latency/BrokerFastFailureTest.java | 1 +
|
|||
|
|
+ .../common/config/AbstractRocksDBStorage.java | 6 +-
|
|||
|
|
+ .../FutureTaskExtThreadPoolExecutor.java | 3 +-
|
|||
|
|
+ .../common/thread/ThreadPoolMonitor.java | 6 +-
|
|||
|
|
+ .../rocketmq/common/utils/ThreadUtils.java | 74 ++++++++++++++++---
|
|||
|
|
+ .../rocketmq/container/BrokerContainer.java | 6 +-
|
|||
|
|
+ .../controller/ControllerManager.java | 14 +---
|
|||
|
|
+ .../controller/impl/DLedgerController.java | 10 +--
|
|||
|
|
+ .../DefaultBrokerHeartbeatManager.java | 3 +-
|
|||
|
|
+ .../rocketmq/namesrv/NamesrvController.java | 22 ++----
|
|||
|
|
+ .../grpc/v2/channel/GrpcChannelManager.java | 6 +-
|
|||
|
|
+ .../remoting/RemotingProtocolServer.java | 4 +-
|
|||
|
|
+ .../proxy/service/ClusterServiceManager.java | 12 +--
|
|||
|
|
+ .../proxy/service/LocalServiceManager.java | 4 +-
|
|||
|
|
+ .../receipt/DefaultReceiptHandleManager.java | 8 +-
|
|||
|
|
+ .../service/route/TopicRouteService.java | 9 +--
|
|||
|
|
+ .../remoting/netty/NettyRemotingClient.java | 4 +-
|
|||
|
|
+ .../remoting/netty/NettyRemotingServer.java | 4 +-
|
|||
|
|
+ .../rocketmq/store/DefaultMessageStore.java | 8 +-
|
|||
|
|
+ .../ha/autoswitch/AutoSwitchHAService.java | 38 +++++-----
|
|||
|
|
+ .../rocketmq/store/kv/CompactionStore.java | 21 +++---
|
|||
|
|
+ .../store/queue/ConsumeQueueStore.java | 4 +-
|
|||
|
|
+ .../store/stats/BrokerStatsManager.java | 14 ++--
|
|||
|
|
+ .../store/timer/TimerMessageStore.java | 6 +-
|
|||
|
|
+ .../apache/rocketmq/test/util/StatUtil.java | 1 -
|
|||
|
|
+ .../common/TieredStoreExecutor.java | 14 ++--
|
|||
|
|
+ .../tools/admin/DefaultMQAdminExtImpl.java | 3 +-
|
|||
|
|
+ 41 files changed, 215 insertions(+), 278 deletions(-)
|
|||
|
|
+ delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFixedThreadPoolExecutor.java
|
|||
|
|
+ delete mode 100644 broker/src/main/java/org/apache/rocketmq/broker/latency/FutureTaskExt.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 6aba70cb2..275b64b1a 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -34,7 +34,6 @@ import org.apache.rocketmq.broker.failover.EscapeBridge;
|
|||
|
|
+ import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap;
|
|||
|
|
+ import org.apache.rocketmq.broker.filter.ConsumerFilterManager;
|
|||
|
|
+ import org.apache.rocketmq.broker.latency.BrokerFastFailure;
|
|||
|
|
+-import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor;
|
|||
|
|
+ import org.apache.rocketmq.broker.longpolling.LmqPullRequestHoldService;
|
|||
|
|
+ import org.apache.rocketmq.broker.longpolling.NotifyMessageArrivingListener;
|
|||
|
|
+ import org.apache.rocketmq.broker.longpolling.PullRequestHoldService;
|
|||
|
|
+@@ -98,6 +97,7 @@ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+ import org.apache.rocketmq.common.stats.MomentStatsItem;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.ServiceProvider;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.Configuration;
|
|||
|
|
+@@ -160,7 +160,6 @@ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ScheduledFuture;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.locks.Lock;
|
|||
|
|
+ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
+@@ -455,10 +454,10 @@ public class BrokerController {
|
|||
|
|
+ * Initialize resources including remoting server and thread executors.
|
|||
|
|
+ */
|
|||
|
|
+ protected void initializeResources() {
|
|||
|
|
+- this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ this.scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("BrokerControllerScheduledThread", true, getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.sendMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.sendMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getSendMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getSendMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -466,7 +465,7 @@ public class BrokerController {
|
|||
|
|
+ this.sendThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("SendMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.pullMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.pullMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getPullMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getPullMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -474,7 +473,7 @@ public class BrokerController {
|
|||
|
|
+ this.pullThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("PullMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.litePullMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.litePullMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getLitePullMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getLitePullMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -482,7 +481,7 @@ public class BrokerController {
|
|||
|
|
+ this.litePullThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("LitePullMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.putMessageFutureExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.putMessageFutureExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getPutMessageFutureThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getPutMessageFutureThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -490,7 +489,7 @@ public class BrokerController {
|
|||
|
|
+ this.putThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("SendMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.ackMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.ackMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getAckMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getAckMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -498,7 +497,7 @@ public class BrokerController {
|
|||
|
|
+ this.ackThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("AckMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.queryMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.queryMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getQueryMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getQueryMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -506,7 +505,7 @@ public class BrokerController {
|
|||
|
|
+ this.queryThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("QueryMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.adminBrokerExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.adminBrokerExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getAdminBrokerThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getAdminBrokerThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -514,7 +513,7 @@ public class BrokerController {
|
|||
|
|
+ this.adminBrokerThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("AdminBrokerThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.clientManageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.clientManageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getClientManageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getClientManageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -522,7 +521,7 @@ public class BrokerController {
|
|||
|
|
+ this.clientManagerThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("ClientManageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.heartbeatExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.heartbeatExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getHeartbeatThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getHeartbeatThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -530,7 +529,7 @@ public class BrokerController {
|
|||
|
|
+ this.heartbeatThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("HeartbeatThread_", true, getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.consumerManageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.consumerManageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getConsumerManageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getConsumerManageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -538,7 +537,7 @@ public class BrokerController {
|
|||
|
|
+ this.consumerManagerThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("ConsumerManageThread_", true, getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.replyMessageExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.replyMessageExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getProcessReplyMessageThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -546,7 +545,7 @@ public class BrokerController {
|
|||
|
|
+ this.replyThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("ProcessReplyMessageThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.endTransactionExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.endTransactionExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getEndTransactionThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getEndTransactionThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -554,7 +553,7 @@ public class BrokerController {
|
|||
|
|
+ this.endTransactionThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("EndTransactionThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.loadBalanceExecutor = new BrokerFixedThreadPoolExecutor(
|
|||
|
|
++ this.loadBalanceExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.brokerConfig.getLoadBalanceProcessorThreadPoolNums(),
|
|||
|
|
+ this.brokerConfig.getLoadBalanceProcessorThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -562,9 +561,9 @@ public class BrokerController {
|
|||
|
|
+ this.loadBalanceThreadPoolQueue,
|
|||
|
|
+ new ThreadFactoryImpl("LoadBalanceProcessorThread_", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+- this.syncBrokerMemberGroupExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ this.syncBrokerMemberGroupExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("BrokerControllerSyncBrokerScheduledThread", getBrokerIdentity()));
|
|||
|
|
+- this.brokerHeartbeatExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ this.brokerHeartbeatExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("BrokerControllerHeartbeatScheduledThread", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+ this.topicQueueMappingCleanService = new TopicQueueMappingCleanService(this);
|
|||
|
|
+@@ -828,8 +827,6 @@ public class BrokerController {
|
|||
|
|
+
|
|||
|
|
+ initializeResources();
|
|||
|
|
+
|
|||
|
|
+- registerProcessor();
|
|||
|
|
+-
|
|||
|
|
+ initializeScheduledTasks();
|
|||
|
|
+
|
|||
|
|
+ initialTransaction();
|
|||
|
|
+@@ -1690,6 +1687,8 @@ public class BrokerController {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }, 10, 5, TimeUnit.SECONDS);
|
|||
|
|
++
|
|||
|
|
++ registerProcessor();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ protected void scheduleSendHeartbeat() {
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java b/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
|
|||
|
|
+index 98e5f450f..cbb81f632 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/client/ClientHousekeepingService.java
|
|||
|
|
+@@ -18,11 +18,11 @@ package org.apache.rocketmq.broker.client;
|
|||
|
|
+
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
+@@ -35,7 +35,7 @@ public class ClientHousekeepingService implements ChannelEventListener {
|
|||
|
|
+
|
|||
|
|
+ public ClientHousekeepingService(final BrokerController brokerController) {
|
|||
|
|
+ this.brokerController = brokerController;
|
|||
|
|
+- scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("ClientHousekeepingScheduledThread", brokerController.getBrokerIdentity()));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
|
|||
|
|
+index 2ce036a0f..d17a2a547 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java
|
|||
|
|
+@@ -22,7 +22,6 @@ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.common.AbstractBrokerRunnable;
|
|||
|
|
+@@ -37,7 +36,7 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen
|
|||
|
|
+ private final BrokerController brokerController;
|
|||
|
|
+ private final int cacheSize = 8096;
|
|||
|
|
+
|
|||
|
|
+- private final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ private final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ ThreadUtils.newGenericThreadFactory("DefaultConsumerIdsChangeListener", true));
|
|||
|
|
+
|
|||
|
|
+ private ConcurrentHashMap<String,List<Channel>> consumerChannelMap = new ConcurrentHashMap<>(cacheSize);
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+index 37c82e434..a989e6e68 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java
|
|||
|
|
+@@ -27,10 +27,8 @@ import java.util.concurrent.ArrayBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ScheduledFuture;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+@@ -42,6 +40,7 @@ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.EpochEntry;
|
|||
|
|
+@@ -107,9 +106,9 @@ public class ReplicasManager {
|
|||
|
|
+ public ReplicasManager(final BrokerController brokerController) {
|
|||
|
|
+ this.brokerController = brokerController;
|
|||
|
|
+ this.brokerOuterAPI = brokerController.getBrokerOuterAPI();
|
|||
|
|
+- this.scheduledService = Executors.newScheduledThreadPool(3, new ThreadFactoryImpl("ReplicasManager_ScheduledService_", brokerController.getBrokerIdentity()));
|
|||
|
|
+- this.executorService = Executors.newFixedThreadPool(3, new ThreadFactoryImpl("ReplicasManager_ExecutorService_", brokerController.getBrokerIdentity()));
|
|||
|
|
+- this.scanExecutor = new ThreadPoolExecutor(4, 10, 60, TimeUnit.SECONDS,
|
|||
|
|
++ this.scheduledService = ThreadUtils.newScheduledThreadPool(3, new ThreadFactoryImpl("ReplicasManager_ScheduledService_", brokerController.getBrokerIdentity()));
|
|||
|
|
++ this.executorService = ThreadUtils.newThreadPoolExecutor(3, new ThreadFactoryImpl("ReplicasManager_ExecutorService_", brokerController.getBrokerIdentity()));
|
|||
|
|
++ this.scanExecutor = ThreadUtils.newThreadPoolExecutor(4, 10, 60, TimeUnit.SECONDS,
|
|||
|
|
+ new ArrayBlockingQueue<>(32), new ThreadFactoryImpl("ReplicasManager_scan_thread_", brokerController.getBrokerIdentity()));
|
|||
|
|
+ this.haService = (AutoSwitchHAService) brokerController.getMessageStore().getHaService();
|
|||
|
|
+ this.brokerConfig = brokerController.getBrokerConfig();
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java b/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
|
|||
|
|
+index 75023ee1b..e6cb97640 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java
|
|||
|
|
+@@ -21,12 +21,12 @@ import io.openmessaging.storage.dledger.DLedgerServer;
|
|||
|
|
+ import io.openmessaging.storage.dledger.MemberState;
|
|||
|
|
+ import io.openmessaging.storage.dledger.utils.DLedgerUtils;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.Future;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+@@ -49,7 +49,7 @@ public class DLedgerRoleChangeHandler implements DLedgerLeaderElector.RoleChange
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ this.dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog();
|
|||
|
|
+ this.dLegerServer = dLedgerCommitLog.getdLedgerServer();
|
|||
|
|
+- this.executorService = Executors.newSingleThreadExecutor(
|
|||
|
|
++ this.executorService = ThreadUtils.newSingleThreadExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("DLegerRoleChangeHandler_", brokerController.getBrokerIdentity()));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java b/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java
|
|||
|
|
+index 7c350fc1d..6a0817480 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/failover/EscapeBridge.java
|
|||
|
|
+@@ -24,7 +24,6 @@ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+@@ -43,6 +42,7 @@ import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+@@ -72,7 +72,7 @@ public class EscapeBridge {
|
|||
|
|
+ public void start() throws Exception {
|
|||
|
|
+ if (brokerController.getBrokerConfig().isEnableSlaveActingMaster() && brokerController.getBrokerConfig().isEnableRemoteEscape()) {
|
|||
|
|
+ final BlockingQueue<Runnable> asyncSenderThreadPoolQueue = new LinkedBlockingQueue<>(50000);
|
|||
|
|
+- this.defaultAsyncSenderExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ this.defaultAsyncSenderExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ Runtime.getRuntime().availableProcessors(),
|
|||
|
|
+ Runtime.getRuntime().availableProcessors(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
|
|||
|
|
+index d3d0bc8ba..3b6e9dc67 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java
|
|||
|
|
+@@ -18,13 +18,14 @@ package org.apache.rocketmq.broker.latency;
|
|||
|
|
+
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.common.AbstractBrokerRunnable;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.RequestTask;
|
|||
|
|
+@@ -43,7 +44,7 @@ public class BrokerFastFailure {
|
|||
|
|
+
|
|||
|
|
+ public BrokerFastFailure(final BrokerController brokerController) {
|
|||
|
|
+ this.brokerController = brokerController;
|
|||
|
|
+- this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ this.scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("BrokerFastFailureScheduledThread", true,
|
|||
|
|
+ brokerController == null ? null : brokerController.getBrokerConfig()));
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFixedThreadPoolExecutor.java b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFixedThreadPoolExecutor.java
|
|||
|
|
+deleted file mode 100644
|
|||
|
|
+index d2d1143a3..000000000
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFixedThreadPoolExecutor.java
|
|||
|
|
++++ /dev/null
|
|||
|
|
+@@ -1,57 +0,0 @@
|
|||
|
|
+-/*
|
|||
|
|
+- * 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.broker.latency;
|
|||
|
|
+-
|
|||
|
|
+-import java.util.concurrent.BlockingQueue;
|
|||
|
|
+-import java.util.concurrent.RejectedExecutionHandler;
|
|||
|
|
+-import java.util.concurrent.RunnableFuture;
|
|||
|
|
+-import java.util.concurrent.ThreadFactory;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-
|
|||
|
|
+-public class BrokerFixedThreadPoolExecutor extends ThreadPoolExecutor {
|
|||
|
|
+- public BrokerFixedThreadPoolExecutor(final int corePoolSize, final int maximumPoolSize, final long keepAliveTime,
|
|||
|
|
+- final TimeUnit unit,
|
|||
|
|
+- final BlockingQueue<Runnable> workQueue) {
|
|||
|
|
+- super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue);
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public BrokerFixedThreadPoolExecutor(final int corePoolSize, final int maximumPoolSize, final long keepAliveTime,
|
|||
|
|
+- final TimeUnit unit,
|
|||
|
|
+- final BlockingQueue<Runnable> workQueue, final ThreadFactory threadFactory) {
|
|||
|
|
+- super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public BrokerFixedThreadPoolExecutor(final int corePoolSize, final int maximumPoolSize, final long keepAliveTime,
|
|||
|
|
+- final TimeUnit unit,
|
|||
|
|
+- final BlockingQueue<Runnable> workQueue, final RejectedExecutionHandler handler) {
|
|||
|
|
+- super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, handler);
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public BrokerFixedThreadPoolExecutor(final int corePoolSize, final int maximumPoolSize, final long keepAliveTime,
|
|||
|
|
+- final TimeUnit unit,
|
|||
|
|
+- final BlockingQueue<Runnable> workQueue, final ThreadFactory threadFactory,
|
|||
|
|
+- final RejectedExecutionHandler handler) {
|
|||
|
|
+- super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Override
|
|||
|
|
+- protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
|
|||
|
|
+- return new FutureTaskExt<>(runnable, value);
|
|||
|
|
+- }
|
|||
|
|
+-}
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/latency/FutureTaskExt.java b/broker/src/main/java/org/apache/rocketmq/broker/latency/FutureTaskExt.java
|
|||
|
|
+deleted file mode 100644
|
|||
|
|
+index f132efaeb..000000000
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/latency/FutureTaskExt.java
|
|||
|
|
++++ /dev/null
|
|||
|
|
+@@ -1,39 +0,0 @@
|
|||
|
|
+-/*
|
|||
|
|
+- * 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.broker.latency;
|
|||
|
|
+-
|
|||
|
|
+-import java.util.concurrent.Callable;
|
|||
|
|
+-import java.util.concurrent.FutureTask;
|
|||
|
|
+-
|
|||
|
|
+-public class FutureTaskExt<V> extends FutureTask<V> {
|
|||
|
|
+- private final Runnable runnable;
|
|||
|
|
+-
|
|||
|
|
+- public FutureTaskExt(final Callable<V> callable) {
|
|||
|
|
+- super(callable);
|
|||
|
|
+- this.runnable = null;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public FutureTaskExt(final Runnable runnable, final V result) {
|
|||
|
|
+- super(runnable, result);
|
|||
|
|
+- this.runnable = runnable;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- public Runnable getRunnable() {
|
|||
|
|
+- return runnable;
|
|||
|
|
+- }
|
|||
|
|
+-}
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
+index ae81e8b11..9dfb8127d 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
+@@ -27,9 +27,9 @@ import java.util.concurrent.ArrayBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
|
|
+ import java.util.concurrent.CountDownLatch;
|
|||
|
|
++import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+-import org.apache.rocketmq.broker.latency.BrokerFixedThreadPoolExecutor;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullStatus;
|
|||
|
|
+ import org.apache.rocketmq.client.exception.MQBrokerException;
|
|||
|
|
+@@ -59,6 +59,7 @@ import org.apache.rocketmq.common.namesrv.DefaultTopAddressing;
|
|||
|
|
+ import org.apache.rocketmq.common.namesrv.TopAddressing;
|
|||
|
|
+ import org.apache.rocketmq.common.sysflag.PullSysFlag;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+@@ -144,7 +145,7 @@ public class BrokerOuterAPI {
|
|||
|
|
+ private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
|
|||
|
|
+ private final RemotingClient remotingClient;
|
|||
|
|
+ private final TopAddressing topAddressing = new DefaultTopAddressing(MixAll.getWSAddr());
|
|||
|
|
+- private final BrokerFixedThreadPoolExecutor brokerOuterExecutor = new BrokerFixedThreadPoolExecutor(4, 10, 1, TimeUnit.MINUTES,
|
|||
|
|
++ private final ExecutorService brokerOuterExecutor = ThreadUtils.newThreadPoolExecutor(4, 10, 1, TimeUnit.MINUTES,
|
|||
|
|
+ new ArrayBlockingQueue<>(32), new ThreadFactoryImpl("brokerOutApi_thread_", true));
|
|||
|
|
+ private final ClientMetadata clientMetadata;
|
|||
|
|
+ private final RpcClient rpcClient;
|
|||
|
|
+@@ -1092,7 +1093,7 @@ public class BrokerOuterAPI {
|
|||
|
|
+ throw new MQBrokerException(response.getCode(), response.getRemark());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public BrokerFixedThreadPoolExecutor getBrokerOuterExecutor() {
|
|||
|
|
++ public ExecutorService getBrokerOuterExecutor() {
|
|||
|
|
+ return brokerOuterExecutor;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
+index 297b14207..0c2e6507b 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/schedule/ScheduleMessageService.java
|
|||
|
|
+@@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+@@ -91,7 +90,7 @@ public class ScheduleMessageService extends ConfigManager {
|
|||
|
|
+ public ScheduleMessageService(final BrokerController brokerController) {
|
|||
|
|
+ this.brokerController = brokerController;
|
|||
|
|
+ this.enableAsyncDeliver = brokerController.getMessageStoreConfig().isEnableScheduleAsyncDeliver();
|
|||
|
|
+- scheduledPersistService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ scheduledPersistService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("ScheduleMessageServicePersistThread", true, brokerController.getBrokerConfig()));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -134,9 +133,9 @@ public class ScheduleMessageService extends ConfigManager {
|
|||
|
|
+ public void start() {
|
|||
|
|
+ if (started.compareAndSet(false, true)) {
|
|||
|
|
+ this.load();
|
|||
|
|
+- this.deliverExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageTimerThread_"));
|
|||
|
|
++ this.deliverExecutorService = ThreadUtils.newScheduledThreadPool(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageTimerThread_"));
|
|||
|
|
+ if (this.enableAsyncDeliver) {
|
|||
|
|
+- this.handleExecutorService = new ScheduledThreadPoolExecutor(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_"));
|
|||
|
|
++ this.handleExecutorService = ThreadUtils.newScheduledThreadPool(this.maxDelayLevel, new ThreadFactoryImpl("ScheduleMessageExecutorHandleThread_"));
|
|||
|
|
+ }
|
|||
|
|
+ for (Map.Entry<Integer, Long> entry : this.delayLevelTable.entrySet()) {
|
|||
|
|
+ Integer level = entry.getKey();
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicRouteInfoManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicRouteInfoManager.java
|
|||
|
|
+index b35564725..11bde5f5f 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicRouteInfoManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicRouteInfoManager.java
|
|||
|
|
+@@ -23,7 +23,6 @@ import java.util.Objects;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.locks.Lock;
|
|||
|
|
+@@ -36,6 +35,7 @@ import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+@@ -66,7 +66,7 @@ public class TopicRouteInfoManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void start() {
|
|||
|
|
+- this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("TopicRouteInfoManagerScheduledThread"));
|
|||
|
|
++ this.scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("TopicRouteInfoManagerScheduledThread"));
|
|||
|
|
+
|
|||
|
|
+ this.scheduledExecutorService.scheduleAtFixedRate(() -> {
|
|||
|
|
+ try {
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/AbstractTransactionalMessageCheckListener.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/AbstractTransactionalMessageCheckListener.java
|
|||
|
|
+index 771d84300..982355d78 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/AbstractTransactionalMessageCheckListener.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/AbstractTransactionalMessageCheckListener.java
|
|||
|
|
+@@ -19,7 +19,6 @@ package org.apache.rocketmq.broker.transaction;
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import java.util.concurrent.ArrayBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+@@ -27,6 +26,7 @@ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader;
|
|||
|
|
+@@ -97,7 +97,7 @@ public abstract class AbstractTransactionalMessageCheckListener {
|
|||
|
|
+
|
|||
|
|
+ public synchronized void initExecutorService() {
|
|||
|
|
+ if (executorService == null) {
|
|||
|
|
+- executorService = new ThreadPoolExecutor(2, 5, 100, TimeUnit.SECONDS, new ArrayBlockingQueue<>(2000),
|
|||
|
|
++ executorService = ThreadUtils.newThreadPoolExecutor(2, 5, 100, TimeUnit.SECONDS, new ArrayBlockingQueue<>(2000),
|
|||
|
|
+ new ThreadFactoryImpl("Transaction-msg-check-thread", brokerController.getBrokerIdentity()), new CallerRunsPolicy());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java
|
|||
|
|
+index 75ad961ce..6035a20ac 100644
|
|||
|
|
+--- a/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java
|
|||
|
|
++++ b/broker/src/test/java/org/apache/rocketmq/broker/BrokerControllerTest.java
|
|||
|
|
+@@ -23,9 +23,9 @@ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+-import org.apache.rocketmq.broker.latency.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
++import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyServerConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.RequestTask;
|
|||
|
|
+diff --git a/broker/src/test/java/org/apache/rocketmq/broker/latency/BrokerFastFailureTest.java b/broker/src/test/java/org/apache/rocketmq/broker/latency/BrokerFastFailureTest.java
|
|||
|
|
+index 5d0f7f9d7..31b547cf1 100644
|
|||
|
|
+--- a/broker/src/test/java/org/apache/rocketmq/broker/latency/BrokerFastFailureTest.java
|
|||
|
|
++++ b/broker/src/test/java/org/apache/rocketmq/broker/latency/BrokerFastFailureTest.java
|
|||
|
|
+@@ -19,6 +19,7 @@ package org.apache.rocketmq.broker.latency;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
++import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.RequestTask;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
+index a720a5be3..6f19a9815 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java
|
|||
|
|
+@@ -23,7 +23,6 @@ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.ArrayBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.Semaphore;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+@@ -33,6 +32,7 @@ import com.google.common.collect.Maps;
|
|||
|
|
+
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.rocksdb.ColumnFamilyDescriptor;
|
|||
|
|
+@@ -82,8 +82,8 @@ public abstract class AbstractRocksDBStorage {
|
|||
|
|
+ private volatile boolean closed;
|
|||
|
|
+
|
|||
|
|
+ private final Semaphore reloadPermit = new Semaphore(1);
|
|||
|
|
+- private final ScheduledExecutorService reloadScheduler = new ScheduledThreadPoolExecutor(1, new ThreadFactoryImpl("RocksDBStorageReloadService_"));
|
|||
|
|
+- private final ThreadPoolExecutor manualCompactionThread = new ThreadPoolExecutor(
|
|||
|
|
++ private final ScheduledExecutorService reloadScheduler = ThreadUtils.newScheduledThreadPool(1, new ThreadFactoryImpl("RocksDBStorageReloadService_"));
|
|||
|
|
++ private final ThreadPoolExecutor manualCompactionThread = (ThreadPoolExecutor) ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ 1, 1, 1000 * 60, TimeUnit.MILLISECONDS,
|
|||
|
|
+ new ArrayBlockingQueue(1),
|
|||
|
|
+ new ThreadFactoryImpl("RocksDBManualCompactionService_"),
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java b/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java
|
|||
|
|
+index 411da9221..7b68873a9 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/thread/FutureTaskExtThreadPoolExecutor.java
|
|||
|
|
+@@ -29,7 +29,8 @@ public class FutureTaskExtThreadPoolExecutor extends ThreadPoolExecutor {
|
|||
|
|
+
|
|||
|
|
+ public FutureTaskExtThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime,
|
|||
|
|
+ TimeUnit unit,
|
|||
|
|
+- BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory,
|
|||
|
|
++ BlockingQueue<Runnable> workQueue,
|
|||
|
|
++ ThreadFactory threadFactory,
|
|||
|
|
+ RejectedExecutionHandler handler) {
|
|||
|
|
+ super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java b/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
|
|||
|
|
+index 49d97a5d7..1bfabbffe 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/thread/ThreadPoolMonitor.java
|
|||
|
|
+@@ -22,12 +22,12 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+@@ -36,7 +36,7 @@ public class ThreadPoolMonitor {
|
|||
|
|
+ private static Logger waterMarkLogger = LoggerFactory.getLogger(ThreadPoolMonitor.class);
|
|||
|
|
+
|
|||
|
|
+ private static final List<ThreadPoolWrapper> MONITOR_EXECUTOR = new CopyOnWriteArrayList<>();
|
|||
|
|
+- private static final ScheduledExecutorService MONITOR_SCHEDULED = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ private static final ScheduledExecutorService MONITOR_SCHEDULED = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryBuilder().setNameFormat("ThreadPoolMonitor-%d").build()
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
+@@ -81,7 +81,7 @@ public class ThreadPoolMonitor {
|
|||
|
|
+ String name,
|
|||
|
|
+ int queueCapacity,
|
|||
|
|
+ List<ThreadPoolStatusMonitor> threadPoolStatusMonitors) {
|
|||
|
|
+- ThreadPoolExecutor executor = new FutureTaskExtThreadPoolExecutor(
|
|||
|
|
++ ThreadPoolExecutor executor = (ThreadPoolExecutor) ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ corePoolSize,
|
|||
|
|
+ maximumPoolSize,
|
|||
|
|
+ keepAliveTime,
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java b/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
|
|||
|
|
+index 4b366d4e3..1644c6360 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/utils/ThreadUtils.java
|
|||
|
|
+@@ -20,38 +20,94 @@ package org.apache.rocketmq.common.utils;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
++import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
++import java.util.concurrent.RejectedExecutionHandler;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.ThreadFactory;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.thread.FutureTaskExtThreadPoolExecutor;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+ public final class ThreadUtils {
|
|||
|
|
+ private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.TOOLS_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+- public static ExecutorService newThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime,
|
|||
|
|
+- TimeUnit unit, BlockingQueue<Runnable> workQueue, String processName, boolean isDaemon) {
|
|||
|
|
+- return new ThreadPoolExecutor(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, newThreadFactory(processName, isDaemon));
|
|||
|
|
++ public static ExecutorService newSingleThreadExecutor(String processName, boolean isDaemon) {
|
|||
|
|
++ return ThreadUtils.newSingleThreadExecutor(newThreadFactory(processName, isDaemon));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public static ExecutorService newSingleThreadExecutor(String processName, boolean isDaemon) {
|
|||
|
|
+- return Executors.newSingleThreadExecutor(newThreadFactory(processName, isDaemon));
|
|||
|
|
++ public static ExecutorService newSingleThreadExecutor(ThreadFactory threadFactory) {
|
|||
|
|
++ return ThreadUtils.newThreadPoolExecutor(1, threadFactory);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ExecutorService newThreadPoolExecutor(int corePoolSize, ThreadFactory threadFactory) {
|
|||
|
|
++ return ThreadUtils.newThreadPoolExecutor(corePoolSize, corePoolSize,
|
|||
|
|
++ 0L, TimeUnit.MILLISECONDS,
|
|||
|
|
++ new LinkedBlockingQueue<>(),
|
|||
|
|
++ threadFactory);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ExecutorService newThreadPoolExecutor(int corePoolSize,
|
|||
|
|
++ int maximumPoolSize,
|
|||
|
|
++ long keepAliveTime,
|
|||
|
|
++ TimeUnit unit, BlockingQueue<Runnable> workQueue,
|
|||
|
|
++ String processName,
|
|||
|
|
++ boolean isDaemon) {
|
|||
|
|
++ return ThreadUtils.newThreadPoolExecutor(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, newThreadFactory(processName, isDaemon));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ExecutorService newThreadPoolExecutor(final int corePoolSize,
|
|||
|
|
++ final int maximumPoolSize,
|
|||
|
|
++ final long keepAliveTime,
|
|||
|
|
++ final TimeUnit unit,
|
|||
|
|
++ final BlockingQueue<Runnable> workQueue,
|
|||
|
|
++ final ThreadFactory threadFactory) {
|
|||
|
|
++ return ThreadUtils.newThreadPoolExecutor(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, new ThreadPoolExecutor.AbortPolicy());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ExecutorService newThreadPoolExecutor(int corePoolSize,
|
|||
|
|
++ int maximumPoolSize,
|
|||
|
|
++ long keepAliveTime,
|
|||
|
|
++ TimeUnit unit,
|
|||
|
|
++ BlockingQueue<Runnable> workQueue,
|
|||
|
|
++ ThreadFactory threadFactory,
|
|||
|
|
++ RejectedExecutionHandler handler) {
|
|||
|
|
++ return new FutureTaskExtThreadPoolExecutor(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static ScheduledExecutorService newSingleThreadScheduledExecutor(String processName, boolean isDaemon) {
|
|||
|
|
+- return Executors.newSingleThreadScheduledExecutor(newThreadFactory(processName, isDaemon));
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(1, processName, isDaemon);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ScheduledExecutorService newSingleThreadScheduledExecutor(ThreadFactory threadFactory) {
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(1, threadFactory);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ScheduledExecutorService newScheduledThreadPool(int corePoolSize) {
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(corePoolSize, Executors.defaultThreadFactory());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public static ScheduledExecutorService newFixedThreadScheduledPool(int nThreads, String processName,
|
|||
|
|
++ public static ScheduledExecutorService newScheduledThreadPool(int corePoolSize, String processName,
|
|||
|
|
+ boolean isDaemon) {
|
|||
|
|
+- return Executors.newScheduledThreadPool(nThreads, newThreadFactory(processName, isDaemon));
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(corePoolSize, newThreadFactory(processName, isDaemon));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ScheduledExecutorService newScheduledThreadPool(int corePoolSize, ThreadFactory threadFactory) {
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(corePoolSize, threadFactory, new ThreadPoolExecutor.AbortPolicy());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static ScheduledExecutorService newScheduledThreadPool(int corePoolSize,
|
|||
|
|
++ ThreadFactory threadFactory,
|
|||
|
|
++ RejectedExecutionHandler handler) {
|
|||
|
|
++ return new ScheduledThreadPoolExecutor(corePoolSize, threadFactory, handler);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static ThreadFactory newThreadFactory(String processName, boolean isDaemon) {
|
|||
|
|
+- return newGenericThreadFactory("Remoting-" + processName, isDaemon);
|
|||
|
|
++ return newGenericThreadFactory("ThreadUtils-" + processName, isDaemon);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static ThreadFactory newGenericThreadFactory(String processName) {
|
|||
|
|
+diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java
|
|||
|
|
+index c6446f058..5b712bc30 100644
|
|||
|
|
+--- a/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java
|
|||
|
|
++++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java
|
|||
|
|
+@@ -47,14 +47,12 @@ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+ public class BrokerContainer implements IBrokerContainer {
|
|||
|
|
+ private static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+- private final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ private final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new BasicThreadFactory.Builder()
|
|||
|
|
+ .namingPattern("BrokerContainerScheduledThread")
|
|||
|
|
+ .daemon(true)
|
|||
|
|
+@@ -143,7 +141,7 @@ public class BrokerContainer implements IBrokerContainer {
|
|||
|
|
+ this.remotingServer = new NettyRemotingServer(this.nettyServerConfig, this.containerClientHouseKeepingService);
|
|||
|
|
+ this.fastRemotingServer = this.remotingServer.newRemotingServer(this.nettyServerConfig.getListenPort() - 2);
|
|||
|
|
+
|
|||
|
|
+- this.brokerContainerExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ this.brokerContainerExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ 1,
|
|||
|
|
+ 1,
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/ControllerManager.java b/controller/src/main/java/org/apache/rocketmq/controller/ControllerManager.java
|
|||
|
|
+index 7c91e70da..3e6b0eba5 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/ControllerManager.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/ControllerManager.java
|
|||
|
|
+@@ -25,8 +25,6 @@ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.Future;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+-import java.util.concurrent.RunnableFuture;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+@@ -34,8 +32,8 @@ import org.apache.rocketmq.common.ControllerConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+-import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.controller.elect.impl.DefaultElectPolicy;
|
|||
|
|
+ import org.apache.rocketmq.controller.impl.DLedgerController;
|
|||
|
|
+ import org.apache.rocketmq.controller.impl.heartbeat.DefaultBrokerHeartbeatManager;
|
|||
|
|
+@@ -93,18 +91,14 @@ public class ControllerManager {
|
|||
|
|
+
|
|||
|
|
+ public boolean initialize() {
|
|||
|
|
+ this.controllerRequestThreadPoolQueue = new LinkedBlockingQueue<>(this.controllerConfig.getControllerRequestThreadPoolQueueCapacity());
|
|||
|
|
+- this.controllerRequestExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ this.controllerRequestExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.controllerConfig.getControllerThreadPoolNums(),
|
|||
|
|
+ this.controllerConfig.getControllerThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+ TimeUnit.MILLISECONDS,
|
|||
|
|
+ this.controllerRequestThreadPoolQueue,
|
|||
|
|
+- new ThreadFactoryImpl("ControllerRequestExecutorThread_")) {
|
|||
|
|
+- @Override
|
|||
|
|
+- protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
|
|||
|
|
+- return new FutureTaskExt<T>(runnable, value);
|
|||
|
|
+- }
|
|||
|
|
+- };
|
|||
|
|
++ new ThreadFactoryImpl("ControllerRequestExecutorThread_"));
|
|||
|
|
++
|
|||
|
|
+ this.notifyService.initialize();
|
|||
|
|
+ if (StringUtils.isEmpty(this.controllerConfig.getControllerDLegerPeers())) {
|
|||
|
|
+ throw new IllegalArgumentException("Attribute value controllerDLegerPeers of ControllerConfig is null or empty");
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/impl/DLedgerController.java b/controller/src/main/java/org/apache/rocketmq/controller/impl/DLedgerController.java
|
|||
|
|
+index fa91f288e..33e4406e4 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/impl/DLedgerController.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/impl/DLedgerController.java
|
|||
|
|
+@@ -32,7 +32,6 @@ import java.util.Map;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ScheduledFuture;
|
|||
|
|
+@@ -44,6 +43,7 @@ import org.apache.rocketmq.common.ControllerConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.ServiceThread;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.controller.Controller;
|
|||
|
|
+ import org.apache.rocketmq.controller.elect.ElectPolicy;
|
|||
|
|
+ import org.apache.rocketmq.controller.elect.impl.DefaultElectPolicy;
|
|||
|
|
+@@ -66,11 +66,11 @@ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.SyncStateSet;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.AlterSyncStateSetRequestHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.admin.CleanControllerBrokerDataRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.GetReplicaInfoRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.admin.CleanControllerBrokerDataRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.register.ApplyBrokerIdRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.register.GetNextBrokerIdRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.register.RegisterBrokerToControllerRequestHeader;
|
|||
|
|
+@@ -136,7 +136,7 @@ public class DLedgerController implements Controller {
|
|||
|
|
+ this.dLedgerServer = new DLedgerServer(dLedgerConfig, nettyServerConfig, nettyClientConfig, channelEventListener);
|
|||
|
|
+ this.dLedgerServer.registerStateMachine(this.statemachine);
|
|||
|
|
+ this.dLedgerServer.getDLedgerLeaderElector().addRoleChangeHandler(this.roleHandler);
|
|||
|
|
+- this.scanInactiveMasterService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DLedgerController_scanInactiveService_"));
|
|||
|
|
++ this.scanInactiveMasterService = ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DLedgerController_scanInactiveService_"));
|
|||
|
|
+ this.brokerLifecycleListeners = new ArrayList<>();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -513,7 +513,7 @@ public class DLedgerController implements Controller {
|
|||
|
|
+ class RoleChangeHandler implements DLedgerLeaderElector.RoleChangeHandler {
|
|||
|
|
+
|
|||
|
|
+ private final String selfId;
|
|||
|
|
+- private final ExecutorService executorService = Executors.newSingleThreadExecutor(new ThreadFactoryImpl("DLedgerControllerRoleChangeHandler_"));
|
|||
|
|
++ private final ExecutorService executorService = ThreadUtils.newSingleThreadExecutor(new ThreadFactoryImpl("DLedgerControllerRoleChangeHandler_"));
|
|||
|
|
+ private volatile MemberState.Role currentRole = MemberState.Role.FOLLOWER;
|
|||
|
|
+
|
|||
|
|
+ public RoleChangeHandler(final String selfId) {
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/DefaultBrokerHeartbeatManager.java b/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/DefaultBrokerHeartbeatManager.java
|
|||
|
|
+index 2fbddb9cd..6ebb2c994 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/DefaultBrokerHeartbeatManager.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/DefaultBrokerHeartbeatManager.java
|
|||
|
|
+@@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.ControllerConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.controller.BrokerHeartbeatManager;
|
|||
|
|
+ import org.apache.rocketmq.controller.helper.BrokerLifecycleListener;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+@@ -66,7 +67,7 @@ public class DefaultBrokerHeartbeatManager implements BrokerHeartbeatManager {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void initialize() {
|
|||
|
|
+- this.scheduledService = Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DefaultBrokerHeartbeatManager_scheduledService_"));
|
|||
|
|
++ this.scheduledService = ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("DefaultBrokerHeartbeatManager_scheduledService_"));
|
|||
|
|
+ this.executor = Executors.newFixedThreadPool(2, new ThreadFactoryImpl("DefaultBrokerHeartbeatManager_executorService_"));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
|
|||
|
|
+index 15c65ebec..be327cffa 100644
|
|||
|
|
+--- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
|
|||
|
|
++++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/NamesrvController.java
|
|||
|
|
+@@ -20,10 +20,7 @@ import java.util.Collections;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+-import java.util.concurrent.RunnableFuture;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.lang3.concurrent.BasicThreadFactory;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+@@ -31,6 +28,7 @@ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.common.namesrv.NamesrvConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.namesrv.kvconfig.KVConfigManager;
|
|||
|
|
+@@ -62,10 +60,10 @@ public class NamesrvController {
|
|||
|
|
+ private final NettyServerConfig nettyServerConfig;
|
|||
|
|
+ private final NettyClientConfig nettyClientConfig;
|
|||
|
|
+
|
|||
|
|
+- private final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ private final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new BasicThreadFactory.Builder().namingPattern("NSScheduledThread").daemon(true).build());
|
|||
|
|
+
|
|||
|
|
+- private final ScheduledExecutorService scanExecutorService = new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ private final ScheduledExecutorService scanExecutorService = ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new BasicThreadFactory.Builder().namingPattern("NSScanScheduledThread").daemon(true).build());
|
|||
|
|
+
|
|||
|
|
+ private final KVConfigManager kvConfigManager;
|
|||
|
|
+@@ -138,20 +136,10 @@ public class NamesrvController {
|
|||
|
|
+
|
|||
|
|
+ private void initiateThreadExecutors() {
|
|||
|
|
+ this.defaultThreadPoolQueue = new LinkedBlockingQueue<>(this.namesrvConfig.getDefaultThreadPoolQueueCapacity());
|
|||
|
|
+- this.defaultExecutor = new ThreadPoolExecutor(this.namesrvConfig.getDefaultThreadPoolNums(), this.namesrvConfig.getDefaultThreadPoolNums(), 1000 * 60, TimeUnit.MILLISECONDS, this.defaultThreadPoolQueue, new ThreadFactoryImpl("RemotingExecutorThread_")) {
|
|||
|
|
+- @Override
|
|||
|
|
+- protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
|
|||
|
|
+- return new FutureTaskExt<>(runnable, value);
|
|||
|
|
+- }
|
|||
|
|
+- };
|
|||
|
|
++ this.defaultExecutor = ThreadUtils.newThreadPoolExecutor(this.namesrvConfig.getDefaultThreadPoolNums(), this.namesrvConfig.getDefaultThreadPoolNums(), 1000 * 60, TimeUnit.MILLISECONDS, this.defaultThreadPoolQueue, new ThreadFactoryImpl("RemotingExecutorThread_"));
|
|||
|
|
+
|
|||
|
|
+ this.clientRequestThreadPoolQueue = new LinkedBlockingQueue<>(this.namesrvConfig.getClientRequestThreadPoolQueueCapacity());
|
|||
|
|
+- this.clientRequestExecutor = new ThreadPoolExecutor(this.namesrvConfig.getClientRequestThreadPoolNums(), this.namesrvConfig.getClientRequestThreadPoolNums(), 1000 * 60, TimeUnit.MILLISECONDS, this.clientRequestThreadPoolQueue, new ThreadFactoryImpl("ClientRequestExecutorThread_")) {
|
|||
|
|
+- @Override
|
|||
|
|
+- protected <T> RunnableFuture<T> newTaskFor(final Runnable runnable, final T value) {
|
|||
|
|
+- return new FutureTaskExt<>(runnable, value);
|
|||
|
|
+- }
|
|||
|
|
+- };
|
|||
|
|
++ this.clientRequestExecutor = ThreadUtils.newThreadPoolExecutor(this.namesrvConfig.getClientRequestThreadPoolNums(), this.namesrvConfig.getClientRequestThreadPoolNums(), 1000 * 60, TimeUnit.MILLISECONDS, this.clientRequestThreadPoolQueue, new ThreadFactoryImpl("ClientRequestExecutorThread_"));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void initiateSslContext() {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcChannelManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcChannelManager.java
|
|||
|
|
+index 14330dd8d..a18cf7600 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcChannelManager.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/channel/GrpcChannelManager.java
|
|||
|
|
+@@ -21,13 +21,13 @@ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+-import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.proxy.grpc.v2.common.GrpcClientSettingsManager;
|
|||
|
|
+@@ -43,7 +43,7 @@ public class GrpcChannelManager implements StartAndShutdown {
|
|||
|
|
+ protected final AtomicLong nonceIdGenerator = new AtomicLong(0);
|
|||
|
|
+ protected final ConcurrentMap<String /* nonce */, ResultFuture> resultNonceFutureMap = new ConcurrentHashMap<>();
|
|||
|
|
+
|
|||
|
|
+- protected final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ protected final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("GrpcChannelManager_")
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
+index bcc9edd09..fe07090d5 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
+@@ -22,7 +22,6 @@ import io.netty.channel.Channel;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+@@ -33,6 +32,7 @@ import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.common.thread.ThreadPoolMonitor;
|
|||
|
|
+ import org.apache.rocketmq.common.thread.ThreadPoolStatusMonitor;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+@@ -178,7 +178,7 @@ public class RemotingProtocolServer implements StartAndShutdown, RemotingProxyOu
|
|||
|
|
+ new ThreadPoolHeadSlowTimeMillsMonitor(config.getRemotingWaitTimeMillsInDefaultQueue())
|
|||
|
|
+ );
|
|||
|
|
+
|
|||
|
|
+- this.timerExecutor = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ this.timerExecutor = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryBuilder().setNameFormat("RemotingServerScheduler-%d").build()
|
|||
|
|
+ );
|
|||
|
|
+ this.timerExecutor.scheduleAtFixedRate(this::cleanExpireRequest, 10, 10, TimeUnit.SECONDS);
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java
|
|||
|
|
+index d2ddfc352..9786cec55 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/ClusterServiceManager.java
|
|||
|
|
+@@ -16,7 +16,6 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.proxy.service;
|
|||
|
|
+
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.client.ClientChannelInfo;
|
|||
|
|
+@@ -27,23 +26,24 @@ import org.apache.rocketmq.broker.client.ProducerChangeListener;
|
|||
|
|
+ import org.apache.rocketmq.broker.client.ProducerGroupEvent;
|
|||
|
|
+ import org.apache.rocketmq.broker.client.ProducerManager;
|
|||
|
|
+ import org.apache.rocketmq.client.common.NameserverAccessConfig;
|
|||
|
|
++import org.apache.rocketmq.client.impl.mqclient.DoNothingClientRemotingProcessor;
|
|||
|
|
++import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.AbstractStartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+-import org.apache.rocketmq.common.utils.AbstractStartAndShutdown;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.admin.AdminService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.admin.DefaultAdminService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.client.ClusterConsumerManager;
|
|||
|
|
++import org.apache.rocketmq.proxy.service.client.ProxyClientRemotingProcessor;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.message.ClusterMessageService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.message.MessageService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.ClusterMetadataService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+-import org.apache.rocketmq.client.impl.mqclient.DoNothingClientRemotingProcessor;
|
|||
|
|
+-import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+-import org.apache.rocketmq.proxy.service.client.ProxyClientRemotingProcessor;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ClusterProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.relay.ProxyRelayService;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.route.ClusterTopicRouteService;
|
|||
|
|
+@@ -73,7 +73,7 @@ public class ClusterServiceManager extends AbstractStartAndShutdown implements S
|
|||
|
|
+ ProxyConfig proxyConfig = ConfigurationManager.getProxyConfig();
|
|||
|
|
+ NameserverAccessConfig nameserverAccessConfig = new NameserverAccessConfig(proxyConfig.getNamesrvAddr(),
|
|||
|
|
+ proxyConfig.getNamesrvDomain(), proxyConfig.getNamesrvDomainSubgroup());
|
|||
|
|
+- this.scheduledExecutorService = Executors.newScheduledThreadPool(3);
|
|||
|
|
++ this.scheduledExecutorService = ThreadUtils.newScheduledThreadPool(3);
|
|||
|
|
+
|
|||
|
|
+ this.messagingClientAPIFactory = new MQClientAPIFactory(
|
|||
|
|
+ nameserverAccessConfig,
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java
|
|||
|
|
+index 4d1ca7b66..59cd92685 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/LocalServiceManager.java
|
|||
|
|
+@@ -16,7 +16,6 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.proxy.service;
|
|||
|
|
+
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+@@ -28,6 +27,7 @@ import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.AbstractStartAndShutdown;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.admin.AdminService;
|
|||
|
|
+@@ -58,7 +58,7 @@ public class LocalServiceManager extends AbstractStartAndShutdown implements Ser
|
|||
|
|
+ private final MQClientAPIFactory mqClientAPIFactory;
|
|||
|
|
+ private final ChannelManager channelManager;
|
|||
|
|
+
|
|||
|
|
+- private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ private final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("LocalServiceManagerScheduledThread"));
|
|||
|
|
+
|
|||
|
|
+ public LocalServiceManager(BrokerController brokerController, RPCHook rpcHook) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java
|
|||
|
|
+index 69f44344a..207603fe8 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/receipt/DefaultReceiptHandleManager.java
|
|||
|
|
+@@ -24,7 +24,6 @@ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+@@ -42,20 +41,21 @@ import org.apache.rocketmq.common.thread.ThreadPoolMonitor;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.AbstractStartAndShutdown;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.ConcurrentHashMapUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+-import org.apache.rocketmq.proxy.common.RenewEvent;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.MessageReceiptHandle;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyContext;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyException;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ProxyExceptionCode;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.ReceiptHandleGroup;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.ReceiptHandleGroupKey;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.RenewEvent;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.RenewStrategyPolicy;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.channel.ChannelHelper;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.ExceptionUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+-import org.apache.rocketmq.proxy.common.ReceiptHandleGroupKey;
|
|||
|
|
+ import org.apache.rocketmq.proxy.service.metadata.MetadataService;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.subscription.RetryPolicy;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig;
|
|||
|
|
+@@ -68,7 +68,7 @@ public class DefaultReceiptHandleManager extends AbstractStartAndShutdown implem
|
|||
|
|
+ protected final StateEventListener<RenewEvent> eventListener;
|
|||
|
|
+ protected final static RetryPolicy RENEW_POLICY = new RenewStrategyPolicy();
|
|||
|
|
+ protected final ScheduledExecutorService scheduledExecutorService =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("RenewalScheduledThread_"));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("RenewalScheduledThread_"));
|
|||
|
|
+ protected final ThreadPoolExecutor renewalWorkerService;
|
|||
|
|
+
|
|||
|
|
+ public DefaultReceiptHandleManager(MetadataService metadataService, ConsumerManager consumerManager, StateEventListener<RenewEvent> eventListener) {
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+index caf62a1e0..ccf094c03 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/route/TopicRouteService.java
|
|||
|
|
+@@ -19,25 +19,24 @@ package org.apache.rocketmq.proxy.service.route;
|
|||
|
|
+ import com.github.benmanes.caffeine.cache.CacheLoader;
|
|||
|
|
+ import com.github.benmanes.caffeine.cache.Caffeine;
|
|||
|
|
+ import com.github.benmanes.caffeine.cache.LoadingCache;
|
|||
|
|
++import com.google.common.base.Optional;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+-
|
|||
|
|
+-import com.google.common.base.Optional;
|
|||
|
|
+ import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.client.exception.MQClientException;
|
|||
|
|
++import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.client.latency.MQFaultStrategy;
|
|||
|
|
+ import org.apache.rocketmq.client.latency.Resolver;
|
|||
|
|
+ import org.apache.rocketmq.client.latency.ServiceDetector;
|
|||
|
|
+-import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.common.thread.ThreadPoolMonitor;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.AbstractStartAndShutdown;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.Address;
|
|||
|
|
+@@ -63,7 +62,7 @@ public abstract class TopicRouteService extends AbstractStartAndShutdown {
|
|||
|
|
+ public TopicRouteService(MQClientAPIFactory mqClientAPIFactory) {
|
|||
|
|
+ ProxyConfig config = ConfigurationManager.getProxyConfig();
|
|||
|
|
+
|
|||
|
|
+- this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ this.scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("TopicRouteService_")
|
|||
|
|
+ );
|
|||
|
|
+ this.cacheRefreshExecutor = ThreadPoolMonitor.createAndMonitor(
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+index 8491f4354..64621dd6c 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+@@ -61,7 +61,6 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
|
+@@ -71,6 +70,7 @@ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
+@@ -142,7 +142,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+
|
|||
|
|
+ this.publicExecutor = Executors.newFixedThreadPool(publicThreadNums, new ThreadFactoryImpl("NettyClientPublicExecutor_"));
|
|||
|
|
+
|
|||
|
|
+- this.scanExecutor = new ThreadPoolExecutor(4, 10, 60, TimeUnit.SECONDS,
|
|||
|
|
++ this.scanExecutor = ThreadUtils.newThreadPoolExecutor(4, 10, 60, TimeUnit.SECONDS,
|
|||
|
|
+ new ArrayBlockingQueue<>(32), new ThreadFactoryImpl("NettyClientScan_thread_"));
|
|||
|
|
+
|
|||
|
|
+ if (eventLoopGroup != null) {
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+index e626260c9..aa0d46542 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+@@ -61,6 +61,7 @@ import org.apache.rocketmq.common.constant.HAProxyConstants;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.BinaryUtil;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
+@@ -83,7 +84,6 @@ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+@@ -171,7 +171,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ScheduledExecutorService buildScheduleExecutor() {
|
|||
|
|
+- return new ScheduledThreadPoolExecutor(1,
|
|||
|
|
++ return ThreadUtils.newScheduledThreadPool(1,
|
|||
|
|
+ new ThreadFactoryImpl("NettyServerScheduler_", true),
|
|||
|
|
+ new ThreadPoolExecutor.DiscardOldestPolicy());
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+index f2a54ddf6..02ea47f13 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+@@ -48,7 +48,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+@@ -83,6 +82,7 @@ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.CleanupPolicyUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.QueueTypeUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.ServiceProvider;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo;
|
|||
|
|
+@@ -205,7 +205,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ private ConcurrentMap<String, TopicConfig> topicConfigTable;
|
|||
|
|
+
|
|||
|
|
+ private final ScheduledExecutorService scheduledCleanQueueExecutorService =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreCleanQueueScheduledThread"));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreCleanQueueScheduledThread"));
|
|||
|
|
+
|
|||
|
|
+ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
|
|||
|
|
+ final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig, final ConcurrentMap<String, TopicConfig> topicConfigTable) throws IOException {
|
|||
|
|
+@@ -253,7 +253,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ this.transientStorePool = new TransientStorePool(messageStoreConfig.getTransientStorePoolSize(), messageStoreConfig.getMappedFileSizeCommitLog());
|
|||
|
|
+
|
|||
|
|
+ this.scheduledExecutorService =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread", getBrokerIdentity()));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("StoreScheduledThread", getBrokerIdentity()));
|
|||
|
|
+
|
|||
|
|
+ this.dispatcherList = new LinkedList<>();
|
|||
|
|
+ this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue());
|
|||
|
|
+@@ -2915,7 +2915,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
+ private final ExecutorService batchDispatchRequestExecutor;
|
|||
|
|
+
|
|||
|
|
+ public MainBatchDispatchRequestService() {
|
|||
|
|
+- batchDispatchRequestExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ batchDispatchRequestExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ DefaultMessageStore.this.getMessageStoreConfig().getBatchDispatchRequestThreadPoolNums(),
|
|||
|
|
+ DefaultMessageStore.this.getMessageStoreConfig().getBatchDispatchRequestThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+index d5393fdca..f20bc3e28 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+@@ -17,10 +17,26 @@
|
|||
|
|
+
|
|||
|
|
+ package org.apache.rocketmq.store.ha.autoswitch;
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
++import java.io.IOException;
|
|||
|
|
++import java.nio.channels.SocketChannel;
|
|||
|
|
++import java.util.ArrayList;
|
|||
|
|
++import java.util.HashSet;
|
|||
|
|
++import java.util.Iterator;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++import java.util.Map;
|
|||
|
|
++import java.util.Objects;
|
|||
|
|
++import java.util.Set;
|
|||
|
|
++import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
++import java.util.concurrent.ExecutorService;
|
|||
|
|
++import java.util.concurrent.locks.Lock;
|
|||
|
|
++import java.util.concurrent.locks.ReadWriteLock;
|
|||
|
|
++import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
|
|
++import java.util.function.Consumer;
|
|||
|
|
++import java.util.stream.Collectors;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.ConcurrentHashMapUtils;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.EpochEntry;
|
|||
|
|
+@@ -36,30 +52,12 @@ import org.apache.rocketmq.store.ha.HAClient;
|
|||
|
|
+ import org.apache.rocketmq.store.ha.HAConnection;
|
|||
|
|
+ import org.apache.rocketmq.store.ha.HAConnectionStateNotificationService;
|
|||
|
|
+
|
|||
|
|
+-import java.io.IOException;
|
|||
|
|
+-import java.nio.channels.SocketChannel;
|
|||
|
|
+-import java.util.ArrayList;
|
|||
|
|
+-import java.util.HashSet;
|
|||
|
|
+-import java.util.List;
|
|||
|
|
+-import java.util.Iterator;
|
|||
|
|
+-import java.util.Map;
|
|||
|
|
+-import java.util.Objects;
|
|||
|
|
+-import java.util.Set;
|
|||
|
|
+-import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+-import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+-import java.util.concurrent.locks.Lock;
|
|||
|
|
+-import java.util.concurrent.locks.ReadWriteLock;
|
|||
|
|
+-import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
|
|
+-import java.util.function.Consumer;
|
|||
|
|
+-import java.util.stream.Collectors;
|
|||
|
|
+-
|
|||
|
|
+ /**
|
|||
|
|
+ * SwitchAble ha service, support switch role to master or slave.
|
|||
|
|
+ */
|
|||
|
|
+ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
+ private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+- private final ExecutorService executorService = Executors.newSingleThreadExecutor(new ThreadFactoryImpl("AutoSwitchHAService_Executor_"));
|
|||
|
|
++ private final ExecutorService executorService = ThreadUtils.newSingleThreadExecutor(new ThreadFactoryImpl("AutoSwitchHAService_Executor_"));
|
|||
|
|
+ private final ConcurrentHashMap<Long/*brokerId*/, Long/*lastCaughtUpTimestamp*/> connectionCaughtUpTimeTable = new ConcurrentHashMap<>();
|
|||
|
|
+ private final List<Consumer<Set<Long/*brokerId*/>>> syncStateSetChangedListeners = new ArrayList<>();
|
|||
|
|
+ private final Set<Long/*brokerId*/> syncStateSet = new HashSet<>();
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/kv/CompactionStore.java b/store/src/main/java/org/apache/rocketmq/store/kv/CompactionStore.java
|
|||
|
|
+index b37c90726..639084fa2 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/kv/CompactionStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/kv/CompactionStore.java
|
|||
|
|
+@@ -16,17 +16,25 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.store.kv;
|
|||
|
|
+
|
|||
|
|
+-import java.util.Random;
|
|||
|
|
++import java.io.File;
|
|||
|
|
++import java.io.IOException;
|
|||
|
|
++import java.nio.file.Files;
|
|||
|
|
++import java.nio.file.Paths;
|
|||
|
|
+ import java.util.Iterator;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Objects;
|
|||
|
|
+ import java.util.Optional;
|
|||
|
|
++import java.util.Random;
|
|||
|
|
++import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
++import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+ import org.apache.rocketmq.common.attribute.CleanupPolicy;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.CleanupPolicyUtils;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+@@ -35,15 +43,6 @@ import org.apache.rocketmq.store.GetMessageResult;
|
|||
|
|
+ import org.apache.rocketmq.store.SelectMappedBufferResult;
|
|||
|
|
+ import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+
|
|||
|
|
+-import java.io.File;
|
|||
|
|
+-import java.io.IOException;
|
|||
|
|
+-import java.nio.file.Files;
|
|||
|
|
+-import java.nio.file.Paths;
|
|||
|
|
+-import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+-import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-
|
|||
|
|
+ public class CompactionStore {
|
|||
|
|
+
|
|||
|
|
+ public static final String COMPACTION_DIR = "compaction";
|
|||
|
|
+@@ -76,7 +75,7 @@ public class CompactionStore {
|
|||
|
|
+ this.positionMgr = new CompactionPositionMgr(compactionPath);
|
|||
|
|
+ this.compactionThreadNum = Math.min(Runtime.getRuntime().availableProcessors(), Math.max(1, config.getCompactionThreadNum()));
|
|||
|
|
+
|
|||
|
|
+- this.compactionSchedule = Executors.newScheduledThreadPool(this.compactionThreadNum,
|
|||
|
|
++ this.compactionSchedule = ThreadUtils.newScheduledThreadPool(this.compactionThreadNum,
|
|||
|
|
+ new ThreadFactoryImpl("compactionSchedule_"));
|
|||
|
|
+ this.offsetMapSize = config.getMaxOffsetMapSize() / compactionThreadNum;
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
+index 8d38503b3..d03d15d65 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
+@@ -24,7 +24,6 @@ import java.util.concurrent.CountDownLatch;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.FutureTask;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+@@ -34,6 +33,7 @@ import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.QueueTypeUtils;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.store.CommitLog;
|
|||
|
|
+@@ -175,7 +175,7 @@ public class ConsumeQueueStore {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ExecutorService buildExecutorService(BlockingQueue<Runnable> blockingQueue, String threadNamePrefix) {
|
|||
|
|
+- return new ThreadPoolExecutor(
|
|||
|
|
++ return ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ this.messageStore.getBrokerConfig().getRecoverThreadPoolNums(),
|
|||
|
|
+ this.messageStore.getBrokerConfig().getRecoverThreadPoolNums(),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
|
|||
|
|
+index 2dd3fc5b5..489d7b4fb 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/stats/BrokerStatsManager.java
|
|||
|
|
+@@ -17,7 +17,6 @@
|
|||
|
|
+ package org.apache.rocketmq.store.stats;
|
|||
|
|
+
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import org.apache.commons.lang3.tuple.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
+@@ -32,13 +31,14 @@ import org.apache.rocketmq.common.statistics.StatisticsItemScheduledPrinter;
|
|||
|
|
+ import org.apache.rocketmq.common.statistics.StatisticsItemStateGetter;
|
|||
|
|
+ import org.apache.rocketmq.common.statistics.StatisticsKindMeta;
|
|||
|
|
+ import org.apache.rocketmq.common.statistics.StatisticsManager;
|
|||
|
|
++import org.apache.rocketmq.common.stats.MomentStatsItemSet;
|
|||
|
|
+ import org.apache.rocketmq.common.stats.Stats;
|
|||
|
|
++import org.apache.rocketmq.common.stats.StatsItem;
|
|||
|
|
++import org.apache.rocketmq.common.stats.StatsItemSet;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+-import org.apache.rocketmq.common.stats.MomentStatsItemSet;
|
|||
|
|
+-import org.apache.rocketmq.common.stats.StatsItem;
|
|||
|
|
+-import org.apache.rocketmq.common.stats.StatsItemSet;
|
|||
|
|
+
|
|||
|
|
+ public class BrokerStatsManager {
|
|||
|
|
+
|
|||
|
|
+@@ -281,11 +281,11 @@ public class BrokerStatsManager {
|
|||
|
|
+
|
|||
|
|
+ private void initScheduleService() {
|
|||
|
|
+ this.scheduledExecutorService =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("BrokerStatsThread", true, brokerConfig));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("BrokerStatsThread", true, brokerConfig));
|
|||
|
|
+ this.commercialExecutor =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("CommercialStatsThread", true, brokerConfig));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("CommercialStatsThread", true, brokerConfig));
|
|||
|
|
+ this.accountExecutor =
|
|||
|
|
+- Executors.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("AccountStatsThread", true, brokerConfig));
|
|||
|
|
++ ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("AccountStatsThread", true, brokerConfig));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MomentStatsItemSet getMomentStatsItemSetFallSize() {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+index 181f7087a..0d50de65a 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+@@ -35,7 +35,6 @@ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ConcurrentSkipListSet;
|
|||
|
|
+ import java.util.concurrent.CountDownLatch;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingDeque;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+@@ -54,6 +53,7 @@ import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.store.ConsumeQueue;
|
|||
|
|
+@@ -174,11 +174,11 @@ public class TimerMessageStore {
|
|||
|
|
+ this.lastBrokerRole = storeConfig.getBrokerRole();
|
|||
|
|
+
|
|||
|
|
+ if (messageStore instanceof DefaultMessageStore) {
|
|||
|
|
+- scheduler = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ scheduler = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("TimerScheduledThread",
|
|||
|
|
+ ((DefaultMessageStore) messageStore).getBrokerIdentity()));
|
|||
|
|
+ } else {
|
|||
|
|
+- scheduler = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
++ scheduler = ThreadUtils.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("TimerScheduledThread"));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java b/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java
|
|||
|
|
+index f3d105bc6..080b7e385 100644
|
|||
|
|
+--- a/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java
|
|||
|
|
++++ b/test/src/main/java/org/apache/rocketmq/test/util/StatUtil.java
|
|||
|
|
+@@ -28,7 +28,6 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicLong;
|
|||
|
|
+-
|
|||
|
|
+ import javax.annotation.Generated;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
+index 6dd0e8846..65d586f43 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/common/TieredStoreExecutor.java
|
|||
|
|
+@@ -20,10 +20,10 @@ import java.util.concurrent.BlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ScheduledThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+
|
|||
|
|
+ public class TieredStoreExecutor {
|
|||
|
|
+
|
|||
|
|
+@@ -43,20 +43,20 @@ public class TieredStoreExecutor {
|
|||
|
|
+ public static ExecutorService compactIndexFileExecutor;
|
|||
|
|
+
|
|||
|
|
+ public static void init() {
|
|||
|
|
+- commonScheduledExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
++ commonScheduledExecutor = ThreadUtils.newScheduledThreadPool(
|
|||
|
|
+ Math.max(4, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+ new ThreadFactoryImpl("TieredCommonExecutor_"));
|
|||
|
|
+
|
|||
|
|
+- commitExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
++ commitExecutor = ThreadUtils.newScheduledThreadPool(
|
|||
|
|
+ Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
+ new ThreadFactoryImpl("TieredCommitExecutor_"));
|
|||
|
|
+
|
|||
|
|
+- cleanExpiredFileExecutor = new ScheduledThreadPoolExecutor(
|
|||
|
|
++ cleanExpiredFileExecutor = ThreadUtils.newScheduledThreadPool(
|
|||
|
|
+ Math.max(4, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+ new ThreadFactoryImpl("TieredCleanFileExecutor_"));
|
|||
|
|
+
|
|||
|
|
+ dispatchThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+- dispatchExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ dispatchExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ Math.max(2, Runtime.getRuntime().availableProcessors()),
|
|||
|
|
+ Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -66,7 +66,7 @@ public class TieredStoreExecutor {
|
|||
|
|
+ new ThreadPoolExecutor.DiscardOldestPolicy());
|
|||
|
|
+
|
|||
|
|
+ fetchDataThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+- fetchDataExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ fetchDataExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ Math.max(16, Runtime.getRuntime().availableProcessors() * 4),
|
|||
|
|
+ Math.max(64, Runtime.getRuntime().availableProcessors() * 8),
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+@@ -75,7 +75,7 @@ public class TieredStoreExecutor {
|
|||
|
|
+ new ThreadFactoryImpl("TieredFetchExecutor_"));
|
|||
|
|
+
|
|||
|
|
+ compactIndexFileThreadPoolQueue = new LinkedBlockingQueue<>(QUEUE_CAPACITY);
|
|||
|
|
+- compactIndexFileExecutor = new ThreadPoolExecutor(
|
|||
|
|
++ compactIndexFileExecutor = ThreadUtils.newThreadPoolExecutor(
|
|||
|
|
+ 1,
|
|||
|
|
+ 1,
|
|||
|
|
+ 1000 * 60,
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
|
|||
|
|
+index fa3596d51..1ebff6d8a 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java
|
|||
|
|
+@@ -66,6 +66,7 @@ import org.apache.rocketmq.common.namesrv.NamesrvUtil;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
+ import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
++import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+@@ -193,7 +194,7 @@ public class DefaultMQAdminExtImpl implements MQAdminExt, MQAdminExtInner {
|
|||
|
|
+
|
|||
|
|
+ int threadPoolCoreSize = Integer.parseInt(System.getProperty("rocketmq.admin.threadpool.coresize", "20"));
|
|||
|
|
+
|
|||
|
|
+- this.threadPoolExecutor = new ThreadPoolExecutor(threadPoolCoreSize, 100, 5, TimeUnit.MINUTES, new LinkedBlockingQueue<>(), new ThreadFactoryImpl("DefaultMQAdminExtImpl_"));
|
|||
|
|
++ this.threadPoolExecutor = (ThreadPoolExecutor) ThreadUtils.newThreadPoolExecutor(threadPoolCoreSize, 100, 5, TimeUnit.MINUTES, new LinkedBlockingQueue<>(), new ThreadFactoryImpl("DefaultMQAdminExtImpl_"));
|
|||
|
|
+
|
|||
|
|
+ break;
|
|||
|
|
+ case RUNNING:
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From dad6b4dadfec7a58e78a6715ec16c2eb6b17ff27 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <ajb459684460@gmail.com>
|
|||
|
|
+Date: Mon, 11 Sep 2023 14:34:10 +0800
|
|||
|
|
+Subject: [PATCH 2/6] [ISSUE #7334] `registerIncrementBrokerData` for single
|
|||
|
|
+ topic update (#7335)
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb459684460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../broker/topic/TopicConfigManager.java | 30 +++++++++++++++----
|
|||
|
|
+ 1 file changed, 25 insertions(+), 5 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+index 4e3c1736c..754605438 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+@@ -290,7 +290,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- this.brokerController.registerBrokerAll(false, true, true);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -394,7 +398,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- this.brokerController.registerBrokerAll(false, true, true);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -435,7 +443,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- this.brokerController.registerBrokerAll(false, true, true);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -461,7 +473,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ dataVersion.nextVersion(stateMachineVersion);
|
|||
|
|
+
|
|||
|
|
+ this.persist();
|
|||
|
|
+- this.brokerController.registerBrokerAll(false, true, true);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -484,7 +500,11 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ dataVersion.nextVersion(stateMachineVersion);
|
|||
|
|
+
|
|||
|
|
+ this.persist();
|
|||
|
|
+- this.brokerController.registerBrokerAll(false, true, true);
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 0dbd0772b99f618f757d42cd64542b83e2100e4f Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ziyi Tan <ajb459684460@gmail.com>
|
|||
|
|
+Date: Mon, 11 Sep 2023 15:48:07 +0800
|
|||
|
|
+Subject: [PATCH 3/6] [ISSUE #7326] Split the request to register to the
|
|||
|
|
+ nameserver (#7325)
|
|||
|
|
+
|
|||
|
|
+Signed-off-by: Ziy1-Tan <ajb459684460@gmail.com>
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/broker/BrokerController.java | 41 +++++++++++--------
|
|||
|
|
+ .../broker/topic/TopicConfigManager.java | 21 ++++++++++
|
|||
|
|
+ .../apache/rocketmq/common/BrokerConfig.java | 24 +++++++++++
|
|||
|
|
+ .../test/route/CreateAndUpdateTopicIT.java | 31 ++++++++++++++
|
|||
|
|
+ 4 files changed, 99 insertions(+), 18 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 275b64b1a..9e49f636d 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1765,29 +1765,34 @@ public class BrokerController {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public synchronized void registerBrokerAll(final boolean checkOrderConfig, boolean oneway, boolean forceRegister) {
|
|||
|
|
++ ConcurrentMap<String, TopicConfig> topicConfigMap = this.getTopicConfigManager().getTopicConfigTable();
|
|||
|
|
++ ConcurrentHashMap<String, TopicConfig> topicConfigTable = new ConcurrentHashMap<>();
|
|||
|
|
+
|
|||
|
|
+- TopicConfigAndMappingSerializeWrapper topicConfigWrapper = new TopicConfigAndMappingSerializeWrapper();
|
|||
|
|
+-
|
|||
|
|
+- topicConfigWrapper.setDataVersion(this.getTopicConfigManager().getDataVersion());
|
|||
|
|
+- topicConfigWrapper.setTopicConfigTable(this.getTopicConfigManager().getTopicConfigTable());
|
|||
|
|
+-
|
|||
|
|
+- topicConfigWrapper.setTopicQueueMappingInfoMap(this.getTopicQueueMappingManager().getTopicQueueMappingTable().entrySet().stream().map(
|
|||
|
|
+- entry -> new AbstractMap.SimpleImmutableEntry<>(entry.getKey(), TopicQueueMappingDetail.cloneAsMappingInfo(entry.getValue()))
|
|||
|
|
+- ).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));
|
|||
|
|
+-
|
|||
|
|
+- if (!PermName.isWriteable(this.getBrokerConfig().getBrokerPermission())
|
|||
|
|
+- || !PermName.isReadable(this.getBrokerConfig().getBrokerPermission())) {
|
|||
|
|
+- ConcurrentHashMap<String, TopicConfig> topicConfigTable = new ConcurrentHashMap<>();
|
|||
|
|
+- for (TopicConfig topicConfig : topicConfigWrapper.getTopicConfigTable().values()) {
|
|||
|
|
+- TopicConfig tmp =
|
|||
|
|
++ for (TopicConfig topicConfig : topicConfigMap.values()) {
|
|||
|
|
++ if (!PermName.isWriteable(this.getBrokerConfig().getBrokerPermission())
|
|||
|
|
++ || !PermName.isReadable(this.getBrokerConfig().getBrokerPermission())) {
|
|||
|
|
++ topicConfigTable.put(topicConfig.getTopicName(),
|
|||
|
|
+ new TopicConfig(topicConfig.getTopicName(), topicConfig.getReadQueueNums(), topicConfig.getWriteQueueNums(),
|
|||
|
|
+- topicConfig.getPerm() & this.brokerConfig.getBrokerPermission(), topicConfig.getTopicSysFlag());
|
|||
|
|
+- topicConfigTable.put(topicConfig.getTopicName(), tmp);
|
|||
|
|
++ topicConfig.getPerm() & getBrokerConfig().getBrokerPermission()));
|
|||
|
|
++ } else {
|
|||
|
|
++ topicConfigTable.put(topicConfig.getTopicName(), topicConfig);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ if (this.brokerConfig.isEnableSplitRegistration()
|
|||
|
|
++ && topicConfigTable.size() >= this.brokerConfig.getSplitRegistrationSize()) {
|
|||
|
|
++ TopicConfigAndMappingSerializeWrapper topicConfigWrapper = this.getTopicConfigManager().buildSerializeWrapper(topicConfigTable);
|
|||
|
|
++ doRegisterBrokerAll(checkOrderConfig, oneway, topicConfigWrapper);
|
|||
|
|
++ topicConfigTable.clear();
|
|||
|
|
+ }
|
|||
|
|
+- topicConfigWrapper.setTopicConfigTable(topicConfigTable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- if (forceRegister || needRegister(this.brokerConfig.getBrokerClusterName(),
|
|||
|
|
++ Map<String, TopicQueueMappingInfo> topicQueueMappingInfoMap = this.getTopicQueueMappingManager().getTopicQueueMappingTable().entrySet().stream()
|
|||
|
|
++ .map(entry -> new AbstractMap.SimpleImmutableEntry<>(entry.getKey(), TopicQueueMappingDetail.cloneAsMappingInfo(entry.getValue())))
|
|||
|
|
++ .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
|||
|
|
++
|
|||
|
|
++ TopicConfigAndMappingSerializeWrapper topicConfigWrapper = this.getTopicConfigManager().
|
|||
|
|
++ buildSerializeWrapper(topicConfigTable, topicQueueMappingInfoMap);
|
|||
|
|
++ if (this.brokerConfig.isEnableSplitRegistration() || forceRegister || needRegister(this.brokerConfig.getBrokerClusterName(),
|
|||
|
|
+ this.getBrokerAddr(),
|
|||
|
|
+ this.brokerConfig.getBrokerName(),
|
|||
|
|
+ this.brokerConfig.getBrokerId(),
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+index 754605438..8537929be 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+@@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
+
|
|||
|
|
+ import com.google.common.collect.ImmutableMap;
|
|||
|
|
+
|
|||
|
|
++import com.google.common.collect.Maps;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerController;
|
|||
|
|
+ import org.apache.rocketmq.broker.BrokerPathConfigHelper;
|
|||
|
|
+@@ -47,7 +48,9 @@ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.DataVersion;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.KVTable;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.body.TopicConfigAndMappingSerializeWrapper;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.statictopic.TopicQueueMappingInfo;
|
|||
|
|
+
|
|||
|
|
+ import static com.google.common.base.Preconditions.checkNotNull;
|
|||
|
|
+
|
|||
|
|
+@@ -609,6 +612,24 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ return topicConfigSerializeWrapper;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public TopicConfigAndMappingSerializeWrapper buildSerializeWrapper(final ConcurrentMap<String, TopicConfig> topicConfigTable) {
|
|||
|
|
++ return buildSerializeWrapper(topicConfigTable, Maps.newHashMap());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public TopicConfigAndMappingSerializeWrapper buildSerializeWrapper(
|
|||
|
|
++ final ConcurrentMap<String, TopicConfig> topicConfigTable,
|
|||
|
|
++ final Map<String, TopicQueueMappingInfo> topicQueueMappingInfoMap
|
|||
|
|
++ ) {
|
|||
|
|
++ TopicConfigAndMappingSerializeWrapper topicConfigWrapper = new TopicConfigAndMappingSerializeWrapper();
|
|||
|
|
++ topicConfigWrapper.setTopicConfigTable(topicConfigTable);
|
|||
|
|
++ topicConfigWrapper.setTopicQueueMappingInfoMap(topicQueueMappingInfoMap);
|
|||
|
|
++ topicConfigWrapper.setDataVersion(this.getDataVersion());
|
|||
|
|
++ if (this.brokerController.getBrokerConfig().isEnableSplitRegistration()) {
|
|||
|
|
++ this.getDataVersion().nextVersion();
|
|||
|
|
++ }
|
|||
|
|
++ return topicConfigWrapper;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public String encode() {
|
|||
|
|
+ return encode(false);
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+index 45d26b29c..0d248c4e1 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
|
|||
|
|
+@@ -396,6 +396,14 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+
|
|||
|
|
+ private boolean enableMixedMessageType = false;
|
|||
|
|
+
|
|||
|
|
++ /**
|
|||
|
|
++ * This flag and deleteTopicWithBrokerRegistration flag in the NameServer cannot be set to true at the same time,
|
|||
|
|
++ * otherwise there will be a loss of routing
|
|||
|
|
++ */
|
|||
|
|
++ private boolean enableSplitRegistration = false;
|
|||
|
|
++
|
|||
|
|
++ private int splitRegistrationSize = 800;
|
|||
|
|
++
|
|||
|
|
+ public long getMaxPopPollingSize() {
|
|||
|
|
+ return maxPopPollingSize;
|
|||
|
|
+ }
|
|||
|
|
+@@ -1731,4 +1739,20 @@ public class BrokerConfig extends BrokerIdentity {
|
|||
|
|
+ public void setEnableMixedMessageType(boolean enableMixedMessageType) {
|
|||
|
|
+ this.enableMixedMessageType = enableMixedMessageType;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableSplitRegistration() {
|
|||
|
|
++ return enableSplitRegistration;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableSplitRegistration(boolean enableSplitRegistration) {
|
|||
|
|
++ this.enableSplitRegistration = enableSplitRegistration;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getSplitRegistrationSize() {
|
|||
|
|
++ return splitRegistrationSize;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setSplitRegistrationSize(int splitRegistrationSize) {
|
|||
|
|
++ this.splitRegistrationSize = splitRegistrationSize;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/test/src/test/java/org/apache/rocketmq/test/route/CreateAndUpdateTopicIT.java b/test/src/test/java/org/apache/rocketmq/test/route/CreateAndUpdateTopicIT.java
|
|||
|
|
+index 7e3c7b871..2370e68c0 100644
|
|||
|
|
+--- a/test/src/test/java/org/apache/rocketmq/test/route/CreateAndUpdateTopicIT.java
|
|||
|
|
++++ b/test/src/test/java/org/apache/rocketmq/test/route/CreateAndUpdateTopicIT.java
|
|||
|
|
+@@ -17,6 +17,7 @@
|
|||
|
|
+
|
|||
|
|
+ package org.apache.rocketmq.test.route;
|
|||
|
|
+
|
|||
|
|
++import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.TopicRouteData;
|
|||
|
|
+ import org.apache.rocketmq.test.base.BaseConf;
|
|||
|
|
+ import org.apache.rocketmq.test.util.MQAdminTestUtils;
|
|||
|
|
+@@ -111,4 +112,34 @@ public class CreateAndUpdateTopicIT extends BaseConf {
|
|||
|
|
+ brokerController3.getBrokerConfig().setEnableSingleTopicRegister(false);
|
|||
|
|
+ namesrvController.getNamesrvConfig().setDeleteTopicWithBrokerRegistration(false);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testCreateOrUpdateTopic_EnableSplitRegistration() {
|
|||
|
|
++ brokerController1.getBrokerConfig().setEnableSplitRegistration(true);
|
|||
|
|
++ brokerController2.getBrokerConfig().setEnableSplitRegistration(true);
|
|||
|
|
++ brokerController3.getBrokerConfig().setEnableSplitRegistration(true);
|
|||
|
|
++
|
|||
|
|
++ String testTopic = "test-topic-";
|
|||
|
|
++
|
|||
|
|
++ for (int i = 0; i < 1000; i++) {
|
|||
|
|
++ TopicConfig topicConfig = new TopicConfig(testTopic + i, 8, 8);
|
|||
|
|
++ brokerController1.getTopicConfigManager().updateTopicConfig(topicConfig);
|
|||
|
|
++ brokerController2.getTopicConfigManager().updateTopicConfig(topicConfig);
|
|||
|
|
++ brokerController3.getTopicConfigManager().updateTopicConfig(topicConfig);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ brokerController1.registerBrokerAll(false, true, true);
|
|||
|
|
++ brokerController2.registerBrokerAll(false, true, true);
|
|||
|
|
++ brokerController3.registerBrokerAll(false, true, true);
|
|||
|
|
++
|
|||
|
|
++ for (int i = 0; i < 1000; i++) {
|
|||
|
|
++ TopicRouteData route = MQAdminTestUtils.examineTopicRouteInfo(NAMESRV_ADDR, testTopic + i);
|
|||
|
|
++ assertThat(route.getBrokerDatas()).hasSize(3);
|
|||
|
|
++ assertThat(route.getQueueDatas()).hasSize(3);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ brokerController1.getBrokerConfig().setEnableSplitRegistration(false);
|
|||
|
|
++ brokerController2.getBrokerConfig().setEnableSplitRegistration(false);
|
|||
|
|
++ brokerController3.getBrokerConfig().setEnableSplitRegistration(false);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From a9e353285cea762b0c5eab567bdfa8e5c8c2d279 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Mon, 11 Sep 2023 15:55:18 +0800
|
|||
|
|
+Subject: [PATCH 4/6] Add the configuration of topicQueueLock number to better
|
|||
|
|
+ support different scenarios (#7317)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/store/CommitLog.java | 2 +-
|
|||
|
|
+ .../java/org/apache/rocketmq/store/TopicQueueLock.java | 8 ++++++++
|
|||
|
|
+ .../rocketmq/store/config/MessageStoreConfig.java | 10 ++++++++++
|
|||
|
|
+ 3 files changed, 19 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
+index e6ee3bacc..456bf2b86 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
+@@ -122,7 +122,7 @@ public class CommitLog implements Swappable {
|
|||
|
|
+
|
|||
|
|
+ this.flushDiskWatcher = new FlushDiskWatcher();
|
|||
|
|
+
|
|||
|
|
+- this.topicQueueLock = new TopicQueueLock();
|
|||
|
|
++ this.topicQueueLock = new TopicQueueLock(messageStore.getMessageStoreConfig().getTopicQueueLockNum());
|
|||
|
|
+
|
|||
|
|
+ this.commitLogSize = messageStore.getMessageStoreConfig().getMappedFileSizeCommitLog();
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java b/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java
|
|||
|
|
+index a78eeed23..5a131b5c3 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/TopicQueueLock.java
|
|||
|
|
+@@ -34,6 +34,14 @@ public class TopicQueueLock {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public TopicQueueLock(int size) {
|
|||
|
|
++ this.size = size;
|
|||
|
|
++ this.lockList = new ArrayList<>(size);
|
|||
|
|
++ for (int i = 0; i < this.size; i++) {
|
|||
|
|
++ this.lockList.add(new ReentrantLock());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public void lock(String topicQueueKey) {
|
|||
|
|
+ Lock lock = this.lockList.get((topicQueueKey.hashCode() & 0x7fffffff) % this.size);
|
|||
|
|
+ lock.lock();
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
+index efb728ac0..9fa448043 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
+@@ -401,6 +401,8 @@ public class MessageStoreConfig {
|
|||
|
|
+ private long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
+ private boolean enableRocksDBLog = false;
|
|||
|
|
+
|
|||
|
|
++ private int topicQueueLockNum = 32;
|
|||
|
|
++
|
|||
|
|
+ public boolean isDebugLockEnable() {
|
|||
|
|
+ return debugLockEnable;
|
|||
|
|
+ }
|
|||
|
|
+@@ -1751,4 +1753,12 @@ public class MessageStoreConfig {
|
|||
|
|
+ public void setEnableRocksDBLog(boolean enableRocksDBLog) {
|
|||
|
|
+ this.enableRocksDBLog = enableRocksDBLog;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public int getTopicQueueLockNum() {
|
|||
|
|
++ return topicQueueLockNum;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setTopicQueueLockNum(int topicQueueLockNum) {
|
|||
|
|
++ this.topicQueueLockNum = topicQueueLockNum;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 57f04c95d3a2ba6b91583058a6e4eda209f72d6e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Mon, 11 Sep 2023 18:23:25 +0800
|
|||
|
|
+Subject: [PATCH 5/6] [ISSUE #7343] Rollback modifications to registerProcessor
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: guyinyou <guyinyou.gyy@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../java/org/apache/rocketmq/broker/BrokerController.java | 4 ++--
|
|||
|
|
+ 1 file changed, 2 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 9e49f636d..13a3feb4e 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -827,6 +827,8 @@ public class BrokerController {
|
|||
|
|
+
|
|||
|
|
+ initializeResources();
|
|||
|
|
+
|
|||
|
|
++ registerProcessor();
|
|||
|
|
++
|
|||
|
|
+ initializeScheduledTasks();
|
|||
|
|
+
|
|||
|
|
+ initialTransaction();
|
|||
|
|
+@@ -1687,8 +1689,6 @@ public class BrokerController {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }, 10, 5, TimeUnit.SECONDS);
|
|||
|
|
+-
|
|||
|
|
+- registerProcessor();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ protected void scheduleSendHeartbeat() {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From dad6ad09d13dadc36b6342671c77f619bbb8c522 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ao Qiao <qiao_ao@foxmail.com>
|
|||
|
|
+Date: Tue, 12 Sep 2023 08:28:45 +0800
|
|||
|
|
+Subject: [PATCH 6/6] [ISSUE #7340] Abstract Duplicate code into a method in
|
|||
|
|
+ `TopicConfigManager` (#7341)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../broker/topic/TopicConfigManager.java | 44 ++++++-------------
|
|||
|
|
+ 1 file changed, 14 insertions(+), 30 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+index 8537929be..511d29e12 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java
|
|||
|
|
+@@ -293,11 +293,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -337,11 +333,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ log.error("createTopicIfAbsent ", e);
|
|||
|
|
+ }
|
|||
|
|
+ if (createNew && register) {
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+ return getTopicConfig(topicConfig.getTopicName());
|
|||
|
|
+ }
|
|||
|
|
+@@ -401,11 +393,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -446,11 +434,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (createNew) {
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return topicConfig;
|
|||
|
|
+@@ -476,11 +460,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ dataVersion.nextVersion(stateMachineVersion);
|
|||
|
|
+
|
|||
|
|
+ this.persist();
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -503,11 +483,7 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ dataVersion.nextVersion(stateMachineVersion);
|
|||
|
|
+
|
|||
|
|
+ this.persist();
|
|||
|
|
+- if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
+- this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
+- } else {
|
|||
|
|
+- this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
+- }
|
|||
|
|
++ registerBrokerData(topicConfig);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -699,6 +675,14 @@ public class TopicConfigManager extends ConfigManager {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ private void registerBrokerData(TopicConfig topicConfig) {
|
|||
|
|
++ if (brokerController.getBrokerConfig().isEnableSingleTopicRegister()) {
|
|||
|
|
++ this.brokerController.registerSingleTopicAll(topicConfig);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.brokerController.registerIncrementBrokerData(topicConfig, dataVersion);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean containsTopic(String topic) {
|
|||
|
|
+ return topicConfigTable.containsKey(topic);
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch018-backport-enhancement-of-tiered-storage.patch b/patch018-backport-enhancement-of-tiered-storage.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..ed79df9c7
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch018-backport-enhancement-of-tiered-storage.patch
|
|||
|
|
@@ -0,0 +1,601 @@
|
|||
|
|
+From 1a8e7cb17cb29ed33b0196b52e452a6e76ade781 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: yuz10 <845238369@qq.com>
|
|||
|
|
+Date: Tue, 12 Sep 2023 19:33:41 +0800
|
|||
|
|
+Subject: [PATCH 1/5] [ISSUE #7345] Fix wrong result of searchOffset in tiered
|
|||
|
|
+ storage
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/file/TieredFlatFile.java | 5 +-
|
|||
|
|
+ .../tieredstore/file/TieredFlatFileTest.java | 46 +++++++++++++++++--
|
|||
|
|
+ 2 files changed, 46 insertions(+), 5 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+index 426c4e09d..d973179ee 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+@@ -365,7 +365,10 @@ public class TieredFlatFile {
|
|||
|
|
+ if (!segmentList.isEmpty()) {
|
|||
|
|
+ return boundaryType == BoundaryType.UPPER ? segmentList.get(0) : segmentList.get(segmentList.size() - 1);
|
|||
|
|
+ }
|
|||
|
|
+- return fileSegmentList.isEmpty() ? null : fileSegmentList.get(fileSegmentList.size() - 1);
|
|||
|
|
++ if (fileSegmentList.isEmpty()) {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++ return boundaryType == BoundaryType.UPPER ? fileSegmentList.get(fileSegmentList.size() - 1) : fileSegmentList.get(0);
|
|||
|
|
+ } finally {
|
|||
|
|
+ fileSegmentLock.readLock().unlock();
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
+index 7a4d05969..7e2fbf201 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileTest.java
|
|||
|
|
+@@ -16,10 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
+-import java.io.IOException;
|
|||
|
|
+-import java.nio.ByteBuffer;
|
|||
|
|
+-import java.util.ArrayList;
|
|||
|
|
+-import java.util.List;
|
|||
|
|
++import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.TieredStoreTestUtil;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.FileSegmentType;
|
|||
|
|
+@@ -35,6 +32,11 @@ import org.junit.Assert;
|
|||
|
|
+ import org.junit.Before;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
++import java.io.IOException;
|
|||
|
|
++import java.nio.ByteBuffer;
|
|||
|
|
++import java.util.ArrayList;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++
|
|||
|
|
+ public class TieredFlatFileTest {
|
|||
|
|
+
|
|||
|
|
+ private final String storePath = TieredStoreTestUtil.getRandomStorePath();
|
|||
|
|
+@@ -301,4 +303,40 @@ public class TieredFlatFileTest {
|
|||
|
|
+ fileQueue.rollingNewFile();
|
|||
|
|
+ Assert.assertEquals(2, fileQueue.getFileSegmentCount());
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testGetFileByTime() {
|
|||
|
|
++ String filePath = TieredStoreUtil.toPath(queue);
|
|||
|
|
++ TieredFlatFile tieredFlatFile = fileQueueFactory.createFlatFileForCommitLog(filePath);
|
|||
|
|
++ TieredFileSegment fileSegment1 = new MemoryFileSegment(FileSegmentType.CONSUME_QUEUE, queue, 1100, storeConfig);
|
|||
|
|
++ fileSegment1.setMinTimestamp(100);
|
|||
|
|
++ fileSegment1.setMaxTimestamp(200);
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment fileSegment2 = new MemoryFileSegment(FileSegmentType.CONSUME_QUEUE, queue, 1100, storeConfig);
|
|||
|
|
++ fileSegment2.setMinTimestamp(200);
|
|||
|
|
++ fileSegment2.setMaxTimestamp(300);
|
|||
|
|
++
|
|||
|
|
++ tieredFlatFile.getFileSegmentList().add(fileSegment1);
|
|||
|
|
++ tieredFlatFile.getFileSegmentList().add(fileSegment2);
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentUpper = tieredFlatFile.getFileByTime(400, BoundaryType.UPPER);
|
|||
|
|
++ Assert.assertEquals(fileSegment2, segmentUpper);
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentLower = tieredFlatFile.getFileByTime(400, BoundaryType.LOWER);
|
|||
|
|
++ Assert.assertEquals(fileSegment2, segmentLower);
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentUpper2 = tieredFlatFile.getFileByTime(0, BoundaryType.UPPER);
|
|||
|
|
++ Assert.assertEquals(fileSegment1, segmentUpper2);
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentLower2 = tieredFlatFile.getFileByTime(0, BoundaryType.LOWER);
|
|||
|
|
++ Assert.assertEquals(fileSegment1, segmentLower2);
|
|||
|
|
++
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentUpper3 = tieredFlatFile.getFileByTime(200, BoundaryType.UPPER);
|
|||
|
|
++ Assert.assertEquals(fileSegment1, segmentUpper3);
|
|||
|
|
++
|
|||
|
|
++ TieredFileSegment segmentLower3 = tieredFlatFile.getFileByTime(200, BoundaryType.LOWER);
|
|||
|
|
++ Assert.assertEquals(fileSegment2, segmentLower3);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From fd32dae2ab59f86dd215eeec405bf4fa6212bcb3 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Tue, 12 Sep 2023 19:58:08 +0800
|
|||
|
|
+Subject: [PATCH 2/5] [ISSUE #6633] Not clear uninitialized files and fix
|
|||
|
|
+ metadata recover (#7342)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/file/TieredFlatFile.java | 53 +++++++------------
|
|||
|
|
+ .../file/TieredFlatFileManager.java | 10 ++--
|
|||
|
|
+ 2 files changed, 22 insertions(+), 41 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+index d973179ee..d96eb6e8f 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFile.java
|
|||
|
|
+@@ -16,7 +16,6 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.tieredstore.file;
|
|||
|
|
+
|
|||
|
|
+-import com.alibaba.fastjson.JSON;
|
|||
|
|
+ import com.google.common.annotations.VisibleForTesting;
|
|||
|
|
+ import java.nio.ByteBuffer;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+@@ -25,13 +24,13 @@ import java.util.Comparator;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+-import java.util.Objects;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
|
|
+ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
|
|
+ import java.util.stream.Collectors;
|
|||
|
|
+ import javax.annotation.Nullable;
|
|||
|
|
++import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.common.AppendResult;
|
|||
|
|
+@@ -43,7 +42,6 @@ import org.apache.rocketmq.tieredstore.metadata.TieredMetadataStore;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.provider.FileSegmentAllocator;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.provider.TieredFileSegment;
|
|||
|
|
+ import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
+-import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+
|
|||
|
|
+ public class TieredFlatFile {
|
|||
|
|
+
|
|||
|
|
+@@ -177,7 +175,10 @@ public class TieredFlatFile {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- private FileSegmentMetadata getOrCreateFileSegmentMetadata(TieredFileSegment fileSegment) {
|
|||
|
|
++ /**
|
|||
|
|
++ * FileQueue Status: Sealed | Sealed | Sealed | Not sealed, Allow appended && Not Full
|
|||
|
|
++ */
|
|||
|
|
++ public void updateFileSegment(TieredFileSegment fileSegment) {
|
|||
|
|
+
|
|||
|
|
+ FileSegmentMetadata metadata = tieredMetadataStore.getFileSegment(
|
|||
|
|
+ this.filePath, fileSegment.getFileType(), fileSegment.getBaseOffset());
|
|||
|
|
+@@ -186,45 +187,24 @@ public class TieredFlatFile {
|
|||
|
|
+ if (metadata == null) {
|
|||
|
|
+ metadata = new FileSegmentMetadata(
|
|||
|
|
+ this.filePath, fileSegment.getBaseOffset(), fileSegment.getFileType().getType());
|
|||
|
|
+- metadata.setCreateTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
+- metadata.setBeginTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
+- metadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
+- if (fileSegment.isClosed()) {
|
|||
|
|
+- metadata.setStatus(FileSegmentMetadata.STATUS_DELETED);
|
|||
|
|
+- }
|
|||
|
|
+- this.tieredMetadataStore.updateFileSegment(metadata);
|
|||
|
|
++ metadata.setCreateTimestamp(System.currentTimeMillis());
|
|||
|
|
+ }
|
|||
|
|
+- return metadata;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- /**
|
|||
|
|
+- * FileQueue Status: Sealed | Sealed | Sealed | Not sealed, Allow appended && Not Full
|
|||
|
|
+- */
|
|||
|
|
+- public void updateFileSegment(TieredFileSegment fileSegment) {
|
|||
|
|
+- FileSegmentMetadata segmentMetadata = getOrCreateFileSegmentMetadata(fileSegment);
|
|||
|
|
+
|
|||
|
|
+- if (segmentMetadata.getStatus() == FileSegmentMetadata.STATUS_NEW
|
|||
|
|
+- && fileSegment.isFull()
|
|||
|
|
+- && !fileSegment.needCommit()) {
|
|||
|
|
++ metadata.setSize(fileSegment.getCommitPosition());
|
|||
|
|
++ metadata.setBeginTimestamp(fileSegment.getMinTimestamp());
|
|||
|
|
++ metadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
+
|
|||
|
|
+- segmentMetadata.markSealed();
|
|||
|
|
++ if (fileSegment.isFull() && !fileSegment.needCommit()) {
|
|||
|
|
++ if (metadata.getStatus() == FileSegmentMetadata.STATUS_NEW) {
|
|||
|
|
++ metadata.markSealed();
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (fileSegment.isClosed()) {
|
|||
|
|
+- segmentMetadata.setStatus(FileSegmentMetadata.STATUS_DELETED);
|
|||
|
|
++ metadata.setStatus(FileSegmentMetadata.STATUS_DELETED);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- segmentMetadata.setSize(fileSegment.getCommitPosition());
|
|||
|
|
+- segmentMetadata.setEndTimestamp(fileSegment.getMaxTimestamp());
|
|||
|
|
+-
|
|||
|
|
+- FileSegmentMetadata metadata = tieredMetadataStore.getFileSegment(
|
|||
|
|
+- this.filePath, fileSegment.getFileType(), fileSegment.getBaseOffset());
|
|||
|
|
+-
|
|||
|
|
+- if (!Objects.equals(metadata, segmentMetadata)) {
|
|||
|
|
+- this.tieredMetadataStore.updateFileSegment(segmentMetadata);
|
|||
|
|
+- logger.debug("TieredFlatFile#UpdateSegmentMetadata, filePath: {}, content: {}",
|
|||
|
|
+- segmentMetadata.getPath(), JSON.toJSONString(segmentMetadata));
|
|||
|
|
+- }
|
|||
|
|
++ this.tieredMetadataStore.updateFileSegment(metadata);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void checkAndFixFileSize() {
|
|||
|
|
+@@ -598,6 +578,9 @@ public class TieredFlatFile {
|
|||
|
|
+ logger.error("TieredFlatFile#destroy: mark file segment: {} is deleted failed", fileSegment.getPath(), e);
|
|||
|
|
+ }
|
|||
|
|
+ fileSegment.destroyFile();
|
|||
|
|
++ if (!fileSegment.exists()) {
|
|||
|
|
++ tieredMetadataStore.deleteFileSegment(filePath, fileType, fileSegment.getBaseOffset());
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+ fileSegmentList.clear();
|
|||
|
|
+ needCommitFileSegmentList.clear();
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+index 7c744af3b..087ea8c9c 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/TieredFlatFileManager.java
|
|||
|
|
+@@ -136,15 +136,13 @@ public class TieredFlatFileManager {
|
|||
|
|
+ TimeUnit.HOURS.toMillis(storeConfig.getTieredStoreFileReservedTime());
|
|||
|
|
+ for (CompositeQueueFlatFile flatFile : deepCopyFlatFileToList()) {
|
|||
|
|
+ TieredStoreExecutor.cleanExpiredFileExecutor.submit(() -> {
|
|||
|
|
+- flatFile.getCompositeFlatFileLock().lock();
|
|||
|
|
+ try {
|
|||
|
|
++ flatFile.getCompositeFlatFileLock().lock();
|
|||
|
|
+ flatFile.cleanExpiredFile(expiredTimeStamp);
|
|||
|
|
+ flatFile.destroyExpiredFile();
|
|||
|
|
+- if (flatFile.getConsumeQueueBaseOffset() == -1) {
|
|||
|
|
+- logger.info("Clean flatFile because file not initialized, topic={}, queueId={}",
|
|||
|
|
+- flatFile.getMessageQueue().getTopic(), flatFile.getMessageQueue().getQueueId());
|
|||
|
|
+- destroyCompositeFile(flatFile.getMessageQueue());
|
|||
|
|
+- }
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ logger.error("Do Clean expired file error, topic={}, queueId={}",
|
|||
|
|
++ flatFile.getMessageQueue().getTopic(), flatFile.getMessageQueue().getQueueId(), t);
|
|||
|
|
+ } finally {
|
|||
|
|
+ flatFile.getCompositeFlatFileLock().unlock();
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 4a8e0d5b851d1f9573cda79b7d2e42ee498809da Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 13 Sep 2023 16:08:03 +0800
|
|||
|
|
+Subject: [PATCH 3/5] [ISSUE #7351] Allow mqadmin to operate slave nodes
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: guyinyou <guyinyou.gyy@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../processor/AdminBrokerProcessor.java | 12 --
|
|||
|
|
+ .../processor/AdminBrokerProcessorTest.java | 106 ------------------
|
|||
|
|
+ 2 files changed, 118 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+index 8fbcd3c94..9e48431be 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+@@ -406,9 +406,6 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ private synchronized RemotingCommand updateAndCreateTopic(ChannelHandlerContext ctx,
|
|||
|
|
+ RemotingCommand request) throws RemotingCommandException {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- if (validateSlave(response)) {
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+ final CreateTopicRequestHeader requestHeader =
|
|||
|
|
+ (CreateTopicRequestHeader) request.decodeCommandCustomHeader(CreateTopicRequestHeader.class);
|
|||
|
|
+
|
|||
|
|
+@@ -519,9 +516,6 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ private synchronized RemotingCommand deleteTopic(ChannelHandlerContext ctx,
|
|||
|
|
+ RemotingCommand request) throws RemotingCommandException {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- if (validateSlave(response)) {
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+ DeleteTopicRequestHeader requestHeader =
|
|||
|
|
+ (DeleteTopicRequestHeader) request.decodeCommandCustomHeader(DeleteTopicRequestHeader.class);
|
|||
|
|
+
|
|||
|
|
+@@ -1413,9 +1407,6 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ private RemotingCommand updateAndCreateSubscriptionGroup(ChannelHandlerContext ctx, RemotingCommand request)
|
|||
|
|
+ throws RemotingCommandException {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- if (validateSlave(response)) {
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+
|
|||
|
|
+ LOGGER.info("AdminBrokerProcessor#updateAndCreateSubscriptionGroup called by {}",
|
|||
|
|
+ RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
|
|||
|
|
+@@ -1480,9 +1471,6 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ private RemotingCommand deleteSubscriptionGroup(ChannelHandlerContext ctx,
|
|||
|
|
+ RemotingCommand request) throws RemotingCommandException {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- if (validateSlave(response)) {
|
|||
|
|
+- return response;
|
|||
|
|
+- }
|
|||
|
|
+ DeleteSubscriptionGroupRequestHeader requestHeader =
|
|||
|
|
+ (DeleteSubscriptionGroupRequestHeader) request.decodeCommandCustomHeader(DeleteSubscriptionGroupRequestHeader.class);
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
+index 9d17011b6..ec252cece 100644
|
|||
|
|
+--- a/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
++++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessorTest.java
|
|||
|
|
+@@ -76,7 +76,6 @@ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfi
|
|||
|
|
+ import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+ import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
+ import org.apache.rocketmq.store.SelectMappedBufferResult;
|
|||
|
|
+-import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
+ import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+ import org.apache.rocketmq.store.logfile.DefaultMappedFile;
|
|||
|
|
+ import org.apache.rocketmq.store.stats.BrokerStats;
|
|||
|
|
+@@ -250,32 +249,6 @@ public class AdminBrokerProcessorTest {
|
|||
|
|
+ assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testUpdateAndCreateTopicOnSlaveInRocksdb() throws Exception {
|
|||
|
|
+- if (notToBeExecuted()) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- initRocksdbTopicManager();
|
|||
|
|
+- testUpdateAndCreateTopicOnSlave();
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testUpdateAndCreateTopicOnSlave() throws Exception {
|
|||
|
|
+- // setup
|
|||
|
|
+- MessageStoreConfig messageStoreConfig = mock(MessageStoreConfig.class);
|
|||
|
|
+- when(messageStoreConfig.getBrokerRole()).thenReturn(BrokerRole.SLAVE);
|
|||
|
|
+- defaultMessageStore = mock(DefaultMessageStore.class);
|
|||
|
|
+- when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig);
|
|||
|
|
+-
|
|||
|
|
+- // test on slave
|
|||
|
|
+- String topic = "TEST_CREATE_TOPIC";
|
|||
|
|
+- RemotingCommand request = buildCreateTopicRequest(topic);
|
|||
|
|
+- RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request);
|
|||
|
|
+- assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+- assertThat(response.getRemark()).isEqualTo("Can't modify topic or subscription group from slave broker, " +
|
|||
|
|
+- "please execute it from master broker.");
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testDeleteTopicInRocksdb() throws Exception {
|
|||
|
|
+ if (notToBeExecuted()) {
|
|||
|
|
+@@ -301,31 +274,6 @@ public class AdminBrokerProcessorTest {
|
|||
|
|
+ assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testDeleteTopicOnSlaveInRocksdb() throws Exception {
|
|||
|
|
+- if (notToBeExecuted()) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- initRocksdbTopicManager();
|
|||
|
|
+- testDeleteTopicOnSlave();
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testDeleteTopicOnSlave() throws Exception {
|
|||
|
|
+- // setup
|
|||
|
|
+- MessageStoreConfig messageStoreConfig = mock(MessageStoreConfig.class);
|
|||
|
|
+- when(messageStoreConfig.getBrokerRole()).thenReturn(BrokerRole.SLAVE);
|
|||
|
|
+- defaultMessageStore = mock(DefaultMessageStore.class);
|
|||
|
|
+- when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig);
|
|||
|
|
+-
|
|||
|
|
+- String topic = "TEST_DELETE_TOPIC";
|
|||
|
|
+- RemotingCommand request = buildDeleteTopicRequest(topic);
|
|||
|
|
+- RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request);
|
|||
|
|
+- assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+- assertThat(response.getRemark()).isEqualTo("Can't modify topic or subscription group from slave broker, " +
|
|||
|
|
+- "please execute it from master broker.");
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testDeleteWithPopRetryTopic() throws Exception {
|
|||
|
|
+ String topic = "topicA";
|
|||
|
|
+@@ -538,36 +486,6 @@ public class AdminBrokerProcessorTest {
|
|||
|
|
+ assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testUpdateAndCreateSubscriptionGroupOnSlaveInRocksdb() throws Exception {
|
|||
|
|
+- initRocksdbSubscriptionManager();
|
|||
|
|
+- testUpdateAndCreateSubscriptionGroupOnSlave();
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testUpdateAndCreateSubscriptionGroupOnSlave() throws RemotingCommandException {
|
|||
|
|
+- // Setup
|
|||
|
|
+- MessageStoreConfig messageStoreConfig = mock(MessageStoreConfig.class);
|
|||
|
|
+- when(messageStoreConfig.getBrokerRole()).thenReturn(BrokerRole.SLAVE);
|
|||
|
|
+- defaultMessageStore = mock(DefaultMessageStore.class);
|
|||
|
|
+- when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig);
|
|||
|
|
+-
|
|||
|
|
+- // Test
|
|||
|
|
+- RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP, null);
|
|||
|
|
+- SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig();
|
|||
|
|
+- subscriptionGroupConfig.setBrokerId(1);
|
|||
|
|
+- subscriptionGroupConfig.setGroupName("groupId");
|
|||
|
|
+- subscriptionGroupConfig.setConsumeEnable(Boolean.TRUE);
|
|||
|
|
+- subscriptionGroupConfig.setConsumeBroadcastEnable(Boolean.TRUE);
|
|||
|
|
+- subscriptionGroupConfig.setRetryMaxTimes(111);
|
|||
|
|
+- subscriptionGroupConfig.setConsumeFromMinEnable(Boolean.TRUE);
|
|||
|
|
+- request.setBody(JSON.toJSON(subscriptionGroupConfig).toString().getBytes());
|
|||
|
|
+- RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request);
|
|||
|
|
+- assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+- assertThat(response.getRemark()).isEqualTo("Can't modify topic or subscription group from slave broker, " +
|
|||
|
|
+- "please execute it from master broker.");
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetAllSubscriptionGroupInRocksdb() throws Exception {
|
|||
|
|
+ initRocksdbSubscriptionManager();
|
|||
|
|
+@@ -596,30 +514,6 @@ public class AdminBrokerProcessorTest {
|
|||
|
|
+ assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testDeleteSubscriptionGroupOnSlaveInRocksdb() throws Exception {
|
|||
|
|
+- initRocksdbSubscriptionManager();
|
|||
|
|
+- testDeleteSubscriptionGroupOnSlave();
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- @Test
|
|||
|
|
+- public void testDeleteSubscriptionGroupOnSlave() throws RemotingCommandException {
|
|||
|
|
+- // Setup
|
|||
|
|
+- MessageStoreConfig messageStoreConfig = mock(MessageStoreConfig.class);
|
|||
|
|
+- when(messageStoreConfig.getBrokerRole()).thenReturn(BrokerRole.SLAVE);
|
|||
|
|
+- defaultMessageStore = mock(DefaultMessageStore.class);
|
|||
|
|
+- when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig);
|
|||
|
|
+-
|
|||
|
|
+- // Test
|
|||
|
|
+- RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, null);
|
|||
|
|
+- request.addExtField("groupName", "GID-Group-Name");
|
|||
|
|
+- request.addExtField("removeOffset", "true");
|
|||
|
|
+- RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request);
|
|||
|
|
+- assertThat(response.getCode()).isEqualTo(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+- assertThat(response.getRemark()).isEqualTo("Can't modify topic or subscription group from slave broker, " +
|
|||
|
|
+- "please execute it from master broker.");
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetTopicStatsInfo() throws RemotingCommandException {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_STATS_INFO, null);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 831fcc76cd7cd362bb6c136c287c624bb7eaf40a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lizhimins <707364882@qq.com>
|
|||
|
|
+Date: Tue, 19 Sep 2023 10:04:04 +0800
|
|||
|
|
+Subject: [PATCH 4/5] [ISSUE #7363] Fix get message from tiered storage return
|
|||
|
|
+ incorrect next pull offset (#7365)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/TieredMessageFetcher.java | 2 +-
|
|||
|
|
+ .../tieredstore/TieredMessageStore.java | 29 ++++++++++---------
|
|||
|
|
+ .../tieredstore/TieredMessageStoreTest.java | 5 ++--
|
|||
|
|
+ 3 files changed, 20 insertions(+), 16 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
+index 766ff64f6..c948fa3fa 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageFetcher.java
|
|||
|
|
+@@ -319,7 +319,7 @@ public class TieredMessageFetcher implements MessageStoreFetcher {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // if cache is miss, immediately pull messages
|
|||
|
|
+- LOGGER.warn("TieredMessageFetcher#getMessageFromCacheAsync: cache miss: " +
|
|||
|
|
++ LOGGER.info("TieredMessageFetcher#getMessageFromCacheAsync: cache miss: " +
|
|||
|
|
+ "topic: {}, queue: {}, queue offset: {}, max message num: {}",
|
|||
|
|
+ mq.getTopic(), mq.getQueueId(), queueOffset, maxCount);
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+index 9fb1b2f01..d7d13d61e 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+@@ -147,6 +147,11 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ public CompletableFuture<GetMessageResult> getMessageAsync(String group, String topic,
|
|||
|
|
+ int queueId, long offset, int maxMsgNums, MessageFilter messageFilter) {
|
|||
|
|
+
|
|||
|
|
++ // For system topic, force reading from local store
|
|||
|
|
++ if (TieredStoreUtil.isSystemTopic(topic) || PopAckConstants.isStartWithRevivePrefix(topic)) {
|
|||
|
|
++ return next.getMessageAsync(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ if (fetchFromCurrentStore(topic, queueId, offset, maxMsgNums)) {
|
|||
|
|
+ logger.trace("GetMessageAsync from current store, topic: {}, queue: {}, offset: {}", topic, queueId, offset);
|
|||
|
|
+ } else {
|
|||
|
|
+@@ -158,6 +163,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ return fetcher
|
|||
|
|
+ .getMessageAsync(group, topic, queueId, offset, maxMsgNums, messageFilter)
|
|||
|
|
+ .thenApply(result -> {
|
|||
|
|
++
|
|||
|
|
+ Attributes latencyAttributes = TieredStoreMetricsManager.newAttributesBuilder()
|
|||
|
|
+ .put(TieredStoreMetricsConstant.LABEL_OPERATION, TieredStoreMetricsConstant.OPERATION_API_GET_MESSAGE)
|
|||
|
|
+ .put(TieredStoreMetricsConstant.LABEL_TOPIC, topic)
|
|||
|
|
+@@ -166,8 +172,7 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes);
|
|||
|
|
+
|
|||
|
|
+ if (result.getStatus() == GetMessageStatus.OFFSET_FOUND_NULL ||
|
|||
|
|
+- result.getStatus() == GetMessageStatus.OFFSET_OVERFLOW_ONE ||
|
|||
|
|
+- result.getStatus() == GetMessageStatus.OFFSET_OVERFLOW_BADLY) {
|
|||
|
|
++ result.getStatus() == GetMessageStatus.NO_MATCHED_LOGIC_QUEUE) {
|
|||
|
|
+
|
|||
|
|
+ if (next.checkInStoreByConsumeOffset(topic, queueId, offset)) {
|
|||
|
|
+ TieredStoreMetricsManager.fallbackTotal.add(1, latencyAttributes);
|
|||
|
|
+@@ -178,14 +183,8 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- // Fetch system topic data from the broker when using the force level.
|
|||
|
|
+- if (result.getStatus() == GetMessageStatus.NO_MATCHED_LOGIC_QUEUE) {
|
|||
|
|
+- if (TieredStoreUtil.isSystemTopic(topic) || PopAckConstants.isStartWithRevivePrefix(topic)) {
|
|||
|
|
+- return next.getMessage(group, topic, queueId, offset, maxMsgNums, messageFilter);
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ if (result.getStatus() != GetMessageStatus.FOUND &&
|
|||
|
|
++ result.getStatus() != GetMessageStatus.NO_MATCHED_LOGIC_QUEUE &&
|
|||
|
|
+ result.getStatus() != GetMessageStatus.OFFSET_OVERFLOW_ONE &&
|
|||
|
|
+ result.getStatus() != GetMessageStatus.OFFSET_OVERFLOW_BADLY) {
|
|||
|
|
+ logger.warn("GetMessageAsync not found and message is not in next store, result: {}, " +
|
|||
|
|
+@@ -206,10 +205,14 @@ public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
+ if (minOffsetInQueue >= 0 && minOffsetInQueue < result.getMinOffset()) {
|
|||
|
|
+ result.setMinOffset(minOffsetInQueue);
|
|||
|
|
+ }
|
|||
|
|
+- long maxOffsetInQueue = next.getMaxOffsetInQueue(topic, queueId);
|
|||
|
|
+- if (maxOffsetInQueue >= 0 && maxOffsetInQueue > result.getMaxOffset()) {
|
|||
|
|
+- result.setMaxOffset(maxOffsetInQueue);
|
|||
|
|
+- }
|
|||
|
|
++
|
|||
|
|
++ // In general, the local cq offset is slightly greater than the commit offset in read message,
|
|||
|
|
++ // so there is no need to update the maximum offset to the local cq offset here,
|
|||
|
|
++ // otherwise it will cause repeated consumption after next begin offset over commit offset.
|
|||
|
|
++
|
|||
|
|
++ logger.trace("GetMessageAsync result, group: {}, topic: {}, queueId: {}, offset: {}, count:{}, {}",
|
|||
|
|
++ group, topic, queueId, offset, maxMsgNums, result);
|
|||
|
|
++
|
|||
|
|
+ return result;
|
|||
|
|
+ }).exceptionally(e -> {
|
|||
|
|
+ logger.error("GetMessageAsync from tiered store failed", e);
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
+index 2451199c2..07af1fc8b 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/TieredMessageStoreTest.java
|
|||
|
|
+@@ -168,7 +168,7 @@ public class TieredMessageStoreTest {
|
|||
|
|
+ GetMessageResult result1 = new GetMessageResult();
|
|||
|
|
+ result1.setStatus(GetMessageStatus.FOUND);
|
|||
|
|
+ GetMessageResult result2 = new GetMessageResult();
|
|||
|
|
+- result2.setStatus(GetMessageStatus.MESSAGE_WAS_REMOVING);
|
|||
|
|
++ result2.setStatus(GetMessageStatus.OFFSET_OVERFLOW_BADLY);
|
|||
|
|
+
|
|||
|
|
+ when(fetcher.getMessageAsync(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any())).thenReturn(CompletableFuture.completedFuture(result1));
|
|||
|
|
+ when(nextStore.getMessage(anyString(), anyString(), anyInt(), anyLong(), anyInt(), any())).thenReturn(result2);
|
|||
|
|
+@@ -188,7 +188,8 @@ public class TieredMessageStoreTest {
|
|||
|
|
+ properties.setProperty("tieredStorageLevel", "3");
|
|||
|
|
+ configuration.update(properties);
|
|||
|
|
+ when(nextStore.checkInStoreByConsumeOffset(anyString(), anyInt(), anyLong())).thenReturn(true);
|
|||
|
|
+- Assert.assertSame(result2, store.getMessage("group", mq.getTopic(), mq.getQueueId(), 0, 0, null));
|
|||
|
|
++ Assert.assertEquals(result2.getStatus(),
|
|||
|
|
++ store.getMessage("group", mq.getTopic(), mq.getQueueId(), 0, 0, null).getStatus());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From f05a8da760dfade411ad56ef874f477988479cf9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Wed, 20 Sep 2023 15:06:21 +0800
|
|||
|
|
+Subject: [PATCH 5/5] Print admin queue watermark in log (#7372)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/broker/BrokerController.java | 1 +
|
|||
|
|
+ 1 file changed, 1 insertion(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 13a3feb4e..53e2e1b62 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1182,6 +1182,7 @@ public class BrokerController {
|
|||
|
|
+ LOG_WATER_MARK.info("[WATERMARK] ClientManager Queue Size: {} SlowTimeMills: {}", this.clientManagerThreadPoolQueue.size(), this.headSlowTimeMills(this.clientManagerThreadPoolQueue));
|
|||
|
|
+ LOG_WATER_MARK.info("[WATERMARK] Heartbeat Queue Size: {} SlowTimeMills: {}", this.heartbeatThreadPoolQueue.size(), this.headSlowTimeMills(this.heartbeatThreadPoolQueue));
|
|||
|
|
+ LOG_WATER_MARK.info("[WATERMARK] Ack Queue Size: {} SlowTimeMills: {}", this.ackThreadPoolQueue.size(), headSlowTimeMills(this.ackThreadPoolQueue));
|
|||
|
|
++ LOG_WATER_MARK.info("[WATERMARK] Admin Queue Size: {} SlowTimeMills: {}", this.adminBrokerThreadPoolQueue.size(), headSlowTimeMills(this.adminBrokerThreadPoolQueue));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MessageStore getMessageStore() {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch019-backport-some-bugfix.patch b/patch019-backport-some-bugfix.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..d85f01c98
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch019-backport-some-bugfix.patch
|
|||
|
|
@@ -0,0 +1,1499 @@
|
|||
|
|
+From 42fcd278ca84f6988d48a7d11271fc81b921d59a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Wed, 20 Sep 2023 15:41:23 +0800
|
|||
|
|
+Subject: [PATCH 01/12] [ISSUE #7374] Prepare to release Apache RocketMQ 5.1.4
|
|||
|
|
+ (#7375)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ common/src/main/java/org/apache/rocketmq/common/MQVersion.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/MQVersion.java b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java
|
|||
|
|
+index bfd07a895..4f1990ff8 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/MQVersion.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java
|
|||
|
|
+@@ -18,7 +18,7 @@ package org.apache.rocketmq.common;
|
|||
|
|
+
|
|||
|
|
+ public class MQVersion {
|
|||
|
|
+
|
|||
|
|
+- public static final int CURRENT_VERSION = Version.V5_1_3.ordinal();
|
|||
|
|
++ public static final int CURRENT_VERSION = Version.V5_1_4.ordinal();
|
|||
|
|
+
|
|||
|
|
+ public static String getVersionDesc(int value) {
|
|||
|
|
+ int length = Version.values().length;
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From b8610d87bb55de1f4413460c05da529dab60c1c1 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Jixiang Jin <lollipop@apache.org>
|
|||
|
|
+Date: Thu, 21 Sep 2023 16:21:44 +0800
|
|||
|
|
+Subject: [PATCH 02/12] Replace loggingMetricExporter with
|
|||
|
|
+ OtlpJsonLoggingMetricExporter. (#7373)
|
|||
|
|
+
|
|||
|
|
+* Replace loggingMetricExporter with OtlpJsonLoggingMetricExporter.
|
|||
|
|
+
|
|||
|
|
+* Fix bazel workspace
|
|||
|
|
+
|
|||
|
|
+* Move OtlpJsonLoggingMetricExporter to proxy and controller.
|
|||
|
|
+
|
|||
|
|
+* Fix Bazel imports.
|
|||
|
|
+---
|
|||
|
|
+ WORKSPACE | 1 +
|
|||
|
|
+ broker/BUILD.bazel | 1 +
|
|||
|
|
+ .../rocketmq/broker/metrics/BrokerMetricsManager.java | 9 +++++----
|
|||
|
|
+ broker/src/main/resources/rmq.broker.logback.xml | 5 +++++
|
|||
|
|
+ common/BUILD.bazel | 1 +
|
|||
|
|
+ common/pom.xml | 4 ++++
|
|||
|
|
+ controller/BUILD.bazel | 1 +
|
|||
|
|
+ .../controller/metrics/ControllerMetricsManager.java | 9 +++++----
|
|||
|
|
+ pom.xml | 5 +++++
|
|||
|
|
+ proxy/BUILD.bazel | 1 +
|
|||
|
|
+ .../rocketmq/proxy/metrics/ProxyMetricsManager.java | 11 ++++++-----
|
|||
|
|
+ proxy/src/main/resources/rmq.proxy.logback.xml | 5 +++++
|
|||
|
|
+ tieredstore/BUILD.bazel | 1 +
|
|||
|
|
+ 13 files changed, 41 insertions(+), 13 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/WORKSPACE b/WORKSPACE
|
|||
|
|
+index 3126f2d1d..8640485ba 100644
|
|||
|
|
+--- a/WORKSPACE
|
|||
|
|
++++ b/WORKSPACE
|
|||
|
|
+@@ -92,6 +92,7 @@ maven_install(
|
|||
|
|
+ "io.opentelemetry:opentelemetry-exporter-prometheus:1.29.0-alpha",
|
|||
|
|
+ "io.opentelemetry:opentelemetry-exporter-logging:1.29.0",
|
|||
|
|
+ "io.opentelemetry:opentelemetry-sdk:1.29.0",
|
|||
|
|
++ "io.opentelemetry:opentelemetry-exporter-logging-otlp:1.29.0",
|
|||
|
|
+ "com.squareup.okio:okio-jvm:3.0.0",
|
|||
|
|
+ "io.opentelemetry:opentelemetry-api:1.29.0",
|
|||
|
|
+ "io.opentelemetry:opentelemetry-sdk-metrics:1.29.0",
|
|||
|
|
+diff --git a/broker/BUILD.bazel b/broker/BUILD.bazel
|
|||
|
|
+index 6adcdc7b9..64cb1b341 100644
|
|||
|
|
+--- a/broker/BUILD.bazel
|
|||
|
|
++++ b/broker/BUILD.bazel
|
|||
|
|
+@@ -44,6 +44,7 @@ java_library(
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_otlp",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_prometheus",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_logging",
|
|||
|
|
++ "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_common",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_metrics",
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
+index 6af5afc14..39af18b9f 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/BrokerMetricsManager.java
|
|||
|
|
+@@ -23,7 +23,7 @@ import io.opentelemetry.api.metrics.LongCounter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.LongHistogram;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.ObservableLongGauge;
|
|||
|
|
+-import io.opentelemetry.exporter.logging.LoggingMetricExporter;
|
|||
|
|
++import io.opentelemetry.exporter.logging.otlp.OtlpJsonLoggingMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporterBuilder;
|
|||
|
|
+ import io.opentelemetry.exporter.prometheus.PrometheusHttpServer;
|
|||
|
|
+@@ -36,6 +36,7 @@ import io.opentelemetry.sdk.metrics.SdkMeterProviderBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.export.MetricExporter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.internal.SdkMeterProviderUtil;
|
|||
|
|
+ import io.opentelemetry.sdk.resources.Resource;
|
|||
|
|
+@@ -113,7 +114,7 @@ public class BrokerMetricsManager {
|
|||
|
|
+ private OtlpGrpcMetricExporter metricExporter;
|
|||
|
|
+ private PeriodicMetricReader periodicMetricReader;
|
|||
|
|
+ private PrometheusHttpServer prometheusHttpServer;
|
|||
|
|
+- private LoggingMetricExporter loggingMetricExporter;
|
|||
|
|
++ private MetricExporter loggingMetricExporter;
|
|||
|
|
+ private Meter brokerMeter;
|
|||
|
|
+
|
|||
|
|
+ public static Supplier<AttributesBuilder> attributesBuilderSupplier = Attributes::builder;
|
|||
|
|
+@@ -327,8 +328,8 @@ public class BrokerMetricsManager {
|
|||
|
|
+ if (metricsExporterType == MetricsExporterType.LOG) {
|
|||
|
|
+ SLF4JBridgeHandler.removeHandlersForRootLogger();
|
|||
|
|
+ SLF4JBridgeHandler.install();
|
|||
|
|
+- loggingMetricExporter = LoggingMetricExporter.create(brokerConfig.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
+- java.util.logging.Logger.getLogger(LoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
++ loggingMetricExporter = OtlpJsonLoggingMetricExporter.create(brokerConfig.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
++ java.util.logging.Logger.getLogger(OtlpJsonLoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
+ periodicMetricReader = PeriodicMetricReader.builder(loggingMetricExporter)
|
|||
|
|
+ .setInterval(brokerConfig.getMetricLoggingExporterIntervalInMills(), TimeUnit.MILLISECONDS)
|
|||
|
|
+ .build();
|
|||
|
|
+diff --git a/broker/src/main/resources/rmq.broker.logback.xml b/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
+index 3c51e59d4..32dc29736 100644
|
|||
|
|
+--- a/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
++++ b/broker/src/main/resources/rmq.broker.logback.xml
|
|||
|
|
+@@ -672,6 +672,11 @@
|
|||
|
|
+ <appender-ref ref="RocketmqTrafficSiftingAppender"/>
|
|||
|
|
+ </logger>
|
|||
|
|
+
|
|||
|
|
++ <!-- Use json formatter to log metrics -->
|
|||
|
|
++ <logger name="io.opentelemetry.exporter.logging.otlp.OtlpJsonLoggingMetricExporter" additivity="false" level="INFO">
|
|||
|
|
++ <appender-ref ref="RocketmqBrokerMetricsSiftingAppender"/>
|
|||
|
|
++ </logger>
|
|||
|
|
++
|
|||
|
|
+ <logger name="io.opentelemetry.exporter.logging.LoggingMetricExporter" additivity="false" level="INFO">
|
|||
|
|
+ <appender-ref ref="RocketmqBrokerMetricsSiftingAppender"/>
|
|||
|
|
+ </logger>
|
|||
|
|
+diff --git a/common/BUILD.bazel b/common/BUILD.bazel
|
|||
|
|
+index a95a19ccd..e6701d0fc 100644
|
|||
|
|
+--- a/common/BUILD.bazel
|
|||
|
|
++++ b/common/BUILD.bazel
|
|||
|
|
+@@ -35,6 +35,7 @@ java_library(
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_common",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_metrics",
|
|||
|
|
++ "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp",
|
|||
|
|
+ "@maven//:org_apache_commons_commons_lang3",
|
|||
|
|
+ "@maven//:org_lz4_lz4_java",
|
|||
|
|
+ "@maven//:io_github_aliyunmq_rocketmq_slf4j_api",
|
|||
|
|
+diff --git a/common/pom.xml b/common/pom.xml
|
|||
|
|
+index 31eb0f087..accc7f0a8 100644
|
|||
|
|
+--- a/common/pom.xml
|
|||
|
|
++++ b/common/pom.xml
|
|||
|
|
+@@ -80,6 +80,10 @@
|
|||
|
|
+ <groupId>io.opentelemetry</groupId>
|
|||
|
|
+ <artifactId>opentelemetry-sdk</artifactId>
|
|||
|
|
+ </dependency>
|
|||
|
|
++ <dependency>
|
|||
|
|
++ <groupId>io.opentelemetry</groupId>
|
|||
|
|
++ <artifactId>opentelemetry-exporter-logging-otlp</artifactId>
|
|||
|
|
++ </dependency>
|
|||
|
|
+ <dependency>
|
|||
|
|
+ <groupId>io.grpc</groupId>
|
|||
|
|
+ <artifactId>grpc-stub</artifactId>
|
|||
|
|
+diff --git a/controller/BUILD.bazel b/controller/BUILD.bazel
|
|||
|
|
+index 843d9dc77..b2b743eb2 100644
|
|||
|
|
+--- a/controller/BUILD.bazel
|
|||
|
|
++++ b/controller/BUILD.bazel
|
|||
|
|
+@@ -49,6 +49,7 @@ java_library(
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_common",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_metrics",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_logging",
|
|||
|
|
++ "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp",
|
|||
|
|
+ "@maven//:org_slf4j_jul_to_slf4j",
|
|||
|
|
+ ],
|
|||
|
|
+ )
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java b/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
+index 650740bcc..be9e77eea 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/metrics/ControllerMetricsManager.java
|
|||
|
|
+@@ -26,7 +26,7 @@ import io.opentelemetry.api.metrics.LongHistogram;
|
|||
|
|
+ import io.opentelemetry.api.metrics.LongUpDownCounter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.ObservableLongGauge;
|
|||
|
|
+-import io.opentelemetry.exporter.logging.LoggingMetricExporter;
|
|||
|
|
++import io.opentelemetry.exporter.logging.otlp.OtlpJsonLoggingMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporterBuilder;
|
|||
|
|
+ import io.opentelemetry.exporter.prometheus.PrometheusHttpServer;
|
|||
|
|
+@@ -38,6 +38,7 @@ import io.opentelemetry.sdk.metrics.SdkMeterProvider;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.SdkMeterProviderBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.View;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.export.MetricExporter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader;
|
|||
|
|
+ import io.opentelemetry.sdk.resources.Resource;
|
|||
|
|
+ import java.io.File;
|
|||
|
|
+@@ -121,7 +122,7 @@ public class ControllerMetricsManager {
|
|||
|
|
+
|
|||
|
|
+ private PrometheusHttpServer prometheusHttpServer;
|
|||
|
|
+
|
|||
|
|
+- private LoggingMetricExporter loggingMetricExporter;
|
|||
|
|
++ private MetricExporter loggingMetricExporter;
|
|||
|
|
+
|
|||
|
|
+ public static ControllerMetricsManager getInstance(ControllerManager controllerManager) {
|
|||
|
|
+ if (instance == null) {
|
|||
|
|
+@@ -364,8 +365,8 @@ public class ControllerMetricsManager {
|
|||
|
|
+ if (type == MetricsExporterType.LOG) {
|
|||
|
|
+ SLF4JBridgeHandler.removeHandlersForRootLogger();
|
|||
|
|
+ SLF4JBridgeHandler.install();
|
|||
|
|
+- loggingMetricExporter = LoggingMetricExporter.create(config.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
+- java.util.logging.Logger.getLogger(LoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
++ loggingMetricExporter = OtlpJsonLoggingMetricExporter.create(config.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
++ java.util.logging.Logger.getLogger(OtlpJsonLoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
+ periodicMetricReader = PeriodicMetricReader.builder(loggingMetricExporter)
|
|||
|
|
+ .setInterval(config.getMetricLoggingExporterIntervalInMills(), TimeUnit.MILLISECONDS)
|
|||
|
|
+ .build();
|
|||
|
|
+diff --git a/pom.xml b/pom.xml
|
|||
|
|
+index 9f0b3eb96..4b382c6da 100644
|
|||
|
|
+--- a/pom.xml
|
|||
|
|
++++ b/pom.xml
|
|||
|
|
+@@ -974,6 +974,11 @@
|
|||
|
|
+ <artifactId>opentelemetry-sdk</artifactId>
|
|||
|
|
+ <version>${opentelemetry.version}</version>
|
|||
|
|
+ </dependency>
|
|||
|
|
++ <dependency>
|
|||
|
|
++ <groupId>io.opentelemetry</groupId>
|
|||
|
|
++ <artifactId>opentelemetry-exporter-logging-otlp</artifactId>
|
|||
|
|
++ <version>${opentelemetry.version}</version>
|
|||
|
|
++ </dependency>
|
|||
|
|
+ <dependency>
|
|||
|
|
+ <groupId>org.slf4j</groupId>
|
|||
|
|
+ <artifactId>jul-to-slf4j</artifactId>
|
|||
|
|
+diff --git a/proxy/BUILD.bazel b/proxy/BUILD.bazel
|
|||
|
|
+index b4f3c16e2..cb7af9254 100644
|
|||
|
|
+--- a/proxy/BUILD.bazel
|
|||
|
|
++++ b/proxy/BUILD.bazel
|
|||
|
|
+@@ -52,6 +52,7 @@ java_library(
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_otlp",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_prometheus",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_exporter_logging",
|
|||
|
|
++ "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_common",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_metrics",
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/metrics/ProxyMetricsManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/metrics/ProxyMetricsManager.java
|
|||
|
|
+index f5050858f..2b8dac5d8 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/metrics/ProxyMetricsManager.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/metrics/ProxyMetricsManager.java
|
|||
|
|
+@@ -21,15 +21,16 @@ import io.opentelemetry.api.common.Attributes;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+ import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+ import io.opentelemetry.api.metrics.ObservableLongGauge;
|
|||
|
|
++import io.opentelemetry.exporter.logging.otlp.OtlpJsonLoggingMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporter;
|
|||
|
|
+ import io.opentelemetry.exporter.otlp.metrics.OtlpGrpcMetricExporterBuilder;
|
|||
|
|
+ import io.opentelemetry.exporter.prometheus.PrometheusHttpServer;
|
|||
|
|
+-import io.opentelemetry.exporter.logging.LoggingMetricExporter;
|
|||
|
|
+ import io.opentelemetry.sdk.OpenTelemetrySdk;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.InstrumentType;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.SdkMeterProvider;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.SdkMeterProviderBuilder;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.data.AggregationTemporality;
|
|||
|
|
++import io.opentelemetry.sdk.metrics.export.MetricExporter;
|
|||
|
|
+ import io.opentelemetry.sdk.metrics.export.PeriodicMetricReader;
|
|||
|
|
+ import io.opentelemetry.sdk.resources.Resource;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+@@ -42,9 +43,9 @@ import org.apache.rocketmq.broker.metrics.BrokerMetricsManager;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.metrics.MetricsExporterType;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.StartAndShutdown;
|
|||
|
|
+-import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
++import org.apache.rocketmq.proxy.config.ProxyConfig;
|
|||
|
|
+ import org.slf4j.bridge.SLF4JBridgeHandler;
|
|||
|
|
+
|
|||
|
|
+ import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.AGGREGATION_DELTA;
|
|||
|
|
+@@ -67,7 +68,7 @@ public class ProxyMetricsManager implements StartAndShutdown {
|
|||
|
|
+ private OtlpGrpcMetricExporter metricExporter;
|
|||
|
|
+ private PeriodicMetricReader periodicMetricReader;
|
|||
|
|
+ private PrometheusHttpServer prometheusHttpServer;
|
|||
|
|
+- private LoggingMetricExporter loggingMetricExporter;
|
|||
|
|
++ private MetricExporter loggingMetricExporter;
|
|||
|
|
+
|
|||
|
|
+ public static ObservableLongGauge proxyUp = null;
|
|||
|
|
+
|
|||
|
|
+@@ -221,8 +222,8 @@ public class ProxyMetricsManager implements StartAndShutdown {
|
|||
|
|
+ if (metricsExporterType == MetricsExporterType.LOG) {
|
|||
|
|
+ SLF4JBridgeHandler.removeHandlersForRootLogger();
|
|||
|
|
+ SLF4JBridgeHandler.install();
|
|||
|
|
+- loggingMetricExporter = LoggingMetricExporter.create(proxyConfig.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
+- java.util.logging.Logger.getLogger(LoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
++ loggingMetricExporter = OtlpJsonLoggingMetricExporter.create(proxyConfig.isMetricsInDelta() ? AggregationTemporality.DELTA : AggregationTemporality.CUMULATIVE);
|
|||
|
|
++ java.util.logging.Logger.getLogger(OtlpJsonLoggingMetricExporter.class.getName()).setLevel(java.util.logging.Level.FINEST);
|
|||
|
|
+ periodicMetricReader = PeriodicMetricReader.builder(loggingMetricExporter)
|
|||
|
|
+ .setInterval(proxyConfig.getMetricLoggingExporterIntervalInMills(), TimeUnit.MILLISECONDS)
|
|||
|
|
+ .build();
|
|||
|
|
+diff --git a/proxy/src/main/resources/rmq.proxy.logback.xml b/proxy/src/main/resources/rmq.proxy.logback.xml
|
|||
|
|
+index d38827f92..f968a45e6 100644
|
|||
|
|
+--- a/proxy/src/main/resources/rmq.proxy.logback.xml
|
|||
|
|
++++ b/proxy/src/main/resources/rmq.proxy.logback.xml
|
|||
|
|
+@@ -418,6 +418,11 @@
|
|||
|
|
+ <appender-ref ref="RocketmqProxyWatermarkAppender" />
|
|||
|
|
+ </logger>
|
|||
|
|
+
|
|||
|
|
++ <!-- Use json formatter to log metrics -->
|
|||
|
|
++ <logger name="io.opentelemetry.exporter.logging.otlp.OtlpJsonLoggingMetricExporter" additivity="false" level="INFO">
|
|||
|
|
++ <appender-ref ref="RocketmqProxyMetricsAppender"/>
|
|||
|
|
++ </logger>
|
|||
|
|
++
|
|||
|
|
+ <logger name="io.opentelemetry.exporter.logging.LoggingMetricExporter" additivity="false" level="INFO">
|
|||
|
|
+ <appender-ref ref="RocketmqProxyMetricsAppender" />
|
|||
|
|
+ </logger>
|
|||
|
|
+diff --git a/tieredstore/BUILD.bazel b/tieredstore/BUILD.bazel
|
|||
|
|
+index 5b3885a4e..dea2c561b 100644
|
|||
|
|
+--- a/tieredstore/BUILD.bazel
|
|||
|
|
++++ b/tieredstore/BUILD.bazel
|
|||
|
|
+@@ -36,6 +36,7 @@ java_library(
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_common",
|
|||
|
|
+ "@maven//:io_opentelemetry_opentelemetry_sdk_metrics",
|
|||
|
|
++ "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp",
|
|||
|
|
+ "@maven//:org_apache_commons_commons_lang3",
|
|||
|
|
+ "@maven//:org_apache_tomcat_annotations_api",
|
|||
|
|
+ "@maven//:com_alibaba_fastjson",
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 1a681bdf9b5c5ab0be446d6394c0cac8768f45d9 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Thu, 21 Sep 2023 19:58:29 +0800
|
|||
|
|
+Subject: [PATCH 03/12] [maven-release-plugin] prepare release
|
|||
|
|
+ rocketmq-all-5.1.4 (#7377)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ acl/pom.xml | 2 +-
|
|||
|
|
+ broker/pom.xml | 2 +-
|
|||
|
|
+ client/pom.xml | 2 +-
|
|||
|
|
+ common/pom.xml | 2 +-
|
|||
|
|
+ container/pom.xml | 2 +-
|
|||
|
|
+ controller/pom.xml | 2 +-
|
|||
|
|
+ distribution/pom.xml | 2 +-
|
|||
|
|
+ example/pom.xml | 2 +-
|
|||
|
|
+ filter/pom.xml | 2 +-
|
|||
|
|
+ namesrv/pom.xml | 2 +-
|
|||
|
|
+ openmessaging/pom.xml | 2 +-
|
|||
|
|
+ pom.xml | 4 ++--
|
|||
|
|
+ proxy/pom.xml | 2 +-
|
|||
|
|
+ remoting/pom.xml | 2 +-
|
|||
|
|
+ srvutil/pom.xml | 2 +-
|
|||
|
|
+ store/pom.xml | 2 +-
|
|||
|
|
+ test/pom.xml | 2 +-
|
|||
|
|
+ tieredstore/pom.xml | 2 +-
|
|||
|
|
+ tools/pom.xml | 2 +-
|
|||
|
|
+ 19 files changed, 20 insertions(+), 20 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/acl/pom.xml b/acl/pom.xml
|
|||
|
|
+index 989c0cf77..9f6838b00 100644
|
|||
|
|
+--- a/acl/pom.xml
|
|||
|
|
++++ b/acl/pom.xml
|
|||
|
|
+@@ -13,7 +13,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <artifactId>rocketmq-acl</artifactId>
|
|||
|
|
+ <name>rocketmq-acl ${project.version}</name>
|
|||
|
|
+diff --git a/broker/pom.xml b/broker/pom.xml
|
|||
|
|
+index 16e026276..d483e67ba 100644
|
|||
|
|
+--- a/broker/pom.xml
|
|||
|
|
++++ b/broker/pom.xml
|
|||
|
|
+@@ -13,7 +13,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/client/pom.xml b/client/pom.xml
|
|||
|
|
+index c59a43889..4febedc6d 100644
|
|||
|
|
+--- a/client/pom.xml
|
|||
|
|
++++ b/client/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/common/pom.xml b/common/pom.xml
|
|||
|
|
+index accc7f0a8..b70873dfa 100644
|
|||
|
|
+--- a/common/pom.xml
|
|||
|
|
++++ b/common/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/container/pom.xml b/container/pom.xml
|
|||
|
|
+index c8499f127..e6c1f4b4d 100644
|
|||
|
|
+--- a/container/pom.xml
|
|||
|
|
++++ b/container/pom.xml
|
|||
|
|
+@@ -18,7 +18,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/controller/pom.xml b/controller/pom.xml
|
|||
|
|
+index 3346c7c82..46a3834c6 100644
|
|||
|
|
+--- a/controller/pom.xml
|
|||
|
|
++++ b/controller/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+ <packaging>jar</packaging>
|
|||
|
|
+diff --git a/distribution/pom.xml b/distribution/pom.xml
|
|||
|
|
+index dbde2d9d4..346c4de35 100644
|
|||
|
|
+--- a/distribution/pom.xml
|
|||
|
|
++++ b/distribution/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <artifactId>rocketmq-distribution</artifactId>
|
|||
|
|
+ <name>rocketmq-distribution ${project.version}</name>
|
|||
|
|
+diff --git a/example/pom.xml b/example/pom.xml
|
|||
|
|
+index 862fc3169..9e7db43f8 100644
|
|||
|
|
+--- a/example/pom.xml
|
|||
|
|
++++ b/example/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/filter/pom.xml b/filter/pom.xml
|
|||
|
|
+index 3fe51ceae..84189066d 100644
|
|||
|
|
+--- a/filter/pom.xml
|
|||
|
|
++++ b/filter/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/namesrv/pom.xml b/namesrv/pom.xml
|
|||
|
|
+index 684b2683c..7c218078a 100644
|
|||
|
|
+--- a/namesrv/pom.xml
|
|||
|
|
++++ b/namesrv/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml
|
|||
|
|
+index aaa4c896c..fd499e3de 100644
|
|||
|
|
+--- a/openmessaging/pom.xml
|
|||
|
|
++++ b/openmessaging/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/pom.xml b/pom.xml
|
|||
|
|
+index 4b382c6da..0e1d04f15 100644
|
|||
|
|
+--- a/pom.xml
|
|||
|
|
++++ b/pom.xml
|
|||
|
|
+@@ -28,7 +28,7 @@
|
|||
|
|
+ <inceptionYear>2012</inceptionYear>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ <packaging>pom</packaging>
|
|||
|
|
+ <name>Apache RocketMQ ${project.version}</name>
|
|||
|
|
+ <url>http://rocketmq.apache.org/</url>
|
|||
|
|
+@@ -37,7 +37,7 @@
|
|||
|
|
+ <url>git@github.com:apache/rocketmq.git</url>
|
|||
|
|
+ <connection>scm:git:git@github.com:apache/rocketmq.git</connection>
|
|||
|
|
+ <developerConnection>scm:git:git@github.com:apache/rocketmq.git</developerConnection>
|
|||
|
|
+- <tag>HEAD</tag>
|
|||
|
|
++ <tag>rocketmq-all-5.1.4</tag>
|
|||
|
|
+ </scm>
|
|||
|
|
+
|
|||
|
|
+ <mailingLists>
|
|||
|
|
+diff --git a/proxy/pom.xml b/proxy/pom.xml
|
|||
|
|
+index 3fbea107a..abf242eee 100644
|
|||
|
|
+--- a/proxy/pom.xml
|
|||
|
|
++++ b/proxy/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/remoting/pom.xml b/remoting/pom.xml
|
|||
|
|
+index 8a43c5c30..fc70cb62e 100644
|
|||
|
|
+--- a/remoting/pom.xml
|
|||
|
|
++++ b/remoting/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/srvutil/pom.xml b/srvutil/pom.xml
|
|||
|
|
+index fa54ad019..d7f946cce 100644
|
|||
|
|
+--- a/srvutil/pom.xml
|
|||
|
|
++++ b/srvutil/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/store/pom.xml b/store/pom.xml
|
|||
|
|
+index 38f04009d..6d6983c5d 100644
|
|||
|
|
+--- a/store/pom.xml
|
|||
|
|
++++ b/store/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/test/pom.xml b/test/pom.xml
|
|||
|
|
+index 8f25c35c9..39090e426 100644
|
|||
|
|
+--- a/test/pom.xml
|
|||
|
|
++++ b/test/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/tieredstore/pom.xml b/tieredstore/pom.xml
|
|||
|
|
+index c476040ba..7b209751f 100644
|
|||
|
|
+--- a/tieredstore/pom.xml
|
|||
|
|
++++ b/tieredstore/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/tools/pom.xml b/tools/pom.xml
|
|||
|
|
+index 1c3b431bc..806787ec9 100644
|
|||
|
|
+--- a/tools/pom.xml
|
|||
|
|
++++ b/tools/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4-SNAPSHOT</version>
|
|||
|
|
++ <version>5.1.4</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 73b3fde83765e066541e3455cd1e6604292a9b7c Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Fri, 22 Sep 2023 10:08:59 +0800
|
|||
|
|
+Subject: [PATCH 04/12] [maven-release-plugin] prepare for next development
|
|||
|
|
+ iteration (#7379)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ acl/pom.xml | 2 +-
|
|||
|
|
+ broker/pom.xml | 2 +-
|
|||
|
|
+ client/pom.xml | 2 +-
|
|||
|
|
+ common/pom.xml | 2 +-
|
|||
|
|
+ container/pom.xml | 2 +-
|
|||
|
|
+ controller/pom.xml | 2 +-
|
|||
|
|
+ distribution/pom.xml | 2 +-
|
|||
|
|
+ example/pom.xml | 2 +-
|
|||
|
|
+ filter/pom.xml | 2 +-
|
|||
|
|
+ namesrv/pom.xml | 2 +-
|
|||
|
|
+ openmessaging/pom.xml | 2 +-
|
|||
|
|
+ pom.xml | 4 ++--
|
|||
|
|
+ proxy/pom.xml | 2 +-
|
|||
|
|
+ remoting/pom.xml | 2 +-
|
|||
|
|
+ srvutil/pom.xml | 2 +-
|
|||
|
|
+ store/pom.xml | 2 +-
|
|||
|
|
+ test/pom.xml | 2 +-
|
|||
|
|
+ tieredstore/pom.xml | 2 +-
|
|||
|
|
+ tools/pom.xml | 2 +-
|
|||
|
|
+ 19 files changed, 20 insertions(+), 20 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/acl/pom.xml b/acl/pom.xml
|
|||
|
|
+index 9f6838b00..8a296e5ae 100644
|
|||
|
|
+--- a/acl/pom.xml
|
|||
|
|
++++ b/acl/pom.xml
|
|||
|
|
+@@ -13,7 +13,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <artifactId>rocketmq-acl</artifactId>
|
|||
|
|
+ <name>rocketmq-acl ${project.version}</name>
|
|||
|
|
+diff --git a/broker/pom.xml b/broker/pom.xml
|
|||
|
|
+index d483e67ba..add83045d 100644
|
|||
|
|
+--- a/broker/pom.xml
|
|||
|
|
++++ b/broker/pom.xml
|
|||
|
|
+@@ -13,7 +13,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/client/pom.xml b/client/pom.xml
|
|||
|
|
+index 4febedc6d..d6fb3889b 100644
|
|||
|
|
+--- a/client/pom.xml
|
|||
|
|
++++ b/client/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/common/pom.xml b/common/pom.xml
|
|||
|
|
+index b70873dfa..6104c3ac6 100644
|
|||
|
|
+--- a/common/pom.xml
|
|||
|
|
++++ b/common/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/container/pom.xml b/container/pom.xml
|
|||
|
|
+index e6c1f4b4d..8af231e01 100644
|
|||
|
|
+--- a/container/pom.xml
|
|||
|
|
++++ b/container/pom.xml
|
|||
|
|
+@@ -18,7 +18,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/controller/pom.xml b/controller/pom.xml
|
|||
|
|
+index 46a3834c6..8432b220b 100644
|
|||
|
|
+--- a/controller/pom.xml
|
|||
|
|
++++ b/controller/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+ <packaging>jar</packaging>
|
|||
|
|
+diff --git a/distribution/pom.xml b/distribution/pom.xml
|
|||
|
|
+index 346c4de35..73474d34a 100644
|
|||
|
|
+--- a/distribution/pom.xml
|
|||
|
|
++++ b/distribution/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+ <artifactId>rocketmq-distribution</artifactId>
|
|||
|
|
+ <name>rocketmq-distribution ${project.version}</name>
|
|||
|
|
+diff --git a/example/pom.xml b/example/pom.xml
|
|||
|
|
+index 9e7db43f8..a8c7f5382 100644
|
|||
|
|
+--- a/example/pom.xml
|
|||
|
|
++++ b/example/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/filter/pom.xml b/filter/pom.xml
|
|||
|
|
+index 84189066d..892f46e9d 100644
|
|||
|
|
+--- a/filter/pom.xml
|
|||
|
|
++++ b/filter/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/namesrv/pom.xml b/namesrv/pom.xml
|
|||
|
|
+index 7c218078a..e320ed573 100644
|
|||
|
|
+--- a/namesrv/pom.xml
|
|||
|
|
++++ b/namesrv/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml
|
|||
|
|
+index fd499e3de..f10c8af6f 100644
|
|||
|
|
+--- a/openmessaging/pom.xml
|
|||
|
|
++++ b/openmessaging/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/pom.xml b/pom.xml
|
|||
|
|
+index 0e1d04f15..4202d4095 100644
|
|||
|
|
+--- a/pom.xml
|
|||
|
|
++++ b/pom.xml
|
|||
|
|
+@@ -28,7 +28,7 @@
|
|||
|
|
+ <inceptionYear>2012</inceptionYear>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ <packaging>pom</packaging>
|
|||
|
|
+ <name>Apache RocketMQ ${project.version}</name>
|
|||
|
|
+ <url>http://rocketmq.apache.org/</url>
|
|||
|
|
+@@ -37,7 +37,7 @@
|
|||
|
|
+ <url>git@github.com:apache/rocketmq.git</url>
|
|||
|
|
+ <connection>scm:git:git@github.com:apache/rocketmq.git</connection>
|
|||
|
|
+ <developerConnection>scm:git:git@github.com:apache/rocketmq.git</developerConnection>
|
|||
|
|
+- <tag>rocketmq-all-5.1.4</tag>
|
|||
|
|
++ <tag>HEAD</tag>
|
|||
|
|
+ </scm>
|
|||
|
|
+
|
|||
|
|
+ <mailingLists>
|
|||
|
|
+diff --git a/proxy/pom.xml b/proxy/pom.xml
|
|||
|
|
+index abf242eee..5c5349a8c 100644
|
|||
|
|
+--- a/proxy/pom.xml
|
|||
|
|
++++ b/proxy/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/remoting/pom.xml b/remoting/pom.xml
|
|||
|
|
+index fc70cb62e..f78480680 100644
|
|||
|
|
+--- a/remoting/pom.xml
|
|||
|
|
++++ b/remoting/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/srvutil/pom.xml b/srvutil/pom.xml
|
|||
|
|
+index d7f946cce..894e9cc6f 100644
|
|||
|
|
+--- a/srvutil/pom.xml
|
|||
|
|
++++ b/srvutil/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/store/pom.xml b/store/pom.xml
|
|||
|
|
+index 6d6983c5d..e979030e8 100644
|
|||
|
|
+--- a/store/pom.xml
|
|||
|
|
++++ b/store/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/test/pom.xml b/test/pom.xml
|
|||
|
|
+index 39090e426..168cbab0b 100644
|
|||
|
|
+--- a/test/pom.xml
|
|||
|
|
++++ b/test/pom.xml
|
|||
|
|
+@@ -20,7 +20,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/tieredstore/pom.xml b/tieredstore/pom.xml
|
|||
|
|
+index 7b209751f..b2ea40bf3 100644
|
|||
|
|
+--- a/tieredstore/pom.xml
|
|||
|
|
++++ b/tieredstore/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+diff --git a/tools/pom.xml b/tools/pom.xml
|
|||
|
|
+index 806787ec9..e1daa57a6 100644
|
|||
|
|
+--- a/tools/pom.xml
|
|||
|
|
++++ b/tools/pom.xml
|
|||
|
|
+@@ -19,7 +19,7 @@
|
|||
|
|
+ <parent>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
+ <artifactId>rocketmq-all</artifactId>
|
|||
|
|
+- <version>5.1.4</version>
|
|||
|
|
++ <version>5.1.5-SNAPSHOT</version>
|
|||
|
|
+ </parent>
|
|||
|
|
+
|
|||
|
|
+ <modelVersion>4.0.0</modelVersion>
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 88a9d939ce110381b3b418370d4711c0c214dc7f Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Ji Juntao <juntao.jjt@alibaba-inc.com>
|
|||
|
|
+Date: Sat, 23 Sep 2023 17:38:27 +0800
|
|||
|
|
+Subject: [PATCH 05/12] [ISSUE #7381] Fix the problem of inaccurate timer
|
|||
|
|
+ message metric (#7382)
|
|||
|
|
+
|
|||
|
|
+* correct the timerMetrics' result.
|
|||
|
|
+
|
|||
|
|
+* for further extension.
|
|||
|
|
+
|
|||
|
|
+* checkstyle.
|
|||
|
|
+
|
|||
|
|
+* use toLong.
|
|||
|
|
+---
|
|||
|
|
+ .../store/timer/TimerMessageStore.java | 20 +++++++++++++++----
|
|||
|
|
+ .../rocketmq/store/timer/TimerMetrics.java | 5 ++++-
|
|||
|
|
+ .../rocketmq/store/timer/TimerRequest.java | 7 +++++--
|
|||
|
|
+ .../store/timer/TimerMetricsTest.java | 10 ++++++++--
|
|||
|
|
+ 4 files changed, 33 insertions(+), 9 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+index 0d50de65a..ac4c61cd6 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+@@ -41,6 +41,7 @@ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.function.Function;
|
|||
|
|
+ import org.apache.commons.collections.CollectionUtils;
|
|||
|
|
++import org.apache.commons.lang3.math.NumberUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.ServiceThread;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicFilterType;
|
|||
|
|
+@@ -599,7 +600,12 @@ public class TimerMessageStore {
|
|||
|
|
+ if (null == msg || null == msg.getProperty(MessageConst.PROPERTY_REAL_TOPIC)) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+- timerMetrics.addAndGet(msg.getProperty(MessageConst.PROPERTY_REAL_TOPIC), value);
|
|||
|
|
++ if (msg.getProperty(TIMER_ENQUEUE_MS) != null
|
|||
|
|
++ && NumberUtils.toLong(msg.getProperty(TIMER_ENQUEUE_MS)) == Long.MAX_VALUE) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ // pass msg into addAndGet, for further more judgement extension.
|
|||
|
|
++ timerMetrics.addAndGet(msg, value);
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ if (frequency.incrementAndGet() % 1000 == 0) {
|
|||
|
|
+ LOGGER.error("error in adding metric", t);
|
|||
|
|
+@@ -1323,6 +1329,7 @@ public class TimerMessageStore {
|
|||
|
|
+ perfCounterTicks.startTick(ENQUEUE_PUT);
|
|||
|
|
+ DefaultStoreMetricsManager.incTimerEnqueueCount(getRealTopic(req.getMsg()));
|
|||
|
|
+ if (shouldRunningDequeue && req.getDelayTime() < currWriteTimeMs) {
|
|||
|
|
++ req.setEnqueueTime(Long.MAX_VALUE);
|
|||
|
|
+ dequeuePutQueue.put(req);
|
|||
|
|
+ } else {
|
|||
|
|
+ boolean doEnqueueRes = doEnqueue(
|
|||
|
|
+@@ -1452,9 +1459,14 @@ public class TimerMessageStore {
|
|||
|
|
+ }
|
|||
|
|
+ try {
|
|||
|
|
+ perfCounterTicks.startTick(DEQUEUE_PUT);
|
|||
|
|
+- DefaultStoreMetricsManager.incTimerDequeueCount(getRealTopic(tr.getMsg()));
|
|||
|
|
+- addMetric(tr.getMsg(), -1);
|
|||
|
|
+- MessageExtBrokerInner msg = convert(tr.getMsg(), tr.getEnqueueTime(), needRoll(tr.getMagic()));
|
|||
|
|
++ MessageExt msgExt = tr.getMsg();
|
|||
|
|
++ DefaultStoreMetricsManager.incTimerDequeueCount(getRealTopic(msgExt));
|
|||
|
|
++ if (tr.getEnqueueTime() == Long.MAX_VALUE) {
|
|||
|
|
++ // never enqueue, mark it.
|
|||
|
|
++ MessageAccessor.putProperty(msgExt, TIMER_ENQUEUE_MS, String.valueOf(Long.MAX_VALUE));
|
|||
|
|
++ }
|
|||
|
|
++ addMetric(msgExt, -1);
|
|||
|
|
++ MessageExtBrokerInner msg = convert(msgExt, tr.getEnqueueTime(), needRoll(tr.getMagic()));
|
|||
|
|
+ doRes = PUT_NEED_RETRY != doPut(msg, needRoll(tr.getMagic()));
|
|||
|
|
+ while (!doRes && !isStopped()) {
|
|||
|
|
+ if (!isRunningDequeue()) {
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java
|
|||
|
|
+index e7b00cc07..7f8fedd8a 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java
|
|||
|
|
+@@ -38,6 +38,8 @@ import java.util.concurrent.locks.Lock;
|
|||
|
|
+ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
+ import org.apache.rocketmq.common.ConfigManager;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+@@ -78,7 +80,8 @@ public class TimerMetrics extends ConfigManager {
|
|||
|
|
+ return distPair.getCount().addAndGet(value);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public long addAndGet(String topic, int value) {
|
|||
|
|
++ public long addAndGet(MessageExt msg, int value) {
|
|||
|
|
++ String topic = msg.getProperty(MessageConst.PROPERTY_REAL_TOPIC);
|
|||
|
|
+ Metric pair = getTopicPair(topic);
|
|||
|
|
+ getDataVersion().nextVersion();
|
|||
|
|
+ pair.setTimeStamp(System.currentTimeMillis());
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerRequest.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerRequest.java
|
|||
|
|
+index 1dd64f759..1b25d355c 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerRequest.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerRequest.java
|
|||
|
|
+@@ -27,8 +27,9 @@ public class TimerRequest {
|
|||
|
|
+ private final int sizePy;
|
|||
|
|
+ private final long delayTime;
|
|||
|
|
+
|
|||
|
|
+- private final long enqueueTime;
|
|||
|
|
+ private final int magic;
|
|||
|
|
++
|
|||
|
|
++ private long enqueueTime;
|
|||
|
|
+ private MessageExt msg;
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+@@ -94,7 +95,9 @@ public class TimerRequest {
|
|||
|
|
+ public void setLatch(CountDownLatch latch) {
|
|||
|
|
+ this.latch = latch;
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
++ public void setEnqueueTime(long enqueueTime) {
|
|||
|
|
++ this.enqueueTime = enqueueTime;
|
|||
|
|
++ }
|
|||
|
|
+ public void idempotentRelease() {
|
|||
|
|
+ idempotentRelease(true);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/store/src/test/java/org/apache/rocketmq/store/timer/TimerMetricsTest.java b/store/src/test/java/org/apache/rocketmq/store/timer/TimerMetricsTest.java
|
|||
|
|
+index b7392cc45..3c7b9b67f 100644
|
|||
|
|
+--- a/store/src/test/java/org/apache/rocketmq/store/timer/TimerMetricsTest.java
|
|||
|
|
++++ b/store/src/test/java/org/apache/rocketmq/store/timer/TimerMetricsTest.java
|
|||
|
|
+@@ -16,6 +16,9 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.store.timer;
|
|||
|
|
+
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageAccessor;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
++import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.junit.Assert;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+@@ -31,8 +34,11 @@ public class TimerMetricsTest {
|
|||
|
|
+
|
|||
|
|
+ TimerMetrics first = new TimerMetrics(baseDir);
|
|||
|
|
+ Assert.assertTrue(first.load());
|
|||
|
|
+- first.addAndGet("AAA", 1000);
|
|||
|
|
+- first.addAndGet("BBB", 2000);
|
|||
|
|
++ MessageExt msg = new MessageExt();
|
|||
|
|
++ MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, "AAA");
|
|||
|
|
++ first.addAndGet(msg, 1000);
|
|||
|
|
++ MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, "BBB");
|
|||
|
|
++ first.addAndGet(msg, 2000);
|
|||
|
|
+ Assert.assertEquals(1000, first.getTimingCount("AAA"));
|
|||
|
|
+ Assert.assertEquals(2000, first.getTimingCount("BBB"));
|
|||
|
|
+ long curr = System.currentTimeMillis();
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From d7e5c4d1a4e048cd97f0b29a96a0fc575927a03e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Mon, 25 Sep 2023 13:37:36 +0800
|
|||
|
|
+Subject: [PATCH 06/12] [ISSUE #7389] Fix the problem that getLastMappedFile
|
|||
|
|
+ function affects performance
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: guyinyou <guyinyou.gyy@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/store/MappedFileQueue.java | 15 +++++++++++++--
|
|||
|
|
+ 1 file changed, 13 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+index 32b90d14f..9a0824829 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
|
|||
|
|
+@@ -377,8 +377,19 @@ public class MappedFileQueue implements Swappable {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MappedFile getLastMappedFile() {
|
|||
|
|
+- MappedFile[] mappedFiles = this.mappedFiles.toArray(new MappedFile[0]);
|
|||
|
|
+- return mappedFiles.length == 0 ? null : mappedFiles[mappedFiles.length - 1];
|
|||
|
|
++ MappedFile mappedFileLast = null;
|
|||
|
|
++ while (!this.mappedFiles.isEmpty()) {
|
|||
|
|
++ try {
|
|||
|
|
++ mappedFileLast = this.mappedFiles.get(this.mappedFiles.size() - 1);
|
|||
|
|
++ break;
|
|||
|
|
++ } catch (IndexOutOfBoundsException e) {
|
|||
|
|
++ //continue;
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ log.error("getLastMappedFile has exception.", e);
|
|||
|
|
++ break;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return mappedFileLast;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean resetOffset(long offset) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3fd43353fdf880deb5d63ba3ad50cc6e3259dc3a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Tue, 26 Sep 2023 13:53:51 +0800
|
|||
|
|
+Subject: [PATCH 07/12] [ISSUE #7393] Add timeout configuration for grpc server
|
|||
|
|
+ (#7394)
|
|||
|
|
+
|
|||
|
|
+* Add timeout configuration for grpc server
|
|||
|
|
+
|
|||
|
|
+* Add proxyConfig
|
|||
|
|
+---
|
|||
|
|
+ .../java/org/apache/rocketmq/proxy/ProxyStartup.java | 1 +
|
|||
|
|
+ .../apache/rocketmq/proxy/config/ProxyConfig.java | 9 +++++++++
|
|||
|
|
+ .../org/apache/rocketmq/proxy/grpc/GrpcServer.java | 10 ++++++++--
|
|||
|
|
+ .../rocketmq/proxy/grpc/GrpcServerBuilder.java | 12 +++++++++++-
|
|||
|
|
+ 4 files changed, 29 insertions(+), 3 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/ProxyStartup.java b/proxy/src/main/java/org/apache/rocketmq/proxy/ProxyStartup.java
|
|||
|
|
+index 06d5f4525..3b2ca99bf 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/ProxyStartup.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/ProxyStartup.java
|
|||
|
|
+@@ -85,6 +85,7 @@ public class ProxyStartup {
|
|||
|
|
+ .addService(ChannelzService.newInstance(100))
|
|||
|
|
+ .addService(ProtoReflectionService.newInstance())
|
|||
|
|
+ .configInterceptor(accessValidators)
|
|||
|
|
++ .shutdownTime(ConfigurationManager.getProxyConfig().getGrpcShutdownTimeSeconds(), TimeUnit.SECONDS)
|
|||
|
|
+ .build();
|
|||
|
|
+ PROXY_START_AND_SHUTDOWN.appendStartAndShutdown(grpcServer);
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+index b2478fec3..c0d00d864 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ProxyConfig.java
|
|||
|
|
+@@ -87,6 +87,7 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ */
|
|||
|
|
+ private String proxyMode = ProxyMode.CLUSTER.name();
|
|||
|
|
+ private Integer grpcServerPort = 8081;
|
|||
|
|
++ private long grpcShutdownTimeSeconds = 30;
|
|||
|
|
+ private int grpcBossLoopNum = 1;
|
|||
|
|
+ private int grpcWorkerLoopNum = PROCESSOR_NUMBER * 2;
|
|||
|
|
+ private boolean enableGrpcEpoll = false;
|
|||
|
|
+@@ -443,6 +444,14 @@ public class ProxyConfig implements ConfigFile {
|
|||
|
|
+ this.grpcServerPort = grpcServerPort;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public long getGrpcShutdownTimeSeconds() {
|
|||
|
|
++ return grpcShutdownTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setGrpcShutdownTimeSeconds(long grpcShutdownTimeSeconds) {
|
|||
|
|
++ this.grpcShutdownTimeSeconds = grpcShutdownTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public boolean isUseEndpointPortFromRequest() {
|
|||
|
|
+ return useEndpointPortFromRequest;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServer.java
|
|||
|
|
+index 1bffa3c0b..d5b896fe1 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServer.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServer.java
|
|||
|
|
+@@ -29,8 +29,14 @@ public class GrpcServer implements StartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ private final Server server;
|
|||
|
|
+
|
|||
|
|
+- protected GrpcServer(Server server) {
|
|||
|
|
++ private final long timeout;
|
|||
|
|
++
|
|||
|
|
++ private final TimeUnit unit;
|
|||
|
|
++
|
|||
|
|
++ protected GrpcServer(Server server, long timeout, TimeUnit unit) {
|
|||
|
|
+ this.server = server;
|
|||
|
|
++ this.timeout = timeout;
|
|||
|
|
++ this.unit = unit;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void start() throws Exception {
|
|||
|
|
+@@ -40,7 +46,7 @@ public class GrpcServer implements StartAndShutdown {
|
|||
|
|
+
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
+ try {
|
|||
|
|
+- this.server.shutdown().awaitTermination(30, TimeUnit.SECONDS);
|
|||
|
|
++ this.server.shutdown().awaitTermination(timeout, unit);
|
|||
|
|
+ log.info("grpc server shutdown successfully.");
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ e.printStackTrace();
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServerBuilder.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServerBuilder.java
|
|||
|
|
+index 9cddd3013..0e79006f6 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServerBuilder.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/GrpcServerBuilder.java
|
|||
|
|
+@@ -41,6 +41,10 @@ public class GrpcServerBuilder {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME);
|
|||
|
|
+ protected NettyServerBuilder serverBuilder;
|
|||
|
|
+
|
|||
|
|
++ protected long time = 30;
|
|||
|
|
++
|
|||
|
|
++ protected TimeUnit unit = TimeUnit.SECONDS;
|
|||
|
|
++
|
|||
|
|
+ public static GrpcServerBuilder newBuilder(ThreadPoolExecutor executor, int port) {
|
|||
|
|
+ return new GrpcServerBuilder(executor, port);
|
|||
|
|
+ }
|
|||
|
|
+@@ -77,6 +81,12 @@ public class GrpcServerBuilder {
|
|||
|
|
+ port, bossLoopNum, workerLoopNum, maxInboundMessageSize);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public GrpcServerBuilder shutdownTime(long time, TimeUnit unit) {
|
|||
|
|
++ this.time = time;
|
|||
|
|
++ this.unit = unit;
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public GrpcServerBuilder addService(BindableService service) {
|
|||
|
|
+ this.serverBuilder.addService(service);
|
|||
|
|
+ return this;
|
|||
|
|
+@@ -93,7 +103,7 @@ public class GrpcServerBuilder {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public GrpcServer build() {
|
|||
|
|
+- return new GrpcServer(this.serverBuilder.build());
|
|||
|
|
++ return new GrpcServer(this.serverBuilder.build(), time, unit);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public GrpcServerBuilder configInterceptor(List<AccessValidator> accessValidators) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From c3b86cd1e3c068bc5847671c899a284e49a2ecdc Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: =?UTF-8?q?=E7=9F=B3=E8=87=BB=E8=87=BB=28Steven=20shi=29?=
|
|||
|
|
+ <shirenchuang@users.noreply.github.com>
|
|||
|
|
+Date: Tue, 26 Sep 2023 16:07:13 +0800
|
|||
|
|
+Subject: [PATCH 08/12] [ISSUE #7398] Fix ExportConfigsCommand NPE (#7399)
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 石臻臻 <shirenchuang.src@cainiao.com>
|
|||
|
|
+---
|
|||
|
|
+ .../command/export/ExportConfigsCommand.java | 42 ++++++++++++-------
|
|||
|
|
+ 1 file changed, 26 insertions(+), 16 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
+index 03613b29c..c3f96d597 100644
|
|||
|
|
+--- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
++++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java
|
|||
|
|
+@@ -20,6 +20,7 @@ import java.util.HashMap;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+ import java.util.Map.Entry;
|
|||
|
|
++import java.util.Arrays;
|
|||
|
|
+ import java.util.Properties;
|
|||
|
|
+
|
|||
|
|
+ import com.alibaba.fastjson.JSON;
|
|||
|
|
+@@ -106,24 +107,33 @@ public class ExportConfigsCommand implements SubCommand {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++
|
|||
|
|
+ private Properties needBrokerProprties(Properties properties) {
|
|||
|
|
++ List<String> propertyKeys = Arrays.asList(
|
|||
|
|
++ "brokerClusterName",
|
|||
|
|
++ "brokerId",
|
|||
|
|
++ "brokerName",
|
|||
|
|
++ "brokerRole",
|
|||
|
|
++ "fileReservedTime",
|
|||
|
|
++ "filterServerNums",
|
|||
|
|
++ "flushDiskType",
|
|||
|
|
++ "maxMessageSize",
|
|||
|
|
++ "messageDelayLevel",
|
|||
|
|
++ "msgTraceTopicName",
|
|||
|
|
++ "slaveReadEnable",
|
|||
|
|
++ "traceOn",
|
|||
|
|
++ "traceTopicEnable",
|
|||
|
|
++ "useTLS",
|
|||
|
|
++ "autoCreateTopicEnable",
|
|||
|
|
++ "autoCreateSubscriptionGroup"
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
+ Properties newProperties = new Properties();
|
|||
|
|
+- newProperties.setProperty("brokerClusterName", properties.getProperty("brokerClusterName"));
|
|||
|
|
+- newProperties.setProperty("brokerId", properties.getProperty("brokerId"));
|
|||
|
|
+- newProperties.setProperty("brokerName", properties.getProperty("brokerName"));
|
|||
|
|
+- newProperties.setProperty("brokerRole", properties.getProperty("brokerRole"));
|
|||
|
|
+- newProperties.setProperty("fileReservedTime", properties.getProperty("fileReservedTime"));
|
|||
|
|
+- newProperties.setProperty("filterServerNums", properties.getProperty("filterServerNums"));
|
|||
|
|
+- newProperties.setProperty("flushDiskType", properties.getProperty("flushDiskType"));
|
|||
|
|
+- newProperties.setProperty("maxMessageSize", properties.getProperty("maxMessageSize"));
|
|||
|
|
+- newProperties.setProperty("messageDelayLevel", properties.getProperty("messageDelayLevel"));
|
|||
|
|
+- newProperties.setProperty("msgTraceTopicName", properties.getProperty("msgTraceTopicName"));
|
|||
|
|
+- newProperties.setProperty("slaveReadEnable", properties.getProperty("slaveReadEnable"));
|
|||
|
|
+- newProperties.setProperty("traceOn", properties.getProperty("traceOn"));
|
|||
|
|
+- newProperties.setProperty("traceTopicEnable", properties.getProperty("traceTopicEnable"));
|
|||
|
|
+- newProperties.setProperty("useTLS", properties.getProperty("useTLS"));
|
|||
|
|
+- newProperties.setProperty("autoCreateTopicEnable", properties.getProperty("autoCreateTopicEnable"));
|
|||
|
|
+- newProperties.setProperty("autoCreateSubscriptionGroup", properties.getProperty("autoCreateSubscriptionGroup"));
|
|||
|
|
++ propertyKeys.stream()
|
|||
|
|
++ .filter(key -> properties.getProperty(key) != null)
|
|||
|
|
++ .forEach(key -> newProperties.setProperty(key, properties.getProperty(key)));
|
|||
|
|
++
|
|||
|
|
+ return newProperties;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 959a98120cea8022498557a308aff35e3d8def2a Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: =?UTF-8?q?=E7=9F=B3=E8=87=BB=E8=87=BB=28Steven=20shi=29?=
|
|||
|
|
+ <shirenchuang@users.noreply.github.com>
|
|||
|
|
+Date: Wed, 27 Sep 2023 01:59:58 +0800
|
|||
|
|
+Subject: [PATCH 09/12] [ISSUE #7400] Fix getBrokerEpochSubCommand NPE
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 石臻臻 <shirenchuang.src@cainiao.com>
|
|||
|
|
+---
|
|||
|
|
+ .../broker/processor/AdminBrokerProcessor.java | 10 ++++++++--
|
|||
|
|
+ 1 file changed, 8 insertions(+), 2 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+index 9e48431be..e77120e15 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java
|
|||
|
|
+@@ -2736,10 +2736,16 @@ public class AdminBrokerProcessor implements NettyRequestProcessor {
|
|||
|
|
+ final ReplicasManager replicasManager = this.brokerController.getReplicasManager();
|
|||
|
|
+ assert replicasManager != null;
|
|||
|
|
+ final BrokerConfig brokerConfig = this.brokerController.getBrokerConfig();
|
|||
|
|
++ final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++
|
|||
|
|
++ if (!brokerConfig.isEnableControllerMode()) {
|
|||
|
|
++ response.setCode(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
++ response.setRemark("this request only for controllerMode ");
|
|||
|
|
++ return response;
|
|||
|
|
++ }
|
|||
|
|
+ final EpochEntryCache entryCache = new EpochEntryCache(brokerConfig.getBrokerClusterName(),
|
|||
|
|
+- brokerConfig.getBrokerName(), brokerConfig.getBrokerId(), replicasManager.getEpochEntries(), this.brokerController.getMessageStore().getMaxPhyOffset());
|
|||
|
|
++ brokerConfig.getBrokerName(), brokerConfig.getBrokerId(), replicasManager.getEpochEntries(), this.brokerController.getMessageStore().getMaxPhyOffset());
|
|||
|
|
+
|
|||
|
|
+- final RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+ response.setBody(entryCache.encode());
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ response.setRemark(null);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 0a6ae4605fef4eaab6262fbd370aba887d8ae58b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: tiger lee <1026203200@qq.com>
|
|||
|
|
+Date: Wed, 27 Sep 2023 14:43:15 +0800
|
|||
|
|
+Subject: [PATCH 10/12] [ISSUE #7396] Fix wrong word in
|
|||
|
|
+ BrokerController#doResterBrokerAll (#7397)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../main/java/org/apache/rocketmq/broker/BrokerController.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+index 53e2e1b62..d4bded600 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java
|
|||
|
|
+@@ -1807,7 +1807,7 @@ public class BrokerController {
|
|||
|
|
+ TopicConfigSerializeWrapper topicConfigWrapper) {
|
|||
|
|
+
|
|||
|
|
+ if (shutdown) {
|
|||
|
|
+- BrokerController.LOG.info("BrokerController#doResterBrokerAll: broker has shutdown, no need to register any more.");
|
|||
|
|
++ BrokerController.LOG.info("BrokerController#doRegisterBrokerAll: broker has shutdown, no need to register any more.");
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ List<RegisterBrokerResult> registerBrokerResultList = this.brokerOuterAPI.registerBrokerAll(
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 4f1b42a7c5557bcadd6b9982a0c9bd876622c69e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: ShuangxiDing <dingshuangxi888@gmail.com>
|
|||
|
|
+Date: Thu, 28 Sep 2023 16:52:02 +0800
|
|||
|
|
+Subject: [PATCH 11/12] [ISSUE #7410] Handle the Exception when the Proxy
|
|||
|
|
+ requests the client
|
|||
|
|
+MIME-Version: 1.0
|
|||
|
|
+Content-Type: text/plain; charset=UTF-8
|
|||
|
|
+Content-Transfer-Encoding: 8bit
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: 徒钟 <shuangxi.dsx@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../remoting/channel/RemotingChannel.java | 23 ++++++++++++++-----
|
|||
|
|
+ 1 file changed, 17 insertions(+), 6 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java
|
|||
|
|
+index 40946cabf..d755fdcc4 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/channel/RemotingChannel.java
|
|||
|
|
+@@ -34,6 +34,7 @@ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.channel.ChannelHelper;
|
|||
|
|
++import org.apache.rocketmq.proxy.common.utils.ExceptionUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.common.utils.FutureUtils;
|
|||
|
|
+ import org.apache.rocketmq.proxy.config.ConfigurationManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.processor.channel.ChannelExtendAttributeGetter;
|
|||
|
|
+@@ -158,10 +159,15 @@ public class RemotingChannel extends ProxyChannel implements RemoteChannelConver
|
|||
|
|
+ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+ ConsumerRunningInfo consumerRunningInfo = ConsumerRunningInfo.decode(response.getBody(), ConsumerRunningInfo.class);
|
|||
|
|
+ responseFuture.complete(new ProxyRelayResult<>(ResponseCode.SUCCESS, "", consumerRunningInfo));
|
|||
|
|
++ } else {
|
|||
|
|
++ String errMsg = String.format("get consumer running info failed, code:%s remark:%s", response.getCode(), response.getRemark());
|
|||
|
|
++ RuntimeException e = new RuntimeException(errMsg);
|
|||
|
|
++ responseFuture.completeExceptionally(e);
|
|||
|
|
+ }
|
|||
|
|
+- String errMsg = String.format("get consumer running info failed, code:%s remark:%s", response.getCode(), response.getRemark());
|
|||
|
|
+- RuntimeException e = new RuntimeException(errMsg);
|
|||
|
|
+- responseFuture.completeExceptionally(e);
|
|||
|
|
++ })
|
|||
|
|
++ .exceptionally(t -> {
|
|||
|
|
++ responseFuture.completeExceptionally(ExceptionUtils.getRealException(t));
|
|||
|
|
++ return null;
|
|||
|
|
+ });
|
|||
|
|
+ return CompletableFuture.completedFuture(null);
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+@@ -183,10 +189,15 @@ public class RemotingChannel extends ProxyChannel implements RemoteChannelConver
|
|||
|
|
+ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+ ConsumeMessageDirectlyResult result = ConsumeMessageDirectlyResult.decode(response.getBody(), ConsumeMessageDirectlyResult.class);
|
|||
|
|
+ responseFuture.complete(new ProxyRelayResult<>(ResponseCode.SUCCESS, "", result));
|
|||
|
|
++ } else {
|
|||
|
|
++ String errMsg = String.format("consume message directly failed, code:%s remark:%s", response.getCode(), response.getRemark());
|
|||
|
|
++ RuntimeException e = new RuntimeException(errMsg);
|
|||
|
|
++ responseFuture.completeExceptionally(e);
|
|||
|
|
+ }
|
|||
|
|
+- String errMsg = String.format("consume message directly failed, code:%s remark:%s", response.getCode(), response.getRemark());
|
|||
|
|
+- RuntimeException e = new RuntimeException(errMsg);
|
|||
|
|
+- responseFuture.completeExceptionally(e);
|
|||
|
|
++ })
|
|||
|
|
++ .exceptionally(t -> {
|
|||
|
|
++ responseFuture.completeExceptionally(ExceptionUtils.getRealException(t));
|
|||
|
|
++ return null;
|
|||
|
|
+ });
|
|||
|
|
+ return CompletableFuture.completedFuture(null);
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From c36bb78e850129b9db40adc5b0e1b9bfd5c8fd2e Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: shriVATSA54 <116296557+shriVATSA54@users.noreply.github.com>
|
|||
|
|
+Date: Sat, 7 Oct 2023 12:22:39 +0530
|
|||
|
|
+Subject: [PATCH 12/12] [ISSUE 7313] Enhancement Optimization Method name
|
|||
|
|
+ (#7420)
|
|||
|
|
+
|
|||
|
|
+* Enhancment/method_name/#7313/
|
|||
|
|
+
|
|||
|
|
+* Enhancment/method_name/#7313/
|
|||
|
|
+
|
|||
|
|
+* Enhancment/method_name/#7313/
|
|||
|
|
+---
|
|||
|
|
+ .../tieredstore/provider/TieredStoreTopicBlackListFilter.java | 2 +-
|
|||
|
|
+ .../rocketmq/tieredstore/provider/TieredStoreTopicFilter.java | 2 +-
|
|||
|
|
+ .../provider/TieredStoreTopicBlackListFilterTest.java | 2 +-
|
|||
|
|
+ 3 files changed, 3 insertions(+), 3 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
+index 50adbb713..f8bf165bc 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilter.java
|
|||
|
|
+@@ -39,7 +39,7 @@ public class TieredStoreTopicBlackListFilter implements TieredStoreTopicFilter {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public void addTopicToWhiteList(String topicName) {
|
|||
|
|
++ public void addTopicToBlackList(String topicName) {
|
|||
|
|
+ this.topicBlackSet.add(topicName);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
+index 3f26b8b02..f983ed6e9 100644
|
|||
|
|
+--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
++++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicFilter.java
|
|||
|
|
+@@ -21,5 +21,5 @@ public interface TieredStoreTopicFilter {
|
|||
|
|
+
|
|||
|
|
+ boolean filterTopic(String topicName);
|
|||
|
|
+
|
|||
|
|
+- void addTopicToWhiteList(String topicName);
|
|||
|
|
++ void addTopicToBlackList(String topicName);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
+index 2bf48173c..fbaafa1b4 100644
|
|||
|
|
+--- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
++++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/provider/TieredStoreTopicBlackListFilterTest.java
|
|||
|
|
+@@ -30,7 +30,7 @@ public class TieredStoreTopicBlackListFilterTest {
|
|||
|
|
+
|
|||
|
|
+ String topicName = "WhiteTopic";
|
|||
|
|
+ Assert.assertFalse(topicFilter.filterTopic(topicName));
|
|||
|
|
+- topicFilter.addTopicToWhiteList(topicName);
|
|||
|
|
++ topicFilter.addTopicToBlackList(topicName);
|
|||
|
|
+ Assert.assertTrue(topicFilter.filterTopic(topicName));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+\ No newline at end of file
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch020-backport-add-goaway-mechanism.patch b/patch020-backport-add-goaway-mechanism.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..122db0325
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch020-backport-add-goaway-mechanism.patch
|
|||
|
|
@@ -0,0 +1,3696 @@
|
|||
|
|
+From 84156084a4c5228e1d2fe21e068fff330bbc40d1 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Sun, 8 Oct 2023 11:13:25 +0800
|
|||
|
|
+Subject: [PATCH 1/7] [ISSUE #7321] Refector NettyRemotingAbstract with unify
|
|||
|
|
+ future implementation (#7322)
|
|||
|
|
+
|
|||
|
|
+* Refector NettyRemotingAbstract
|
|||
|
|
+
|
|||
|
|
+* Add invoke with future method
|
|||
|
|
+
|
|||
|
|
+* Deprecate InvokeCallback#operationComplete
|
|||
|
|
+
|
|||
|
|
+* Add operationSuccess and operationException for InvokeCallback
|
|||
|
|
+
|
|||
|
|
+* fix unit test
|
|||
|
|
+
|
|||
|
|
+* fix unit test
|
|||
|
|
+
|
|||
|
|
+* Keep InvokeCallback#operationComplete
|
|||
|
|
+
|
|||
|
|
+* Optimize invokeAsyncImpl operationComplete
|
|||
|
|
+
|
|||
|
|
+* Add unit test for NettyRemotingClient
|
|||
|
|
+
|
|||
|
|
+* fix checkstyle
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/broker/out/BrokerOuterAPI.java | 147 +++++----
|
|||
|
|
+ .../rocketmq/client/impl/MQAdminImpl.java | 71 ++--
|
|||
|
|
+ .../rocketmq/client/impl/MQClientAPIImpl.java | 239 +++++++-------
|
|||
|
|
+ .../client/impl/mqclient/MQClientAPIExt.java | 309 ++++++++----------
|
|||
|
|
+ .../client/impl/MQClientAPIImplTest.java | 12 +-
|
|||
|
|
+ .../remoting/RemotingProtocolServer.java | 22 +-
|
|||
|
|
+ .../service/mqclient/MQClientAPIExtTest.java | 97 +++---
|
|||
|
|
+ .../rocketmq/remoting/InvokeCallback.java | 15 +
|
|||
|
|
+ .../rocketmq/remoting/RemotingClient.java | 27 +-
|
|||
|
|
+ .../remoting/netty/NettyRemotingAbstract.java | 123 ++++---
|
|||
|
|
+ .../remoting/netty/NettyRemotingClient.java | 33 +-
|
|||
|
|
+ .../remoting/netty/ResponseFuture.java | 15 +
|
|||
|
|
+ .../rocketmq/remoting/rpc/RpcClientImpl.java | 29 +-
|
|||
|
|
+ .../rocketmq/remoting/RemotingServerTest.java | 22 +-
|
|||
|
|
+ .../rocketmq/remoting/netty/MockChannel.java | 21 +-
|
|||
|
|
+ .../remoting/netty/MockChannelPromise.java | 191 +++++++++++
|
|||
|
|
+ .../netty/NettyRemotingAbstractTest.java | 54 ++-
|
|||
|
|
+ .../netty/NettyRemotingClientTest.java | 185 ++++++++++-
|
|||
|
|
+ 18 files changed, 1029 insertions(+), 583 deletions(-)
|
|||
|
|
+ rename client/src/main/java/org/apache/rocketmq/client/impl/BaseInvokeCallback.java => remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannel.java (57%)
|
|||
|
|
+ create mode 100644 remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannelPromise.java
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
+index 9dfb8127d..6fde48dd9 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java
|
|||
|
|
+@@ -73,6 +73,7 @@ import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyRemotingClient;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.BrokerSyncInfo;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.DataVersion;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+@@ -107,6 +108,8 @@ import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeaderV2;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.SendMessageResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.AlterSyncStateSetRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterRequestHeader;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.GetReplicaInfoRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.controller.GetReplicaInfoResponseHeader;
|
|||
|
|
+@@ -124,8 +127,6 @@ import org.apache.rocketmq.remoting.protocol.header.namesrv.RegisterBrokerReques
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.RegisterBrokerResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.RegisterTopicRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.namesrv.UnRegisterBrokerRequestHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterRequestHeader;
|
|||
|
|
+-import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.namesrv.RegisterBrokerResult;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.BrokerData;
|
|||
|
|
+@@ -151,7 +152,6 @@ public class BrokerOuterAPI {
|
|||
|
|
+ private final RpcClient rpcClient;
|
|||
|
|
+ private String nameSrvAddr = null;
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ public BrokerOuterAPI(final NettyClientConfig nettyClientConfig) {
|
|||
|
|
+ this(nettyClientConfig, new DynamicalExtFieldRPCHook(), new ClientMetadata());
|
|||
|
|
+ }
|
|||
|
|
+@@ -459,7 +459,7 @@ public class BrokerOuterAPI {
|
|||
|
|
+ * @param filterServerList
|
|||
|
|
+ * @param oneway
|
|||
|
|
+ * @param timeoutMills
|
|||
|
|
+- * @param compressed default false
|
|||
|
|
++ * @param compressed default false
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ public List<RegisterBrokerResult> registerBrokerAll(
|
|||
|
|
+@@ -643,7 +643,6 @@ public class BrokerOuterAPI {
|
|||
|
|
+ queueDatas.add(queueData);
|
|||
|
|
+ final byte[] topicRouteBody = topicRouteData.encode();
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ List<String> nameServerAddressList = this.remotingClient.getNameServerAddressList();
|
|||
|
|
+ final CountDownLatch countDownLatch = new CountDownLatch(nameServerAddressList.size());
|
|||
|
|
+ for (final String namesrvAddr : nameServerAddressList) {
|
|||
|
|
+@@ -910,25 +909,33 @@ public class BrokerOuterAPI {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, null);
|
|||
|
|
+
|
|||
|
|
+ request.setBody(requestBody.encode());
|
|||
|
|
+- this.remotingClient.invokeAsync(addr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- if (callback == null) {
|
|||
|
|
+- return;
|
|||
|
|
++ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- try {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+- LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(),
|
|||
|
|
+- LockBatchResponseBody.class);
|
|||
|
|
+- Set<MessageQueue> messageQueues = responseBody.getLockOKMQSet();
|
|||
|
|
+- callback.onSuccess(messageQueues);
|
|||
|
|
+- } else {
|
|||
|
|
+- callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- }
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ if (callback == null) {
|
|||
|
|
++ return;
|
|||
|
|
+ }
|
|||
|
|
+- } catch (Throwable ignored) {
|
|||
|
|
++ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
++ LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(),
|
|||
|
|
++ LockBatchResponseBody.class);
|
|||
|
|
++ Set<MessageQueue> messageQueues = responseBody.getLockOKMQSet();
|
|||
|
|
++ callback.onSuccess(messageQueues);
|
|||
|
|
++ } else {
|
|||
|
|
++ callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ if (callback == null) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ callback.onException(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+@@ -942,22 +949,30 @@ public class BrokerOuterAPI {
|
|||
|
|
+
|
|||
|
|
+ request.setBody(requestBody.encode());
|
|||
|
|
+
|
|||
|
|
+- this.remotingClient.invokeAsync(addr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- if (callback == null) {
|
|||
|
|
+- return;
|
|||
|
|
++ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- try {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+- callback.onSuccess();
|
|||
|
|
+- } else {
|
|||
|
|
+- callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- }
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ if (callback == null) {
|
|||
|
|
++ return;
|
|||
|
|
+ }
|
|||
|
|
+- } catch (Throwable ignored) {
|
|||
|
|
++ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
++ callback.onSuccess();
|
|||
|
|
++ } else {
|
|||
|
|
++ callback.onException(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ if (callback == null) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ callback.onException(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+@@ -983,21 +998,27 @@ public class BrokerOuterAPI {
|
|||
|
|
+ CompletableFuture<SendResult> cf = new CompletableFuture<>();
|
|||
|
|
+ final String msgId = msg.getMsgId();
|
|||
|
|
+ try {
|
|||
|
|
+- this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (null != response) {
|
|||
|
|
+- SendResult sendResult = null;
|
|||
|
|
++ this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ try {
|
|||
|
|
+- sendResult = this.processSendResponse(brokerName, msg, response);
|
|||
|
|
++ SendResult sendResult = processSendResponse(brokerName, msg, response);
|
|||
|
|
+ cf.complete(sendResult);
|
|||
|
|
+ } catch (MQBrokerException | RemotingCommandException e) {
|
|||
|
|
+ LOGGER.error("processSendResponse in sendMessageToSpecificBrokerAsync failed, msgId=" + msgId, e);
|
|||
|
|
+ cf.completeExceptionally(e);
|
|||
|
|
+ }
|
|||
|
|
+- } else {
|
|||
|
|
+- cf.complete(null);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ cf.completeExceptionally(throwable);
|
|||
|
|
++ }
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ LOGGER.error("invokeAsync failed in sendMessageToSpecificBrokerAsync, msgId=" + msgId, t);
|
|||
|
|
+@@ -1057,7 +1078,7 @@ public class BrokerOuterAPI {
|
|||
|
|
+ }
|
|||
|
|
+ if (sendStatus != null) {
|
|||
|
|
+ SendMessageResponseHeader responseHeader =
|
|||
|
|
+- (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class);
|
|||
|
|
++ (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class);
|
|||
|
|
+
|
|||
|
|
+ //If namespace not null , reset Topic without namespace.
|
|||
|
|
+ String topic = msg.getTopic();
|
|||
|
|
+@@ -1073,8 +1094,8 @@ public class BrokerOuterAPI {
|
|||
|
|
+ uniqMsgId = sb.toString();
|
|||
|
|
+ }
|
|||
|
|
+ SendResult sendResult = new SendResult(sendStatus,
|
|||
|
|
+- uniqMsgId,
|
|||
|
|
+- responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset());
|
|||
|
|
++ uniqMsgId,
|
|||
|
|
++ responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset());
|
|||
|
|
+ sendResult.setTransactionId(responseHeader.getTransactionId());
|
|||
|
|
+ String regionId = response.getExtFields().get(MessageConst.PROPERTY_MSG_REGION);
|
|||
|
|
+ String traceOn = response.getExtFields().get(MessageConst.PROPERTY_TRACE_SWITCH);
|
|||
|
|
+@@ -1218,8 +1239,9 @@ public class BrokerOuterAPI {
|
|||
|
|
+ /**
|
|||
|
|
+ * Broker try to elect itself as a master in broker set
|
|||
|
|
+ */
|
|||
|
|
+- public Pair<ElectMasterResponseHeader, Set<Long>> brokerElect(String controllerAddress, String clusterName, String brokerName,
|
|||
|
|
+- Long brokerId) throws Exception {
|
|||
|
|
++ public Pair<ElectMasterResponseHeader, Set<Long>> brokerElect(String controllerAddress, String clusterName,
|
|||
|
|
++ String brokerName,
|
|||
|
|
++ Long brokerId) throws Exception {
|
|||
|
|
+
|
|||
|
|
+ final ElectMasterRequestHeader requestHeader = ElectMasterRequestHeader.ofBrokerTrigger(clusterName, brokerName, brokerId);
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_ELECT_MASTER, requestHeader);
|
|||
|
|
+@@ -1237,7 +1259,8 @@ public class BrokerOuterAPI {
|
|||
|
|
+ throw new MQBrokerException(response.getCode(), response.getRemark());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public GetNextBrokerIdResponseHeader getNextBrokerId(final String clusterName, final String brokerName, final String controllerAddress) throws Exception {
|
|||
|
|
++ public GetNextBrokerIdResponseHeader getNextBrokerId(final String clusterName, final String brokerName,
|
|||
|
|
++ final String controllerAddress) throws Exception {
|
|||
|
|
+ final GetNextBrokerIdRequestHeader requestHeader = new GetNextBrokerIdRequestHeader(clusterName, brokerName);
|
|||
|
|
+ final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_GET_NEXT_BROKER_ID, requestHeader);
|
|||
|
|
+ final RemotingCommand response = this.remotingClient.invokeSync(controllerAddress, request, 3000);
|
|||
|
|
+@@ -1248,7 +1271,8 @@ public class BrokerOuterAPI {
|
|||
|
|
+ throw new MQBrokerException(response.getCode(), response.getRemark());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public ApplyBrokerIdResponseHeader applyBrokerId(final String clusterName, final String brokerName, final Long brokerId, final String registerCheckCode, final String controllerAddress) throws Exception {
|
|||
|
|
++ public ApplyBrokerIdResponseHeader applyBrokerId(final String clusterName, final String brokerName,
|
|||
|
|
++ final Long brokerId, final String registerCheckCode, final String controllerAddress) throws Exception {
|
|||
|
|
+ final ApplyBrokerIdRequestHeader requestHeader = new ApplyBrokerIdRequestHeader(clusterName, brokerName, brokerId, registerCheckCode);
|
|||
|
|
+ final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_APPLY_BROKER_ID, requestHeader);
|
|||
|
|
+ final RemotingCommand response = this.remotingClient.invokeSync(controllerAddress, request, 3000);
|
|||
|
|
+@@ -1259,7 +1283,9 @@ public class BrokerOuterAPI {
|
|||
|
|
+ throw new MQBrokerException(response.getCode(), response.getRemark());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public Pair<RegisterBrokerToControllerResponseHeader, Set<Long>> registerBrokerToController(final String clusterName, final String brokerName, final Long brokerId, final String brokerAddress, final String controllerAddress) throws Exception {
|
|||
|
|
++ public Pair<RegisterBrokerToControllerResponseHeader, Set<Long>> registerBrokerToController(
|
|||
|
|
++ final String clusterName, final String brokerName, final Long brokerId, final String brokerAddress,
|
|||
|
|
++ final String controllerAddress) throws Exception {
|
|||
|
|
+ final RegisterBrokerToControllerRequestHeader requestHeader = new RegisterBrokerToControllerRequestHeader(clusterName, brokerName, brokerId, brokerAddress);
|
|||
|
|
+ final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_REGISTER_BROKER, requestHeader);
|
|||
|
|
+ final RemotingCommand response = this.remotingClient.invokeSync(controllerAddress, request, 3000);
|
|||
|
|
+@@ -1355,16 +1381,25 @@ public class BrokerOuterAPI {
|
|||
|
|
+
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, requestHeader);
|
|||
|
|
+ CompletableFuture<PullResult> pullResultFuture = new CompletableFuture<>();
|
|||
|
|
+- this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- if (responseFuture.getCause() != null) {
|
|||
|
|
+- pullResultFuture.complete(new PullResult(PullStatus.NO_MATCHED_MSG, -1, -1, -1, new ArrayList<>()));
|
|||
|
|
+- return;
|
|||
|
|
++ this.remotingClient.invokeAsync(brokerAddr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+- try {
|
|||
|
|
+- PullResultExt pullResultExt = this.processPullResponse(responseFuture.getResponseCommand(), brokerAddr);
|
|||
|
|
+- this.processPullResult(pullResultExt, brokerName, queueId);
|
|||
|
|
+- pullResultFuture.complete(pullResultExt);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ try {
|
|||
|
|
++ PullResultExt pullResultExt = processPullResponse(response, brokerAddr);
|
|||
|
|
++ processPullResult(pullResultExt, brokerName, queueId);
|
|||
|
|
++ pullResultFuture.complete(pullResultExt);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ pullResultFuture.complete(new PullResult(PullStatus.NO_MATCHED_MSG, -1, -1, -1, new ArrayList<>()));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
+ pullResultFuture.complete(new PullResult(PullStatus.NO_MATCHED_MSG, -1, -1, -1, new ArrayList<>()));
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java
|
|||
|
|
+index 1ef3a9483..83835bd3d 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java
|
|||
|
|
+@@ -44,6 +44,8 @@ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageId;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.common.utils.NetworkUtil;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
++import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingCommandException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+@@ -55,8 +57,6 @@ import org.apache.rocketmq.remoting.protocol.header.QueryMessageRequestHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.header.QueryMessageResponseHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.BrokerData;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.route.TopicRouteData;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+-import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+
|
|||
|
|
+ public class MQAdminImpl {
|
|||
|
|
+
|
|||
|
|
+@@ -357,44 +357,51 @@ public class MQAdminImpl {
|
|||
|
|
+ new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ try {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- switch (response.getCode()) {
|
|||
|
|
+- case ResponseCode.SUCCESS: {
|
|||
|
|
+- QueryMessageResponseHeader responseHeader = null;
|
|||
|
|
+- try {
|
|||
|
|
+- responseHeader =
|
|||
|
|
+- (QueryMessageResponseHeader) response
|
|||
|
|
+- .decodeCommandCustomHeader(QueryMessageResponseHeader.class);
|
|||
|
|
+- } catch (RemotingCommandException e) {
|
|||
|
|
+- log.error("decodeCommandCustomHeader exception", e);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- List<MessageExt> wrappers =
|
|||
|
|
+- MessageDecoder.decodes(ByteBuffer.wrap(response.getBody()), true);
|
|||
|
|
+-
|
|||
|
|
+- QueryResult qr = new QueryResult(responseHeader.getIndexLastUpdateTimestamp(), wrappers);
|
|||
|
|
+- try {
|
|||
|
|
+- lock.writeLock().lock();
|
|||
|
|
+- queryResultList.add(qr);
|
|||
|
|
+- } finally {
|
|||
|
|
+- lock.writeLock().unlock();
|
|||
|
|
+- }
|
|||
|
|
+- break;
|
|||
|
|
++ switch (response.getCode()) {
|
|||
|
|
++ case ResponseCode.SUCCESS: {
|
|||
|
|
++ QueryMessageResponseHeader responseHeader = null;
|
|||
|
|
++ try {
|
|||
|
|
++ responseHeader =
|
|||
|
|
++ (QueryMessageResponseHeader) response
|
|||
|
|
++ .decodeCommandCustomHeader(QueryMessageResponseHeader.class);
|
|||
|
|
++ } catch (RemotingCommandException e) {
|
|||
|
|
++ log.error("decodeCommandCustomHeader exception", e);
|
|||
|
|
++ return;
|
|||
|
|
+ }
|
|||
|
|
+- default:
|
|||
|
|
+- log.warn("getResponseCommand failed, {} {}", response.getCode(), response.getRemark());
|
|||
|
|
+- break;
|
|||
|
|
++
|
|||
|
|
++ List<MessageExt> wrappers =
|
|||
|
|
++ MessageDecoder.decodes(ByteBuffer.wrap(response.getBody()), true);
|
|||
|
|
++
|
|||
|
|
++ QueryResult qr = new QueryResult(responseHeader.getIndexLastUpdateTimestamp(), wrappers);
|
|||
|
|
++ try {
|
|||
|
|
++ lock.writeLock().lock();
|
|||
|
|
++ queryResultList.add(qr);
|
|||
|
|
++ } finally {
|
|||
|
|
++ lock.writeLock().unlock();
|
|||
|
|
++ }
|
|||
|
|
++ break;
|
|||
|
|
+ }
|
|||
|
|
+- } else {
|
|||
|
|
+- log.warn("getResponseCommand return null");
|
|||
|
|
++ default:
|
|||
|
|
++ log.warn("getResponseCommand failed, {} {}", response.getCode(), response.getRemark());
|
|||
|
|
++ break;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ } finally {
|
|||
|
|
+ countDownLatch.countDown();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ log.error("queryMessage error, requestHeader={}", requestHeader);
|
|||
|
|
++ countDownLatch.countDown();
|
|||
|
|
++ }
|
|||
|
|
+ }, isUniqKey);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.warn("queryMessage exception", e);
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+index 3201a493f..2407e5737 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.client.ClientConfig;
|
|||
|
|
+ import org.apache.rocketmq.client.Validators;
|
|||
|
|
+-import org.apache.rocketmq.client.common.ClientErrorCode;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckCallback;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.AckStatus;
|
|||
|
|
+@@ -653,10 +652,13 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- long cost = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (null == sendCallback && response != null) {
|
|||
|
|
+
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ long cost = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
++ if (null == sendCallback) {
|
|||
|
|
+ try {
|
|||
|
|
+ SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr);
|
|||
|
|
+ if (context != null && sendResult != null) {
|
|||
|
|
+@@ -666,46 +668,47 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false, true);
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, false, true);
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- if (response != null) {
|
|||
|
|
++ try {
|
|||
|
|
++ SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr);
|
|||
|
|
++ assert sendResult != null;
|
|||
|
|
++ if (context != null) {
|
|||
|
|
++ context.setSendResult(sendResult);
|
|||
|
|
++ context.getProducer().executeSendMessageHookAfter(context);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ try {
|
|||
|
|
+- SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr);
|
|||
|
|
+- assert sendResult != null;
|
|||
|
|
+- if (context != null) {
|
|||
|
|
+- context.setSendResult(sendResult);
|
|||
|
|
+- context.getProducer().executeSendMessageHookAfter(context);
|
|||
|
|
+- }
|
|||
|
|
++ sendCallback.onSuccess(sendResult);
|
|||
|
|
++ } catch (Throwable e) {
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+- try {
|
|||
|
|
+- sendCallback.onSuccess(sendResult);
|
|||
|
|
+- } catch (Throwable e) {
|
|||
|
|
+- }
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, false, true);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, true, true);
|
|||
|
|
++ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
++ retryTimesWhenSendFailed, times, e, context, false, producer);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), false, true);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true, true);
|
|||
|
|
+- onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+- retryTimesWhenSendFailed, times, e, context, false, producer);
|
|||
|
|
+- }
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, true, true);
|
|||
|
|
++ long cost = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
++ if (throwable instanceof RemotingSendRequestException) {
|
|||
|
|
++ MQClientException ex = new MQClientException("send request failed", throwable);
|
|||
|
|
++ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
++ retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
++ } else if (throwable instanceof RemotingTimeoutException) {
|
|||
|
|
++ MQClientException ex = new MQClientException("wait response timeout, cost=" + cost, throwable);
|
|||
|
|
++ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
++ retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+ } else {
|
|||
|
|
+- producer.updateFaultItem(brokerName, System.currentTimeMillis() - responseFuture.getBeginTimestamp(), true, true);
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- MQClientException ex = new MQClientException("send request failed", responseFuture.getCause());
|
|||
|
|
+- onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+- retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- MQClientException ex = new MQClientException("wait response timeout " + responseFuture.getTimeoutMillis() + "ms",
|
|||
|
|
+- responseFuture.getCause());
|
|||
|
|
+- onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+- retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+- } else {
|
|||
|
|
+- MQClientException ex = new MQClientException("unknow reseaon", responseFuture.getCause());
|
|||
|
|
+- onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
+- retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+- }
|
|||
|
|
++ MQClientException ex = new MQClientException("unknow reseaon", throwable);
|
|||
|
|
++ onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance,
|
|||
|
|
++ retryTimesWhenSendFailed, times, ex, context, true, producer);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+@@ -857,30 +860,25 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ final long timeoutMillis, final PopCallback popCallback
|
|||
|
|
+ ) throws RemotingException, InterruptedException {
|
|||
|
|
+ final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.POP_MESSAGE, requestHeader);
|
|||
|
|
+- this.remotingClient.invokeAsync(addr, request, timeoutMillis, new BaseInvokeCallback(MQClientAPIImpl.this) {
|
|||
|
|
++ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void onComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- PopResult
|
|||
|
|
+- popResult = MQClientAPIImpl.this.processPopResponse(brokerName, response, requestHeader.getTopic(), requestHeader);
|
|||
|
|
+- assert popResult != null;
|
|||
|
|
+- popCallback.onSuccess(popResult);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- popCallback.onException(e);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- popCallback.onException(new MQClientException(ClientErrorCode.CONNECT_BROKER_EXCEPTION, "send request failed to " + addr + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- popCallback.onException(new MQClientException(ClientErrorCode.ACCESS_BROKER_TIMEOUT, "wait response from " + addr + " timeout :" + responseFuture.getTimeoutMillis() + "ms" + ". Request: " + request,
|
|||
|
|
+- responseFuture.getCause()));
|
|||
|
|
+- } else {
|
|||
|
|
+- popCallback.onException(new MQClientException("unknown reason. addr: " + addr + ", timeoutMillis: " + timeoutMillis + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- }
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ try {
|
|||
|
|
++ PopResult popResult = MQClientAPIImpl.this.processPopResponse(brokerName, response, requestHeader.getTopic(), requestHeader);
|
|||
|
|
++ popCallback.onSuccess(popResult);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ popCallback.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ popCallback.onException(throwable);
|
|||
|
|
++ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -959,34 +957,26 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ request.setBody(requestBody.encode());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+- this.remotingClient.invokeAsync(addr, request, timeOut, new BaseInvokeCallback(MQClientAPIImpl.this) {
|
|||
|
|
++ this.remotingClient.invokeAsync(addr, request, timeOut, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+- public void onComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- AckResult ackResult = new AckResult();
|
|||
|
|
+- if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
+- ackResult.setStatus(AckStatus.OK);
|
|||
|
|
+- } else {
|
|||
|
|
+- ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+- }
|
|||
|
|
+- ackCallback.onSuccess(ackResult);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- ackCallback.onException(e);
|
|||
|
|
+- }
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
+ } else {
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- ackCallback.onException(new MQClientException(ClientErrorCode.CONNECT_BROKER_EXCEPTION, "send request failed to " + addr + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- ackCallback.onException(new MQClientException(ClientErrorCode.ACCESS_BROKER_TIMEOUT, "wait response from " + addr + " timeout :" + responseFuture.getTimeoutMillis() + "ms" + ". Request: " + request,
|
|||
|
|
+- responseFuture.getCause()));
|
|||
|
|
+- } else {
|
|||
|
|
+- ackCallback.onException(new MQClientException("unknown reason. addr: " + addr + ", timeoutMillis: " + timeOut + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- }
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+ }
|
|||
|
|
++ ackCallback.onSuccess(ackResult);
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ ackCallback.onException(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+@@ -999,39 +989,37 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ final AckCallback ackCallback
|
|||
|
|
+ ) throws RemotingException, MQBrokerException, InterruptedException {
|
|||
|
|
+ final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHANGE_MESSAGE_INVISIBLETIME, requestHeader);
|
|||
|
|
+- this.remotingClient.invokeAsync(addr, request, timeoutMillis, new BaseInvokeCallback(MQClientAPIImpl.this) {
|
|||
|
|
++ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void onComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- ChangeInvisibleTimeResponseHeader responseHeader = (ChangeInvisibleTimeResponseHeader) response.decodeCommandCustomHeader(ChangeInvisibleTimeResponseHeader.class);
|
|||
|
|
+- AckResult ackResult = new AckResult();
|
|||
|
|
+- if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
+- ackResult.setStatus(AckStatus.OK);
|
|||
|
|
+- ackResult.setPopTime(responseHeader.getPopTime());
|
|||
|
|
+- ackResult.setExtraInfo(ExtraInfoUtil
|
|||
|
|
+- .buildExtraInfo(requestHeader.getOffset(), responseHeader.getPopTime(), responseHeader.getInvisibleTime(),
|
|||
|
|
+- responseHeader.getReviveQid(), requestHeader.getTopic(), brokerName, requestHeader.getQueueId()) + MessageConst.KEY_SEPARATOR
|
|||
|
|
+- + requestHeader.getOffset());
|
|||
|
|
+- } else {
|
|||
|
|
+- ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+- }
|
|||
|
|
+- ackCallback.onSuccess(ackResult);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- ackCallback.onException(e);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- ackCallback.onException(new MQClientException(ClientErrorCode.CONNECT_BROKER_EXCEPTION, "send request failed to " + addr + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- ackCallback.onException(new MQClientException(ClientErrorCode.ACCESS_BROKER_TIMEOUT, "wait response from " + addr + " timeout :" + responseFuture.getTimeoutMillis() + "ms" + ". Request: " + request,
|
|||
|
|
+- responseFuture.getCause()));
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ try {
|
|||
|
|
++ ChangeInvisibleTimeResponseHeader responseHeader = (ChangeInvisibleTimeResponseHeader) response.decodeCommandCustomHeader(ChangeInvisibleTimeResponseHeader.class);
|
|||
|
|
++ AckResult ackResult = new AckResult();
|
|||
|
|
++ if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
++ ackResult.setStatus(AckStatus.OK);
|
|||
|
|
++ ackResult.setPopTime(responseHeader.getPopTime());
|
|||
|
|
++ ackResult.setExtraInfo(ExtraInfoUtil
|
|||
|
|
++ .buildExtraInfo(requestHeader.getOffset(), responseHeader.getPopTime(), responseHeader.getInvisibleTime(),
|
|||
|
|
++ responseHeader.getReviveQid(), requestHeader.getTopic(), brokerName, requestHeader.getQueueId()) + MessageConst.KEY_SEPARATOR
|
|||
|
|
++ + requestHeader.getOffset());
|
|||
|
|
+ } else {
|
|||
|
|
+- ackCallback.onException(new MQClientException("unknown reason. addr: " + addr + ", timeoutMillis: " + timeoutMillis + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
++ ackResult.setStatus(AckStatus.NO_EXIST);
|
|||
|
|
+ }
|
|||
|
|
++ ackCallback.onSuccess(ackResult);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ ackCallback.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ ackCallback.onException(throwable);
|
|||
|
|
++ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -1044,26 +1032,23 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- PullResult pullResult = MQClientAPIImpl.this.processPullResponse(response, addr);
|
|||
|
|
+- assert pullResult != null;
|
|||
|
|
+- pullCallback.onSuccess(pullResult);
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- pullCallback.onException(e);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- pullCallback.onException(new MQClientException(ClientErrorCode.CONNECT_BROKER_EXCEPTION, "send request failed to " + addr + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- pullCallback.onException(new MQClientException(ClientErrorCode.ACCESS_BROKER_TIMEOUT, "wait response from " + addr + " timeout :" + responseFuture.getTimeoutMillis() + "ms" + ". Request: " + request,
|
|||
|
|
+- responseFuture.getCause()));
|
|||
|
|
+- } else {
|
|||
|
|
+- pullCallback.onException(new MQClientException("unknown reason. addr: " + addr + ", timeoutMillis: " + timeoutMillis + ". Request: " + request, responseFuture.getCause()));
|
|||
|
|
+- }
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ try {
|
|||
|
|
++ PullResult pullResult = MQClientAPIImpl.this.processPullResponse(response, addr);
|
|||
|
|
++ pullCallback.onSuccess(pullResult);
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ pullCallback.onException(e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ pullCallback.onException(throwable);
|
|||
|
|
++ }
|
|||
|
|
+ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+index d7c8ef8d9..f3102e175 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/mqclient/MQClientAPIExt.java
|
|||
|
|
+@@ -30,7 +30,6 @@ import org.apache.rocketmq.client.consumer.PullCallback;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullResult;
|
|||
|
|
+ import org.apache.rocketmq.client.consumer.PullStatus;
|
|||
|
|
+ import org.apache.rocketmq.client.exception.MQBrokerException;
|
|||
|
|
+-import org.apache.rocketmq.client.exception.MQClientException;
|
|||
|
|
+ import org.apache.rocketmq.client.exception.OffsetNotFoundException;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.ClientRemotingProcessor;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.CommunicationMode;
|
|||
|
|
+@@ -47,6 +46,7 @@ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
++import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingCommandException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyClientConfig;
|
|||
|
|
+@@ -106,19 +106,6 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- protected static MQClientException processNullResponseErr(ResponseFuture responseFuture) {
|
|||
|
|
+- MQClientException ex;
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- ex = new MQClientException("send request failed", responseFuture.getCause());
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- ex = new MQClientException("wait response timeout " + responseFuture.getTimeoutMillis() + "ms",
|
|||
|
|
+- responseFuture.getCause());
|
|||
|
|
+- } else {
|
|||
|
|
+- ex = new MQClientException("unknown reason", responseFuture.getCause());
|
|||
|
|
+- }
|
|||
|
|
+- return ex;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+ public CompletableFuture<Void> sendHeartbeatOneway(
|
|||
|
|
+ String brokerAddr,
|
|||
|
|
+ HeartbeatData heartbeatData,
|
|||
|
|
+@@ -146,24 +133,15 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ request.setLanguage(clientConfig.getLanguage());
|
|||
|
|
+ request.setBody(heartbeatData.encode());
|
|||
|
|
+
|
|||
|
|
+- CompletableFuture<Integer> future = new CompletableFuture<>();
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
+- future.complete(response.getVersion());
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr));
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
+- }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<Integer> future0 = new CompletableFuture<>();
|
|||
|
|
++ if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
++ future0.complete(response.getVersion());
|
|||
|
|
++ } else {
|
|||
|
|
++ future0.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr));
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<SendResult> sendMessageAsync(
|
|||
|
|
+@@ -177,24 +155,15 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE_V2, requestHeaderV2);
|
|||
|
|
+ request.setBody(msg.getBody());
|
|||
|
|
+
|
|||
|
|
+- CompletableFuture<SendResult> future = new CompletableFuture<>();
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- future.complete(this.processSendResponse(brokerName, msg, response, brokerAddr));
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- future.completeExceptionally(e);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
+- }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<SendResult> future0 = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ future0.complete(this.processSendResponse(brokerName, msg, response, brokerAddr));
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ future0.completeExceptionally(e);
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<SendResult> sendMessageAsync(
|
|||
|
|
+@@ -216,17 +185,14 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ msgBatch.setBody(body);
|
|||
|
|
+
|
|||
|
|
+ request.setBody(body);
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- try {
|
|||
|
|
+- future.complete(this.processSendResponse(brokerName, msgBatch, response, brokerAddr));
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- future.completeExceptionally(e);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<SendResult> future0 = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ future0.complete(processSendResponse(brokerName, msgBatch, response, brokerAddr));
|
|||
|
|
++ } catch (Exception e) {
|
|||
|
|
++ future0.completeExceptionally(e);
|
|||
|
|
+ }
|
|||
|
|
++ return future0;
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ future.completeExceptionally(t);
|
|||
|
|
+@@ -240,21 +206,7 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ ) {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONSUMER_SEND_MSG_BACK, requestHeader);
|
|||
|
|
+-
|
|||
|
|
+- CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- future.complete(response);
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
+- }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<PopResult> popMessageAsync(
|
|||
|
|
+@@ -402,38 +354,31 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ QueryConsumerOffsetRequestHeader requestHeader,
|
|||
|
|
+ long timeoutMillis
|
|||
|
|
+ ) {
|
|||
|
|
+- CompletableFuture<Long> future = new CompletableFuture<>();
|
|||
|
|
+- try {
|
|||
|
|
+- RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUMER_OFFSET, requestHeader);
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- switch (response.getCode()) {
|
|||
|
|
+- case ResponseCode.SUCCESS: {
|
|||
|
|
+- try {
|
|||
|
|
+- QueryConsumerOffsetResponseHeader responseHeader =
|
|||
|
|
+- (QueryConsumerOffsetResponseHeader) response.decodeCommandCustomHeader(QueryConsumerOffsetResponseHeader.class);
|
|||
|
|
+- future.complete(responseHeader.getOffset());
|
|||
|
|
+- } catch (RemotingCommandException e) {
|
|||
|
|
+- future.completeExceptionally(e);
|
|||
|
|
+- }
|
|||
|
|
+- break;
|
|||
|
|
+- }
|
|||
|
|
+- case ResponseCode.QUERY_NOT_FOUND: {
|
|||
|
|
+- future.completeExceptionally(new OffsetNotFoundException(response.getCode(), response.getRemark(), brokerAddr));
|
|||
|
|
+- break;
|
|||
|
|
+- }
|
|||
|
|
+- default:
|
|||
|
|
+- break;
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUMER_OFFSET, requestHeader);
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<Long> future0 = new CompletableFuture<>();
|
|||
|
|
++ switch (response.getCode()) {
|
|||
|
|
++ case ResponseCode.SUCCESS: {
|
|||
|
|
++ try {
|
|||
|
|
++ QueryConsumerOffsetResponseHeader responseHeader =
|
|||
|
|
++ (QueryConsumerOffsetResponseHeader) response.decodeCommandCustomHeader(QueryConsumerOffsetResponseHeader.class);
|
|||
|
|
++ future0.complete(responseHeader.getOffset());
|
|||
|
|
++ } catch (RemotingCommandException e) {
|
|||
|
|
++ future0.completeExceptionally(e);
|
|||
|
|
+ }
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ break;
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ case ResponseCode.QUERY_NOT_FOUND: {
|
|||
|
|
++ future0.completeExceptionally(new OffsetNotFoundException(response.getCode(), response.getRemark(), brokerAddr));
|
|||
|
|
++ break;
|
|||
|
|
++ }
|
|||
|
|
++ default: {
|
|||
|
|
++ future0.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ break;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<Void> updateConsumerOffsetOneWay(
|
|||
|
|
+@@ -461,9 +406,14 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<List<String>> future = new CompletableFuture<>();
|
|||
|
|
+ try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
++ this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ switch (response.getCode()) {
|
|||
|
|
+ case ResponseCode.SUCCESS: {
|
|||
|
|
+ if (response.getBody() != null) {
|
|||
|
|
+@@ -485,8 +435,11 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ break;
|
|||
|
|
+ }
|
|||
|
|
+ future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+@@ -501,9 +454,14 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<Long> future = new CompletableFuture<>();
|
|||
|
|
+ try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
++ this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
+ try {
|
|||
|
|
+ GetMaxOffsetResponseHeader responseHeader = (GetMaxOffsetResponseHeader) response.decodeCommandCustomHeader(GetMaxOffsetResponseHeader.class);
|
|||
|
|
+@@ -513,8 +471,11 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+@@ -529,9 +490,14 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<Long> future = new CompletableFuture<>();
|
|||
|
|
+ try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
++ this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ if (ResponseCode.SUCCESS == response.getCode()) {
|
|||
|
|
+ try {
|
|||
|
|
+ GetMinOffsetResponseHeader responseHeader = (GetMinOffsetResponseHeader) response.decodeCommandCustomHeader(GetMinOffsetResponseHeader.class);
|
|||
|
|
+@@ -541,8 +507,11 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+@@ -555,57 +524,41 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+ long timeoutMillis) {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, requestHeader);
|
|||
|
|
+
|
|||
|
|
+- CompletableFuture<Long> future = new CompletableFuture<>();
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+- try {
|
|||
|
|
+- SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.decodeCommandCustomHeader(SearchOffsetResponseHeader.class);
|
|||
|
|
+- future.complete(responseHeader.getOffset());
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+- future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<Long> future0 = new CompletableFuture<>();
|
|||
|
|
++ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
++ try {
|
|||
|
|
++ SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.decodeCommandCustomHeader(SearchOffsetResponseHeader.class);
|
|||
|
|
++ future0.complete(responseHeader.getOffset());
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future0.completeExceptionally(t);
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ } else {
|
|||
|
|
++ future0.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<Set<MessageQueue>> lockBatchMQWithFuture(String brokerAddr,
|
|||
|
|
+ LockBatchRequestBody requestBody, long timeoutMillis) {
|
|||
|
|
+- CompletableFuture<Set<MessageQueue>> future = new CompletableFuture<>();
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, null);
|
|||
|
|
+ request.setBody(requestBody.encode());
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invokeAsync(brokerAddr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
+- if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+- try {
|
|||
|
|
+- LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(), LockBatchResponseBody.class);
|
|||
|
|
+- Set<MessageQueue> messageQueues = responseBody.getLockOKMQSet();
|
|||
|
|
+- future.complete(messageQueues);
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
+- future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(processNullResponseErr(responseFuture));
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<Set<MessageQueue>> future0 = new CompletableFuture<>();
|
|||
|
|
++ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
++ try {
|
|||
|
|
++ LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(), LockBatchResponseBody.class);
|
|||
|
|
++ Set<MessageQueue> messageQueues = responseBody.getLockOKMQSet();
|
|||
|
|
++ future0.complete(messageQueues);
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future0.completeExceptionally(t);
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Exception e) {
|
|||
|
|
+- future.completeExceptionally(e);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ } else {
|
|||
|
|
++ future0.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<Void> unlockBatchMQOneway(String brokerAddr,
|
|||
|
|
+@@ -624,25 +577,21 @@ public class MQClientAPIExt extends MQClientAPIImpl {
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<Boolean> notification(String brokerAddr, NotificationRequestHeader requestHeader,
|
|||
|
|
+ long timeoutMillis) {
|
|||
|
|
+- CompletableFuture<Boolean> future = new CompletableFuture<>();
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.NOTIFICATION, requestHeader);
|
|||
|
|
+- try {
|
|||
|
|
+- this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenAccept(response -> {
|
|||
|
|
+- if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
+- try {
|
|||
|
|
+- NotificationResponseHeader responseHeader = (NotificationResponseHeader) response.decodeCommandCustomHeader(NotificationResponseHeader.class);
|
|||
|
|
+- future.complete(responseHeader.isHasMsg());
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ return this.getRemotingClient().invoke(brokerAddr, request, timeoutMillis).thenCompose(response -> {
|
|||
|
|
++ CompletableFuture<Boolean> future0 = new CompletableFuture<>();
|
|||
|
|
++ if (response.getCode() == ResponseCode.SUCCESS) {
|
|||
|
|
++ try {
|
|||
|
|
++ NotificationResponseHeader responseHeader = (NotificationResponseHeader) response.decodeCommandCustomHeader(NotificationResponseHeader.class);
|
|||
|
|
++ future0.complete(responseHeader.isHasMsg());
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future0.completeExceptionally(t);
|
|||
|
|
+ }
|
|||
|
|
+- });
|
|||
|
|
+- } catch (Throwable t) {
|
|||
|
|
+- future.completeExceptionally(t);
|
|||
|
|
+- }
|
|||
|
|
+- return future;
|
|||
|
|
++ } else {
|
|||
|
|
++ future0.completeExceptionally(new MQBrokerException(response.getCode(), response.getRemark()));
|
|||
|
|
++ }
|
|||
|
|
++ return future0;
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CompletableFuture<RemotingCommand> invoke(String brokerAddr, RemotingCommand request, long timeoutMillis) {
|
|||
|
|
+diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java
|
|||
|
|
+index d13f2cfe4..c152d38ea 100644
|
|||
|
|
+--- a/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java
|
|||
|
|
++++ b/client/src/test/java/org/apache/rocketmq/client/impl/MQClientAPIImplTest.java
|
|||
|
|
+@@ -212,7 +212,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ RemotingCommand request = mock.getArgument(1);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+ responseFuture.setResponseCommand(createSendMessageSuccessResponse(request));
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+@@ -386,7 +386,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ RemotingCommand request = mock.getArgument(1);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+ responseFuture.setResponseCommand(createSendMessageSuccessResponse(request));
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(Matchers.anyString(), Matchers.any(RemotingCommand.class), Matchers.anyLong(), Matchers.any(InvokeCallback.class));
|
|||
|
|
+@@ -472,7 +472,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ message.putUserProperty("key", "value");
|
|||
|
|
+ response.setBody(MessageDecoder.encode(message, false));
|
|||
|
|
+ responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+@@ -543,7 +543,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ message.getProperties().put(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET, String.valueOf(0));
|
|||
|
|
+ response.setBody(MessageDecoder.encode(message, false));
|
|||
|
|
+ responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+@@ -585,7 +585,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ response.setOpaque(request.getOpaque());
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+@@ -622,7 +622,7 @@ public class MQClientAPIImplTest {
|
|||
|
|
+ responseHeader.setPopTime(System.currentTimeMillis());
|
|||
|
|
+ responseHeader.setInvisibleTime(10 * 1000L);
|
|||
|
|
+ responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
+index fe07090d5..3227d1e1c 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/RemotingProtocolServer.java
|
|||
|
|
+@@ -26,7 +26,6 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+ import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.rocketmq.acl.AccessValidator;
|
|||
|
|
+-import org.apache.rocketmq.client.exception.MQClientException;
|
|||
|
|
+ import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+ import org.apache.rocketmq.common.future.FutureTaskExt;
|
|||
|
|
+ import org.apache.rocketmq.common.thread.ThreadPoolMonitor;
|
|||
|
|
+@@ -51,10 +50,12 @@ import org.apache.rocketmq.proxy.remoting.channel.RemotingChannelManager;
|
|||
|
|
+ import org.apache.rocketmq.proxy.remoting.pipeline.AuthenticationPipeline;
|
|||
|
|
+ import org.apache.rocketmq.proxy.remoting.pipeline.RequestPipeline;
|
|||
|
|
+ import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
++import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RemotingServer;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyServerConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.RequestTask;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.TlsSystemConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+@@ -239,12 +240,21 @@ public class RemotingProtocolServer implements StartAndShutdown, RemotingProxyOu
|
|||
|
|
+ long timeoutMillis) {
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
+ try {
|
|||
|
|
+- this.defaultRemotingServer.invokeAsync(channel, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- if (responseFuture.getResponseCommand() == null) {
|
|||
|
|
+- future.completeExceptionally(new MQClientException("response is null after send request to client", responseFuture.getCause()));
|
|||
|
|
+- return;
|
|||
|
|
++ this.defaultRemotingServer.invokeAsync(channel, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
+ }
|
|||
|
|
+- future.complete(responseFuture.getResponseCommand());
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+ future.completeExceptionally(t);
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+index 3f3a4ae40..e2d05b0f5 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/mqclient/MQClientAPIExtTest.java
|
|||
|
|
+@@ -24,6 +24,7 @@ import java.util.ArrayList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.UUID;
|
|||
|
|
++import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ThreadLocalRandom;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
|
+ import java.util.stream.Collectors;
|
|||
|
|
+@@ -85,6 +86,7 @@ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.Mockito.doAnswer;
|
|||
|
|
++import static org.mockito.Mockito.doReturn;
|
|||
|
|
+
|
|||
|
|
+ @RunWith(MockitoJUnitRunner.class)
|
|||
|
|
+ public class MQClientAPIExtTest {
|
|||
|
|
+@@ -109,13 +111,9 @@ public class MQClientAPIExtTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testSendHeartbeatAsync() throws Exception {
|
|||
|
|
+- doAnswer((Answer<Void>) mock -> {
|
|||
|
|
+- InvokeCallback invokeCallback = mock.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, 0, 3000, invokeCallback, null);
|
|||
|
|
+- responseFuture.putResponse(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""));
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ future.complete(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""));
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ assertNotNull(mqClientAPI.sendHeartbeatAsync(BROKER_ADDR, new HeartbeatData(), TIMEOUT).get());
|
|||
|
|
+ }
|
|||
|
|
+@@ -123,20 +121,16 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testSendMessageAsync() throws Exception {
|
|||
|
|
+ AtomicReference<String> msgIdRef = new AtomicReference<>();
|
|||
|
|
+- doAnswer((Answer<Void>) mock -> {
|
|||
|
|
+- InvokeCallback invokeCallback = mock.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, 0, 3000, invokeCallback, null);
|
|||
|
|
+- RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
|
|||
|
|
+- SendMessageResponseHeader sendMessageResponseHeader = (SendMessageResponseHeader) response.readCustomHeader();
|
|||
|
|
+- sendMessageResponseHeader.setMsgId(msgIdRef.get());
|
|||
|
|
+- sendMessageResponseHeader.setQueueId(0);
|
|||
|
|
+- sendMessageResponseHeader.setQueueOffset(1L);
|
|||
|
|
+- response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- response.makeCustomHeaderToNet();
|
|||
|
|
+- responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
|
|||
|
|
++ SendMessageResponseHeader sendMessageResponseHeader = (SendMessageResponseHeader) response.readCustomHeader();
|
|||
|
|
++ sendMessageResponseHeader.setMsgId(msgIdRef.get());
|
|||
|
|
++ sendMessageResponseHeader.setQueueId(0);
|
|||
|
|
++ sendMessageResponseHeader.setQueueOffset(1L);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ response.makeCustomHeaderToNet();
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ MessageExt messageExt = createMessage();
|
|||
|
|
+ msgIdRef.set(MessageClientIDSetter.getUniqID(messageExt));
|
|||
|
|
+@@ -150,20 +144,16 @@ public class MQClientAPIExtTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testSendMessageListAsync() throws Exception {
|
|||
|
|
+- doAnswer((Answer<Void>) mock -> {
|
|||
|
|
+- InvokeCallback invokeCallback = mock.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, 0, 3000, invokeCallback, null);
|
|||
|
|
+- RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
|
|||
|
|
+- SendMessageResponseHeader sendMessageResponseHeader = (SendMessageResponseHeader) response.readCustomHeader();
|
|||
|
|
+- sendMessageResponseHeader.setMsgId("");
|
|||
|
|
+- sendMessageResponseHeader.setQueueId(0);
|
|||
|
|
+- sendMessageResponseHeader.setQueueOffset(1L);
|
|||
|
|
+- response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- response.makeCustomHeaderToNet();
|
|||
|
|
+- responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(SendMessageResponseHeader.class);
|
|||
|
|
++ SendMessageResponseHeader sendMessageResponseHeader = (SendMessageResponseHeader) response.readCustomHeader();
|
|||
|
|
++ sendMessageResponseHeader.setMsgId("");
|
|||
|
|
++ sendMessageResponseHeader.setQueueId(0);
|
|||
|
|
++ sendMessageResponseHeader.setQueueOffset(1L);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ response.makeCustomHeaderToNet();
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ List<MessageExt> messageExtList = new ArrayList<>();
|
|||
|
|
+ StringBuilder sb = new StringBuilder();
|
|||
|
|
+@@ -182,13 +172,9 @@ public class MQClientAPIExtTest {
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testSendMessageBackAsync() throws Exception {
|
|||
|
|
+- doAnswer((Answer<Void>) mock -> {
|
|||
|
|
+- InvokeCallback invokeCallback = mock.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, 0, 3000, invokeCallback, null);
|
|||
|
|
+- responseFuture.putResponse(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""));
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ future.complete(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, ""));
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ RemotingCommand remotingCommand = mqClientAPI.sendMessageBackAsync(BROKER_ADDR, new ConsumerSendMsgBackRequestHeader(), TIMEOUT)
|
|||
|
|
+ .get();
|
|||
|
|
+@@ -285,7 +271,7 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ body.setConsumerIdList(clientIds);
|
|||
|
|
+ response.setBody(body.encode());
|
|||
|
|
+ responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
++ invokeCallback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
+
|
|||
|
|
+@@ -302,7 +288,7 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ response.setCode(ResponseCode.SYSTEM_ERROR);
|
|||
|
|
+ response.makeCustomHeaderToNet();
|
|||
|
|
+ responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
++ invokeCallback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
+
|
|||
|
|
+@@ -322,7 +308,7 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ response.makeCustomHeaderToNet();
|
|||
|
|
+ responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
++ invokeCallback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
+
|
|||
|
|
+@@ -335,18 +321,15 @@ public class MQClientAPIExtTest {
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testSearchOffsetAsync() throws Exception {
|
|||
|
|
+ long offset = ThreadLocalRandom.current().nextLong();
|
|||
|
|
+- doAnswer((Answer<Void>) mock -> {
|
|||
|
|
+- InvokeCallback invokeCallback = mock.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, 0, 3000, invokeCallback, null);
|
|||
|
|
+- RemotingCommand response = RemotingCommand.createResponseCommand(SearchOffsetResponseHeader.class);
|
|||
|
|
+- SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.readCustomHeader();
|
|||
|
|
+- responseHeader.setOffset(offset);
|
|||
|
|
+- response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- response.makeCustomHeaderToNet();
|
|||
|
|
+- responseFuture.putResponse(response);
|
|||
|
|
+- invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any());
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(SearchOffsetResponseHeader.class);
|
|||
|
|
++ SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.readCustomHeader();
|
|||
|
|
++ responseHeader.setOffset(offset);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ response.makeCustomHeaderToNet();
|
|||
|
|
++ future.complete(response);
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ SearchOffsetRequestHeader requestHeader = new SearchOffsetRequestHeader();
|
|||
|
|
+ requestHeader.setTopic(TOPIC);
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/InvokeCallback.java b/remoting/src/main/java/org/apache/rocketmq/remoting/InvokeCallback.java
|
|||
|
|
+index ce78fa923..6be491745 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/InvokeCallback.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/InvokeCallback.java
|
|||
|
|
+@@ -17,7 +17,22 @@
|
|||
|
|
+ package org.apache.rocketmq.remoting;
|
|||
|
|
+
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+
|
|||
|
|
+ public interface InvokeCallback {
|
|||
|
|
++ /**
|
|||
|
|
++ * This method is expected to be invoked after {@link #operationSucceed(RemotingCommand)}
|
|||
|
|
++ * or {@link #operationFail(Throwable)}
|
|||
|
|
++ *
|
|||
|
|
++ * @param responseFuture the returned object contains response or exception
|
|||
|
|
++ */
|
|||
|
|
+ void operationComplete(final ResponseFuture responseFuture);
|
|||
|
|
++
|
|||
|
|
++ default void operationSucceed(final RemotingCommand response) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ default void operationFail(final Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
|
|||
|
|
+index ff0b3df95..c8389eedb 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/RemotingClient.java
|
|||
|
|
+@@ -20,11 +20,11 @@ import java.util.List;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingConnectException;
|
|||
|
|
+-import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+
|
|||
|
|
+ public interface RemotingClient extends RemotingService {
|
|||
|
|
+@@ -51,18 +51,21 @@ public interface RemotingClient extends RemotingService {
|
|||
|
|
+ final long timeoutMillis) {
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
+ try {
|
|||
|
|
+- invokeAsync(addr, request, timeoutMillis, responseFuture -> {
|
|||
|
|
+- RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
+- if (response != null) {
|
|||
|
|
++ invokeAsync(addr, request, timeoutMillis, new InvokeCallback() {
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ future.complete(response);
|
|||
|
|
+- } else {
|
|||
|
|
+- if (!responseFuture.isSendRequestOK()) {
|
|||
|
|
+- future.completeExceptionally(new RemotingSendRequestException(addr, responseFuture.getCause()));
|
|||
|
|
+- } else if (responseFuture.isTimeout()) {
|
|||
|
|
+- future.completeExceptionally(new RemotingTimeoutException(addr, timeoutMillis, responseFuture.getCause()));
|
|||
|
|
+- } else {
|
|||
|
|
+- future.completeExceptionally(new RemotingException(request.toString(), responseFuture.getCause()));
|
|||
|
|
+- }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ } catch (Throwable t) {
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+index fce2de267..12e66f913 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+@@ -23,20 +23,23 @@ import io.netty.handler.ssl.SslContext;
|
|||
|
|
+ import io.netty.handler.ssl.SslHandler;
|
|||
|
|
+ import io.netty.util.concurrent.Future;
|
|||
|
|
+ import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+-import java.net.SocketAddress;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.Iterator;
|
|||
|
|
+ import java.util.LinkedList;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Map.Entry;
|
|||
|
|
++import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
++import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
+ import java.util.concurrent.RejectedExecutionException;
|
|||
|
|
+ import java.util.concurrent.Semaphore;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
++import java.util.concurrent.TimeoutException;
|
|||
|
|
++import java.util.concurrent.atomic.AtomicReference;
|
|||
|
|
+ import java.util.function.Consumer;
|
|||
|
|
+ import javax.annotation.Nullable;
|
|||
|
|
+ import org.apache.rocketmq.common.AbortProcessException;
|
|||
|
|
+@@ -125,7 +128,7 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ * Constructor, specifying capacity of one-way and asynchronous semaphores.
|
|||
|
|
+ *
|
|||
|
|
+ * @param permitsOneway Number of permits for one-way requests.
|
|||
|
|
+- * @param permitsAsync Number of permits for asynchronous requests.
|
|||
|
|
++ * @param permitsAsync Number of permits for asynchronous requests.
|
|||
|
|
+ */
|
|||
|
|
+ public NettyRemotingAbstract(final int permitsOneway, final int permitsAsync) {
|
|||
|
|
+ this.semaphoreOneway = new Semaphore(permitsOneway, true);
|
|||
|
|
+@@ -367,8 +370,7 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ responseFuture.release();
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+- log.warn("receive response, but not matched any request, " + RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
|
|||
|
|
+- log.warn(cmd.toString());
|
|||
|
|
++ log.warn("receive response, cmd={}, but not matched any request, address={}", cmd, RemotingHelper.parseChannelRemoteAddr(ctx.channel()));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -467,57 +469,68 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ public RemotingCommand invokeSyncImpl(final Channel channel, final RemotingCommand request,
|
|||
|
|
+ final long timeoutMillis)
|
|||
|
|
+ throws InterruptedException, RemotingSendRequestException, RemotingTimeoutException {
|
|||
|
|
+- //get the request id
|
|||
|
|
+- final int opaque = request.getOpaque();
|
|||
|
|
+-
|
|||
|
|
+ try {
|
|||
|
|
+- final ResponseFuture responseFuture = new ResponseFuture(channel, opaque, timeoutMillis, null, null);
|
|||
|
|
+- this.responseTable.put(opaque, responseFuture);
|
|||
|
|
+- final SocketAddress addr = channel.remoteAddress();
|
|||
|
|
+- channel.writeAndFlush(request).addListener((ChannelFutureListener) f -> {
|
|||
|
|
+- if (f.isSuccess()) {
|
|||
|
|
+- responseFuture.setSendRequestOK(true);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+-
|
|||
|
|
+- responseFuture.setSendRequestOK(false);
|
|||
|
|
+- responseTable.remove(opaque);
|
|||
|
|
+- responseFuture.setCause(f.cause());
|
|||
|
|
+- responseFuture.putResponse(null);
|
|||
|
|
+- log.warn("Failed to write a request command to {}, caused by underlying I/O operation failure", addr);
|
|||
|
|
+- });
|
|||
|
|
++ return invokeImpl(channel, request, timeoutMillis).thenApply(ResponseFuture::getResponseCommand)
|
|||
|
|
++ .get(timeoutMillis, TimeUnit.MILLISECONDS);
|
|||
|
|
++ } catch (ExecutionException e) {
|
|||
|
|
++ throw new RemotingSendRequestException(channel.remoteAddress().toString(), e.getCause());
|
|||
|
|
++ } catch (TimeoutException e) {
|
|||
|
|
++ throw new RemotingTimeoutException(channel.remoteAddress().toString(), timeoutMillis, e.getCause());
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+
|
|||
|
|
+- RemotingCommand responseCommand = responseFuture.waitResponse(timeoutMillis);
|
|||
|
|
+- if (null == responseCommand) {
|
|||
|
|
+- if (responseFuture.isSendRequestOK()) {
|
|||
|
|
+- throw new RemotingTimeoutException(RemotingHelper.parseSocketAddressAddr(addr), timeoutMillis,
|
|||
|
|
+- responseFuture.getCause());
|
|||
|
|
+- } else {
|
|||
|
|
+- throw new RemotingSendRequestException(RemotingHelper.parseSocketAddressAddr(addr), responseFuture.getCause());
|
|||
|
|
+- }
|
|||
|
|
++ public CompletableFuture<ResponseFuture> invokeImpl(final Channel channel, final RemotingCommand request,
|
|||
|
|
++ final long timeoutMillis) {
|
|||
|
|
++ String channelRemoteAddr = RemotingHelper.parseChannelRemoteAddr(channel);
|
|||
|
|
++ doBeforeRpcHooks(channelRemoteAddr, request);
|
|||
|
|
++ return invoke0(channel, request, timeoutMillis).whenComplete((v, t) -> {
|
|||
|
|
++ if (t == null) {
|
|||
|
|
++ doAfterRpcHooks(channelRemoteAddr, request, v.getResponseCommand());
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+- return responseCommand;
|
|||
|
|
+- } finally {
|
|||
|
|
+- this.responseTable.remove(opaque);
|
|||
|
|
+- }
|
|||
|
|
++ });
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void invokeAsyncImpl(final Channel channel, final RemotingCommand request, final long timeoutMillis,
|
|||
|
|
+- final InvokeCallback invokeCallback)
|
|||
|
|
+- throws InterruptedException, RemotingTooMuchRequestException, RemotingTimeoutException, RemotingSendRequestException {
|
|||
|
|
++ protected CompletableFuture<ResponseFuture> invoke0(final Channel channel, final RemotingCommand request,
|
|||
|
|
++ final long timeoutMillis) {
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
+ long beginStartTime = System.currentTimeMillis();
|
|||
|
|
+ final int opaque = request.getOpaque();
|
|||
|
|
+- boolean acquired = this.semaphoreAsync.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
|
|||
|
|
++
|
|||
|
|
++ boolean acquired;
|
|||
|
|
++ try {
|
|||
|
|
++ acquired = this.semaphoreAsync.tryAcquire(timeoutMillis, TimeUnit.MILLISECONDS);
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
+ if (acquired) {
|
|||
|
|
+ final SemaphoreReleaseOnlyOnce once = new SemaphoreReleaseOnlyOnce(this.semaphoreAsync);
|
|||
|
|
+ long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+ if (timeoutMillis < costTime) {
|
|||
|
|
+ once.release();
|
|||
|
|
+- throw new RemotingTimeoutException("invokeAsyncImpl call timeout");
|
|||
|
|
++ future.completeExceptionally(new RemotingTimeoutException("invokeAsyncImpl call timeout"));
|
|||
|
|
++ return future;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- final ResponseFuture responseFuture = new ResponseFuture(channel, opaque, timeoutMillis - costTime, invokeCallback, once);
|
|||
|
|
++ AtomicReference<ResponseFuture> responseFutureReference = new AtomicReference<>();
|
|||
|
|
++ final ResponseFuture responseFuture = new ResponseFuture(channel, opaque, request, timeoutMillis - costTime,
|
|||
|
|
++ new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ future.complete(responseFutureReference.get());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ future.completeExceptionally(throwable);
|
|||
|
|
++ }
|
|||
|
|
++ }, once);
|
|||
|
|
++ responseFutureReference.set(responseFuture);
|
|||
|
|
+ this.responseTable.put(opaque, responseFuture);
|
|||
|
|
+ try {
|
|||
|
|
+ channel.writeAndFlush(request).addListener((ChannelFutureListener) f -> {
|
|||
|
|
+@@ -528,15 +541,17 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ requestFail(opaque);
|
|||
|
|
+ log.warn("send a request command to channel <{}> failed.", RemotingHelper.parseChannelRemoteAddr(channel));
|
|||
|
|
+ });
|
|||
|
|
++ return future;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ responseTable.remove(opaque);
|
|||
|
|
+ responseFuture.release();
|
|||
|
|
+ log.warn("send a request command to channel <" + RemotingHelper.parseChannelRemoteAddr(channel) + "> Exception", e);
|
|||
|
|
+- throw new RemotingSendRequestException(RemotingHelper.parseChannelRemoteAddr(channel), e);
|
|||
|
|
++ future.completeExceptionally(new RemotingSendRequestException(RemotingHelper.parseChannelRemoteAddr(channel), e));
|
|||
|
|
++ return future;
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+ if (timeoutMillis <= 0) {
|
|||
|
|
+- throw new RemotingTooMuchRequestException("invokeAsyncImpl invoke too fast");
|
|||
|
|
++ future.completeExceptionally(new RemotingTooMuchRequestException("invokeAsyncImpl invoke too fast"));
|
|||
|
|
+ } else {
|
|||
|
|
+ String info =
|
|||
|
|
+ String.format("invokeAsyncImpl tryAcquire semaphore timeout, %dms, waiting thread nums: %d semaphoreAsyncValue: %d",
|
|||
|
|
+@@ -545,11 +560,31 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ this.semaphoreAsync.availablePermits()
|
|||
|
|
+ );
|
|||
|
|
+ log.warn(info);
|
|||
|
|
+- throw new RemotingTimeoutException(info);
|
|||
|
|
++ future.completeExceptionally(new RemotingTimeoutException(info));
|
|||
|
|
+ }
|
|||
|
|
++ return future;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public void invokeAsyncImpl(final Channel channel, final RemotingCommand request, final long timeoutMillis,
|
|||
|
|
++ final InvokeCallback invokeCallback) {
|
|||
|
|
++ invokeImpl(channel, request, timeoutMillis)
|
|||
|
|
++ .whenComplete((v, t) -> {
|
|||
|
|
++ if (t == null) {
|
|||
|
|
++ invokeCallback.operationComplete(v);
|
|||
|
|
++ } else {
|
|||
|
|
++ ResponseFuture responseFuture = new ResponseFuture(channel, request.getOpaque(), request, timeoutMillis, null, null);
|
|||
|
|
++ responseFuture.setCause(t);
|
|||
|
|
++ invokeCallback.operationComplete(responseFuture);
|
|||
|
|
++ }
|
|||
|
|
++ })
|
|||
|
|
++ .thenAccept(responseFuture -> invokeCallback.operationSucceed(responseFuture.getResponseCommand()))
|
|||
|
|
++ .exceptionally(t -> {
|
|||
|
|
++ invokeCallback.operationFail(t);
|
|||
|
|
++ return null;
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private void requestFail(final int opaque) {
|
|||
|
|
+ ResponseFuture responseFuture = responseTable.remove(opaque);
|
|||
|
|
+ if (responseFuture != null) {
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+index 64621dd6c..d784351a5 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+@@ -527,15 +527,13 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ if (channel != null && channel.isActive()) {
|
|||
|
|
+ long left = timeoutMillis;
|
|||
|
|
+ try {
|
|||
|
|
+- doBeforeRpcHooks(channelRemoteAddr, request);
|
|||
|
|
+ long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+ left -= costTime;
|
|||
|
|
+ if (left <= 0) {
|
|||
|
|
+ throw new RemotingTimeoutException("invokeSync call the addr[" + channelRemoteAddr + "] timeout");
|
|||
|
|
+ }
|
|||
|
|
+ RemotingCommand response = this.invokeSyncImpl(channel, request, left);
|
|||
|
|
+- doAfterRpcHooks(channelRemoteAddr, request, response);
|
|||
|
|
+- this.updateChannelLastResponseTime(addr);
|
|||
|
|
++ updateChannelLastResponseTime(addr);
|
|||
|
|
+ return response;
|
|||
|
|
+ } catch (RemotingSendRequestException e) {
|
|||
|
|
+ LOGGER.warn("invokeSync: send request exception, so close the channel[{}]", channelRemoteAddr);
|
|||
|
|
+@@ -727,18 +725,11 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ final Channel channel = this.getAndCreateChannel(addr);
|
|||
|
|
+ String channelRemoteAddr = RemotingHelper.parseChannelRemoteAddr(channel);
|
|||
|
|
+ if (channel != null && channel.isActive()) {
|
|||
|
|
+- try {
|
|||
|
|
+- doBeforeRpcHooks(channelRemoteAddr, request);
|
|||
|
|
+- long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
+- if (timeoutMillis < costTime) {
|
|||
|
|
+- throw new RemotingTooMuchRequestException("invokeAsync call the addr[" + channelRemoteAddr + "] timeout");
|
|||
|
|
+- }
|
|||
|
|
+- this.invokeAsyncImpl(channel, request, timeoutMillis - costTime, new InvokeCallbackWrapper(invokeCallback, addr));
|
|||
|
|
+- } catch (RemotingSendRequestException e) {
|
|||
|
|
+- LOGGER.warn("invokeAsync: send request exception, so close the channel[{}]", channelRemoteAddr);
|
|||
|
|
+- this.closeChannel(addr, channel);
|
|||
|
|
+- throw e;
|
|||
|
|
++ long costTime = System.currentTimeMillis() - beginStartTime;
|
|||
|
|
++ if (timeoutMillis < costTime) {
|
|||
|
|
++ throw new RemotingTooMuchRequestException("invokeAsync call the addr[" + channelRemoteAddr + "] timeout");
|
|||
|
|
+ }
|
|||
|
|
++ this.invokeAsyncImpl(channel, request, timeoutMillis - costTime, new InvokeCallbackWrapper(invokeCallback, addr));
|
|||
|
|
+ } else {
|
|||
|
|
+ this.closeChannel(addr, channel);
|
|||
|
|
+ throw new RemotingConnectException(addr);
|
|||
|
|
+@@ -931,11 +922,19 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- if (responseFuture != null && responseFuture.isSendRequestOK() && responseFuture.getResponseCommand() != null) {
|
|||
|
|
+- NettyRemotingClient.this.updateChannelLastResponseTime(addr);
|
|||
|
|
+- }
|
|||
|
|
+ this.invokeCallback.operationComplete(responseFuture);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ updateChannelLastResponseTime(addr);
|
|||
|
|
++ this.invokeCallback.operationSucceed(response);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(final Throwable throwable) {
|
|||
|
|
++ this.invokeCallback.operationFail(throwable);
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class NettyClientHandler extends SimpleChannelInboundHandler<RemotingCommand> {
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
|
|||
|
|
+index 19f705d74..0882818fe 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/ResponseFuture.java
|
|||
|
|
+@@ -22,6 +22,9 @@ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.common.SemaphoreReleaseOnlyOnce;
|
|||
|
|
++import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
++import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
|
|||
|
|
++import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+
|
|||
|
|
+ public class ResponseFuture {
|
|||
|
|
+@@ -59,6 +62,18 @@ public class ResponseFuture {
|
|||
|
|
+ public void executeInvokeCallback() {
|
|||
|
|
+ if (invokeCallback != null) {
|
|||
|
|
+ if (this.executeCallbackOnlyOnce.compareAndSet(false, true)) {
|
|||
|
|
++ RemotingCommand response = getResponseCommand();
|
|||
|
|
++ if (response != null) {
|
|||
|
|
++ invokeCallback.operationSucceed(response);
|
|||
|
|
++ } else {
|
|||
|
|
++ if (!isSendRequestOK()) {
|
|||
|
|
++ invokeCallback.operationFail(new RemotingSendRequestException(channel.remoteAddress().toString(), getCause()));
|
|||
|
|
++ } else if (isTimeout()) {
|
|||
|
|
++ invokeCallback.operationFail(new RemotingTimeoutException(channel.remoteAddress().toString(), getTimeoutMillis(), getCause()));
|
|||
|
|
++ } else {
|
|||
|
|
++ invokeCallback.operationFail(new RemotingException(getRequestCommand().toString(), getCause()));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+ invokeCallback.operationComplete(this);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/rpc/RpcClientImpl.java b/remoting/src/main/java/org/apache/rocketmq/remoting/rpc/RpcClientImpl.java
|
|||
|
|
+index 133e0ed31..5328e8845 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/rpc/RpcClientImpl.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/rpc/RpcClientImpl.java
|
|||
|
|
+@@ -160,31 +160,38 @@ public class RpcClientImpl implements RpcClient {
|
|||
|
|
+ InvokeCallback callback = new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
+- RemotingCommand responseCommand = responseFuture.getResponseCommand();
|
|||
|
|
+- if (responseCommand == null) {
|
|||
|
|
+- processFailedResponse(addr, requestCommand, responseFuture, rpcResponsePromise);
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ try {
|
|||
|
|
+- switch (responseCommand.getCode()) {
|
|||
|
|
++ switch (response.getCode()) {
|
|||
|
|
+ case ResponseCode.SUCCESS:
|
|||
|
|
+ case ResponseCode.PULL_NOT_FOUND:
|
|||
|
|
+ case ResponseCode.PULL_RETRY_IMMEDIATELY:
|
|||
|
|
+ case ResponseCode.PULL_OFFSET_MOVED:
|
|||
|
|
+ PullMessageResponseHeader responseHeader =
|
|||
|
|
+- (PullMessageResponseHeader) responseCommand.decodeCommandCustomHeader(PullMessageResponseHeader.class);
|
|||
|
|
+- rpcResponsePromise.setSuccess(new RpcResponse(responseCommand.getCode(), responseHeader, responseCommand.getBody()));
|
|||
|
|
++ (PullMessageResponseHeader) response.decodeCommandCustomHeader(PullMessageResponseHeader.class);
|
|||
|
|
++ rpcResponsePromise.setSuccess(new RpcResponse(response.getCode(), responseHeader, response.getBody()));
|
|||
|
|
+ default:
|
|||
|
|
+- RpcResponse rpcResponse = new RpcResponse(new RpcException(responseCommand.getCode(), "unexpected remote response code"));
|
|||
|
|
++ RpcResponse rpcResponse = new RpcResponse(new RpcException(response.getCode(), "unexpected remote response code"));
|
|||
|
|
+ rpcResponsePromise.setSuccess(rpcResponse);
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+- String errorMessage = "process failed. addr: " + addr + ", timeoutMillis: " + responseFuture.getTimeoutMillis() + ". Request: " + requestCommand;
|
|||
|
|
+- RpcResponse rpcResponse = new RpcResponse(new RpcException(ResponseCode.RPC_UNKNOWN, errorMessage, e));
|
|||
|
|
++ String errorMessage = "process failed. addr: " + addr + ", timeoutMillis: " + timeoutMillis + ". Request: " + requestCommand;
|
|||
|
|
++ RpcResponse rpcResponse = new RpcResponse(new RpcException(ResponseCode.RPC_UNKNOWN, errorMessage, e));
|
|||
|
|
+ rpcResponsePromise.setSuccess(rpcResponse);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++ String errorMessage = "process failed. addr: " + addr + ". Request: " + requestCommand;
|
|||
|
|
++ RpcResponse rpcResponse = new RpcResponse(new RpcException(ResponseCode.RPC_UNKNOWN, errorMessage, throwable));
|
|||
|
|
++ rpcResponsePromise.setSuccess(rpcResponse);
|
|||
|
|
++ }
|
|||
|
|
+ };
|
|||
|
|
+
|
|||
|
|
+ this.remotingClient.invokeAsync(addr, requestCommand, timeoutMillis, callback);
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
|
|||
|
|
+index 90072960b..d0da0eb2e 100644
|
|||
|
|
+--- a/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/RemotingServerTest.java
|
|||
|
|
+@@ -26,12 +26,12 @@ import org.apache.rocketmq.remoting.exception.RemotingConnectException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+-import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+-import org.apache.rocketmq.remoting.netty.NettyServerConfig;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyClientConfig;
|
|||
|
|
+-import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyRemotingClient;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.NettyRemotingServer;
|
|||
|
|
+ import org.apache.rocketmq.remoting.netty.NettyRequestProcessor;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.NettyServerConfig;
|
|||
|
|
++import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.LanguageCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.junit.AfterClass;
|
|||
|
|
+@@ -40,7 +40,6 @@ import org.junit.Test;
|
|||
|
|
+
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+ import static org.junit.Assert.assertNotNull;
|
|||
|
|
+-import static org.junit.Assert.assertTrue;
|
|||
|
|
+
|
|||
|
|
+ public class RemotingServerTest {
|
|||
|
|
+ private static RemotingServer remotingServer;
|
|||
|
|
+@@ -122,10 +121,19 @@ public class RemotingServerTest {
|
|||
|
|
+ remotingClient.invokeAsync("localhost:" + remotingServer.localListenPort(), request, 1000 * 3, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ latch.countDown();
|
|||
|
|
+- assertTrue(responseFuture != null);
|
|||
|
|
+- assertThat(responseFuture.getResponseCommand().getLanguage()).isEqualTo(LanguageCode.JAVA);
|
|||
|
|
+- assertThat(responseFuture.getResponseCommand().getExtFields()).hasSize(2);
|
|||
|
|
++ assertThat(response.getLanguage()).isEqualTo(LanguageCode.JAVA);
|
|||
|
|
++ assertThat(response.getExtFields()).hasSize(2);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+ });
|
|||
|
|
+ latch.await();
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/BaseInvokeCallback.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannel.java
|
|||
|
|
+similarity index 57%
|
|||
|
|
+rename from client/src/main/java/org/apache/rocketmq/client/impl/BaseInvokeCallback.java
|
|||
|
|
+rename to remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannel.java
|
|||
|
|
+index 80188832e..8ddcdf35d 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/BaseInvokeCallback.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannel.java
|
|||
|
|
+@@ -15,23 +15,14 @@
|
|||
|
|
+ * limitations under the License.
|
|||
|
|
+ */
|
|||
|
|
+
|
|||
|
|
+-package org.apache.rocketmq.client.impl;
|
|||
|
|
++package org.apache.rocketmq.remoting.netty;
|
|||
|
|
+
|
|||
|
|
+-import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+-import org.apache.rocketmq.remoting.netty.ResponseFuture;
|
|||
|
|
+-
|
|||
|
|
+-public abstract class BaseInvokeCallback implements InvokeCallback {
|
|||
|
|
+- private final MQClientAPIImpl mqClientAPI;
|
|||
|
|
+-
|
|||
|
|
+- public BaseInvokeCallback(MQClientAPIImpl mqClientAPI) {
|
|||
|
|
+- this.mqClientAPI = mqClientAPI;
|
|||
|
|
+- }
|
|||
|
|
++import io.netty.channel.ChannelFuture;
|
|||
|
|
++import io.netty.channel.local.LocalChannel;
|
|||
|
|
+
|
|||
|
|
++public class MockChannel extends LocalChannel {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void operationComplete(final ResponseFuture responseFuture) {
|
|||
|
|
+- mqClientAPI.execRpcHooksAfterRequest(responseFuture);
|
|||
|
|
+- onComplete(responseFuture);
|
|||
|
|
++ public ChannelFuture writeAndFlush(Object msg) {
|
|||
|
|
++ return new MockChannelPromise(MockChannel.this);
|
|||
|
|
+ }
|
|||
|
|
+-
|
|||
|
|
+- public abstract void onComplete(final ResponseFuture responseFuture);
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannelPromise.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannelPromise.java
|
|||
|
|
+new file mode 100644
|
|||
|
|
+index 000000000..9c3a35487
|
|||
|
|
+--- /dev/null
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/MockChannelPromise.java
|
|||
|
|
+@@ -0,0 +1,191 @@
|
|||
|
|
++/*
|
|||
|
|
++ * 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.remoting.netty;
|
|||
|
|
++
|
|||
|
|
++import io.netty.channel.Channel;
|
|||
|
|
++import io.netty.channel.ChannelPromise;
|
|||
|
|
++import io.netty.util.concurrent.Future;
|
|||
|
|
++import io.netty.util.concurrent.GenericFutureListener;
|
|||
|
|
++import java.util.concurrent.ExecutionException;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
++import java.util.concurrent.TimeoutException;
|
|||
|
|
++import org.jetbrains.annotations.NotNull;
|
|||
|
|
++
|
|||
|
|
++public class MockChannelPromise implements ChannelPromise {
|
|||
|
|
++ protected Channel channel;
|
|||
|
|
++
|
|||
|
|
++ public MockChannelPromise(Channel channel) {
|
|||
|
|
++ this.channel = channel;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Channel channel() {
|
|||
|
|
++ return channel;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise setSuccess(Void result) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise setSuccess() {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean trySuccess() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise setFailure(Throwable cause) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise addListener(GenericFutureListener<? extends Future<? super Void>> listener) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise addListeners(GenericFutureListener<? extends Future<? super Void>>... listeners) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise removeListener(GenericFutureListener<? extends Future<? super Void>> listener) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise removeListeners(GenericFutureListener<? extends Future<? super Void>>... listeners) {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise sync() throws InterruptedException {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise syncUninterruptibly() {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise await() throws InterruptedException {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise awaitUninterruptibly() {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelPromise unvoid() {
|
|||
|
|
++ return this;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean isVoid() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean trySuccess(Void result) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean tryFailure(Throwable cause) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean setUncancellable() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean isSuccess() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean isCancellable() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Throwable cause() {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean await(long timeout, TimeUnit unit) throws InterruptedException {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean await(long timeoutMillis) throws InterruptedException {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean awaitUninterruptibly(long timeout, TimeUnit unit) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean awaitUninterruptibly(long timeoutMillis) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Void getNow() {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean cancel(boolean mayInterruptIfRunning) {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean isCancelled() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public boolean isDone() {
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Void get() throws InterruptedException, ExecutionException {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public Void get(long timeout,
|
|||
|
|
++ @NotNull java.util.concurrent.TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
|
|||
|
|
++ return null;
|
|||
|
|
++ }
|
|||
|
|
++}
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
|
|||
|
|
+index 8381c132b..dbbea86ea 100644
|
|||
|
|
+--- a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstractTest.java
|
|||
|
|
+@@ -39,9 +39,19 @@ public class NettyRemotingAbstractTest {
|
|||
|
|
+ final Semaphore semaphore = new Semaphore(0);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, 1, 3000, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void operationComplete(final ResponseFuture responseFuture) {
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ assertThat(semaphore.availablePermits()).isEqualTo(0);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
+ }, new SemaphoreReleaseOnlyOnce(semaphore));
|
|||
|
|
+
|
|||
|
|
+ remotingAbstract.responseTable.putIfAbsent(1, responseFuture);
|
|||
|
|
+@@ -75,9 +85,19 @@ public class NettyRemotingAbstractTest {
|
|||
|
|
+ final Semaphore semaphore = new Semaphore(0);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, 1, 3000, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void operationComplete(final ResponseFuture responseFuture) {
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
+ assertThat(semaphore.availablePermits()).isEqualTo(0);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
+ }, new SemaphoreReleaseOnlyOnce(semaphore));
|
|||
|
|
+
|
|||
|
|
+ remotingAbstract.responseTable.putIfAbsent(1, responseFuture);
|
|||
|
|
+@@ -98,7 +118,18 @@ public class NettyRemotingAbstractTest {
|
|||
|
|
+ // mock timeout
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, dummyId, -1000, new InvokeCallback() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void operationComplete(final ResponseFuture responseFuture) {
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
+ }
|
|||
|
|
+ }, null);
|
|||
|
|
+ remotingAbstract.responseTable.putIfAbsent(dummyId, responseFuture);
|
|||
|
|
+@@ -111,7 +142,22 @@ public class NettyRemotingAbstractTest {
|
|||
|
|
+ final Semaphore semaphore = new Semaphore(0);
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(1, null);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, 1, request, 3000,
|
|||
|
|
+- responseFuture1 -> assertThat(semaphore.availablePermits()).isEqualTo(0), new SemaphoreReleaseOnlyOnce(semaphore));
|
|||
|
|
++ new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationSucceed(RemotingCommand response) {
|
|||
|
|
++ assertThat(semaphore.availablePermits()).isEqualTo(0);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationFail(Throwable throwable) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++ }, new SemaphoreReleaseOnlyOnce(semaphore));
|
|||
|
|
+
|
|||
|
|
+ remotingAbstract.responseTable.putIfAbsent(1, responseFuture);
|
|||
|
|
+ RemotingCommand response = RemotingCommand.createResponseCommand(0, "Foo");
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
+index 8fabbb21d..e72e7bd53 100644
|
|||
|
|
+--- a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
+@@ -16,10 +16,17 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.remoting.netty;
|
|||
|
|
+
|
|||
|
|
++import io.netty.channel.Channel;
|
|||
|
|
++import io.netty.channel.ChannelFuture;
|
|||
|
|
++import io.netty.channel.local.LocalChannel;
|
|||
|
|
+ import java.util.concurrent.CompletableFuture;
|
|||
|
|
++import java.util.concurrent.ExecutionException;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+ import java.util.concurrent.Executors;
|
|||
|
|
++import java.util.concurrent.Semaphore;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
++import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
++import org.apache.rocketmq.remoting.common.SemaphoreReleaseOnlyOnce;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingConnectException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
|
|||
|
|
+@@ -29,23 +36,33 @@ import org.apache.rocketmq.remoting.protocol.RequestCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+ import org.junit.Test;
|
|||
|
|
+ import org.junit.runner.RunWith;
|
|||
|
|
++import org.mockito.Mock;
|
|||
|
|
+ import org.mockito.Spy;
|
|||
|
|
+ import org.mockito.junit.MockitoJUnitRunner;
|
|||
|
|
+
|
|||
|
|
+ import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
++import static org.assertj.core.api.Assertions.assertThatThrownBy;
|
|||
|
|
+ import static org.assertj.core.api.AssertionsForClassTypes.catchThrowable;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
++import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
+ import static org.mockito.Mockito.doAnswer;
|
|||
|
|
++import static org.mockito.Mockito.doReturn;
|
|||
|
|
++import static org.mockito.Mockito.mock;
|
|||
|
|
++import static org.mockito.Mockito.never;
|
|||
|
|
++import static org.mockito.Mockito.times;
|
|||
|
|
++import static org.mockito.Mockito.verify;
|
|||
|
|
+
|
|||
|
|
+ @RunWith(MockitoJUnitRunner.class)
|
|||
|
|
+ public class NettyRemotingClientTest {
|
|||
|
|
+ @Spy
|
|||
|
|
+ private NettyRemotingClient remotingClient = new NettyRemotingClient(new NettyClientConfig());
|
|||
|
|
++ @Mock
|
|||
|
|
++ private RPCHook rpcHookMock;
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+- public void testSetCallbackExecutor() throws NoSuchFieldException, IllegalAccessException {
|
|||
|
|
++ public void testSetCallbackExecutor() {
|
|||
|
|
+ ExecutorService customized = Executors.newCachedThreadPool();
|
|||
|
|
+ remotingClient.setCallbackExecutor(customized);
|
|||
|
|
+ assertThat(remotingClient.getCallbackExecutor()).isEqualTo(customized);
|
|||
|
|
+@@ -61,7 +78,7 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+ ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+ responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+
|
|||
|
|
+@@ -78,9 +95,7 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ doAnswer(invocation -> {
|
|||
|
|
+ InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+- responseFuture.setSendRequestOK(false);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationFail(new RemotingSendRequestException(null));
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+
|
|||
|
|
+@@ -97,8 +112,7 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ doAnswer(invocation -> {
|
|||
|
|
+ InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), -1L, null, null);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationFail(new RemotingTimeoutException(""));
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+
|
|||
|
|
+@@ -115,8 +129,7 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+ doAnswer(invocation -> {
|
|||
|
|
+ InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+- callback.operationComplete(responseFuture);
|
|||
|
|
++ callback.operationFail(new RemotingException(null));
|
|||
|
|
+ return null;
|
|||
|
|
+ }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
+
|
|||
|
|
+@@ -134,4 +147,158 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ assertThat(e.getMessage()).contains(addr);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvoke0() throws ExecutionException, InterruptedException {
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ Channel channel = new MockChannel() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelFuture writeAndFlush(Object msg) {
|
|||
|
|
++ ResponseFuture responseFuture = remotingClient.responseTable.get(request.getOpaque());
|
|||
|
|
++ responseFuture.setResponseCommand(response);
|
|||
|
|
++ responseFuture.executeInvokeCallback();
|
|||
|
|
++ return super.writeAndFlush(msg);
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = remotingClient.invoke0(channel, request, 1000L);
|
|||
|
|
++ assertThat(future.get().getResponseCommand()).isEqualTo(response);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvoke0WithException() {
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ Channel channel = new MockChannel() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public ChannelFuture writeAndFlush(Object msg) {
|
|||
|
|
++ ResponseFuture responseFuture = remotingClient.responseTable.get(request.getOpaque());
|
|||
|
|
++ responseFuture.executeInvokeCallback();
|
|||
|
|
++ return super.writeAndFlush(msg);
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = remotingClient.invoke0(channel, request, 1000L);
|
|||
|
|
++ assertThatThrownBy(future::get).getCause().isInstanceOf(RemotingException.class);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvokeSync() throws RemotingSendRequestException, RemotingTimeoutException, InterruptedException {
|
|||
|
|
++ remotingClient.registerRPCHook(rpcHookMock);
|
|||
|
|
++
|
|||
|
|
++ Channel channel = new LocalChannel();
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ ResponseFuture responseFuture = new ResponseFuture(channel, request.getOpaque(), request, 1000, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++ }, new SemaphoreReleaseOnlyOnce(new Semaphore(1)));
|
|||
|
|
++ responseFuture.setResponseCommand(response);
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
++ future.complete(responseFuture);
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke0(any(Channel.class), any(RemotingCommand.class), anyLong());
|
|||
|
|
++ RemotingCommand actual = remotingClient.invokeSyncImpl(channel, request, 1000);
|
|||
|
|
++ assertThat(actual).isEqualTo(response);
|
|||
|
|
++
|
|||
|
|
++ verify(rpcHookMock).doBeforeRequest(anyString(), eq(request));
|
|||
|
|
++ verify(rpcHookMock).doAfterResponse(anyString(), eq(request), eq(response));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvokeAsync() {
|
|||
|
|
++ remotingClient.registerRPCHook(rpcHookMock);
|
|||
|
|
++ Channel channel = new LocalChannel();
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ ResponseFuture responseFuture = new ResponseFuture(channel, request.getOpaque(), request, 1000, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++ }, new SemaphoreReleaseOnlyOnce(new Semaphore(1)));
|
|||
|
|
++ responseFuture.setResponseCommand(response);
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
++ future.complete(responseFuture);
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke0(any(Channel.class), any(RemotingCommand.class), anyLong());
|
|||
|
|
++
|
|||
|
|
++ InvokeCallback callback = mock(InvokeCallback.class);
|
|||
|
|
++ remotingClient.invokeAsyncImpl(channel, request, 1000, callback);
|
|||
|
|
++ verify(callback, times(1)).operationSucceed(eq(response));
|
|||
|
|
++ verify(callback, times(1)).operationComplete(eq(responseFuture));
|
|||
|
|
++ verify(callback, never()).operationFail(any());
|
|||
|
|
++
|
|||
|
|
++ verify(rpcHookMock).doBeforeRequest(anyString(), eq(request));
|
|||
|
|
++ verify(rpcHookMock).doAfterResponse(anyString(), eq(request), eq(response));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvokeAsyncFail() {
|
|||
|
|
++ remotingClient.registerRPCHook(rpcHookMock);
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++
|
|||
|
|
++ Channel channel = new LocalChannel();
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
++ future.completeExceptionally(new RemotingException(null));
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke0(any(Channel.class), any(RemotingCommand.class), anyLong());
|
|||
|
|
++
|
|||
|
|
++ InvokeCallback callback = mock(InvokeCallback.class);
|
|||
|
|
++ remotingClient.invokeAsyncImpl(channel, request, 1000, callback);
|
|||
|
|
++ verify(callback, never()).operationSucceed(any());
|
|||
|
|
++ verify(callback, times(1)).operationComplete(any());
|
|||
|
|
++ verify(callback, times(1)).operationFail(any());
|
|||
|
|
++
|
|||
|
|
++ verify(rpcHookMock).doBeforeRequest(anyString(), eq(request));
|
|||
|
|
++ verify(rpcHookMock, never()).doAfterResponse(anyString(), eq(request), any());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvokeImpl() throws ExecutionException, InterruptedException {
|
|||
|
|
++ remotingClient.registerRPCHook(rpcHookMock);
|
|||
|
|
++ Channel channel = new LocalChannel();
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
++ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
++ ResponseFuture responseFuture = new ResponseFuture(channel, request.getOpaque(), request, 1000, new InvokeCallback() {
|
|||
|
|
++ @Override
|
|||
|
|
++ public void operationComplete(ResponseFuture responseFuture) {
|
|||
|
|
++
|
|||
|
|
++ }
|
|||
|
|
++ }, new SemaphoreReleaseOnlyOnce(new Semaphore(1)));
|
|||
|
|
++ responseFuture.setResponseCommand(response);
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
++ future.complete(responseFuture);
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke0(any(Channel.class), any(RemotingCommand.class), anyLong());
|
|||
|
|
++
|
|||
|
|
++ CompletableFuture<ResponseFuture> future0 = remotingClient.invokeImpl(channel, request, 1000);
|
|||
|
|
++ assertThat(future0.get()).isEqualTo(responseFuture);
|
|||
|
|
++
|
|||
|
|
++ verify(rpcHookMock).doBeforeRequest(anyString(), eq(request));
|
|||
|
|
++ verify(rpcHookMock).doAfterResponse(anyString(), eq(request), eq(response));
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testInvokeImplFail() {
|
|||
|
|
++ remotingClient.registerRPCHook(rpcHookMock);
|
|||
|
|
++ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
++
|
|||
|
|
++ Channel channel = new LocalChannel();
|
|||
|
|
++ CompletableFuture<ResponseFuture> future = new CompletableFuture<>();
|
|||
|
|
++ future.completeExceptionally(new RemotingException(null));
|
|||
|
|
++
|
|||
|
|
++ doReturn(future).when(remotingClient).invoke0(any(Channel.class), any(RemotingCommand.class), anyLong());
|
|||
|
|
++
|
|||
|
|
++ assertThatThrownBy(() -> remotingClient.invokeImpl(channel, request, 1000).get()).getCause().isInstanceOf(RemotingException.class);
|
|||
|
|
++
|
|||
|
|
++ verify(rpcHookMock).doBeforeRequest(anyString(), eq(request));
|
|||
|
|
++ verify(rpcHookMock, never()).doAfterResponse(anyString(), eq(request), any());
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From b9ffe0f9576f68b8a37cf3e2f68051658ae5a9a2 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Sun, 8 Oct 2023 16:33:44 +0800
|
|||
|
|
+Subject: [PATCH 2/7] [ISSUE #7296] Add ChannelEventListener for
|
|||
|
|
+ MQClientAPIImpl (#7324)
|
|||
|
|
+
|
|||
|
|
+* Add ChannelEventListener for MQClientAPIImpl
|
|||
|
|
+
|
|||
|
|
+* add heartbeat when channel connect
|
|||
|
|
+
|
|||
|
|
+* remove log
|
|||
|
|
+
|
|||
|
|
+* Add enableHeartbeatChannelEventListener for ClientConfig
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/client/ClientConfig.java | 55 ++++++++++++++-----
|
|||
|
|
+ .../rocketmq/client/impl/MQClientAPIImpl.java | 9 ++-
|
|||
|
|
+ .../client/impl/factory/MQClientInstance.java | 35 +++++++++++-
|
|||
|
|
+ .../remoting/netty/NettyRemotingClient.java | 2 +
|
|||
|
|
+ 4 files changed, 85 insertions(+), 16 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
+index bb0fe3522..f9843cc02 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/ClientConfig.java
|
|||
|
|
+@@ -94,6 +94,8 @@ public class ClientConfig {
|
|||
|
|
+ private boolean sendLatencyEnable = Boolean.parseBoolean(System.getProperty(SEND_LATENCY_ENABLE, "false"));
|
|||
|
|
+ private boolean startDetectorEnable = Boolean.parseBoolean(System.getProperty(START_DETECTOR_ENABLE, "false"));
|
|||
|
|
+
|
|||
|
|
++ private boolean enableHeartbeatChannelEventListener = true;
|
|||
|
|
++
|
|||
|
|
+ public String buildMQClientId() {
|
|||
|
|
+ StringBuilder sb = new StringBuilder();
|
|||
|
|
+ sb.append(this.getClientIP());
|
|||
|
|
+@@ -201,6 +203,7 @@ public class ClientConfig {
|
|||
|
|
+ this.useHeartbeatV2 = cc.useHeartbeatV2;
|
|||
|
|
+ this.startDetectorEnable = cc.startDetectorEnable;
|
|||
|
|
+ this.sendLatencyEnable = cc.sendLatencyEnable;
|
|||
|
|
++ this.enableHeartbeatChannelEventListener = cc.enableHeartbeatChannelEventListener;
|
|||
|
|
+ this.detectInterval = cc.detectInterval;
|
|||
|
|
+ this.detectTimeout = cc.detectTimeout;
|
|||
|
|
+ }
|
|||
|
|
+@@ -228,6 +231,7 @@ public class ClientConfig {
|
|||
|
|
+ cc.enableStreamRequestType = enableStreamRequestType;
|
|||
|
|
+ cc.useHeartbeatV2 = useHeartbeatV2;
|
|||
|
|
+ cc.startDetectorEnable = startDetectorEnable;
|
|||
|
|
++ cc.enableHeartbeatChannelEventListener = enableHeartbeatChannelEventListener;
|
|||
|
|
+ cc.sendLatencyEnable = sendLatencyEnable;
|
|||
|
|
+ cc.detectInterval = detectInterval;
|
|||
|
|
+ cc.detectTimeout = detectTimeout;
|
|||
|
|
+@@ -418,6 +422,14 @@ public class ClientConfig {
|
|||
|
|
+ this.startDetectorEnable = startDetectorEnable;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public boolean isEnableHeartbeatChannelEventListener() {
|
|||
|
|
++ return enableHeartbeatChannelEventListener;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableHeartbeatChannelEventListener(boolean enableHeartbeatChannelEventListener) {
|
|||
|
|
++ this.enableHeartbeatChannelEventListener = enableHeartbeatChannelEventListener;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public int getDetectTimeout() {
|
|||
|
|
+ return this.detectTimeout;
|
|||
|
|
+ }
|
|||
|
|
+@@ -444,19 +456,34 @@ public class ClientConfig {
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String toString() {
|
|||
|
|
+- return "ClientConfig [namesrvAddr=" + namesrvAddr
|
|||
|
|
+- + ", clientIP=" + clientIP + ", instanceName=" + instanceName
|
|||
|
|
+- + ", clientCallbackExecutorThreads=" + clientCallbackExecutorThreads
|
|||
|
|
+- + ", pollNameServerInterval=" + pollNameServerInterval
|
|||
|
|
+- + ", heartbeatBrokerInterval=" + heartbeatBrokerInterval
|
|||
|
|
+- + ", persistConsumerOffsetInterval=" + persistConsumerOffsetInterval
|
|||
|
|
+- + ", pullTimeDelayMillsWhenException=" + pullTimeDelayMillsWhenException
|
|||
|
|
+- + ", unitMode=" + unitMode + ", unitName=" + unitName
|
|||
|
|
+- + ", vipChannelEnabled=" + vipChannelEnabled + ", useTLS=" + useTLS
|
|||
|
|
+- + ", socksProxyConfig=" + socksProxyConfig + ", language=" + language.name()
|
|||
|
|
+- + ", namespace=" + namespace + ", mqClientApiTimeout=" + mqClientApiTimeout
|
|||
|
|
+- + ", decodeReadBody=" + decodeReadBody + ", decodeDecompressBody=" + decodeDecompressBody
|
|||
|
|
+- + ", sendLatencyEnable=" + sendLatencyEnable + ", startDetectorEnable=" + startDetectorEnable
|
|||
|
|
+- + ", enableStreamRequestType=" + enableStreamRequestType + ", useHeartbeatV2=" + useHeartbeatV2 + "]";
|
|||
|
|
++ return "ClientConfig{" +
|
|||
|
|
++ "namesrvAddr='" + namesrvAddr + '\'' +
|
|||
|
|
++ ", clientIP='" + clientIP + '\'' +
|
|||
|
|
++ ", instanceName='" + instanceName + '\'' +
|
|||
|
|
++ ", clientCallbackExecutorThreads=" + clientCallbackExecutorThreads +
|
|||
|
|
++ ", namespace='" + namespace + '\'' +
|
|||
|
|
++ ", namespaceInitialized=" + namespaceInitialized +
|
|||
|
|
++ ", accessChannel=" + accessChannel +
|
|||
|
|
++ ", pollNameServerInterval=" + pollNameServerInterval +
|
|||
|
|
++ ", heartbeatBrokerInterval=" + heartbeatBrokerInterval +
|
|||
|
|
++ ", persistConsumerOffsetInterval=" + persistConsumerOffsetInterval +
|
|||
|
|
++ ", pullTimeDelayMillsWhenException=" + pullTimeDelayMillsWhenException +
|
|||
|
|
++ ", unitMode=" + unitMode +
|
|||
|
|
++ ", unitName='" + unitName + '\'' +
|
|||
|
|
++ ", decodeReadBody=" + decodeReadBody +
|
|||
|
|
++ ", decodeDecompressBody=" + decodeDecompressBody +
|
|||
|
|
++ ", vipChannelEnabled=" + vipChannelEnabled +
|
|||
|
|
++ ", useHeartbeatV2=" + useHeartbeatV2 +
|
|||
|
|
++ ", useTLS=" + useTLS +
|
|||
|
|
++ ", socksProxyConfig='" + socksProxyConfig + '\'' +
|
|||
|
|
++ ", mqClientApiTimeout=" + mqClientApiTimeout +
|
|||
|
|
++ ", detectTimeout=" + detectTimeout +
|
|||
|
|
++ ", detectInterval=" + detectInterval +
|
|||
|
|
++ ", language=" + language +
|
|||
|
|
++ ", enableStreamRequestType=" + enableStreamRequestType +
|
|||
|
|
++ ", sendLatencyEnable=" + sendLatencyEnable +
|
|||
|
|
++ ", startDetectorEnable=" + startDetectorEnable +
|
|||
|
|
++ ", enableHeartbeatChannelEventListener=" + enableHeartbeatChannelEventListener +
|
|||
|
|
++ '}';
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+index 2407e5737..e152be811 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java
|
|||
|
|
+@@ -79,6 +79,7 @@ import org.apache.rocketmq.common.sysflag.PullSysFlag;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+ import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
++import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
+ import org.apache.rocketmq.remoting.CommandCustomHeader;
|
|||
|
|
+ import org.apache.rocketmq.remoting.InvokeCallback;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+@@ -246,10 +247,16 @@ public class MQClientAPIImpl implements NameServerUpdateCallback {
|
|||
|
|
+ public MQClientAPIImpl(final NettyClientConfig nettyClientConfig,
|
|||
|
|
+ final ClientRemotingProcessor clientRemotingProcessor,
|
|||
|
|
+ RPCHook rpcHook, final ClientConfig clientConfig) {
|
|||
|
|
++ this(nettyClientConfig, clientRemotingProcessor, rpcHook, clientConfig, null);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public MQClientAPIImpl(final NettyClientConfig nettyClientConfig,
|
|||
|
|
++ final ClientRemotingProcessor clientRemotingProcessor,
|
|||
|
|
++ RPCHook rpcHook, final ClientConfig clientConfig, final ChannelEventListener channelEventListener) {
|
|||
|
|
+ this.clientConfig = clientConfig;
|
|||
|
|
+ topAddressing = new DefaultTopAddressing(MixAll.getWSAddr(), clientConfig.getUnitName());
|
|||
|
|
+ topAddressing.registerChangeCallBack(this);
|
|||
|
|
+- this.remotingClient = new NettyRemotingClient(nettyClientConfig, null);
|
|||
|
|
++ this.remotingClient = new NettyRemotingClient(nettyClientConfig, channelEventListener);
|
|||
|
|
+ this.clientRemotingProcessor = clientRemotingProcessor;
|
|||
|
|
+
|
|||
|
|
+ // Inject stream rpc hook first to make reserve field signature
|
|||
|
|
+diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
+index 9484b26f8..09534a176 100644
|
|||
|
|
+--- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
++++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java
|
|||
|
|
+@@ -16,6 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.client.impl.factory;
|
|||
|
|
+
|
|||
|
|
++import io.netty.channel.Channel;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+@@ -65,6 +66,7 @@ import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueue;
|
|||
|
|
+ import org.apache.rocketmq.common.message.MessageQueueAssignment;
|
|||
|
|
+ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
++import org.apache.rocketmq.remoting.ChannelEventListener;
|
|||
|
|
+ import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+ import org.apache.rocketmq.remoting.common.HeartbeatV2Result;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingException;
|
|||
|
|
+@@ -151,7 +153,38 @@ public class MQClientInstance {
|
|||
|
|
+ this.nettyClientConfig.setUseTLS(clientConfig.isUseTLS());
|
|||
|
|
+ this.nettyClientConfig.setSocksProxyConfig(clientConfig.getSocksProxyConfig());
|
|||
|
|
+ ClientRemotingProcessor clientRemotingProcessor = new ClientRemotingProcessor(this);
|
|||
|
|
+- this.mQClientAPIImpl = new MQClientAPIImpl(this.nettyClientConfig, clientRemotingProcessor, rpcHook, clientConfig);
|
|||
|
|
++ ChannelEventListener channelEventListener;
|
|||
|
|
++ if (clientConfig.isEnableHeartbeatChannelEventListener()) {
|
|||
|
|
++ channelEventListener = new ChannelEventListener() {
|
|||
|
|
++ private final ConcurrentMap<String, HashMap<Long, String>> brokerAddrTable = MQClientInstance.this.brokerAddrTable;
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onChannelConnect(String remoteAddr, Channel channel) {
|
|||
|
|
++ for (Map.Entry<String, HashMap<Long, String>> addressEntry : brokerAddrTable.entrySet()) {
|
|||
|
|
++ for (String address : addressEntry.getValue().values()) {
|
|||
|
|
++ if (address.equals(remoteAddr)) {
|
|||
|
|
++ sendHeartbeatToAllBrokerWithLockV2(false);
|
|||
|
|
++ break;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onChannelClose(String remoteAddr, Channel channel) {
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onChannelException(String remoteAddr, Channel channel) {
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public void onChannelIdle(String remoteAddr, Channel channel) {
|
|||
|
|
++ }
|
|||
|
|
++ };
|
|||
|
|
++ } else {
|
|||
|
|
++ channelEventListener = null;
|
|||
|
|
++ }
|
|||
|
|
++ this.mQClientAPIImpl = new MQClientAPIImpl(this.nettyClientConfig, clientRemotingProcessor, rpcHook, clientConfig, channelEventListener);
|
|||
|
|
+
|
|||
|
|
+ if (this.clientConfig.getNamesrvAddr() != null) {
|
|||
|
|
+ this.mQClientAPIImpl.updateNameServerAddressList(this.clientConfig.getNamesrvAddr());
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+index d784351a5..8631d0447 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+@@ -229,6 +229,8 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ handler.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ nettyEventExecutor.start();
|
|||
|
|
++
|
|||
|
|
+ TimerTask timerTaskScanResponseTable = new TimerTask() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void run(Timeout timeout) {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 3808387e1389278edbe4ef023d200ecb3015622b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: lk <xdkxlk@outlook.com>
|
|||
|
|
+Date: Mon, 9 Oct 2023 16:07:56 +0800
|
|||
|
|
+Subject: [PATCH 3/7] [ISSUE #7429] clean channel map when CLIENT_UNREGISTER in
|
|||
|
|
+ proxy
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../service/sysmessage/HeartbeatSyncer.java | 31 ++++++---
|
|||
|
|
+ .../sysmessage/HeartbeatSyncerTest.java | 68 +++++++++++++++++++
|
|||
|
|
+ 2 files changed, 88 insertions(+), 11 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java
|
|||
|
|
+index f70c06b8f..fee3ea87d 100644
|
|||
|
|
+--- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java
|
|||
|
|
++++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java
|
|||
|
|
+@@ -18,6 +18,7 @@
|
|||
|
|
+ package org.apache.rocketmq.proxy.service.sysmessage;
|
|||
|
|
+
|
|||
|
|
+ import com.alibaba.fastjson.JSON;
|
|||
|
|
++import io.netty.channel.Channel;
|
|||
|
|
+ import java.nio.charset.StandardCharsets;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+ import java.util.Map;
|
|||
|
|
+@@ -73,16 +74,8 @@ public class HeartbeatSyncer extends AbstractSystemMessageSyncer {
|
|||
|
|
+ );
|
|||
|
|
+ this.consumerManager.appendConsumerIdsChangeListener(new ConsumerIdsChangeListener() {
|
|||
|
|
+ @Override
|
|||
|
|
+- public void handle(ConsumerGroupEvent event, String s, Object... args) {
|
|||
|
|
+- if (event == ConsumerGroupEvent.CLIENT_UNREGISTER) {
|
|||
|
|
+- if (args == null || args.length < 1) {
|
|||
|
|
+- return;
|
|||
|
|
+- }
|
|||
|
|
+- if (args[0] instanceof ClientChannelInfo) {
|
|||
|
|
+- ClientChannelInfo clientChannelInfo = (ClientChannelInfo) args[0];
|
|||
|
|
+- remoteChannelMap.remove(clientChannelInfo.getChannel().id().asLongText());
|
|||
|
|
+- }
|
|||
|
|
+- }
|
|||
|
|
++ public void handle(ConsumerGroupEvent event, String group, Object... args) {
|
|||
|
|
++ processConsumerGroupEvent(event, group, args);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+@@ -98,6 +91,18 @@ public class HeartbeatSyncer extends AbstractSystemMessageSyncer {
|
|||
|
|
+ super.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ protected void processConsumerGroupEvent(ConsumerGroupEvent event, String group, Object... args) {
|
|||
|
|
++ if (event == ConsumerGroupEvent.CLIENT_UNREGISTER) {
|
|||
|
|
++ if (args == null || args.length < 1) {
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ if (args[0] instanceof ClientChannelInfo) {
|
|||
|
|
++ ClientChannelInfo clientChannelInfo = (ClientChannelInfo) args[0];
|
|||
|
|
++ remoteChannelMap.remove(buildKey(group, clientChannelInfo.getChannel()));
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public void onConsumerRegister(String consumerGroup, ClientChannelInfo clientChannelInfo,
|
|||
|
|
+ ConsumeType consumeType, MessageModel messageModel, ConsumeFromWhere consumeFromWhere,
|
|||
|
|
+ Set<SubscriptionData> subList) {
|
|||
|
|
+@@ -189,7 +194,7 @@ public class HeartbeatSyncer extends AbstractSystemMessageSyncer {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ RemoteChannel decodedChannel = RemoteChannel.decode(data.getChannelData());
|
|||
|
|
+- RemoteChannel channel = remoteChannelMap.computeIfAbsent(data.getGroup() + "@" + decodedChannel.id().asLongText(), key -> decodedChannel);
|
|||
|
|
++ RemoteChannel channel = remoteChannelMap.computeIfAbsent(buildKey(data.getGroup(), decodedChannel), key -> decodedChannel);
|
|||
|
|
+ channel.setExtendAttribute(decodedChannel.getChannelExtendAttribute());
|
|||
|
|
+ ClientChannelInfo clientChannelInfo = new ClientChannelInfo(
|
|||
|
|
+ channel,
|
|||
|
|
+@@ -228,4 +233,8 @@ public class HeartbeatSyncer extends AbstractSystemMessageSyncer {
|
|||
|
|
+ // use local address, remoting port and grpc port to build unique local proxy Id
|
|||
|
|
+ return proxyConfig.getLocalServeAddr() + "%" + proxyConfig.getRemotingListenPort() + "%" + proxyConfig.getGrpcServerPort();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ private static String buildKey(String group, Channel channel) {
|
|||
|
|
++ return group + "@" + channel.id().asLongText();
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
+index 43fba3d03..9a2c5e343 100644
|
|||
|
|
+--- a/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
++++ b/proxy/src/test/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncerTest.java
|
|||
|
|
+@@ -27,6 +27,7 @@ import com.google.common.collect.Sets;
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+ import io.netty.channel.ChannelId;
|
|||
|
|
+ import java.time.Duration;
|
|||
|
|
++import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+ import java.util.HashSet;
|
|||
|
|
+ import java.util.List;
|
|||
|
|
+@@ -35,6 +36,7 @@ import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.stream.Collectors;
|
|||
|
|
+ import org.apache.commons.lang3.RandomStringUtils;
|
|||
|
|
+ import org.apache.rocketmq.broker.client.ClientChannelInfo;
|
|||
|
|
++import org.apache.rocketmq.broker.client.ConsumerGroupEvent;
|
|||
|
|
+ import org.apache.rocketmq.broker.client.ConsumerManager;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.mqclient.MQClientAPIExt;
|
|||
|
|
+ import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory;
|
|||
|
|
+@@ -320,6 +322,72 @@ public class HeartbeatSyncerTest extends InitConfigTest {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testProcessConsumerGroupEventForRemoting() {
|
|||
|
|
++ String consumerGroup = "consumerGroup";
|
|||
|
|
++ Channel channel = createMockChannel();
|
|||
|
|
++ RemotingProxyOutClient remotingProxyOutClient = mock(RemotingProxyOutClient.class);
|
|||
|
|
++ RemotingChannel remotingChannel = new RemotingChannel(remotingProxyOutClient, proxyRelayService, channel, clientId, Collections.emptySet());
|
|||
|
|
++ ClientChannelInfo clientChannelInfo = new ClientChannelInfo(
|
|||
|
|
++ remotingChannel,
|
|||
|
|
++ clientId,
|
|||
|
|
++ LanguageCode.JAVA,
|
|||
|
|
++ 4
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
++ testProcessConsumerGroupEvent(consumerGroup, clientChannelInfo);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Test
|
|||
|
|
++ public void testProcessConsumerGroupEventForGrpcV2() {
|
|||
|
|
++ String consumerGroup = "consumerGroup";
|
|||
|
|
++ GrpcClientSettingsManager grpcClientSettingsManager = mock(GrpcClientSettingsManager.class);
|
|||
|
|
++ GrpcChannelManager grpcChannelManager = mock(GrpcChannelManager.class);
|
|||
|
|
++ GrpcClientChannel grpcClientChannel = new GrpcClientChannel(
|
|||
|
|
++ proxyRelayService, grpcClientSettingsManager, grpcChannelManager,
|
|||
|
|
++ ProxyContext.create().setRemoteAddress(remoteAddress).setLocalAddress(localAddress),
|
|||
|
|
++ clientId);
|
|||
|
|
++ ClientChannelInfo clientChannelInfo = new ClientChannelInfo(
|
|||
|
|
++ grpcClientChannel,
|
|||
|
|
++ clientId,
|
|||
|
|
++ LanguageCode.JAVA,
|
|||
|
|
++ 5
|
|||
|
|
++ );
|
|||
|
|
++
|
|||
|
|
++ testProcessConsumerGroupEvent(consumerGroup, clientChannelInfo);
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ private void testProcessConsumerGroupEvent(String consumerGroup, ClientChannelInfo clientChannelInfo) {
|
|||
|
|
++ HeartbeatSyncer heartbeatSyncer = new HeartbeatSyncer(topicRouteService, adminService, consumerManager, mqClientAPIFactory, null);
|
|||
|
|
++ SendResult okSendResult = new SendResult();
|
|||
|
|
++ okSendResult.setSendStatus(SendStatus.SEND_OK);
|
|||
|
|
++
|
|||
|
|
++ ArgumentCaptor<Message> messageArgumentCaptor = ArgumentCaptor.forClass(Message.class);
|
|||
|
|
++ doReturn(CompletableFuture.completedFuture(okSendResult)).when(this.mqClientAPIExt)
|
|||
|
|
++ .sendMessageAsync(anyString(), anyString(), messageArgumentCaptor.capture(), any(), anyLong());
|
|||
|
|
++
|
|||
|
|
++ heartbeatSyncer.onConsumerRegister(
|
|||
|
|
++ consumerGroup,
|
|||
|
|
++ clientChannelInfo,
|
|||
|
|
++ ConsumeType.CONSUME_PASSIVELY,
|
|||
|
|
++ MessageModel.CLUSTERING,
|
|||
|
|
++ ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET,
|
|||
|
|
++ Collections.emptySet()
|
|||
|
|
++ );
|
|||
|
|
++ await().atMost(Duration.ofSeconds(3)).until(() -> messageArgumentCaptor.getAllValues().size() == 1);
|
|||
|
|
++
|
|||
|
|
++ // change local serve addr, to simulate other proxy receive messages
|
|||
|
|
++ heartbeatSyncer.localProxyId = RandomStringUtils.randomAlphabetic(10);
|
|||
|
|
++ ArgumentCaptor<ClientChannelInfo> channelInfoArgumentCaptor = ArgumentCaptor.forClass(ClientChannelInfo.class);
|
|||
|
|
++ doReturn(true).when(consumerManager).registerConsumer(anyString(), channelInfoArgumentCaptor.capture(), any(), any(), any(), any(), anyBoolean());
|
|||
|
|
++
|
|||
|
|
++ heartbeatSyncer.consumeMessage(convertFromMessage(messageArgumentCaptor.getAllValues()), null);
|
|||
|
|
++ assertEquals(1, heartbeatSyncer.remoteChannelMap.size());
|
|||
|
|
++
|
|||
|
|
++ heartbeatSyncer.processConsumerGroupEvent(ConsumerGroupEvent.CLIENT_UNREGISTER, consumerGroup, channelInfoArgumentCaptor.getValue());
|
|||
|
|
++ assertTrue(heartbeatSyncer.remoteChannelMap.isEmpty());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ private MessageExt convertFromMessage(Message message) {
|
|||
|
|
+ MessageExt messageExt = new MessageExt();
|
|||
|
|
+ messageExt.setTopic(message.getTopic());
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 0027a1486d4f2d6f7dce3010751167e883783945 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: redlsz <szliu0927@gmail.com>
|
|||
|
|
+Date: Mon, 9 Oct 2023 16:52:10 +0800
|
|||
|
|
+Subject: [PATCH 4/7] [ISSUE #7412] Fix pop revive message error when reput
|
|||
|
|
+ checkpoint
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../org/apache/rocketmq/broker/processor/PopReviveService.java | 1 +
|
|||
|
|
+ 1 file changed, 1 insertion(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
+index 93167db37..d5174d3d1 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java
|
|||
|
|
+@@ -595,6 +595,7 @@ public class PopReviveService extends ServiceThread {
|
|||
|
|
+ newCk.setCId(oldCK.getCId());
|
|||
|
|
+ newCk.setTopic(oldCK.getTopic());
|
|||
|
|
+ newCk.setQueueId(oldCK.getQueueId());
|
|||
|
|
++ newCk.setBrokerName(oldCK.getBrokerName());
|
|||
|
|
+ newCk.addDiff(0);
|
|||
|
|
+ MessageExtBrokerInner ckMsg = brokerController.getPopMessageProcessor().buildCkMsg(newCk, queueId);
|
|||
|
|
+ brokerController.getMessageStore().putMessage(ckMsg);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From b18e564addbcff50165a5e1d9d4ab7db789d901b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Mon, 9 Oct 2023 21:43:01 +0800
|
|||
|
|
+Subject: [PATCH 5/7] [ISSUE #7431] Fix flaky test of
|
|||
|
|
+ DLedgerControllerTest#testBrokerLifecycleListener (#7432)
|
|||
|
|
+
|
|||
|
|
+* Fix flaky test of DLedgerControllerTest#testBrokerLifecycleListener
|
|||
|
|
+---
|
|||
|
|
+ .../impl/DLedgerControllerTest.java | 26 ++++++++++++-------
|
|||
|
|
+ 1 file changed, 17 insertions(+), 9 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/DLedgerControllerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/DLedgerControllerTest.java
|
|||
|
|
+index 595a5cb65..d6e5449c5 100644
|
|||
|
|
+--- a/controller/src/test/java/org/apache/rocketmq/controller/impl/DLedgerControllerTest.java
|
|||
|
|
++++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/DLedgerControllerTest.java
|
|||
|
|
+@@ -63,7 +63,8 @@ public class DLedgerControllerTest {
|
|||
|
|
+ private List<String> baseDirs;
|
|||
|
|
+ private List<DLedgerController> controllers;
|
|||
|
|
+
|
|||
|
|
+- public DLedgerController launchController(final String group, final String peers, final String selfId, final boolean isEnableElectUncleanMaster) {
|
|||
|
|
++ public DLedgerController launchController(final String group, final String peers, final String selfId,
|
|||
|
|
++ final boolean isEnableElectUncleanMaster) {
|
|||
|
|
+ String tmpdir = System.getProperty("java.io.tmpdir");
|
|||
|
|
+ final String path = (StringUtils.endsWith(tmpdir, File.separator) ? tmpdir : tmpdir + File.separator) + group + File.separator + selfId;
|
|||
|
|
+ baseDirs.add(path);
|
|||
|
|
+@@ -121,11 +122,11 @@ public class DLedgerControllerTest {
|
|||
|
|
+ final RegisterBrokerToControllerRequestHeader registerBrokerToControllerRequestHeader = new RegisterBrokerToControllerRequestHeader(clusterName, brokerName, nextBrokerId, brokerAddress);
|
|||
|
|
+ RemotingCommand remotingCommand2 = leader.registerBroker(registerBrokerToControllerRequestHeader).get(2, TimeUnit.SECONDS);
|
|||
|
|
+
|
|||
|
|
+-
|
|||
|
|
+ assertEquals(ResponseCode.SUCCESS, remotingCommand2.getCode());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- public void brokerTryElectMaster(Controller leader, String clusterName, String brokerName, String brokerAddress, Long brokerId,
|
|||
|
|
++ public void brokerTryElectMaster(Controller leader, String clusterName, String brokerName, String brokerAddress,
|
|||
|
|
++ Long brokerId,
|
|||
|
|
+ boolean exceptSuccess) throws Exception {
|
|||
|
|
+ final ElectMasterRequestHeader electMasterRequestHeader = ElectMasterRequestHeader.ofBrokerTrigger(clusterName, brokerName, brokerId);
|
|||
|
|
+ RemotingCommand command = leader.electMaster(electMasterRequestHeader).get(2, TimeUnit.SECONDS);
|
|||
|
|
+@@ -186,9 +187,9 @@ public class DLedgerControllerTest {
|
|||
|
|
+ registerNewBroker(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[1], 2L);
|
|||
|
|
+ registerNewBroker(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[2], 3L);
|
|||
|
|
+ // try elect
|
|||
|
|
+- brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[0], 1L,true);
|
|||
|
|
+- brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[1], 2L, false);
|
|||
|
|
+- brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[2], 3L,false);
|
|||
|
|
++ brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[0], 1L, true);
|
|||
|
|
++ brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[1], 2L, false);
|
|||
|
|
++ brokerTryElectMaster(leader, DEFAULT_CLUSTER_NAME, DEFAULT_BROKER_NAME, DEFAULT_IP[2], 3L, false);
|
|||
|
|
+ final RemotingCommand getInfoResponse = leader.getReplicaInfo(new GetReplicaInfoRequestHeader(DEFAULT_BROKER_NAME)).get(10, TimeUnit.SECONDS);
|
|||
|
|
+ final GetReplicaInfoResponseHeader replicaInfo = (GetReplicaInfoResponseHeader) getInfoResponse.readCustomHeader();
|
|||
|
|
+ assertEquals(1, replicaInfo.getMasterEpoch().intValue());
|
|||
|
|
+@@ -239,6 +240,8 @@ public class DLedgerControllerTest {
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testBrokerLifecycleListener() throws Exception {
|
|||
|
|
+ final DLedgerController leader = mockMetaData(false);
|
|||
|
|
++
|
|||
|
|
++ assertTrue(leader.isLeaderState());
|
|||
|
|
+ // Mock that master broker has been inactive, and try to elect a new master from sync-state-set
|
|||
|
|
+ // But we shut down two controller, so the ElectMasterEvent will be appended to DLedger failed.
|
|||
|
|
+ // So the statemachine still keep the stale master's information
|
|||
|
|
+@@ -247,15 +250,20 @@ public class DLedgerControllerTest {
|
|||
|
|
+ dLedgerController.shutdown();
|
|||
|
|
+ controllers.remove(dLedgerController);
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
+ final ElectMasterRequestHeader request = ElectMasterRequestHeader.ofControllerTrigger(DEFAULT_BROKER_NAME);
|
|||
|
|
+ setBrokerElectPolicy(leader, 1L);
|
|||
|
|
+ Exception exception = null;
|
|||
|
|
++ RemotingCommand remotingCommand = null;
|
|||
|
|
+ try {
|
|||
|
|
+- leader.electMaster(request).get(5, TimeUnit.SECONDS);
|
|||
|
|
++ remotingCommand = leader.electMaster(request).get(5, TimeUnit.SECONDS);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ exception = e;
|
|||
|
|
+ }
|
|||
|
|
+- assertNotNull(exception);
|
|||
|
|
++
|
|||
|
|
++ assertTrue(exception != null ||
|
|||
|
|
++ remotingCommand != null && remotingCommand.getCode() == ResponseCode.CONTROLLER_NOT_LEADER);
|
|||
|
|
++
|
|||
|
|
+ // Shut down leader controller
|
|||
|
|
+ leader.shutdown();
|
|||
|
|
+ controllers.remove(leader);
|
|||
|
|
+@@ -272,7 +280,7 @@ public class DLedgerControllerTest {
|
|||
|
|
+ setBrokerAlivePredicate(newLeader, 1L);
|
|||
|
|
+ // Check if the statemachine is stale
|
|||
|
|
+ final RemotingCommand resp = newLeader.getReplicaInfo(new GetReplicaInfoRequestHeader(DEFAULT_BROKER_NAME)).
|
|||
|
|
+- get(10, TimeUnit.SECONDS);
|
|||
|
|
++ get(10, TimeUnit.SECONDS);
|
|||
|
|
+ final GetReplicaInfoResponseHeader replicaInfo = (GetReplicaInfoResponseHeader) resp.readCustomHeader();
|
|||
|
|
+ assertEquals(1, replicaInfo.getMasterBrokerId().longValue());
|
|||
|
|
+ assertEquals(1, replicaInfo.getMasterEpoch().intValue());
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 38d3d5d95d371ac89f7d491a4c8719b4a22c60e1 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: mxsm <ljbmxsm@gmail.com>
|
|||
|
|
+Date: Tue, 10 Oct 2023 09:37:04 +0800
|
|||
|
|
+Subject: [PATCH 6/7] [ISSUE #7433]Update the version in the README.md document
|
|||
|
|
+ to 5.1.4 (#7434)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ README.md | 8 ++++----
|
|||
|
|
+ 1 file changed, 4 insertions(+), 4 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/README.md b/README.md
|
|||
|
|
+index 56d253ce1..5aaa2ba73 100644
|
|||
|
|
+--- a/README.md
|
|||
|
|
++++ b/README.md
|
|||
|
|
+@@ -49,21 +49,21 @@ $ java -version
|
|||
|
|
+ java version "1.8.0_121"
|
|||
|
|
+ ```
|
|||
|
|
+
|
|||
|
|
+-For Windows users, click [here](https://dist.apache.org/repos/dist/release/rocketmq/5.1.3/rocketmq-all-5.1.3-bin-release.zip) to download the 5.1.3 RocketMQ binary release,
|
|||
|
|
++For Windows users, click [here](https://dist.apache.org/repos/dist/release/rocketmq/5.1.4/rocketmq-all-5.1.4-bin-release.zip) to download the 5.1.4 RocketMQ binary release,
|
|||
|
|
+ unpack it to your local disk, such as `D:\rocketmq`.
|
|||
|
|
+ For macOS and Linux users, execute following commands:
|
|||
|
|
+
|
|||
|
|
+ ```shell
|
|||
|
|
+ # Download release from the Apache mirror
|
|||
|
|
+-$ wget https://dist.apache.org/repos/dist/release/rocketmq/5.1.3/rocketmq-all-5.1.3-bin-release.zip
|
|||
|
|
++$ wget https://dist.apache.org/repos/dist/release/rocketmq/5.1.4/rocketmq-all-5.1.4-bin-release.zip
|
|||
|
|
+
|
|||
|
|
+ # Unpack the release
|
|||
|
|
+-$ unzip rocketmq-all-5.1.3-bin-release.zip
|
|||
|
|
++$ unzip rocketmq-all-5.1.4-bin-release.zip
|
|||
|
|
+ ```
|
|||
|
|
+
|
|||
|
|
+ Prepare a terminal and change to the extracted `bin` directory:
|
|||
|
|
+ ```shell
|
|||
|
|
+-$ cd rocketmq-all-5.1.3-bin-release/bin
|
|||
|
|
++$ cd rocketmq-all-5.1.4-bin-release/bin
|
|||
|
|
+ ```
|
|||
|
|
+
|
|||
|
|
+ **1) Start NameServer**
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 4acb43ecee03e429d036e3ff4c28bd402d1b30c7 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Zhouxiang Zhan <zhouxzhan@apache.org>
|
|||
|
|
+Date: Tue, 10 Oct 2023 13:54:01 +0800
|
|||
|
|
+Subject: [PATCH 7/7] [ISSUE #7330] Add goaway and reconnection mechanism
|
|||
|
|
+ (#7331)
|
|||
|
|
+
|
|||
|
|
+* Add shutdown wait for NettyRemotingServer
|
|||
|
|
+
|
|||
|
|
+* Add goaway and reconnection mechanism
|
|||
|
|
+
|
|||
|
|
+* Add client version check
|
|||
|
|
+
|
|||
|
|
+* Add enableTransparentRetry for NettyClientConfig
|
|||
|
|
+
|
|||
|
|
+* Add enableReconnectForGoAway for NettyClientConfig
|
|||
|
|
+
|
|||
|
|
+* fix unit test
|
|||
|
|
+
|
|||
|
|
+* fix client version check
|
|||
|
|
+---
|
|||
|
|
+ .../remoting/netty/NettyClientConfig.java | 30 ++++
|
|||
|
|
+ .../remoting/netty/NettyRemotingAbstract.java | 15 ++
|
|||
|
|
+ .../remoting/netty/NettyRemotingClient.java | 153 ++++++++++++++++--
|
|||
|
|
+ .../remoting/netty/NettyRemotingServer.java | 31 ++--
|
|||
|
|
+ .../remoting/netty/NettyServerConfig.java | 19 +++
|
|||
|
|
+ .../remoting/protocol/ResponseCode.java | 2 +
|
|||
|
|
+ .../netty/NettyRemotingClientTest.java | 39 ++---
|
|||
|
|
+ 7 files changed, 239 insertions(+), 50 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
|
|||
|
|
+index b2e7df754..c28288786 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyClientConfig.java
|
|||
|
|
+@@ -53,6 +53,12 @@ public class NettyClientConfig {
|
|||
|
|
+ private boolean disableCallbackExecutor = false;
|
|||
|
|
+ private boolean disableNettyWorkerGroup = false;
|
|||
|
|
+
|
|||
|
|
++ private long maxReconnectIntervalTimeSeconds = 60;
|
|||
|
|
++
|
|||
|
|
++ private boolean enableReconnectForGoAway = true;
|
|||
|
|
++
|
|||
|
|
++ private boolean enableTransparentRetry = true;
|
|||
|
|
++
|
|||
|
|
+ public boolean isClientCloseSocketIfTimeout() {
|
|||
|
|
+ return clientCloseSocketIfTimeout;
|
|||
|
|
+ }
|
|||
|
|
+@@ -181,6 +187,30 @@ public class NettyClientConfig {
|
|||
|
|
+ this.disableNettyWorkerGroup = disableNettyWorkerGroup;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ public long getMaxReconnectIntervalTimeSeconds() {
|
|||
|
|
++ return maxReconnectIntervalTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setMaxReconnectIntervalTimeSeconds(long maxReconnectIntervalTimeSeconds) {
|
|||
|
|
++ this.maxReconnectIntervalTimeSeconds = maxReconnectIntervalTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableReconnectForGoAway() {
|
|||
|
|
++ return enableReconnectForGoAway;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableReconnectForGoAway(boolean enableReconnectForGoAway) {
|
|||
|
|
++ this.enableReconnectForGoAway = enableReconnectForGoAway;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableTransparentRetry() {
|
|||
|
|
++ return enableTransparentRetry;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableTransparentRetry(boolean enableTransparentRetry) {
|
|||
|
|
++ this.enableTransparentRetry = enableTransparentRetry;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public String getSocksProxyConfig() {
|
|||
|
|
+ return socksProxyConfig;
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+index 12e66f913..07ace28ea 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingAbstract.java
|
|||
|
|
+@@ -40,9 +40,11 @@ import java.util.concurrent.Semaphore;
|
|||
|
|
+ import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import java.util.concurrent.TimeoutException;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
|
++import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
|
+ import java.util.function.Consumer;
|
|||
|
|
+ import javax.annotation.Nullable;
|
|||
|
|
+ import org.apache.rocketmq.common.AbortProcessException;
|
|||
|
|
++import org.apache.rocketmq.common.MQVersion;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ServiceThread;
|
|||
|
|
+ import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+@@ -60,6 +62,7 @@ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.metrics.RemotingMetricsManager;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+
|
|||
|
|
+ import static org.apache.rocketmq.remoting.metrics.RemotingMetricsConstant.LABEL_IS_LONG_POLLING;
|
|||
|
|
+ import static org.apache.rocketmq.remoting.metrics.RemotingMetricsConstant.LABEL_REQUEST_CODE;
|
|||
|
|
+@@ -120,6 +123,8 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+ */
|
|||
|
|
+ protected List<RPCHook> rpcHooks = new ArrayList<>();
|
|||
|
|
+
|
|||
|
|
++ protected AtomicBoolean isShuttingDown = new AtomicBoolean(false);
|
|||
|
|
++
|
|||
|
|
+ static {
|
|||
|
|
+ NettyLogger.initNettyLogger();
|
|||
|
|
+ }
|
|||
|
|
+@@ -264,6 +269,16 @@ public abstract class NettyRemotingAbstract {
|
|||
|
|
+
|
|||
|
|
+ Runnable run = buildProcessRequestHandler(ctx, cmd, pair, opaque);
|
|||
|
|
+
|
|||
|
|
++ if (isShuttingDown.get()) {
|
|||
|
|
++ if (cmd.getVersion() > MQVersion.Version.V5_1_4.ordinal()) {
|
|||
|
|
++ final RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.GO_AWAY,
|
|||
|
|
++ "please go away");
|
|||
|
|
++ response.setOpaque(opaque);
|
|||
|
|
++ writeResponse(ctx.channel(), cmd, response);
|
|||
|
|
++ return;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ if (pair.getObject1().rejectRequest()) {
|
|||
|
|
+ final RemotingCommand response = RemotingCommand.createResponseCommand(RemotingSysResponseCode.SYSTEM_BUSY,
|
|||
|
|
+ "[REJECTREQUEST]system busy, start flow control for a while");
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+index 8631d0447..4bc51bd83 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java
|
|||
|
|
+@@ -18,6 +18,7 @@ package org.apache.rocketmq.remoting.netty;
|
|||
|
|
+
|
|||
|
|
+ import com.alibaba.fastjson.JSON;
|
|||
|
|
+ import com.alibaba.fastjson.TypeReference;
|
|||
|
|
++import com.google.common.base.Stopwatch;
|
|||
|
|
+ import io.netty.bootstrap.Bootstrap;
|
|||
|
|
+ import io.netty.buffer.PooledByteBufAllocator;
|
|||
|
|
+ import io.netty.channel.Channel;
|
|||
|
|
+@@ -48,6 +49,7 @@ import java.io.IOException;
|
|||
|
|
+ import java.net.InetSocketAddress;
|
|||
|
|
+ import java.net.SocketAddress;
|
|||
|
|
+ import java.security.cert.CertificateException;
|
|||
|
|
++import java.time.Duration;
|
|||
|
|
+ import java.util.ArrayList;
|
|||
|
|
+ import java.util.Collections;
|
|||
|
|
+ import java.util.HashMap;
|
|||
|
|
+@@ -57,6 +59,7 @@ import java.util.Map;
|
|||
|
|
+ import java.util.Random;
|
|||
|
|
+ import java.util.Set;
|
|||
|
|
+ import java.util.concurrent.ArrayBlockingQueue;
|
|||
|
|
++import java.util.concurrent.CompletableFuture;
|
|||
|
|
+ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+ import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+ import java.util.concurrent.ExecutorService;
|
|||
|
|
+@@ -66,6 +69,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
|
+ import java.util.concurrent.locks.Lock;
|
|||
|
|
+ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
|
++import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+ import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+@@ -82,6 +86,7 @@ import org.apache.rocketmq.remoting.exception.RemotingSendRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
++import org.apache.rocketmq.remoting.protocol.ResponseCode;
|
|||
|
|
+ import org.apache.rocketmq.remoting.proxy.SocksProxyConfig;
|
|||
|
|
+
|
|||
|
|
+ public class NettyRemotingClient extends NettyRemotingAbstract implements RemotingClient {
|
|||
|
|
+@@ -97,6 +102,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ private final Map<String /* cidr */, SocksProxyConfig /* proxy */> proxyMap = new HashMap<>();
|
|||
|
|
+ private final ConcurrentHashMap<String /* cidr */, Bootstrap> bootstrapMap = new ConcurrentHashMap<>();
|
|||
|
|
+ private final ConcurrentMap<String /* addr */, ChannelWrapper> channelTables = new ConcurrentHashMap<>();
|
|||
|
|
++ private final ConcurrentMap<Channel, ChannelWrapper> channelWrapperTables = new ConcurrentHashMap<>();
|
|||
|
|
+
|
|||
|
|
+ private final HashedWheelTimer timer = new HashedWheelTimer(r -> new Thread(r, "ClientHouseKeepingService"));
|
|||
|
|
+
|
|||
|
|
+@@ -356,9 +362,10 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ this.timer.stop();
|
|||
|
|
+
|
|||
|
|
+ for (String addr : this.channelTables.keySet()) {
|
|||
|
|
+- this.closeChannel(addr, this.channelTables.get(addr).getChannel());
|
|||
|
|
++ this.channelTables.get(addr).close();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ this.channelWrapperTables.clear();
|
|||
|
|
+ this.channelTables.clear();
|
|||
|
|
+
|
|||
|
|
+ this.eventLoopGroupWorker.shutdownGracefully();
|
|||
|
|
+@@ -416,7 +423,10 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (removeItemFromTable) {
|
|||
|
|
+- this.channelTables.remove(addrRemote);
|
|||
|
|
++ ChannelWrapper channelWrapper = this.channelWrapperTables.remove(channel);
|
|||
|
|
++ if (channelWrapper != null && channelWrapper.tryClose(channel)) {
|
|||
|
|
++ this.channelTables.remove(addrRemote);
|
|||
|
|
++ }
|
|||
|
|
+ LOGGER.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+@@ -463,7 +473,10 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (removeItemFromTable) {
|
|||
|
|
+- this.channelTables.remove(addrRemote);
|
|||
|
|
++ ChannelWrapper channelWrapper = this.channelWrapperTables.remove(channel);
|
|||
|
|
++ if (channelWrapper != null && channelWrapper.tryClose(channel)) {
|
|||
|
|
++ this.channelTables.remove(addrRemote);
|
|||
|
|
++ }
|
|||
|
|
+ LOGGER.info("closeChannel: the channel[{}] was removed from channel table", addrRemote);
|
|||
|
|
+ RemotingHelper.closeChannel(channel);
|
|||
|
|
+ }
|
|||
|
|
+@@ -511,7 +524,7 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ if (addr.contains(namesrvAddr)) {
|
|||
|
|
+ ChannelWrapper channelWrapper = this.channelTables.get(addr);
|
|||
|
|
+ if (channelWrapper != null) {
|
|||
|
|
+- closeChannel(channelWrapper.getChannel());
|
|||
|
|
++ channelWrapper.close();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+@@ -689,8 +702,9 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ ChannelFuture channelFuture = fetchBootstrap(addr)
|
|||
|
|
+ .connect(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
|
|||
|
|
+ LOGGER.info("createChannel: begin to connect remote host[{}] asynchronously", addr);
|
|||
|
|
+- cw = new ChannelWrapper(channelFuture);
|
|||
|
|
++ cw = new ChannelWrapper(addr, channelFuture);
|
|||
|
|
+ this.channelTables.put(addr, cw);
|
|||
|
|
++ this.channelWrapperTables.put(channelFuture.channel(), cw);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ LOGGER.error("createChannel: create channel exception", e);
|
|||
|
|
+@@ -758,6 +772,64 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<RemotingCommand> invoke(String addr, RemotingCommand request,
|
|||
|
|
++ long timeoutMillis) {
|
|||
|
|
++ CompletableFuture<RemotingCommand> future = new CompletableFuture<>();
|
|||
|
|
++ try {
|
|||
|
|
++ final Channel channel = this.getAndCreateChannel(addr);
|
|||
|
|
++ if (channel != null && channel.isActive()) {
|
|||
|
|
++ return invokeImpl(channel, request, timeoutMillis).whenComplete((v, t) -> {
|
|||
|
|
++ if (t == null) {
|
|||
|
|
++ updateChannelLastResponseTime(addr);
|
|||
|
|
++ }
|
|||
|
|
++ }).thenApply(ResponseFuture::getResponseCommand);
|
|||
|
|
++ } else {
|
|||
|
|
++ this.closeChannel(addr, channel);
|
|||
|
|
++ future.completeExceptionally(new RemotingConnectException(addr));
|
|||
|
|
++ }
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ future.completeExceptionally(t);
|
|||
|
|
++ }
|
|||
|
|
++ return future;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public CompletableFuture<ResponseFuture> invokeImpl(final Channel channel, final RemotingCommand request,
|
|||
|
|
++ final long timeoutMillis) {
|
|||
|
|
++ Stopwatch stopwatch = Stopwatch.createStarted();
|
|||
|
|
++ return super.invokeImpl(channel, request, timeoutMillis).thenCompose(responseFuture -> {
|
|||
|
|
++ RemotingCommand response = responseFuture.getResponseCommand();
|
|||
|
|
++ if (response.getCode() == ResponseCode.GO_AWAY) {
|
|||
|
|
++ if (nettyClientConfig.isEnableReconnectForGoAway()) {
|
|||
|
|
++ ChannelWrapper channelWrapper = channelWrapperTables.computeIfPresent(channel, (channel0, channelWrapper0) -> {
|
|||
|
|
++ try {
|
|||
|
|
++ if (channelWrapper0.reconnect()) {
|
|||
|
|
++ LOGGER.info("Receive go away from channel {}, recreate the channel", channel0);
|
|||
|
|
++ channelWrapperTables.put(channelWrapper0.getChannel(), channelWrapper0);
|
|||
|
|
++ }
|
|||
|
|
++ } catch (Throwable t) {
|
|||
|
|
++ LOGGER.error("Channel {} reconnect error", channelWrapper0, t);
|
|||
|
|
++ }
|
|||
|
|
++ return channelWrapper0;
|
|||
|
|
++ });
|
|||
|
|
++ if (channelWrapper != null) {
|
|||
|
|
++ if (nettyClientConfig.isEnableTransparentRetry()) {
|
|||
|
|
++ long duration = stopwatch.elapsed(TimeUnit.MILLISECONDS);
|
|||
|
|
++ stopwatch.stop();
|
|||
|
|
++ RemotingCommand retryRequest = RemotingCommand.createRequestCommand(request.getCode(), request.readCustomHeader());
|
|||
|
|
++ Channel retryChannel = channelWrapper.getChannel();
|
|||
|
|
++ if (channel != retryChannel) {
|
|||
|
|
++ return super.invokeImpl(retryChannel, retryRequest, timeoutMillis - duration);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return CompletableFuture.completedFuture(responseFuture);
|
|||
|
|
++ });
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ @Override
|
|||
|
|
+ public void registerProcessor(int requestCode, NettyRequestProcessor processor, ExecutorService executor) {
|
|||
|
|
+ ExecutorService executorThis = executor;
|
|||
|
|
+@@ -877,30 +949,41 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- static class ChannelWrapper {
|
|||
|
|
+- private final ChannelFuture channelFuture;
|
|||
|
|
++ class ChannelWrapper {
|
|||
|
|
++ private final ReentrantReadWriteLock lock;
|
|||
|
|
++ private ChannelFuture channelFuture;
|
|||
|
|
+ // only affected by sync or async request, oneway is not included.
|
|||
|
|
++ private ChannelFuture channelToClose;
|
|||
|
|
+ private long lastResponseTime;
|
|||
|
|
++ private volatile long lastReconnectTimestamp = 0L;
|
|||
|
|
++ private final String channelAddress;
|
|||
|
|
+
|
|||
|
|
+- public ChannelWrapper(ChannelFuture channelFuture) {
|
|||
|
|
++ public ChannelWrapper(String address, ChannelFuture channelFuture) {
|
|||
|
|
++ this.lock = new ReentrantReadWriteLock();
|
|||
|
|
+ this.channelFuture = channelFuture;
|
|||
|
|
+ this.lastResponseTime = System.currentTimeMillis();
|
|||
|
|
++ this.channelAddress = address;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean isOK() {
|
|||
|
|
+- return this.channelFuture.channel() != null && this.channelFuture.channel().isActive();
|
|||
|
|
++ return getChannel() != null && getChannel().isActive();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean isWritable() {
|
|||
|
|
+- return this.channelFuture.channel().isWritable();
|
|||
|
|
++ return getChannel().isWritable();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private Channel getChannel() {
|
|||
|
|
+- return this.channelFuture.channel();
|
|||
|
|
++ return getChannelFuture().channel();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ChannelFuture getChannelFuture() {
|
|||
|
|
+- return channelFuture;
|
|||
|
|
++ lock.readLock().lock();
|
|||
|
|
++ try {
|
|||
|
|
++ return this.channelFuture;
|
|||
|
|
++ } finally {
|
|||
|
|
++ lock.readLock().unlock();
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getLastResponseTime() {
|
|||
|
|
+@@ -910,6 +993,52 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ public void updateLastResponseTime() {
|
|||
|
|
+ this.lastResponseTime = System.currentTimeMillis();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean reconnect() {
|
|||
|
|
++ if (lock.writeLock().tryLock()) {
|
|||
|
|
++ try {
|
|||
|
|
++ if (lastReconnectTimestamp == 0L || System.currentTimeMillis() - lastReconnectTimestamp > Duration.ofSeconds(nettyClientConfig.getMaxReconnectIntervalTimeSeconds()).toMillis()) {
|
|||
|
|
++ channelToClose = channelFuture;
|
|||
|
|
++ String[] hostAndPort = getHostAndPort(channelAddress);
|
|||
|
|
++ channelFuture = fetchBootstrap(channelAddress)
|
|||
|
|
++ .connect(hostAndPort[0], Integer.parseInt(hostAndPort[1]));
|
|||
|
|
++ lastReconnectTimestamp = System.currentTimeMillis();
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++ } finally {
|
|||
|
|
++ lock.writeLock().unlock();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public boolean tryClose(Channel channel) {
|
|||
|
|
++ try {
|
|||
|
|
++ lock.readLock().lock();
|
|||
|
|
++ if (channelFuture != null) {
|
|||
|
|
++ if (channelFuture.channel().equals(channel)) {
|
|||
|
|
++ return true;
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ } finally {
|
|||
|
|
++ lock.readLock().unlock();
|
|||
|
|
++ }
|
|||
|
|
++ return false;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void close() {
|
|||
|
|
++ try {
|
|||
|
|
++ lock.writeLock().lock();
|
|||
|
|
++ if (channelFuture != null) {
|
|||
|
|
++ closeChannel(channelFuture.channel());
|
|||
|
|
++ }
|
|||
|
|
++ if (channelToClose != null) {
|
|||
|
|
++ closeChannel(channelToClose.channel());
|
|||
|
|
++ }
|
|||
|
|
++ } finally {
|
|||
|
|
++ lock.writeLock().unlock();
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class InvokeCallbackWrapper implements InvokeCallback {
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+index aa0d46542..735d36168 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingServer.java
|
|||
|
|
+@@ -53,6 +53,19 @@ import io.netty.util.HashedWheelTimer;
|
|||
|
|
+ import io.netty.util.Timeout;
|
|||
|
|
+ import io.netty.util.TimerTask;
|
|||
|
|
+ import io.netty.util.concurrent.DefaultEventExecutorGroup;
|
|||
|
|
++import java.io.IOException;
|
|||
|
|
++import java.net.InetSocketAddress;
|
|||
|
|
++import java.security.cert.CertificateException;
|
|||
|
|
++import java.time.Duration;
|
|||
|
|
++import java.util.List;
|
|||
|
|
++import java.util.NoSuchElementException;
|
|||
|
|
++import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
++import java.util.concurrent.ConcurrentMap;
|
|||
|
|
++import java.util.concurrent.ExecutorService;
|
|||
|
|
++import java.util.concurrent.Executors;
|
|||
|
|
++import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
++import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
++import java.util.concurrent.TimeUnit;
|
|||
|
|
+ import org.apache.commons.collections.CollectionUtils;
|
|||
|
|
+ import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.rocketmq.common.Pair;
|
|||
|
|
+@@ -74,19 +87,6 @@ import org.apache.rocketmq.remoting.exception.RemotingTimeoutException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException;
|
|||
|
|
+ import org.apache.rocketmq.remoting.protocol.RemotingCommand;
|
|||
|
|
+
|
|||
|
|
+-import java.io.IOException;
|
|||
|
|
+-import java.net.InetSocketAddress;
|
|||
|
|
+-import java.security.cert.CertificateException;
|
|||
|
|
+-import java.util.List;
|
|||
|
|
+-import java.util.NoSuchElementException;
|
|||
|
|
+-import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+-import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+-import java.util.concurrent.ExecutorService;
|
|||
|
|
+-import java.util.concurrent.Executors;
|
|||
|
|
+-import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+-import java.util.concurrent.ThreadPoolExecutor;
|
|||
|
|
+-import java.util.concurrent.TimeUnit;
|
|||
|
|
+-
|
|||
|
|
+ @SuppressWarnings("NullableProblems")
|
|||
|
|
+ public class NettyRemotingServer extends NettyRemotingAbstract implements RemotingServer {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.ROCKETMQ_REMOTING_NAME);
|
|||
|
|
+@@ -305,6 +305,10 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+ @Override
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
+ try {
|
|||
|
|
++ if (nettyServerConfig.isEnableShutdownGracefully() && isShuttingDown.compareAndSet(false, true)) {
|
|||
|
|
++ Thread.sleep(Duration.ofSeconds(nettyServerConfig.getShutdownWaitTimeSeconds()).toMillis());
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ this.timer.stop();
|
|||
|
|
+
|
|||
|
|
+ this.eventLoopGroupBoss.shutdownGracefully();
|
|||
|
|
+@@ -736,6 +740,7 @@ public class NettyRemotingServer extends NettyRemotingAbstract implements Remoti
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void shutdown() {
|
|||
|
|
++ isShuttingDown.set(true);
|
|||
|
|
+ if (this.serverChannel != null) {
|
|||
|
|
+ try {
|
|||
|
|
+ this.serverChannel.close().await(5, TimeUnit.SECONDS);
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java
|
|||
|
|
+index 59ef2c84f..756661f62 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyServerConfig.java
|
|||
|
|
+@@ -38,6 +38,9 @@ public class NettyServerConfig implements Cloneable {
|
|||
|
|
+ private int serverSocketBacklog = NettySystemConfig.socketBacklog;
|
|||
|
|
+ private boolean serverPooledByteBufAllocatorEnable = true;
|
|||
|
|
+
|
|||
|
|
++ private boolean enableShutdownGracefully = false;
|
|||
|
|
++ private int shutdownWaitTimeSeconds = 30;
|
|||
|
|
++
|
|||
|
|
+ /**
|
|||
|
|
+ * make install
|
|||
|
|
+ *
|
|||
|
|
+@@ -171,4 +174,20 @@ public class NettyServerConfig implements Cloneable {
|
|||
|
|
+ public void setWriteBufferHighWaterMark(int writeBufferHighWaterMark) {
|
|||
|
|
+ this.writeBufferHighWaterMark = writeBufferHighWaterMark;
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public boolean isEnableShutdownGracefully() {
|
|||
|
|
++ return enableShutdownGracefully;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setEnableShutdownGracefully(boolean enableShutdownGracefully) {
|
|||
|
|
++ this.enableShutdownGracefully = enableShutdownGracefully;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public int getShutdownWaitTimeSeconds() {
|
|||
|
|
++ return shutdownWaitTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public void setShutdownWaitTimeSeconds(int shutdownWaitTimeSeconds) {
|
|||
|
|
++ this.shutdownWaitTimeSeconds = shutdownWaitTimeSeconds;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java
|
|||
|
|
+index e81dadf2e..be945c48f 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/ResponseCode.java
|
|||
|
|
+@@ -99,6 +99,8 @@ public class ResponseCode extends RemotingSysResponseCode {
|
|||
|
|
+ public static final int RPC_SEND_TO_CHANNEL_FAILED = -1004;
|
|||
|
|
+ public static final int RPC_TIME_OUT = -1006;
|
|||
|
|
+
|
|||
|
|
++ public static final int GO_AWAY = 1500;
|
|||
|
|
++
|
|||
|
|
+ /**
|
|||
|
|
+ * Controller response code
|
|||
|
|
+ */
|
|||
|
|
+diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
+index e72e7bd53..1cc6b4f46 100644
|
|||
|
|
+--- a/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
++++ b/remoting/src/test/java/org/apache/rocketmq/remoting/netty/NettyRemotingClientTest.java
|
|||
|
|
+@@ -47,7 +47,6 @@ import static org.mockito.ArgumentMatchers.any;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.anyString;
|
|||
|
|
+ import static org.mockito.ArgumentMatchers.eq;
|
|||
|
|
+-import static org.mockito.Mockito.doAnswer;
|
|||
|
|
+ import static org.mockito.Mockito.doReturn;
|
|||
|
|
+ import static org.mockito.Mockito.mock;
|
|||
|
|
+ import static org.mockito.Mockito.never;
|
|||
|
|
+@@ -74,13 +73,11 @@ public class NettyRemotingClientTest {
|
|||
|
|
+
|
|||
|
|
+ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- doAnswer(invocation -> {
|
|||
|
|
+- InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
+- responseFuture.setResponseCommand(response);
|
|||
|
|
+- callback.operationSucceed(responseFuture.getResponseCommand());
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
++ ResponseFuture responseFuture = new ResponseFuture(null, request.getOpaque(), 3 * 1000, null, null);
|
|||
|
|
++ responseFuture.setResponseCommand(response);
|
|||
|
|
++ CompletableFuture<RemotingCommand> future0 = new CompletableFuture<>();
|
|||
|
|
++ future0.complete(responseFuture.getResponseCommand());
|
|||
|
|
++ doReturn(future0).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = remotingClient.invoke("0.0.0.0", request, 1000);
|
|||
|
|
+ RemotingCommand actual = future.get();
|
|||
|
|
+@@ -93,11 +90,9 @@ public class NettyRemotingClientTest {
|
|||
|
|
+
|
|||
|
|
+ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- doAnswer(invocation -> {
|
|||
|
|
+- InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- callback.operationFail(new RemotingSendRequestException(null));
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
++ CompletableFuture<RemotingCommand> future0 = new CompletableFuture<>();
|
|||
|
|
++ future0.completeExceptionally(new RemotingSendRequestException(null));
|
|||
|
|
++ doReturn(future0).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = remotingClient.invoke("0.0.0.0", request, 1000);
|
|||
|
|
+ Throwable thrown = catchThrowable(future::get);
|
|||
|
|
+@@ -110,11 +105,9 @@ public class NettyRemotingClientTest {
|
|||
|
|
+
|
|||
|
|
+ RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+ response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- doAnswer(invocation -> {
|
|||
|
|
+- InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- callback.operationFail(new RemotingTimeoutException(""));
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
++ CompletableFuture<RemotingCommand> future0 = new CompletableFuture<>();
|
|||
|
|
++ future0.completeExceptionally(new RemotingTimeoutException(""));
|
|||
|
|
++ doReturn(future0).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = remotingClient.invoke("0.0.0.0", request, 1000);
|
|||
|
|
+ Throwable thrown = catchThrowable(future::get);
|
|||
|
|
+@@ -125,13 +118,9 @@ public class NettyRemotingClientTest {
|
|||
|
|
+ public void testRemotingException() throws Exception {
|
|||
|
|
+ RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, null);
|
|||
|
|
+
|
|||
|
|
+- RemotingCommand response = RemotingCommand.createResponseCommand(null);
|
|||
|
|
+- response.setCode(ResponseCode.SUCCESS);
|
|||
|
|
+- doAnswer(invocation -> {
|
|||
|
|
+- InvokeCallback callback = invocation.getArgument(3);
|
|||
|
|
+- callback.operationFail(new RemotingException(null));
|
|||
|
|
+- return null;
|
|||
|
|
+- }).when(remotingClient).invokeAsync(anyString(), any(RemotingCommand.class), anyLong(), any(InvokeCallback.class));
|
|||
|
|
++ CompletableFuture<RemotingCommand> future0 = new CompletableFuture<>();
|
|||
|
|
++ future0.completeExceptionally(new RemotingException(""));
|
|||
|
|
++ doReturn(future0).when(remotingClient).invoke(anyString(), any(RemotingCommand.class), anyLong());
|
|||
|
|
+
|
|||
|
|
+ CompletableFuture<RemotingCommand> future = remotingClient.invoke("0.0.0.0", request, 1000);
|
|||
|
|
+ Throwable thrown = catchThrowable(future::get);
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/patch021-backport-some-enhancements.patch b/patch021-backport-some-enhancements.patch
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..839b65fea
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/patch021-backport-some-enhancements.patch
|
|||
|
|
@@ -0,0 +1,344 @@
|
|||
|
|
+From dc3f22ffe9eb83ace991b68921076093c7c0da5f Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: LetLetMe <43874697+LetLetMe@users.noreply.github.com>
|
|||
|
|
+Date: Tue, 10 Oct 2023 17:39:23 +0800
|
|||
|
|
+Subject: [PATCH 1/6] add getter for class Message ,fix json serialize bug
|
|||
|
|
+ (#7439)
|
|||
|
|
+
|
|||
|
|
+Co-authored-by: LetLetMe <allen.hyt@alibaba-inc.com>
|
|||
|
|
+---
|
|||
|
|
+ .../rocketmq/common/message/Message.java | 24 ++++++++++++++++++-
|
|||
|
|
+ 1 file changed, 23 insertions(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/message/Message.java b/common/src/main/java/org/apache/rocketmq/common/message/Message.java
|
|||
|
|
+index e02b526a1..c7997c473 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/message/Message.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/message/Message.java
|
|||
|
|
+@@ -218,14 +218,36 @@ public class Message implements Serializable {
|
|||
|
|
+ public void setDelayTimeSec(long sec) {
|
|||
|
|
+ this.putProperty(MessageConst.PROPERTY_TIMER_DELAY_SEC, String.valueOf(sec));
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public long getDelayTimeSec() {
|
|||
|
|
++ String t = this.getProperty(MessageConst.PROPERTY_TIMER_DELAY_SEC);
|
|||
|
|
++ if (t != null) {
|
|||
|
|
++ return Long.parseLong(t);
|
|||
|
|
++ }
|
|||
|
|
++ return 0;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public void setDelayTimeMs(long timeMs) {
|
|||
|
|
+ this.putProperty(MessageConst.PROPERTY_TIMER_DELAY_MS, String.valueOf(timeMs));
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ public long getDelayTimeMs() {
|
|||
|
|
++ String t = this.getProperty(MessageConst.PROPERTY_TIMER_DELAY_MS);
|
|||
|
|
++ if (t != null) {
|
|||
|
|
++ return Long.parseLong(t);
|
|||
|
|
++ }
|
|||
|
|
++ return 0;
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
+ public void setDeliverTimeMs(long timeMs) {
|
|||
|
|
+ this.putProperty(MessageConst.PROPERTY_TIMER_DELIVER_MS, String.valueOf(timeMs));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getDeliverTimeMs() {
|
|||
|
|
+- return Long.parseLong(this.getUserProperty(MessageConst.PROPERTY_TIMER_DELIVER_MS));
|
|||
|
|
++ String t = this.getProperty(MessageConst.PROPERTY_TIMER_DELIVER_MS);
|
|||
|
|
++ if (t != null) {
|
|||
|
|
++ return Long.parseLong(t);
|
|||
|
|
++ }
|
|||
|
|
++ return 0;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 7e4879a3bc120d6289aabc8354a2811f349ac8a6 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: fujian-zfj <2573259572@qq.com>
|
|||
|
|
+Date: Wed, 11 Oct 2023 14:45:07 +0800
|
|||
|
|
+Subject: [PATCH 2/6] [ISSUE #7441] Fix log "Init the confirmOffset" keep
|
|||
|
|
+ printing error in controller mode (#7442)
|
|||
|
|
+
|
|||
|
|
+* typo int readme[ecosystem]
|
|||
|
|
+
|
|||
|
|
+* fix keep printing log problem
|
|||
|
|
+---
|
|||
|
|
+ store/src/main/java/org/apache/rocketmq/store/CommitLog.java | 2 +-
|
|||
|
|
+ 1 file changed, 1 insertion(+), 1 deletion(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
+index 456bf2b86..f98e9a284 100644
|
|||
|
|
+--- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
++++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
+@@ -580,7 +580,7 @@ public class CommitLog implements Swappable {
|
|||
|
|
+ return this.defaultMessageStore.getMaxPhyOffset();
|
|||
|
|
+ }
|
|||
|
|
+ // First time it will compute the confirmOffset.
|
|||
|
|
+- if (this.confirmOffset <= 0) {
|
|||
|
|
++ if (this.confirmOffset < 0) {
|
|||
|
|
+ setConfirmOffset(((AutoSwitchHAService) this.defaultMessageStore.getHaService()).computeConfirmOffset());
|
|||
|
|
+ log.info("Init the confirmOffset to {}.", this.confirmOffset);
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 5d492c338258d07613103e6ae16df4c6fa5b3838 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: rongtong <jinrongtong5@163.com>
|
|||
|
|
+Date: Fri, 13 Oct 2023 11:23:30 +0800
|
|||
|
|
+Subject: [PATCH 3/6] [ISSUE #7444] Fix testCalculateFileSizeInPath test can
|
|||
|
|
+ not rerun in same environment (#7445)
|
|||
|
|
+
|
|||
|
|
+* Fix testCalculateFileSizeInPath test can not rerun in same environment
|
|||
|
|
+
|
|||
|
|
+* Ensure that files are always deleted
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/common/UtilAllTest.java | 83 +++++++++++--------
|
|||
|
|
+ 1 file changed, 48 insertions(+), 35 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java b/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
|
|||
|
|
+index f568a65f4..a0653d7fc 100644
|
|||
|
|
+--- a/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
|
|||
|
|
++++ b/common/src/test/java/org/apache/rocketmq/common/UtilAllTest.java
|
|||
|
|
+@@ -238,41 +238,54 @@ public class UtilAllTest {
|
|||
|
|
+ */
|
|||
|
|
+ String basePath = System.getProperty("java.io.tmpdir") + File.separator + "testCalculateFileSizeInPath";
|
|||
|
|
+ File baseFile = new File(basePath);
|
|||
|
|
+- // test empty path
|
|||
|
|
+- assertEquals(0, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
+-
|
|||
|
|
+- // create baseDir
|
|||
|
|
+- assertTrue(baseFile.mkdirs());
|
|||
|
|
+-
|
|||
|
|
+- File file0 = new File(baseFile, "file_0");
|
|||
|
|
+- assertTrue(file0.createNewFile());
|
|||
|
|
+- writeFixedBytesToFile(file0, 1313);
|
|||
|
|
+-
|
|||
|
|
+- assertEquals(1313, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
+-
|
|||
|
|
+- // build a file tree like above
|
|||
|
|
+- File dir1 = new File(baseFile, "dir_1");
|
|||
|
|
+- dir1.mkdirs();
|
|||
|
|
+- File file10 = new File(dir1, "file_1_0");
|
|||
|
|
+- File file11 = new File(dir1, "file_1_1");
|
|||
|
|
+- File dir12 = new File(dir1, "dir_1_2");
|
|||
|
|
+- dir12.mkdirs();
|
|||
|
|
+- File file120 = new File(dir12, "file_1_2_0");
|
|||
|
|
+- File dir2 = new File(baseFile, "dir_2");
|
|||
|
|
+- dir2.mkdirs();
|
|||
|
|
+-
|
|||
|
|
+- // write all file with 1313 bytes data
|
|||
|
|
+- assertTrue(file10.createNewFile());
|
|||
|
|
+- writeFixedBytesToFile(file10, 1313);
|
|||
|
|
+- assertTrue(file11.createNewFile());
|
|||
|
|
+- writeFixedBytesToFile(file11, 1313);
|
|||
|
|
+- assertTrue(file120.createNewFile());
|
|||
|
|
+- writeFixedBytesToFile(file120, 1313);
|
|||
|
|
+-
|
|||
|
|
+- assertEquals(1313 * 4, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
+-
|
|||
|
|
+- // clear all file
|
|||
|
|
+- baseFile.deleteOnExit();
|
|||
|
|
++ try {
|
|||
|
|
++ // test empty path
|
|||
|
|
++ assertEquals(0, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
++
|
|||
|
|
++ // create baseDir
|
|||
|
|
++ assertTrue(baseFile.mkdirs());
|
|||
|
|
++
|
|||
|
|
++ File file0 = new File(baseFile, "file_0");
|
|||
|
|
++ assertTrue(file0.createNewFile());
|
|||
|
|
++ writeFixedBytesToFile(file0, 1313);
|
|||
|
|
++
|
|||
|
|
++ assertEquals(1313, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
++
|
|||
|
|
++ // build a file tree like above
|
|||
|
|
++ File dir1 = new File(baseFile, "dir_1");
|
|||
|
|
++ dir1.mkdirs();
|
|||
|
|
++ File file10 = new File(dir1, "file_1_0");
|
|||
|
|
++ File file11 = new File(dir1, "file_1_1");
|
|||
|
|
++ File dir12 = new File(dir1, "dir_1_2");
|
|||
|
|
++ dir12.mkdirs();
|
|||
|
|
++ File file120 = new File(dir12, "file_1_2_0");
|
|||
|
|
++ File dir2 = new File(baseFile, "dir_2");
|
|||
|
|
++ dir2.mkdirs();
|
|||
|
|
++
|
|||
|
|
++ // write all file with 1313 bytes data
|
|||
|
|
++ assertTrue(file10.createNewFile());
|
|||
|
|
++ writeFixedBytesToFile(file10, 1313);
|
|||
|
|
++ assertTrue(file11.createNewFile());
|
|||
|
|
++ writeFixedBytesToFile(file11, 1313);
|
|||
|
|
++ assertTrue(file120.createNewFile());
|
|||
|
|
++ writeFixedBytesToFile(file120, 1313);
|
|||
|
|
++
|
|||
|
|
++ assertEquals(1313 * 4, UtilAll.calculateFileSizeInPath(baseFile));
|
|||
|
|
++ } finally {
|
|||
|
|
++ deleteFolder(baseFile);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++
|
|||
|
|
++ public static void deleteFolder(File folder) {
|
|||
|
|
++ if (folder.isDirectory()) {
|
|||
|
|
++ File[] files = folder.listFiles();
|
|||
|
|
++ if (files != null) {
|
|||
|
|
++ for (File file : files) {
|
|||
|
|
++ deleteFolder(file);
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ }
|
|||
|
|
++ folder.delete();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void writeFixedBytesToFile(File file, int size) throws Exception {
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 28427d40129e3aa0c6f951535617e5cac0a8211b Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: Lei Sun <yuncun.sl@alibaba-inc.com>
|
|||
|
|
+Date: Fri, 13 Oct 2023 13:42:27 +0800
|
|||
|
|
+Subject: [PATCH 4/6] [ISSUE #7425] Add RoccketmqControllerConsole log to fix
|
|||
|
|
+ bug (#7458)
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../org/apache/rocketmq/common/constant/LoggerName.java | 1 +
|
|||
|
|
+ .../org/apache/rocketmq/controller/ControllerStartup.java | 7 ++++---
|
|||
|
|
+ controller/src/main/resources/rmq.controller.logback.xml | 4 ++++
|
|||
|
|
+ 3 files changed, 9 insertions(+), 3 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
+index cb04b00b3..61310893f 100644
|
|||
|
|
+--- a/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
++++ b/common/src/main/java/org/apache/rocketmq/common/constant/LoggerName.java
|
|||
|
|
+@@ -21,6 +21,7 @@ public class LoggerName {
|
|||
|
|
+ public static final String NAMESRV_LOGGER_NAME = "RocketmqNamesrv";
|
|||
|
|
+ public static final String NAMESRV_CONSOLE_LOGGER_NAME = "RocketmqNamesrvConsole";
|
|||
|
|
+ public static final String CONTROLLER_LOGGER_NAME = "RocketmqController";
|
|||
|
|
++ public static final String CONTROLLER_CONSOLE_NAME = "RocketmqControllerConsole";
|
|||
|
|
+ public static final String NAMESRV_WATER_MARK_LOGGER_NAME = "RocketmqNamesrvWaterMark";
|
|||
|
|
+ public static final String BROKER_LOGGER_NAME = "RocketmqBroker";
|
|||
|
|
+ public static final String BROKER_CONSOLE_NAME = "RocketmqConsole";
|
|||
|
|
+diff --git a/controller/src/main/java/org/apache/rocketmq/controller/ControllerStartup.java b/controller/src/main/java/org/apache/rocketmq/controller/ControllerStartup.java
|
|||
|
|
+index 401720d05..9e96a704d 100644
|
|||
|
|
+--- a/controller/src/main/java/org/apache/rocketmq/controller/ControllerStartup.java
|
|||
|
|
++++ b/controller/src/main/java/org/apache/rocketmq/controller/ControllerStartup.java
|
|||
|
|
+@@ -94,9 +94,10 @@ public class ControllerStartup {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (commandLine.hasOption('p')) {
|
|||
|
|
+- MixAll.printObjectProperties(null, controllerConfig);
|
|||
|
|
+- MixAll.printObjectProperties(null, nettyServerConfig);
|
|||
|
|
+- MixAll.printObjectProperties(null, nettyClientConfig);
|
|||
|
|
++ Logger console = LoggerFactory.getLogger(LoggerName.CONTROLLER_CONSOLE_NAME);
|
|||
|
|
++ MixAll.printObjectProperties(console, controllerConfig);
|
|||
|
|
++ MixAll.printObjectProperties(console, nettyServerConfig);
|
|||
|
|
++ MixAll.printObjectProperties(console, nettyClientConfig);
|
|||
|
|
+ System.exit(0);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+diff --git a/controller/src/main/resources/rmq.controller.logback.xml b/controller/src/main/resources/rmq.controller.logback.xml
|
|||
|
|
+index bb158213a..18083e8f9 100644
|
|||
|
|
+--- a/controller/src/main/resources/rmq.controller.logback.xml
|
|||
|
|
++++ b/controller/src/main/resources/rmq.controller.logback.xml
|
|||
|
|
+@@ -116,6 +116,10 @@
|
|||
|
|
+ <appender-ref ref="RocketmqControllerAppender"/>
|
|||
|
|
+ </logger>
|
|||
|
|
+
|
|||
|
|
++ <logger name="RocketmqControllerConsole" additivity="false" level="INFO">
|
|||
|
|
++ <appender-ref ref="STDOUT"/>
|
|||
|
|
++ </logger>
|
|||
|
|
++
|
|||
|
|
+ <logger name="RocketmqCommon" additivity="false" level="INFO">
|
|||
|
|
+ <appender-ref ref="RocketmqControllerAppender"/>
|
|||
|
|
+ </logger>
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From dc62d7f2e1ce4f99364599f8e23d65fd88eb1cd4 Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: LetLetMe <43874697+LetLetMe@users.noreply.github.com>
|
|||
|
|
+Date: Fri, 13 Oct 2023 13:45:48 +0800
|
|||
|
|
+Subject: [PATCH 5/6] [ISSUE #7451] Override toString for
|
|||
|
|
+ TopicConfigAndQueueMapping
|
|||
|
|
+
|
|||
|
|
+---
|
|||
|
|
+ .../statictopic/TopicConfigAndQueueMapping.java | 10 ++++++++++
|
|||
|
|
+ 1 file changed, 10 insertions(+)
|
|||
|
|
+
|
|||
|
|
+diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicConfigAndQueueMapping.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicConfigAndQueueMapping.java
|
|||
|
|
+index c937fec23..d13692735 100644
|
|||
|
|
+--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicConfigAndQueueMapping.java
|
|||
|
|
++++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicConfigAndQueueMapping.java
|
|||
|
|
+@@ -16,6 +16,7 @@
|
|||
|
|
+ */
|
|||
|
|
+ package org.apache.rocketmq.remoting.protocol.statictopic;
|
|||
|
|
+
|
|||
|
|
++import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+ import org.apache.commons.lang3.builder.EqualsBuilder;
|
|||
|
|
+ import org.apache.commons.lang3.builder.HashCodeBuilder;
|
|||
|
|
+ import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+@@ -60,4 +61,13 @@ public class TopicConfigAndQueueMapping extends TopicConfig {
|
|||
|
|
+ .append(mappingDetail)
|
|||
|
|
+ .toHashCode();
|
|||
|
|
+ }
|
|||
|
|
++
|
|||
|
|
++ @Override
|
|||
|
|
++ public String toString() {
|
|||
|
|
++ String string = super.toString();
|
|||
|
|
++ if (StringUtils.isNotBlank(string)) {
|
|||
|
|
++ string = string.substring(0, string.length() - 1) + ", mappingDetail=" + mappingDetail + "]";
|
|||
|
|
++ }
|
|||
|
|
++ return string;
|
|||
|
|
++ }
|
|||
|
|
+ }
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+From 2113fa371b9c2bf7c512f8ad234e51c616f1362c Mon Sep 17 00:00:00 2001
|
|||
|
|
+From: guyinyou <36399867+guyinyou@users.noreply.github.com>
|
|||
|
|
+Date: Fri, 13 Oct 2023 13:47:09 +0800
|
|||
|
|
+Subject: [PATCH 6/6] [ISSUE #7453] Fix the problem in constructing the
|
|||
|
|
+ GetMessageResult (#7456)
|
|||
|
|
+
|
|||
|
|
+* Fix the problem in constructing the GetMessageResult
|
|||
|
|
+
|
|||
|
|
+* Optimize the initialization size of GetMessageResult
|
|||
|
|
+---
|
|||
|
|
+ .../apache/rocketmq/broker/processor/PeekMessageProcessor.java | 3 +--
|
|||
|
|
+ .../apache/rocketmq/broker/processor/PopMessageProcessor.java | 3 +--
|
|||
|
|
+ 2 files changed, 2 insertions(+), 4 deletions(-)
|
|||
|
|
+
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java
|
|||
|
|
+index a8358c4ff..e1e0e13e5 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PeekMessageProcessor.java
|
|||
|
|
+@@ -129,8 +129,7 @@ public class PeekMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
+ }
|
|||
|
|
+ int randomQ = random.nextInt(100);
|
|||
|
|
+ int reviveQid = randomQ % this.brokerController.getBrokerConfig().getReviveQueueNum();
|
|||
|
|
+- int commercialSizePerMsg = this.brokerController.getBrokerConfig().getCommercialSizePerMsg();
|
|||
|
|
+- GetMessageResult getMessageResult = new GetMessageResult(commercialSizePerMsg);
|
|||
|
|
++ GetMessageResult getMessageResult = new GetMessageResult(requestHeader.getMaxMsgNums());
|
|||
|
|
+ boolean needRetry = randomQ % 5 == 0;
|
|||
|
|
+ long popTime = System.currentTimeMillis();
|
|||
|
|
+ long restNum = 0;
|
|||
|
|
+diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
+index 441f7de08..0d9bdf143 100644
|
|||
|
|
+--- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
++++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java
|
|||
|
|
+@@ -347,8 +347,7 @@ public class PopMessageProcessor implements NettyRequestProcessor {
|
|||
|
|
+ reviveQid = (int) Math.abs(ckMessageNumber.getAndIncrement() % this.brokerController.getBrokerConfig().getReviveQueueNum());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+- int commercialSizePerMsg = this.brokerController.getBrokerConfig().getCommercialSizePerMsg();
|
|||
|
|
+- GetMessageResult getMessageResult = new GetMessageResult(commercialSizePerMsg);
|
|||
|
|
++ GetMessageResult getMessageResult = new GetMessageResult(requestHeader.getMaxMsgNums());
|
|||
|
|
+ ExpressionMessageFilter finalMessageFilter = messageFilter;
|
|||
|
|
+ StringBuilder finalOrderCountInfo = orderCountInfo;
|
|||
|
|
+
|
|||
|
|
+--
|
|||
|
|
+2.32.0.windows.2
|
|||
|
|
+
|
|||
|
|
diff --git a/pom.xml b/pom.xml
|
|||
|
|
index 4202d4095..a3f7c2270 100644
|
|||
|
|
--- a/pom.xml
|
|||
|
|
+++ b/pom.xml
|
|||
|
|
@@ -137,7 +137,7 @@
|
|||
|
|
<opentelemetry-exporter-prometheus.version>1.29.0-alpha</opentelemetry-exporter-prometheus.version>
|
|||
|
|
<jul-to-slf4j.version>2.0.6</jul-to-slf4j.version>
|
|||
|
|
<s3.version>2.20.29</s3.version>
|
|||
|
|
- <rocksdb.version>1.0.3</rocksdb.version>
|
|||
|
|
+ <rocksdb.version>1.0.2</rocksdb.version>
|
|||
|
|
<jackson-databind.version>2.13.4.2</jackson-databind.version>
|
|||
|
|
|
|||
|
|
<!-- Test dependencies -->
|
|||
|
|
@@ -713,7 +713,7 @@
|
|||
|
|
<version>${slf4j-api.version}</version>
|
|||
|
|
</dependency>
|
|||
|
|
<dependency>
|
|||
|
|
- <groupId>io.github.aliyunmq</groupId>
|
|||
|
|
+ <groupId>org.apache.rocketmq</groupId>
|
|||
|
|
<artifactId>rocketmq-rocksdb</artifactId>
|
|||
|
|
<version>${rocksdb.version}</version>
|
|||
|
|
</dependency>
|
|||
|
|
diff --git a/store/pom.xml b/store/pom.xml
|
|||
|
|
index e979030e8..e1e616123 100644
|
|||
|
|
--- a/store/pom.xml
|
|||
|
|
+++ b/store/pom.xml
|
|||
|
|
@@ -58,6 +58,10 @@
|
|||
|
|
<groupId>com.google.guava</groupId>
|
|||
|
|
<artifactId>guava</artifactId>
|
|||
|
|
</dependency>
|
|||
|
|
+ <dependency>
|
|||
|
|
+ <groupId>commons-io</groupId>
|
|||
|
|
+ <artifactId>commons-io</artifactId>
|
|||
|
|
+ </dependency>
|
|||
|
|
<!-- Required by DLedger -->
|
|||
|
|
<dependency>
|
|||
|
|
<groupId>org.slf4j</groupId>
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
index f98e9a284..93102799b 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java
|
|||
|
|
@@ -60,6 +60,8 @@ import org.apache.rocketmq.store.ha.HAService;
|
|||
|
|
import org.apache.rocketmq.store.ha.autoswitch.AutoSwitchHAService;
|
|||
|
|
import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.util.LibC;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
import sun.nio.ch.DirectBuffer;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
@@ -299,8 +301,9 @@ public class CommitLog implements Swappable {
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* When the normal exit, data recovery, all memory data have been flush
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
- public void recoverNormally(long maxPhyOffsetOfConsumeQueue) {
|
|||
|
|
+ public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException {
|
|||
|
|
boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
|
|||
|
|
boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
|
|||
|
|
final List<MappedFile> mappedFiles = this.mappedFileQueue.getMappedFiles();
|
|||
|
|
@@ -369,21 +372,22 @@ public class CommitLog implements Swappable {
|
|||
|
|
this.setConfirmOffset(lastValidMsgPhyOffset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- this.mappedFileQueue.setFlushedWhere(processOffset);
|
|||
|
|
- this.mappedFileQueue.setCommittedWhere(processOffset);
|
|||
|
|
- this.mappedFileQueue.truncateDirtyFiles(processOffset);
|
|||
|
|
-
|
|||
|
|
// Clear ConsumeQueue redundant data
|
|||
|
|
if (maxPhyOffsetOfConsumeQueue >= processOffset) {
|
|||
|
|
log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
|
|||
|
|
this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ this.mappedFileQueue.setFlushedWhere(processOffset);
|
|||
|
|
+ this.mappedFileQueue.setCommittedWhere(processOffset);
|
|||
|
|
+ this.mappedFileQueue.truncateDirtyFiles(processOffset);
|
|||
|
|
} else {
|
|||
|
|
// Commitlog case files are deleted
|
|||
|
|
log.warn("The commitlog files are deleted, and delete the consume queue files");
|
|||
|
|
this.mappedFileQueue.setFlushedWhere(0);
|
|||
|
|
this.mappedFileQueue.setCommittedWhere(0);
|
|||
|
|
- this.defaultMessageStore.destroyLogics();
|
|||
|
|
+ this.defaultMessageStore.getQueueStore().destroy();
|
|||
|
|
+ this.defaultMessageStore.getQueueStore().loadAfterDestroy();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -626,8 +630,10 @@ public class CommitLog implements Swappable {
|
|||
|
|
return -1;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- @Deprecated
|
|||
|
|
- public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) {
|
|||
|
|
+ /**
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException {
|
|||
|
|
// recover by the minimum time stamp
|
|||
|
|
boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover();
|
|||
|
|
boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable();
|
|||
|
|
@@ -705,6 +711,9 @@ public class CommitLog implements Swappable {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ // only for rocksdb mode
|
|||
|
|
+ this.getMessageStore().finishCommitLogDispatch();
|
|||
|
|
+
|
|||
|
|
processOffset += mappedFileOffset;
|
|||
|
|
if (this.defaultMessageStore.getBrokerConfig().isEnableControllerMode()) {
|
|||
|
|
if (this.defaultMessageStore.getConfirmOffset() < this.defaultMessageStore.getMinPhyOffset()) {
|
|||
|
|
@@ -717,22 +726,24 @@ public class CommitLog implements Swappable {
|
|||
|
|
} else {
|
|||
|
|
this.setConfirmOffset(lastValidMsgPhyOffset);
|
|||
|
|
}
|
|||
|
|
- this.mappedFileQueue.setFlushedWhere(processOffset);
|
|||
|
|
- this.mappedFileQueue.setCommittedWhere(processOffset);
|
|||
|
|
- this.mappedFileQueue.truncateDirtyFiles(processOffset);
|
|||
|
|
|
|||
|
|
// Clear ConsumeQueue redundant data
|
|||
|
|
if (maxPhyOffsetOfConsumeQueue >= processOffset) {
|
|||
|
|
log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset);
|
|||
|
|
this.defaultMessageStore.truncateDirtyLogicFiles(processOffset);
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ this.mappedFileQueue.setFlushedWhere(processOffset);
|
|||
|
|
+ this.mappedFileQueue.setCommittedWhere(processOffset);
|
|||
|
|
+ this.mappedFileQueue.truncateDirtyFiles(processOffset);
|
|||
|
|
}
|
|||
|
|
// Commitlog case files are deleted
|
|||
|
|
else {
|
|||
|
|
log.warn("The commitlog files are deleted, and delete the consume queue files");
|
|||
|
|
this.mappedFileQueue.setFlushedWhere(0);
|
|||
|
|
this.mappedFileQueue.setCommittedWhere(0);
|
|||
|
|
- this.defaultMessageStore.destroyLogics();
|
|||
|
|
+ this.defaultMessageStore.getQueueStore().destroy();
|
|||
|
|
+ this.defaultMessageStore.getQueueStore().loadAfterDestroy();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -755,7 +766,7 @@ public class CommitLog implements Swappable {
|
|||
|
|
this.getMessageStore().onCommitLogAppend(msg, result, commitLogFile);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private boolean isMappedFileMatchedRecover(final MappedFile mappedFile) {
|
|||
|
|
+ private boolean isMappedFileMatchedRecover(final MappedFile mappedFile) throws RocksDBException {
|
|||
|
|
ByteBuffer byteBuffer = mappedFile.sliceByteBuffer();
|
|||
|
|
|
|||
|
|
int magicCode = byteBuffer.getInt(MessageDecoder.MESSAGE_MAGIC_CODE_POSITION);
|
|||
|
|
@@ -763,28 +774,37 @@ public class CommitLog implements Swappable {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- int sysFlag = byteBuffer.getInt(MessageDecoder.SYSFLAG_POSITION);
|
|||
|
|
- int bornhostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
|
|||
|
|
- int msgStoreTimePos = 4 + 4 + 4 + 4 + 4 + 8 + 8 + 4 + 8 + bornhostLength;
|
|||
|
|
- long storeTimestamp = byteBuffer.getLong(msgStoreTimePos);
|
|||
|
|
- if (0 == storeTimestamp) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable()
|
|||
|
|
- && this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe()) {
|
|||
|
|
- if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestampIndex()) {
|
|||
|
|
- log.info("find check timestamp, {} {}",
|
|||
|
|
- storeTimestamp,
|
|||
|
|
- UtilAll.timeMillisToHumanString(storeTimestamp));
|
|||
|
|
+ if (this.defaultMessageStore.getMessageStoreConfig().isEnableRocksDBStore()) {
|
|||
|
|
+ final long maxPhyOffsetInConsumeQueue = this.defaultMessageStore.getQueueStore().getMaxPhyOffsetInConsumeQueue();
|
|||
|
|
+ long phyOffset = byteBuffer.getLong(MessageDecoder.MESSAGE_PHYSIC_OFFSET_POSITION);
|
|||
|
|
+ if (phyOffset <= maxPhyOffsetInConsumeQueue) {
|
|||
|
|
+ log.info("find check. beginPhyOffset: {}, maxPhyOffsetInConsumeQueue: {}", phyOffset, maxPhyOffsetInConsumeQueue);
|
|||
|
|
return true;
|
|||
|
|
}
|
|||
|
|
} else {
|
|||
|
|
- if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestamp()) {
|
|||
|
|
- log.info("find check timestamp, {} {}",
|
|||
|
|
- storeTimestamp,
|
|||
|
|
- UtilAll.timeMillisToHumanString(storeTimestamp));
|
|||
|
|
- return true;
|
|||
|
|
+ int sysFlag = byteBuffer.getInt(MessageDecoder.SYSFLAG_POSITION);
|
|||
|
|
+ int bornHostLength = (sysFlag & MessageSysFlag.BORNHOST_V6_FLAG) == 0 ? 8 : 20;
|
|||
|
|
+ int msgStoreTimePos = 4 + 4 + 4 + 4 + 4 + 8 + 8 + 4 + 8 + bornHostLength;
|
|||
|
|
+ long storeTimestamp = byteBuffer.getLong(msgStoreTimePos);
|
|||
|
|
+ if (0 == storeTimestamp) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (this.defaultMessageStore.getMessageStoreConfig().isMessageIndexEnable()
|
|||
|
|
+ && this.defaultMessageStore.getMessageStoreConfig().isMessageIndexSafe()) {
|
|||
|
|
+ if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestampIndex()) {
|
|||
|
|
+ log.info("find check timestamp, {} {}",
|
|||
|
|
+ storeTimestamp,
|
|||
|
|
+ UtilAll.timeMillisToHumanString(storeTimestamp));
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+ if (storeTimestamp <= this.defaultMessageStore.getStoreCheckpoint().getMinTimestamp()) {
|
|||
|
|
+ log.info("find check timestamp, {} {}",
|
|||
|
|
+ storeTimestamp,
|
|||
|
|
+ UtilAll.timeMillisToHumanString(storeTimestamp));
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -958,8 +978,6 @@ public class CommitLog implements Swappable {
|
|||
|
|
beginTimeInLock = 0;
|
|||
|
|
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, result));
|
|||
|
|
case UNKNOWN_ERROR:
|
|||
|
|
- beginTimeInLock = 0;
|
|||
|
|
- return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
|
|||
|
|
default:
|
|||
|
|
beginTimeInLock = 0;
|
|||
|
|
return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
|
|||
|
|
@@ -974,6 +992,8 @@ public class CommitLog implements Swappable {
|
|||
|
|
if (AppendMessageStatus.PUT_OK.equals(result.getStatus())) {
|
|||
|
|
this.defaultMessageStore.increaseOffset(msg, getMessageNum(msg));
|
|||
|
|
}
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
|
|||
|
|
} finally {
|
|||
|
|
topicQueueLock.unlock(topicQueueKey);
|
|||
|
|
}
|
|||
|
|
@@ -997,7 +1017,7 @@ public class CommitLog implements Swappable {
|
|||
|
|
|
|||
|
|
public CompletableFuture<PutMessageResult> asyncPutMessages(final MessageExtBatch messageExtBatch) {
|
|||
|
|
messageExtBatch.setStoreTimestamp(System.currentTimeMillis());
|
|||
|
|
- AppendMessageResult result;
|
|||
|
|
+ AppendMessageResult result = null;
|
|||
|
|
|
|||
|
|
StoreStatsService storeStatsService = this.defaultMessageStore.getStoreStatsService();
|
|||
|
|
|
|||
|
|
@@ -1133,7 +1153,9 @@ public class CommitLog implements Swappable {
|
|||
|
|
if (AppendMessageStatus.PUT_OK.equals(result.getStatus())) {
|
|||
|
|
this.defaultMessageStore.increaseOffset(messageExtBatch, (short) putMessageContext.getBatchSize());
|
|||
|
|
}
|
|||
|
|
- } finally {
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, result));
|
|||
|
|
+ } finally {
|
|||
|
|
topicQueueLock.unlock(topicQueueKey);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java
|
|||
|
|
index 9d6fa6ad9..f3a7b7c5c 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/CommitLogDispatcher.java
|
|||
|
|
@@ -17,6 +17,8 @@
|
|||
|
|
|
|||
|
|
package org.apache.rocketmq.store;
|
|||
|
|
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
/**
|
|||
|
|
* Dispatcher of commit log.
|
|||
|
|
*/
|
|||
|
|
@@ -25,6 +27,7 @@ public interface CommitLogDispatcher {
|
|||
|
|
/**
|
|||
|
|
* Dispatch messages from store to build consume queues, indexes, and filter data
|
|||
|
|
* @param request dispatch message request
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
- void dispatch(final DispatchRequest request);
|
|||
|
|
+ void dispatch(final DispatchRequest request) throws RocksDBException;
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
index 56bee2af3..623509c8b 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java
|
|||
|
|
@@ -24,13 +24,12 @@ import java.util.Map;
|
|||
|
|
import org.apache.commons.lang3.StringUtils;
|
|||
|
|
import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageAccessor;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
-import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
-import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
@@ -39,9 +38,9 @@ import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
import org.apache.rocketmq.store.queue.CqUnit;
|
|||
|
|
import org.apache.rocketmq.store.queue.FileQueueLifeCycle;
|
|||
|
|
+import org.apache.rocketmq.store.queue.MultiDispatch;
|
|||
|
|
import org.apache.rocketmq.store.queue.QueueOffsetOperator;
|
|||
|
|
import org.apache.rocketmq.store.queue.ReferredIterator;
|
|||
|
|
-import org.apache.rocketmq.store.timer.TimerMessageStore;
|
|||
|
|
|
|||
|
|
public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
@@ -703,7 +702,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
this.messageStore.getStoreCheckpoint().setPhysicMsgTimestamp(request.getStoreTimestamp());
|
|||
|
|
}
|
|||
|
|
this.messageStore.getStoreCheckpoint().setLogicsMsgTimestamp(request.getStoreTimestamp());
|
|||
|
|
- if (checkMultiDispatchQueue(request)) {
|
|||
|
|
+ if (MultiDispatch.checkMultiDispatchQueue(this.messageStore.getMessageStoreConfig(), request)) {
|
|||
|
|
multiDispatchLmqQueue(request, maxRetries);
|
|||
|
|
}
|
|||
|
|
return;
|
|||
|
|
@@ -725,25 +724,6 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
this.messageStore.getRunningFlags().makeLogicsQueueError();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private boolean checkMultiDispatchQueue(DispatchRequest dispatchRequest) {
|
|||
|
|
- if (!this.messageStore.getMessageStoreConfig().isEnableMultiDispatch()
|
|||
|
|
- || dispatchRequest.getTopic().startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)
|
|||
|
|
- || dispatchRequest.getTopic().equals(TimerMessageStore.TIMER_TOPIC)
|
|||
|
|
- || dispatchRequest.getTopic().equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC)) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
- Map<String, String> prop = dispatchRequest.getPropertiesMap();
|
|||
|
|
- if (prop == null || prop.isEmpty()) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
- String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
- String multiQueueOffset = prop.get(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET);
|
|||
|
|
- if (StringUtils.isBlank(multiDispatchQueue) || StringUtils.isBlank(multiQueueOffset)) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
- return true;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
private void multiDispatchLmqQueue(DispatchRequest request, int maxRetries) {
|
|||
|
|
Map<String, String> prop = request.getPropertiesMap();
|
|||
|
|
String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
@@ -765,9 +745,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
queueId = 0;
|
|||
|
|
}
|
|||
|
|
doDispatchLmqQueue(request, maxRetries, queueName, queueOffset, queueId);
|
|||
|
|
-
|
|||
|
|
}
|
|||
|
|
- return;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
private void doDispatchLmqQueue(DispatchRequest request, int maxRetries, String queueName, long queueOffset,
|
|||
|
|
@@ -802,7 +780,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
|
|||
|
|
// Handling the multi dispatch message. In the context of a light message queue (as defined in RIP-28),
|
|||
|
|
// light message queues are constructed based on message properties, which requires special handling of queue offset of the light message queue.
|
|||
|
|
- if (!isNeedHandleMultiDispatch(msg)) {
|
|||
|
|
+ if (!MultiDispatch.isNeedHandleMultiDispatch(this.messageStore.getMessageStoreConfig(), msg.getTopic())) {
|
|||
|
|
return;
|
|||
|
|
}
|
|||
|
|
String multiDispatchQueue = msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
@@ -812,14 +790,14 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER);
|
|||
|
|
Long[] queueOffsets = new Long[queues.length];
|
|||
|
|
for (int i = 0; i < queues.length; i++) {
|
|||
|
|
- String key = queueKey(queues[i], msg);
|
|||
|
|
- if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(key)) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queues[i])) {
|
|||
|
|
+ String key = MultiDispatch.lmqQueueKey(queues[i]);
|
|||
|
|
queueOffsets[i] = queueOffsetOperator.getLmqOffset(key);
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET,
|
|||
|
|
StringUtils.join(queueOffsets, MixAll.MULTI_DISPATCH_QUEUE_SPLITTER));
|
|||
|
|
- removeWaitStorePropertyString(msg);
|
|||
|
|
+ msg.removeWaitStorePropertyString();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
@@ -830,7 +808,7 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
|
|||
|
|
// Handling the multi dispatch message. In the context of a light message queue (as defined in RIP-28),
|
|||
|
|
// light message queues are constructed based on message properties, which requires special handling of queue offset of the light message queue.
|
|||
|
|
- if (!isNeedHandleMultiDispatch(msg)) {
|
|||
|
|
+ if (!MultiDispatch.isNeedHandleMultiDispatch(this.messageStore.getMessageStoreConfig(), msg.getTopic())) {
|
|||
|
|
return;
|
|||
|
|
}
|
|||
|
|
String multiDispatchQueue = msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
@@ -839,45 +817,13 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
}
|
|||
|
|
String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER);
|
|||
|
|
for (int i = 0; i < queues.length; i++) {
|
|||
|
|
- String key = queueKey(queues[i], msg);
|
|||
|
|
- if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(key)) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queues[i])) {
|
|||
|
|
+ String key = MultiDispatch.lmqQueueKey(queues[i]);
|
|||
|
|
queueOffsetOperator.increaseLmqOffset(key, (short) 1);
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public boolean isNeedHandleMultiDispatch(MessageExtBrokerInner msg) {
|
|||
|
|
- return messageStore.getMessageStoreConfig().isEnableMultiDispatch()
|
|||
|
|
- && !msg.getTopic().startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)
|
|||
|
|
- && !msg.getTopic().equals(TimerMessageStore.TIMER_TOPIC)
|
|||
|
|
- && !msg.getTopic().equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public String queueKey(String queueName, MessageExtBrokerInner msgInner) {
|
|||
|
|
- StringBuilder keyBuilder = new StringBuilder();
|
|||
|
|
- keyBuilder.append(queueName);
|
|||
|
|
- keyBuilder.append('-');
|
|||
|
|
- int queueId = msgInner.getQueueId();
|
|||
|
|
- if (messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queueName)) {
|
|||
|
|
- queueId = 0;
|
|||
|
|
- }
|
|||
|
|
- keyBuilder.append(queueId);
|
|||
|
|
- return keyBuilder.toString();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- private void removeWaitStorePropertyString(MessageExtBrokerInner msgInner) {
|
|||
|
|
- if (msgInner.getProperties().containsKey(MessageConst.PROPERTY_WAIT_STORE_MSG_OK)) {
|
|||
|
|
- // There is no need to store "WAIT=true", remove it from propertiesString to save 9 bytes for each message.
|
|||
|
|
- // It works for most case. In some cases msgInner.setPropertiesString invoked later and replace it.
|
|||
|
|
- String waitStoreMsgOKValue = msgInner.getProperties().remove(MessageConst.PROPERTY_WAIT_STORE_MSG_OK);
|
|||
|
|
- msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
|
|||
|
|
- // Reput to properties, since msgInner.isWaitStoreMsgOK() will be invoked later
|
|||
|
|
- msgInner.getProperties().put(MessageConst.PROPERTY_WAIT_STORE_MSG_OK, waitStoreMsgOKValue);
|
|||
|
|
- } else {
|
|||
|
|
- msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties()));
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode,
|
|||
|
|
final long cqOffset) {
|
|||
|
|
|
|||
|
|
@@ -965,6 +911,11 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
return new ConsumeQueueIterator(sbr);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public ReferredIterator<CqUnit> iterateFrom(long startIndex, int count) {
|
|||
|
|
+ return iterateFrom(startIndex);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public CqUnit get(long offset) {
|
|||
|
|
ReferredIterator<CqUnit> it = iterateFrom(offset);
|
|||
|
|
@@ -974,6 +925,20 @@ public class ConsumeQueue implements ConsumeQueueInterface, FileQueueLifeCycle {
|
|||
|
|
return it.nextAndRelease();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getCqUnitAndStoreTime(long index) {
|
|||
|
|
+ CqUnit cqUnit = get(index);
|
|||
|
|
+ Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit);
|
|||
|
|
+ return new Pair<>(cqUnit, messageStoreTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getEarliestUnitAndStoreTime() {
|
|||
|
|
+ CqUnit cqUnit = getEarliestUnit();
|
|||
|
|
+ Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit);
|
|||
|
|
+ return new Pair<>(cqUnit, messageStoreTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public CqUnit getEarliestUnit() {
|
|||
|
|
/**
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
index 02ea47f13..99a54e2d7 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
|
|||
|
|
@@ -105,32 +105,35 @@ import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager;
|
|||
|
|
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
import org.apache.rocketmq.store.queue.ConsumeQueueStore;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
|
|||
|
|
import org.apache.rocketmq.store.queue.CqUnit;
|
|||
|
|
import org.apache.rocketmq.store.queue.ReferredIterator;
|
|||
|
|
import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
import org.apache.rocketmq.store.timer.TimerMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.util.PerfCounter;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
public class DefaultMessageStore implements MessageStore {
|
|||
|
|
- private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+ protected static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+ protected static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
|
|||
|
|
|
|||
|
|
public final PerfCounter.Ticks perfs = new PerfCounter.Ticks(LOGGER);
|
|||
|
|
|
|||
|
|
private final MessageStoreConfig messageStoreConfig;
|
|||
|
|
// CommitLog
|
|||
|
|
- private final CommitLog commitLog;
|
|||
|
|
+ protected final CommitLog commitLog;
|
|||
|
|
|
|||
|
|
- private final ConsumeQueueStore consumeQueueStore;
|
|||
|
|
+ protected final ConsumeQueueStoreInterface consumeQueueStore;
|
|||
|
|
|
|||
|
|
private final FlushConsumeQueueService flushConsumeQueueService;
|
|||
|
|
|
|||
|
|
- private final CleanCommitLogService cleanCommitLogService;
|
|||
|
|
+ protected final CleanCommitLogService cleanCommitLogService;
|
|||
|
|
|
|||
|
|
private final CleanConsumeQueueService cleanConsumeQueueService;
|
|||
|
|
|
|||
|
|
private final CorrectLogicOffsetService correctLogicOffsetService;
|
|||
|
|
|
|||
|
|
- private final IndexService indexService;
|
|||
|
|
+ protected final IndexService indexService;
|
|||
|
|
|
|||
|
|
private final AllocateMappedFileService allocateMappedFileService;
|
|||
|
|
|
|||
|
|
@@ -147,7 +150,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
private final TransientStorePool transientStorePool;
|
|||
|
|
|
|||
|
|
- private final RunningFlags runningFlags = new RunningFlags();
|
|||
|
|
+ protected final RunningFlags runningFlags = new RunningFlags();
|
|||
|
|
private final SystemClock systemClock = new SystemClock();
|
|||
|
|
|
|||
|
|
private final ScheduledExecutorService scheduledExecutorService;
|
|||
|
|
@@ -156,6 +159,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
private final BrokerConfig brokerConfig;
|
|||
|
|
|
|||
|
|
private volatile boolean shutdown = true;
|
|||
|
|
+ protected boolean notifyMessageArriveInBatch = false;
|
|||
|
|
|
|||
|
|
private StoreCheckpoint storeCheckpoint;
|
|||
|
|
private TimerMessageStore timerMessageStore;
|
|||
|
|
@@ -182,7 +186,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
private volatile long brokerInitMaxOffset = -1L;
|
|||
|
|
|
|||
|
|
- protected List<PutMessageHook> putMessageHookList = new ArrayList<>();
|
|||
|
|
+ private List<PutMessageHook> putMessageHookList = new ArrayList<>();
|
|||
|
|
|
|||
|
|
private SendMessageBackHook sendMessageBackHook;
|
|||
|
|
|
|||
|
|
@@ -222,12 +226,12 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
this.commitLog = new CommitLog(this);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- this.consumeQueueStore = new ConsumeQueueStore(this, this.messageStoreConfig);
|
|||
|
|
+ this.consumeQueueStore = createConsumeQueueStore();
|
|||
|
|
|
|||
|
|
- this.flushConsumeQueueService = new FlushConsumeQueueService();
|
|||
|
|
+ this.flushConsumeQueueService = createFlushConsumeQueueService();
|
|||
|
|
this.cleanCommitLogService = new CleanCommitLogService();
|
|||
|
|
- this.cleanConsumeQueueService = new CleanConsumeQueueService();
|
|||
|
|
- this.correctLogicOffsetService = new CorrectLogicOffsetService();
|
|||
|
|
+ this.cleanConsumeQueueService = createCleanConsumeQueueService();
|
|||
|
|
+ this.correctLogicOffsetService = createCorrectLogicOffsetService();
|
|||
|
|
this.storeStatsService = new StoreStatsService(getBrokerIdentity());
|
|||
|
|
this.indexService = new IndexService(this);
|
|||
|
|
|
|||
|
|
@@ -273,6 +277,22 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
parseDelayLevel();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public ConsumeQueueStoreInterface createConsumeQueueStore() {
|
|||
|
|
+ return new ConsumeQueueStore(this);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CleanConsumeQueueService createCleanConsumeQueueService() {
|
|||
|
|
+ return new CleanConsumeQueueService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public FlushConsumeQueueService createFlushConsumeQueueService() {
|
|||
|
|
+ return new FlushConsumeQueueService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public CorrectLogicOffsetService createCorrectLogicOffsetService() {
|
|||
|
|
+ return new CorrectLogicOffsetService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public boolean parseDelayLevel() {
|
|||
|
|
HashMap<String, Long> timeUnitTable = new HashMap<>();
|
|||
|
|
timeUnitTable.put("s", 1000L);
|
|||
|
|
@@ -305,7 +325,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void truncateDirtyLogicFiles(long phyOffset) {
|
|||
|
|
+ public void truncateDirtyLogicFiles(long phyOffset) throws RocksDBException {
|
|||
|
|
this.consumeQueueStore.truncateDirty(phyOffset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -393,6 +413,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
this.flushConsumeQueueService.start();
|
|||
|
|
this.commitLog.start();
|
|||
|
|
+ this.consumeQueueStore.start();
|
|||
|
|
this.storeStatsService.start();
|
|||
|
|
|
|||
|
|
if (this.haService != null) {
|
|||
|
|
@@ -481,6 +502,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
this.storeStatsService.shutdown();
|
|||
|
|
this.commitLog.shutdown();
|
|||
|
|
this.reputMessageService.shutdown();
|
|||
|
|
+ this.consumeQueueStore.shutdown();
|
|||
|
|
// dispatch-related services must be shut down after reputMessageService
|
|||
|
|
this.indexService.shutdown();
|
|||
|
|
if (this.compactionService != null) {
|
|||
|
|
@@ -515,7 +537,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
public void destroy() {
|
|||
|
|
- this.destroyLogics();
|
|||
|
|
+ this.consumeQueueStore.destroy();
|
|||
|
|
this.commitLog.destroy();
|
|||
|
|
this.indexService.destroy();
|
|||
|
|
this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir()));
|
|||
|
|
@@ -541,11 +563,6 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return commitLogSize + consumeQueueSize + indexFileSize;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- @Override
|
|||
|
|
- public void destroyLogics() {
|
|||
|
|
- this.consumeQueueStore.destroy();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
@Override
|
|||
|
|
public CompletableFuture<PutMessageResult> asyncPutMessage(MessageExtBrokerInner msg) {
|
|||
|
|
|
|||
|
|
@@ -687,7 +704,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return commitLog;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void truncateDirtyFiles(long offsetToTruncate) {
|
|||
|
|
+ public void truncateDirtyFiles(long offsetToTruncate) throws RocksDBException {
|
|||
|
|
|
|||
|
|
LOGGER.info("truncate dirty files to {}", offsetToTruncate);
|
|||
|
|
|
|||
|
|
@@ -700,12 +717,12 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
long oldReputFromOffset = this.reputMessageService.getReputFromOffset();
|
|||
|
|
|
|||
|
|
- // truncate commitLog
|
|||
|
|
- this.commitLog.truncateDirtyFiles(offsetToTruncate);
|
|||
|
|
-
|
|||
|
|
// truncate consume queue
|
|||
|
|
this.truncateDirtyLogicFiles(offsetToTruncate);
|
|||
|
|
|
|||
|
|
+ // truncate commitLog
|
|||
|
|
+ this.commitLog.truncateDirtyFiles(offsetToTruncate);
|
|||
|
|
+
|
|||
|
|
this.recoverTopicQueueTable();
|
|||
|
|
|
|||
|
|
if (!messageStoreConfig.isEnableBuildConsumeQueueConcurrently()) {
|
|||
|
|
@@ -723,7 +740,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public boolean truncateFiles(long offsetToTruncate) {
|
|||
|
|
+ public boolean truncateFiles(long offsetToTruncate) throws RocksDBException {
|
|||
|
|
if (offsetToTruncate >= this.getMaxPhyOffset()) {
|
|||
|
|
LOGGER.info("no need to truncate files, truncate offset is {}, max physical offset is {}", offsetToTruncate, this.getMaxPhyOffset());
|
|||
|
|
return true;
|
|||
|
|
@@ -825,17 +842,19 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
while (getResult.getBufferTotalSize() <= 0
|
|||
|
|
&& nextBeginOffset < maxOffset
|
|||
|
|
&& cqFileNum++ < this.messageStoreConfig.getTravelCqFileNumWhenGetMessage()) {
|
|||
|
|
- ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(nextBeginOffset);
|
|||
|
|
-
|
|||
|
|
- if (bufferConsumeQueue == null) {
|
|||
|
|
- status = GetMessageStatus.OFFSET_FOUND_NULL;
|
|||
|
|
- nextBeginOffset = nextOffsetCorrection(nextBeginOffset, this.consumeQueueStore.rollNextFile(consumeQueue, nextBeginOffset));
|
|||
|
|
- LOGGER.warn("consumer request topic: " + topic + "offset: " + offset + " minOffset: " + minOffset + " maxOffset: "
|
|||
|
|
- + maxOffset + ", but access logic queue failed. Correct nextBeginOffset to " + nextBeginOffset);
|
|||
|
|
- break;
|
|||
|
|
- }
|
|||
|
|
+ ReferredIterator<CqUnit> bufferConsumeQueue = null;
|
|||
|
|
|
|||
|
|
try {
|
|||
|
|
+ bufferConsumeQueue = consumeQueue.iterateFrom(nextBeginOffset, maxMsgNums);
|
|||
|
|
+
|
|||
|
|
+ if (bufferConsumeQueue == null) {
|
|||
|
|
+ status = GetMessageStatus.OFFSET_FOUND_NULL;
|
|||
|
|
+ nextBeginOffset = nextOffsetCorrection(nextBeginOffset, this.consumeQueueStore.rollNextFile(consumeQueue, nextBeginOffset));
|
|||
|
|
+ LOGGER.warn("consumer request topic: " + topic + ", offset: " + offset + ", minOffset: " + minOffset + ", maxOffset: "
|
|||
|
|
+ + maxOffset + ", but access logic queue failed. Correct nextBeginOffset to " + nextBeginOffset);
|
|||
|
|
+ break;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
long nextPhyFileStartOffset = Long.MIN_VALUE;
|
|||
|
|
while (bufferConsumeQueue.hasNext()
|
|||
|
|
&& nextBeginOffset < maxOffset) {
|
|||
|
|
@@ -905,8 +924,13 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
status = GetMessageStatus.FOUND;
|
|||
|
|
nextPhyFileStartOffset = Long.MIN_VALUE;
|
|||
|
|
}
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getMessage Failed. cid: {}, topic: {}, queueId: {}, offset: {}, minOffset: {}, maxOffset: {}, {}",
|
|||
|
|
+ group, topic, queueId, offset, minOffset, maxOffset, e.getMessage());
|
|||
|
|
} finally {
|
|||
|
|
- bufferConsumeQueue.release();
|
|||
|
|
+ if (bufferConsumeQueue != null) {
|
|||
|
|
+ bufferConsumeQueue.release();
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -975,12 +999,12 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
public long getMinOffsetInQueue(String topic, int queueId) {
|
|||
|
|
- ConsumeQueueInterface logic = this.findConsumeQueue(topic, queueId);
|
|||
|
|
- if (logic != null) {
|
|||
|
|
- return logic.getMinOffsetInQueue();
|
|||
|
|
+ try {
|
|||
|
|
+ return this.consumeQueueStore.getMinOffsetInQueue(topic, queueId);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getMinOffsetInQueue Failed. topic: {}, queueId: {}", topic, queueId, e);
|
|||
|
|
+ return -1;
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
- return -1;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
@@ -997,38 +1021,27 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) {
|
|||
|
|
ConsumeQueueInterface consumeQueue = findConsumeQueue(topic, queueId);
|
|||
|
|
if (consumeQueue != null) {
|
|||
|
|
-
|
|||
|
|
- ReferredIterator<CqUnit> bufferConsumeQueue = consumeQueue.iterateFrom(consumeQueueOffset);
|
|||
|
|
- if (bufferConsumeQueue != null) {
|
|||
|
|
- try {
|
|||
|
|
- if (bufferConsumeQueue.hasNext()) {
|
|||
|
|
- long offsetPy = bufferConsumeQueue.next().getPos();
|
|||
|
|
- return offsetPy;
|
|||
|
|
- }
|
|||
|
|
- } finally {
|
|||
|
|
- bufferConsumeQueue.release();
|
|||
|
|
- }
|
|||
|
|
+ CqUnit cqUnit = consumeQueue.get(consumeQueueOffset);
|
|||
|
|
+ if (cqUnit != null) {
|
|||
|
|
+ return cqUnit.getPos();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
return 0;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) {
|
|||
|
|
- return getOffsetInQueueByTime(topic, queueId, timestamp, BoundaryType.LOWER);
|
|||
|
|
+ return this.getOffsetInQueueByTime(topic, queueId, timestamp, BoundaryType.LOWER);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) {
|
|||
|
|
- ConsumeQueueInterface logic = this.findConsumeQueue(topic, queueId);
|
|||
|
|
- if (logic != null) {
|
|||
|
|
- long resultOffset = logic.getOffsetInQueueByTime(timestamp, boundaryType);
|
|||
|
|
- // Make sure the result offset is in valid range.
|
|||
|
|
- resultOffset = Math.max(resultOffset, logic.getMinOffsetInQueue());
|
|||
|
|
- resultOffset = Math.min(resultOffset, logic.getMaxOffsetInQueue());
|
|||
|
|
- return resultOffset;
|
|||
|
|
+ try {
|
|||
|
|
+ return this.consumeQueueStore.getOffsetInQueueByTime(topic, queueId, timestamp, boundaryType);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getOffsetInQueueByTime Failed. topic: {}, queueId: {}, timestamp: {} boundaryType: {}, {}",
|
|||
|
|
+ topic, queueId, timestamp, boundaryType, e.getMessage());
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
return 0;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -1088,6 +1101,10 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return StorePathConfigHelper.getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir());
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public MessageArrivingListener getMessageArrivingListener() {
|
|||
|
|
+ return messageArrivingListener;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public HashMap<String, String> getRuntimeInfo() {
|
|||
|
|
HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
|
|||
|
|
@@ -1121,7 +1138,6 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return this.commitLog.getMaxOffset();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
-
|
|||
|
|
@Override
|
|||
|
|
public long getMinPhyOffset() {
|
|||
|
|
return this.commitLog.getMinOffset();
|
|||
|
|
@@ -1141,7 +1157,10 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
public long getEarliestMessageTime(String topic, int queueId) {
|
|||
|
|
ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
|
|||
|
|
if (logicQueue != null) {
|
|||
|
|
- return getStoreTime(logicQueue.getEarliestUnit());
|
|||
|
|
+ Pair<CqUnit, Long> pair = logicQueue.getEarliestUnitAndStoreTime();
|
|||
|
|
+ if (pair != null && pair.getObject2() != null) {
|
|||
|
|
+ return pair.getObject2();
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
return -1;
|
|||
|
|
@@ -1152,19 +1171,6 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return CompletableFuture.completedFuture(getEarliestMessageTime(topic, queueId));
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- protected long getStoreTime(CqUnit result) {
|
|||
|
|
- if (result != null) {
|
|||
|
|
- try {
|
|||
|
|
- final long phyOffset = result.getPos();
|
|||
|
|
- final int size = result.getSize();
|
|||
|
|
- long storeTime = this.getCommitLog().pickupStoreTimestamp(phyOffset, size);
|
|||
|
|
- return storeTime;
|
|||
|
|
- } catch (Exception e) {
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- return -1;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
@Override
|
|||
|
|
public long getEarliestMessageTime() {
|
|||
|
|
long minPhyOffset = this.getMinPhyOffset();
|
|||
|
|
@@ -1179,13 +1185,16 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
public long getMessageStoreTimeStamp(String topic, int queueId, long consumeQueueOffset) {
|
|||
|
|
ConsumeQueueInterface logicQueue = this.findConsumeQueue(topic, queueId);
|
|||
|
|
if (logicQueue != null) {
|
|||
|
|
- return getStoreTime(logicQueue.get(consumeQueueOffset));
|
|||
|
|
+ Pair<CqUnit, Long> pair = logicQueue.getCqUnitAndStoreTime(consumeQueueOffset);
|
|||
|
|
+ if (pair != null && pair.getObject2() != null) {
|
|||
|
|
+ return pair.getObject2();
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
return -1;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- @Override public CompletableFuture<Long> getMessageStoreTimeStampAsync(String topic, int queueId,
|
|||
|
|
+ @Override
|
|||
|
|
+ public CompletableFuture<Long> getMessageStoreTimeStampAsync(String topic, int queueId,
|
|||
|
|
long consumeQueueOffset) {
|
|||
|
|
return CompletableFuture.completedFuture(getMessageStoreTimeStamp(topic, queueId, consumeQueueOffset));
|
|||
|
|
}
|
|||
|
|
@@ -1354,6 +1363,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
* If offset table is cleaned, and old messages are dispatching after the old consume queue is cleaned,
|
|||
|
|
* consume queue will be created with old offset, then later message with new offset table can not be
|
|||
|
|
* dispatched to consume queue.
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
@Override
|
|||
|
|
public int deleteTopics(final Set<String> deleteTopics) {
|
|||
|
|
@@ -1363,17 +1373,19 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
int deleteCount = 0;
|
|||
|
|
for (String topic : deleteTopics) {
|
|||
|
|
- ConcurrentMap<Integer, ConsumeQueueInterface> queueTable =
|
|||
|
|
- this.consumeQueueStore.getConsumeQueueTable().get(topic);
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> queueTable = this.consumeQueueStore.findConsumeQueueMap(topic);
|
|||
|
|
|
|||
|
|
if (queueTable == null || queueTable.isEmpty()) {
|
|||
|
|
continue;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
for (ConsumeQueueInterface cq : queueTable.values()) {
|
|||
|
|
- this.consumeQueueStore.destroy(cq);
|
|||
|
|
- LOGGER.info("DeleteTopic: ConsumeQueue has been cleaned, topic={}, queueId={}",
|
|||
|
|
- cq.getTopic(), cq.getQueueId());
|
|||
|
|
+ try {
|
|||
|
|
+ this.consumeQueueStore.destroy(cq);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ LOGGER.error("DeleteTopic: ConsumeQueue cleans error!, topic={}, queueId={}", cq.getTopic(), cq.getQueueId(), e);
|
|||
|
|
+ }
|
|||
|
|
+ LOGGER.info("DeleteTopic: ConsumeQueue has been cleaned, topic={}, queueId={}", cq.getTopic(), cq.getQueueId());
|
|||
|
|
this.consumeQueueStore.removeTopicQueueTable(cq.getTopic(), cq.getQueueId());
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -1852,14 +1864,18 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return file.exists();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private void recover(final boolean lastExitOK) {
|
|||
|
|
- boolean recoverConcurrently = this.brokerConfig.isRecoverConcurrently();
|
|||
|
|
+ private boolean isRecoverConcurrently() {
|
|||
|
|
+ return this.brokerConfig.isRecoverConcurrently() && !this.messageStoreConfig.isEnableRocksDBStore();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void recover(final boolean lastExitOK) throws RocksDBException {
|
|||
|
|
+ boolean recoverConcurrently = this.isRecoverConcurrently();
|
|||
|
|
LOGGER.info("message store recover mode: {}", recoverConcurrently ? "concurrent" : "normal");
|
|||
|
|
|
|||
|
|
// recover consume queue
|
|||
|
|
long recoverConsumeQueueStart = System.currentTimeMillis();
|
|||
|
|
this.recoverConsumeQueue();
|
|||
|
|
- long maxPhyOffsetOfConsumeQueue = this.getMaxOffsetInConsumeQueue();
|
|||
|
|
+ long maxPhyOffsetOfConsumeQueue = this.consumeQueueStore.getMaxPhyOffsetInConsumeQueue();
|
|||
|
|
long recoverConsumeQueueEnd = System.currentTimeMillis();
|
|||
|
|
|
|||
|
|
// recover commitlog
|
|||
|
|
@@ -1894,23 +1910,25 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return messageStoreConfig;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public void finishCommitLogDispatch() {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public TransientStorePool getTransientStorePool() {
|
|||
|
|
return transientStorePool;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
private void recoverConsumeQueue() {
|
|||
|
|
- if (!this.brokerConfig.isRecoverConcurrently()) {
|
|||
|
|
+ if (!this.isRecoverConcurrently()) {
|
|||
|
|
this.consumeQueueStore.recover();
|
|||
|
|
} else {
|
|||
|
|
this.consumeQueueStore.recoverConcurrently();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private long getMaxOffsetInConsumeQueue() {
|
|||
|
|
- return this.consumeQueueStore.getMaxOffsetInConsumeQueue();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
+ @Override
|
|||
|
|
public void recoverTopicQueueTable() {
|
|||
|
|
long minPhyOffset = this.commitLog.getMinOffset();
|
|||
|
|
this.consumeQueueStore.recoverOffsetTable(minPhyOffset);
|
|||
|
|
@@ -1949,13 +1967,17 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return runningFlags;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void doDispatch(DispatchRequest req) {
|
|||
|
|
+ public void doDispatch(DispatchRequest req) throws RocksDBException {
|
|||
|
|
for (CommitLogDispatcher dispatcher : this.dispatcherList) {
|
|||
|
|
dispatcher.dispatch(req);
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void putMessagePositionInfo(DispatchRequest dispatchRequest) {
|
|||
|
|
+ /**
|
|||
|
|
+ * @param dispatchRequest
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ protected void putMessagePositionInfo(DispatchRequest dispatchRequest) throws RocksDBException {
|
|||
|
|
this.consumeQueueStore.putMessagePositionInfoWrapper(dispatchRequest);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -2054,7 +2076,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public ConsumeQueueStore getQueueStore() {
|
|||
|
|
+ public ConsumeQueueStoreInterface getQueueStore() {
|
|||
|
|
return consumeQueueStore;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -2065,7 +2087,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile,
|
|||
|
|
- boolean isRecover, boolean isFileEnd) {
|
|||
|
|
+ boolean isRecover, boolean isFileEnd) throws RocksDBException {
|
|||
|
|
if (doDispatch && !isFileEnd) {
|
|||
|
|
this.doDispatch(dispatchRequest);
|
|||
|
|
}
|
|||
|
|
@@ -2082,7 +2104,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void assignOffset(MessageExtBrokerInner msg) {
|
|||
|
|
+ public void assignOffset(MessageExtBrokerInner msg) throws RocksDBException {
|
|||
|
|
final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag());
|
|||
|
|
|
|||
|
|
if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) {
|
|||
|
|
@@ -2127,12 +2149,12 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
class CommitLogDispatcherBuildConsumeQueue implements CommitLogDispatcher {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void dispatch(DispatchRequest request) {
|
|||
|
|
+ public void dispatch(DispatchRequest request) throws RocksDBException {
|
|||
|
|
final int tranType = MessageSysFlag.getTransactionValue(request.getSysFlag());
|
|||
|
|
switch (tranType) {
|
|||
|
|
case MessageSysFlag.TRANSACTION_NOT_TYPE:
|
|||
|
|
case MessageSysFlag.TRANSACTION_COMMIT_TYPE:
|
|||
|
|
- DefaultMessageStore.this.putMessagePositionInfo(request);
|
|||
|
|
+ putMessagePositionInfo(request);
|
|||
|
|
break;
|
|||
|
|
case MessageSysFlag.TRANSACTION_PREPARED_TYPE:
|
|||
|
|
case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE:
|
|||
|
|
@@ -2278,7 +2300,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
return DefaultMessageStore.this.brokerConfig.getIdentifier() + CleanCommitLogService.class.getSimpleName();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private boolean isTimeToDelete() {
|
|||
|
|
+ protected boolean isTimeToDelete() {
|
|||
|
|
String when = DefaultMessageStore.this.getMessageStoreConfig().getDeleteWhen();
|
|||
|
|
if (UtilAll.isItTimeToDo(when)) {
|
|||
|
|
DefaultMessageStore.LOGGER.info("it's time to reclaim disk space, " + when);
|
|||
|
|
@@ -2436,7 +2458,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
class CleanConsumeQueueService {
|
|||
|
|
- private long lastPhysicalMinOffset = 0;
|
|||
|
|
+ protected long lastPhysicalMinOffset = 0;
|
|||
|
|
|
|||
|
|
public void run() {
|
|||
|
|
try {
|
|||
|
|
@@ -2446,7 +2468,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private void deleteExpiredFiles() {
|
|||
|
|
+ protected void deleteExpiredFiles() {
|
|||
|
|
int deleteLogicsFilesInterval = DefaultMessageStore.this.getMessageStoreConfig().getDeleteConsumeQueueFilesInterval();
|
|||
|
|
|
|||
|
|
long minOffset = DefaultMessageStore.this.commitLog.getMinOffset();
|
|||
|
|
@@ -2551,7 +2573,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
if (cqUnit.getPos() >= minPhyOffset) {
|
|||
|
|
|
|||
|
|
- // Normal case, do not need correct.
|
|||
|
|
+ // Normal case, do not need to correct.
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
@@ -2741,6 +2763,18 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public void notifyMessageArriveIfNecessary(DispatchRequest dispatchRequest) {
|
|||
|
|
+ if (DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
|
|||
|
|
+ && DefaultMessageStore.this.messageArrivingListener != null) {
|
|||
|
|
+ DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
|
|||
|
|
+ dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
|
|||
|
|
+ dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
|
|||
|
|
+ dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
|
|||
|
|
+ DefaultMessageStore.this.reputMessageService.notifyMessageArrive4MultiQueue(dispatchRequest);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
class ReputMessageService extends ServiceThread {
|
|||
|
|
|
|||
|
|
protected volatile long reputFromOffset = 0;
|
|||
|
|
@@ -2810,13 +2844,8 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
if (size > 0) {
|
|||
|
|
DefaultMessageStore.this.doDispatch(dispatchRequest);
|
|||
|
|
|
|||
|
|
- if (DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
|
|||
|
|
- && DefaultMessageStore.this.messageArrivingListener != null) {
|
|||
|
|
- DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
|
|||
|
|
- dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
|
|||
|
|
- dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
|
|||
|
|
- dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
|
|||
|
|
- notifyMessageArrive4MultiQueue(dispatchRequest);
|
|||
|
|
+ if (!notifyMessageArriveInBatch) {
|
|||
|
|
+ notifyMessageArriveIfNecessary(dispatchRequest);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
this.reputFromOffset += size;
|
|||
|
|
@@ -2850,9 +2879,14 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.info("dispatch message to cq exception. reputFromOffset: {}", this.reputFromOffset, e);
|
|||
|
|
+ return;
|
|||
|
|
} finally {
|
|||
|
|
result.release();
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ finishCommitLogDispatch();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -2989,7 +3023,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
// dispatchRequestsList:[
|
|||
|
|
// {dispatchRequests:[{dispatchRequest}, {dispatchRequest}]},
|
|||
|
|
// {dispatchRequests:[{dispatchRequest}, {dispatchRequest}]}]
|
|||
|
|
- private void dispatch() {
|
|||
|
|
+ private void dispatch() throws Exception {
|
|||
|
|
dispatchRequestsList.clear();
|
|||
|
|
dispatchRequestOrderlyQueue.get(dispatchRequestsList);
|
|||
|
|
if (!dispatchRequestsList.isEmpty()) {
|
|||
|
|
@@ -2997,21 +3031,15 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
for (DispatchRequest dispatchRequest : dispatchRequests) {
|
|||
|
|
DefaultMessageStore.this.doDispatch(dispatchRequest);
|
|||
|
|
// wake up long-polling
|
|||
|
|
- if (DefaultMessageStore.this.brokerConfig.isLongPollingEnable()
|
|||
|
|
- && DefaultMessageStore.this.messageArrivingListener != null) {
|
|||
|
|
- DefaultMessageStore.this.messageArrivingListener.arriving(dispatchRequest.getTopic(),
|
|||
|
|
- dispatchRequest.getQueueId(), dispatchRequest.getConsumeQueueOffset() + 1,
|
|||
|
|
- dispatchRequest.getTagsCode(), dispatchRequest.getStoreTimestamp(),
|
|||
|
|
- dispatchRequest.getBitMap(), dispatchRequest.getPropertiesMap());
|
|||
|
|
- DefaultMessageStore.this.reputMessageService.notifyMessageArrive4MultiQueue(dispatchRequest);
|
|||
|
|
- }
|
|||
|
|
+ DefaultMessageStore.this.notifyMessageArriveIfNecessary(dispatchRequest);
|
|||
|
|
+
|
|||
|
|
if (!DefaultMessageStore.this.getMessageStoreConfig().isDuplicationEnable() &&
|
|||
|
|
- DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
|
|||
|
|
+ DefaultMessageStore.this.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE) {
|
|||
|
|
DefaultMessageStore.this.storeStatsService
|
|||
|
|
- .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
|
|||
|
|
+ .getSinglePutMessageTopicTimesTotal(dispatchRequest.getTopic()).add(1);
|
|||
|
|
DefaultMessageStore.this.storeStatsService
|
|||
|
|
- .getSinglePutMessageTopicSizeTotal(dispatchRequest.getTopic())
|
|||
|
|
- .add(dispatchRequest.getMsgSize());
|
|||
|
|
+ .getSinglePutMessageTopicSizeTotal(dispatchRequest.getTopic())
|
|||
|
|
+ .add(dispatchRequest.getMsgSize());
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
@@ -3079,7 +3107,7 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
public void doReput() {
|
|||
|
|
if (this.reputFromOffset < DefaultMessageStore.this.commitLog.getMinOffset()) {
|
|||
|
|
LOGGER.warn("The reputFromOffset={} is smaller than minPyOffset={}, this usually indicate that the dispatch behind too much and the commitlog has expired.",
|
|||
|
|
- this.reputFromOffset, DefaultMessageStore.this.commitLog.getMinOffset());
|
|||
|
|
+ this.reputFromOffset, DefaultMessageStore.this.commitLog.getMinOffset());
|
|||
|
|
this.reputFromOffset = DefaultMessageStore.this.commitLog.getMinOffset();
|
|||
|
|
}
|
|||
|
|
for (boolean doNext = true; this.isCommitLogAvailable() && doNext; ) {
|
|||
|
|
@@ -3138,6 +3166,9 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
result.release();
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ // only for rocksdb mode
|
|||
|
|
+ finishCommitLogDispatch();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
@@ -3180,8 +3211,8 @@ public class DefaultMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
if (this.isCommitLogAvailable()) {
|
|||
|
|
LOGGER.warn("shutdown concurrentReputMessageService, but CommitLog have not finish to be dispatched, CommitLog max" +
|
|||
|
|
- " offset={}, reputFromOffset={}", DefaultMessageStore.this.commitLog.getMaxOffset(),
|
|||
|
|
- this.reputFromOffset);
|
|||
|
|
+ " offset={}, reputFromOffset={}", DefaultMessageStore.this.commitLog.getMaxOffset(),
|
|||
|
|
+ this.reputFromOffset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
this.mainBatchDispatchRequestService.shutdown();
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
index 989cbbe31..814c6d1bf 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java
|
|||
|
|
@@ -16,10 +16,6 @@
|
|||
|
|
*/
|
|||
|
|
package org.apache.rocketmq.store;
|
|||
|
|
|
|||
|
|
-import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
-import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
import java.nio.ByteBuffer;
|
|||
|
|
import java.util.HashMap;
|
|||
|
|
import java.util.LinkedList;
|
|||
|
|
@@ -40,10 +36,15 @@ import org.apache.rocketmq.store.hook.PutMessageHook;
|
|||
|
|
import org.apache.rocketmq.store.hook.SendMessageBackHook;
|
|||
|
|
import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
-import org.apache.rocketmq.store.queue.ConsumeQueueStore;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
|
|||
|
|
import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
import org.apache.rocketmq.store.timer.TimerMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.util.PerfCounter;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* This class defines contracting interfaces to implement, allowing third-party vendor to use customized message store.
|
|||
|
|
@@ -545,7 +546,7 @@ public interface MessageStore {
|
|||
|
|
void setConfirmOffset(long phyOffset);
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
- * Check if the operation system page cache is busy or not.
|
|||
|
|
+ * Check if the operating system page cache is busy or not.
|
|||
|
|
*
|
|||
|
|
* @return true if the OS page cache is busy; false otherwise.
|
|||
|
|
*/
|
|||
|
|
@@ -620,9 +621,18 @@ public interface MessageStore {
|
|||
|
|
* @param commitLogFile commit log file
|
|||
|
|
* @param isRecover is from recover process
|
|||
|
|
* @param isFileEnd if the dispatch request represents 'file end'
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile,
|
|||
|
|
- boolean isRecover, boolean isFileEnd);
|
|||
|
|
+ boolean isRecover, boolean isFileEnd) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Only used in rocksdb mode, because we build consumeQueue in batch(default 16 dispatchRequests)
|
|||
|
|
+ * It will be triggered in two cases:
|
|||
|
|
+ * @see org.apache.rocketmq.store.DefaultMessageStore.ReputMessageService#doReput
|
|||
|
|
+ * @see CommitLog#recoverAbnormally
|
|||
|
|
+ */
|
|||
|
|
+ void finishCommitLogDispatch();
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* Get the message store config
|
|||
|
|
@@ -691,13 +701,9 @@ public interface MessageStore {
|
|||
|
|
* Truncate dirty logic files
|
|||
|
|
*
|
|||
|
|
* @param phyOffset physical offset
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
- void truncateDirtyLogicFiles(long phyOffset);
|
|||
|
|
-
|
|||
|
|
- /**
|
|||
|
|
- * Destroy logics files
|
|||
|
|
- */
|
|||
|
|
- void destroyLogics();
|
|||
|
|
+ void truncateDirtyLogicFiles(long phyOffset) throws RocksDBException;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* Unlock mappedFile
|
|||
|
|
@@ -718,7 +724,7 @@ public interface MessageStore {
|
|||
|
|
*
|
|||
|
|
* @return the queue store
|
|||
|
|
*/
|
|||
|
|
- ConsumeQueueStore getQueueStore();
|
|||
|
|
+ ConsumeQueueStoreInterface getQueueStore();
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* If 'sync disk flush' is configured in this message store
|
|||
|
|
@@ -739,8 +745,9 @@ public interface MessageStore {
|
|||
|
|
* yourself.
|
|||
|
|
*
|
|||
|
|
* @param msg message
|
|||
|
|
+ * @throws RocksDBException
|
|||
|
|
*/
|
|||
|
|
- void assignOffset(MessageExtBrokerInner msg);
|
|||
|
|
+ void assignOffset(MessageExtBrokerInner msg) throws RocksDBException;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* Increase queue offset in memory table. If there is a race condition, you need to lock/unlock this method
|
|||
|
|
@@ -835,14 +842,15 @@ public interface MessageStore {
|
|||
|
|
*
|
|||
|
|
* @param offsetToTruncate offset to truncate
|
|||
|
|
* @return true if truncate succeed, false otherwise
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
- boolean truncateFiles(long offsetToTruncate);
|
|||
|
|
+ boolean truncateFiles(long offsetToTruncate) throws RocksDBException;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
- * Check if the offset is align with one message.
|
|||
|
|
+ * Check if the offset is aligned with one message.
|
|||
|
|
*
|
|||
|
|
* @param offset offset to check
|
|||
|
|
- * @return true if align, false otherwise
|
|||
|
|
+ * @return true if aligned, false otherwise
|
|||
|
|
*/
|
|||
|
|
boolean isOffsetAligned(long offset);
|
|||
|
|
|
|||
|
|
@@ -971,4 +979,14 @@ public interface MessageStore {
|
|||
|
|
* @param attributesBuilderSupplier metrics attributes builder
|
|||
|
|
*/
|
|||
|
|
void initMetrics(Meter meter, Supplier<AttributesBuilder> attributesBuilderSupplier);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Recover topic queue table
|
|||
|
|
+ */
|
|||
|
|
+ void recoverTopicQueueTable();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * notify message arrive if necessary
|
|||
|
|
+ */
|
|||
|
|
+ void notifyMessageArriveIfNecessary(DispatchRequest dispatchRequest);
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..87ccb5474
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/RocksDBMessageStore.java
|
|||
|
|
@@ -0,0 +1,169 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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;
|
|||
|
|
+
|
|||
|
|
+import java.io.IOException;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
+import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+import org.apache.rocketmq.store.config.StorePathConfigHelper;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
|
|||
|
|
+import org.apache.rocketmq.store.queue.RocksDBConsumeQueue;
|
|||
|
|
+import org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore;
|
|||
|
|
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBMessageStore extends DefaultMessageStore {
|
|||
|
|
+
|
|||
|
|
+ public RocksDBMessageStore(final MessageStoreConfig messageStoreConfig, final BrokerStatsManager brokerStatsManager,
|
|||
|
|
+ final MessageArrivingListener messageArrivingListener, final BrokerConfig brokerConfig, final ConcurrentMap<String, TopicConfig> topicConfigTable) throws
|
|||
|
|
+ IOException {
|
|||
|
|
+ super(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig, topicConfigTable);
|
|||
|
|
+ notifyMessageArriveInBatch = true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConsumeQueueStoreInterface createConsumeQueueStore() {
|
|||
|
|
+ return new RocksDBConsumeQueueStore(this);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CleanConsumeQueueService createCleanConsumeQueueService() {
|
|||
|
|
+ return new RocksDBCleanConsumeQueueService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public FlushConsumeQueueService createFlushConsumeQueueService() {
|
|||
|
|
+ return new RocksDBFlushConsumeQueueService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CorrectLogicOffsetService createCorrectLogicOffsetService() {
|
|||
|
|
+ return new RocksDBCorrectLogicOffsetService();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Try to set topicQueueTable = new HashMap<>(), otherwise it will cause bug when broker role changes.
|
|||
|
|
+ * And unlike method in DefaultMessageStore, we don't need to really recover topic queue table advance,
|
|||
|
|
+ * because we can recover topic queue table from rocksdb when we need to use it.
|
|||
|
|
+ * @see RocksDBConsumeQueue#assignQueueOffset
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recoverTopicQueueTable() {
|
|||
|
|
+ this.consumeQueueStore.setTopicQueueTable(new ConcurrentHashMap<>());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void finishCommitLogDispatch() {
|
|||
|
|
+ try {
|
|||
|
|
+ putMessagePositionInfo(null);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.info("try to finish commitlog dispatch error.", e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConsumeQueueInterface getConsumeQueue(String topic, int queueId) {
|
|||
|
|
+ return findConsumeQueue(topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class RocksDBCleanConsumeQueueService extends CleanConsumeQueueService {
|
|||
|
|
+ private final double diskSpaceWarningLevelRatio =
|
|||
|
|
+ Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceWarningLevelRatio", "0.90"));
|
|||
|
|
+
|
|||
|
|
+ private final double diskSpaceCleanForciblyRatio =
|
|||
|
|
+ Double.parseDouble(System.getProperty("rocketmq.broker.diskSpaceCleanForciblyRatio", "0.85"));
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ protected void deleteExpiredFiles() {
|
|||
|
|
+
|
|||
|
|
+ long minOffset = RocksDBMessageStore.this.commitLog.getMinOffset();
|
|||
|
|
+ if (minOffset > this.lastPhysicalMinOffset) {
|
|||
|
|
+ this.lastPhysicalMinOffset = minOffset;
|
|||
|
|
+
|
|||
|
|
+ boolean spaceFull = isSpaceToDelete();
|
|||
|
|
+ boolean timeUp = cleanCommitLogService.isTimeToDelete();
|
|||
|
|
+ if (spaceFull || timeUp) {
|
|||
|
|
+ RocksDBMessageStore.this.consumeQueueStore.cleanExpired(minOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ RocksDBMessageStore.this.indexService.deleteExpiredFile(minOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean isSpaceToDelete() {
|
|||
|
|
+ double ratio = RocksDBMessageStore.this.getMessageStoreConfig().getDiskMaxUsedSpaceRatio() / 100.0;
|
|||
|
|
+
|
|||
|
|
+ String storePathLogics = StorePathConfigHelper
|
|||
|
|
+ .getStorePathConsumeQueue(RocksDBMessageStore.this.getMessageStoreConfig().getStorePathRootDir());
|
|||
|
|
+ double logicsRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathLogics);
|
|||
|
|
+ if (logicsRatio > diskSpaceWarningLevelRatio) {
|
|||
|
|
+ boolean diskOk = RocksDBMessageStore.this.runningFlags.getAndMakeLogicDiskFull();
|
|||
|
|
+ if (diskOk) {
|
|||
|
|
+ RocksDBMessageStore.LOGGER.error("logics disk maybe full soon " + logicsRatio + ", so mark disk full");
|
|||
|
|
+ }
|
|||
|
|
+ } else if (logicsRatio > diskSpaceCleanForciblyRatio) {
|
|||
|
|
+ } else {
|
|||
|
|
+ boolean diskOk = RocksDBMessageStore.this.runningFlags.getAndMakeLogicDiskOK();
|
|||
|
|
+ if (!diskOk) {
|
|||
|
|
+ RocksDBMessageStore.LOGGER.info("logics disk space OK " + logicsRatio + ", so mark disk ok");
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ if (logicsRatio < 0 || logicsRatio > ratio) {
|
|||
|
|
+ RocksDBMessageStore.LOGGER.info("logics disk maybe full soon, so reclaim space, " + logicsRatio);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class RocksDBFlushConsumeQueueService extends FlushConsumeQueueService {
|
|||
|
|
+ /**
|
|||
|
|
+ * There is no need to flush consume queue,
|
|||
|
|
+ * we put all consume queues in RocksDBConsumeQueueStore,
|
|||
|
|
+ * it depends on rocksdb to flush consume queue to disk(sorted string table),
|
|||
|
|
+ * we even don't flush WAL of consume store, since we think it can recover consume queue from commitlog.
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void run() {
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ class RocksDBCorrectLogicOffsetService extends CorrectLogicOffsetService {
|
|||
|
|
+ /**
|
|||
|
|
+ * There is no need to correct min offset of consume queue, we already fix this problem.
|
|||
|
|
+ * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable#getMinCqOffset
|
|||
|
|
+ */
|
|||
|
|
+ public void run() {
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long estimateMessageCount(String topic, int queueId, long from, long to, MessageFilter filter) {
|
|||
|
|
+ // todo
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java b/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java
|
|||
|
|
index 2ae6879aa..91fcb155a 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java
|
|||
|
|
@@ -30,6 +30,8 @@ public class RunningFlags {
|
|||
|
|
|
|||
|
|
private static final int FENCED_BIT = 1 << 5;
|
|||
|
|
|
|||
|
|
+ private static final int LOGIC_DISK_FULL_BIT = 1 << 5;
|
|||
|
|
+
|
|||
|
|
private volatile int flagBits = 0;
|
|||
|
|
|
|||
|
|
public RunningFlags() {
|
|||
|
|
@@ -63,6 +65,10 @@ public class RunningFlags {
|
|||
|
|
return result;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public void clearLogicsQueueError() {
|
|||
|
|
+ this.flagBits &= ~WRITE_LOGICS_QUEUE_ERROR_BIT;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public boolean getAndMakeWriteable() {
|
|||
|
|
boolean result = this.isWriteable();
|
|||
|
|
if (!result) {
|
|||
|
|
@@ -72,7 +78,7 @@ public class RunningFlags {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
public boolean isWriteable() {
|
|||
|
|
- if ((this.flagBits & (NOT_WRITEABLE_BIT | WRITE_LOGICS_QUEUE_ERROR_BIT | DISK_FULL_BIT | WRITE_INDEX_FILE_ERROR_BIT | FENCED_BIT)) == 0) {
|
|||
|
|
+ if ((this.flagBits & (NOT_WRITEABLE_BIT | WRITE_LOGICS_QUEUE_ERROR_BIT | DISK_FULL_BIT | WRITE_INDEX_FILE_ERROR_BIT | FENCED_BIT | LOGIC_DISK_FULL_BIT)) == 0) {
|
|||
|
|
return true;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -81,7 +87,7 @@ public class RunningFlags {
|
|||
|
|
|
|||
|
|
//for consume queue, just ignore the DISK_FULL_BIT
|
|||
|
|
public boolean isCQWriteable() {
|
|||
|
|
- if ((this.flagBits & (NOT_WRITEABLE_BIT | WRITE_LOGICS_QUEUE_ERROR_BIT | WRITE_INDEX_FILE_ERROR_BIT)) == 0) {
|
|||
|
|
+ if ((this.flagBits & (NOT_WRITEABLE_BIT | WRITE_LOGICS_QUEUE_ERROR_BIT | WRITE_INDEX_FILE_ERROR_BIT | LOGIC_DISK_FULL_BIT)) == 0) {
|
|||
|
|
return true;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -139,4 +145,16 @@ public class RunningFlags {
|
|||
|
|
this.flagBits &= ~DISK_FULL_BIT;
|
|||
|
|
return result;
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ public boolean getAndMakeLogicDiskFull() {
|
|||
|
|
+ boolean result = !((this.flagBits & LOGIC_DISK_FULL_BIT) == LOGIC_DISK_FULL_BIT);
|
|||
|
|
+ this.flagBits |= LOGIC_DISK_FULL_BIT;
|
|||
|
|
+ return result;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public boolean getAndMakeLogicDiskOK() {
|
|||
|
|
+ boolean result = !((this.flagBits & LOGIC_DISK_FULL_BIT) == LOGIC_DISK_FULL_BIT);
|
|||
|
|
+ this.flagBits &= ~LOGIC_DISK_FULL_BIT;
|
|||
|
|
+ return result;
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
index 9fa448043..028facbdc 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
|
|||
|
|
@@ -397,8 +397,10 @@ public class MessageStoreConfig {
|
|||
|
|
private int batchDispatchRequestThreadPoolNums = 16;
|
|||
|
|
|
|||
|
|
// rocksdb mode
|
|||
|
|
+ private long cleanRocksDBDirtyCQIntervalMin = 60;
|
|||
|
|
+ private long statRocksDBCQIntervalSec = 10;
|
|||
|
|
+ private long memTableFlushIntervalMs = 60 * 60 * 1000L;
|
|||
|
|
private boolean realTimePersistRocksDBConfig = true;
|
|||
|
|
- private long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
private boolean enableRocksDBLog = false;
|
|||
|
|
|
|||
|
|
private int topicQueueLockNum = 32;
|
|||
|
|
@@ -499,6 +501,10 @@ public class MessageStoreConfig {
|
|||
|
|
this.mappedFileSizeCommitLog = mappedFileSizeCommitLog;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public boolean isEnableRocksDBStore() {
|
|||
|
|
+ return StoreType.DEFAULT_ROCKSDB.getStoreType().equalsIgnoreCase(this.storeType);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public String getStoreType() {
|
|||
|
|
return storeType;
|
|||
|
|
}
|
|||
|
|
@@ -508,7 +514,6 @@ public class MessageStoreConfig {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
public int getMappedFileSizeConsumeQueue() {
|
|||
|
|
-
|
|||
|
|
int factor = (int) Math.ceil(this.mappedFileSizeConsumeQueue / (ConsumeQueue.CQ_STORE_UNIT_SIZE * 1.0));
|
|||
|
|
return (int) (factor * ConsumeQueue.CQ_STORE_UNIT_SIZE);
|
|||
|
|
}
|
|||
|
|
@@ -1738,12 +1743,28 @@ public class MessageStoreConfig {
|
|||
|
|
this.realTimePersistRocksDBConfig = realTimePersistRocksDBConfig;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public long getMemTableFlushInterval() {
|
|||
|
|
- return memTableFlushInterval;
|
|||
|
|
+ public long getStatRocksDBCQIntervalSec() {
|
|||
|
|
+ return statRocksDBCQIntervalSec;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void setStatRocksDBCQIntervalSec(long statRocksDBCQIntervalSec) {
|
|||
|
|
+ this.statRocksDBCQIntervalSec = statRocksDBCQIntervalSec;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getCleanRocksDBDirtyCQIntervalMin() {
|
|||
|
|
+ return cleanRocksDBDirtyCQIntervalMin;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void setCleanRocksDBDirtyCQIntervalMin(long cleanRocksDBDirtyCQIntervalMin) {
|
|||
|
|
+ this.cleanRocksDBDirtyCQIntervalMin = cleanRocksDBDirtyCQIntervalMin;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getMemTableFlushIntervalMs() {
|
|||
|
|
+ return memTableFlushIntervalMs;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void setMemTableFlushInterval(long memTableFlushInterval) {
|
|||
|
|
- this.memTableFlushInterval = memTableFlushInterval;
|
|||
|
|
+ public void setMemTableFlushIntervalMs(long memTableFlushIntervalMs) {
|
|||
|
|
+ this.memTableFlushIntervalMs = memTableFlushIntervalMs;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
public boolean isEnableRocksDBLog() {
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
index d5f6acdc0..70371d83b 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java
|
|||
|
|
@@ -16,26 +16,13 @@
|
|||
|
|
*/
|
|||
|
|
package org.apache.rocketmq.store.dledger;
|
|||
|
|
|
|||
|
|
-import io.openmessaging.storage.dledger.AppendFuture;
|
|||
|
|
-import io.openmessaging.storage.dledger.BatchAppendFuture;
|
|||
|
|
-import io.openmessaging.storage.dledger.DLedgerConfig;
|
|||
|
|
-import io.openmessaging.storage.dledger.DLedgerServer;
|
|||
|
|
-import io.openmessaging.storage.dledger.entry.DLedgerEntry;
|
|||
|
|
-import io.openmessaging.storage.dledger.protocol.AppendEntryRequest;
|
|||
|
|
-import io.openmessaging.storage.dledger.protocol.AppendEntryResponse;
|
|||
|
|
-import io.openmessaging.storage.dledger.protocol.BatchAppendEntryRequest;
|
|||
|
|
-import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode;
|
|||
|
|
-import io.openmessaging.storage.dledger.store.file.DLedgerMmapFileStore;
|
|||
|
|
-import io.openmessaging.storage.dledger.store.file.MmapFile;
|
|||
|
|
-import io.openmessaging.storage.dledger.store.file.MmapFileList;
|
|||
|
|
-import io.openmessaging.storage.dledger.store.file.SelectMmapBufferResult;
|
|||
|
|
-import io.openmessaging.storage.dledger.utils.DLedgerUtils;
|
|||
|
|
import java.net.Inet6Address;
|
|||
|
|
import java.net.InetSocketAddress;
|
|||
|
|
import java.nio.ByteBuffer;
|
|||
|
|
import java.util.LinkedList;
|
|||
|
|
import java.util.List;
|
|||
|
|
import java.util.concurrent.CompletableFuture;
|
|||
|
|
+
|
|||
|
|
import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageExtBatch;
|
|||
|
|
@@ -54,6 +41,22 @@ import org.apache.rocketmq.store.SelectMappedBufferResult;
|
|||
|
|
import org.apache.rocketmq.store.StoreStatsService;
|
|||
|
|
import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+import io.openmessaging.storage.dledger.AppendFuture;
|
|||
|
|
+import io.openmessaging.storage.dledger.BatchAppendFuture;
|
|||
|
|
+import io.openmessaging.storage.dledger.DLedgerConfig;
|
|||
|
|
+import io.openmessaging.storage.dledger.DLedgerServer;
|
|||
|
|
+import io.openmessaging.storage.dledger.entry.DLedgerEntry;
|
|||
|
|
+import io.openmessaging.storage.dledger.protocol.AppendEntryRequest;
|
|||
|
|
+import io.openmessaging.storage.dledger.protocol.AppendEntryResponse;
|
|||
|
|
+import io.openmessaging.storage.dledger.protocol.BatchAppendEntryRequest;
|
|||
|
|
+import io.openmessaging.storage.dledger.protocol.DLedgerResponseCode;
|
|||
|
|
+import io.openmessaging.storage.dledger.store.file.DLedgerMmapFileStore;
|
|||
|
|
+import io.openmessaging.storage.dledger.store.file.MmapFile;
|
|||
|
|
+import io.openmessaging.storage.dledger.store.file.MmapFileList;
|
|||
|
|
+import io.openmessaging.storage.dledger.store.file.SelectMmapBufferResult;
|
|||
|
|
+import io.openmessaging.storage.dledger.utils.DLedgerUtils;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* Store all metadata downtime for recovery, data protection reliability
|
|||
|
|
@@ -269,7 +272,7 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
|
|||
|
|
return null;
|
|||
|
|
}
|
|||
|
|
-
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public boolean getData(final long offset, final int size, final ByteBuffer byteBuffer) {
|
|||
|
|
if (offset < dividedCommitlogOffset) {
|
|||
|
|
@@ -287,7 +290,7 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private void recover(long maxPhyOffsetOfConsumeQueue) {
|
|||
|
|
+ private void recover(long maxPhyOffsetOfConsumeQueue) throws RocksDBException {
|
|||
|
|
dLedgerFileStore.load();
|
|||
|
|
if (dLedgerFileList.getMappedFiles().size() > 0) {
|
|||
|
|
dLedgerFileStore.recover();
|
|||
|
|
@@ -341,12 +344,12 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void recoverNormally(long maxPhyOffsetOfConsumeQueue) {
|
|||
|
|
+ public void recoverNormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException {
|
|||
|
|
recover(maxPhyOffsetOfConsumeQueue);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) {
|
|||
|
|
+ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBException {
|
|||
|
|
recover(maxPhyOffsetOfConsumeQueue);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -469,9 +472,6 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
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);
|
|||
|
|
- } catch (Exception e) {
|
|||
|
|
- log.error("Put message error", e);
|
|||
|
|
- return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
|
|||
|
|
} finally {
|
|||
|
|
beginTimeInDledgerLock = 0;
|
|||
|
|
putMessageLock.unlock();
|
|||
|
|
@@ -482,6 +482,9 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
defaultMessageStore.increaseOffset(msg, getMessageNum(msg));
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.error("Put message error", e);
|
|||
|
|
+ return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
|
|||
|
|
} finally {
|
|||
|
|
topicQueueLock.unlock(topicQueueKey);
|
|||
|
|
}
|
|||
|
|
@@ -611,9 +614,6 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, firstWroteOffset, encodeResult.totalMsgLen,
|
|||
|
|
msgIdBuilder.toString(), System.currentTimeMillis(), queueOffset, elapsedTimeInLock);
|
|||
|
|
appendResult.setMsgNum(msgNum);
|
|||
|
|
- } catch (Exception e) {
|
|||
|
|
- log.error("Put message error", e);
|
|||
|
|
- return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
|
|||
|
|
} finally {
|
|||
|
|
beginTimeInDledgerLock = 0;
|
|||
|
|
putMessageLock.unlock();
|
|||
|
|
@@ -626,7 +626,10 @@ public class DLedgerCommitLog extends CommitLog {
|
|||
|
|
|
|||
|
|
defaultMessageStore.increaseOffset(messageExtBatch, (short) batchNum);
|
|||
|
|
|
|||
|
|
- } finally {
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.error("Put message error", e);
|
|||
|
|
+ return CompletableFuture.completedFuture(new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)));
|
|||
|
|
+ } finally {
|
|||
|
|
topicQueueLock.unlock(encodeResult.queueOffsetKey);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
|
|||
|
|
index 467da603d..aaea7d690 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAService.java
|
|||
|
|
@@ -25,6 +25,7 @@ import org.apache.rocketmq.remoting.protocol.body.HARuntimeInfo;
|
|||
|
|
import org.apache.rocketmq.store.CommitLog;
|
|||
|
|
import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
public interface HAService {
|
|||
|
|
|
|||
|
|
@@ -53,7 +54,7 @@ public interface HAService {
|
|||
|
|
*
|
|||
|
|
* @param masterEpoch the new masterEpoch
|
|||
|
|
*/
|
|||
|
|
- default boolean changeToMaster(int masterEpoch) {
|
|||
|
|
+ default boolean changeToMaster(int masterEpoch) throws RocksDBException {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAClient.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAClient.java
|
|||
|
|
index 936db0c4c..176c25a96 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAClient.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAClient.java
|
|||
|
|
@@ -432,7 +432,7 @@ public class AutoSwitchHAClient extends ServiceThread implements HAClient {
|
|||
|
|
/**
|
|||
|
|
* Compare the master and slave's epoch file, find consistent point, do truncate.
|
|||
|
|
*/
|
|||
|
|
- private boolean doTruncate(List<EpochEntry> masterEpochEntries, long masterEndOffset) throws IOException {
|
|||
|
|
+ private boolean doTruncate(List<EpochEntry> masterEpochEntries, long masterEndOffset) throws Exception {
|
|||
|
|
if (this.epochCache.getEntrySize() == 0) {
|
|||
|
|
// If epochMap is empty, means the broker is a new replicas
|
|||
|
|
LOGGER.info("Slave local epochCache is empty, skip truncate log");
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
index f20bc3e28..64dad9aef 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java
|
|||
|
|
@@ -51,6 +51,7 @@ import org.apache.rocketmq.store.ha.GroupTransferService;
|
|||
|
|
import org.apache.rocketmq.store.ha.HAClient;
|
|||
|
|
import org.apache.rocketmq.store.ha.HAConnection;
|
|||
|
|
import org.apache.rocketmq.store.ha.HAConnectionStateNotificationService;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
* SwitchAble ha service, support switch role to master or slave.
|
|||
|
|
@@ -111,7 +112,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public boolean changeToMaster(int masterEpoch) {
|
|||
|
|
+ public boolean changeToMaster(int masterEpoch) throws RocksDBException {
|
|||
|
|
final int lastEpoch = this.epochCache.lastEpoch();
|
|||
|
|
if (masterEpoch < lastEpoch) {
|
|||
|
|
LOGGER.warn("newMasterEpoch {} < lastEpoch {}, fail to change to master", masterEpoch, lastEpoch);
|
|||
|
|
@@ -315,7 +316,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
final EpochEntry currentLeaderEpoch = this.epochCache.lastEntry();
|
|||
|
|
if (slaveMaxOffset >= currentLeaderEpoch.getStartOffset()) {
|
|||
|
|
LOGGER.info("The slave {} has caught up, slaveMaxOffset: {}, confirmOffset: {}, epoch: {}, leader epoch startOffset: {}.",
|
|||
|
|
- slaveBrokerId, slaveMaxOffset, confirmOffset, currentLeaderEpoch.getEpoch(), currentLeaderEpoch.getStartOffset());
|
|||
|
|
+ slaveBrokerId, slaveMaxOffset, confirmOffset, currentLeaderEpoch.getEpoch(), currentLeaderEpoch.getStartOffset());
|
|||
|
|
currentSyncStateSet.add(slaveBrokerId);
|
|||
|
|
markSynchronizingSyncStateSet(currentSyncStateSet);
|
|||
|
|
// Notify the upper layer that syncStateSet changed.
|
|||
|
|
@@ -491,7 +492,7 @@ public class AutoSwitchHAService extends DefaultHAService {
|
|||
|
|
/**
|
|||
|
|
* Try to truncate incomplete msg transferred from master.
|
|||
|
|
*/
|
|||
|
|
- public long truncateInvalidMsg() {
|
|||
|
|
+ public long truncateInvalidMsg() throws RocksDBException {
|
|||
|
|
long dispatchBehind = this.defaultMessageStore.dispatchBehindBytes();
|
|||
|
|
if (dispatchBehind <= 0) {
|
|||
|
|
LOGGER.info("Dispatch complete, skip truncate");
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
index ab9fc6da7..2f2ce9812 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java
|
|||
|
|
@@ -17,10 +17,6 @@
|
|||
|
|
|
|||
|
|
package org.apache.rocketmq.store.plugin;
|
|||
|
|
|
|||
|
|
-import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
-import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
import java.nio.ByteBuffer;
|
|||
|
|
import java.util.HashMap;
|
|||
|
|
import java.util.LinkedList;
|
|||
|
|
@@ -55,10 +51,16 @@ import org.apache.rocketmq.store.hook.PutMessageHook;
|
|||
|
|
import org.apache.rocketmq.store.hook.SendMessageBackHook;
|
|||
|
|
import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
-import org.apache.rocketmq.store.queue.ConsumeQueueStore;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface;
|
|||
|
|
import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
import org.apache.rocketmq.store.timer.TimerMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.util.PerfCounter;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
|
|||
|
|
public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
protected MessageStore next = null;
|
|||
|
|
@@ -457,7 +459,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public boolean truncateFiles(long offsetToTruncate) {
|
|||
|
|
+ public boolean truncateFiles(long offsetToTruncate) throws RocksDBException {
|
|||
|
|
return next.truncateFiles(offsetToTruncate);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -511,7 +513,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
public void onCommitLogDispatch(DispatchRequest dispatchRequest, boolean doDispatch, MappedFile commitLogFile,
|
|||
|
|
- boolean isRecover, boolean isFileEnd) {
|
|||
|
|
+ boolean isRecover, boolean isFileEnd) throws RocksDBException {
|
|||
|
|
next.onCommitLogDispatch(dispatchRequest, doDispatch, commitLogFile, isRecover, isFileEnd);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -551,15 +553,10 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void truncateDirtyLogicFiles(long phyOffset) {
|
|||
|
|
+ public void truncateDirtyLogicFiles(long phyOffset) throws RocksDBException {
|
|||
|
|
next.truncateDirtyLogicFiles(phyOffset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- @Override
|
|||
|
|
- public void destroyLogics() {
|
|||
|
|
- next.destroyLogics();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
@Override
|
|||
|
|
public void unlockMappedFile(MappedFile unlockMappedFile) {
|
|||
|
|
next.unlockMappedFile(unlockMappedFile);
|
|||
|
|
@@ -571,7 +568,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public ConsumeQueueStore getQueueStore() {
|
|||
|
|
+ public ConsumeQueueStoreInterface getQueueStore() {
|
|||
|
|
return next.getQueueStore();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -586,7 +583,7 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Override
|
|||
|
|
- public void assignOffset(MessageExtBrokerInner msg) {
|
|||
|
|
+ public void assignOffset(MessageExtBrokerInner msg) throws RocksDBException {
|
|||
|
|
next.assignOffset(msg);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -649,4 +646,19 @@ public abstract class AbstractPluginMessageStore implements MessageStore {
|
|||
|
|
public void initMetrics(Meter meter, Supplier<AttributesBuilder> attributesBuilderSupplier) {
|
|||
|
|
next.initMetrics(meter, attributesBuilderSupplier);
|
|||
|
|
}
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void finishCommitLogDispatch() {
|
|||
|
|
+ next.finishCommitLogDispatch();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recoverTopicQueueTable() {
|
|||
|
|
+ next.recoverTopicQueueTable();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void notifyMessageArriveIfNecessary(DispatchRequest dispatchRequest) {
|
|||
|
|
+ next.notifyMessageArriveIfNecessary(dispatchRequest);
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..30054fa50
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/AbstractConsumeQueueStore.java
|
|||
|
|
@@ -0,0 +1,105 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+public abstract class AbstractConsumeQueueStore implements ConsumeQueueStoreInterface {
|
|||
|
|
+ protected static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ protected final DefaultMessageStore messageStore;
|
|||
|
|
+ protected final MessageStoreConfig messageStoreConfig;
|
|||
|
|
+ protected final QueueOffsetOperator queueOffsetOperator = new QueueOffsetOperator();
|
|||
|
|
+ protected final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
|
|||
|
|
+
|
|||
|
|
+ public AbstractConsumeQueueStore(DefaultMessageStore messageStore) {
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ this.messageStoreConfig = messageStore.getMessageStoreConfig();
|
|||
|
|
+ this.consumeQueueTable = new ConcurrentHashMap<>(32);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request) {
|
|||
|
|
+ consumeQueue.putMessagePositionInfoWrapper(request);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Long getMaxOffset(String topic, int queueId) {
|
|||
|
|
+ return this.queueOffsetOperator.currentQueueOffset(topic + "-" + queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void setTopicQueueTable(ConcurrentMap<String, Long> topicQueueTable) {
|
|||
|
|
+ this.queueOffsetOperator.setTopicQueueTable(topicQueueTable);
|
|||
|
|
+ this.queueOffsetOperator.setLmqTopicQueueTable(topicQueueTable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConcurrentMap getTopicQueueTable() {
|
|||
|
|
+ return this.queueOffsetOperator.getTopicQueueTable();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void assignQueueOffset(MessageExtBrokerInner msg) throws RocksDBException {
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = findOrCreateConsumeQueue(msg.getTopic(), msg.getQueueId());
|
|||
|
|
+ consumeQueue.assignQueueOffset(this.queueOffsetOperator, msg);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void increaseQueueOffset(MessageExtBrokerInner msg, short messageNum) {
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = findOrCreateConsumeQueue(msg.getTopic(), msg.getQueueId());
|
|||
|
|
+ consumeQueue.increaseQueueOffset(this.queueOffsetOperator, msg, messageNum);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void removeTopicQueueTable(String topic, Integer queueId) {
|
|||
|
|
+ this.queueOffsetOperator.remove(topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> getConsumeQueueTable() {
|
|||
|
|
+ return this.consumeQueueTable;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConcurrentMap<Integer, ConsumeQueueInterface> findConsumeQueueMap(String topic) {
|
|||
|
|
+ return this.consumeQueueTable.get(topic);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getStoreTime(CqUnit cqUnit) {
|
|||
|
|
+ if (cqUnit != null) {
|
|||
|
|
+ try {
|
|||
|
|
+ final long phyOffset = cqUnit.getPos();
|
|||
|
|
+ final int size = cqUnit.getSize();
|
|||
|
|
+ long storeTime = this.messageStore.getCommitLog().pickupStoreTimestamp(phyOffset, size);
|
|||
|
|
+ return storeTime;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return -1;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
|
|||
|
|
index 387c233bf..7108c835c 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/BatchConsumeQueue.java
|
|||
|
|
@@ -24,6 +24,7 @@ import java.util.Map;
|
|||
|
|
import java.util.concurrent.ConcurrentSkipListMap;
|
|||
|
|
import java.util.function.Function;
|
|||
|
|
import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
@@ -311,6 +312,11 @@ public class BatchConsumeQueue implements ConsumeQueueInterface {
|
|||
|
|
return new BatchConsumeQueueIterator(sbr);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public ReferredIterator<CqUnit> iterateFrom(long startIndex, int count) {
|
|||
|
|
+ return iterateFrom(startIndex);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public CqUnit get(long offset) {
|
|||
|
|
ReferredIterator<CqUnit> it = iterateFrom(offset);
|
|||
|
|
@@ -320,6 +326,20 @@ public class BatchConsumeQueue implements ConsumeQueueInterface {
|
|||
|
|
return it.nextAndRelease();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getCqUnitAndStoreTime(long index) {
|
|||
|
|
+ CqUnit cqUnit = get(index);
|
|||
|
|
+ Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit);
|
|||
|
|
+ return new Pair<>(cqUnit, messageStoreTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getEarliestUnitAndStoreTime() {
|
|||
|
|
+ CqUnit cqUnit = getEarliestUnit();
|
|||
|
|
+ Long messageStoreTime = this.messageStore.getQueueStore().getStoreTime(cqUnit);
|
|||
|
|
+ return new Pair<>(cqUnit, messageStoreTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
@Override
|
|||
|
|
public CqUnit getEarliestUnit() {
|
|||
|
|
return get(minOffsetInQueue);
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
|
|||
|
|
index 55d080829..c65f2a68b 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueInterface.java
|
|||
|
|
@@ -18,10 +18,12 @@
|
|||
|
|
package org.apache.rocketmq.store.queue;
|
|||
|
|
|
|||
|
|
import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
import org.apache.rocketmq.store.MessageFilter;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
public interface ConsumeQueueInterface extends FileQueueLifeCycle {
|
|||
|
|
/**
|
|||
|
|
@@ -44,6 +46,16 @@ public interface ConsumeQueueInterface extends FileQueueLifeCycle {
|
|||
|
|
*/
|
|||
|
|
ReferredIterator<CqUnit> iterateFrom(long startIndex);
|
|||
|
|
|
|||
|
|
+ /**
|
|||
|
|
+ * Get the units from the start offset.
|
|||
|
|
+ *
|
|||
|
|
+ * @param startIndex start index
|
|||
|
|
+ * @param count the unit counts will be iterated
|
|||
|
|
+ * @return the unit iterateFrom
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ ReferredIterator<CqUnit> iterateFrom(long startIndex, int count) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
/**
|
|||
|
|
* Get cq unit at specified index
|
|||
|
|
* @param index index
|
|||
|
|
@@ -51,6 +63,18 @@ public interface ConsumeQueueInterface extends FileQueueLifeCycle {
|
|||
|
|
*/
|
|||
|
|
CqUnit get(long index);
|
|||
|
|
|
|||
|
|
+ /**
|
|||
|
|
+ * Get earliest cq unit
|
|||
|
|
+ * @return the cq unit and message storeTime at index
|
|||
|
|
+ */
|
|||
|
|
+ Pair<CqUnit, Long> getCqUnitAndStoreTime(long index);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Get earliest cq unit
|
|||
|
|
+ * @return earliest cq unit and message storeTime
|
|||
|
|
+ */
|
|||
|
|
+ Pair<CqUnit, Long> getEarliestUnitAndStoreTime();
|
|||
|
|
+
|
|||
|
|
/**
|
|||
|
|
* Get earliest cq unit
|
|||
|
|
* @return earliest cq unit
|
|||
|
|
@@ -153,8 +177,9 @@ public interface ConsumeQueueInterface extends FileQueueLifeCycle {
|
|||
|
|
* Assign queue offset.
|
|||
|
|
* @param queueOffsetAssigner the delegated queue offset assigner
|
|||
|
|
* @param msg message itself
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
*/
|
|||
|
|
- void assignQueueOffset(QueueOffsetOperator queueOffsetAssigner, MessageExtBrokerInner msg);
|
|||
|
|
+ void assignQueueOffset(QueueOffsetOperator queueOffsetAssigner, MessageExtBrokerInner msg) throws RocksDBException;
|
|||
|
|
|
|||
|
|
|
|||
|
|
/**
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
index d03d15d65..616511b67 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStore.java
|
|||
|
|
@@ -16,64 +16,128 @@
|
|||
|
|
*/
|
|||
|
|
package org.apache.rocketmq.store.queue;
|
|||
|
|
|
|||
|
|
+import java.io.File;
|
|||
|
|
import java.nio.ByteBuffer;
|
|||
|
|
import java.util.ArrayList;
|
|||
|
|
+import java.util.Iterator;
|
|||
|
|
import java.util.List;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+import java.util.Map.Entry;
|
|||
|
|
+import java.util.Objects;
|
|||
|
|
+import java.util.Optional;
|
|||
|
|
import java.util.concurrent.BlockingQueue;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
import java.util.concurrent.CountDownLatch;
|
|||
|
|
import java.util.concurrent.ExecutorService;
|
|||
|
|
import java.util.concurrent.FutureTask;
|
|||
|
|
import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
|
import java.util.concurrent.TimeUnit;
|
|||
|
|
+import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
-import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
import org.apache.rocketmq.common.utils.QueueTypeUtils;
|
|||
|
|
import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
-import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
-import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
import org.apache.rocketmq.store.CommitLog;
|
|||
|
|
import org.apache.rocketmq.store.ConsumeQueue;
|
|||
|
|
import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
-import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
import org.apache.rocketmq.store.SelectMappedBufferResult;
|
|||
|
|
-import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
-
|
|||
|
|
-import java.io.File;
|
|||
|
|
-import java.util.Iterator;
|
|||
|
|
-import java.util.Map;
|
|||
|
|
-import java.util.Objects;
|
|||
|
|
-import java.util.Optional;
|
|||
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
-import java.util.concurrent.ConcurrentMap;
|
|||
|
|
|
|||
|
|
import static java.lang.String.format;
|
|||
|
|
import static org.apache.rocketmq.store.config.StorePathConfigHelper.getStorePathBatchConsumeQueue;
|
|||
|
|
import static org.apache.rocketmq.store.config.StorePathConfigHelper.getStorePathConsumeQueue;
|
|||
|
|
|
|||
|
|
-public class ConsumeQueueStore {
|
|||
|
|
- private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+public class ConsumeQueueStore extends AbstractConsumeQueueStore {
|
|||
|
|
|
|||
|
|
- protected final DefaultMessageStore messageStore;
|
|||
|
|
- protected final MessageStoreConfig messageStoreConfig;
|
|||
|
|
- protected final QueueOffsetOperator queueOffsetOperator = new QueueOffsetOperator();
|
|||
|
|
- protected final ConcurrentMap<String/* topic */, ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface>> consumeQueueTable;
|
|||
|
|
+ public ConsumeQueueStore(DefaultMessageStore messageStore) {
|
|||
|
|
+ super(messageStore);
|
|||
|
|
+ }
|
|||
|
|
|
|||
|
|
- public ConsumeQueueStore(DefaultMessageStore messageStore, MessageStoreConfig messageStoreConfig) {
|
|||
|
|
- this.messageStore = messageStore;
|
|||
|
|
- this.messageStoreConfig = messageStoreConfig;
|
|||
|
|
- this.consumeQueueTable = new ConcurrentHashMap<>(32);
|
|||
|
|
+ @Override
|
|||
|
|
+ public void start() {
|
|||
|
|
+ log.info("Default ConsumeQueueStore start!");
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private FileQueueLifeCycle getLifeCycle(String topic, int queueId) {
|
|||
|
|
- return findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean load() {
|
|||
|
|
+ boolean cqLoadResult = loadConsumeQueues(getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), CQType.SimpleCQ);
|
|||
|
|
+ boolean bcqLoadResult = loadConsumeQueues(getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), CQType.BatchCQ);
|
|||
|
|
+ return cqLoadResult && bcqLoadResult;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean loadAfterDestroy() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recover() {
|
|||
|
|
+ for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
+ for (ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
+ this.recover(logic);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean recoverConcurrently() {
|
|||
|
|
+ int count = 0;
|
|||
|
|
+ for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
+ count += maps.values().size();
|
|||
|
|
+ }
|
|||
|
|
+ final CountDownLatch countDownLatch = new CountDownLatch(count);
|
|||
|
|
+ BlockingQueue<Runnable> recoverQueue = new LinkedBlockingQueue<>();
|
|||
|
|
+ final ExecutorService executor = buildExecutorService(recoverQueue, "RecoverConsumeQueueThread_");
|
|||
|
|
+ List<FutureTask<Boolean>> result = new ArrayList<>(count);
|
|||
|
|
+ try {
|
|||
|
|
+ for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
+ for (final ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
+ FutureTask<Boolean> futureTask = new FutureTask<>(() -> {
|
|||
|
|
+ boolean ret = true;
|
|||
|
|
+ try {
|
|||
|
|
+ logic.recover();
|
|||
|
|
+ } catch (Throwable e) {
|
|||
|
|
+ ret = false;
|
|||
|
|
+ log.error("Exception occurs while recover consume queue concurrently, " +
|
|||
|
|
+ "topic={}, queueId={}", logic.getTopic(), logic.getQueueId(), e);
|
|||
|
|
+ } finally {
|
|||
|
|
+ countDownLatch.countDown();
|
|||
|
|
+ }
|
|||
|
|
+ return ret;
|
|||
|
|
+ });
|
|||
|
|
+
|
|||
|
|
+ result.add(futureTask);
|
|||
|
|
+ executor.submit(futureTask);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ countDownLatch.await();
|
|||
|
|
+ for (FutureTask<Boolean> task : result) {
|
|||
|
|
+ if (task != null && task.isDone()) {
|
|||
|
|
+ if (!task.get()) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.error("Exception occurs while recover consume queue concurrently", e);
|
|||
|
|
+ return false;
|
|||
|
|
+ } finally {
|
|||
|
|
+ executor.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean shutdown() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
public long rollNextFile(ConsumeQueueInterface consumeQueue, final long offset) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.rollNextFile(offset);
|
|||
|
|
@@ -83,32 +147,53 @@ public class ConsumeQueueStore {
|
|||
|
|
consumeQueue.correctMinOffset(minCommitLogOffset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- /**
|
|||
|
|
- * Apply the dispatched request and build the consume queue. This function should be idempotent.
|
|||
|
|
- *
|
|||
|
|
- * @param consumeQueue consume queue
|
|||
|
|
- * @param request dispatch request
|
|||
|
|
- */
|
|||
|
|
- public void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request) {
|
|||
|
|
- consumeQueue.putMessagePositionInfoWrapper(request);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
+ @Override
|
|||
|
|
public void putMessagePositionInfoWrapper(DispatchRequest dispatchRequest) {
|
|||
|
|
ConsumeQueueInterface cq = this.findOrCreateConsumeQueue(dispatchRequest.getTopic(), dispatchRequest.getQueueId());
|
|||
|
|
this.putMessagePositionInfoWrapper(cq, dispatchRequest);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public List<ByteBuffer> rangeQuery(String topic, int queueId, long startIndex, int num) {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ByteBuffer get(String topic, int queueId, long startIndex) {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxOffsetInQueue(String topic, int queueId) {
|
|||
|
|
+ ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ if (logic != null) {
|
|||
|
|
+ return logic.getMaxOffsetInQueue();
|
|||
|
|
+ }
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) {
|
|||
|
|
+ ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ if (logic != null) {
|
|||
|
|
+ long resultOffset = logic.getOffsetInQueueByTime(timestamp, boundaryType);
|
|||
|
|
+ // Make sure the result offset is in valid range.
|
|||
|
|
+ resultOffset = Math.max(resultOffset, logic.getMinOffsetInQueue());
|
|||
|
|
+ resultOffset = Math.min(resultOffset, logic.getMaxOffsetInQueue());
|
|||
|
|
+ return resultOffset;
|
|||
|
|
+ }
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private FileQueueLifeCycle getLifeCycle(String topic, int queueId) {
|
|||
|
|
+ return findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public boolean load(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.load();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public boolean load() {
|
|||
|
|
- boolean cqLoadResult = loadConsumeQueues(getStorePathConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), CQType.SimpleCQ);
|
|||
|
|
- boolean bcqLoadResult = loadConsumeQueues(getStorePathBatchConsumeQueue(this.messageStoreConfig.getStorePathRootDir()), CQType.BatchCQ);
|
|||
|
|
- return cqLoadResult && bcqLoadResult;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
private boolean loadConsumeQueues(String storePath, CQType cqType) {
|
|||
|
|
File dirLogic = new File(storePath);
|
|||
|
|
File[] fileTopicList = dirLogic.listFiles();
|
|||
|
|
@@ -189,62 +274,17 @@ public class ConsumeQueueStore {
|
|||
|
|
fileQueueLifeCycle.recover();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void recover() {
|
|||
|
|
- for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
- for (ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
- this.recover(logic);
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public boolean recoverConcurrently() {
|
|||
|
|
- int count = 0;
|
|||
|
|
- for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
- count += maps.values().size();
|
|||
|
|
- }
|
|||
|
|
- final CountDownLatch countDownLatch = new CountDownLatch(count);
|
|||
|
|
- BlockingQueue<Runnable> recoverQueue = new LinkedBlockingQueue<>();
|
|||
|
|
- final ExecutorService executor = buildExecutorService(recoverQueue, "RecoverConsumeQueueThread_");
|
|||
|
|
- List<FutureTask<Boolean>> result = new ArrayList<>(count);
|
|||
|
|
- try {
|
|||
|
|
- for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
- for (final ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
- FutureTask<Boolean> futureTask = new FutureTask<>(() -> {
|
|||
|
|
- boolean ret = true;
|
|||
|
|
- try {
|
|||
|
|
- logic.recover();
|
|||
|
|
- } catch (Throwable e) {
|
|||
|
|
- ret = false;
|
|||
|
|
- log.error("Exception occurs while recover consume queue concurrently, " +
|
|||
|
|
- "topic={}, queueId={}", logic.getTopic(), logic.getQueueId(), e);
|
|||
|
|
- } finally {
|
|||
|
|
- countDownLatch.countDown();
|
|||
|
|
- }
|
|||
|
|
- return ret;
|
|||
|
|
- });
|
|||
|
|
-
|
|||
|
|
- result.add(futureTask);
|
|||
|
|
- executor.submit(futureTask);
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- countDownLatch.await();
|
|||
|
|
- for (FutureTask<Boolean> task : result) {
|
|||
|
|
- if (task != null && task.isDone()) {
|
|||
|
|
- if (!task.get()) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- }
|
|||
|
|
- } catch (Exception e) {
|
|||
|
|
- log.error("Exception occurs while recover consume queue concurrently", e);
|
|||
|
|
- return false;
|
|||
|
|
- } finally {
|
|||
|
|
- executor.shutdown();
|
|||
|
|
+ @Override
|
|||
|
|
+ public Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId) {
|
|||
|
|
+ ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ if (logic != null) {
|
|||
|
|
+ return logic.getMaxPhysicOffset();
|
|||
|
|
}
|
|||
|
|
- return true;
|
|||
|
|
+ return null;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public long getMaxOffsetInConsumeQueue() {
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxPhyOffsetInConsumeQueue() {
|
|||
|
|
long maxPhysicOffset = -1L;
|
|||
|
|
for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
for (ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
@@ -256,11 +296,22 @@ public class ConsumeQueueStore {
|
|||
|
|
return maxPhysicOffset;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMinOffsetInQueue(String topic, int queueId) {
|
|||
|
|
+ ConsumeQueueInterface logic = findOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
+ if (logic != null) {
|
|||
|
|
+ return logic.getMinOffsetInQueue();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return -1;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public void checkSelf(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
fileQueueLifeCycle.checkSelf();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public void checkSelf() {
|
|||
|
|
for (Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> topicEntry : this.consumeQueueTable.entrySet()) {
|
|||
|
|
for (Map.Entry<Integer, ConsumeQueueInterface> cqEntry : topicEntry.getValue().entrySet()) {
|
|||
|
|
@@ -269,16 +320,19 @@ public class ConsumeQueueStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.flush(flushLeastPages);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public void destroy(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
fileQueueLifeCycle.destroy();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.deleteExpiredFile(minCommitLogPos);
|
|||
|
|
@@ -300,21 +354,20 @@ public class ConsumeQueueStore {
|
|||
|
|
fileQueueLifeCycle.cleanSwappedMap(forceCleanSwapIntervalMs);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.isFirstFileAvailable();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public boolean isFirstFileExist(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
FileQueueLifeCycle fileQueueLifeCycle = getLifeCycle(consumeQueue.getTopic(), consumeQueue.getQueueId());
|
|||
|
|
return fileQueueLifeCycle.isFirstFileExist();
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) {
|
|||
|
|
- return doFindOrCreateConsumeQueue(topic, queueId);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- private ConsumeQueueInterface doFindOrCreateConsumeQueue(String topic, int queueId) {
|
|||
|
|
ConcurrentMap<Integer, ConsumeQueueInterface> map = consumeQueueTable.get(topic);
|
|||
|
|
if (null == map) {
|
|||
|
|
ConcurrentMap<Integer, ConsumeQueueInterface> newMap = new ConcurrentHashMap<>(128);
|
|||
|
|
@@ -361,46 +414,15 @@ public class ConsumeQueueStore {
|
|||
|
|
return logic;
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public Long getMaxOffset(String topic, int queueId) {
|
|||
|
|
- return this.queueOffsetOperator.currentQueueOffset(topic + "-" + queueId);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void setTopicQueueTable(ConcurrentMap<String, Long> topicQueueTable) {
|
|||
|
|
- this.queueOffsetOperator.setTopicQueueTable(topicQueueTable);
|
|||
|
|
- this.queueOffsetOperator.setLmqTopicQueueTable(topicQueueTable);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public ConcurrentMap getTopicQueueTable() {
|
|||
|
|
- return this.queueOffsetOperator.getTopicQueueTable();
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
public void setBatchTopicQueueTable(ConcurrentMap<String, Long> batchTopicQueueTable) {
|
|||
|
|
this.queueOffsetOperator.setBatchTopicQueueTable(batchTopicQueueTable);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void assignQueueOffset(MessageExtBrokerInner msg) {
|
|||
|
|
- ConsumeQueueInterface consumeQueue = findOrCreateConsumeQueue(msg.getTopic(), msg.getQueueId());
|
|||
|
|
- consumeQueue.assignQueueOffset(this.queueOffsetOperator, msg);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public void increaseQueueOffset(MessageExtBrokerInner msg, short messageNum) {
|
|||
|
|
- ConsumeQueueInterface consumeQueue = findOrCreateConsumeQueue(msg.getTopic(), msg.getQueueId());
|
|||
|
|
- consumeQueue.increaseQueueOffset(this.queueOffsetOperator, msg, messageNum);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
public void updateQueueOffset(String topic, int queueId, long offset) {
|
|||
|
|
String topicQueueKey = topic + "-" + queueId;
|
|||
|
|
this.queueOffsetOperator.updateQueueOffset(topicQueueKey, offset);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void removeTopicQueueTable(String topic, Integer queueId) {
|
|||
|
|
- this.queueOffsetOperator.remove(topic, queueId);
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
- public ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> getConsumeQueueTable() {
|
|||
|
|
- return consumeQueueTable;
|
|||
|
|
- }
|
|||
|
|
-
|
|||
|
|
private void putConsumeQueue(final String topic, final int queueId, final ConsumeQueueInterface consumeQueue) {
|
|||
|
|
ConcurrentMap<Integer/* queueId */, ConsumeQueueInterface> map = this.consumeQueueTable.get(topic);
|
|||
|
|
if (null == map) {
|
|||
|
|
@@ -412,6 +434,7 @@ public class ConsumeQueueStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public void recoverOffsetTable(long minPhyOffset) {
|
|||
|
|
ConcurrentMap<String, Long> cqOffsetTable = new ConcurrentHashMap<>(1024);
|
|||
|
|
ConcurrentMap<String, Long> bcqOffsetTable = new ConcurrentHashMap<>(1024);
|
|||
|
|
@@ -431,7 +454,7 @@ public class ConsumeQueueStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- //Correct unSubmit consumeOffset
|
|||
|
|
+ // Correct unSubmit consumeOffset
|
|||
|
|
if (messageStoreConfig.isDuplicationEnable()) {
|
|||
|
|
SelectMappedBufferResult lastBuffer = null;
|
|||
|
|
long startReadOffset = messageStore.getCommitLog().getConfirmOffset() == -1 ? 0 : messageStore.getCommitLog().getConfirmOffset();
|
|||
|
|
@@ -476,6 +499,7 @@ public class ConsumeQueueStore {
|
|||
|
|
this.setBatchTopicQueueTable(bcqOffsetTable);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public void destroy() {
|
|||
|
|
for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
for (ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
@@ -484,8 +508,9 @@ public class ConsumeQueueStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public void cleanExpired(long minCommitLogOffset) {
|
|||
|
|
- Iterator<Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
|
|||
|
|
+ Iterator<Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>>> it = this.consumeQueueTable.entrySet().iterator();
|
|||
|
|
while (it.hasNext()) {
|
|||
|
|
Map.Entry<String, ConcurrentMap<Integer, ConsumeQueueInterface>> next = it.next();
|
|||
|
|
String topic = next.getKey();
|
|||
|
|
@@ -526,14 +551,16 @@ public class ConsumeQueueStore {
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- public void truncateDirty(long phyOffset) {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void truncateDirty(long offsetToTruncate) {
|
|||
|
|
for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
for (ConsumeQueueInterface logic : maps.values()) {
|
|||
|
|
- this.truncateDirtyLogicFiles(logic, phyOffset);
|
|||
|
|
+ this.truncateDirtyLogicFiles(logic, offsetToTruncate);
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ @Override
|
|||
|
|
public long getTotalSize() {
|
|||
|
|
long totalSize = 0;
|
|||
|
|
for (ConcurrentMap<Integer, ConsumeQueueInterface> maps : this.consumeQueueTable.values()) {
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..268803dcc
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/ConsumeQueueStoreInterface.java
|
|||
|
|
@@ -0,0 +1,289 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.nio.ByteBuffer;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+public interface ConsumeQueueStoreInterface {
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Start the consumeQueueStore
|
|||
|
|
+ */
|
|||
|
|
+ void start();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Load from file.
|
|||
|
|
+ * @return true if loaded successfully.
|
|||
|
|
+ */
|
|||
|
|
+ boolean load();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * load after destroy
|
|||
|
|
+ */
|
|||
|
|
+ boolean loadAfterDestroy();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Recover from file.
|
|||
|
|
+ */
|
|||
|
|
+ void recover();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Recover concurrently from file.
|
|||
|
|
+ * @return true if recovered successfully.
|
|||
|
|
+ */
|
|||
|
|
+ boolean recoverConcurrently();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Shutdown the consumeQueueStore
|
|||
|
|
+ * @return true if shutdown successfully.
|
|||
|
|
+ */
|
|||
|
|
+ boolean shutdown();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * destroy all consumeQueues
|
|||
|
|
+ */
|
|||
|
|
+ void destroy();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * destroy the specific consumeQueue
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ void destroy(ConsumeQueueInterface consumeQueue) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Flush cache to file.
|
|||
|
|
+ * @param consumeQueue the consumeQueue will be flushed
|
|||
|
|
+ * @param flushLeastPages the minimum number of pages to be flushed
|
|||
|
|
+ * @return true if any data has been flushed.
|
|||
|
|
+ */
|
|||
|
|
+ boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * clean expired data from minPhyOffset
|
|||
|
|
+ * @param minPhyOffset
|
|||
|
|
+ */
|
|||
|
|
+ void cleanExpired(long minPhyOffset);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Check files.
|
|||
|
|
+ */
|
|||
|
|
+ void checkSelf();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Delete expired files ending at min commit log position.
|
|||
|
|
+ * @param consumeQueue
|
|||
|
|
+ * @param minCommitLogPos min commit log position
|
|||
|
|
+ * @return deleted file numbers.
|
|||
|
|
+ */
|
|||
|
|
+ int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Is the first file available?
|
|||
|
|
+ * @param consumeQueue
|
|||
|
|
+ * @return true if it's available
|
|||
|
|
+ */
|
|||
|
|
+ boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Does the first file exist?
|
|||
|
|
+ * @param consumeQueue
|
|||
|
|
+ * @return true if it exists
|
|||
|
|
+ */
|
|||
|
|
+ boolean isFirstFileExist(ConsumeQueueInterface consumeQueue);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Roll to next file.
|
|||
|
|
+ * @param consumeQueue
|
|||
|
|
+ * @param offset next beginning offset
|
|||
|
|
+ * @return the beginning offset of the next file
|
|||
|
|
+ */
|
|||
|
|
+ long rollNextFile(ConsumeQueueInterface consumeQueue, final long offset);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * truncate dirty data
|
|||
|
|
+ * @param offsetToTruncate
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ void truncateDirty(long offsetToTruncate) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Apply the dispatched request and build the consume queue. This function should be idempotent.
|
|||
|
|
+ *
|
|||
|
|
+ * @param consumeQueue consume queue
|
|||
|
|
+ * @param request dispatch request
|
|||
|
|
+ */
|
|||
|
|
+ void putMessagePositionInfoWrapper(ConsumeQueueInterface consumeQueue, DispatchRequest request);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Apply the dispatched request. This function should be idempotent.
|
|||
|
|
+ *
|
|||
|
|
+ * @param request dispatch request
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode will throw exception
|
|||
|
|
+ */
|
|||
|
|
+ void putMessagePositionInfoWrapper(DispatchRequest request) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * range query cqUnit(ByteBuffer) in rocksdb
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @param startIndex
|
|||
|
|
+ * @param num
|
|||
|
|
+ * @return the byteBuffer list of the topic-queueId in rocksdb
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ List<ByteBuffer> rangeQuery(final String topic, final int queueId, final long startIndex, final int num) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * query cqUnit(ByteBuffer) in rocksdb
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @param startIndex
|
|||
|
|
+ * @return the byteBuffer of the topic-queueId in rocksdb
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ ByteBuffer get(final String topic, final int queueId, final long startIndex) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get consumeQueue table
|
|||
|
|
+ * @return the consumeQueue table
|
|||
|
|
+ */
|
|||
|
|
+ ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> getConsumeQueueTable();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Assign queue offset.
|
|||
|
|
+ * @param msg message itself
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ void assignQueueOffset(MessageExtBrokerInner msg) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Increase queue offset.
|
|||
|
|
+ * @param msg message itself
|
|||
|
|
+ * @param messageNum message number
|
|||
|
|
+ */
|
|||
|
|
+ void increaseQueueOffset(MessageExtBrokerInner msg, short messageNum);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * recover topicQueue table by minPhyOffset
|
|||
|
|
+ * @param minPhyOffset
|
|||
|
|
+ */
|
|||
|
|
+ void recoverOffsetTable(long minPhyOffset);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * set topicQueue table
|
|||
|
|
+ * @param topicQueueTable
|
|||
|
|
+ */
|
|||
|
|
+ void setTopicQueueTable(ConcurrentMap<String, Long> topicQueueTable);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * remove topic-queueId from topicQueue table
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ */
|
|||
|
|
+ void removeTopicQueueTable(String topic, Integer queueId);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get topicQueue table
|
|||
|
|
+ * @return the topicQueue table
|
|||
|
|
+ */
|
|||
|
|
+ ConcurrentMap getTopicQueueTable();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get the max physical offset in consumeQueue
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get maxOffset of specific topic-queueId in topicQueue table
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @return the max offset in QueueOffsetOperator
|
|||
|
|
+ */
|
|||
|
|
+ Long getMaxOffset(String topic, int queueId);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get max physic offset in consumeQueue
|
|||
|
|
+ * @return the max physic offset in consumeQueue
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ long getMaxPhyOffsetInConsumeQueue() throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get min logic offset of specific topic-queueId in consumeQueue
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @return the min logic offset of specific topic-queueId in consumeQueue
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ long getMinOffsetInQueue(final String topic, final int queueId) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get max logic offset of specific topic-queueId in consumeQueue
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @return the max logic offset of specific topic-queueId in consumeQueue
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ long getMaxOffsetInQueue(final String topic, final int queueId) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Get the message whose timestamp is the smallest, greater than or equal to the given time and when there are more
|
|||
|
|
+ * than one message satisfy the condition, decide which one to return based on boundaryType.
|
|||
|
|
+ * @param timestamp timestamp
|
|||
|
|
+ * @param boundaryType Lower or Upper
|
|||
|
|
+ * @return the offset(index)
|
|||
|
|
+ * @throws RocksDBException only in rocksdb mode
|
|||
|
|
+ */
|
|||
|
|
+ long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) throws RocksDBException;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * find or create the consumeQueue
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @return the consumeQueue
|
|||
|
|
+ */
|
|||
|
|
+ ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * find the consumeQueueMap of topic
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @return the consumeQueueMap of topic
|
|||
|
|
+ */
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> findConsumeQueueMap(String topic);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * get the total size of all consumeQueue
|
|||
|
|
+ * @return the total size of all consumeQueue
|
|||
|
|
+ */
|
|||
|
|
+ long getTotalSize();
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Get store time from commitlog by cqUnit
|
|||
|
|
+ * @param cqUnit
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ long getStoreTime(CqUnit cqUnit);
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/MultiDispatch.java b/store/src/main/java/org/apache/rocketmq/store/queue/MultiDispatch.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..d6291d908
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/MultiDispatch.java
|
|||
|
|
@@ -0,0 +1,76 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+
|
|||
|
|
+import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
+import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+
|
|||
|
|
+public class MultiDispatch {
|
|||
|
|
+
|
|||
|
|
+ public static String lmqQueueKey(String queueName) {
|
|||
|
|
+ StringBuilder keyBuilder = new StringBuilder();
|
|||
|
|
+ keyBuilder.append(queueName);
|
|||
|
|
+ keyBuilder.append('-');
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ keyBuilder.append(queueId);
|
|||
|
|
+ return keyBuilder.toString();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static boolean isNeedHandleMultiDispatch(MessageStoreConfig messageStoreConfig, String topic) {
|
|||
|
|
+ return messageStoreConfig.isEnableMultiDispatch()
|
|||
|
|
+ && !topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)
|
|||
|
|
+ && !topic.startsWith(TopicValidator.SYSTEM_TOPIC_PREFIX)
|
|||
|
|
+ && !topic.equals(TopicValidator.RMQ_SYS_SCHEDULE_TOPIC);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static boolean checkMultiDispatchQueue(MessageStoreConfig messageStoreConfig, DispatchRequest dispatchRequest) {
|
|||
|
|
+ if (!isNeedHandleMultiDispatch(messageStoreConfig, dispatchRequest.getTopic())) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ Map<String, String> prop = dispatchRequest.getPropertiesMap();
|
|||
|
|
+ if (prop == null || prop.isEmpty()) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ String multiDispatchQueue = prop.get(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
+ String multiQueueOffset = prop.get(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET);
|
|||
|
|
+ if (StringUtils.isBlank(multiDispatchQueue) || StringUtils.isBlank(multiQueueOffset)) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static List<DispatchRequest> checkMultiDispatchQueue(MessageStoreConfig messageStoreConfig, List<DispatchRequest> dispatchRequests) {
|
|||
|
|
+ if (!messageStoreConfig.isEnableMultiDispatch() || dispatchRequests == null || dispatchRequests.size() == 0) {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ List<DispatchRequest> result = new ArrayList<>();
|
|||
|
|
+ for (DispatchRequest dispatchRequest : dispatchRequests) {
|
|||
|
|
+ if (checkMultiDispatchQueue(messageStoreConfig, dispatchRequest)) {
|
|||
|
|
+ result.add(dispatchRequest);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return dispatchRequests;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetOperator.java b/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetOperator.java
|
|||
|
|
index 2545bbf52..8da374828 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetOperator.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/QueueOffsetOperator.java
|
|||
|
|
@@ -41,6 +41,10 @@ public class QueueOffsetOperator {
|
|||
|
|
return ConcurrentHashMapUtils.computeIfAbsent(this.topicQueueTable, topicQueueKey, k -> 0L);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public Long getTopicQueueNextOffset(String topicQueueKey) {
|
|||
|
|
+ return this.topicQueueTable.get(topicQueueKey);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public void increaseQueueOffset(String topicQueueKey, short messageNum) {
|
|||
|
|
Long queueOffset = ConcurrentHashMapUtils.computeIfAbsent(this.topicQueueTable, topicQueueKey, k -> 0L);
|
|||
|
|
topicQueueTable.put(topicQueueKey, queueOffset + messageNum);
|
|||
|
|
@@ -63,6 +67,10 @@ public class QueueOffsetOperator {
|
|||
|
|
return ConcurrentHashMapUtils.computeIfAbsent(this.lmqTopicQueueTable, topicQueueKey, k -> 0L);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
+ public Long getLmqTopicQueueNextOffset(String topicQueueKey) {
|
|||
|
|
+ return this.lmqTopicQueueTable.get(topicQueueKey);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
public void increaseLmqOffset(String topicQueueKey, short messageNum) {
|
|||
|
|
Long lmqOffset = ConcurrentHashMapUtils.computeIfAbsent(this.lmqTopicQueueTable, topicQueueKey, k -> 0L);
|
|||
|
|
this.lmqTopicQueueTable.put(topicQueueKey, lmqOffset + messageNum);
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..759be395d
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueue.java
|
|||
|
|
@@ -0,0 +1,437 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.nio.ByteBuffer;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+
|
|||
|
|
+import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
+import org.apache.rocketmq.common.attribute.CQType;
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageAccessor;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.ConsumeQueue;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.MessageFilter;
|
|||
|
|
+import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBConsumeQueue implements ConsumeQueueInterface {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ private final MessageStore messageStore;
|
|||
|
|
+ private final String topic;
|
|||
|
|
+ private final int queueId;
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueue(final MessageStore messageStore, final String topic, final int queueId) {
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ this.topic = topic;
|
|||
|
|
+ this.queueId = queueId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueue(final String topic, final int queueId) {
|
|||
|
|
+ this.messageStore = null;
|
|||
|
|
+ this.topic = topic;
|
|||
|
|
+ this.queueId = queueId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean load() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recover() {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void checkSelf() {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean flush(final int flushLeastPages) {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void destroy() {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void truncateDirtyLogicFiles(long maxCommitLogPos) {
|
|||
|
|
+ // ignored
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public int deleteExpiredFile(long minCommitLogPos) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long rollNextFile(long nextBeginOffset) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean isFirstFileAvailable() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean isFirstFileExist() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void cleanSwappedMap(long forceCleanSwapIntervalMs) {
|
|||
|
|
+ // ignore
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxOffsetInQueue() {
|
|||
|
|
+ try {
|
|||
|
|
+ return this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getMaxOffsetInQueue Failed. topic: {}, queueId: {}", topic, queueId, e);
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMessageTotalInQueue() {
|
|||
|
|
+ try {
|
|||
|
|
+ long maxOffsetInQueue = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId);
|
|||
|
|
+ long minOffsetInQueue = this.messageStore.getQueueStore().getMinOffsetInQueue(topic, queueId);
|
|||
|
|
+ return maxOffsetInQueue - minOffsetInQueue;
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getMessageTotalInQueue Failed. topic: {}, queueId: {}, {}", topic, queueId, e);
|
|||
|
|
+ }
|
|||
|
|
+ return -1;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * We already implement it in RocksDBConsumeQueueStore.
|
|||
|
|
+ * @see RocksDBConsumeQueueStore#getOffsetInQueueByTime
|
|||
|
|
+ * @param timestamp timestamp
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getOffsetInQueueByTime(long timestamp) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * We already implement it in RocksDBConsumeQueueStore.
|
|||
|
|
+ * @see RocksDBConsumeQueueStore#getOffsetInQueueByTime
|
|||
|
|
+ * @param timestamp timestamp
|
|||
|
|
+ * @param boundaryType Lower or Upper
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getOffsetInQueueByTime(long timestamp, BoundaryType boundaryType) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxPhysicOffset() {
|
|||
|
|
+ Long maxPhyOffset = this.messageStore.getQueueStore().getMaxPhyOffsetInConsumeQueue(topic, queueId);
|
|||
|
|
+ return maxPhyOffset == null ? -1 : maxPhyOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMinLogicOffset() {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CQType getCQType() {
|
|||
|
|
+ return CQType.RocksDBCQ;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getTotalSize() {
|
|||
|
|
+ // ignored
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public int getUnitSize() {
|
|||
|
|
+ // attention: unitSize should equal to 'ConsumeQueue.CQ_STORE_UNIT_SIZE'
|
|||
|
|
+ return ConsumeQueue.CQ_STORE_UNIT_SIZE;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Ignored, we already implement this method
|
|||
|
|
+ * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable#getMinCqOffset(String, int)
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void correctMinOffset(long minCommitLogOffset) {
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Ignored, in rocksdb mode, we build cq in RocksDBConsumeQueueStore
|
|||
|
|
+ * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore#putMessagePosition()
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void putMessagePositionInfoWrapper(DispatchRequest request) {
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void assignQueueOffset(QueueOffsetOperator queueOffsetOperator, MessageExtBrokerInner msg) throws RocksDBException {
|
|||
|
|
+ String topicQueueKey = getTopic() + "-" + getQueueId();
|
|||
|
|
+ Long queueOffset = queueOffsetOperator.getTopicQueueNextOffset(topicQueueKey);
|
|||
|
|
+ if (queueOffset == null) {
|
|||
|
|
+ // we will recover topic queue table from rocksdb when we use it.
|
|||
|
|
+ queueOffset = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId);
|
|||
|
|
+ queueOffsetOperator.updateQueueOffset(topicQueueKey, queueOffset);
|
|||
|
|
+ }
|
|||
|
|
+ msg.setQueueOffset(queueOffset);
|
|||
|
|
+
|
|||
|
|
+ // Handling the multi dispatch message. In the context of a light message queue (as defined in RIP-28),
|
|||
|
|
+ // light message queues are constructed based on message properties, which requires special handling of queue offset of the light message queue.
|
|||
|
|
+ if (!MultiDispatch.isNeedHandleMultiDispatch(this.messageStore.getMessageStoreConfig(), msg.getTopic())) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String multiDispatchQueue = msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
+ if (StringUtils.isBlank(multiDispatchQueue)) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER);
|
|||
|
|
+ Long[] queueOffsets = new Long[queues.length];
|
|||
|
|
+ for (int i = 0; i < queues.length; i++) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queues[i])) {
|
|||
|
|
+ String key = MultiDispatch.lmqQueueKey(queues[i]);
|
|||
|
|
+ queueOffsets[i] = queueOffsetOperator.getLmqTopicQueueNextOffset(key);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ MessageAccessor.putProperty(msg, MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET,
|
|||
|
|
+ StringUtils.join(queueOffsets, MixAll.MULTI_DISPATCH_QUEUE_SPLITTER));
|
|||
|
|
+ msg.removeWaitStorePropertyString();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void increaseQueueOffset(QueueOffsetOperator queueOffsetOperator, MessageExtBrokerInner msg, short messageNum) {
|
|||
|
|
+ String topicQueueKey = getTopic() + "-" + getQueueId();
|
|||
|
|
+ queueOffsetOperator.increaseQueueOffset(topicQueueKey, messageNum);
|
|||
|
|
+
|
|||
|
|
+ // Handling the multi dispatch message. In the context of a light message queue (as defined in RIP-28),
|
|||
|
|
+ // light message queues are constructed based on message properties, which requires special handling of queue offset of the light message queue.
|
|||
|
|
+ if (!MultiDispatch.isNeedHandleMultiDispatch(this.messageStore.getMessageStoreConfig(), msg.getTopic())) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String multiDispatchQueue = msg.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH);
|
|||
|
|
+ if (StringUtils.isBlank(multiDispatchQueue)) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String[] queues = multiDispatchQueue.split(MixAll.MULTI_DISPATCH_QUEUE_SPLITTER);
|
|||
|
|
+ for (int i = 0; i < queues.length; i++) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableLmq() && MixAll.isLmq(queues[i])) {
|
|||
|
|
+ String key = MultiDispatch.lmqQueueKey(queues[i]);
|
|||
|
|
+ queueOffsetOperator.increaseLmqOffset(key, (short) 1);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long estimateMessageCount(long from, long to, MessageFilter filter) {
|
|||
|
|
+ // todo
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMinOffsetInQueue() {
|
|||
|
|
+ return this.messageStore.getMinOffsetInQueue(this.topic, this.queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private int pullNum(long cqOffset, long maxCqOffset) {
|
|||
|
|
+ long diffLong = maxCqOffset - cqOffset;
|
|||
|
|
+ if (diffLong < Integer.MAX_VALUE) {
|
|||
|
|
+ int diffInt = (int) diffLong;
|
|||
|
|
+ return diffInt > 16 ? 16 : diffInt;
|
|||
|
|
+ }
|
|||
|
|
+ return 16;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ReferredIterator<CqUnit> iterateFrom(final long startIndex) {
|
|||
|
|
+ try {
|
|||
|
|
+ long maxCqOffset = getMaxOffsetInQueue();
|
|||
|
|
+ if (startIndex < maxCqOffset) {
|
|||
|
|
+ int num = pullNum(startIndex, maxCqOffset);
|
|||
|
|
+ return iterateFrom0(startIndex, num);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ log.error("[RocksDBConsumeQueue] iterateFrom error!", e);
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ReferredIterator<CqUnit> iterateFrom(long startIndex, int count) throws RocksDBException {
|
|||
|
|
+ long maxCqOffset = getMaxOffsetInQueue();
|
|||
|
|
+ if (startIndex < maxCqOffset) {
|
|||
|
|
+ int num = Math.min((int)(maxCqOffset - startIndex), count);
|
|||
|
|
+ return iterateFrom0(startIndex, num);
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CqUnit get(long index) {
|
|||
|
|
+ Pair<CqUnit, Long> pair = getCqUnitAndStoreTime(index);
|
|||
|
|
+ return pair == null ? null : pair.getObject1();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getCqUnitAndStoreTime(long index) {
|
|||
|
|
+ ByteBuffer byteBuffer;
|
|||
|
|
+ try {
|
|||
|
|
+ byteBuffer = this.messageStore.getQueueStore().get(topic, queueId, index);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getUnitAndStoreTime Failed. topic: {}, queueId: {}", topic, queueId, e);
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ if (byteBuffer == null || byteBuffer.remaining() < RocksDBConsumeQueueTable.CQ_UNIT_SIZE) {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ long phyOffset = byteBuffer.getLong();
|
|||
|
|
+ int size = byteBuffer.getInt();
|
|||
|
|
+ long tagCode = byteBuffer.getLong();
|
|||
|
|
+ long messageStoreTime = byteBuffer.getLong();
|
|||
|
|
+ return new Pair<>(new CqUnit(index, phyOffset, size, tagCode), messageStoreTime);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Pair<CqUnit, Long> getEarliestUnitAndStoreTime() {
|
|||
|
|
+ try {
|
|||
|
|
+ long minOffset = this.messageStore.getQueueStore().getMinOffsetInQueue(topic, queueId);
|
|||
|
|
+ return getCqUnitAndStoreTime(minOffset);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getEarliestUnitAndStoreTime Failed. topic: {}, queueId: {}", topic, queueId, e);
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CqUnit getEarliestUnit() {
|
|||
|
|
+ Pair<CqUnit, Long> pair = getEarliestUnitAndStoreTime();
|
|||
|
|
+ return pair == null ? null : pair.getObject1();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CqUnit getLatestUnit() {
|
|||
|
|
+ try {
|
|||
|
|
+ long maxOffset = this.messageStore.getQueueStore().getMaxOffsetInQueue(topic, queueId);
|
|||
|
|
+ return get(maxOffset);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("getLatestUnit Failed. topic: {}, queueId: {}, {}", topic, queueId, e.getMessage());
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getLastOffset() {
|
|||
|
|
+ return getMaxPhysicOffset();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ReferredIterator<CqUnit> iterateFrom0(final long startIndex, final int count) throws RocksDBException {
|
|||
|
|
+ List<ByteBuffer> byteBufferList = this.messageStore.getQueueStore().rangeQuery(topic, queueId, startIndex, count);
|
|||
|
|
+ if (byteBufferList == null || byteBufferList.isEmpty()) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ log.warn("iterateFrom0 - find nothing, startIndex:{}, count:{}", startIndex, count);
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ return new RocksDBConsumeQueueIterator(byteBufferList, startIndex);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String getTopic() {
|
|||
|
|
+ return topic;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public int getQueueId() {
|
|||
|
|
+ return queueId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private class RocksDBConsumeQueueIterator implements ReferredIterator<CqUnit> {
|
|||
|
|
+ private final List<ByteBuffer> byteBufferList;
|
|||
|
|
+ private final long startIndex;
|
|||
|
|
+ private final int totalCount;
|
|||
|
|
+ private int currentIndex;
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueueIterator(final List<ByteBuffer> byteBufferList, final long startIndex) {
|
|||
|
|
+ this.byteBufferList = byteBufferList;
|
|||
|
|
+ this.startIndex = startIndex;
|
|||
|
|
+ this.totalCount = byteBufferList.size();
|
|||
|
|
+ this.currentIndex = 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean hasNext() {
|
|||
|
|
+ return this.currentIndex < this.totalCount;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CqUnit next() {
|
|||
|
|
+ if (!hasNext()) {
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+ final int currentIndex = this.currentIndex;
|
|||
|
|
+ final ByteBuffer byteBuffer = this.byteBufferList.get(currentIndex);
|
|||
|
|
+ CqUnit cqUnit = new CqUnit(this.startIndex + currentIndex, byteBuffer.getLong(), byteBuffer.getInt(), byteBuffer.getLong());
|
|||
|
|
+ this.currentIndex++;
|
|||
|
|
+ return cqUnit;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void remove() {
|
|||
|
|
+ throw new UnsupportedOperationException("remove");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void release() {
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public CqUnit nextAndRelease() {
|
|||
|
|
+ try {
|
|||
|
|
+ return next();
|
|||
|
|
+ } finally {
|
|||
|
|
+ release();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..6fa66282e
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueOffsetTable.java
|
|||
|
|
@@ -0,0 +1,641 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.nio.ByteBuffer;
|
|||
|
|
+import java.util.HashMap;
|
|||
|
|
+import java.util.HashSet;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+import java.util.Set;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
+import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.ConsumeQueue;
|
|||
|
|
+import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.rocksdb.ConsumeQueueRocksDBStorage;
|
|||
|
|
+import org.rocksdb.ColumnFamilyHandle;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+import org.rocksdb.RocksIterator;
|
|||
|
|
+import org.rocksdb.WriteBatch;
|
|||
|
|
+
|
|||
|
|
+import static org.apache.rocketmq.common.utils.DataConverter.CHARSET_UTF8;
|
|||
|
|
+import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore.CTRL_1;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBConsumeQueueOffsetTable {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ROCKSDB_LOG = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ private static final byte[] MAX_BYTES = "max".getBytes(CHARSET_UTF8);
|
|||
|
|
+ private static final byte[] MIN_BYTES = "min".getBytes(CHARSET_UTF8);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Rocksdb ConsumeQueue's Offset unit. Format:
|
|||
|
|
+ *
|
|||
|
|
+ * <pre>
|
|||
|
|
+ * ┌─────────────────────────┬───────────┬───────────────────────┬───────────┬───────────┬───────────┬─────────────┐
|
|||
|
|
+ * │ Topic Bytes Array Size │ CTRL_1 │ Topic Bytes Array │ CTRL_1 │ Max(Min) │ CTRL_1 │ QueueId │
|
|||
|
|
+ * │ (4 Bytes) │ (1 Bytes) │ (n Bytes) │ (1 Bytes) │ (3 Bytes) │ (1 Bytes) │ (4 Bytes) │
|
|||
|
|
+ * ├─────────────────────────┴───────────┴───────────────────────┴───────────┴───────────┴───────────┴─────────────┤
|
|||
|
|
+ * │ Key Unit │
|
|||
|
|
+ * │ │
|
|||
|
|
+ * </pre>
|
|||
|
|
+ *
|
|||
|
|
+ * <pre>
|
|||
|
|
+ * ┌─────────────────────────────┬────────────────────────┐
|
|||
|
|
+ * │ CommitLog Physical Offset │ ConsumeQueue Offset │
|
|||
|
|
+ * │ (8 Bytes) │ (8 Bytes) │
|
|||
|
|
+ * ├─────────────────────────────┴────────────────────────┤
|
|||
|
|
+ * │ Value Unit │
|
|||
|
|
+ * │ │
|
|||
|
|
+ * </pre>
|
|||
|
|
+ * ConsumeQueue's Offset unit. Size: CommitLog Physical Offset(8) + ConsumeQueue Offset(8) = 16 Bytes
|
|||
|
|
+ */
|
|||
|
|
+ private static final int OFFSET_PHY_OFFSET = 0;
|
|||
|
|
+ private static final int OFFSET_CQ_OFFSET = 8;
|
|||
|
|
+ /**
|
|||
|
|
+ *
|
|||
|
|
+ * ┌─────────────────────────┬───────────┬───────────┬───────────┬───────────┬─────────────┐
|
|||
|
|
+ * │ Topic Bytes Array Size │ CTRL_1 │ CTRL_1 │ Max(Min) │ CTRL_1 │ QueueId │
|
|||
|
|
+ * │ (4 Bytes) │ (1 Bytes) │ (1 Bytes) │ (3 Bytes) │ (1 Bytes) │ (4 Bytes) │
|
|||
|
|
+ * ├─────────────────────────┴───────────┴───────────┴───────────┴───────────┴─────────────┤
|
|||
|
|
+ */
|
|||
|
|
+ private static final int OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES = 4 + 1 + 1 + 3 + 1 + 4;
|
|||
|
|
+ private static final int OFFSET_VALUE_LENGTH = 8 + 8;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * We use a new system topic='CHECKPOINT_TOPIC' to record the maxPhyOffset built by CQ dispatch thread.
|
|||
|
|
+ * @see ConsumeQueueStore#getMaxPhyOffsetInConsumeQueue(), we use it to find the maxPhyOffset built by CQ dispatch thread.
|
|||
|
|
+ * If we do not record the maxPhyOffset, it may take us a long time to start traversing from the head of
|
|||
|
|
+ * RocksDBConsumeQueueOffsetTable to find it.
|
|||
|
|
+ */
|
|||
|
|
+ private static final String MAX_PHYSICAL_OFFSET_CHECKPOINT = TopicValidator.RMQ_SYS_ROCKSDB_OFFSET_TOPIC;
|
|||
|
|
+ private static final byte[] MAX_PHYSICAL_OFFSET_CHECKPOINT_BYTES = MAX_PHYSICAL_OFFSET_CHECKPOINT.getBytes(CHARSET_UTF8);
|
|||
|
|
+ private static final int INNER_CHECKPOINT_TOPIC_LEN = OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + MAX_PHYSICAL_OFFSET_CHECKPOINT_BYTES.length;
|
|||
|
|
+ private static final ByteBuffer INNER_CHECKPOINT_TOPIC = ByteBuffer.allocateDirect(INNER_CHECKPOINT_TOPIC_LEN);
|
|||
|
|
+ private static final byte[] MAX_PHYSICAL_OFFSET_CHECKPOINT_KEY = new byte[INNER_CHECKPOINT_TOPIC_LEN];
|
|||
|
|
+ private final ByteBuffer maxPhyOffsetBB;
|
|||
|
|
+ static {
|
|||
|
|
+ buildOffsetKeyByteBuffer0(INNER_CHECKPOINT_TOPIC, MAX_PHYSICAL_OFFSET_CHECKPOINT_BYTES, 0, true);
|
|||
|
|
+ INNER_CHECKPOINT_TOPIC.position(0).limit(INNER_CHECKPOINT_TOPIC_LEN);
|
|||
|
|
+ INNER_CHECKPOINT_TOPIC.get(MAX_PHYSICAL_OFFSET_CHECKPOINT_KEY);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private final RocksDBConsumeQueueTable rocksDBConsumeQueueTable;
|
|||
|
|
+ private final ConsumeQueueRocksDBStorage rocksDBStorage;
|
|||
|
|
+ private final DefaultMessageStore messageStore;
|
|||
|
|
+
|
|||
|
|
+ private ColumnFamilyHandle offsetCFH;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Although we have already put max(min) consumeQueueOffset and physicalOffset in rocksdb, we still hope to get them
|
|||
|
|
+ * from heap to avoid accessing rocksdb.
|
|||
|
|
+ * @see ConsumeQueue#getMaxPhysicOffset(), maxPhysicOffset --> topicQueueMaxCqOffset
|
|||
|
|
+ * @see ConsumeQueue#getMinLogicOffset(), minLogicOffset --> topicQueueMinOffset
|
|||
|
|
+ */
|
|||
|
|
+ private final Map<String/* topic-queueId */, PhyAndCQOffset> topicQueueMinOffset;
|
|||
|
|
+ private final Map<String/* topic-queueId */, Long> topicQueueMaxCqOffset;
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueueOffsetTable(RocksDBConsumeQueueTable rocksDBConsumeQueueTable,
|
|||
|
|
+ ConsumeQueueRocksDBStorage rocksDBStorage, DefaultMessageStore messageStore) {
|
|||
|
|
+ this.rocksDBConsumeQueueTable = rocksDBConsumeQueueTable;
|
|||
|
|
+ this.rocksDBStorage = rocksDBStorage;
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ this.topicQueueMinOffset = new ConcurrentHashMap(1024);
|
|||
|
|
+ this.topicQueueMaxCqOffset = new ConcurrentHashMap(1024);
|
|||
|
|
+
|
|||
|
|
+ this.maxPhyOffsetBB = ByteBuffer.allocateDirect(8);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void load() {
|
|||
|
|
+ this.offsetCFH = this.rocksDBStorage.getOffsetCFHandle();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void updateTempTopicQueueMaxOffset(final Pair<ByteBuffer, ByteBuffer> offsetBBPair,
|
|||
|
|
+ final byte[] topicBytes, final DispatchRequest request,
|
|||
|
|
+ final Map<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> tempTopicQueueMaxOffsetMap) {
|
|||
|
|
+ buildOffsetKeyAndValueByteBuffer(offsetBBPair, topicBytes, request);
|
|||
|
|
+ ByteBuffer topicQueueId = offsetBBPair.getObject1();
|
|||
|
|
+ ByteBuffer maxOffsetBB = offsetBBPair.getObject2();
|
|||
|
|
+ Pair<ByteBuffer, DispatchRequest> old = tempTopicQueueMaxOffsetMap.get(topicQueueId);
|
|||
|
|
+ if (old == null) {
|
|||
|
|
+ tempTopicQueueMaxOffsetMap.put(topicQueueId, new Pair(maxOffsetBB, request));
|
|||
|
|
+ } else {
|
|||
|
|
+ long oldMaxOffset = old.getObject1().getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+ long maxOffset = maxOffsetBB.getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+ if (maxOffset >= oldMaxOffset) {
|
|||
|
|
+ ERROR_LOG.error("cqOffset invalid1. old: {}, now: {}", oldMaxOffset, maxOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void putMaxPhyAndCqOffset(final Map<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> tempTopicQueueMaxOffsetMap,
|
|||
|
|
+ final WriteBatch writeBatch, final long maxPhyOffset) throws RocksDBException {
|
|||
|
|
+ for (Map.Entry<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> entry : tempTopicQueueMaxOffsetMap.entrySet()) {
|
|||
|
|
+ writeBatch.put(this.offsetCFH, entry.getKey(), entry.getValue().getObject1());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ appendMaxPhyOffset(writeBatch, maxPhyOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void putHeapMaxCqOffset(final Map<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> tempTopicQueueMaxOffsetMap) {
|
|||
|
|
+ for (Map.Entry<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> entry : tempTopicQueueMaxOffsetMap.entrySet()) {
|
|||
|
|
+ DispatchRequest request = entry.getValue().getObject2();
|
|||
|
|
+ putHeapMaxCqOffset(request.getTopic(), request.getQueueId(), request.getConsumeQueueOffset());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * When topic is deleted, we clean up its offset info in rocksdb.
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @throws RocksDBException
|
|||
|
|
+ */
|
|||
|
|
+ public void destroyOffset(String topic, int queueId, WriteBatch writeBatch) throws RocksDBException {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final ByteBuffer minOffsetKey = buildOffsetKeyByteBuffer(topicBytes, queueId, false);
|
|||
|
|
+ byte[] minOffsetBytes = this.rocksDBStorage.getOffset(minOffsetKey.array());
|
|||
|
|
+ Long startCQOffset = (minOffsetBytes != null) ? ByteBuffer.wrap(minOffsetBytes).getLong(OFFSET_CQ_OFFSET) : null;
|
|||
|
|
+
|
|||
|
|
+ final ByteBuffer maxOffsetKey = buildOffsetKeyByteBuffer(topicBytes, queueId, true);
|
|||
|
|
+ byte[] maxOffsetBytes = this.rocksDBStorage.getOffset(maxOffsetKey.array());
|
|||
|
|
+ Long endCQOffset = (maxOffsetBytes != null) ? ByteBuffer.wrap(maxOffsetBytes).getLong(OFFSET_CQ_OFFSET) : null;
|
|||
|
|
+
|
|||
|
|
+ writeBatch.delete(this.offsetCFH, minOffsetKey.array());
|
|||
|
|
+ writeBatch.delete(this.offsetCFH, maxOffsetKey.array());
|
|||
|
|
+
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ removeHeapMinCqOffset(topicQueueId);
|
|||
|
|
+ removeHeapMaxCqOffset(topicQueueId);
|
|||
|
|
+
|
|||
|
|
+ log.info("RocksDB offset table delete topic: {}, queueId: {}, minOffset: {}, maxOffset: {}", topic, queueId,
|
|||
|
|
+ startCQOffset, endCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void appendMaxPhyOffset(final WriteBatch writeBatch, final long maxPhyOffset) throws RocksDBException {
|
|||
|
|
+ final ByteBuffer maxPhyOffsetBB = this.maxPhyOffsetBB;
|
|||
|
|
+ maxPhyOffsetBB.position(0).limit(8);
|
|||
|
|
+ maxPhyOffsetBB.putLong(maxPhyOffset);
|
|||
|
|
+ maxPhyOffsetBB.flip();
|
|||
|
|
+
|
|||
|
|
+ INNER_CHECKPOINT_TOPIC.position(0).limit(INNER_CHECKPOINT_TOPIC_LEN);
|
|||
|
|
+ writeBatch.put(this.offsetCFH, INNER_CHECKPOINT_TOPIC, maxPhyOffsetBB);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getMaxPhyOffset() throws RocksDBException {
|
|||
|
|
+ byte[] valueBytes = this.rocksDBStorage.getOffset(MAX_PHYSICAL_OFFSET_CHECKPOINT_KEY);
|
|||
|
|
+ if (valueBytes == null) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+ ByteBuffer valueBB = ByteBuffer.wrap(valueBytes);
|
|||
|
|
+ return valueBB.getLong(0);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Traverse the offset table to find dirty topic
|
|||
|
|
+ * @param existTopicSet
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ public Map<String, Set<Integer>> iterateOffsetTable2FindDirty(final Set<String> existTopicSet) {
|
|||
|
|
+ Map<String/* topic */, Set<Integer/* queueId */>> topicQueueIdToBeDeletedMap = new HashMap<>();
|
|||
|
|
+
|
|||
|
|
+ RocksIterator iterator = null;
|
|||
|
|
+ try {
|
|||
|
|
+ iterator = rocksDBStorage.seekOffsetCF();
|
|||
|
|
+ if (iterator == null) {
|
|||
|
|
+ return topicQueueIdToBeDeletedMap;
|
|||
|
|
+ }
|
|||
|
|
+ for (iterator.seekToFirst(); iterator.isValid(); iterator.next()) {
|
|||
|
|
+ byte[] key = iterator.key();
|
|||
|
|
+ byte[] value = iterator.value();
|
|||
|
|
+ if (key == null || key.length <= OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES
|
|||
|
|
+ || value == null || value.length != OFFSET_VALUE_LENGTH) {
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+ ByteBuffer keyBB = ByteBuffer.wrap(key);
|
|||
|
|
+ int topicLen = keyBB.getInt(0);
|
|||
|
|
+ byte[] topicBytes = new byte[topicLen];
|
|||
|
|
+ /**
|
|||
|
|
+ * "Topic Bytes Array Size" + "CTRL_1" = 4 + 1
|
|||
|
|
+ */
|
|||
|
|
+ keyBB.position(4 + 1);
|
|||
|
|
+ keyBB.get(topicBytes);
|
|||
|
|
+ String topic = new String(topicBytes, CHARSET_UTF8);
|
|||
|
|
+ if (TopicValidator.isSystemTopic(topic)) {
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * "Topic Bytes Array Size" + "CTRL_1" + "Topic Bytes Array" + "CTRL_1" + "Max(min)" + "CTRL_1"
|
|||
|
|
+ * = 4 + 1 + topicLen + 1 + 3 + 1
|
|||
|
|
+ */
|
|||
|
|
+ int queueId = keyBB.getInt(4 + 1 + topicLen + 1 + 3 + 1);
|
|||
|
|
+
|
|||
|
|
+ if (!existTopicSet.contains(topic)) {
|
|||
|
|
+ ByteBuffer valueBB = ByteBuffer.wrap(value);
|
|||
|
|
+ long cqOffset = valueBB.getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+
|
|||
|
|
+ Set<Integer> topicQueueIdSet = topicQueueIdToBeDeletedMap.get(topic);
|
|||
|
|
+ if (topicQueueIdSet == null) {
|
|||
|
|
+ Set<Integer> newSet = new HashSet<>();
|
|||
|
|
+ newSet.add(queueId);
|
|||
|
|
+ topicQueueIdToBeDeletedMap.put(topic, newSet);
|
|||
|
|
+ } else {
|
|||
|
|
+ topicQueueIdSet.add(queueId);
|
|||
|
|
+ }
|
|||
|
|
+ ERROR_LOG.info("RocksDBConsumeQueueOffsetTable has dirty cqOffset. topic: {}, queueId: {}, cqOffset: {}",
|
|||
|
|
+ topic, queueId, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ ERROR_LOG.error("iterateOffsetTable2MarkDirtyCQ Failed.", e);
|
|||
|
|
+ } finally {
|
|||
|
|
+ if (iterator != null) {
|
|||
|
|
+ iterator.close();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return topicQueueIdToBeDeletedMap;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public Long getMaxCqOffset(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ Long maxCqOffset = getHeapMaxCqOffset(topic, queueId);
|
|||
|
|
+
|
|||
|
|
+ if (maxCqOffset == null) {
|
|||
|
|
+ final ByteBuffer byteBuffer = getMaxPhyAndCqOffsetInKV(topic, queueId);
|
|||
|
|
+ maxCqOffset = (byteBuffer != null) ? byteBuffer.getLong(OFFSET_CQ_OFFSET) : null;
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ this.topicQueueMaxCqOffset.putIfAbsent(topicQueueId, maxCqOffset != null ? maxCqOffset : -1L);
|
|||
|
|
+ if (messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("updateMaxOffsetInQueue. {}, {}", topicQueueId, maxCqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ return maxCqOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * truncate dirty offset in rocksdb
|
|||
|
|
+ * @param offsetToTruncate
|
|||
|
|
+ * @throws RocksDBException
|
|||
|
|
+ */
|
|||
|
|
+ public void truncateDirty(long offsetToTruncate) throws RocksDBException {
|
|||
|
|
+ correctMaxPyhOffset(offsetToTruncate);
|
|||
|
|
+
|
|||
|
|
+ ConcurrentMap<String, TopicConfig> allTopicConfigMap = this.messageStore.getTopicConfigs();
|
|||
|
|
+ if (allTopicConfigMap == null) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ for (TopicConfig topicConfig : allTopicConfigMap.values()) {
|
|||
|
|
+ for (int i = 0; i < topicConfig.getWriteQueueNums(); i++) {
|
|||
|
|
+ truncateDirtyOffset(topicConfig.getTopicName(), i);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private Pair<Boolean, Long> isMinOffsetOk(final String topic, final int queueId, final long minPhyOffset) throws RocksDBException {
|
|||
|
|
+ PhyAndCQOffset phyAndCQOffset = getHeapMinOffset(topic, queueId);
|
|||
|
|
+ if (phyAndCQOffset != null) {
|
|||
|
|
+ final long phyOffset = phyAndCQOffset.getPhyOffset();
|
|||
|
|
+ final long cqOffset = phyAndCQOffset.getCqOffset();
|
|||
|
|
+
|
|||
|
|
+ return (phyOffset >= minPhyOffset) ? new Pair(true, cqOffset) : new Pair(false, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ ByteBuffer byteBuffer = getMinPhyAndCqOffsetInKV(topic, queueId);
|
|||
|
|
+ if (byteBuffer == null) {
|
|||
|
|
+ return new Pair(false, 0L);
|
|||
|
|
+ }
|
|||
|
|
+ final long phyOffset = byteBuffer.getLong(OFFSET_PHY_OFFSET);
|
|||
|
|
+ final long cqOffset = byteBuffer.getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+ if (phyOffset >= minPhyOffset) {
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ PhyAndCQOffset newPhyAndCQOffset = new PhyAndCQOffset(phyOffset, cqOffset);
|
|||
|
|
+ this.topicQueueMinOffset.putIfAbsent(topicQueueId, newPhyAndCQOffset);
|
|||
|
|
+ if (messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("updateMinOffsetInQueue. {}, {}", topicQueueId, newPhyAndCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+ return new Pair(true, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ return new Pair(false, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void truncateDirtyOffset(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ final ByteBuffer byteBuffer = getMaxPhyAndCqOffsetInKV(topic, queueId);
|
|||
|
|
+ if (byteBuffer == null) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ long maxPhyOffset = byteBuffer.getLong(OFFSET_PHY_OFFSET);
|
|||
|
|
+ long maxCqOffset = byteBuffer.getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+ long maxPhyOffsetInCQ = getMaxPhyOffset();
|
|||
|
|
+
|
|||
|
|
+ if (maxPhyOffset >= maxPhyOffsetInCQ) {
|
|||
|
|
+ correctMaxCqOffset(topic, queueId, maxCqOffset, maxPhyOffsetInCQ);
|
|||
|
|
+ Long newMaxCqOffset = getHeapMaxCqOffset(topic, queueId);
|
|||
|
|
+ ROCKSDB_LOG.warn("truncateDirtyLogicFile topic: {}, queueId: {} from {} to {}", topic, queueId,
|
|||
|
|
+ maxPhyOffset, newMaxCqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void correctMaxPyhOffset(long maxPhyOffset) throws RocksDBException {
|
|||
|
|
+ if (!this.rocksDBStorage.hold()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ try {
|
|||
|
|
+ WriteBatch writeBatch = new WriteBatch();
|
|||
|
|
+ long oldMaxPhyOffset = getMaxPhyOffset();
|
|||
|
|
+ if (oldMaxPhyOffset <= maxPhyOffset) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ log.info("correctMaxPyhOffset, oldMaxPhyOffset={}, newMaxPhyOffset={}", oldMaxPhyOffset, maxPhyOffset);
|
|||
|
|
+ appendMaxPhyOffset(writeBatch, maxPhyOffset);
|
|||
|
|
+ this.rocksDBStorage.batchPut(writeBatch);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("correctMaxPyhOffset Failed.", e);
|
|||
|
|
+ throw e;
|
|||
|
|
+ } finally {
|
|||
|
|
+ this.rocksDBStorage.release();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getMinCqOffset(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ final long minPhyOffset = this.messageStore.getMinPhyOffset();
|
|||
|
|
+ Pair<Boolean, Long> pair = isMinOffsetOk(topic, queueId, minPhyOffset);
|
|||
|
|
+ final long cqOffset = pair.getObject2();
|
|||
|
|
+ if (!pair.getObject1() && correctMinCqOffset(topic, queueId, cqOffset, minPhyOffset)) {
|
|||
|
|
+ PhyAndCQOffset phyAndCQOffset = getHeapMinOffset(topic, queueId);
|
|||
|
|
+ if (phyAndCQOffset != null) {
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("getMinOffsetInQueue miss heap. topic: {}, queueId: {}, old: {}, new: {}",
|
|||
|
|
+ topic, queueId, cqOffset, phyAndCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+ return phyAndCQOffset.getCqOffset();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return cqOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public Long getMaxPhyOffset(String topic, int queueId) {
|
|||
|
|
+ try {
|
|||
|
|
+ ByteBuffer byteBuffer = getMaxPhyAndCqOffsetInKV(topic, queueId);
|
|||
|
|
+ if (byteBuffer != null) {
|
|||
|
|
+ return byteBuffer.getLong(OFFSET_PHY_OFFSET);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ ERROR_LOG.info("getMaxPhyOffset error. topic: {}, queueId: {}", topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+ return null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer getMinPhyAndCqOffsetInKV(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ return getPhyAndCqOffsetInKV(topic, queueId, false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer getMaxPhyAndCqOffsetInKV(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ return getPhyAndCqOffsetInKV(topic, queueId, true);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer getPhyAndCqOffsetInKV(String topic, int queueId, boolean max) throws RocksDBException {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final ByteBuffer keyBB = buildOffsetKeyByteBuffer(topicBytes, queueId, max);
|
|||
|
|
+
|
|||
|
|
+ byte[] value = this.rocksDBStorage.getOffset(keyBB.array());
|
|||
|
|
+ return (value != null) ? ByteBuffer.wrap(value) : null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private String buildTopicQueueId(final String topic, final int queueId) {
|
|||
|
|
+ return topic + "-" + queueId;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void putHeapMinCqOffset(final String topic, final int queueId, final long minPhyOffset, final long minCQOffset) {
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ PhyAndCQOffset phyAndCQOffset = new PhyAndCQOffset(minPhyOffset, minCQOffset);
|
|||
|
|
+ this.topicQueueMinOffset.put(topicQueueId, phyAndCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void putHeapMaxCqOffset(final String topic, final int queueId, final long maxCQOffset) {
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ Long oldMaxCqOffset = this.topicQueueMaxCqOffset.put(topicQueueId, maxCQOffset);
|
|||
|
|
+ if (oldMaxCqOffset != null && oldMaxCqOffset > maxCQOffset) {
|
|||
|
|
+ ERROR_LOG.error("cqOffset invalid0. old: {}, now: {}", oldMaxCqOffset, maxCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private PhyAndCQOffset getHeapMinOffset(final String topic, final int queueId) {
|
|||
|
|
+ return this.topicQueueMinOffset.get(buildTopicQueueId(topic, queueId));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private Long getHeapMaxCqOffset(final String topic, final int queueId) {
|
|||
|
|
+ String topicQueueId = buildTopicQueueId(topic, queueId);
|
|||
|
|
+ return this.topicQueueMaxCqOffset.get(topicQueueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private PhyAndCQOffset removeHeapMinCqOffset(String topicQueueId) {
|
|||
|
|
+ return this.topicQueueMinOffset.remove(topicQueueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private Long removeHeapMaxCqOffset(String topicQueueId) {
|
|||
|
|
+ return this.topicQueueMaxCqOffset.remove(topicQueueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void updateCqOffset(final String topic, final int queueId, final long phyOffset,
|
|||
|
|
+ final long cqOffset, boolean max) throws RocksDBException {
|
|||
|
|
+ if (!this.rocksDBStorage.hold()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ WriteBatch writeBatch = new WriteBatch();
|
|||
|
|
+ try {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final ByteBuffer offsetKey = buildOffsetKeyByteBuffer(topicBytes, queueId, max);
|
|||
|
|
+
|
|||
|
|
+ final ByteBuffer offsetValue = buildOffsetValueByteBuffer(phyOffset, cqOffset);
|
|||
|
|
+ writeBatch.put(this.offsetCFH, offsetKey.array(), offsetValue.array());
|
|||
|
|
+ this.rocksDBStorage.batchPut(writeBatch);
|
|||
|
|
+
|
|||
|
|
+ if (max) {
|
|||
|
|
+ putHeapMaxCqOffset(topic, queueId, cqOffset);
|
|||
|
|
+ } else {
|
|||
|
|
+ putHeapMinCqOffset(topic, queueId, phyOffset, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("updateCqOffset({}) failed.", max ? "max" : "min", e);
|
|||
|
|
+ throw e;
|
|||
|
|
+ } finally {
|
|||
|
|
+ writeBatch.close();
|
|||
|
|
+ this.rocksDBStorage.release();
|
|||
|
|
+ if (messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("updateCqOffset({}). topic: {}, queueId: {}, phyOffset: {}, cqOffset: {}",
|
|||
|
|
+ max ? "max" : "min", topic, queueId, phyOffset, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean correctMaxCqOffset(final String topic, final int queueId, final long maxCQOffset,
|
|||
|
|
+ final long maxPhyOffsetInCQ) throws RocksDBException {
|
|||
|
|
+ // 'getMinOffsetInQueue' may correct minCqOffset and put it into heap
|
|||
|
|
+ long minCQOffset = getMinCqOffset(topic, queueId);
|
|||
|
|
+ PhyAndCQOffset minPhyAndCQOffset = getHeapMinOffset(topic, queueId);
|
|||
|
|
+ if (minPhyAndCQOffset == null
|
|||
|
|
+ || minPhyAndCQOffset.getCqOffset() != minCQOffset
|
|||
|
|
+ || minPhyAndCQOffset.getPhyOffset() > maxPhyOffsetInCQ) {
|
|||
|
|
+ ROCKSDB_LOG.info("[BUG] correctMaxCqOffset error! topic: {}, queueId: {}, maxPhyOffsetInCQ: {}, "
|
|||
|
|
+ + "minCqOffset: {}, phyAndCQOffset: {}",
|
|||
|
|
+ topic, queueId, maxPhyOffsetInCQ, minCQOffset, minPhyAndCQOffset);
|
|||
|
|
+ throw new RocksDBException("correctMaxCqOffset error");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ long high = maxCQOffset;
|
|||
|
|
+ long low = minCQOffset;
|
|||
|
|
+ PhyAndCQOffset targetPhyAndCQOffset = this.rocksDBConsumeQueueTable.binarySearchInCQ(topic, queueId, high,
|
|||
|
|
+ low, maxPhyOffsetInCQ, false);
|
|||
|
|
+
|
|||
|
|
+ long targetCQOffset = targetPhyAndCQOffset.getCqOffset();
|
|||
|
|
+ long targetPhyOffset = targetPhyAndCQOffset.getPhyOffset();
|
|||
|
|
+
|
|||
|
|
+ if (targetCQOffset == -1) {
|
|||
|
|
+ if (maxCQOffset != minCQOffset) {
|
|||
|
|
+ updateCqOffset(topic, queueId, minPhyAndCQOffset.getPhyOffset(), minCQOffset, true);
|
|||
|
|
+ }
|
|||
|
|
+ if (messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("correct error. {}, {}, {}, {}, {}", topic, queueId, minCQOffset, maxCQOffset, minPhyAndCQOffset.getPhyOffset());
|
|||
|
|
+ }
|
|||
|
|
+ return false;
|
|||
|
|
+ } else {
|
|||
|
|
+ updateCqOffset(topic, queueId, targetPhyOffset, targetCQOffset, true);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean correctMinCqOffset(final String topic, final int queueId,
|
|||
|
|
+ final long minCQOffset, final long minPhyOffset) throws RocksDBException {
|
|||
|
|
+ final ByteBuffer maxBB = getMaxPhyAndCqOffsetInKV(topic, queueId);
|
|||
|
|
+ if (maxBB == null) {
|
|||
|
|
+ updateCqOffset(topic, queueId, minPhyOffset, 0L, false);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+ final long maxPhyOffset = maxBB.getLong(OFFSET_PHY_OFFSET);
|
|||
|
|
+ final long maxCQOffset = maxBB.getLong(OFFSET_CQ_OFFSET);
|
|||
|
|
+
|
|||
|
|
+ if (maxPhyOffset < minPhyOffset) {
|
|||
|
|
+ updateCqOffset(topic, queueId, minPhyOffset, maxCQOffset + 1, false);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ long high = maxCQOffset;
|
|||
|
|
+ long low = minCQOffset;
|
|||
|
|
+ PhyAndCQOffset phyAndCQOffset = this.rocksDBConsumeQueueTable.binarySearchInCQ(topic, queueId, high, low,
|
|||
|
|
+ minPhyOffset, true);
|
|||
|
|
+ long targetCQOffset = phyAndCQOffset.getCqOffset();
|
|||
|
|
+ long targetPhyOffset = phyAndCQOffset.getPhyOffset();
|
|||
|
|
+
|
|||
|
|
+ if (targetCQOffset == -1) {
|
|||
|
|
+ if (maxCQOffset != minCQOffset) {
|
|||
|
|
+ updateCqOffset(topic, queueId, maxPhyOffset, maxCQOffset, false);
|
|||
|
|
+ }
|
|||
|
|
+ if (messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("correct error. {}, {}, {}, {}, {}", topic, queueId, minCQOffset, maxCQOffset, minPhyOffset);
|
|||
|
|
+ }
|
|||
|
|
+ return false;
|
|||
|
|
+ } else {
|
|||
|
|
+ updateCqOffset(topic, queueId, targetPhyOffset, targetCQOffset, false);
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static Pair<ByteBuffer, ByteBuffer> getOffsetByteBufferPair() {
|
|||
|
|
+ ByteBuffer offsetKey = ByteBuffer.allocateDirect(RocksDBConsumeQueueStore.MAX_KEY_LEN);
|
|||
|
|
+ ByteBuffer offsetValue = ByteBuffer.allocateDirect(OFFSET_VALUE_LENGTH);
|
|||
|
|
+ return new Pair<>(offsetKey, offsetValue);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildOffsetKeyAndValueByteBuffer(final Pair<ByteBuffer, ByteBuffer> offsetBBPair,
|
|||
|
|
+ final byte[] topicBytes, final DispatchRequest request) {
|
|||
|
|
+ final ByteBuffer offsetKey = offsetBBPair.getObject1();
|
|||
|
|
+ buildOffsetKeyByteBuffer(offsetKey, topicBytes, request.getQueueId(), true);
|
|||
|
|
+
|
|||
|
|
+ final ByteBuffer offsetValue = offsetBBPair.getObject2();
|
|||
|
|
+ buildOffsetValueByteBuffer(offsetValue, request.getCommitLogOffset(), request.getConsumeQueueOffset());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer buildOffsetKeyByteBuffer(final byte[] topicBytes, final int queueId, final boolean max) {
|
|||
|
|
+ ByteBuffer byteBuffer = ByteBuffer.allocate(OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicBytes.length);
|
|||
|
|
+ buildOffsetKeyByteBuffer0(byteBuffer, topicBytes, queueId, max);
|
|||
|
|
+ return byteBuffer;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildOffsetKeyByteBuffer(final ByteBuffer byteBuffer, final byte[] topicBytes, final int queueId, final boolean max) {
|
|||
|
|
+ byteBuffer.position(0).limit(OFFSET_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicBytes.length);
|
|||
|
|
+ buildOffsetKeyByteBuffer0(byteBuffer, topicBytes, queueId, max);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private static void buildOffsetKeyByteBuffer0(final ByteBuffer byteBuffer, final byte[] topicBytes, final int queueId,
|
|||
|
|
+ final boolean max) {
|
|||
|
|
+ byteBuffer.putInt(topicBytes.length).put(CTRL_1).put(topicBytes).put(CTRL_1);
|
|||
|
|
+ if (max) {
|
|||
|
|
+ byteBuffer.put(MAX_BYTES);
|
|||
|
|
+ } else {
|
|||
|
|
+ byteBuffer.put(MIN_BYTES);
|
|||
|
|
+ }
|
|||
|
|
+ byteBuffer.put(CTRL_1).putInt(queueId);
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildOffsetValueByteBuffer(final ByteBuffer byteBuffer, final long phyOffset, final long cqOffset) {
|
|||
|
|
+ byteBuffer.position(0).limit(OFFSET_VALUE_LENGTH);
|
|||
|
|
+ buildOffsetValueByteBuffer0(byteBuffer, phyOffset, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer buildOffsetValueByteBuffer(final long phyOffset, final long cqOffset) {
|
|||
|
|
+ final ByteBuffer byteBuffer = ByteBuffer.allocate(OFFSET_VALUE_LENGTH);
|
|||
|
|
+ buildOffsetValueByteBuffer0(byteBuffer, phyOffset, cqOffset);
|
|||
|
|
+ return byteBuffer;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildOffsetValueByteBuffer0(final ByteBuffer byteBuffer, final long phyOffset, final long cqOffset) {
|
|||
|
|
+ byteBuffer.putLong(phyOffset).putLong(cqOffset);
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ static class PhyAndCQOffset {
|
|||
|
|
+ private final long phyOffset;
|
|||
|
|
+ private final long cqOffset;
|
|||
|
|
+
|
|||
|
|
+ public PhyAndCQOffset(final long phyOffset, final long cqOffset) {
|
|||
|
|
+ this.phyOffset = phyOffset;
|
|||
|
|
+ this.cqOffset = cqOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getPhyOffset() {
|
|||
|
|
+ return this.phyOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long getCqOffset() {
|
|||
|
|
+ return this.cqOffset;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String toString() {
|
|||
|
|
+ return "[cqOffset=" + cqOffset + ", phyOffset=" + phyOffset + "]";
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..78456cfcd
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueStore.java
|
|||
|
|
@@ -0,0 +1,441 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.io.File;
|
|||
|
|
+import java.nio.ByteBuffer;
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.HashMap;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+import java.util.Set;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+import java.util.concurrent.Executors;
|
|||
|
|
+import java.util.concurrent.ScheduledExecutorService;
|
|||
|
|
+import java.util.concurrent.TimeUnit;
|
|||
|
|
+
|
|||
|
|
+import org.apache.commons.io.FileUtils;
|
|||
|
|
+import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
+import org.apache.rocketmq.common.ThreadFactoryImpl;
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
+import org.apache.rocketmq.store.config.StorePathConfigHelper;
|
|||
|
|
+import org.apache.rocketmq.store.rocksdb.ConsumeQueueRocksDBStorage;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+import org.rocksdb.WriteBatch;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBConsumeQueueStore extends AbstractConsumeQueueStore {
|
|||
|
|
+ private static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ROCKSDB_LOG = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ public static final byte CTRL_0 = '\u0000';
|
|||
|
|
+ public static final byte CTRL_1 = '\u0001';
|
|||
|
|
+ public static final byte CTRL_2 = '\u0002';
|
|||
|
|
+
|
|||
|
|
+ private static final int BATCH_SIZE = 16;
|
|||
|
|
+ public static final int MAX_KEY_LEN = 300;
|
|||
|
|
+
|
|||
|
|
+ private final ScheduledExecutorService scheduledExecutorService;
|
|||
|
|
+ private final String storePath;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * we use two tables with different ColumnFamilyHandle, called RocksDBConsumeQueueTable and RocksDBConsumeQueueOffsetTable.
|
|||
|
|
+ * 1.RocksDBConsumeQueueTable uses to store CqUnit[physicalOffset, msgSize, tagHashCode, msgStoreTime]
|
|||
|
|
+ * 2.RocksDBConsumeQueueOffsetTable uses to store physicalOffset and consumeQueueOffset(@see PhyAndCQOffset) of topic-queueId
|
|||
|
|
+ */
|
|||
|
|
+ private final ConsumeQueueRocksDBStorage rocksDBStorage;
|
|||
|
|
+ private final RocksDBConsumeQueueTable rocksDBConsumeQueueTable;
|
|||
|
|
+ private final RocksDBConsumeQueueOffsetTable rocksDBConsumeQueueOffsetTable;
|
|||
|
|
+
|
|||
|
|
+ private final WriteBatch writeBatch;
|
|||
|
|
+ private final List<DispatchRequest> bufferDRList;
|
|||
|
|
+ private final List<Pair<ByteBuffer, ByteBuffer>> cqBBPairList;
|
|||
|
|
+ private final List<Pair<ByteBuffer, ByteBuffer>> offsetBBPairList;
|
|||
|
|
+ private final Map<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> tempTopicQueueMaxOffsetMap;
|
|||
|
|
+ private volatile boolean isCQError = false;
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueueStore(DefaultMessageStore messageStore) {
|
|||
|
|
+ super(messageStore);
|
|||
|
|
+
|
|||
|
|
+ this.storePath = StorePathConfigHelper.getStorePathConsumeQueue(messageStoreConfig.getStorePathRootDir());
|
|||
|
|
+ this.rocksDBStorage = new ConsumeQueueRocksDBStorage(messageStore, storePath, 4);
|
|||
|
|
+ this.rocksDBConsumeQueueTable = new RocksDBConsumeQueueTable(rocksDBStorage, messageStore);
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable = new RocksDBConsumeQueueOffsetTable(rocksDBConsumeQueueTable, rocksDBStorage, messageStore);
|
|||
|
|
+
|
|||
|
|
+ this.writeBatch = new WriteBatch();
|
|||
|
|
+ this.bufferDRList = new ArrayList(BATCH_SIZE);
|
|||
|
|
+ this.cqBBPairList = new ArrayList(BATCH_SIZE);
|
|||
|
|
+ this.offsetBBPairList = new ArrayList(BATCH_SIZE);
|
|||
|
|
+ for (int i = 0; i < BATCH_SIZE; i++) {
|
|||
|
|
+ this.cqBBPairList.add(RocksDBConsumeQueueTable.getCQByteBufferPair());
|
|||
|
|
+ this.offsetBBPairList.add(RocksDBConsumeQueueOffsetTable.getOffsetByteBufferPair());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ this.tempTopicQueueMaxOffsetMap = new HashMap<>();
|
|||
|
|
+ this.scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(
|
|||
|
|
+ new ThreadFactoryImpl("RocksDBConsumeQueueStoreScheduledThread", messageStore.getBrokerIdentity()));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void start() {
|
|||
|
|
+ log.info("RocksDB ConsumeQueueStore start!");
|
|||
|
|
+ this.scheduledExecutorService.scheduleAtFixedRate(() -> {
|
|||
|
|
+ this.rocksDBStorage.statRocksdb(ROCKSDB_LOG);
|
|||
|
|
+ }, 10, this.messageStoreConfig.getStatRocksDBCQIntervalSec(), TimeUnit.SECONDS);
|
|||
|
|
+
|
|||
|
|
+ this.scheduledExecutorService.scheduleWithFixedDelay(() -> {
|
|||
|
|
+ cleanDirty(messageStore.getTopicConfigs().keySet());
|
|||
|
|
+ }, 10, this.messageStoreConfig.getCleanRocksDBDirtyCQIntervalMin(), TimeUnit.MINUTES);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void cleanDirty(final Set<String> existTopicSet) {
|
|||
|
|
+ try {
|
|||
|
|
+ Map<String, Set<Integer>> topicQueueIdToBeDeletedMap =
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.iterateOffsetTable2FindDirty(existTopicSet);
|
|||
|
|
+
|
|||
|
|
+ for (Map.Entry<String, Set<Integer>> entry : topicQueueIdToBeDeletedMap.entrySet()) {
|
|||
|
|
+ String topic = entry.getKey();
|
|||
|
|
+ for (int queueId : entry.getValue()) {
|
|||
|
|
+ destroy(new RocksDBConsumeQueue(topic, queueId));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ log.error("cleanUnusedTopic Failed.", e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean load() {
|
|||
|
|
+ boolean result = this.rocksDBStorage.start();
|
|||
|
|
+ this.rocksDBConsumeQueueTable.load();
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.load();
|
|||
|
|
+ log.info("load rocksdb consume queue {}.", result ? "OK" : "Failed");
|
|||
|
|
+ return result;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean loadAfterDestroy() {
|
|||
|
|
+ return this.load();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recover() {
|
|||
|
|
+ // ignored
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean recoverConcurrently() {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean shutdown() {
|
|||
|
|
+ this.scheduledExecutorService.shutdown();
|
|||
|
|
+ return shutdownInner();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean shutdownInner() {
|
|||
|
|
+ return this.rocksDBStorage.shutdown();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void putMessagePositionInfoWrapper(DispatchRequest request) throws RocksDBException {
|
|||
|
|
+ if (request == null || this.bufferDRList.size() >= BATCH_SIZE) {
|
|||
|
|
+ putMessagePosition();
|
|||
|
|
+ }
|
|||
|
|
+ if (request != null) {
|
|||
|
|
+ this.bufferDRList.add(request);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void putMessagePosition() throws RocksDBException {
|
|||
|
|
+ final int maxRetries = 30;
|
|||
|
|
+ for (int i = 0; i < maxRetries; i++) {
|
|||
|
|
+ if (putMessagePosition0()) {
|
|||
|
|
+ if (this.isCQError) {
|
|||
|
|
+ this.messageStore.getRunningFlags().clearLogicsQueueError();
|
|||
|
|
+ this.isCQError = false;
|
|||
|
|
+ }
|
|||
|
|
+ return;
|
|||
|
|
+ } else {
|
|||
|
|
+ ERROR_LOG.warn("{} put cq Failed. retryTime: {}", i);
|
|||
|
|
+ try {
|
|||
|
|
+ Thread.sleep(100);
|
|||
|
|
+ } catch (InterruptedException ignored) {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ if (!this.isCQError) {
|
|||
|
|
+ ERROR_LOG.error("[BUG] put CQ Failed.");
|
|||
|
|
+ this.messageStore.getRunningFlags().makeLogicsQueueError();
|
|||
|
|
+ this.isCQError = true;
|
|||
|
|
+ }
|
|||
|
|
+ throw new RocksDBException("put CQ Failed");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean putMessagePosition0() {
|
|||
|
|
+ if (!this.rocksDBStorage.hold()) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ final Map<ByteBuffer, Pair<ByteBuffer, DispatchRequest>> tempTopicQueueMaxOffsetMap = this.tempTopicQueueMaxOffsetMap;
|
|||
|
|
+ try {
|
|||
|
|
+ final List<DispatchRequest> bufferDRList = this.bufferDRList;
|
|||
|
|
+ final int size = bufferDRList.size();
|
|||
|
|
+ if (size == 0) {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+ final List<Pair<ByteBuffer, ByteBuffer>> cqBBPairList = this.cqBBPairList;
|
|||
|
|
+ final List<Pair<ByteBuffer, ByteBuffer>> offsetBBPairList = this.offsetBBPairList;
|
|||
|
|
+ final WriteBatch writeBatch = this.writeBatch;
|
|||
|
|
+
|
|||
|
|
+ long maxPhyOffset = 0;
|
|||
|
|
+ for (int i = size - 1; i >= 0; i--) {
|
|||
|
|
+ final DispatchRequest request = bufferDRList.get(i);
|
|||
|
|
+ final byte[] topicBytes = request.getTopic().getBytes(DataConverter.CHARSET_UTF8);
|
|||
|
|
+
|
|||
|
|
+ this.rocksDBConsumeQueueTable.buildAndPutCQByteBuffer(cqBBPairList.get(i), topicBytes, request, writeBatch);
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.updateTempTopicQueueMaxOffset(offsetBBPairList.get(i),
|
|||
|
|
+ topicBytes, request, tempTopicQueueMaxOffsetMap);
|
|||
|
|
+
|
|||
|
|
+ final int msgSize = request.getMsgSize();
|
|||
|
|
+ final long phyOffset = request.getCommitLogOffset();
|
|||
|
|
+ if (phyOffset + msgSize >= maxPhyOffset) {
|
|||
|
|
+ maxPhyOffset = phyOffset + msgSize;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.putMaxPhyAndCqOffset(tempTopicQueueMaxOffsetMap, writeBatch, maxPhyOffset);
|
|||
|
|
+
|
|||
|
|
+ // clear writeBatch in batchPut
|
|||
|
|
+ this.rocksDBStorage.batchPut(writeBatch);
|
|||
|
|
+
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.putHeapMaxCqOffset(tempTopicQueueMaxOffsetMap);
|
|||
|
|
+
|
|||
|
|
+ long storeTimeStamp = bufferDRList.get(size - 1).getStoreTimestamp();
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().getBrokerRole() == BrokerRole.SLAVE
|
|||
|
|
+ || this.messageStore.getMessageStoreConfig().isEnableDLegerCommitLog()) {
|
|||
|
|
+ this.messageStore.getStoreCheckpoint().setPhysicMsgTimestamp(storeTimeStamp);
|
|||
|
|
+ }
|
|||
|
|
+ this.messageStore.getStoreCheckpoint().setLogicsMsgTimestamp(storeTimeStamp);
|
|||
|
|
+
|
|||
|
|
+ notifyMessageArriveAndClear();
|
|||
|
|
+ return true;
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ ERROR_LOG.error("putMessagePosition0 Failed.", e);
|
|||
|
|
+ return false;
|
|||
|
|
+ } finally {
|
|||
|
|
+ tempTopicQueueMaxOffsetMap.clear();
|
|||
|
|
+ this.rocksDBStorage.release();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void notifyMessageArriveAndClear() {
|
|||
|
|
+ final List<DispatchRequest> bufferDRList = this.bufferDRList;
|
|||
|
|
+ try {
|
|||
|
|
+ for (DispatchRequest dp : bufferDRList) {
|
|||
|
|
+ this.messageStore.notifyMessageArriveIfNecessary(dp);
|
|||
|
|
+ }
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ ERROR_LOG.error("notifyMessageArriveAndClear Failed.", e);
|
|||
|
|
+ } finally {
|
|||
|
|
+ bufferDRList.clear();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public List<ByteBuffer> rangeQuery(final String topic, final int queueId, final long startIndex, final int num) throws RocksDBException {
|
|||
|
|
+ return this.rocksDBConsumeQueueTable.rangeQuery(topic, queueId, startIndex, num);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ByteBuffer get(final String topic, final int queueId, final long cqOffset) throws RocksDBException {
|
|||
|
|
+ return this.rocksDBConsumeQueueTable.getCQInKV(topic, queueId, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Ignored, we do not need to recover topicQueueTable and correct minLogicOffset. Because we will correct them
|
|||
|
|
+ * when we use them, we call it lazy correction.
|
|||
|
|
+ * @see RocksDBConsumeQueue#increaseQueueOffset(QueueOffsetOperator, MessageExtBrokerInner, short)
|
|||
|
|
+ * @see org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable#getMinCqOffset(String, int)
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void recoverOffsetTable(long minPhyOffset) {
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void destroy() {
|
|||
|
|
+ try {
|
|||
|
|
+ shutdownInner();
|
|||
|
|
+ FileUtils.deleteDirectory(new File(this.storePath));
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ ERROR_LOG.error("destroy cq Failed. {}", this.storePath, e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void destroy(ConsumeQueueInterface consumeQueue) throws RocksDBException {
|
|||
|
|
+ String topic = consumeQueue.getTopic();
|
|||
|
|
+ int queueId = consumeQueue.getQueueId();
|
|||
|
|
+ if (StringUtils.isEmpty(topic) || queueId < 0 || !this.rocksDBStorage.hold()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ WriteBatch writeBatch = new WriteBatch();
|
|||
|
|
+ try {
|
|||
|
|
+ this.rocksDBConsumeQueueTable.destroyCQ(topic, queueId, writeBatch);
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.destroyOffset(topic, queueId, writeBatch);
|
|||
|
|
+
|
|||
|
|
+ this.rocksDBStorage.batchPut(writeBatch);
|
|||
|
|
+ } catch (RocksDBException e) {
|
|||
|
|
+ ERROR_LOG.error("kv deleteTopic {} Failed.", topic, e);
|
|||
|
|
+ throw e;
|
|||
|
|
+ } finally {
|
|||
|
|
+ writeBatch.close();
|
|||
|
|
+ this.rocksDBStorage.release();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean flush(ConsumeQueueInterface consumeQueue, int flushLeastPages) {
|
|||
|
|
+ try {
|
|||
|
|
+ this.rocksDBStorage.flushWAL();
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ }
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void checkSelf() {
|
|||
|
|
+ // ignored
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public int deleteExpiredFile(ConsumeQueueInterface consumeQueue, long minCommitLogPos) {
|
|||
|
|
+ // ignored
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * We do not need to truncate dirty CQ in RocksDBConsumeQueueTable, Because dirty CQ in RocksDBConsumeQueueTable
|
|||
|
|
+ * will be rewritten by new KV when new messages are appended or will be cleaned up when topics are deleted.
|
|||
|
|
+ * But dirty offset info in RocksDBConsumeQueueOffsetTable must be truncated, because we use offset info in
|
|||
|
|
+ * RocksDBConsumeQueueOffsetTable to rebuild topicQueueTable(@see RocksDBConsumeQueue#increaseQueueOffset).
|
|||
|
|
+ * @param offsetToTruncate
|
|||
|
|
+ * @throws RocksDBException
|
|||
|
|
+ */
|
|||
|
|
+ @Override
|
|||
|
|
+ public void truncateDirty(long offsetToTruncate) throws RocksDBException {
|
|||
|
|
+ long maxPhyOffsetInRocksdb = getMaxPhyOffsetInConsumeQueue();
|
|||
|
|
+ if (offsetToTruncate >= maxPhyOffsetInRocksdb) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ this.rocksDBConsumeQueueOffsetTable.truncateDirty(offsetToTruncate);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void cleanExpired(final long minPhyOffset) {
|
|||
|
|
+ this.rocksDBStorage.manualCompaction(minPhyOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) throws RocksDBException {
|
|||
|
|
+ final long minPhysicOffset = this.messageStore.getMinPhyOffset();
|
|||
|
|
+ long low = this.rocksDBConsumeQueueOffsetTable.getMinCqOffset(topic, queueId);
|
|||
|
|
+ Long high = this.rocksDBConsumeQueueOffsetTable.getMaxCqOffset(topic, queueId);
|
|||
|
|
+ if (high == null || high == -1) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+ return this.rocksDBConsumeQueueTable.binarySearchInCQByTime(topic, queueId, high, low, timestamp, minPhysicOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxOffsetInQueue(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ Long maxOffset = this.rocksDBConsumeQueueOffsetTable.getMaxCqOffset(topic, queueId);
|
|||
|
|
+ return (maxOffset != null) ? maxOffset + 1 : 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMinOffsetInQueue(String topic, int queueId) throws RocksDBException {
|
|||
|
|
+ return this.rocksDBConsumeQueueOffsetTable.getMinCqOffset(topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Long getMaxPhyOffsetInConsumeQueue(String topic, int queueId) {
|
|||
|
|
+ return this.rocksDBConsumeQueueOffsetTable.getMaxPhyOffset(topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getMaxPhyOffsetInConsumeQueue() throws RocksDBException {
|
|||
|
|
+ return this.rocksDBConsumeQueueOffsetTable.getMaxPhyOffset();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public ConsumeQueueInterface findOrCreateConsumeQueue(String topic, int queueId) {
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> map = this.consumeQueueTable.get(topic);
|
|||
|
|
+ if (null == map) {
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> newMap = new ConcurrentHashMap<>(128);
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> oldMap = this.consumeQueueTable.putIfAbsent(topic, newMap);
|
|||
|
|
+ if (oldMap != null) {
|
|||
|
|
+ map = oldMap;
|
|||
|
|
+ } else {
|
|||
|
|
+ map = newMap;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface logic = map.get(queueId);
|
|||
|
|
+ if (logic != null) {
|
|||
|
|
+ return logic;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface newLogic = new RocksDBConsumeQueue(this.messageStore, topic, queueId);
|
|||
|
|
+ ConsumeQueueInterface oldLogic = map.putIfAbsent(queueId, newLogic);
|
|||
|
|
+
|
|||
|
|
+ return oldLogic != null ? oldLogic : newLogic;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long rollNextFile(ConsumeQueueInterface consumeQueue, long offset) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean isFirstFileExist(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public boolean isFirstFileAvailable(ConsumeQueueInterface consumeQueue) {
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public long getTotalSize() {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTable.java b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTable.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..0a735ea27
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/queue/RocksDBConsumeQueueTable.java
|
|||
|
|
@@ -0,0 +1,312 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.queue;
|
|||
|
|
+
|
|||
|
|
+import java.nio.ByteBuffer;
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.Pair;
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
+import org.apache.rocketmq.store.DispatchRequest;
|
|||
|
|
+import org.apache.rocketmq.store.queue.RocksDBConsumeQueueOffsetTable.PhyAndCQOffset;
|
|||
|
|
+import org.apache.rocketmq.store.rocksdb.ConsumeQueueRocksDBStorage;
|
|||
|
|
+import org.rocksdb.ColumnFamilyHandle;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+import org.rocksdb.WriteBatch;
|
|||
|
|
+
|
|||
|
|
+import static org.apache.rocketmq.common.utils.DataConverter.CHARSET_UTF8;
|
|||
|
|
+import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore.CTRL_0;
|
|||
|
|
+import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore.CTRL_1;
|
|||
|
|
+import static org.apache.rocketmq.store.queue.RocksDBConsumeQueueStore.CTRL_2;
|
|||
|
|
+
|
|||
|
|
+/**
|
|||
|
|
+ * We use RocksDBConsumeQueueTable to store cqUnit.
|
|||
|
|
+ */
|
|||
|
|
+public class RocksDBConsumeQueueTable {
|
|||
|
|
+ private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ROCKSDB_LOG = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME);
|
|||
|
|
+ private static final Logger ERROR_LOG = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME);
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Rocksdb ConsumeQueue's store unit. Format:
|
|||
|
|
+ *
|
|||
|
|
+ * <pre>
|
|||
|
|
+ * ┌─────────────────────────┬───────────┬───────────────────────┬───────────┬───────────┬───────────┬───────────────────────┐
|
|||
|
|
+ * │ Topic Bytes Array Size │ CTRL_1 │ Topic Bytes Array │ CTRL_1 │ QueueId │ CTRL_1 │ ConsumeQueue Offset │
|
|||
|
|
+ * │ (4 Bytes) │ (1 Bytes) │ (n Bytes) │ (1 Bytes) │ (4 Bytes) │ (1 Bytes) │ (8 Bytes) │
|
|||
|
|
+ * ├─────────────────────────┴───────────┴───────────────────────┴───────────┴───────────┴───────────┴───────────────────────┤
|
|||
|
|
+ * │ Key Unit │
|
|||
|
|
+ * │ │
|
|||
|
|
+ * </pre>
|
|||
|
|
+ *
|
|||
|
|
+ * <pre>
|
|||
|
|
+ * ┌─────────────────────────────┬───────────────────┬──────────────────┬──────────────────┐
|
|||
|
|
+ * │ CommitLog Physical Offset │ Body Size │ Tag HashCode │ Msg Store Time │
|
|||
|
|
+ * │ (8 Bytes) │ (4 Bytes) │ (8 Bytes) │ (8 Bytes) │
|
|||
|
|
+ * ├─────────────────────────────┴───────────────────┴──────────────────┴──────────────────┤
|
|||
|
|
+ * │ Value Unit │
|
|||
|
|
+ * │ │
|
|||
|
|
+ * </pre>
|
|||
|
|
+ * ConsumeQueue's store unit. Size:
|
|||
|
|
+ * CommitLog Physical Offset(8) + Body Size(4) + Tag HashCode(8) + Msg Store Time(8) = 28 Bytes
|
|||
|
|
+ */
|
|||
|
|
+ private static final int PHY_OFFSET_OFFSET = 0;
|
|||
|
|
+ private static final int PHY_MSG_LEN_OFFSET = 8;
|
|||
|
|
+ private static final int MSG_TAG_HASHCODE_OFFSET = 12;
|
|||
|
|
+ private static final int MSG_STORE_TIME_SIZE_OFFSET = 20;
|
|||
|
|
+ public static final int CQ_UNIT_SIZE = 8 + 4 + 8 + 8;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * ┌─────────────────────────┬───────────┬───────────┬───────────┬───────────┬───────────────────────┐
|
|||
|
|
+ * │ Topic Bytes Array Size │ CTRL_1 │ CTRL_1 │ QueueId │ CTRL_1 │ ConsumeQueue Offset │
|
|||
|
|
+ * │ (4 Bytes) │ (1 Bytes) │ (1 Bytes) │ (4 Bytes) │ (1 Bytes) │ (8 Bytes) │
|
|||
|
|
+ * ├─────────────────────────┴───────────┴───────────┴───────────┴───────────┴───────────────────────┤
|
|||
|
|
+ */
|
|||
|
|
+ private static final int CQ_KEY_LENGTH_WITHOUT_TOPIC_BYTES = 4 + 1 + 1 + 4 + 1 + 8;
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * ┌─────────────────────────┬───────────┬───────────┬───────────┬───────────────────┐
|
|||
|
|
+ * │ Topic Bytes Array Size │ CTRL_1 │ CTRL_1 │ QueueId │ CTRL_0(CTRL_2) │
|
|||
|
|
+ * │ (4 Bytes) │ (1 Bytes) │ (1 Bytes) │ (4 Bytes) │ (1 Bytes) │
|
|||
|
|
+ * ├─────────────────────────┴───────────┴───────────┴───────────┴───────────────────┤
|
|||
|
|
+ */
|
|||
|
|
+ private static final int DELETE_CQ_KEY_LENGTH_WITHOUT_TOPIC_BYTES = 4 + 1 + 1 + 4 + 1;
|
|||
|
|
+
|
|||
|
|
+ private final ConsumeQueueRocksDBStorage rocksDBStorage;
|
|||
|
|
+ private final DefaultMessageStore messageStore;
|
|||
|
|
+
|
|||
|
|
+ private ColumnFamilyHandle defaultCFH;
|
|||
|
|
+
|
|||
|
|
+ public RocksDBConsumeQueueTable(ConsumeQueueRocksDBStorage rocksDBStorage, DefaultMessageStore messageStore) {
|
|||
|
|
+ this.rocksDBStorage = rocksDBStorage;
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void load() {
|
|||
|
|
+ this.defaultCFH = this.rocksDBStorage.getDefaultCFHandle();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void buildAndPutCQByteBuffer(final Pair<ByteBuffer, ByteBuffer> cqBBPair,
|
|||
|
|
+ final byte[] topicBytes, final DispatchRequest request, final WriteBatch writeBatch) throws RocksDBException {
|
|||
|
|
+ final ByteBuffer cqKey = cqBBPair.getObject1();
|
|||
|
|
+ buildCQKeyByteBuffer(cqKey, topicBytes, request.getQueueId(), request.getConsumeQueueOffset());
|
|||
|
|
+
|
|||
|
|
+ final ByteBuffer cqValue = cqBBPair.getObject2();
|
|||
|
|
+ buildCQValueByteBuffer(cqValue, request.getCommitLogOffset(), request.getMsgSize(), request.getTagsCode(), request.getStoreTimestamp());
|
|||
|
|
+
|
|||
|
|
+ writeBatch.put(this.defaultCFH, cqKey, cqValue);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ByteBuffer getCQInKV(final String topic, final int queueId, final long cqOffset) throws RocksDBException {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final ByteBuffer keyBB = buildCQKeyByteBuffer(topicBytes, queueId, cqOffset);
|
|||
|
|
+ byte[] value = this.rocksDBStorage.getCQ(keyBB.array());
|
|||
|
|
+ return (value != null) ? ByteBuffer.wrap(value) : null;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public List<ByteBuffer> rangeQuery(final String topic, final int queueId, final long startIndex, final int num) throws RocksDBException {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final List<ColumnFamilyHandle> defaultCFHList = new ArrayList(num);
|
|||
|
|
+ final ByteBuffer[] resultList = new ByteBuffer[num];
|
|||
|
|
+ final List<Integer> kvIndexList = new ArrayList(num);
|
|||
|
|
+ final List<byte[]> kvKeyList = new ArrayList(num);
|
|||
|
|
+ for (int i = 0; i < num; i++) {
|
|||
|
|
+ final ByteBuffer keyBB = buildCQKeyByteBuffer(topicBytes, queueId, startIndex + i);
|
|||
|
|
+ kvIndexList.add(i);
|
|||
|
|
+ kvKeyList.add(keyBB.array());
|
|||
|
|
+ defaultCFHList.add(this.defaultCFH);
|
|||
|
|
+ }
|
|||
|
|
+ int keyNum = kvIndexList.size();
|
|||
|
|
+ if (keyNum > 0) {
|
|||
|
|
+ List<byte[]> kvValueList = this.rocksDBStorage.multiGet(defaultCFHList, kvKeyList);
|
|||
|
|
+ final int valueNum = kvValueList.size();
|
|||
|
|
+ if (keyNum != valueNum) {
|
|||
|
|
+ throw new RocksDBException("rocksdb bug, multiGet");
|
|||
|
|
+ }
|
|||
|
|
+ for (int i = 0; i < valueNum; i++) {
|
|||
|
|
+ byte[] value = kvValueList.get(i);
|
|||
|
|
+ if (value == null) {
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+ ByteBuffer byteBuffer = ByteBuffer.wrap(value);
|
|||
|
|
+ resultList[kvIndexList.get(i)] = byteBuffer;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ final int resultSize = resultList.length;
|
|||
|
|
+ List<ByteBuffer> bbValueList = new ArrayList(resultSize);
|
|||
|
|
+ for (int i = 0; i < resultSize; i++) {
|
|||
|
|
+ ByteBuffer byteBuffer = resultList[i];
|
|||
|
|
+ if (byteBuffer == null) {
|
|||
|
|
+ break;
|
|||
|
|
+ }
|
|||
|
|
+ bbValueList.add(byteBuffer);
|
|||
|
|
+ }
|
|||
|
|
+ return bbValueList;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * When topic is deleted, we clean up its CqUnit in rocksdb.
|
|||
|
|
+ * @param topic
|
|||
|
|
+ * @param queueId
|
|||
|
|
+ * @throws RocksDBException
|
|||
|
|
+ */
|
|||
|
|
+ public void destroyCQ(final String topic, final int queueId, WriteBatch writeBatch) throws RocksDBException {
|
|||
|
|
+ final byte[] topicBytes = topic.getBytes(CHARSET_UTF8);
|
|||
|
|
+ final ByteBuffer cqStartKey = buildDeleteCQKey(true, topicBytes, queueId);
|
|||
|
|
+ final ByteBuffer cqEndKey = buildDeleteCQKey(false, topicBytes, queueId);
|
|||
|
|
+
|
|||
|
|
+ writeBatch.deleteRange(this.defaultCFH, cqStartKey.array(), cqEndKey.array());
|
|||
|
|
+
|
|||
|
|
+ log.info("Rocksdb consumeQueue table delete topic. {}, {}", topic, queueId);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public long binarySearchInCQByTime(String topic, int queueId, long high, long low, long timestamp,
|
|||
|
|
+ long minPhysicOffset) throws RocksDBException {
|
|||
|
|
+ long result = 0;
|
|||
|
|
+ long targetOffset = -1L, leftOffset = -1L, rightOffset = -1L;
|
|||
|
|
+ long leftValue = -1L, rightValue = -1L;
|
|||
|
|
+ while (high >= low) {
|
|||
|
|
+ long midOffset = low + ((high - low) >>> 1);
|
|||
|
|
+ ByteBuffer byteBuffer = getCQInKV(topic, queueId, midOffset);
|
|||
|
|
+ if (byteBuffer == null) {
|
|||
|
|
+ ERROR_LOG.warn("binarySearchInCQByTimeStamp Failed. topic: {}, queueId: {}, timestamp: {}, result: null",
|
|||
|
|
+ topic, queueId, timestamp);
|
|||
|
|
+ low = midOffset + 1;
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ long phyOffset = byteBuffer.getLong(PHY_OFFSET_OFFSET);
|
|||
|
|
+ if (phyOffset < minPhysicOffset) {
|
|||
|
|
+ low = midOffset + 1;
|
|||
|
|
+ leftOffset = midOffset;
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+ long storeTime = byteBuffer.getLong(MSG_STORE_TIME_SIZE_OFFSET);
|
|||
|
|
+ if (storeTime < 0) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ } else if (storeTime == timestamp) {
|
|||
|
|
+ targetOffset = midOffset;
|
|||
|
|
+ break;
|
|||
|
|
+ } else if (storeTime > timestamp) {
|
|||
|
|
+ high = midOffset - 1;
|
|||
|
|
+ rightOffset = midOffset;
|
|||
|
|
+ rightValue = storeTime;
|
|||
|
|
+ } else {
|
|||
|
|
+ low = midOffset + 1;
|
|||
|
|
+ leftOffset = midOffset;
|
|||
|
|
+ leftValue = storeTime;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ if (targetOffset != -1) {
|
|||
|
|
+ result = targetOffset;
|
|||
|
|
+ } else {
|
|||
|
|
+ if (leftValue == -1) {
|
|||
|
|
+ result = rightOffset;
|
|||
|
|
+ } else if (rightValue == -1) {
|
|||
|
|
+ result = leftOffset;
|
|||
|
|
+ } else {
|
|||
|
|
+ result = Math.abs(timestamp - leftValue) > Math.abs(timestamp - rightValue) ? rightOffset : leftOffset;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return result;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public PhyAndCQOffset binarySearchInCQ(String topic, int queueId, long high, long low, long targetPhyOffset,
|
|||
|
|
+ boolean min) throws RocksDBException {
|
|||
|
|
+ long resultCQOffset = -1L;
|
|||
|
|
+ long resultPhyOffset = -1L;
|
|||
|
|
+ while (high >= low) {
|
|||
|
|
+ long midCQOffset = low + ((high - low) >>> 1);
|
|||
|
|
+ ByteBuffer byteBuffer = getCQInKV(topic, queueId, midCQOffset);
|
|||
|
|
+ if (this.messageStore.getMessageStoreConfig().isEnableRocksDBLog()) {
|
|||
|
|
+ ROCKSDB_LOG.warn("binarySearchInCQ. {}, {}, {}, {}, {}", topic, queueId, midCQOffset, low, high);
|
|||
|
|
+ }
|
|||
|
|
+ if (byteBuffer == null) {
|
|||
|
|
+ low = midCQOffset + 1;
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ final long phyOffset = byteBuffer.getLong(PHY_OFFSET_OFFSET);
|
|||
|
|
+ if (phyOffset == targetPhyOffset) {
|
|||
|
|
+ if (min) {
|
|||
|
|
+ resultCQOffset = midCQOffset;
|
|||
|
|
+ resultPhyOffset = phyOffset;
|
|||
|
|
+ }
|
|||
|
|
+ break;
|
|||
|
|
+ } else if (phyOffset > targetPhyOffset) {
|
|||
|
|
+ high = midCQOffset - 1;
|
|||
|
|
+ if (min) {
|
|||
|
|
+ resultCQOffset = midCQOffset;
|
|||
|
|
+ resultPhyOffset = phyOffset;
|
|||
|
|
+ }
|
|||
|
|
+ } else {
|
|||
|
|
+ low = midCQOffset + 1;
|
|||
|
|
+ if (!min) {
|
|||
|
|
+ resultCQOffset = midCQOffset;
|
|||
|
|
+ resultPhyOffset = phyOffset;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return new PhyAndCQOffset(resultPhyOffset, resultCQOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static Pair<ByteBuffer, ByteBuffer> getCQByteBufferPair() {
|
|||
|
|
+ ByteBuffer cqKey = ByteBuffer.allocateDirect(RocksDBConsumeQueueStore.MAX_KEY_LEN);
|
|||
|
|
+ ByteBuffer cqValue = ByteBuffer.allocateDirect(CQ_UNIT_SIZE);
|
|||
|
|
+ return new Pair<>(cqKey, cqValue);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer buildCQKeyByteBuffer(final byte[] topicBytes, final int queueId, final long cqOffset) {
|
|||
|
|
+ final ByteBuffer byteBuffer = ByteBuffer.allocate(CQ_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicBytes.length);
|
|||
|
|
+ buildCQKeyByteBuffer0(byteBuffer, topicBytes, queueId, cqOffset);
|
|||
|
|
+ return byteBuffer;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCQKeyByteBuffer(final ByteBuffer byteBuffer, final byte[] topicBytes, final int queueId, final long cqOffset) {
|
|||
|
|
+ byteBuffer.position(0).limit(CQ_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicBytes.length);
|
|||
|
|
+ buildCQKeyByteBuffer0(byteBuffer, topicBytes, queueId, cqOffset);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCQKeyByteBuffer0(final ByteBuffer byteBuffer, final byte[] topicBytes, final int queueId, final long cqOffset) {
|
|||
|
|
+ byteBuffer.putInt(topicBytes.length).put(CTRL_1).put(topicBytes).put(CTRL_1).putInt(queueId).put(CTRL_1).putLong(cqOffset);
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCQValueByteBuffer(final ByteBuffer byteBuffer, final long phyOffset, final int msgSize, final long tagsCode, final long storeTimestamp) {
|
|||
|
|
+ byteBuffer.position(0).limit(CQ_UNIT_SIZE);
|
|||
|
|
+ buildCQValueByteBuffer0(byteBuffer, phyOffset, msgSize, tagsCode, storeTimestamp);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void buildCQValueByteBuffer0(final ByteBuffer byteBuffer, final long phyOffset, final int msgSize,
|
|||
|
|
+ final long tagsCode, final long storeTimestamp) {
|
|||
|
|
+ byteBuffer.putLong(phyOffset).putInt(msgSize).putLong(tagsCode).putLong(storeTimestamp);
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private ByteBuffer buildDeleteCQKey(final boolean start, final byte[] topicBytes, final int queueId) {
|
|||
|
|
+ final ByteBuffer byteBuffer = ByteBuffer.allocate(DELETE_CQ_KEY_LENGTH_WITHOUT_TOPIC_BYTES + topicBytes.length);
|
|||
|
|
+
|
|||
|
|
+ byteBuffer.putInt(topicBytes.length).put(CTRL_1).put(topicBytes).put(CTRL_1).putInt(queueId).put(start ? CTRL_0 : CTRL_2);
|
|||
|
|
+ byteBuffer.flip();
|
|||
|
|
+ return byteBuffer;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueCompactionFilterFactory.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueCompactionFilterFactory.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..aa796c4d3
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueCompactionFilterFactory.java
|
|||
|
|
@@ -0,0 +1,47 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.rocksdb;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.constant.LoggerName;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
+import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
+import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
+import org.rocksdb.AbstractCompactionFilter;
|
|||
|
|
+import org.rocksdb.AbstractCompactionFilterFactory;
|
|||
|
|
+import org.rocksdb.RemoveConsumeQueueCompactionFilter;
|
|||
|
|
+
|
|||
|
|
+public class ConsumeQueueCompactionFilterFactory extends AbstractCompactionFilterFactory<RemoveConsumeQueueCompactionFilter> {
|
|||
|
|
+ private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKSDB_LOGGER_NAME);
|
|||
|
|
+ private final MessageStore messageStore;
|
|||
|
|
+
|
|||
|
|
+ public ConsumeQueueCompactionFilterFactory(final MessageStore messageStore) {
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String name() {
|
|||
|
|
+ return "ConsumeQueueCompactionFilterFactory";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public RemoveConsumeQueueCompactionFilter createCompactionFilter(final AbstractCompactionFilter.Context context) {
|
|||
|
|
+ long minPhyOffset = this.messageStore.getMinPhyOffset();
|
|||
|
|
+ LOGGER.info("manualCompaction minPhyOffset: {}, isFull: {}, isManual: {}",
|
|||
|
|
+ minPhyOffset, context.isFullCompaction(), context.isManualCompaction());
|
|||
|
|
+ return new RemoveConsumeQueueCompactionFilter(minPhyOffset);
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueRocksDBStorage.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueRocksDBStorage.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..362684560
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/ConsumeQueueRocksDBStorage.java
|
|||
|
|
@@ -0,0 +1,133 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.rocksdb;
|
|||
|
|
+
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+import org.apache.rocketmq.common.config.AbstractRocksDBStorage;
|
|||
|
|
+import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
+import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
+import org.rocksdb.ColumnFamilyDescriptor;
|
|||
|
|
+import org.rocksdb.ColumnFamilyHandle;
|
|||
|
|
+import org.rocksdb.ColumnFamilyOptions;
|
|||
|
|
+import org.rocksdb.CompactRangeOptions;
|
|||
|
|
+import org.rocksdb.ReadOptions;
|
|||
|
|
+import org.rocksdb.RocksDB;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
+import org.rocksdb.RocksIterator;
|
|||
|
|
+import org.rocksdb.WriteBatch;
|
|||
|
|
+import org.rocksdb.WriteOptions;
|
|||
|
|
+
|
|||
|
|
+public class ConsumeQueueRocksDBStorage extends AbstractRocksDBStorage {
|
|||
|
|
+ private final MessageStore messageStore;
|
|||
|
|
+ private volatile ColumnFamilyHandle offsetCFHandle;
|
|||
|
|
+
|
|||
|
|
+ public ConsumeQueueRocksDBStorage(final MessageStore messageStore, final String dbPath, final int prefixLen) {
|
|||
|
|
+ this.messageStore = messageStore;
|
|||
|
|
+ this.dbPath = dbPath;
|
|||
|
|
+ this.readOnly = false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void initOptions() {
|
|||
|
|
+ this.options = RocksDBOptionsFactory.createDBOptions();
|
|||
|
|
+
|
|||
|
|
+ this.writeOptions = new WriteOptions();
|
|||
|
|
+ this.writeOptions.setSync(false);
|
|||
|
|
+ this.writeOptions.setDisableWAL(true);
|
|||
|
|
+ this.writeOptions.setNoSlowdown(true);
|
|||
|
|
+
|
|||
|
|
+ this.totalOrderReadOptions = new ReadOptions();
|
|||
|
|
+ this.totalOrderReadOptions.setPrefixSameAsStart(false);
|
|||
|
|
+ this.totalOrderReadOptions.setTotalOrderSeek(false);
|
|||
|
|
+
|
|||
|
|
+ this.compactRangeOptions = new CompactRangeOptions();
|
|||
|
|
+ this.compactRangeOptions.setBottommostLevelCompaction(CompactRangeOptions.BottommostLevelCompaction.kForce);
|
|||
|
|
+ this.compactRangeOptions.setAllowWriteStall(true);
|
|||
|
|
+ this.compactRangeOptions.setExclusiveManualCompaction(false);
|
|||
|
|
+ this.compactRangeOptions.setChangeLevel(true);
|
|||
|
|
+ this.compactRangeOptions.setTargetLevel(-1);
|
|||
|
|
+ this.compactRangeOptions.setMaxSubcompactions(4);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ protected boolean postLoad() {
|
|||
|
|
+ try {
|
|||
|
|
+ UtilAll.ensureDirOK(this.dbPath);
|
|||
|
|
+
|
|||
|
|
+ initOptions();
|
|||
|
|
+
|
|||
|
|
+ final List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList();
|
|||
|
|
+
|
|||
|
|
+ ColumnFamilyOptions cqCfOptions = RocksDBOptionsFactory.createCQCFOptions(this.messageStore);
|
|||
|
|
+ this.cfOptions.add(cqCfOptions);
|
|||
|
|
+ cfDescriptors.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cqCfOptions));
|
|||
|
|
+
|
|||
|
|
+ ColumnFamilyOptions offsetCfOptions = RocksDBOptionsFactory.createOffsetCFOptions();
|
|||
|
|
+ this.cfOptions.add(offsetCfOptions);
|
|||
|
|
+ cfDescriptors.add(new ColumnFamilyDescriptor("offset".getBytes(DataConverter.CHARSET_UTF8), offsetCfOptions));
|
|||
|
|
+
|
|||
|
|
+ final List<ColumnFamilyHandle> cfHandles = new ArrayList();
|
|||
|
|
+ open(cfDescriptors, cfHandles);
|
|||
|
|
+
|
|||
|
|
+ this.defaultCFHandle = cfHandles.get(0);
|
|||
|
|
+ this.offsetCFHandle = cfHandles.get(1);
|
|||
|
|
+ } catch (final Exception e) {
|
|||
|
|
+ LOGGER.error("postLoad Failed. {}", this.dbPath, e);
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+ return true;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ protected void preShutdown() {
|
|||
|
|
+ this.offsetCFHandle.close();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public byte[] getCQ(final byte[] keyBytes) throws RocksDBException {
|
|||
|
|
+ return get(this.defaultCFHandle, this.totalOrderReadOptions, keyBytes);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public byte[] getOffset(final byte[] keyBytes) throws RocksDBException {
|
|||
|
|
+ return get(this.offsetCFHandle, this.totalOrderReadOptions, keyBytes);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public List<byte[]> multiGet(final List<ColumnFamilyHandle> cfhList, final List<byte[]> keys) throws RocksDBException {
|
|||
|
|
+ return multiGet(this.totalOrderReadOptions, cfhList, keys);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void batchPut(final WriteBatch batch) throws RocksDBException {
|
|||
|
|
+ batchPut(this.writeOptions, batch);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public void manualCompaction(final long minPhyOffset) {
|
|||
|
|
+ try {
|
|||
|
|
+ manualCompaction(minPhyOffset, this.compactRangeOptions);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ LOGGER.error("manualCompaction Failed. minPhyOffset: {}", minPhyOffset, e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public RocksIterator seekOffsetCF() {
|
|||
|
|
+ return this.db.newIterator(this.offsetCFHandle, this.totalOrderReadOptions);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public ColumnFamilyHandle getOffsetCFHandle() {
|
|||
|
|
+ return this.offsetCFHandle;
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
\ No newline at end of file
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..a3a99d334
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java
|
|||
|
|
@@ -0,0 +1,161 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.rocksdb;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.config.ConfigRocksDBStorage;
|
|||
|
|
+import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
+import org.rocksdb.BlockBasedTableConfig;
|
|||
|
|
+import org.rocksdb.BloomFilter;
|
|||
|
|
+import org.rocksdb.ColumnFamilyOptions;
|
|||
|
|
+import org.rocksdb.CompactionOptionsUniversal;
|
|||
|
|
+import org.rocksdb.CompactionStopStyle;
|
|||
|
|
+import org.rocksdb.CompactionStyle;
|
|||
|
|
+import org.rocksdb.CompressionType;
|
|||
|
|
+import org.rocksdb.DBOptions;
|
|||
|
|
+import org.rocksdb.DataBlockIndexType;
|
|||
|
|
+import org.rocksdb.IndexType;
|
|||
|
|
+import org.rocksdb.InfoLogLevel;
|
|||
|
|
+import org.rocksdb.LRUCache;
|
|||
|
|
+import org.rocksdb.RateLimiter;
|
|||
|
|
+import org.rocksdb.SkipListMemTableConfig;
|
|||
|
|
+import org.rocksdb.Statistics;
|
|||
|
|
+import org.rocksdb.StatsLevel;
|
|||
|
|
+import org.rocksdb.StringAppendOperator;
|
|||
|
|
+import org.rocksdb.WALRecoveryMode;
|
|||
|
|
+import org.rocksdb.util.SizeUnit;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBOptionsFactory {
|
|||
|
|
+
|
|||
|
|
+ public static ColumnFamilyOptions createCQCFOptions(final MessageStore messageStore) {
|
|||
|
|
+ BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig().
|
|||
|
|
+ setFormatVersion(5).
|
|||
|
|
+ setIndexType(IndexType.kBinarySearch).
|
|||
|
|
+ setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash).
|
|||
|
|
+ setDataBlockHashTableUtilRatio(0.75).
|
|||
|
|
+ setBlockSize(32 * SizeUnit.KB).
|
|||
|
|
+ setMetadataBlockSize(4 * SizeUnit.KB).
|
|||
|
|
+ setFilterPolicy(new BloomFilter(16, false)).
|
|||
|
|
+ setCacheIndexAndFilterBlocks(false).
|
|||
|
|
+ setCacheIndexAndFilterBlocksWithHighPriority(true).
|
|||
|
|
+ setPinL0FilterAndIndexBlocksInCache(false).
|
|||
|
|
+ setPinTopLevelIndexAndFilter(true).
|
|||
|
|
+ setBlockCache(new LRUCache(1024 * SizeUnit.MB, 8, false)).
|
|||
|
|
+ setWholeKeyFiltering(true);
|
|||
|
|
+
|
|||
|
|
+ ColumnFamilyOptions columnFamilyOptions = new ColumnFamilyOptions();
|
|||
|
|
+ CompactionOptionsUniversal compactionOption = new CompactionOptionsUniversal();
|
|||
|
|
+ compactionOption.setSizeRatio(100).
|
|||
|
|
+ setMaxSizeAmplificationPercent(25).
|
|||
|
|
+ setAllowTrivialMove(true).
|
|||
|
|
+ setMinMergeWidth(2).
|
|||
|
|
+ setMaxMergeWidth(Integer.MAX_VALUE).
|
|||
|
|
+ setStopStyle(CompactionStopStyle.CompactionStopStyleTotalSize).
|
|||
|
|
+ setCompressionSizePercent(-1);
|
|||
|
|
+ return columnFamilyOptions.setMaxWriteBufferNumber(4).
|
|||
|
|
+ setWriteBufferSize(128 * SizeUnit.MB).
|
|||
|
|
+ setMinWriteBufferNumberToMerge(1).
|
|||
|
|
+ setTableFormatConfig(blockBasedTableConfig).
|
|||
|
|
+ setMemTableConfig(new SkipListMemTableConfig()).
|
|||
|
|
+ setCompressionType(CompressionType.LZ4_COMPRESSION).
|
|||
|
|
+ setBottommostCompressionType(CompressionType.ZSTD_COMPRESSION).
|
|||
|
|
+ setNumLevels(7).
|
|||
|
|
+ setCompactionStyle(CompactionStyle.UNIVERSAL).
|
|||
|
|
+ setCompactionOptionsUniversal(compactionOption).
|
|||
|
|
+ setMaxCompactionBytes(100 * SizeUnit.GB).
|
|||
|
|
+ setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB).
|
|||
|
|
+ setHardPendingCompactionBytesLimit(256 * SizeUnit.GB).
|
|||
|
|
+ setLevel0FileNumCompactionTrigger(2).
|
|||
|
|
+ setLevel0SlowdownWritesTrigger(8).
|
|||
|
|
+ setLevel0StopWritesTrigger(10).
|
|||
|
|
+ setTargetFileSizeBase(256 * SizeUnit.MB).
|
|||
|
|
+ setTargetFileSizeMultiplier(2).
|
|||
|
|
+ setMergeOperator(new StringAppendOperator()).
|
|||
|
|
+ setCompactionFilterFactory(new ConsumeQueueCompactionFilterFactory(messageStore)).
|
|||
|
|
+ setReportBgIoStats(true).
|
|||
|
|
+ setOptimizeFiltersForHits(true);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public static ColumnFamilyOptions createOffsetCFOptions() {
|
|||
|
|
+ BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig().
|
|||
|
|
+ setFormatVersion(5).
|
|||
|
|
+ setIndexType(IndexType.kBinarySearch).
|
|||
|
|
+ setDataBlockIndexType(DataBlockIndexType.kDataBlockBinarySearch).
|
|||
|
|
+ setBlockSize(32 * SizeUnit.KB).
|
|||
|
|
+ setFilterPolicy(new BloomFilter(16, false)).
|
|||
|
|
+ setCacheIndexAndFilterBlocks(false).
|
|||
|
|
+ setCacheIndexAndFilterBlocksWithHighPriority(true).
|
|||
|
|
+ setPinL0FilterAndIndexBlocksInCache(false).
|
|||
|
|
+ setPinTopLevelIndexAndFilter(true).
|
|||
|
|
+ setBlockCache(new LRUCache(128 * SizeUnit.MB, 8, false)).
|
|||
|
|
+ setWholeKeyFiltering(true);
|
|||
|
|
+
|
|||
|
|
+ ColumnFamilyOptions columnFamilyOptions = new ColumnFamilyOptions();
|
|||
|
|
+ return columnFamilyOptions.setMaxWriteBufferNumber(4).
|
|||
|
|
+ setWriteBufferSize(64 * SizeUnit.MB).
|
|||
|
|
+ setMinWriteBufferNumberToMerge(1).
|
|||
|
|
+ setTableFormatConfig(blockBasedTableConfig).
|
|||
|
|
+ setMemTableConfig(new SkipListMemTableConfig()).
|
|||
|
|
+ setCompressionType(CompressionType.NO_COMPRESSION).
|
|||
|
|
+ setNumLevels(7).
|
|||
|
|
+ setCompactionStyle(CompactionStyle.LEVEL).
|
|||
|
|
+ setLevel0FileNumCompactionTrigger(2).
|
|||
|
|
+ setLevel0SlowdownWritesTrigger(8).
|
|||
|
|
+ setLevel0StopWritesTrigger(10).
|
|||
|
|
+ setTargetFileSizeBase(64 * SizeUnit.MB).
|
|||
|
|
+ setTargetFileSizeMultiplier(2).
|
|||
|
|
+ setMaxBytesForLevelBase(256 * SizeUnit.MB).
|
|||
|
|
+ setMaxBytesForLevelMultiplier(2).
|
|||
|
|
+ setMergeOperator(new StringAppendOperator()).
|
|||
|
|
+ setInplaceUpdateSupport(true);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ /**
|
|||
|
|
+ * Create a rocksdb db options, the user must take care to close it after closing db.
|
|||
|
|
+ * @return
|
|||
|
|
+ */
|
|||
|
|
+ public static DBOptions createDBOptions() {
|
|||
|
|
+ //Turn based on https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
|
|||
|
|
+ // and http://gitlab.alibaba-inc.com/aloha/aloha/blob/branch_2_5_0/jstorm-core/src/main/java/com/alibaba/jstorm/cache/rocksdb/RocksDbOptionsFactory.java
|
|||
|
|
+ DBOptions options = new DBOptions();
|
|||
|
|
+ Statistics statistics = new Statistics();
|
|||
|
|
+ statistics.setStatsLevel(StatsLevel.EXCEPT_DETAILED_TIMERS);
|
|||
|
|
+ return options.
|
|||
|
|
+ setDbLogDir(ConfigRocksDBStorage.getDBLogDir()).
|
|||
|
|
+ setInfoLogLevel(InfoLogLevel.INFO_LEVEL).
|
|||
|
|
+ setWalRecoveryMode(WALRecoveryMode.PointInTimeRecovery).
|
|||
|
|
+ setManualWalFlush(true).
|
|||
|
|
+ setMaxTotalWalSize(0).
|
|||
|
|
+ setWalSizeLimitMB(0).
|
|||
|
|
+ setWalTtlSeconds(0).
|
|||
|
|
+ setCreateIfMissing(true).
|
|||
|
|
+ setCreateMissingColumnFamilies(true).
|
|||
|
|
+ setMaxOpenFiles(-1).
|
|||
|
|
+ setMaxLogFileSize(1 * SizeUnit.GB).
|
|||
|
|
+ setKeepLogFileNum(5).
|
|||
|
|
+ setMaxManifestFileSize(1 * SizeUnit.GB).
|
|||
|
|
+ setAllowConcurrentMemtableWrite(false).
|
|||
|
|
+ setStatistics(statistics).
|
|||
|
|
+ setAtomicFlush(true).
|
|||
|
|
+ setMaxBackgroundJobs(32).
|
|||
|
|
+ setMaxSubcompactions(8).
|
|||
|
|
+ setParanoidChecks(true).
|
|||
|
|
+ setDelayedWriteRate(16 * SizeUnit.MB).
|
|||
|
|
+ setRateLimiter(new RateLimiter(100 * SizeUnit.MB)).
|
|||
|
|
+ setUseDirectIoForFlushAndCompaction(false).
|
|||
|
|
+ setUseDirectReads(false);
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
index ac4c61cd6..3ab51a26d 100644
|
|||
|
|
--- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
+++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java
|
|||
|
|
@@ -57,7 +57,6 @@ import org.apache.rocketmq.common.topic.TopicValidator;
|
|||
|
|
import org.apache.rocketmq.common.utils.ThreadUtils;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.Logger;
|
|||
|
|
import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
|
|||
|
|
-import org.apache.rocketmq.store.ConsumeQueue;
|
|||
|
|
import org.apache.rocketmq.store.DefaultMessageStore;
|
|||
|
|
import org.apache.rocketmq.store.MessageStore;
|
|||
|
|
import org.apache.rocketmq.store.PutMessageResult;
|
|||
|
|
@@ -66,6 +65,9 @@ import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
import org.apache.rocketmq.store.logfile.MappedFile;
|
|||
|
|
import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
+import org.apache.rocketmq.store.queue.CqUnit;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ReferredIterator;
|
|||
|
|
import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
import org.apache.rocketmq.store.util.PerfCounter;
|
|||
|
|
|
|||
|
|
@@ -333,7 +335,7 @@ public class TimerMessageStore {
|
|||
|
|
// if not, use cq offset.
|
|||
|
|
long msgQueueOffset = messageExt.getQueueOffset();
|
|||
|
|
int queueId = messageExt.getQueueId();
|
|||
|
|
- ConsumeQueue cq = (ConsumeQueue) this.messageStore.getConsumeQueue(TIMER_TOPIC, queueId);
|
|||
|
|
+ ConsumeQueueInterface cq = this.messageStore.getConsumeQueue(TIMER_TOPIC, queueId);
|
|||
|
|
if (null == cq) {
|
|||
|
|
return msgQueueOffset;
|
|||
|
|
}
|
|||
|
|
@@ -346,15 +348,18 @@ public class TimerMessageStore {
|
|||
|
|
offsetPy, sizePy);
|
|||
|
|
break;
|
|||
|
|
}
|
|||
|
|
- SelectMappedBufferResult bufferCQ = cq.getIndexBuffer(tmpOffset);
|
|||
|
|
- if (null == bufferCQ) {
|
|||
|
|
- // offset in msg may be greater than offset of cq.
|
|||
|
|
- tmpOffset -= 1;
|
|||
|
|
- continue;
|
|||
|
|
- }
|
|||
|
|
+ ReferredIterator<CqUnit> iterator = null;
|
|||
|
|
try {
|
|||
|
|
- long offsetPyTemp = bufferCQ.getByteBuffer().getLong();
|
|||
|
|
- int sizePyTemp = bufferCQ.getByteBuffer().getInt();
|
|||
|
|
+ iterator = cq.iterateFrom(tmpOffset);
|
|||
|
|
+ CqUnit cqUnit = null;
|
|||
|
|
+ if (null == iterator || (cqUnit = iterator.next()) == null) {
|
|||
|
|
+ // offset in msg may be greater than offset of cq.
|
|||
|
|
+ tmpOffset -= 1;
|
|||
|
|
+ continue;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ long offsetPyTemp = cqUnit.getPos();
|
|||
|
|
+ int sizePyTemp = cqUnit.getSize();
|
|||
|
|
if (offsetPyTemp == offsetPy && sizePyTemp == sizePy) {
|
|||
|
|
LOGGER.info("reviseQueueOffset check cq offset ok. {}, {}, {}",
|
|||
|
|
tmpOffset, offsetPyTemp, sizePyTemp);
|
|||
|
|
@@ -365,7 +370,9 @@ public class TimerMessageStore {
|
|||
|
|
} catch (Throwable e) {
|
|||
|
|
LOGGER.error("reviseQueueOffset check cq offset error.", e);
|
|||
|
|
} finally {
|
|||
|
|
- bufferCQ.release();
|
|||
|
|
+ if (iterator != null) {
|
|||
|
|
+ iterator.release();
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@@ -633,7 +640,7 @@ public class TimerMessageStore {
|
|||
|
|
if (!isRunningEnqueue()) {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
- ConsumeQueue cq = (ConsumeQueue) this.messageStore.getConsumeQueue(TIMER_TOPIC, queueId);
|
|||
|
|
+ ConsumeQueueInterface cq = this.messageStore.getConsumeQueue(TIMER_TOPIC, queueId);
|
|||
|
|
if (null == cq) {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
@@ -643,18 +650,22 @@ public class TimerMessageStore {
|
|||
|
|
currQueueOffset = cq.getMinOffsetInQueue();
|
|||
|
|
}
|
|||
|
|
long offset = currQueueOffset;
|
|||
|
|
- SelectMappedBufferResult bufferCQ = cq.getIndexBuffer(offset);
|
|||
|
|
- if (null == bufferCQ) {
|
|||
|
|
- return false;
|
|||
|
|
- }
|
|||
|
|
+ ReferredIterator<CqUnit> iterator = null;
|
|||
|
|
try {
|
|||
|
|
+ iterator = cq.iterateFrom(offset);
|
|||
|
|
+ if (null == iterator) {
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
int i = 0;
|
|||
|
|
- for (; i < bufferCQ.getSize(); i += ConsumeQueue.CQ_STORE_UNIT_SIZE) {
|
|||
|
|
+ while (iterator.hasNext()) {
|
|||
|
|
+ i++;
|
|||
|
|
perfCounterTicks.startTick("enqueue_get");
|
|||
|
|
try {
|
|||
|
|
- long offsetPy = bufferCQ.getByteBuffer().getLong();
|
|||
|
|
- int sizePy = bufferCQ.getByteBuffer().getInt();
|
|||
|
|
- bufferCQ.getByteBuffer().getLong(); //tags code
|
|||
|
|
+ CqUnit cqUnit = iterator.next();
|
|||
|
|
+ long offsetPy = cqUnit.getPos();
|
|||
|
|
+ int sizePy = cqUnit.getSize();
|
|||
|
|
+ cqUnit.getTagsCode(); //tags code
|
|||
|
|
MessageExt msgExt = getMessageByCommitOffset(offsetPy, sizePy);
|
|||
|
|
if (null == msgExt) {
|
|||
|
|
perfCounterTicks.getCounter("enqueue_get_miss");
|
|||
|
|
@@ -663,7 +674,7 @@ public class TimerMessageStore {
|
|||
|
|
lastEnqueueButExpiredStoreTime = msgExt.getStoreTimestamp();
|
|||
|
|
long delayedTime = Long.parseLong(msgExt.getProperty(TIMER_OUT_MS));
|
|||
|
|
// use CQ offset, not offset in Message
|
|||
|
|
- msgExt.setQueueOffset(offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE));
|
|||
|
|
+ msgExt.setQueueOffset(offset + i);
|
|||
|
|
TimerRequest timerRequest = new TimerRequest(offsetPy, sizePy, delayedTime, System.currentTimeMillis(), MAGIC_DEFAULT, msgExt);
|
|||
|
|
// System.out.printf("build enqueue request, %s%n", timerRequest);
|
|||
|
|
while (!enqueuePutQueue.offer(timerRequest, 3, TimeUnit.SECONDS)) {
|
|||
|
|
@@ -687,14 +698,16 @@ public class TimerMessageStore {
|
|||
|
|
if (!isRunningEnqueue()) {
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
- currQueueOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE);
|
|||
|
|
+ currQueueOffset = offset + i;
|
|||
|
|
}
|
|||
|
|
- currQueueOffset = offset + (i / ConsumeQueue.CQ_STORE_UNIT_SIZE);
|
|||
|
|
+ currQueueOffset = offset + i;
|
|||
|
|
return i > 0;
|
|||
|
|
} catch (Exception e) {
|
|||
|
|
LOGGER.error("Unknown exception in enqueuing", e);
|
|||
|
|
} finally {
|
|||
|
|
- bufferCQ.release();
|
|||
|
|
+ if (iterator != null) {
|
|||
|
|
+ iterator.release();
|
|||
|
|
+ }
|
|||
|
|
}
|
|||
|
|
return false;
|
|||
|
|
}
|
|||
|
|
@@ -1642,7 +1655,7 @@ public class TimerMessageStore {
|
|||
|
|
if (System.currentTimeMillis() - start > storeConfig.getTimerProgressLogIntervalMs()) {
|
|||
|
|
start = System.currentTimeMillis();
|
|||
|
|
long tmpQueueOffset = currQueueOffset;
|
|||
|
|
- ConsumeQueue cq = (ConsumeQueue) messageStore.getConsumeQueue(TIMER_TOPIC, 0);
|
|||
|
|
+ ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0);
|
|||
|
|
long maxOffsetInQueue = cq == null ? 0 : cq.getMaxOffsetInQueue();
|
|||
|
|
TimerMessageStore.LOGGER.info("[{}]Timer progress-check commitRead:[{}] currRead:[{}] currWrite:[{}] readBehind:{} currReadOffset:{} offsetBehind:{} behindMaster:{} " +
|
|||
|
|
"enqPutQueue:{} deqGetQueue:{} deqPutQueue:{} allCongestNum:{} enqExpiredStoreTime:{}",
|
|||
|
|
@@ -1685,7 +1698,7 @@ public class TimerMessageStore {
|
|||
|
|
|
|||
|
|
public long getEnqueueBehindMessages() {
|
|||
|
|
long tmpQueueOffset = currQueueOffset;
|
|||
|
|
- ConsumeQueue cq = (ConsumeQueue) messageStore.getConsumeQueue(TIMER_TOPIC, 0);
|
|||
|
|
+ ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0);
|
|||
|
|
long maxOffsetInQueue = cq == null ? 0 : cq.getMaxOffsetInQueue();
|
|||
|
|
return maxOffsetInQueue - tmpQueueOffset;
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
|
|||
|
|
index 12d1e5723..1d09ca86e 100644
|
|||
|
|
--- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java
|
|||
|
|
@@ -428,9 +428,10 @@ public class DefaultMessageStoreTest {
|
|||
|
|
|
|||
|
|
private long getStoreTime(CqUnit cqUnit) {
|
|||
|
|
try {
|
|||
|
|
- Method getStoreTime = getDefaultMessageStore().getClass().getDeclaredMethod("getStoreTime", CqUnit.class);
|
|||
|
|
+ Class abstractConsumeQueueStore = getDefaultMessageStore().getQueueStore().getClass().getSuperclass();
|
|||
|
|
+ Method getStoreTime = abstractConsumeQueueStore.getDeclaredMethod("getStoreTime", CqUnit.class);
|
|||
|
|
getStoreTime.setAccessible(true);
|
|||
|
|
- return (long) getStoreTime.invoke(getDefaultMessageStore(), cqUnit);
|
|||
|
|
+ return (long) getStoreTime.invoke(getDefaultMessageStore().getQueueStore(), cqUnit);
|
|||
|
|
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
|
|||
|
|
throw new RuntimeException(e);
|
|||
|
|
}
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
|
|||
|
|
index 85626a332..2447bbf68 100644
|
|||
|
|
--- a/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/MultiDispatchTest.java
|
|||
|
|
@@ -28,9 +28,11 @@ import org.apache.rocketmq.common.message.MessageConst;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+import org.apache.rocketmq.store.queue.MultiDispatch;
|
|||
|
|
import org.junit.After;
|
|||
|
|
import org.junit.Before;
|
|||
|
|
import org.junit.Test;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
import static org.apache.rocketmq.store.config.StorePathConfigHelper.getStorePathConsumeQueue;
|
|||
|
|
import static org.junit.Assert.assertEquals;
|
|||
|
|
@@ -69,15 +71,15 @@ public class MultiDispatchTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void queueKey() {
|
|||
|
|
+ public void lmqQueueKey() {
|
|||
|
|
MessageExtBrokerInner messageExtBrokerInner = mock(MessageExtBrokerInner.class);
|
|||
|
|
when(messageExtBrokerInner.getQueueId()).thenReturn(2);
|
|||
|
|
- String ret = consumeQueue.queueKey("%LMQ%lmq123", messageExtBrokerInner);
|
|||
|
|
+ String ret = MultiDispatch.lmqQueueKey("%LMQ%lmq123");
|
|||
|
|
assertEquals(ret, "%LMQ%lmq123-0");
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void wrapMultiDispatch() {
|
|||
|
|
+ public void wrapMultiDispatch() throws RocksDBException {
|
|||
|
|
MessageExtBrokerInner messageExtBrokerInner = buildMessageMultiQueue();
|
|||
|
|
messageStore.assignOffset(messageExtBrokerInner);
|
|||
|
|
assertEquals(messageExtBrokerInner.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET), "0,0");
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java b/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..acf5edf51
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/RocksDBMessageStoreTest.java
|
|||
|
|
@@ -0,0 +1,1060 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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;
|
|||
|
|
+
|
|||
|
|
+import java.io.File;
|
|||
|
|
+import java.io.RandomAccessFile;
|
|||
|
|
+import java.lang.reflect.InvocationTargetException;
|
|||
|
|
+import java.lang.reflect.Method;
|
|||
|
|
+import java.net.InetAddress;
|
|||
|
|
+import java.net.InetSocketAddress;
|
|||
|
|
+import java.net.SocketAddress;
|
|||
|
|
+import java.net.UnknownHostException;
|
|||
|
|
+import java.nio.MappedByteBuffer;
|
|||
|
|
+import java.nio.channels.FileChannel;
|
|||
|
|
+import java.nio.channels.OverlappingFileLockException;
|
|||
|
|
+import java.nio.charset.StandardCharsets;
|
|||
|
|
+import java.util.ArrayList;
|
|||
|
|
+import java.util.Arrays;
|
|||
|
|
+import java.util.HashSet;
|
|||
|
|
+import java.util.List;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+import java.util.Random;
|
|||
|
|
+import java.util.UUID;
|
|||
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|||
|
|
+import java.util.concurrent.ConcurrentMap;
|
|||
|
|
+import java.util.concurrent.TimeUnit;
|
|||
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
|
+
|
|||
|
|
+import com.google.common.collect.Sets;
|
|||
|
|
+
|
|||
|
|
+import org.apache.rocketmq.common.BrokerConfig;
|
|||
|
|
+import org.apache.rocketmq.common.MixAll;
|
|||
|
|
+import org.apache.rocketmq.common.TopicConfig;
|
|||
|
|
+import org.apache.rocketmq.common.UtilAll;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageBatch;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageDecoder;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExt;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBatch;
|
|||
|
|
+import org.apache.rocketmq.common.message.MessageExtBrokerInner;
|
|||
|
|
+import org.apache.rocketmq.store.config.BrokerRole;
|
|||
|
|
+import org.apache.rocketmq.store.config.FlushDiskType;
|
|||
|
|
+import org.apache.rocketmq.store.config.MessageStoreConfig;
|
|||
|
|
+import org.apache.rocketmq.store.config.StorePathConfigHelper;
|
|||
|
|
+import org.apache.rocketmq.store.queue.ConsumeQueueInterface;
|
|||
|
|
+import org.apache.rocketmq.store.queue.CqUnit;
|
|||
|
|
+import org.apache.rocketmq.store.stats.BrokerStatsManager;
|
|||
|
|
+import org.assertj.core.util.Strings;
|
|||
|
|
+import org.junit.After;
|
|||
|
|
+import org.junit.Assert;
|
|||
|
|
+import org.junit.Before;
|
|||
|
|
+import org.junit.Test;
|
|||
|
|
+import org.junit.runner.RunWith;
|
|||
|
|
+import org.mockito.junit.MockitoJUnitRunner;
|
|||
|
|
+
|
|||
|
|
+import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
+import static org.junit.Assert.assertEquals;
|
|||
|
|
+import static org.junit.Assert.assertTrue;
|
|||
|
|
+
|
|||
|
|
+@RunWith(MockitoJUnitRunner.class)
|
|||
|
|
+public class RocksDBMessageStoreTest {
|
|||
|
|
+ private final String storeMessage = "Once, there was a chance for me!";
|
|||
|
|
+ private final String messageTopic = "FooBar";
|
|||
|
|
+ private final String storeType = StoreType.DEFAULT_ROCKSDB.getStoreType();
|
|||
|
|
+ private int queueTotal = 100;
|
|||
|
|
+ private AtomicInteger queueId = new AtomicInteger(0);
|
|||
|
|
+ private SocketAddress bornHost;
|
|||
|
|
+ private SocketAddress storeHost;
|
|||
|
|
+ private byte[] messageBody;
|
|||
|
|
+ private MessageStore messageStore;
|
|||
|
|
+
|
|||
|
|
+ @Before
|
|||
|
|
+ public void init() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ storeHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123);
|
|||
|
|
+ bornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0);
|
|||
|
|
+
|
|||
|
|
+ messageStore = buildMessageStore();
|
|||
|
|
+ boolean load = messageStore.load();
|
|||
|
|
+ assertTrue(load);
|
|||
|
|
+ messageStore.start();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test(expected = OverlappingFileLockException.class)
|
|||
|
|
+ public void test_repeat_restart() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ throw new OverlappingFileLockException();
|
|||
|
|
+ }
|
|||
|
|
+ queueTotal = 1;
|
|||
|
|
+ messageBody = storeMessage.getBytes();
|
|||
|
|
+
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
|
|||
|
|
+ messageStoreConfig.setMappedFileSizeCommitLog(1024 * 8);
|
|||
|
|
+ messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 4);
|
|||
|
|
+ messageStoreConfig.setMaxHashSlotNum(100);
|
|||
|
|
+ messageStoreConfig.setMaxIndexNum(100 * 10);
|
|||
|
|
+ messageStoreConfig.setStorePathRootDir(System.getProperty("java.io.tmpdir") + File.separator + "store");
|
|||
|
|
+ messageStoreConfig.setHaListenPort(0);
|
|||
|
|
+ MessageStore master = new RocksDBMessageStore(messageStoreConfig, null, new MyMessageArrivingListener(), new BrokerConfig(), new ConcurrentHashMap<>());
|
|||
|
|
+
|
|||
|
|
+ boolean load = master.load();
|
|||
|
|
+ assertTrue(load);
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ master.start();
|
|||
|
|
+ master.start();
|
|||
|
|
+ } finally {
|
|||
|
|
+ master.shutdown();
|
|||
|
|
+ master.destroy();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @After
|
|||
|
|
+ public void destroy() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ messageStore.shutdown();
|
|||
|
|
+ messageStore.destroy();
|
|||
|
|
+
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
|
|||
|
|
+ File file = new File(messageStoreConfig.getStorePathRootDir());
|
|||
|
|
+ UtilAll.deleteFile(file);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageStore buildMessageStore() throws Exception {
|
|||
|
|
+ return buildMessageStore(null, "");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageStore buildMessageStore(String storePathRootDir, String topic) throws Exception {
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = new MessageStoreConfig();
|
|||
|
|
+ messageStoreConfig.setMappedFileSizeCommitLog(1024 * 1024 * 10);
|
|||
|
|
+ messageStoreConfig.setMappedFileSizeConsumeQueue(1024 * 1024 * 10);
|
|||
|
|
+ messageStoreConfig.setMaxHashSlotNum(10000);
|
|||
|
|
+ messageStoreConfig.setMaxIndexNum(100 * 100);
|
|||
|
|
+ messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH);
|
|||
|
|
+ messageStoreConfig.setFlushIntervalConsumeQueue(1);
|
|||
|
|
+ messageStoreConfig.setStoreType(storeType);
|
|||
|
|
+ messageStoreConfig.setHaListenPort(0);
|
|||
|
|
+ if (Strings.isNullOrEmpty(storePathRootDir)) {
|
|||
|
|
+ UUID uuid = UUID.randomUUID();
|
|||
|
|
+ storePathRootDir = System.getProperty("java.io.tmpdir") + File.separator + "store-" + uuid.toString();
|
|||
|
|
+ }
|
|||
|
|
+ messageStoreConfig.setStorePathRootDir(storePathRootDir);
|
|||
|
|
+ ConcurrentMap<String, TopicConfig> topicConfigTable = new ConcurrentHashMap<>();
|
|||
|
|
+ topicConfigTable.put(topic, new TopicConfig(topic, 1, 1));
|
|||
|
|
+ return new RocksDBMessageStore(messageStoreConfig,
|
|||
|
|
+ new BrokerStatsManager("simpleTest", true),
|
|||
|
|
+ new MyMessageArrivingListener(),
|
|||
|
|
+ new BrokerConfig(), topicConfigTable);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testWriteAndRead() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ long ipv4HostMsgs = 10;
|
|||
|
|
+ long ipv6HostMsgs = 10;
|
|||
|
|
+ long totalMsgs = ipv4HostMsgs + ipv6HostMsgs;
|
|||
|
|
+ queueTotal = 1;
|
|||
|
|
+ messageBody = storeMessage.getBytes();
|
|||
|
|
+ for (long i = 0; i < ipv4HostMsgs; i++) {
|
|||
|
|
+ messageStore.putMessage(buildMessage());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ for (long i = 0; i < ipv6HostMsgs; i++) {
|
|||
|
|
+ messageStore.putMessage(buildIPv6HostMessage());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ for (long i = 0; i < totalMsgs; i++) {
|
|||
|
|
+ GetMessageResult result = messageStore.getMessage("GROUP_A", "FooBar", 0, i, 1024 * 1024, null);
|
|||
|
|
+ assertThat(result).isNotNull();
|
|||
|
|
+ result.release();
|
|||
|
|
+ }
|
|||
|
|
+ verifyThatMasterIsFunctional(totalMsgs, messageStore);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testLookMessageByOffset_OffsetIsFirst() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = new Random().nextInt(10);
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ int firstOffset = 0;
|
|||
|
|
+ AppendMessageResult[] appendMessageResultArray = putMessages(totalCount, topic, queueId);
|
|||
|
|
+ AppendMessageResult firstResult = appendMessageResultArray[0];
|
|||
|
|
+
|
|||
|
|
+ MessageExt messageExt = messageStore.lookMessageByOffset(firstResult.getWroteOffset());
|
|||
|
|
+ MessageExt messageExt1 = getDefaultMessageStore().lookMessageByOffset(firstResult.getWroteOffset(), firstResult.getWroteBytes());
|
|||
|
|
+
|
|||
|
|
+ assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, firstOffset));
|
|||
|
|
+ assertThat(new String(messageExt1.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, firstOffset));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testLookMessageByOffset_OffsetIsLast() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = new Random().nextInt(10);
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResultArray = putMessages(totalCount, topic, queueId);
|
|||
|
|
+ int lastIndex = totalCount - 1;
|
|||
|
|
+ AppendMessageResult lastResult = appendMessageResultArray[lastIndex];
|
|||
|
|
+
|
|||
|
|
+ MessageExt messageExt = getDefaultMessageStore().lookMessageByOffset(lastResult.getWroteOffset(), lastResult.getWroteBytes());
|
|||
|
|
+
|
|||
|
|
+ assertThat(new String(messageExt.getBody())).isEqualTo(buildMessageBodyByOffset(storeMessage, lastIndex));
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testLookMessageByOffset_OffsetIsOutOfBound() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = new Random().nextInt(10);
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResultArray = putMessages(totalCount, topic, queueId);
|
|||
|
|
+ long lastOffset = getMaxOffset(appendMessageResultArray);
|
|||
|
|
+
|
|||
|
|
+ MessageExt messageExt = getDefaultMessageStore().lookMessageByOffset(lastOffset);
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageExt).isNull();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId);
|
|||
|
|
+ for (AppendMessageResult appendMessageResult : appendMessageResults) {
|
|||
|
|
+ long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp());
|
|||
|
|
+ CqUnit cqUnit = consumeQueue.get(offset);
|
|||
|
|
+ assertThat(cqUnit.getPos()).isEqualTo(appendMessageResult.getWroteOffset());
|
|||
|
|
+ assertThat(cqUnit.getSize()).isEqualTo(appendMessageResult.getWroteBytes());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime_TimestampIsSkewing() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ int skewing = 2;
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId);
|
|||
|
|
+ for (AppendMessageResult appendMessageResult : appendMessageResults) {
|
|||
|
|
+ long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() - skewing);
|
|||
|
|
+ CqUnit cqUnit = consumeQueue.get(offset);
|
|||
|
|
+ assertThat(cqUnit.getPos()).isEqualTo(appendMessageResult.getWroteOffset());
|
|||
|
|
+ assertThat(cqUnit.getSize()).isEqualTo(appendMessageResult.getWroteBytes());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime_TimestampSkewingIsLarge() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ int skewing = 20000;
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId);
|
|||
|
|
+ for (AppendMessageResult appendMessageResult : appendMessageResults) {
|
|||
|
|
+ long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() - skewing);
|
|||
|
|
+ CqUnit cqUnit = consumeQueue.get(offset);
|
|||
|
|
+ assertThat(cqUnit.getPos()).isEqualTo(appendMessageResults[0].getWroteOffset());
|
|||
|
|
+ assertThat(cqUnit.getSize()).isEqualTo(appendMessageResults[0].getWroteBytes());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime_ConsumeQueueNotFound1() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ int wrongQueueId = 1;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, false);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ long offset = messageStore.getOffsetInQueueByTime(topic, wrongQueueId, appendMessageResults[0].getStoreTimestamp());
|
|||
|
|
+
|
|||
|
|
+ assertThat(offset).isEqualTo(0);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime_ConsumeQueueNotFound2() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ int wrongQueueId = 1;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ putMessages(totalCount, topic, queueId, false);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ long messageStoreTimeStamp = messageStore.getMessageStoreTimeStamp(topic, wrongQueueId, 0);
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageStoreTimeStamp).isEqualTo(-1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetOffsetInQueueByTime_ConsumeQueueOffsetNotExist() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ int wrongQueueId = 1;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ putMessages(totalCount, topic, queueId, true);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ long messageStoreTimeStamp = messageStore.getMessageStoreTimeStamp(topic, wrongQueueId, -1);
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageStoreTimeStamp).isEqualTo(-1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetMessageStoreTimeStamp() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, false);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId);
|
|||
|
|
+ int minOffsetInQueue = (int) consumeQueue.getMinOffsetInQueue();
|
|||
|
|
+ for (int i = minOffsetInQueue; i < consumeQueue.getMaxOffsetInQueue(); i++) {
|
|||
|
|
+ long messageStoreTimeStamp = messageStore.getMessageStoreTimeStamp(topic, queueId, i);
|
|||
|
|
+ assertThat(messageStoreTimeStamp).isEqualTo(appendMessageResults[i].getStoreTimestamp());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetStoreTime_ParamIsNull() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ long storeTime = getStoreTime(null);
|
|||
|
|
+
|
|||
|
|
+ assertThat(storeTime).isEqualTo(-1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetStoreTime_EverythingIsOk() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ final int totalCount = 10;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ String topic = "FooBar";
|
|||
|
|
+ AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, false);
|
|||
|
|
+ //Thread.sleep(10);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ ConsumeQueueInterface consumeQueue = messageStore.getConsumeQueue(topic, queueId);
|
|||
|
|
+
|
|||
|
|
+ for (int i = 0; i < totalCount; i++) {
|
|||
|
|
+ CqUnit cqUnit = consumeQueue.get(i);
|
|||
|
|
+ long storeTime = getStoreTime(cqUnit);
|
|||
|
|
+ assertThat(storeTime).isEqualTo(appendMessageResults[i].getStoreTimestamp());
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetStoreTime_PhyOffsetIsLessThanCommitLogMinOffset() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ long phyOffset = -10;
|
|||
|
|
+ int size = 138;
|
|||
|
|
+ CqUnit cqUnit = new CqUnit(0, phyOffset, size, 0);
|
|||
|
|
+ long storeTime = getStoreTime(cqUnit);
|
|||
|
|
+
|
|||
|
|
+ assertThat(storeTime).isEqualTo(-1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutMessage_whenMessagePropertyIsTooLong() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String topicName = "messagePropertyIsTooLongTest";
|
|||
|
|
+ MessageExtBrokerInner illegalMessage = buildSpecifyLengthPropertyMessage("123".getBytes(StandardCharsets.UTF_8), topicName, Short.MAX_VALUE + 1);
|
|||
|
|
+ assertEquals(messageStore.putMessage(illegalMessage).getPutMessageStatus(), PutMessageStatus.PROPERTIES_SIZE_EXCEEDED);
|
|||
|
|
+ assertEquals(0L, messageStore.getQueueStore().getMaxOffset(topicName, 0).longValue());
|
|||
|
|
+ MessageExtBrokerInner normalMessage = buildSpecifyLengthPropertyMessage("123".getBytes(StandardCharsets.UTF_8), topicName, 100);
|
|||
|
|
+ assertEquals(messageStore.putMessage(normalMessage).getPutMessageStatus(), PutMessageStatus.PUT_OK);
|
|||
|
|
+ assertEquals(1L, messageStore.getQueueStore().getMaxOffset(topicName, 0).longValue());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private RocksDBMessageStore getDefaultMessageStore() {
|
|||
|
|
+ return (RocksDBMessageStore) this.messageStore;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private AppendMessageResult[] putMessages(int totalCount, String topic, int queueId) {
|
|||
|
|
+ return putMessages(totalCount, topic, queueId, false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private AppendMessageResult[] putMessages(int totalCount, String topic, int queueId, boolean interval) {
|
|||
|
|
+ AppendMessageResult[] appendMessageResultArray = new AppendMessageResult[totalCount];
|
|||
|
|
+ for (int i = 0; i < totalCount; i++) {
|
|||
|
|
+ String messageBody = buildMessageBodyByOffset(storeMessage, i);
|
|||
|
|
+
|
|||
|
|
+ MessageExtBrokerInner msgInner =
|
|||
|
|
+ i < totalCount / 2 ? buildMessage(messageBody.getBytes(), topic) : buildIPv6HostMessage(messageBody.getBytes(), topic);
|
|||
|
|
+ msgInner.setQueueId(queueId);
|
|||
|
|
+ PutMessageResult result = messageStore.putMessage(msgInner);
|
|||
|
|
+ appendMessageResultArray[i] = result.getAppendMessageResult();
|
|||
|
|
+ assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.PUT_OK);
|
|||
|
|
+ if (interval) {
|
|||
|
|
+ try {
|
|||
|
|
+ Thread.sleep(10);
|
|||
|
|
+ } catch (InterruptedException e) {
|
|||
|
|
+ throw new RuntimeException("Thread sleep ERROR");
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ return appendMessageResultArray;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private long getMaxOffset(AppendMessageResult[] appendMessageResultArray) {
|
|||
|
|
+ if (appendMessageResultArray == null) {
|
|||
|
|
+ return 0;
|
|||
|
|
+ }
|
|||
|
|
+ AppendMessageResult last = appendMessageResultArray[appendMessageResultArray.length - 1];
|
|||
|
|
+ return last.getWroteOffset() + last.getWroteBytes();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private String buildMessageBodyByOffset(String message, long i) {
|
|||
|
|
+ return String.format("%s offset %d", message, i);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private long getStoreTime(CqUnit cqUnit) {
|
|||
|
|
+ try {
|
|||
|
|
+ Class abstractConsumeQueueStore = getDefaultMessageStore().getQueueStore().getClass().getSuperclass();
|
|||
|
|
+ Method getStoreTime = abstractConsumeQueueStore.getDeclaredMethod("getStoreTime", CqUnit.class);
|
|||
|
|
+ getStoreTime.setAccessible(true);
|
|||
|
|
+ return (long) getStoreTime.invoke(getDefaultMessageStore().getQueueStore(), cqUnit);
|
|||
|
|
+ } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
|
|||
|
|
+ throw new RuntimeException(e);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageExtBrokerInner buildMessage(byte[] messageBody, String topic) {
|
|||
|
|
+ MessageExtBrokerInner msg = new MessageExtBrokerInner();
|
|||
|
|
+ msg.setTopic(topic);
|
|||
|
|
+ msg.setTags("TAG1");
|
|||
|
|
+ msg.setKeys("Hello");
|
|||
|
|
+ msg.setBody(messageBody);
|
|||
|
|
+ msg.setKeys(String.valueOf(System.currentTimeMillis()));
|
|||
|
|
+ msg.setQueueId(Math.abs(queueId.getAndIncrement()) % queueTotal);
|
|||
|
|
+ msg.setSysFlag(0);
|
|||
|
|
+ msg.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
+ msg.setStoreHost(storeHost);
|
|||
|
|
+ msg.setBornHost(bornHost);
|
|||
|
|
+ msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
|
|||
|
|
+ return msg;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageExtBrokerInner buildSpecifyLengthPropertyMessage(byte[] messageBody, String topic, int length) {
|
|||
|
|
+ StringBuilder stringBuilder = new StringBuilder();
|
|||
|
|
+ Random random = new Random();
|
|||
|
|
+ for (int i = 0; i < length; i++) {
|
|||
|
|
+ stringBuilder.append(random.nextInt(10));
|
|||
|
|
+ }
|
|||
|
|
+ MessageExtBrokerInner msg = new MessageExtBrokerInner();
|
|||
|
|
+ msg.putUserProperty("test", stringBuilder.toString());
|
|||
|
|
+ msg.setTopic(topic);
|
|||
|
|
+ msg.setTags("TAG1");
|
|||
|
|
+ msg.setKeys("Hello");
|
|||
|
|
+ msg.setBody(messageBody);
|
|||
|
|
+ msg.setQueueId(0);
|
|||
|
|
+ msg.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
+ msg.setStoreHost(storeHost);
|
|||
|
|
+ msg.setBornHost(bornHost);
|
|||
|
|
+ msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties()));
|
|||
|
|
+ return msg;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageExtBrokerInner buildIPv6HostMessage(byte[] messageBody, String topic) {
|
|||
|
|
+ MessageExtBrokerInner msg = new MessageExtBrokerInner();
|
|||
|
|
+ msg.setTopic(topic);
|
|||
|
|
+ msg.setTags("TAG1");
|
|||
|
|
+ msg.setKeys("Hello");
|
|||
|
|
+ msg.setBody(messageBody);
|
|||
|
|
+ msg.setMsgId("24084004018081003FAA1DDE2B3F898A00002A9F0000000000000CA0");
|
|||
|
|
+ msg.setKeys(String.valueOf(System.currentTimeMillis()));
|
|||
|
|
+ msg.setQueueId(Math.abs(queueId.getAndIncrement()) % queueTotal);
|
|||
|
|
+ msg.setSysFlag(0);
|
|||
|
|
+ msg.setBornHostV6Flag();
|
|||
|
|
+ msg.setStoreHostAddressV6Flag();
|
|||
|
|
+ msg.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
+ try {
|
|||
|
|
+ msg.setBornHost(new InetSocketAddress(InetAddress.getByName("1050:0000:0000:0000:0005:0600:300c:326b"), 0));
|
|||
|
|
+ } catch (UnknownHostException e) {
|
|||
|
|
+ e.printStackTrace();
|
|||
|
|
+ assertThat(Boolean.FALSE).isTrue();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ msg.setStoreHost(new InetSocketAddress(InetAddress.getByName("::1"), 0));
|
|||
|
|
+ } catch (UnknownHostException e) {
|
|||
|
|
+ e.printStackTrace();
|
|||
|
|
+ assertThat(Boolean.FALSE).isTrue();
|
|||
|
|
+ }
|
|||
|
|
+ return msg;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageExtBrokerInner buildMessage() {
|
|||
|
|
+ return buildMessage(messageBody, messageTopic);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ public MessageExtBatch buildMessageBatch(MessageBatch msgBatch) {
|
|||
|
|
+ MessageExtBatch msgExtBatch = new MessageExtBatch();
|
|||
|
|
+ msgExtBatch.setTopic(messageTopic);
|
|||
|
|
+ msgExtBatch.setTags("TAG1");
|
|||
|
|
+ msgExtBatch.setKeys("Hello");
|
|||
|
|
+ msgExtBatch.setBody(msgBatch.getBody());
|
|||
|
|
+ msgExtBatch.setKeys(String.valueOf(System.currentTimeMillis()));
|
|||
|
|
+ msgExtBatch.setQueueId(Math.abs(queueId.getAndIncrement()) % queueTotal);
|
|||
|
|
+ msgExtBatch.setSysFlag(0);
|
|||
|
|
+ msgExtBatch.setBornTimestamp(System.currentTimeMillis());
|
|||
|
|
+ msgExtBatch.setStoreHost(storeHost);
|
|||
|
|
+ msgExtBatch.setBornHost(bornHost);
|
|||
|
|
+ return msgExtBatch;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGroupCommit() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ long totalMsgs = 10;
|
|||
|
|
+ queueTotal = 1;
|
|||
|
|
+ messageBody = storeMessage.getBytes();
|
|||
|
|
+ for (long i = 0; i < totalMsgs; i++) {
|
|||
|
|
+ messageStore.putMessage(buildMessage());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ for (long i = 0; i < totalMsgs; i++) {
|
|||
|
|
+ GetMessageResult result = messageStore.getMessage("GROUP_A", "TOPIC_A", 0, i, 1024 * 1024, null);
|
|||
|
|
+ assertThat(result).isNotNull();
|
|||
|
|
+ result.release();
|
|||
|
|
+ }
|
|||
|
|
+ verifyThatMasterIsFunctional(totalMsgs, messageStore);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testMaxOffset() throws InterruptedException {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ int firstBatchMessages = 3;
|
|||
|
|
+ int queueId = 0;
|
|||
|
|
+ messageBody = storeMessage.getBytes();
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(0);
|
|||
|
|
+
|
|||
|
|
+ for (int i = 0; i < firstBatchMessages; i++) {
|
|||
|
|
+ final MessageExtBrokerInner msg = buildMessage();
|
|||
|
|
+ msg.setQueueId(queueId);
|
|||
|
|
+ messageStore.putMessage(msg);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ while (messageStore.dispatchBehindBytes() != 0) {
|
|||
|
|
+ TimeUnit.MILLISECONDS.sleep(1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(firstBatchMessages);
|
|||
|
|
+
|
|||
|
|
+ // Disable the dispatcher
|
|||
|
|
+ messageStore.getDispatcherList().clear();
|
|||
|
|
+
|
|||
|
|
+ int secondBatchMessages = 2;
|
|||
|
|
+
|
|||
|
|
+ for (int i = 0; i < secondBatchMessages; i++) {
|
|||
|
|
+ final MessageExtBrokerInner msg = buildMessage();
|
|||
|
|
+ msg.setQueueId(queueId);
|
|||
|
|
+ messageStore.putMessage(msg);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId)).isEqualTo(firstBatchMessages);
|
|||
|
|
+ assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId, true)).isEqualTo(firstBatchMessages);
|
|||
|
|
+ assertThat(messageStore.getMaxOffsetInQueue(messageTopic, queueId, false)).isEqualTo(firstBatchMessages + secondBatchMessages);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private MessageExtBrokerInner buildIPv6HostMessage() {
|
|||
|
|
+ return buildIPv6HostMessage(messageBody, "FooBar");
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void verifyThatMasterIsFunctional(long totalMsgs, MessageStore master) {
|
|||
|
|
+ for (long i = 0; i < totalMsgs; i++) {
|
|||
|
|
+ master.putMessage(buildMessage());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ for (long i = 0; i < totalMsgs; i++) {
|
|||
|
|
+ GetMessageResult result = master.getMessage("GROUP_A", "FooBar", 0, i, 1024 * 1024, null);
|
|||
|
|
+ assertThat(result).isNotNull();
|
|||
|
|
+ result.release();
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPullSize() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String topic = "pullSizeTopic";
|
|||
|
|
+
|
|||
|
|
+ for (int i = 0; i < 32; i++) {
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ }
|
|||
|
|
+ // wait for consume queue build
|
|||
|
|
+ // the sleep time should be great than consume queue flush interval
|
|||
|
|
+ //Thread.sleep(100);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ String group = "simple";
|
|||
|
|
+ GetMessageResult getMessageResult32 = messageStore.getMessage(group, topic, 0, 0, 32, null);
|
|||
|
|
+ assertThat(getMessageResult32.getMessageBufferList().size()).isEqualTo(32);
|
|||
|
|
+ getMessageResult32.release();
|
|||
|
|
+
|
|||
|
|
+ GetMessageResult getMessageResult20 = messageStore.getMessage(group, topic, 0, 0, 20, null);
|
|||
|
|
+ assertThat(getMessageResult20.getMessageBufferList().size()).isEqualTo(20);
|
|||
|
|
+
|
|||
|
|
+ getMessageResult20.release();
|
|||
|
|
+ GetMessageResult getMessageResult45 = messageStore.getMessage(group, topic, 0, 0, 10, null);
|
|||
|
|
+ assertThat(getMessageResult45.getMessageBufferList().size()).isEqualTo(10);
|
|||
|
|
+ getMessageResult45.release();
|
|||
|
|
+
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testRecover() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ String topic = "recoverTopic";
|
|||
|
|
+ messageBody = storeMessage.getBytes();
|
|||
|
|
+ for (int i = 0; i < 100; i++) {
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ // Thread.sleep(100);//wait for build consumer queue
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+
|
|||
|
|
+ long maxPhyOffset = messageStore.getMaxPhyOffset();
|
|||
|
|
+ long maxCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
|
|||
|
|
+
|
|||
|
|
+ //1.just reboot
|
|||
|
|
+ messageStore.shutdown();
|
|||
|
|
+ String storeRootDir = ((RocksDBMessageStore) messageStore).getMessageStoreConfig().getStorePathRootDir();
|
|||
|
|
+ messageStore = buildMessageStore(storeRootDir, topic);
|
|||
|
|
+ boolean load = messageStore.load();
|
|||
|
|
+ assertTrue(load);
|
|||
|
|
+ messageStore.start();
|
|||
|
|
+ assertTrue(maxPhyOffset == messageStore.getMaxPhyOffset());
|
|||
|
|
+ assertTrue(maxCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
|
|||
|
|
+
|
|||
|
|
+ //2.damage commit-log and reboot normal
|
|||
|
|
+ for (int i = 0; i < 100; i++) {
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ }
|
|||
|
|
+ //Thread.sleep(100);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ long secondLastPhyOffset = messageStore.getMaxPhyOffset();
|
|||
|
|
+ long secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
|
|||
|
|
+
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
+ messageStore.shutdown();
|
|||
|
|
+
|
|||
|
|
+ //damage last message
|
|||
|
|
+ damageCommitLog((RocksDBMessageStore) messageStore, secondLastPhyOffset);
|
|||
|
|
+
|
|||
|
|
+ //reboot
|
|||
|
|
+ messageStore = buildMessageStore(storeRootDir, topic);
|
|||
|
|
+ load = messageStore.load();
|
|||
|
|
+ assertTrue(load);
|
|||
|
|
+ messageStore.start();
|
|||
|
|
+ assertTrue(secondLastPhyOffset == messageStore.getMaxPhyOffset());
|
|||
|
|
+ assertTrue(secondLastCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
|
|||
|
|
+
|
|||
|
|
+ //3.damage commitlog and reboot abnormal
|
|||
|
|
+ for (int i = 0; i < 100; i++) {
|
|||
|
|
+ messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ }
|
|||
|
|
+ //Thread.sleep(100);
|
|||
|
|
+ StoreTestUtil.waitCommitLogReput((RocksDBMessageStore) messageStore);
|
|||
|
|
+ secondLastPhyOffset = messageStore.getMaxPhyOffset();
|
|||
|
|
+ secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0);
|
|||
|
|
+
|
|||
|
|
+ messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ messageStore.shutdown();
|
|||
|
|
+
|
|||
|
|
+ //damage last message
|
|||
|
|
+ damageCommitLog((RocksDBMessageStore) messageStore, secondLastPhyOffset);
|
|||
|
|
+ //add abort file
|
|||
|
|
+ String fileName = StorePathConfigHelper.getAbortFile(((RocksDBMessageStore) messageStore).getMessageStoreConfig().getStorePathRootDir());
|
|||
|
|
+ File file = new File(fileName);
|
|||
|
|
+ UtilAll.ensureDirOK(file.getParent());
|
|||
|
|
+ file.createNewFile();
|
|||
|
|
+
|
|||
|
|
+ messageStore = buildMessageStore(storeRootDir, topic);
|
|||
|
|
+ load = messageStore.load();
|
|||
|
|
+ assertTrue(load);
|
|||
|
|
+ messageStore.start();
|
|||
|
|
+ assertTrue(secondLastPhyOffset == messageStore.getMaxPhyOffset());
|
|||
|
|
+ assertTrue(secondLastCqOffset == messageStore.getMaxOffsetInQueue(topic, 0));
|
|||
|
|
+
|
|||
|
|
+ //message write again
|
|||
|
|
+ for (int i = 0; i < 100; i++) {
|
|||
|
|
+ messageExtBrokerInner = buildMessage();
|
|||
|
|
+ messageExtBrokerInner.setTopic(topic);
|
|||
|
|
+ messageExtBrokerInner.setQueueId(0);
|
|||
|
|
+ messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testStorePathOK() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ if (messageStore instanceof RocksDBMessageStore) {
|
|||
|
|
+ assertTrue(fileExists(((RocksDBMessageStore) messageStore).getStorePathPhysic()));
|
|||
|
|
+ assertTrue(fileExists(((RocksDBMessageStore) messageStore).getStorePathLogic()));
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean fileExists(String path) {
|
|||
|
|
+ if (path != null) {
|
|||
|
|
+ File f = new File(path);
|
|||
|
|
+ return f.exists();
|
|||
|
|
+ }
|
|||
|
|
+ return false;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private void damageCommitLog(RocksDBMessageStore store, long offset) throws Exception {
|
|||
|
|
+ assertThat(store).isNotNull();
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = store.getMessageStoreConfig();
|
|||
|
|
+ File file = new File(messageStoreConfig.getStorePathCommitLog() + File.separator + "00000000000000000000");
|
|||
|
|
+ try (RandomAccessFile raf = new RandomAccessFile(file, "rw");
|
|||
|
|
+ FileChannel fileChannel = raf.getChannel()) {
|
|||
|
|
+ MappedByteBuffer mappedByteBuffer = fileChannel.map(FileChannel.MapMode.READ_WRITE, 0, 1024 * 1024 * 10);
|
|||
|
|
+ int bodyLen = mappedByteBuffer.getInt((int) offset + 84);
|
|||
|
|
+ int topicLenIndex = (int) offset + 84 + bodyLen + 4;
|
|||
|
|
+ mappedByteBuffer.position(topicLenIndex);
|
|||
|
|
+ mappedByteBuffer.putInt(0);
|
|||
|
|
+ mappedByteBuffer.putInt(0);
|
|||
|
|
+ mappedByteBuffer.putInt(0);
|
|||
|
|
+ mappedByteBuffer.putInt(0);
|
|||
|
|
+ mappedByteBuffer.force();
|
|||
|
|
+ fileChannel.force(true);
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutMsgExceedsMaxLength() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ messageBody = new byte[4 * 1024 * 1024 + 1];
|
|||
|
|
+ MessageExtBrokerInner msg = buildMessage();
|
|||
|
|
+
|
|||
|
|
+ PutMessageResult result = messageStore.putMessage(msg);
|
|||
|
|
+ assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutMsgBatchExceedsMaxLength() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ messageBody = new byte[4 * 1024 * 1024 + 1];
|
|||
|
|
+ MessageExtBrokerInner msg1 = buildMessage();
|
|||
|
|
+ MessageExtBrokerInner msg2 = buildMessage();
|
|||
|
|
+ MessageExtBrokerInner msg3 = buildMessage();
|
|||
|
|
+
|
|||
|
|
+ MessageBatch msgBatch = MessageBatch.generateFromList(Arrays.asList(msg1, msg2, msg3));
|
|||
|
|
+ msgBatch.setBody(msgBatch.encode());
|
|||
|
|
+
|
|||
|
|
+ MessageExtBatch msgExtBatch = buildMessageBatch(msgBatch);
|
|||
|
|
+
|
|||
|
|
+ try {
|
|||
|
|
+ PutMessageResult result = this.messageStore.putMessages(msgExtBatch);
|
|||
|
|
+ } catch (Exception e) {
|
|||
|
|
+ assertThat(e.getMessage()).contains("message body size exceeded");
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutMsgWhenReplicasNotEnough() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = ((RocksDBMessageStore) this.messageStore).getMessageStoreConfig();
|
|||
|
|
+ messageStoreConfig.setBrokerRole(BrokerRole.SYNC_MASTER);
|
|||
|
|
+ messageStoreConfig.setTotalReplicas(2);
|
|||
|
|
+ messageStoreConfig.setInSyncReplicas(2);
|
|||
|
|
+ messageStoreConfig.setEnableAutoInSyncReplicas(false);
|
|||
|
|
+ ((RocksDBMessageStore) this.messageStore).getBrokerConfig().setEnableSlaveActingMaster(true);
|
|||
|
|
+ this.messageStore.setAliveReplicaNumInGroup(1);
|
|||
|
|
+
|
|||
|
|
+ MessageExtBrokerInner msg = buildMessage();
|
|||
|
|
+ PutMessageResult result = this.messageStore.putMessage(msg);
|
|||
|
|
+ assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.IN_SYNC_REPLICAS_NOT_ENOUGH);
|
|||
|
|
+ ((RocksDBMessageStore) this.messageStore).getBrokerConfig().setEnableSlaveActingMaster(false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutMsgWhenAdaptiveDegradation() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = ((RocksDBMessageStore) this.messageStore).getMessageStoreConfig();
|
|||
|
|
+ messageStoreConfig.setBrokerRole(BrokerRole.SYNC_MASTER);
|
|||
|
|
+ messageStoreConfig.setTotalReplicas(2);
|
|||
|
|
+ messageStoreConfig.setInSyncReplicas(2);
|
|||
|
|
+ messageStoreConfig.setEnableAutoInSyncReplicas(true);
|
|||
|
|
+ ((RocksDBMessageStore) this.messageStore).getBrokerConfig().setEnableSlaveActingMaster(true);
|
|||
|
|
+ this.messageStore.setAliveReplicaNumInGroup(1);
|
|||
|
|
+
|
|||
|
|
+ MessageExtBrokerInner msg = buildMessage();
|
|||
|
|
+ PutMessageResult result = this.messageStore.putMessage(msg);
|
|||
|
|
+ assertThat(result.getPutMessageStatus()).isEqualTo(PutMessageStatus.PUT_OK);
|
|||
|
|
+ ((RocksDBMessageStore) this.messageStore).getBrokerConfig().setEnableSlaveActingMaster(false);
|
|||
|
|
+ messageStoreConfig.setEnableAutoInSyncReplicas(false);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testGetBulkCommitLogData() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ RocksDBMessageStore defaultMessageStore = (RocksDBMessageStore) messageStore;
|
|||
|
|
+
|
|||
|
|
+ messageBody = new byte[2 * 1024 * 1024];
|
|||
|
|
+
|
|||
|
|
+ for (int i = 0; i < 10; i++) {
|
|||
|
|
+ MessageExtBrokerInner msg1 = buildMessage();
|
|||
|
|
+ messageStore.putMessage(msg1);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ List<SelectMappedBufferResult> bufferResultList = defaultMessageStore.getBulkCommitLogData(0, (int) defaultMessageStore.getMaxPhyOffset());
|
|||
|
|
+ List<MessageExt> msgList = new ArrayList<>();
|
|||
|
|
+ for (SelectMappedBufferResult bufferResult : bufferResultList) {
|
|||
|
|
+ msgList.addAll(MessageDecoder.decodesBatch(bufferResult.getByteBuffer(), true, false, false));
|
|||
|
|
+ bufferResult.release();
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ assertThat(msgList.size()).isEqualTo(10);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testPutLongMessage() throws Exception {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ CommitLog commitLog = ((RocksDBMessageStore) messageStore).getCommitLog();
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = ((RocksDBMessageStore) messageStore).getMessageStoreConfig();
|
|||
|
|
+ MessageExtEncoder.PutMessageThreadLocal putMessageThreadLocal = commitLog.getPutMessageThreadLocal().get();
|
|||
|
|
+
|
|||
|
|
+ //body size, topic size, properties size exactly equal to max size
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[messageStoreConfig.getMaxMessageSize()]);
|
|||
|
|
+ messageExtBrokerInner.setTopic(new String(new byte[127]));
|
|||
|
|
+ messageExtBrokerInner.setPropertiesString(new String(new byte[Short.MAX_VALUE]));
|
|||
|
|
+ PutMessageResult encodeResult1 = putMessageThreadLocal.getEncoder().encode(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(encodeResult1 == null);
|
|||
|
|
+
|
|||
|
|
+ //body size exactly more than max message body size
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[messageStoreConfig.getMaxMessageSize() + 1]);
|
|||
|
|
+ PutMessageResult encodeResult2 = putMessageThreadLocal.getEncoder().encode(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(encodeResult2.getPutMessageStatus() == PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+
|
|||
|
|
+ //body size exactly equal to max message size
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[messageStoreConfig.getMaxMessageSize() + 64 * 1024]);
|
|||
|
|
+ PutMessageResult encodeResult3 = putMessageThreadLocal.getEncoder().encode(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(encodeResult3.getPutMessageStatus() == PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+
|
|||
|
|
+ //message properties length more than properties maxSize
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[messageStoreConfig.getMaxMessageSize()]);
|
|||
|
|
+ messageExtBrokerInner.setPropertiesString(new String(new byte[Short.MAX_VALUE + 1]));
|
|||
|
|
+ PutMessageResult encodeResult4 = putMessageThreadLocal.getEncoder().encode(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(encodeResult4.getPutMessageStatus() == PutMessageStatus.PROPERTIES_SIZE_EXCEEDED);
|
|||
|
|
+
|
|||
|
|
+ //message length more than buffer length capacity
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[messageStoreConfig.getMaxMessageSize()]);
|
|||
|
|
+ messageExtBrokerInner.setTopic(new String(new byte[Short.MAX_VALUE]));
|
|||
|
|
+ messageExtBrokerInner.setPropertiesString(new String(new byte[Short.MAX_VALUE]));
|
|||
|
|
+ PutMessageResult encodeResult5 = putMessageThreadLocal.getEncoder().encode(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(encodeResult5.getPutMessageStatus() == PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testDynamicMaxMessageSize() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageExtBrokerInner messageExtBrokerInner = buildMessage();
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = ((RocksDBMessageStore) messageStore).getMessageStoreConfig();
|
|||
|
|
+ int originMaxMessageSize = messageStoreConfig.getMaxMessageSize();
|
|||
|
|
+
|
|||
|
|
+ messageExtBrokerInner.setBody(new byte[originMaxMessageSize + 10]);
|
|||
|
|
+ PutMessageResult putMessageResult = messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(putMessageResult.getPutMessageStatus() == PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+
|
|||
|
|
+ int newMaxMessageSize = originMaxMessageSize + 10;
|
|||
|
|
+ messageStoreConfig.setMaxMessageSize(newMaxMessageSize);
|
|||
|
|
+ putMessageResult = messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(putMessageResult.getPutMessageStatus() == PutMessageStatus.PUT_OK);
|
|||
|
|
+
|
|||
|
|
+ messageStoreConfig.setMaxMessageSize(10);
|
|||
|
|
+ putMessageResult = messageStore.putMessage(messageExtBrokerInner);
|
|||
|
|
+ assertTrue(putMessageResult.getPutMessageStatus() == PutMessageStatus.MESSAGE_ILLEGAL);
|
|||
|
|
+
|
|||
|
|
+ messageStoreConfig.setMaxMessageSize(originMaxMessageSize);
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testDeleteTopics() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = messageStore.getMessageStoreConfig();
|
|||
|
|
+ ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> consumeQueueTable =
|
|||
|
|
+ ((RocksDBMessageStore) messageStore).getConsumeQueueTable();
|
|||
|
|
+ for (int i = 0; i < 10; i++) {
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> cqTable = new ConcurrentHashMap<>();
|
|||
|
|
+ String topicName = "topic-" + i;
|
|||
|
|
+ for (int j = 0; j < 4; j++) {
|
|||
|
|
+ ConsumeQueue consumeQueue = new ConsumeQueue(topicName, j, messageStoreConfig.getStorePathRootDir(),
|
|||
|
|
+ messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore);
|
|||
|
|
+ cqTable.put(j, consumeQueue);
|
|||
|
|
+ }
|
|||
|
|
+ consumeQueueTable.put(topicName, cqTable);
|
|||
|
|
+ }
|
|||
|
|
+ Assert.assertEquals(consumeQueueTable.size(), 10);
|
|||
|
|
+ HashSet<String> resultSet = Sets.newHashSet("topic-3", "topic-5");
|
|||
|
|
+ messageStore.deleteTopics(Sets.difference(consumeQueueTable.keySet(), resultSet));
|
|||
|
|
+ Assert.assertEquals(consumeQueueTable.size(), 2);
|
|||
|
|
+ Assert.assertEquals(resultSet, consumeQueueTable.keySet());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Test
|
|||
|
|
+ public void testCleanUnusedTopic() {
|
|||
|
|
+ if (notExecuted()) {
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ MessageStoreConfig messageStoreConfig = messageStore.getMessageStoreConfig();
|
|||
|
|
+ ConcurrentMap<String, ConcurrentMap<Integer, ConsumeQueueInterface>> consumeQueueTable =
|
|||
|
|
+ ((RocksDBMessageStore) messageStore).getConsumeQueueTable();
|
|||
|
|
+ for (int i = 0; i < 10; i++) {
|
|||
|
|
+ ConcurrentMap<Integer, ConsumeQueueInterface> cqTable = new ConcurrentHashMap<>();
|
|||
|
|
+ String topicName = "topic-" + i;
|
|||
|
|
+ for (int j = 0; j < 4; j++) {
|
|||
|
|
+ ConsumeQueue consumeQueue = new ConsumeQueue(topicName, j, messageStoreConfig.getStorePathRootDir(),
|
|||
|
|
+ messageStoreConfig.getMappedFileSizeConsumeQueue(), messageStore);
|
|||
|
|
+ cqTable.put(j, consumeQueue);
|
|||
|
|
+ }
|
|||
|
|
+ consumeQueueTable.put(topicName, cqTable);
|
|||
|
|
+ }
|
|||
|
|
+ Assert.assertEquals(consumeQueueTable.size(), 10);
|
|||
|
|
+ HashSet<String> resultSet = Sets.newHashSet("topic-3", "topic-5");
|
|||
|
|
+ messageStore.cleanUnusedTopic(resultSet);
|
|||
|
|
+ Assert.assertEquals(consumeQueueTable.size(), 2);
|
|||
|
|
+ Assert.assertEquals(resultSet, consumeQueueTable.keySet());
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private class MyMessageArrivingListener implements MessageArrivingListener {
|
|||
|
|
+ @Override
|
|||
|
|
+ public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime,
|
|||
|
|
+ byte[] filterBitMap, Map<String, String> properties) {
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ private boolean notExecuted() {
|
|||
|
|
+ return MixAll.isMac();
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
+
|
|||
|
|
+
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java
|
|||
|
|
index b2d99c3ed..17a2b5e19 100644
|
|||
|
|
--- a/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java
|
|||
|
|
@@ -38,11 +38,16 @@ public class StoreTestUtil {
|
|||
|
|
|
|||
|
|
public static boolean isCommitLogAvailable(DefaultMessageStore store) {
|
|||
|
|
try {
|
|||
|
|
+ Field serviceField = null;
|
|||
|
|
+ if (store instanceof RocksDBMessageStore) {
|
|||
|
|
+ serviceField = store.getClass().getSuperclass().getDeclaredField("reputMessageService");
|
|||
|
|
+ } else {
|
|||
|
|
+ serviceField = store.getClass().getDeclaredField("reputMessageService");
|
|||
|
|
+ }
|
|||
|
|
|
|||
|
|
- Field serviceField = store.getClass().getDeclaredField("reputMessageService");
|
|||
|
|
serviceField.setAccessible(true);
|
|||
|
|
DefaultMessageStore.ReputMessageService reputService =
|
|||
|
|
- (DefaultMessageStore.ReputMessageService) serviceField.get(store);
|
|||
|
|
+ (DefaultMessageStore.ReputMessageService) serviceField.get(store);
|
|||
|
|
|
|||
|
|
Method method = DefaultMessageStore.ReputMessageService.class.getDeclaredMethod("isCommitLogAvailable");
|
|||
|
|
method.setAccessible(true);
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/ha/HAServerTest.java b/store/src/test/java/org/apache/rocketmq/store/ha/HAServerTest.java
|
|||
|
|
index 54174ac16..fa8f41dbf 100644
|
|||
|
|
--- a/store/src/test/java/org/apache/rocketmq/store/ha/HAServerTest.java
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/ha/HAServerTest.java
|
|||
|
|
@@ -36,7 +36,7 @@ import org.junit.Before;
|
|||
|
|
import org.junit.Test;
|
|||
|
|
import org.junit.runner.RunWith;
|
|||
|
|
import org.mockito.junit.MockitoJUnitRunner;
|
|||
|
|
-
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
import static org.assertj.core.api.Assertions.assertThat;
|
|||
|
|
import static org.awaitility.Awaitility.await;
|
|||
|
|
import static org.mockito.ArgumentMatchers.anyLong;
|
|||
|
|
@@ -114,7 +114,7 @@ public class HAServerTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void inSyncReplicasNums() throws IOException {
|
|||
|
|
+ public void inSyncReplicasNums() throws IOException, RocksDBException {
|
|||
|
|
DefaultMessageStore messageStore = mockMessageStore();
|
|||
|
|
doReturn(123L).when(messageStore).getMaxPhyOffset();
|
|||
|
|
doReturn(123L).when(messageStore).getMasterFlushedOffset();
|
|||
|
|
@@ -150,7 +150,7 @@ public class HAServerTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void isSlaveOK() throws IOException {
|
|||
|
|
+ public void isSlaveOK() throws IOException, RocksDBException {
|
|||
|
|
DefaultMessageStore messageStore = mockMessageStore();
|
|||
|
|
doReturn(123L).when(messageStore).getMaxPhyOffset();
|
|||
|
|
doReturn(123L).when(messageStore).getMasterFlushedOffset();
|
|||
|
|
@@ -175,7 +175,8 @@ public class HAServerTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void putRequest_SingleAck() throws IOException, ExecutionException, InterruptedException, TimeoutException {
|
|||
|
|
+ public void putRequest_SingleAck()
|
|||
|
|
+ throws IOException, ExecutionException, InterruptedException, TimeoutException, RocksDBException {
|
|||
|
|
CommitLog.GroupCommitRequest request = new CommitLog.GroupCommitRequest(124, 4000, 1);
|
|||
|
|
this.haService.putRequest(request);
|
|||
|
|
|
|||
|
|
@@ -192,7 +193,8 @@ public class HAServerTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void putRequest_MultipleAckAndRequests() throws IOException, ExecutionException, InterruptedException {
|
|||
|
|
+ public void putRequest_MultipleAckAndRequests()
|
|||
|
|
+ throws IOException, ExecutionException, InterruptedException, RocksDBException {
|
|||
|
|
CommitLog.GroupCommitRequest oneAck = new CommitLog.GroupCommitRequest(124, 4000, 2);
|
|||
|
|
this.haService.putRequest(oneAck);
|
|||
|
|
|
|||
|
|
@@ -218,7 +220,7 @@ public class HAServerTest {
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
@Test
|
|||
|
|
- public void getPush2SlaveMaxOffset() throws IOException {
|
|||
|
|
+ public void getPush2SlaveMaxOffset() throws IOException, RocksDBException {
|
|||
|
|
DefaultMessageStore messageStore = mockMessageStore();
|
|||
|
|
doReturn(123L).when(messageStore).getMaxPhyOffset();
|
|||
|
|
doReturn(123L).when(messageStore).getMasterFlushedOffset();
|
|||
|
|
@@ -256,7 +258,7 @@ public class HAServerTest {
|
|||
|
|
this.haClientList.add(haClient);
|
|||
|
|
}
|
|||
|
|
|
|||
|
|
- private DefaultMessageStore mockMessageStore() throws IOException {
|
|||
|
|
+ private DefaultMessageStore mockMessageStore() throws IOException, RocksDBException {
|
|||
|
|
DefaultMessageStore messageStore = mock(DefaultMessageStore.class);
|
|||
|
|
BrokerConfig brokerConfig = mock(BrokerConfig.class);
|
|||
|
|
|
|||
|
|
diff --git a/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java b/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java
|
|||
|
|
index 27dcff141..db5c5af4c 100644
|
|||
|
|
--- a/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java
|
|||
|
|
+++ b/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java
|
|||
|
|
@@ -41,6 +41,7 @@ import org.junit.Assert;
|
|||
|
|
import org.junit.Assume;
|
|||
|
|
import org.junit.Ignore;
|
|||
|
|
import org.junit.Test;
|
|||
|
|
+import org.rocksdb.RocksDBException;
|
|||
|
|
|
|||
|
|
import java.io.File;
|
|||
|
|
import java.net.InetAddress;
|
|||
|
|
@@ -180,7 +181,7 @@ public class AutoSwitchHATest {
|
|||
|
|
|
|||
|
|
private boolean changeMasterAndPutMessage(DefaultMessageStore master, MessageStoreConfig masterConfig,
|
|||
|
|
DefaultMessageStore slave, long slaveId, MessageStoreConfig slaveConfig, int epoch, String masterHaAddress,
|
|||
|
|
- int totalPutMessageNums) {
|
|||
|
|
+ int totalPutMessageNums) throws RocksDBException {
|
|||
|
|
|
|||
|
|
boolean flag = true;
|
|||
|
|
// Change role
|
|||
|
|
diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
index d7d13d61e..edaa5d19f 100644
|
|||
|
|
--- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
+++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java
|
|||
|
|
@@ -16,17 +16,14 @@
|
|||
|
|
*/
|
|||
|
|
package org.apache.rocketmq.tieredstore;
|
|||
|
|
|
|||
|
|
-import com.google.common.base.Stopwatch;
|
|||
|
|
-import io.opentelemetry.api.common.Attributes;
|
|||
|
|
-import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
-import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
-import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
import java.util.List;
|
|||
|
|
import java.util.Set;
|
|||
|
|
import java.util.concurrent.CompletableFuture;
|
|||
|
|
import java.util.concurrent.TimeUnit;
|
|||
|
|
import java.util.function.Supplier;
|
|||
|
|
+
|
|||
|
|
+import com.google.common.base.Stopwatch;
|
|||
|
|
+
|
|||
|
|
import org.apache.commons.lang3.StringUtils;
|
|||
|
|
import org.apache.rocketmq.common.BoundaryType;
|
|||
|
|
import org.apache.rocketmq.common.MixAll;
|
|||
|
|
@@ -55,6 +52,12 @@ import org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsConstant;
|
|||
|
|
import org.apache.rocketmq.tieredstore.metrics.TieredStoreMetricsManager;
|
|||
|
|
import org.apache.rocketmq.tieredstore.util.TieredStoreUtil;
|
|||
|
|
|
|||
|
|
+import io.opentelemetry.api.common.Attributes;
|
|||
|
|
+import io.opentelemetry.api.common.AttributesBuilder;
|
|||
|
|
+import io.opentelemetry.api.metrics.Meter;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.InstrumentSelector;
|
|||
|
|
+import io.opentelemetry.sdk.metrics.ViewBuilder;
|
|||
|
|
+
|
|||
|
|
public class TieredMessageStore extends AbstractPluginMessageStore {
|
|||
|
|
|
|||
|
|
protected static final Logger logger = LoggerFactory.getLogger(TieredStoreUtil.TIERED_STORE_LOGGER_NAME);
|
|||
|
|
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
index 2a7d3fba4..1ecb1fa2c 100644
|
|||
|
|
--- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java
|
|||
|
|
@@ -106,8 +106,8 @@ public class ExportMetadataInRocksDBCommand implements SubCommand {
|
|||
|
|
final Map<String, JSONObject> jsonConfig = new HashMap<>();
|
|||
|
|
final Map<String, JSONObject> configTable = new HashMap<>();
|
|||
|
|
iterateKvStore(kvStore, (key, value) -> {
|
|||
|
|
- final String configKey = new String(key, DataConverter.charset);
|
|||
|
|
- final String configValue = new String(value, DataConverter.charset);
|
|||
|
|
+ final String configKey = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final String configValue = new String(value, DataConverter.CHARSET_UTF8);
|
|||
|
|
final JSONObject jsonObject = JSONObject.parseObject(configValue);
|
|||
|
|
configTable.put(configKey, jsonObject);
|
|||
|
|
}
|
|||
|
|
@@ -120,8 +120,8 @@ public class ExportMetadataInRocksDBCommand implements SubCommand {
|
|||
|
|
} else {
|
|||
|
|
AtomicLong count = new AtomicLong(0);
|
|||
|
|
iterateKvStore(kvStore, (key, value) -> {
|
|||
|
|
- final String configKey = new String(key, DataConverter.charset);
|
|||
|
|
- final String configValue = new String(value, DataConverter.charset);
|
|||
|
|
+ final String configKey = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final String configValue = new String(value, DataConverter.CHARSET_UTF8);
|
|||
|
|
System.out.printf("%d, Key: %s, Value: %s%n", count.incrementAndGet(), configKey, configValue);
|
|||
|
|
});
|
|||
|
|
}
|
|||
|
|
diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
new file mode 100644
|
|||
|
|
index 000000000..b987ad873
|
|||
|
|
--- /dev/null
|
|||
|
|
+++ b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java
|
|||
|
|
@@ -0,0 +1,118 @@
|
|||
|
|
+/*
|
|||
|
|
+ * 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.tools.command.metadata;
|
|||
|
|
+
|
|||
|
|
+import com.alibaba.fastjson.JSONObject;
|
|||
|
|
+import org.apache.commons.cli.CommandLine;
|
|||
|
|
+import org.apache.commons.cli.Option;
|
|||
|
|
+import org.apache.commons.cli.Options;
|
|||
|
|
+import org.apache.commons.lang3.StringUtils;
|
|||
|
|
+import org.apache.rocketmq.common.config.RocksDBConfigManager;
|
|||
|
|
+import org.apache.rocketmq.common.utils.DataConverter;
|
|||
|
|
+import org.apache.rocketmq.remoting.RPCHook;
|
|||
|
|
+import org.apache.rocketmq.tools.command.SubCommand;
|
|||
|
|
+import org.apache.rocketmq.tools.command.SubCommandException;
|
|||
|
|
+
|
|||
|
|
+import java.io.File;
|
|||
|
|
+import java.util.HashMap;
|
|||
|
|
+import java.util.Map;
|
|||
|
|
+
|
|||
|
|
+public class RocksDBConfigToJsonCommand implements SubCommand {
|
|||
|
|
+ private static final String TOPICS_JSON_CONFIG = "topics";
|
|||
|
|
+ private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups";
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandName() {
|
|||
|
|
+ return "rocksDBConfigToJson";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public String commandDesc() {
|
|||
|
|
+ return "Convert RocksDB kv config (topics/subscriptionGroups) to json";
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public Options buildCommandlineOptions(Options options) {
|
|||
|
|
+ Option pathOption = new Option("p", "path", true,
|
|||
|
|
+ "Absolute path to the metadata directory");
|
|||
|
|
+ pathOption.setRequired(true);
|
|||
|
|
+ options.addOption(pathOption);
|
|||
|
|
+
|
|||
|
|
+ Option configTypeOption = new Option("t", "configType", true, "Name of kv config, e.g. " +
|
|||
|
|
+ "topics/subscriptionGroups");
|
|||
|
|
+ configTypeOption.setRequired(true);
|
|||
|
|
+ options.addOption(configTypeOption);
|
|||
|
|
+
|
|||
|
|
+ return options;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ @Override
|
|||
|
|
+ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException {
|
|||
|
|
+ String path = commandLine.getOptionValue("path").trim();
|
|||
|
|
+ if (StringUtils.isEmpty(path) || !new File(path).exists()) {
|
|||
|
|
+ System.out.print("Rocksdb path is invalid.\n");
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+
|
|||
|
|
+ String configType = commandLine.getOptionValue("configType").trim().toLowerCase();
|
|||
|
|
+
|
|||
|
|
+ final long memTableFlushInterval = 60 * 60 * 1000L;
|
|||
|
|
+ RocksDBConfigManager kvConfigManager = new RocksDBConfigManager(memTableFlushInterval);
|
|||
|
|
+ try {
|
|||
|
|
+ if (TOPICS_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+ // for topics.json
|
|||
|
|
+ final Map<String, JSONObject> topicsJsonConfig = new HashMap<>();
|
|||
|
|
+ final Map<String, JSONObject> topicConfigTable = new HashMap<>();
|
|||
|
|
+ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
+ final String topic = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final String topicConfig = new String(value, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final JSONObject jsonObject = JSONObject.parseObject(topicConfig);
|
|||
|
|
+ topicConfigTable.put(topic, jsonObject);
|
|||
|
|
+ });
|
|||
|
|
+
|
|||
|
|
+ if (isLoad) {
|
|||
|
|
+ topicsJsonConfig.put("topicConfigTable", (JSONObject) JSONObject.toJSON(topicConfigTable));
|
|||
|
|
+ final String topicsJsonStr = JSONObject.toJSONString(topicsJsonConfig, true);
|
|||
|
|
+ System.out.print(topicsJsonStr + "\n");
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ if (SUBSCRIPTION_GROUP_JSON_CONFIG.toLowerCase().equals(configType)) {
|
|||
|
|
+ // for subscriptionGroup.json
|
|||
|
|
+ final Map<String, JSONObject> subscriptionGroupJsonConfig = new HashMap<>();
|
|||
|
|
+ final Map<String, JSONObject> subscriptionGroupTable = new HashMap<>();
|
|||
|
|
+ boolean isLoad = kvConfigManager.load(path, (key, value) -> {
|
|||
|
|
+ final String subscriptionGroup = new String(key, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final String subscriptionGroupConfig = new String(value, DataConverter.CHARSET_UTF8);
|
|||
|
|
+ final JSONObject jsonObject = JSONObject.parseObject(subscriptionGroupConfig);
|
|||
|
|
+ subscriptionGroupTable.put(subscriptionGroup, jsonObject);
|
|||
|
|
+ });
|
|||
|
|
+
|
|||
|
|
+ if (isLoad) {
|
|||
|
|
+ subscriptionGroupJsonConfig.put("subscriptionGroupTable",
|
|||
|
|
+ (JSONObject) JSONObject.toJSON(subscriptionGroupTable));
|
|||
|
|
+ final String subscriptionGroupJsonStr = JSONObject.toJSONString(subscriptionGroupJsonConfig, true);
|
|||
|
|
+ System.out.print(subscriptionGroupJsonStr + "\n");
|
|||
|
|
+ return;
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+ System.out.print("Config type was not recognized, configType=" + configType + "\n");
|
|||
|
|
+ } finally {
|
|||
|
|
+ kvConfigManager.stop();
|
|||
|
|
+ }
|
|||
|
|
+ }
|
|||
|
|
+}
|
|||
|
|
--
|
|||
|
|
2.32.0.windows.2
|
|||
|
|
|