diff --git a/.gitignore b/.gitignore index 8abdfd8fd6e..264f48d0dd7 100644 --- a/.gitignore +++ b/.gitignore @@ -10,6 +10,6 @@ devenv *.versionsBackup !NOTICE-BIN !LICENSE-BIN -.DS_Store -localbin -nohup.out +.DS_Store +localbin +nohup.out diff --git a/README.md b/README.md index 686ba462945..f964f46fc99 100644 --- a/README.md +++ b/README.md @@ -21,6 +21,8 @@ It offers a variety of features: * Various message filter mechanics such as SQL and Tag * Docker images for isolated testing and cloud isolated clusters * Feature-rich administrative dashboard for configuration, metrics and monitoring +* Access control list +* Message trace ---------- @@ -46,3 +48,5 @@ We always welcome new contributions, whether for trivial cleanups, big new featu ---------- ## License [Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html) Copyright (C) Apache Software Foundation + + diff --git a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java index 34bb1b439e5..bd50e1292ad 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainAccessValidator.java @@ -52,6 +52,11 @@ public AccessResource parse(RemotingCommand request, String remoteAddr) { } else { accessResource.setWhiteRemoteAddress(remoteAddr); } + + if (request.getExtFields() == null) { + throw new AclException("request's extFields value is null"); + } + accessResource.setRequestCode(request.getCode()); accessResource.setAccessKey(request.getExtFields().get(SessionCredentials.ACCESS_KEY)); accessResource.setSignature(request.getExtFields().get(SessionCredentials.SIGNATURE)); diff --git a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionLoader.java b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionLoader.java index 1da7380b6d9..9148422ff10 100644 --- a/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionLoader.java +++ b/acl/src/main/java/org/apache/rocketmq/acl/plain/PlainPermissionLoader.java @@ -130,8 +130,8 @@ void checkPerm(PlainAccessResource needCheckedAccess, PlainAccessResource ownedA if (!ownedPermMap.containsKey(resource)) { // Check the default perm - byte ownedPerm = isGroup ? needCheckedAccess.getDefaultGroupPerm() : - needCheckedAccess.getDefaultTopicPerm(); + byte ownedPerm = isGroup ? ownedAccess.getDefaultGroupPerm() : + ownedAccess.getDefaultTopicPerm(); if (!Permission.checkPermission(neededPerm, ownedPerm)) { throw new AclException(String.format("No default permission for %s", PlainAccessResource.printStr(resource, isGroup))); } 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 16e770206cb..e7b6f2d68af 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 @@ -43,6 +43,7 @@ public class PlainAccessValidatorTest { @Before public void init() { System.setProperty("rocketmq.home.dir", "src/test/resources"); + System.setProperty("rocketmq.acl.plain.file", "/conf/plain_acl.yml"); plainAccessValidator = new PlainAccessValidator(); sessionCredentials = new SessionCredentials(); sessionCredentials.setAccessKey("RocketMQ"); @@ -115,6 +116,22 @@ public void validateSendMessageV2Test() { plainAccessValidator.validate(accessResource); } + @Test(expected = AclException.class) + public void validateForAdminCommandWithOutAclRPCHook() { + RemotingCommand consumerOffsetAdminRequest = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_CONSUMER_OFFSET, null); + plainAccessValidator.parse(consumerOffsetAdminRequest, "192.168.0.1:9876"); + + RemotingCommand subscriptionGroupAdminRequest = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_SUBSCRIPTIONGROUP_CONFIG, null); + plainAccessValidator.parse(subscriptionGroupAdminRequest, "192.168.0.1:9876"); + + RemotingCommand delayOffsetAdminRequest = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_DELAY_OFFSET, null); + plainAccessValidator.parse(delayOffsetAdminRequest, "192.168.0.1:9876"); + + RemotingCommand allTopicConfigAdminRequest = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_TOPIC_CONFIG, null); + plainAccessValidator.parse(allTopicConfigAdminRequest, "192.168.0.1:9876"); + + } + @Test public void validatePullMessageTest() { PullMessageRequestHeader pullMessageRequestHeader=new PullMessageRequestHeader(); diff --git a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionLoaderTest.java b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionLoaderTest.java index ebbc4fd260c..575c9018743 100644 --- a/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionLoaderTest.java +++ b/acl/src/test/java/org/apache/rocketmq/acl/plain/PlainPermissionLoaderTest.java @@ -158,10 +158,10 @@ public void checkPerm() { } @Test(expected = AclException.class) - public void checkErrorPerm() { + public void checkErrorPermDefaultValueNotMatch() { plainAccessResource = new PlainAccessResource(); - plainAccessResource.addResourceAndPerm("topicF", Permission.SUB); + plainAccessResource.addResourceAndPerm("topicF", Permission.PUB); plainPermissionLoader.checkPerm(plainAccessResource, SUBPlainAccessResource); } @Test(expected = AclException.class) 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 c3431ca41e5..427f8611890 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -27,6 +27,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; @@ -39,6 +40,7 @@ import org.apache.rocketmq.broker.client.ProducerManager; import org.apache.rocketmq.broker.client.net.Broker2Client; import org.apache.rocketmq.broker.client.rebalance.RebalanceLockManager; +import org.apache.rocketmq.broker.dledger.DLedgerRoleChangeHandler; import org.apache.rocketmq.broker.filter.CommitLogDispatcherCalcBitMap; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.broker.filtersrv.FilterServerManager; @@ -99,6 +101,7 @@ import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.dledger.DLedgerCommitLog; import org.apache.rocketmq.store.stats.BrokerStats; import org.apache.rocketmq.store.stats.BrokerStatsManager; @@ -158,6 +161,7 @@ public class BrokerController { private TransactionalMessageCheckService transactionalMessageCheckService; private TransactionalMessageService transactionalMessageService; private AbstractTransactionalMessageCheckListener transactionalMessageCheckListener; + private Future slaveSyncFuture; public BrokerController( @@ -234,6 +238,10 @@ public boolean initialize() throws CloneNotSupportedException { this.messageStore = new DefaultMessageStore(this.messageStoreConfig, this.brokerStatsManager, this.messageArrivingListener, this.brokerConfig); + if (messageStoreConfig.isEnableDLegerCommitLog()) { + DLedgerRoleChangeHandler roleChangeHandler = new DLedgerRoleChangeHandler(this, (DefaultMessageStore) messageStore); + ((DLedgerCommitLog)((DefaultMessageStore) messageStore).getCommitLog()).getdLedgerServer().getdLedgerLeaderElector().addRoleChangeHandler(roleChangeHandler); + } this.brokerStats = new BrokerStats((DefaultMessageStore) this.messageStore); //load plugin MessageStorePluginContext context = new MessageStorePluginContext(messageStoreConfig, brokerStatsManager, messageArrivingListener, brokerConfig); @@ -396,37 +404,26 @@ public void run() { }, 1000 * 10, 1000 * 60 * 2, TimeUnit.MILLISECONDS); } - if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) { - if (this.messageStoreConfig.getHaMasterAddress() != null && this.messageStoreConfig.getHaMasterAddress().length() >= 6) { - this.messageStore.updateHaMasterAddress(this.messageStoreConfig.getHaMasterAddress()); - this.updateMasterHAServerAddrPeriodically = false; - } else { - this.updateMasterHAServerAddrPeriodically = true; - } - - this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { - - @Override - public void run() { - try { - BrokerController.this.slaveSynchronize.syncAll(); - } catch (Throwable e) { - log.error("ScheduledTask syncAll slave exception", e); - } + if (!messageStoreConfig.isEnableDLegerCommitLog()) { + if (BrokerRole.SLAVE == this.messageStoreConfig.getBrokerRole()) { + if (this.messageStoreConfig.getHaMasterAddress() != null && this.messageStoreConfig.getHaMasterAddress().length() >= 6) { + this.messageStore.updateHaMasterAddress(this.messageStoreConfig.getHaMasterAddress()); + this.updateMasterHAServerAddrPeriodically = false; + } else { + this.updateMasterHAServerAddrPeriodically = true; } - }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS); - } else { - this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { - - @Override - public void run() { - try { - BrokerController.this.printMasterAndSlaveDiff(); - } catch (Throwable e) { - log.error("schedule printMasterAndSlaveDiff error.", e); + } else { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + BrokerController.this.printMasterAndSlaveDiff(); + } catch (Throwable e) { + log.error("schedule printMasterAndSlaveDiff error.", e); + } } - } - }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS); + }, 1000 * 10, 1000 * 60, TimeUnit.MILLISECONDS); + } } if (TlsSystemConfig.tlsMode != TlsMode.DISABLED) { @@ -855,6 +852,13 @@ public void start() throws Exception { this.filterServerManager.start(); } + if (!messageStoreConfig.isEnableDLegerCommitLog()) { + startProcessorByHa(messageStoreConfig.getBrokerRole()); + handleSlaveSynchronize(messageStoreConfig.getBrokerRole()); + } + + + this.registerBrokerAll(true, false, true); this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @@ -877,12 +881,7 @@ public void run() { this.brokerFastFailure.start(); } - if (BrokerRole.SLAVE != messageStoreConfig.getBrokerRole()) { - if (this.transactionalMessageCheckService != null) { - log.info("Start transaction service!"); - this.transactionalMessageCheckService.start(); - } - } + } public synchronized void registerIncrementBrokerData(TopicConfig topicConfig, DataVersion dataVersion) { @@ -1101,4 +1100,118 @@ public BlockingQueue getEndTransactionThreadPoolQueue() { return endTransactionThreadPoolQueue; } + + + + private void handleSlaveSynchronize(BrokerRole role) { + if (role == BrokerRole.SLAVE) { + if (null != slaveSyncFuture) { + slaveSyncFuture.cancel(false); + } + this.slaveSynchronize.setMasterAddr(null); + slaveSyncFuture = this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + try { + BrokerController.this.slaveSynchronize.syncAll(); + } + catch (Throwable e) { + log.error("ScheduledTask SlaveSynchronize syncAll error.", e); + } + } + }, 1000 * 3, 1000 * 10, TimeUnit.MILLISECONDS); + } else { + //handle the slave synchronise + if (null != slaveSyncFuture) { + slaveSyncFuture.cancel(false); + } + this.slaveSynchronize.setMasterAddr(null); + } + } + + public void changeToSlave(int brokerId) { + log.info("Begin to change to slave brokerName={} brokerId={}", brokerConfig.getBrokerName(), brokerId); + + //change the role + brokerConfig.setBrokerId(brokerId == 0 ? 1 : brokerId); //TO DO check + messageStoreConfig.setBrokerRole(BrokerRole.SLAVE); + + //handle the scheduled service + try { + this.messageStore.handleScheduleMessageService(BrokerRole.SLAVE); + } catch (Throwable t) { + log.error("[MONITOR] handleScheduleMessageService failed when changing to slave", t); + } + + //handle the transactional service + try { + this.shutdownProcessorByHa(); + } catch (Throwable t) { + log.error("[MONITOR] shutdownProcessorByHa failed when changing to slave", t); + } + + //handle the slave synchronise + handleSlaveSynchronize(BrokerRole.SLAVE); + + try { + this.registerBrokerAll(true, true, brokerConfig.isForceRegister()); + } catch (Throwable ignored) { + + } + log.info("Finish to change to slave brokerName={} brokerId={}", brokerConfig.getBrokerName(), brokerId); + } + + + + public void changeToMaster(BrokerRole role) { + if (role == BrokerRole.SLAVE) { + return; + } + log.info("Begin to change to master brokerName={}", brokerConfig.getBrokerName()); + + //handle the slave synchronise + handleSlaveSynchronize(role); + + //handle the scheduled service + try { + this.messageStore.handleScheduleMessageService(role); + } catch (Throwable t) { + log.error("[MONITOR] handleScheduleMessageService failed when changing to master", t); + } + + //handle the transactional service + try { + this.startProcessorByHa(BrokerRole.SYNC_MASTER); + } catch (Throwable t) { + log.error("[MONITOR] startProcessorByHa failed when changing to master", t); + } + + //if the operations above are totally successful, we change to master + brokerConfig.setBrokerId(0); //TO DO check + messageStoreConfig.setBrokerRole(role); + + try { + this.registerBrokerAll(true, true, brokerConfig.isForceRegister()); + } catch (Throwable ignored) { + + } + log.info("Finish to change to master brokerName={}", brokerConfig.getBrokerName()); + } + + private void startProcessorByHa(BrokerRole role) { + if (BrokerRole.SLAVE != role) { + if (this.transactionalMessageCheckService != null) { + this.transactionalMessageCheckService.start(); + } + } + } + + private void shutdownProcessorByHa() { + if (this.transactionalMessageCheckService != null) { + this.transactionalMessageCheckService.shutdown(true); + } + } + + + } 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 new file mode 100644 index 00000000000..09bf10c327d --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/dledger/DLedgerRoleChangeHandler.java @@ -0,0 +1,104 @@ +/* + * 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.dledger; + +import io.openmessaging.storage.dledger.DLedgerLeaderElector; +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 org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.config.BrokerRole; +import org.apache.rocketmq.store.dledger.DLedgerCommitLog; + +public class DLedgerRoleChangeHandler implements DLedgerLeaderElector.RoleChangeHandler { + + private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); + private ExecutorService executorService = Executors.newSingleThreadExecutor(new ThreadFactoryImpl("DLegerRoleChangeHandler_")); + private BrokerController brokerController; + private DefaultMessageStore messageStore; + private DLedgerCommitLog dLedgerCommitLog; + private DLedgerServer dLegerServer; + public DLedgerRoleChangeHandler(BrokerController brokerController, DefaultMessageStore messageStore) { + this.brokerController = brokerController; + this.messageStore = messageStore; + this.dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog(); + this.dLegerServer = dLedgerCommitLog.getdLedgerServer(); + } + + @Override public void handle(long term, MemberState.Role role) { + Runnable runnable = new Runnable() { + @Override public void run() { + long start = System.currentTimeMillis(); + try { + boolean succ = true; + log.info("Begin handling broker role change term={} role={} currStoreRole={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole()); + switch (role) { + case CANDIDATE: + if (messageStore.getMessageStoreConfig().getBrokerRole() != BrokerRole.SLAVE) { + brokerController.changeToSlave(dLedgerCommitLog.getId()); + } + break; + case FOLLOWER: + brokerController.changeToSlave(dLedgerCommitLog.getId()); + break; + case LEADER: + while (true) { + if (!dLegerServer.getMemberState().isLeader()) { + succ = false; + break; + } + if (dLegerServer.getdLedgerStore().getLedgerEndIndex() == -1) { + break; + } + if (dLegerServer.getdLedgerStore().getLedgerEndIndex() == dLegerServer.getdLedgerStore().getCommittedIndex() + && messageStore.dispatchBehindBytes() == 0) { + break; + } + Thread.sleep(100); + } + if (succ) { + messageStore.recoverTopicQueueTable(); + brokerController.changeToMaster(BrokerRole.SYNC_MASTER); + } + break; + default: + break; + } + log.info("Finish handling broker role change succ={} term={} role={} currStoreRole={} cost={}", succ, term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start)); + } catch (Throwable t) { + log.info("[MONITOR]Failed handling broker role change term={} role={} currStoreRole={} cost={}", term, role, messageStore.getMessageStoreConfig().getBrokerRole(), DLedgerUtils.elapsed(start), t); + } + } + }; + executorService.submit(runnable); + } + + @Override public void startup() { + + } + + @Override public void shutdown() { + executorService.shutdown(); + } +} 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 4dee01cbff0..6caa2358e32 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 @@ -152,7 +152,7 @@ public void run() { registerBrokerResultList.add(result); } - log.info("register broker to name server {} OK", namesrvAddr); + log.info("register broker[{}]to name server {} OK", brokerId, namesrvAddr); } catch (Exception e) { log.warn("registerBroker Exception, {}", namesrvAddr, e); } finally { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java index b5e608558cb..971237929ab 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/ClientManageProcessor.java @@ -23,7 +23,6 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.constant.PermName; import org.apache.rocketmq.common.filter.ExpressionType; -import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.common.protocol.RequestCode; import org.apache.rocketmq.common.protocol.ResponseCode; import org.apache.rocketmq.common.protocol.body.CheckClientRequestBody; @@ -36,6 +35,7 @@ import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.common.sysflag.TopicSysFlag; import org.apache.rocketmq.filter.FilterFactory; +import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; import org.apache.rocketmq.remoting.exception.RemotingCommandException; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/slave/SlaveSynchronize.java b/broker/src/main/java/org/apache/rocketmq/broker/slave/SlaveSynchronize.java index 643a812fa02..7b5e5645a0b 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/slave/SlaveSynchronize.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/slave/SlaveSynchronize.java @@ -54,7 +54,7 @@ public void syncAll() { private void syncTopicConfig() { String masterAddrBak = this.masterAddr; - if (masterAddrBak != null) { + if (masterAddrBak != null && !masterAddrBak.equals(brokerController.getBrokerAddr())) { try { TopicConfigSerializeWrapper topicWrapper = this.brokerController.getBrokerOuterAPI().getAllTopicConfig(masterAddrBak); @@ -78,7 +78,7 @@ private void syncTopicConfig() { private void syncConsumerOffset() { String masterAddrBak = this.masterAddr; - if (masterAddrBak != null) { + if (masterAddrBak != null && !masterAddrBak.equals(brokerController.getBrokerAddr())) { try { ConsumerOffsetSerializeWrapper offsetWrapper = this.brokerController.getBrokerOuterAPI().getAllConsumerOffset(masterAddrBak); @@ -94,7 +94,7 @@ private void syncConsumerOffset() { private void syncDelayOffset() { String masterAddrBak = this.masterAddr; - if (masterAddrBak != null) { + if (masterAddrBak != null && !masterAddrBak.equals(brokerController.getBrokerAddr())) { try { String delayOffset = this.brokerController.getBrokerOuterAPI().getAllDelayOffset(masterAddrBak); @@ -118,7 +118,7 @@ private void syncDelayOffset() { private void syncSubscriptionGroupConfig() { String masterAddrBak = this.masterAddr; - if (masterAddrBak != null) { + if (masterAddrBak != null && !masterAddrBak.equals(brokerController.getBrokerAddr())) { try { SubscriptionGroupWrapper subscriptionWrapper = this.brokerController.getBrokerOuterAPI() diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionalMessageCheckService.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionalMessageCheckService.java index 5d515d6f11b..a2cc0baa9e7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionalMessageCheckService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionalMessageCheckService.java @@ -22,36 +22,15 @@ import org.apache.rocketmq.logging.InternalLogger; import org.apache.rocketmq.logging.InternalLoggerFactory; -import java.util.concurrent.atomic.AtomicBoolean; - public class TransactionalMessageCheckService extends ServiceThread { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.TRANSACTION_LOGGER_NAME); private BrokerController brokerController; - private final AtomicBoolean started = new AtomicBoolean(false); - public TransactionalMessageCheckService(BrokerController brokerController) { this.brokerController = brokerController; } - @Override - public void start() { - if (started.compareAndSet(false, true)) { - super.start(); - this.brokerController.getTransactionalMessageService().open(); - } - } - - @Override - public void shutdown(boolean interrupt) { - if (started.compareAndSet(true, false)) { - super.shutdown(interrupt); - this.brokerController.getTransactionalMessageService().close(); - this.brokerController.getTransactionalMessageCheckListener().shutDown(); - } - } - @Override public String getServiceName() { return TransactionalMessageCheckService.class.getSimpleName(); 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 1c227af1502..e1549b15177 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 @@ -100,7 +100,7 @@ private boolean putBackHalfMsgQueue(MessageExt msgExt, long offset) { msgExt.setCommitLogOffset( putMessageResult.getAppendMessageResult().getWroteOffset()); msgExt.setMsgId(putMessageResult.getAppendMessageResult().getMsgId()); - log.info( + log.debug( "Send check message, the offset={} restored in queueOffset={} " + "commitLogOffset={} " + "newMsgId={} realMsgId={} topic={}", @@ -127,7 +127,7 @@ public void check(long transactionTimeout, int transactionCheckMax, log.warn("The queue of topic is empty :" + topic); return; } - log.info("Check topic={}, queues={}", topic, msgQueues); + log.debug("Check topic={}, queues={}", topic, msgQueues); for (MessageQueue messageQueue : msgQueues) { long startTime = System.currentTimeMillis(); MessageQueue opQueue = getOpQueue(messageQueue); @@ -168,7 +168,7 @@ public void check(long transactionTimeout, int transactionCheckMax, break; } if (getResult.getPullResult().getPullStatus() == PullStatus.NO_NEW_MSG) { - log.info("No new msg, the miss offset={} in={}, continue check={}, pull result={}", i, + log.debug("No new msg, the miss offset={} in={}, continue check={}, pull result={}", i, messageQueue, getMessageNullCount, getResult.getPullResult()); break; } else { @@ -187,7 +187,7 @@ public void check(long transactionTimeout, int transactionCheckMax, continue; } if (msgExt.getStoreTimestamp() >= startTime) { - log.info("Fresh stored. the miss offset={}, check it later, store={}", i, + log.debug("Fresh stored. the miss offset={}, check it later, store={}", i, new Date(msgExt.getStoreTimestamp())); break; } @@ -206,7 +206,7 @@ public void check(long transactionTimeout, int transactionCheckMax, } } else { if ((0 <= valueOfCurrentMinusBorn) && (valueOfCurrentMinusBorn < checkImmunityTime)) { - log.info("New arrived, the miss offset={}, check it later checkImmunity={}, born={}", i, + log.debug("New arrived, the miss offset={}, check it later checkImmunity={}, born={}", i, checkImmunityTime, new Date(msgExt.getBornTimestamp())); break; } 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 80347d1052e..57d2eda1869 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 @@ -548,10 +548,10 @@ private void sendHeartbeatToAllBroker() { } } catch (Exception e) { if (this.isBrokerInNameServer(addr)) { - log.info("send heart beat to broker[{} {} {}] failed", brokerName, id, addr); + log.info("send heart beat to broker[{} {} {}] failed", brokerName, id, addr, e); } else { log.info("send heart beat to broker[{} {} {}] exception, because the broker not up, forget it", brokerName, - id, addr); + id, addr, e); } } } @@ -1046,20 +1046,8 @@ public int findBrokerVersion(String brokerName, String brokerAddr) { if (this.brokerVersionTable.get(brokerName).containsKey(brokerAddr)) { return this.brokerVersionTable.get(brokerName).get(brokerAddr); } - } else { - HeartbeatData heartbeatData = prepareHeartbeatData(); - try { - int version = this.mQClientAPIImpl.sendHearbeat(brokerAddr, heartbeatData, 3000); - return version; - } catch (Exception e) { - if (this.isBrokerInNameServer(brokerAddr)) { - log.info("send heart beat to broker[{} {}] failed", brokerName, brokerAddr); - } else { - log.info("send heart beat to broker[{} {}] exception, because the broker not up, forget it", brokerName, - brokerAddr); - } - } } + //To do need to fresh the version return 0; } 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 90f4f787635..6b8fdbd73d4 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 @@ -236,6 +236,7 @@ public void shutdown(final boolean shutdownFactory) { break; case RUNNING: this.mQClientFactory.unregisterProducer(this.defaultMQProducer.getProducerGroup()); + this.defaultAsyncSenderExecutor.shutdown(); if (shutdownFactory) { this.mQClientFactory.shutdown(); } diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java new file mode 100644 index 00000000000..921743c6be8 --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ConsumeMessageConcurrentlyServiceTest.java @@ -0,0 +1,213 @@ +/* + * 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.impl.consumer; + +import org.apache.rocketmq.client.consumer.*; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.impl.CommunicationMode; +import org.apache.rocketmq.client.impl.FindBrokerResult; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.client.stat.ConsumerStatsManager; +import org.apache.rocketmq.common.message.MessageClientExt; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.common.protocol.header.PullMessageRequestHeader; +import org.apache.rocketmq.common.stats.StatsItem; +import org.apache.rocketmq.common.stats.StatsItemSet; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Spy; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; + +import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.*; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class ConsumeMessageConcurrentlyServiceTest { + private String consumerGroup; + private String topic = "FooBar"; + private String brokerName = "BrokerA"; + private MQClientInstance mQClientFactory; + + @Mock + private MQClientAPIImpl mQClientAPIImpl; + private PullAPIWrapper pullAPIWrapper; + private RebalancePushImpl rebalancePushImpl; + private DefaultMQPushConsumer pushConsumer; + + @Before + public void init() throws Exception { + consumerGroup = "FooBarGroup" + System.currentTimeMillis(); + pushConsumer = new DefaultMQPushConsumer(consumerGroup); + pushConsumer.setNamesrvAddr("127.0.0.1:9876"); + pushConsumer.setPullInterval(60 * 1000); + + pushConsumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + + DefaultMQPushConsumerImpl pushConsumerImpl = pushConsumer.getDefaultMQPushConsumerImpl(); + rebalancePushImpl = spy(new RebalancePushImpl(pushConsumer.getDefaultMQPushConsumerImpl())); + Field field = DefaultMQPushConsumerImpl.class.getDeclaredField("rebalanceImpl"); + field.setAccessible(true); + field.set(pushConsumerImpl, rebalancePushImpl); + pushConsumer.subscribe(topic, "*"); + pushConsumer.start(); + + mQClientFactory = spy(pushConsumerImpl.getmQClientFactory()); + field = DefaultMQPushConsumerImpl.class.getDeclaredField("mQClientFactory"); + field.setAccessible(true); + field.set(pushConsumerImpl, mQClientFactory); + + field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); + field.setAccessible(true); + field.set(mQClientFactory, mQClientAPIImpl); + + pullAPIWrapper = spy(new PullAPIWrapper(mQClientFactory, consumerGroup, false)); + field = DefaultMQPushConsumerImpl.class.getDeclaredField("pullAPIWrapper"); + field.setAccessible(true); + field.set(pushConsumerImpl, pullAPIWrapper); + + pushConsumer.getDefaultMQPushConsumerImpl().getRebalanceImpl().setmQClientFactory(mQClientFactory); + mQClientFactory.registerConsumer(consumerGroup, pushConsumerImpl); + + when(mQClientFactory.getMQClientAPIImpl().pullMessage(anyString(), any(PullMessageRequestHeader.class), + anyLong(), any(CommunicationMode.class), nullable(PullCallback.class))) + .thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock mock) throws Throwable { + PullMessageRequestHeader requestHeader = mock.getArgument(1); + MessageClientExt messageClientExt = new MessageClientExt(); + messageClientExt.setTopic(topic); + messageClientExt.setQueueId(0); + messageClientExt.setMsgId("123"); + messageClientExt.setBody(new byte[] {'a'}); + messageClientExt.setOffsetMsgId("234"); + messageClientExt.setBornHost(new InetSocketAddress(8080)); + messageClientExt.setStoreHost(new InetSocketAddress(8080)); + PullResult pullResult = createPullResult(requestHeader, PullStatus.FOUND, Collections.singletonList(messageClientExt)); + ((PullCallback) mock.getArgument(4)).onSuccess(pullResult); + return pullResult; + } + }); + + doReturn(new FindBrokerResult("127.0.0.1:10912", false)).when(mQClientFactory).findBrokerAddressInSubscribe(anyString(), anyLong(), anyBoolean()); + Set messageQueueSet = new HashSet(); + messageQueueSet.add(createPullRequest().getMessageQueue()); + pushConsumer.getDefaultMQPushConsumerImpl().updateTopicSubscribeInfo(topic, messageQueueSet); + } + + @Ignore + @Test + public void testPullMessage_ConsumeSuccess() throws InterruptedException, RemotingException, MQBrokerException, NoSuchFieldException,Exception { + final CountDownLatch countDownLatch = new CountDownLatch(1); + final MessageExt[] messageExts = new MessageExt[1]; + + ConsumeMessageConcurrentlyService normalServie = new ConsumeMessageConcurrentlyService(pushConsumer.getDefaultMQPushConsumerImpl(), new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, + ConsumeConcurrentlyContext context) { + messageExts[0] = msgs.get(0); + countDownLatch.countDown(); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + pushConsumer.getDefaultMQPushConsumerImpl().setConsumeMessageService(normalServie); + + PullMessageService pullMessageService = mQClientFactory.getPullMessageService(); + pullMessageService.executePullRequestImmediately(createPullRequest()); + countDownLatch.await(); + + Thread.sleep(1000); + + org.apache.rocketmq.common.protocol.body.ConsumeStatus stats = normalServie.getConsumerStatsManager().consumeStatus(pushConsumer.getDefaultMQPushConsumerImpl().groupName(),topic); + + ConsumerStatsManager mgr = normalServie.getConsumerStatsManager(); + + Field statItmeSetField = mgr.getClass().getDeclaredField("topicAndGroupConsumeOKTPS"); + statItmeSetField.setAccessible(true); + + StatsItemSet itemSet = (StatsItemSet)statItmeSetField.get(mgr); + StatsItem item = itemSet.getAndCreateStatsItem(topic + "@" + pushConsumer.getDefaultMQPushConsumerImpl().groupName()); + + assertThat(item.getValue().get()).isGreaterThan(0L); + assertThat(messageExts[0].getTopic()).isEqualTo(topic); + assertThat(messageExts[0].getBody()).isEqualTo(new byte[] {'a'}); + } + + @After + public void terminate() { + pushConsumer.shutdown(); + } + + private PullRequest createPullRequest() { + PullRequest pullRequest = new PullRequest(); + pullRequest.setConsumerGroup(consumerGroup); + pullRequest.setNextOffset(1024); + + MessageQueue messageQueue = new MessageQueue(); + messageQueue.setBrokerName(brokerName); + messageQueue.setQueueId(0); + messageQueue.setTopic(topic); + pullRequest.setMessageQueue(messageQueue); + ProcessQueue processQueue = new ProcessQueue(); + processQueue.setLocked(true); + processQueue.setLastLockTimestamp(System.currentTimeMillis()); + pullRequest.setProcessQueue(processQueue); + + return pullRequest; + } + + private PullResultExt createPullResult(PullMessageRequestHeader requestHeader, PullStatus pullStatus, + List messageExtList) throws Exception { + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + for (MessageExt messageExt : messageExtList) { + outputStream.write(MessageDecoder.encode(messageExt, false)); + } + return new PullResultExt(pullStatus, requestHeader.getQueueOffset() + messageExtList.size(), 123, 2048, messageExtList, 0, outputStream.toByteArray()); + } +} diff --git a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java index 53ca3290826..ae46e78fa62 100644 --- a/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java +++ b/common/src/main/java/org/apache/rocketmq/common/ServiceThread.java @@ -27,18 +27,29 @@ public abstract class ServiceThread implements Runnable { private static final long JOIN_TIME = 90 * 1000; - protected final Thread thread; + private Thread thread; protected final CountDownLatch2 waitPoint = new CountDownLatch2(1); protected volatile AtomicBoolean hasNotified = new AtomicBoolean(false); protected volatile boolean stopped = false; + protected boolean isDaemon = false; + + //Make it able to restart the thread + private final AtomicBoolean started = new AtomicBoolean(false); public ServiceThread() { - this.thread = new Thread(this, this.getServiceName()); + } public abstract String getServiceName(); public void start() { + log.info("Try to start service thread:{} started:{} lastThread:{}", getServiceName(), started.get(), thread); + if (!started.compareAndSet(false, true)) { + return; + } + stopped = false; + this.thread = new Thread(this, getServiceName()); + this.thread.setDaemon(isDaemon); this.thread.start(); } @@ -47,6 +58,10 @@ public void shutdown() { } public void shutdown(final boolean interrupt) { + log.info("Try to shutdown service thread:{} started:{} lastThread:{}", getServiceName(), started.get(), thread); + if (!started.compareAndSet(true, false)) { + return; + } this.stopped = true; log.info("shutdown thread " + this.getServiceName() + " interrupt " + interrupt); @@ -75,11 +90,16 @@ public long getJointime() { return JOIN_TIME; } + @Deprecated public void stop() { this.stop(false); } + @Deprecated public void stop(final boolean interrupt) { + if (!started.get()) { + return; + } this.stopped = true; log.info("stop thread " + this.getServiceName() + " interrupt " + interrupt); @@ -93,6 +113,9 @@ public void stop(final boolean interrupt) { } public void makeStop() { + if (!started.get()) { + return; + } this.stopped = true; log.info("makestop thread " + this.getServiceName()); } @@ -128,4 +151,12 @@ protected void onWaitEnd() { public boolean isStopped() { return stopped; } + + public boolean isDaemon() { + return isDaemon; + } + + public void setDaemon(boolean daemon) { + isDaemon = daemon; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java index 1eb9f4d9948..af0b638d3d0 100644 --- a/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java +++ b/common/src/main/java/org/apache/rocketmq/common/message/MessageDecoder.java @@ -41,6 +41,7 @@ public class MessageDecoder { public final static int MESSAGE_MAGIC_CODE = -626843481; public static final char NAME_VALUE_SEPARATOR = 1; public static final char PROPERTY_SEPARATOR = 2; + public static final int PHY_POS_POSITION = 4 + 4 + 4 + 4 + 4 + 8; public static final int BODY_SIZE_POSITION = 4 // 1 TOTALSIZE + 4 // 2 MAGICCODE + 4 // 3 BODYCRC diff --git a/common/src/main/java/org/apache/rocketmq/common/stats/MomentStatsItemSet.java b/common/src/main/java/org/apache/rocketmq/common/stats/MomentStatsItemSet.java index d3b5596611e..a38b53ffd53 100644 --- a/common/src/main/java/org/apache/rocketmq/common/stats/MomentStatsItemSet.java +++ b/common/src/main/java/org/apache/rocketmq/common/stats/MomentStatsItemSet.java @@ -79,10 +79,10 @@ public MomentStatsItem getAndCreateStatsItem(final String statsKey) { if (null == statsItem) { statsItem = new MomentStatsItem(this.statsName, statsKey, this.scheduledExecutorService, this.log); - MomentStatsItem prev = this.statsItemTable.put(statsKey, statsItem); - - if (null == prev) { + MomentStatsItem prev = this.statsItemTable.putIfAbsent(statsKey, statsItem); + if (null != prev) { + statsItem = prev; // statsItem.init(); } } diff --git a/common/src/main/java/org/apache/rocketmq/common/stats/StatsItemSet.java b/common/src/main/java/org/apache/rocketmq/common/stats/StatsItemSet.java index 9a0caaa112d..b368651618a 100644 --- a/common/src/main/java/org/apache/rocketmq/common/stats/StatsItemSet.java +++ b/common/src/main/java/org/apache/rocketmq/common/stats/StatsItemSet.java @@ -162,10 +162,10 @@ public StatsItem getAndCreateStatsItem(final String statsKey) { StatsItem statsItem = this.statsItemTable.get(statsKey); if (null == statsItem) { statsItem = new StatsItem(this.statsName, statsKey, this.scheduledExecutorService, this.log); - StatsItem prev = this.statsItemTable.put(statsKey, statsItem); - - if (null == prev) { + StatsItem prev = this.statsItemTable.putIfAbsent(statsKey, statsItem); + if (null != prev) { + statsItem = prev; // statsItem.init(); } } diff --git a/common/src/test/java/org/apache/rocketmq/common/admin/TopicStatsTableTest.java b/common/src/test/java/org/apache/rocketmq/common/admin/TopicStatsTableTest.java new file mode 100644 index 00000000000..22ea926af8c --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/admin/TopicStatsTableTest.java @@ -0,0 +1,89 @@ +/* + * 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.admin; + +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + + +public class TopicStatsTableTest { + + private volatile TopicStatsTable topicStatsTable; + + private static final String TEST_TOPIC = "test_topic"; + + private static final String TEST_BROKER = "test_broker"; + + private static final int QUEUE_ID = 1; + + private static final long CURRENT_TIME_MILLIS = System.currentTimeMillis(); + + private static final long MAX_OFFSET = CURRENT_TIME_MILLIS + 100; + + private static final long MIN_OFFSET = CURRENT_TIME_MILLIS - 100; + + @Before + public void buildTopicStatsTable() { + HashMap offsetTableMap = new HashMap(); + + MessageQueue messageQueue = new MessageQueue(TEST_TOPIC, TEST_BROKER, QUEUE_ID); + + TopicOffset topicOffset = new TopicOffset(); + topicOffset.setLastUpdateTimestamp(CURRENT_TIME_MILLIS); + topicOffset.setMinOffset(MIN_OFFSET); + topicOffset.setMaxOffset(MAX_OFFSET); + + offsetTableMap.put(messageQueue, topicOffset); + + topicStatsTable = new TopicStatsTable(); + topicStatsTable.setOffsetTable(offsetTableMap); + } + + @Test + public void testGetOffsetTable() throws Exception { + validateTopicStatsTable(topicStatsTable); + } + + @Test + public void testFromJson() throws Exception { + String json = RemotingSerializable.toJson(topicStatsTable, true); + TopicStatsTable fromJson = RemotingSerializable.fromJson(json, TopicStatsTable.class); + + validateTopicStatsTable(fromJson); + } + + private static void validateTopicStatsTable(TopicStatsTable topicStatsTable) throws Exception { + Map.Entry savedTopicStatsTableMap = topicStatsTable.getOffsetTable().entrySet().iterator().next(); + MessageQueue savedMessageQueue = savedTopicStatsTableMap.getKey(); + TopicOffset savedTopicOffset = savedTopicStatsTableMap.getValue(); + + Assert.assertTrue(savedMessageQueue.getTopic().equals(TEST_TOPIC)); + Assert.assertTrue(savedMessageQueue.getBrokerName().equals(TEST_BROKER)); + Assert.assertTrue(savedMessageQueue.getQueueId() == QUEUE_ID); + + Assert.assertTrue(savedTopicOffset.getLastUpdateTimestamp() == CURRENT_TIME_MILLIS); + Assert.assertTrue(savedTopicOffset.getMaxOffset() == MAX_OFFSET); + Assert.assertTrue(savedTopicOffset.getMinOffset() == MIN_OFFSET); + } + +} diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/ClusterInfoTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/ClusterInfoTest.java new file mode 100644 index 00000000000..bfdd872153c --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/ClusterInfoTest.java @@ -0,0 +1,98 @@ +/* + * 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.protocol; + +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.protocol.body.ClusterInfo; +import org.apache.rocketmq.common.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + +import static org.junit.Assert.*; + +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Set; + +public class ClusterInfoTest { + + @Test + public void testFormJson() throws Exception { + ClusterInfo clusterInfo = buildClusterInfo(); + byte[] data = clusterInfo.encode(); + ClusterInfo json = RemotingSerializable.decode(data, ClusterInfo.class); + + assertNotNull(json); + assertNotNull(json.getClusterAddrTable()); + assertTrue(json.getClusterAddrTable().containsKey("DEFAULT_CLUSTER")); + assertTrue(json.getClusterAddrTable().get("DEFAULT_CLUSTER").contains("master")); + assertNotNull(json.getBrokerAddrTable()); + assertTrue(json.getBrokerAddrTable().containsKey("master")); + assertEquals(json.getBrokerAddrTable().get("master").getBrokerName(), "master"); + assertEquals(json.getBrokerAddrTable().get("master").getCluster(), "DEFAULT_CLUSTER"); + assertEquals(json.getBrokerAddrTable().get("master").getBrokerAddrs().get(MixAll.MASTER_ID), MixAll.getLocalhostByNetworkInterface()); + } + + @Test + public void testRetrieveAllClusterNames() throws Exception { + ClusterInfo clusterInfo = buildClusterInfo(); + byte[] data = clusterInfo.encode(); + ClusterInfo json = RemotingSerializable.decode(data, ClusterInfo.class); + + assertArrayEquals(new String[]{"DEFAULT_CLUSTER"}, json.retrieveAllClusterNames()); + } + + + @Test + public void testRetrieveAllAddrByCluster() throws Exception { + ClusterInfo clusterInfo = buildClusterInfo(); + byte[] data = clusterInfo.encode(); + ClusterInfo json = RemotingSerializable.decode(data, ClusterInfo.class); + + assertArrayEquals(new String[]{MixAll.getLocalhostByNetworkInterface()}, json.retrieveAllAddrByCluster("DEFAULT_CLUSTER")); + } + + + private ClusterInfo buildClusterInfo() throws Exception { + ClusterInfo clusterInfo = new ClusterInfo(); + HashMap brokerAddrTable = new HashMap(); + HashMap> clusterAddrTable = new HashMap>(); + + //build brokerData + BrokerData brokerData = new BrokerData(); + brokerData.setBrokerName("master"); + brokerData.setCluster("DEFAULT_CLUSTER"); + + //build brokerAddrs + HashMap brokerAddrs = new HashMap(); + brokerAddrs.put(MixAll.MASTER_ID, MixAll.getLocalhostByNetworkInterface()); + + brokerData.setBrokerAddrs(brokerAddrs); + brokerAddrTable.put("master", brokerData); + + Set brokerNames = new HashSet(); + brokerNames.add("master"); + + clusterAddrTable.put("DEFAULT_CLUSTER", brokerNames); + + clusterInfo.setBrokerAddrTable(brokerAddrTable); + clusterInfo.setClusterAddrTable(clusterAddrTable); + return clusterInfo; + } +} \ No newline at end of file diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBodyTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBodyTest.java new file mode 100644 index 00000000000..22bc6b39ddc --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/CheckClientRequestBodyTest.java @@ -0,0 +1,43 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +public class CheckClientRequestBodyTest { + + @Test + public void testFromJson() { + SubscriptionData subscriptionData = new SubscriptionData(); + String expectedClientId = "defalutId"; + String expectedGroup = "defaultGroup"; + CheckClientRequestBody checkClientRequestBody = new CheckClientRequestBody(); + checkClientRequestBody.setClientId(expectedClientId); + checkClientRequestBody.setGroup(expectedGroup); + checkClientRequestBody.setSubscriptionData(subscriptionData); + String json = RemotingSerializable.toJson(checkClientRequestBody, true); + CheckClientRequestBody fromJson = RemotingSerializable.fromJson(json, CheckClientRequestBody.class); + assertThat(fromJson.getClientId()).isEqualTo(expectedClientId); + assertThat(fromJson.getGroup()).isEqualTo(expectedGroup); + assertThat(fromJson.getSubscriptionData()).isEqualTo(subscriptionData); + } +} \ No newline at end of file diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumeStatsListTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumeStatsListTest.java new file mode 100644 index 00000000000..088ca05ee63 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumeStatsListTest.java @@ -0,0 +1,62 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.common.admin.ConsumeStats; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ConsumeStatsListTest { + + @Test + public void testFromJson() { + ConsumeStats consumeStats = new ConsumeStats(); + ArrayList consumeStatsListValue = new ArrayList(); + consumeStatsListValue.add(consumeStats); + HashMap> map = new HashMap>(); + map.put("subscriptionGroupName", consumeStatsListValue); + List>> consumeStatsListValue2 = new ArrayList>>(); + consumeStatsListValue2.add(map); + + String brokerAddr = "brokerAddr"; + long totalDiff = 12352L; + ConsumeStatsList consumeStatsList = new ConsumeStatsList(); + consumeStatsList.setBrokerAddr(brokerAddr); + consumeStatsList.setTotalDiff(totalDiff); + consumeStatsList.setConsumeStatsList(consumeStatsListValue2); + + String toJson = RemotingSerializable.toJson(consumeStatsList, true); + ConsumeStatsList fromJson = RemotingSerializable.fromJson(toJson, ConsumeStatsList.class); + + assertThat(fromJson.getBrokerAddr()).isEqualTo(brokerAddr); + assertThat(fromJson.getTotalDiff()).isEqualTo(totalDiff); + + List>> fromJsonConsumeStatsList = fromJson.getConsumeStatsList(); + assertThat(fromJsonConsumeStatsList).isInstanceOf(List.class); + + ConsumeStats fromJsonConsumeStats = fromJsonConsumeStatsList.get(0).get("subscriptionGroupName").get(0); + assertThat(fromJsonConsumeStats).isExactlyInstanceOf(ConsumeStats.class); + } +} \ No newline at end of file diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumerConnectionTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumerConnectionTest.java new file mode 100644 index 00000000000..be1460ebcad --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ConsumerConnectionTest.java @@ -0,0 +1,83 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.protocol.heartbeat.ConsumeType; +import org.apache.rocketmq.common.protocol.heartbeat.MessageModel; +import org.apache.rocketmq.common.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ConsumerConnectionTest { + + @Test + public void testFromJson() { + ConsumerConnection consumerConnection = new ConsumerConnection(); + HashSet connections = new HashSet(); + Connection conn = new Connection(); + connections.add(conn); + + ConcurrentHashMap subscriptionTable = new ConcurrentHashMap(); + SubscriptionData subscriptionData = new SubscriptionData(); + subscriptionTable.put("topicA", subscriptionData); + + ConsumeType consumeType = ConsumeType.CONSUME_ACTIVELY; + MessageModel messageModel = MessageModel.CLUSTERING; + ConsumeFromWhere consumeFromWhere = ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET; + + consumerConnection.setConnectionSet(connections); + consumerConnection.setSubscriptionTable(subscriptionTable); + consumerConnection.setConsumeType(consumeType); + consumerConnection.setMessageModel(messageModel); + consumerConnection.setConsumeFromWhere(consumeFromWhere); + + String json = RemotingSerializable.toJson(consumerConnection, true); + ConsumerConnection fromJson = RemotingSerializable.fromJson(json, ConsumerConnection.class); + assertThat(fromJson.getConsumeType()).isEqualTo(ConsumeType.CONSUME_ACTIVELY); + assertThat(fromJson.getMessageModel()).isEqualTo(MessageModel.CLUSTERING); + + HashSet connectionSet = fromJson.getConnectionSet(); + assertThat(connectionSet).isInstanceOf(Set.class); + + SubscriptionData data = fromJson.getSubscriptionTable().get("topicA"); + assertThat(data).isExactlyInstanceOf(SubscriptionData.class); + } + + @Test + public void testComputeMinVersion() { + ConsumerConnection consumerConnection = new ConsumerConnection(); + HashSet connections = new HashSet(); + Connection conn1 = new Connection(); + conn1.setVersion(1); + connections.add(conn1); + Connection conn2 = new Connection(); + conn2.setVersion(10); + connections.add(conn2); + consumerConnection.setConnectionSet(connections); + + int version = consumerConnection.computeMinVersion(); + assertThat(version).isEqualTo(1); + } +} diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.java new file mode 100644 index 00000000000..836733c5069 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/KVTableTest.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.common.protocol.body; + +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; + +import static org.assertj.core.api.Assertions.assertThat; + +public class KVTableTest { + + @Test + public void testFromJson() throws Exception { + HashMap table = new HashMap(); + table.put("key1", "value1"); + table.put("key2", "value2"); + + KVTable kvTable = new KVTable(); + kvTable.setTable(table); + + String json = RemotingSerializable.toJson(kvTable, true); + KVTable fromJson = RemotingSerializable.fromJson(json, KVTable.class); + + assertThat(fromJson).isNotEqualTo(kvTable); + assertThat(fromJson.getTable().get("key1")).isEqualTo(kvTable.getTable().get("key1")); + assertThat(fromJson.getTable().get("key2")).isEqualTo(kvTable.getTable().get("key2")); + } + +} diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/QueryCorrectionOffsetBodyTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/QueryCorrectionOffsetBodyTest.java new file mode 100644 index 00000000000..d0c4b5026c7 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/QueryCorrectionOffsetBodyTest.java @@ -0,0 +1,43 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class QueryCorrectionOffsetBodyTest { + + @Test + public void testFromJson() throws Exception { + QueryCorrectionOffsetBody qcob = new QueryCorrectionOffsetBody(); + Map offsetMap = new HashMap(); + offsetMap.put(1, 100L); + offsetMap.put(2, 200L); + qcob.setCorrectionOffsets(offsetMap); + String json = RemotingSerializable.toJson(qcob, true); + QueryCorrectionOffsetBody fromJson = RemotingSerializable.fromJson(json, QueryCorrectionOffsetBody.class); + assertThat(fromJson.getCorrectionOffsets().get(1)).isEqualTo(100L); + assertThat(fromJson.getCorrectionOffsets().get(2)).isEqualTo(200L); + assertThat(fromJson.getCorrectionOffsets().size()).isEqualTo(2); + } +} diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/ResetOffsetBodyTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ResetOffsetBodyTest.java new file mode 100644 index 00000000000..f9559a92c10 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/ResetOffsetBodyTest.java @@ -0,0 +1,46 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ResetOffsetBodyTest { + + @Test + public void testFromJson() throws Exception { + ResetOffsetBody rob = new ResetOffsetBody(); + Map offsetMap = new HashMap(); + MessageQueue queue = new MessageQueue(); + queue.setQueueId(1); + queue.setBrokerName("brokerName"); + queue.setTopic("topic"); + offsetMap.put(queue, 100L); + rob.setOffsetTable(offsetMap); + String json = RemotingSerializable.toJson(rob, true); + ResetOffsetBody fromJson = RemotingSerializable.fromJson(json, ResetOffsetBody.class); + assertThat(fromJson.getOffsetTable().get(queue)).isEqualTo(100L); + assertThat(fromJson.getOffsetTable().size()).isEqualTo(1); + } +} diff --git a/common/src/test/java/org/apache/rocketmq/common/protocol/body/SubscriptionGroupWrapperTest.java b/common/src/test/java/org/apache/rocketmq/common/protocol/body/SubscriptionGroupWrapperTest.java new file mode 100644 index 00000000000..ffd7f61022a --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/protocol/body/SubscriptionGroupWrapperTest.java @@ -0,0 +1,49 @@ +/* + * 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.protocol.body; + +import org.apache.rocketmq.common.DataVersion; +import org.apache.rocketmq.common.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.junit.Test; +import java.util.concurrent.ConcurrentHashMap; +import static org.assertj.core.api.Assertions.assertThat; + +public class SubscriptionGroupWrapperTest { + + @Test + public void testFromJson(){ + SubscriptionGroupWrapper subscriptionGroupWrapper = new SubscriptionGroupWrapper(); + ConcurrentHashMap subscriptions = new ConcurrentHashMap(); + SubscriptionGroupConfig subscriptionGroupConfig = new SubscriptionGroupConfig(); + subscriptionGroupConfig.setConsumeBroadcastEnable(true); + subscriptionGroupConfig.setBrokerId(1234); + subscriptionGroupConfig.setGroupName("Consumer-group-one"); + subscriptions.put("Consumer-group-one", subscriptionGroupConfig); + subscriptionGroupWrapper.setSubscriptionGroupTable(subscriptions); + DataVersion dataVersion = new DataVersion(); + dataVersion.nextVersion(); + subscriptionGroupWrapper.setDataVersion(dataVersion); + String json = RemotingSerializable.toJson(subscriptionGroupWrapper, true); + SubscriptionGroupWrapper fromJson = RemotingSerializable.fromJson(json, SubscriptionGroupWrapper.class); + assertThat(fromJson.getSubscriptionGroupTable()).containsKey("Consumer-group-one"); + assertThat(fromJson.getSubscriptionGroupTable().get("Consumer-group-one").getGroupName()).isEqualTo("Consumer-group-one"); + assertThat(fromJson.getSubscriptionGroupTable().get("Consumer-group-one").getBrokerId()).isEqualTo(1234); + } + +} diff --git a/common/src/test/java/org/apache/rocketmq/common/stats/StatsItemSetTest.java b/common/src/test/java/org/apache/rocketmq/common/stats/StatsItemSetTest.java new file mode 100644 index 00000000000..bd6550d7be7 --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/stats/StatsItemSetTest.java @@ -0,0 +1,93 @@ +/* + * 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.stats; + +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.junit.After; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class StatsItemSetTest { + + private ThreadPoolExecutor executor; + private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + + @Test + public void test_getAndCreateStatsItem_multiThread() throws InterruptedException { + assertEquals(20L, test_unit().longValue()); + } + + @Test + public void test_getAndCreateMomentStatsItem_multiThread() throws InterruptedException { + assertEquals(10, test_unit_moment().longValue()); + } + + private AtomicLong test_unit() throws InterruptedException { + final StatsItemSet statsItemSet = new StatsItemSet("topicTest", scheduler, null); + executor = new ThreadPoolExecutor(10, 20, 10, TimeUnit.SECONDS, + new ArrayBlockingQueue(100), new ThreadFactoryImpl("testMultiThread")); + for (int i = 0; i < 10; i++) { + executor.submit(new Runnable() { + @Override + public void run() { + statsItemSet.addValue("topicTest", 2, 1); + } + }); + } + while (true) { + if (executor.getCompletedTaskCount() == 10) { + break; + } + Thread.sleep(1000); + } + return statsItemSet.getStatsItem("topicTest").getValue(); + } + + private AtomicLong test_unit_moment() throws InterruptedException { + final MomentStatsItemSet statsItemSet = new MomentStatsItemSet("topicTest", scheduler, null); + executor = new ThreadPoolExecutor(10, 20, 10, TimeUnit.SECONDS, + new ArrayBlockingQueue(100), new ThreadFactoryImpl("testMultiThread")); + for (int i = 0; i < 10; i++) { + executor.submit(new Runnable() { + @Override + public void run() { + statsItemSet.setValue("test", 10); + } + }); + } + while (true) { + if (executor.getCompletedTaskCount() == 10) { + break; + } + Thread.sleep(1000); + } + return statsItemSet.getAndCreateStatsItem("test").getValue(); + } + + @After + public void shutdown() { + executor.shutdown(); + } +} \ No newline at end of file diff --git a/distribution/bin/dledger/fast-try.sh b/distribution/bin/dledger/fast-try.sh new file mode 100644 index 00000000000..ff8a96036d1 --- /dev/null +++ b/distribution/bin/dledger/fast-try.sh @@ -0,0 +1,102 @@ +#!/bin/sh + +# 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. + +## Revise the base dir +CURRENT_DIR="$(cd "$(dirname "$0")"; pwd)" +RMQ_DIR=$CURRENT_DIR/../.. +cd $RMQ_DIR + +function startNameserver() { + export JAVA_OPT_EXT=" -Xms512m -Xmx512m " + nohup bin/mqnamesrv & +} + +function startBroker() { + export JAVA_OPT_EXT=" -Xms1g -Xmx1g " + conf_name=$1 + nohup bin/mqbroker -c $conf_name & +} + +function stopNameserver() { + PIDS=$(ps -ef|grep java|grep NamesrvStartup|grep -v grep|awk '{print $2}') + if [ ! -z "$PIDS" ]; then + kill -s TERM $PIDS + fi +} + +function stopBroker() { + conf_name=$1 + PIDS=$(ps -ef|grep java|grep BrokerStartup|grep $conf_name|grep -v grep|awk '{print $2}') + i=1 + while [ ! -z "$PIDS" -a $i -lt 5 ] + do + echo "Waiting to kill ..." + kill -s TERM $PIDS + ((i=$i+1)) + sleep 2 + PIDS=$(ps -ef|grep java|grep BrokerStartup|grep $conf_name|grep -v grep|awk '{print $2}') + done + PIDS=$(ps -ef|grep java|grep BrokerStartup|grep $conf_name|grep -v grep|awk '{print $2}') + if [ ! -z "$PIDS" ]; then + kill -9 $PIDS + fi +} + +function stopAll() { + ps -ef|grep java|grep BrokerStartup|grep -v grep|awk '{print $2}'|xargs kill + stopNameserver + stopBroker ./conf/dledger/broker-n0.conf + stopBroker ./conf/dledger/broker-n1.conf + stopBroker ./conf/dledger/broker-n2.conf +} + +function startAll() { + startNameserver + startBroker ./conf/dledger/broker-n0.conf + startBroker ./conf/dledger/broker-n1.conf + startBroker ./conf/dledger/broker-n2.conf +} + +function checkConf() { + if [ ! -f ./conf/dledger/broker-n0.conf -o ! -f ./conf/dledger/broker-n1.conf -o ! -f ./conf/dledger/broker-n2.conf ]; then + echo "Make sure the ./conf/dledger/broker-n0.conf, ./conf/dledger/broker-n1.conf, ./conf/dledger/broker-n2.conf exists" + exit -1 + fi +} + + + +## Main +if [ $# -lt 1 ]; then + echo "Usage: sh $0 start|stop" + exit -1 +fi +action=$1 +checkConf +case $action in + "start") + startAll + exit + ;; + "stop") + stopAll + ;; + *) + echo "Usage: sh $0 start|stop" + ;; +esac + diff --git a/distribution/conf/dledger/broker-n0.conf b/distribution/conf/dledger/broker-n0.conf new file mode 100644 index 00000000000..5351e497dc8 --- /dev/null +++ b/distribution/conf/dledger/broker-n0.conf @@ -0,0 +1,27 @@ +# 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. + +brokerClusterName = RaftCluster +brokerName=RaftNode00 +listenPort=30911 +namesrvAddr=127.0.0.1:9876 +storePathRootDir=/tmp/rmqstore/node00 +storePathCommitLog=/tmp/rmqstore/node00/commitlog +enableDLegerCommitLog=true +dLegerGroup=RaftNode00 +dLegerPeers=n0-127.0.0.1:40911;n1-127.0.0.1:40912;n2-127.0.0.1:40913 +## must be unique +dLegerSelfId=n0 +sendMessageThreadPoolNums=16 diff --git a/distribution/conf/dledger/broker-n1.conf b/distribution/conf/dledger/broker-n1.conf new file mode 100644 index 00000000000..6aaf8f9309c --- /dev/null +++ b/distribution/conf/dledger/broker-n1.conf @@ -0,0 +1,27 @@ +# 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. + +brokerClusterName = RaftCluster +brokerName=RaftNode00 +listenPort=30921 +namesrvAddr=127.0.0.1:9876 +storePathRootDir=/tmp/rmqstore/node01 +storePathCommitLog=/tmp/rmqstore/node01/commitlog +enableDLegerCommitLog=true +dLegerGroup=RaftNode00 +dLegerPeers=n0-127.0.0.1:40911;n1-127.0.0.1:40912;n2-127.0.0.1:40913 +## must be unique +dLegerSelfId=n1 +sendMessageThreadPoolNums=16 diff --git a/distribution/conf/dledger/broker-n2.conf b/distribution/conf/dledger/broker-n2.conf new file mode 100644 index 00000000000..c863d89ee82 --- /dev/null +++ b/distribution/conf/dledger/broker-n2.conf @@ -0,0 +1,27 @@ +# 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. + +brokerClusterName = RaftCluster +brokerName=RaftNode00 +listenPort=30931 +namesrvAddr=127.0.0.1:9876 +storePathRootDir=/tmp/rmqstore/node02 +storePathCommitLog=/tmp/rmqstore/node02/commitlog +enableDLegerCommitLog=true +dLegerGroup=RaftNode00 +dLegerPeers=n0-127.0.0.1:40911;n1-127.0.0.1:40912;n2-127.0.0.1:40913 +## must be unique +dLegerSelfId=n2 +sendMessageThreadPoolNums=16 diff --git a/distribution/release.xml b/distribution/release.xml index 76bf5fbb873..ae9eb081875 100644 --- a/distribution/release.xml +++ b/distribution/release.xml @@ -40,7 +40,7 @@ - bin/* + bin/** 0755 diff --git a/docs/cn/RocketMQ_Example.md b/docs/cn/RocketMQ_Example.md new file mode 100644 index 00000000000..cfd78ea405f --- /dev/null +++ b/docs/cn/RocketMQ_Example.md @@ -0,0 +1,956 @@ +1 基本样例 +-------- + +在基本样例中我们提供如下的功能场景: + +* 使用RocketMQ发送三种类型的消息:同步消息、异步消息和单向消息。其中前两种消息是可靠的,因为会有发送是否成功的应答。 +* 使用RocketMQ来消费接收到的消息。 + +### 1.1 加入依赖: + +`maven:` +``` + + org.apache.rocketmq + rocketmq-client + 4.3.0 + +``` +`gradle` +``` +compile 'org.apache.rocketmq:rocketmq-client:4.3.0' +``` +### 1.2 消息发送 + +#### 1、Producer端发送同步消息 + +这种可靠性同步地发送方式使用的比较广泛,比如:重要的消息通知,短信通知。 +```java +public class SyncProducer { + public static void main(String[] args) throws Exception { + // 实例化消息生产者Producer + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // 设置NameServer的地址 + producer.setNamesrvAddr("localhost:9876"); + // 启动Producer实例 + producer.start(); + for (int i = 0; i < 100; i++) { + // 创建消息,并指定Topic,Tag和消息体 + Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); + // 发送消息到一个Broker + SendResult sendResult = producer.send(msg); + // 通过sendResult返回消息是否成功送达 + System.out.printf("%s%n", sendResult); + } + // 如果不再发送消息,关闭Producer实例。 + producer.shutdown(); + } +} +``` +#### 2、发送异步消息 + +异步消息通常用在对响应时间敏感的业务场景,即发送端不能容忍长时间地等待Broker的响应。 + +```java +public class AsyncProducer { + public static void main(String[] args) throws Exception { + // 实例化消息生产者Producer + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // 设置NameServer的地址 + producer.setNamesrvAddr("localhost:9876"); + // 启动Producer实例 + producer.start(); + producer.setRetryTimesWhenSendAsyncFailed(0); + for (int i = 0; i < 100; i++) { + final int index = i; + // 创建消息,并指定Topic,Tag和消息体 + Message msg = new Message("TopicTest", + "TagA", + "OrderID188", + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + // SendCallback接收异步返回结果的回调 + producer.send(msg, new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + System.out.printf("%-10d OK %s %n", index, + sendResult.getMsgId()); + } + @Override + public void onException(Throwable e) { + System.out.printf("%-10d Exception %s %n", index, e); + e.printStackTrace(); + } + }); + } + // 如果不再发送消息,关闭Producer实例。 + producer.shutdown(); + } +} +``` + +#### 3、单向发送消息 + +这种方式主要用在不特别关心发送结果的场景,例如日志发送。 + +```java +public class OnewayProducer { + public static void main(String[] args) throws Exception{ + // 实例化消息生产者Producer + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // 设置NameServer的地址 + producer.setNamesrvAddr("localhost:9876"); + // 启动Producer实例 + producer.start(); + for (int i = 0; i < 100; i++) { + // 创建消息,并指定Topic,Tag和消息体 + Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); + // 发送单向消息,没有任何返回结果 + producer.sendOneway(msg); + + } + // 如果不再发送消息,关闭Producer实例。 + producer.shutdown(); + } +} +``` + +### 1.3 消费消息 + +```java +public class Consumer { + + public static void main(String[] args) throws InterruptedException, MQClientException { + + // 实例化消费者 + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name"); + + // 设置NameServer的地址 + consumer.setNamesrvAddr("localhost:9876"); + + // 订阅一个或者多个Topic,以及Tag来过滤需要消费的消息 + consumer.subscribe("TopicTest", "*"); + // 注册回调实现类来处理从broker拉取回来的消息 + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + System.out.printf("%s Receive New Messages: %s %n", Thread.currentThread().getName(), msgs); + // 标记该消息已经被成功消费 + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + // 启动消费者实例 + consumer.start(); + System.out.printf("Consumer Started.%n"); + } +} +``` + +2 顺序消息样例 +---------- + +消息有序指的是可以按照消息的发送顺序来消费(FIFO)。RocketMQ可以严格的保证消息有序,可以分为分区有序或者全局有序。 + +顺序消费的原理解析,在默认的情况下消息发送会采取Round Robin轮询方式把消息发送到不同的queue(分区队列);而消费消息的时候从多个queue上拉取消息,这种情况发送和消费是不能保证顺序。但是如果控制发送的顺序消息只依次发送到同一个queue中,消费的时候只从这个queue上依次拉取,则就保证了顺序。当发送和消费参与的queue只有一个,则是全局有序;如果多个queue参与,则为分区有序,即相对每个queue,消息都是有序的。 + +下面用订单进行分区有序的示例。一个订单的顺序流程是:创建、付款、推送、完成。订单号相同的消息会被先后发送到同一个队列中,消费时,同一个OrderId获取到的肯定是同一个队列。 + +### 2.1 顺序消息生产 + +```java +package org.apache.rocketmq.example.order2; + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.MessageQueueSelector; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageQueue; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +/** +* Producer,发送顺序消息 +*/ +public class Producer { + + public static void main(String[] args) throws Exception { + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + + producer.setNamesrvAddr("127.0.0.1:9876"); + + producer.start(); + + String[] tags = new String[]{"TagA", "TagC", "TagD"}; + + // 订单列表 + List orderList = new Producer().buildOrders(); + + Date date = new Date(); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + String dateStr = sdf.format(date); + for (int i = 0; i < 10; i++) { + // 加个时间前缀 + String body = dateStr + " Hello RocketMQ " + orderList.get(i); + Message msg = new Message("TopicTest", tags[i % tags.length], "KEY" + i, body.getBytes()); + + SendResult sendResult = producer.send(msg, new MessageQueueSelector() { + @Override + public MessageQueue select(List mqs, Message msg, Object arg) { + Long id = (Long) arg; //根据订单id选择发送queue + long index = id % mqs.size(); + return mqs.get((int) index); + } + }, orderList.get(i).getOrderId());//订单id + + System.out.println(String.format("SendResult status:%s, queueId:%d, body:%s", + sendResult.getSendStatus(), + sendResult.getMessageQueue().getQueueId(), + body)); + } + + producer.shutdown(); + } + + /** + * 订单的步骤 + */ + private static class OrderStep { + private long orderId; + private String desc; + + public long getOrderId() { + return orderId; + } + + public void setOrderId(long orderId) { + this.orderId = orderId; + } + + public String getDesc() { + return desc; + } + + public void setDesc(String desc) { + this.desc = desc; + } + + @Override + public String toString() { + return "OrderStep{" + + "orderId=" + orderId + + ", desc='" + desc + '\'' + + '}'; + } + } + + /** + * 生成模拟订单数据 + */ + private List buildOrders() { + List orderList = new ArrayList(); + + OrderStep orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("创建"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("创建"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("付款"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("创建"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("付款"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("付款"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("完成"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("推送"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("完成"); + orderList.add(orderDemo); + + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("完成"); + orderList.add(orderDemo); + + return orderList; + } +} +``` + +### 2.2 顺序消费消息 + +```java +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; +import java.util.List; + +package org.apache.rocketmq.example.order2; + +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.message.MessageExt; + +import java.util.List; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +/** +* 顺序消息消费,带事务方式(应用可控制Offset什么时候提交) +*/ +public class ConsumerInOrder { + + public static void main(String[] args) throws Exception { + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_3"); + consumer.setNamesrvAddr("127.0.0.1:9876"); + /** + * 设置Consumer第一次启动是从队列头部开始消费还是队列尾部开始消费
+ * 如果非第一次启动,那么按照上次消费的位置继续消费 + */ + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + + consumer.subscribe("TopicTest", "TagA || TagC || TagD"); + + consumer.registerMessageListener(new MessageListenerOrderly() { + + Random random = new Random(); + + @Override + public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderlyContext context) { + context.setAutoCommit(true); + for (MessageExt msg : msgs) { + // 可以看到每个queue有唯一的consume线程来消费, 订单对每个queue(分区)有序 + System.out.println("consumeThread=" + Thread.currentThread().getName() + "queueId=" + msg.getQueueId() + ", content:" + new String(msg.getBody())); + } + + try { + //模拟业务逻辑处理中... + TimeUnit.SECONDS.sleep(random.nextInt(10)); + } catch (Exception e) { + e.printStackTrace(); + } + return ConsumeOrderlyStatus.SUCCESS; + } + }); + + consumer.start(); + + System.out.println("Consumer Started."); + } +} +``` + +3 延时消息样例 +---------- + +### 3.1 启动消费者等待传入订阅消息 + +```java + +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; +import java.util.List; + +public class ScheduledMessageConsumer { + public static void main(String[] args) throws Exception { + // 实例化消费者 + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("ExampleConsumer"); + // 订阅Topics + consumer.subscribe("TestTopic", "*"); + // 注册消息监听者 + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List messages, ConsumeConcurrentlyContext context) { + for (MessageExt message : messages) { + // Print approximate delay time period + System.out.println("Receive message[msgId=" + message.getMsgId() + "] " + (System.currentTimeMillis() - message.getStoreTimestamp()) + "ms later"); + } + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + // 启动消费者 + consumer.start(); + } +} + +``` + +### 3.2 发送延时消息 + +```java + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; + +public class ScheduledMessageProducer { + public static void main(String[] args) throws Exception { + // 实例化一个生产者来产生延时消息 + DefaultMQProducer producer = new DefaultMQProducer("ExampleProducerGroup"); + // 启动生产者 + producer.start(); + int totalMessagesToSend = 100; + for (int i = 0; i < totalMessagesToSend; i++) { + Message message = new Message("TestTopic", ("Hello scheduled message " + i).getBytes()); + // 设置延时等级3,这个消息将在10s之后发送(现在只支持固定的几个时间,详看delayTimeLevel) + message.setDelayTimeLevel(3); + // 发送消息 + producer.send(message); + } + // 关闭生产者 + producer.shutdown(); + } +} +``` + +### 3.3 验证 + +您将会看到消息的消费比存储时间晚10秒。 + +### 3.4 延时消息的使用场景 +比如电商里,提交了一个订单就可以发送一个延时消息,1h后去检查这个订单的状态,如果还是未付款就取消订单释放库存。 + +### 3.5 延时消息的使用限制 + +```java +// org/apache/rocketmq/store/config/MessageStoreConfig.java + +private String messageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"; +``` +现在RocketMq并不支持任意时间的延时,需要设置几个固定的延时等级,从1s到2h分别对应着等级1到18 +消息消费失败会进入延时消息队列,消息发送时间与设置的延时等级和重试次数有关,详见代码`SendMessageProcessor.java` + + +4 批量消息样例 +---------- + +批量发送消息能显著提高传递小消息的性能。限制是这些批量消息应该有相同的topic,相同的waitStoreMsgOK,而且不能是延时消息。此外,这一批消息的总大小不应超过4MB。 + +### 4.1 发送批量消息 + +如果您每次只发送不超过4MB的消息,则很容易使用批处理,样例如下: + +```java +String topic = "BatchTest"; +List messages = new ArrayList<>(); +messages.add(new Message(topic, "TagA", "OrderID001", "Hello world 0".getBytes())); +messages.add(new Message(topic, "TagA", "OrderID002", "Hello world 1".getBytes())); +messages.add(new Message(topic, "TagA", "OrderID003", "Hello world 2".getBytes())); +try { + producer.send(messages); +} catch (Exception e) { + e.printStackTrace(); + //处理error +} + +``` + +### 4.2 消息列表分割 + +复杂度只有当你发送大批量时才会增长,你可能不确定它是否超过了大小限制(4MB)。这时候你最好把你的消息列表分割一下: + +```java + +public class ListSplitter implements Iterator> { + private final int SIZE_LIMIT = 1024 * 1024 * 4; + private final List messages; + private int currIndex; + public ListSplitter(List messages) { + this.messages = messages; + } + @Override public boolean hasNext() { + return currIndex < messages.size(); + } + @Override public List next() { + int nextIndex = currIndex; + int totalSize = 0; + for (; nextIndex < messages.size(); nextIndex++) { + Message message = messages.get(nextIndex); + int tmpSize = message.getTopic().length() + message.getBody().length; + Map properties = message.getProperties(); + for (Map.Entry entry : properties.entrySet()) { + tmpSize += entry.getKey().length() + entry.getValue().length(); + } + tmpSize = tmpSize + 20; // 增加日志的开销20字节 + if (tmpSize > SIZE_LIMIT) { + //单个消息超过了最大的限制 + //忽略,否则会阻塞分裂的进程 + if (nextIndex - currIndex == 0) { + //假如下一个子列表没有元素,则添加这个子列表然后退出循环,否则只是退出循环 + nextIndex++; + } + break; + } + if (tmpSize + totalSize > SIZE_LIMIT) { + break; + } else { + totalSize += tmpSize; + } + + } + List subList = messages.subList(currIndex, nextIndex); + currIndex = nextIndex; + return subList; + } +} +//把大的消息分裂成若干个小的消息 +ListSplitter splitter = new ListSplitter(messages); +while (splitter.hasNext()) { + try { + List listItem = splitter.next(); + producer.send(listItem); + } catch (Exception e) { + e.printStackTrace(); + //处理error + } +} +``` + +5 过滤消息样例 +---------- + +在大多数情况下,TAG是一个简单而有用的设计,其可以来选择您想要的消息。例如: + +```java +DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_EXAMPLE"); +consumer.subscribe("TOPIC", "TAGA || TAGB || TAGC"); +``` + +消费者将接收包含TAGA或TAGB或TAGC的消息。但是限制是一个消息只能有一个标签,这对于复杂的场景可能不起作用。在这种情况下,可以使用SQL表达式筛选消息。SQL特性可以通过发送消息时的属性来进行计算。在RocketMQ定义的语法下,可以实现一些简单的逻辑。下面是一个例子: +``` +------------ +| message | +|----------| a > 5 AND b = 'abc' +| a = 10 | --------------------> Gotten +| b = 'abc'| +| c = true | +------------ +------------ +| message | +|----------| a > 5 AND b = 'abc' +| a = 1 | --------------------> Missed +| b = 'abc'| +| c = true | +------------ +``` +### 5.1 基本语法 + +RocketMQ只定义了一些基本语法来支持这个特性。你也可以很容易地扩展它。 + +- 数值比较,比如:**>,>=,<,<=,BETWEEN,=;** +- 字符比较,比如:**=,<>,IN;** +- **IS NULL** 或者 **IS NOT NULL;** +- 逻辑符号 **AND,OR,NOT;** + +常量支持类型为: + +- 数值,比如:**123,3.1415;** +- 字符,比如:**'abc',必须用单引号包裹起来;** +- **NULL**,特殊的常量 +- 布尔值,**TRUE** 或 **FALSE** + +只有使用push模式的消费者才能用使用SQL92标准的sql语句,接口如下: +``` +public void subscribe(finalString topic, final MessageSelector messageSelector) +``` + +### 5.2 使用样例 + +#### 1、生产者样例 + +发送消息时,你能通过`putUserProperty`来设置消息的属性 + +```java +DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); +producer.start(); +Message msg = new Message("TopicTest", + tag, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) +); +// 设置一些属性 +msg.putUserProperty("a", String.valueOf(i)); +SendResult sendResult = producer.send(msg); + +producer.shutdown(); +``` + +#### 2、消费者样例 + +用MessageSelector.bySql来使用sql筛选消息 + +```java +DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_4"); +// 只有订阅的消息有这个属性a, a >=0 and a <= 3 +consumer.subscribe("TopicTest", MessageSelector.bySql("a between 0 and 3"); +consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } +}); +consumer.start(); + +``` + +6 消息事务样例 +---------- + +事务消息共有三种状态,提交状态、回滚状态、中间状态: + +1. TransactionStatus.CommitTransaction: 提交事务,它允许消费者消费此消息。 +2. TransactionStatus.RollbackTransaction: 回滚事务,它代表该消息将被删除,不允许被消费。 +3. TransactionStatus.Unknown: 中间状态,它代表需要检查消息队列来确定状态。 + +### 6.1 发送事务消息样例 + +#### 1、创建事务性生产者 + +使用 `TransactionMQProducer`类创建生产者,并指定唯一的 `ProducerGroup`,就可以设置自定义线程池来处理这些检查请求。执行本地事务后、需要根据执行结果对消息队列进行回复。回传的事务状态在请参考前一节。 + +```java +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; +import java.util.List; +public class TransactionProducer { + public static void main(String[] args) throws MQClientException, InterruptedException { + TransactionListener transactionListener = new TransactionListenerImpl(); + TransactionMQProducer producer = new TransactionMQProducer("please_rename_unique_group_name"); + ExecutorService executorService = new ThreadPoolExecutor(2, 5, 100, TimeUnit.SECONDS, new ArrayBlockingQueue(2000), new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + Thread thread = new Thread(r); + thread.setName("client-transaction-msg-check-thread"); + return thread; + } + }); + producer.setExecutorService(executorService); + producer.setTransactionListener(transactionListener); + producer.start(); + String[] tags = new String[] {"TagA", "TagB", "TagC", "TagD", "TagE"}; + for (int i = 0; i < 10; i++) { + try { + Message msg = + new Message("TopicTest1234", tags[i % tags.length], "KEY" + i, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET)); + SendResult sendResult = producer.sendMessageInTransaction(msg, null); + System.out.printf("%s%n", sendResult); + Thread.sleep(10); + } catch (MQClientException | UnsupportedEncodingException e) { + e.printStackTrace(); + } + } + for (int i = 0; i < 100000; i++) { + Thread.sleep(1000); + } + producer.shutdown(); + } +} + +``` +#### 2、实现事务的监听接口 + +当发送半消息成功时,我们使用 `executeLocalTransaction` 方法来执行本地事务。它返回前一节中提到的三个事务状态之一。`checkLocalTranscation` 方法用于检查本地事务状态,并回应消息队列的检查请求。它也是返回前一节中提到的三个事务状态之一。 + +```java +public class TransactionListenerImpl implements TransactionListener { + private AtomicInteger transactionIndex = new AtomicInteger(0); + private ConcurrentHashMap localTrans = new ConcurrentHashMap<>(); + @Override + public LocalTransactionState executeLocalTransaction(Message msg, Object arg) { + int value = transactionIndex.getAndIncrement(); + int status = value % 3; + localTrans.put(msg.getTransactionId(), status); + return LocalTransactionState.UNKNOW; + } + @Override + public LocalTransactionState checkLocalTransaction(MessageExt msg) { + Integer status = localTrans.get(msg.getTransactionId()); + if (null != status) { + switch (status) { + case 0: + return LocalTransactionState.UNKNOW; + case 1: + return LocalTransactionState.COMMIT_MESSAGE; + case 2: + return LocalTransactionState.ROLLBACK_MESSAGE; + } + } + return LocalTransactionState.COMMIT_MESSAGE; + } +} + +``` + +### 6.2 事务消息使用上的限制 + +1. 事务消息不支持延时消息和批量消息。 +2. 为了避免单个消息被检查太多次而导致半队列消息累积,我们默认将单个消息的检查次数限制为 15 次,但是用户可以通过 Broker 配置文件的 `transactionCheckMax`参数来修改此限制。如果已经检查某条消息超过 N 次的话( N = `transactionCheckMax` ) 则 Broker 将丢弃此消息,并在默认情况下同时打印错误日志。用户可以通过重写 `AbstractTransactionCheckListener` 类来修改这个行为。 +3. 事务消息将在 Broker 配置文件中的参数 transactionMsgTimeout 这样的特定时间长度之后被检查。当发送事务消息时,用户还可以通过设置用户属性 CHECK_IMMUNITY_TIME_IN_SECONDS 来改变这个限制,该参数优先于 `transactionMsgTimeout` 参数。 +4. 事务性消息可能不止一次被检查或消费。 +5. 提交给用户的目标主题消息可能会失败,目前这依日志的记录而定。它的高可用性通过 RocketMQ 本身的高可用性机制来保证,如果希望确保事务消息不丢失、并且事务完整性得到保证,建议使用同步的双重写入机制。 +6. 事务消息的生产者 ID 不能与其他类型消息的生产者 ID 共享。与其他类型的消息不同,事务消息允许反向查询、MQ服务器能通过它们的生产者 ID 查询到消费者。 + +7 Logappender样例 +----------------- + +RocketMQ日志提供log4j、log4j2和logback日志框架作为业务应用,下面是配置样例 + +### 7.1 log4j样例 + +按下面样例使用log4j属性配置 +``` +log4j.appender.mq=org.apache.rocketmq.logappender.log4j.RocketmqLog4jAppender +log4j.appender.mq.Tag=yourTag +log4j.appender.mq.Topic=yourLogTopic +log4j.appender.mq.ProducerGroup=yourLogGroup +log4j.appender.mq.NameServerAddress=yourRocketmqNameserverAddress +log4j.appender.mq.layout=org.apache.log4j.PatternLayout +log4j.appender.mq.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-4r [%t] (%F:%L) %-5p - %m%n +``` +按下面样例使用log4j xml配置来使用异步添加日志 +``` + +      + + + +``` +### 7.2 log4j2样例 + +用log4j2时,配置如下,如果想要非阻塞,只需要使用异步添加引用即可 +``` + + +``` +### 7.3 logback样例 +``` +yourTagyourLogTopicyourLogGroupyourRocketmqNameserverAddress +      %date %p %t - %m%n + +1024802000true + +``` + +8 OpenMessaging样例 +--------------- + + [OpenMessaging](https://www.google.com/url?q=http://openmessaging.cloud/&sa=D&ust=1546524111089000)旨在建立消息和流处理规范,以为金融、电子商务、物联网和大数据领域提供通用框架及工业级指导方案。在分布式异构环境中,设计原则是面向云、简单、灵活和独立于语言。符合这些规范将帮助企业方便的开发跨平台和操作系统的异构消息传递应用程序。提供了openmessaging-api 0.3.0-alpha的部分实现,下面的示例演示如何基于OpenMessaging访问RocketMQ。 + +### 8.1 OMSProducer样例 + +下面的示例演示如何在同步、异步或单向传输中向RocketMQ代理发送消息。 + +```java +import io.openmessaging.Future; +import io.openmessaging.FutureListener; +import io.openmessaging.Message; +import io.openmessaging.MessagingAccessPoint; +import io.openmessaging.OMS; +import io.openmessaging.producer.Producer; +import io.openmessaging.producer.SendResult; +import java.nio.charset.Charset; +import java.util.concurrent.CountDownLatch; + +public class SimpleProducer { + public static void main(String[] args) { + final MessagingAccessPoint messagingAccessPoint = + OMS.getMessagingAccessPoint("oms:rocketmq://localhost:9876/default:default"); + final Producer producer = messagingAccessPoint.createProducer(); + messagingAccessPoint.startup(); + System.out.printf("MessagingAccessPoint startup OK%n"); + producer.startup(); + System.out.printf("Producer startup OK%n"); + { + Message message = producer.createBytesMessage("OMS_HELLO_TOPIC", "OMS_HELLO_BODY".getBytes(Charset.forName("UTF-8"))); + SendResult sendResult = producer.send(message); + //final Void aVoid = result.get(3000L); + System.out.printf("Send async message OK, msgId: %s%n", sendResult.messageId()); + } + final CountDownLatch countDownLatch = new CountDownLatch(1); + { + final Future result = producer.sendAsync(producer.createBytesMessage("OMS_HELLO_TOPIC", "OMS_HELLO_BODY".getBytes(Charset.forName("UTF-8")))); + result.addListener(new FutureListener() { + @Override + public void operationComplete(Future future) { + if (future.getThrowable() != null) { + System.out.printf("Send async message Failed, error: %s%n", future.getThrowable().getMessage()); + } else { + System.out.printf("Send async message OK, msgId: %s%n", future.get().messageId()); + } + countDownLatch.countDown(); + } + }); + } + { + producer.sendOneway(producer.createBytesMessage("OMS_HELLO_TOPIC", "OMS_HELLO_BODY".getBytes(Charset.forName("UTF-8")))); + System.out.printf("Send oneway message OK%n"); + } + try { + countDownLatch.await(); + Thread.sleep(500); // 等一些时间来发送消息 + } catch (InterruptedException ignore) { + } + producer.shutdown(); + } +} +``` + +### 8.2 OMSPullConsumer + +用OMS PullConsumer 来从指定的队列中拉取消息 + +```java +import io.openmessaging.Message; +import io.openmessaging.MessagingAccessPoint; +import io.openmessaging.OMS; +import io.openmessaging.OMSBuiltinKeys; +import io.openmessaging.consumer.PullConsumer; +import io.openmessaging.producer.Producer; +import io.openmessaging.producer.SendResult; + +public class SimplePullConsumer { + public static void main(String[] args) { + final MessagingAccessPoint messagingAccessPoint = + OMS.getMessagingAccessPoint("oms:rocketmq://localhost:9876/default:default"); + messagingAccessPoint.startup(); + final Producer producer = messagingAccessPoint.createProducer(); + final PullConsumer consumer = messagingAccessPoint.createPullConsumer( + OMS.newKeyValue().put(OMSBuiltinKeys.CONSUMER_ID, "OMS_CONSUMER")); + messagingAccessPoint.startup(); + System.out.printf("MessagingAccessPoint startup OK%n"); + final String queueName = "TopicTest"; + producer.startup(); + Message msg = producer.createBytesMessage(queueName, "Hello Open Messaging".getBytes()); + SendResult sendResult = producer.send(msg); + System.out.printf("Send Message OK. MsgId: %s%n", sendResult.messageId()); + producer.shutdown(); + consumer.attachQueue(queueName); + consumer.startup(); + System.out.printf("Consumer startup OK%n"); + // 运行直到发现一个消息被发送了 + boolean stop = false; + while (!stop) { + Message message = consumer.receive(); + if (message != null) { + String msgId = message.sysHeaders().getString(Message.BuiltinKeys.MESSAGE_ID); + System.out.printf("Received one message: %s%n", msgId); + consumer.ack(msgId); + if (!stop) { + stop = msgId.equalsIgnoreCase(sendResult.messageId()); + } + } else { + System.out.printf("Return without any message%n"); + } + } + consumer.shutdown(); + messagingAccessPoint.shutdown(); + } +} +``` + +### 8.3 OMSPushConsumer + +以下示范如何将 OMS PushConsumer 添加到指定的队列,并通过 MessageListener 消费这些消息。 + +```java +import io.openmessaging.Message; +import io.openmessaging.MessagingAccessPoint; +import io.openmessaging.OMS; +import io.openmessaging.OMSBuiltinKeys; +import io.openmessaging.consumer.MessageListener; +import io.openmessaging.consumer.PushConsumer; + +public class SimplePushConsumer { + public static void main(String[] args) { + final MessagingAccessPoint messagingAccessPoint = OMS + .getMessagingAccessPoint("oms:rocketmq://localhost:9876/default:default"); + final PushConsumer consumer = messagingAccessPoint. + createPushConsumer(OMS.newKeyValue().put(OMSBuiltinKeys.CONSUMER_ID, "OMS_CONSUMER")); + messagingAccessPoint.startup(); + System.out.printf("MessagingAccessPoint startup OK%n"); + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + consumer.shutdown(); + messagingAccessPoint.shutdown(); + } + })); + consumer.attachQueue("OMS_HELLO_TOPIC", new MessageListener() { + @Override + public void onReceived(Message message, Context context) { + System.out.printf("Received one message: %s%n", message.sysHeaders().getString(Message.BuiltinKeys.MESSAGE_ID)); + context.ack(); + } + }); + consumer.startup(); + System.out.printf("Consumer startup OK%n"); + } +} +``` \ No newline at end of file diff --git a/docs/cn/acl/user_guide.md b/docs/cn/acl/user_guide.md new file mode 100644 index 00000000000..deeb2fa56af --- /dev/null +++ b/docs/cn/acl/user_guide.md @@ -0,0 +1,82 @@ +# 权限控制 +## 前言 +该文档主要介绍如何快速部署和使用支持权限控制特性的RocketMQ 集群。 + +## 1.权限控制特性介绍 +权限控制(ACL)主要为RocketMQ提供Topic资源级别的用户访问控制。用户在使用RocketMQ权限控制时,可以在Client客户端通过 RPCHook注入AccessKey和SecretKey签名;同时,将对应的权限控制属性(包括Topic访问权限、IP白名单和AccessKey和SecretKey签名等)设置在distribution/conf/plain_acl.yml的配置文件中。Broker端对AccessKey所拥有的权限进行校验,校验不过,抛出异常; +ACL客户端可以参考:**org.apache.rocketmq.example.simple**包下面的**AclClient**代码。 + +## 2. 权限控制的定义与属性值 +### 2.1权限定义 +对RocketMQ的Topic资源访问权限控制定义主要如下表所示,分为以下四种 + + +| 权限 | 含义 | +| --- | --- | +| DENY | 拒绝 | +| ANY | PUB 或者 SUB 权限 | +| PUB | 发送权限 | +| SUB | 订阅权限 | + +### 2.2 权限定义的关键属性 +| 字段 | 取值 | 含义 | +| --- | --- | --- | +| globalWhiteRemoteAddresses | \*;192.168.\*.\*;192.168.0.1 | 全局IP白名单 | +| accessKey | 字符串 | Access Key | +| secretKey | 字符串 | Secret Key | +| whiteRemoteAddress | \*;192.168.\*.\*;192.168.0.1 | 用户IP白名单 | +| admin | true;false | 是否管理员账户 | +| defaultTopicPerm | DENY;PUB;SUB;PUB\|SUB | 默认的Topic权限 | +| defaultGroupPerm | DENY;PUB;SUB;PUB\|SUB | 默认的ConsumerGroup权限 | +| topicPerms | topic=权限 | 各个Topic的权限 | +| groupPerms | group=权限 | 各个ConsumerGroup的权限 | + +具体可以参考**distribution/conf/plain_acl.yml**配置文件 + +## 3. 支持权限控制的集群部署 +在**distribution/conf/plain_acl.yml**配置文件中按照上述说明定义好权限属性后,打开**aclEnable**开关变量即可开启RocketMQ集群的ACL特性。这里贴出Broker端开启ACL特性的properties配置文件内容: +``` +brokerClusterName=DefaultCluster +brokerName=broker-a +brokerId=0 +deleteWhen=04 +fileReservedTime=48 +brokerRole=ASYNC_MASTER +flushDiskType=ASYNC_FLUSH +storePathRootDir=/data/rocketmq/rootdir-a-m +storePathCommitLog=/data/rocketmq/commitlog-a-m +autoCreateSubscriptionGroup=true +## if acl is open,the flag will be true +aclEnable=true +listenPort=10911 +brokerIP1=XX.XX.XX.XX1 +namesrvAddr=XX.XX.XX.XX:9876 +``` + +## 4. 权限控制主要流程 +ACL主要流程分为两部分,主要包括权限解析和权限校验。 + +### 4.1 权限解析 +Broker端对客户端的RequestCommand请求进行解析,拿到需要鉴权的属性字段。 +主要包括: +(1)AccessKey:类似于用户名,代指用户主体,权限数据与之对应; +(2)Signature:客户根据 SecretKey 签名得到的串,服务端再用SecretKey进行签名验证; + +### 4.2 权限校验 +Broker端对权限的校验逻辑主要分为以下几步: +(1)检查是否命中全局 IP 白名单;如果是,则认为校验通过;否则走 2; +(2)检查是否命中用户 IP 白名单;如果是,则认为校验通过;否则走 3; +(3)校验签名,校验不通过,抛出异常;校验通过,则走 4; +(4)对用户请求所需的权限 和 用户所拥有的权限进行校验;不通过,抛出异常; +用户所需权限的校验需要注意已下内容: +(1)特殊的请求例如 UPDATE_AND_CREATE_TOPIC 等,只能由 admin 账户进行操作; +(2)对于某个资源,如果有显性配置权限,则采用配置的权限;如果没有显性配置权限,则采用默认的权限; + +## 5. 热加载修改后权限控制定义 +RocketrMQ的权限控制存储的默认实现是基于yml配置文件。用户可以动态修改权限控制定义的属性,而不需重新启动Broker服务节点。 + + + + + + diff --git a/docs/cn/architecture.md b/docs/cn/architecture.md index a26bcbdd085..12493a67eae 100644 --- a/docs/cn/architecture.md +++ b/docs/cn/architecture.md @@ -5,19 +5,18 @@ RocketMQ架构上主要分为四部分,如上图所示: +- Producer:消息发布的角色,支持分布式集群方式部署。Producer通过MQ的负载均衡模块选择相应的Broker集群队列进行消息投递,投递的过程支持快速失败并且低延迟。 -- Producer:消息发布的角色,支持分布式集群方式部署。producer通过MQ的负载均衡模块选择相应的Broker集群队列进行消息投递。投递的过程支持快速失败并且低延迟 +- Consumer:消息消费的角色,支持分布式集群方式部署。支持以push推,pull拉两种模式对消息进行消费。同时也支持集群方式和广播方式的消费,它提供实时消息订阅机制,可以满足大多数用户的需求。 -- Consumer:消息消费者的角色,支持分布式集群方式部署。支持以push推,pull拉两种模式对消息进行消费。同时也支持集群方式和广播形式的消费,它提供实时消息订阅机制,可以满足大多数用户的需求 +- NameServer:NameServer是一个非常简单的Topic路由注册中心,其角色类似Dubbo中的zookeeper,支持Broker的动态注册与发现。主要包括两个功能:Broker管理,NameServer接受Broker集群的注册信息并且保存下来作为路由信息的基本数据。然后提供心跳检测机制,检查Broker是否还存活;路由信息管理,每个NameServer将保存关于Broker集群的整个路由信息和用于客户端查询的队列信息。然后Producer和Conumser通过NameServer就可以知道整个Broker集群的路由信息,从而进行消息的投递和消费。NameServer通常也是集群的方式部署,各实例间相互不进行信息通讯。Broker是向每一台NameServer注册自己的路由信息,所以每一个NameServer实例上面都保存一份完整的路由信息。当某个NameServer因某种原因下线了,Broker仍然可以向其它NameServer同步其路由信息,Producer,Consumer仍然可以动态感知Broker的路由的信息。 -- NameServer:NameServer是一个非常简单的Topic路由注册中心,其角色类似dubbo中的zookeeper,支持Broker的动态注册与发现。主要包括两个功能:Broker管理,NameServer接受Broker集群的注册信息并且保存下来作为路由信息的基本数据。然后提供心跳检测机制,检查Broker是否还存活。路由信息管理。每个NameServer将保存关于Broker集群的整个路由信息和用于客户端查询的队列信息。然后Producer和Conumser通过NameServer就可以知道整个Broker集群的路由信息,从而进行消息的投递和消费。NameServer通常也是集群的方式部署,各实例间相互不进行信息通讯。Broker是向每一台NameServer注册自己的路由信息,所以每一个NameServer实例上面都保存一份完整的路由信息。当某个NameServer因某种原因下线了,Broker仍然可以向其它NameServer同步其路由信息,Produce,Consumer仍然可以动态感知Broker的路由的信息。 - -- BrokerServer:Broker主要负责消息的存储、投递和查询以及服务高可用保证,为了实现这些功能broker包含了以下几个重要子模块。 -1. Remoting Module:整个broker的实体,负责处理来自clients端的请求。 -2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的topic订阅信息 +- BrokerServer:Broker主要负责消息的存储、投递和查询以及服务高可用保证,为了实现这些功能,Broker包含了以下几个重要子模块。 +1. Remoting Module:整个Broker的实体,负责处理来自clients端的请求。 +2. Client Manager:负责管理客户端(Producer/Consumer)和维护Consumer的Topic订阅信息 3. Store Service:提供方便简单的API接口处理消息存储到物理硬盘和查询功能。 -4. HA Service:高可用服务,提供master broker 和 slave broker之间的数据同步功能。 -5. Index Service:根据特定的Message key对投递到broker的消息进行索引服务,以提供消息的快速查询。 +4. HA Service:高可用服务,提供Master Broker 和 Slave Broker之间的数据同步功能。 +5. Index Service:根据特定的Message key对投递到Broker的消息进行索引服务,以提供消息的快速查询。 ![](image/rocketmq_architecture_2.png) @@ -29,18 +28,18 @@ RocketMQ架构上主要分为四部分,如上图所示: ### RocketMQ 网络部署特点 -- NameServer 是一个几乎无状态节点,可集群部署,节点之间无任何信息同步。 +- NameServer是一个几乎无状态节点,可集群部署,节点之间无任何信息同步。 -- Broker 部署相对复杂,Broker 分为Master 与Slave,一个Master 可以对应多个Slave,但是一个Slave 只能对应一个Master,Master 与Slave 的对应关系通过指定相同的BrokerName,不同的BrokerId 来定义,BrokerId 为0 表示Master,非0 表示Slave。Master 也可以部署多个。每个Broker 与Name Server 集群中的所有节点建立长连接,定时注册Topic 信息到所有Name Server。 注意:当前RocketMQ版本在部署架构上支持一master多slave,但只有brokerId=1的从服务器才会参与消息的读负载。 +- Broker部署相对复杂,Broker分为Master与Slave,一个Master可以对应多个Slave,但是一个Slave只能对应一个Master,Master与Slave 的对应关系通过指定相同的BrokerName,不同的BrokerId 来定义,BrokerId为0表示Master,非0表示Slave。Master也可以部署多个。每个Broker与NameServer集群中的所有节点建立长连接,定时注册Topic信息到所有NameServer。 注意:当前RocketMQ版本在部署架构上支持一Master多Slave,但只有BrokerId=1的从服务器才会参与消息的读负载。 -- Producer与NameServer 集群中的其中一个节点(随机选择)建立长连接,定期从NameServer 获取Topic 路由信息,并向提供Topic 服务的Master 建立长连接,且定时向Master 发送心跳。Producer 完全无状态,可集群部署。 +- Producer与NameServer集群中的其中一个节点(随机选择)建立长连接,定期从NameServer获取Topic路由信息,并向提供Topic 服务的Master建立长连接,且定时向Master发送心跳。Producer完全无状态,可集群部署。 -- Consumer与NameServer 集群中的其中一个节点(随机选择)建立长连接,定期从NameServer取Topic路由信息,并向提供Topic服务的Master、Slave建立长连接,且定时向Master、Slave发送心跳。Consumer既可以从Master订阅消息,也可以从Slave订阅消息,消费者在向master拉取消息时Master服务器会根据拉取偏移量与最大偏移量的距离(判断是否读老消息,产生读IO),以及从服务器是否可读等因素建议下一次是从Master还是Slave拉取。 +- Consumer与NameServer集群中的其中一个节点(随机选择)建立长连接,定期从NameServer获取Topic路由信息,并向提供Topic服务的Master、Slave建立长连接,且定时向Master、Slave发送心跳。Consumer既可以从Master订阅消息,也可以从Slave订阅消息,消费者在向Master拉取消息时,Master服务器会根据拉取偏移量与最大偏移量的距离(判断是否读老消息,产生读I/O),以及从服务器是否可读等因素建议下一次是从Master还是Slave拉取。 -结合部署结构图,描述集群工作流程: +结合部署架构图,描述集群工作流程: -1. 启动Namesrv,Namesrv起来后监听端口,等待Broker、Produer、Consumer连上来,相当于一个路由控制中心。 -2. Broker启动,跟所有的Namesrv保持长连接,定时发送心跳包。心跳包中包含当前Broker信息(IP+端口等)以及存储所有topic信息。注册成功后,namesrv集群中就有Topic跟Broker的映射关系 -3. 收发消息前,先创建topic,创建topic时需要指定该topic要存储在哪些Broker上。也可以在发送消息时自动创建Topic -4. Producer发送消息,启动时先跟Namesrv集群中的其中一台建立长连接,并从Namesrv中获取当前发送的Topic存在哪些Broker上,轮询从队列列表中选择一个队列,然后与队列所在的Broker建立长连接从而向Broker发消息 -5. Consumer跟Producer类似。跟其中一台Namesrv建立长连接,获取当前订阅Topic存在哪些Broker上,然后直接跟Broker建立连接通道,开始消费消息 +- 启动NameServer,NameServer起来后监听端口,等待Broker、Producer、Consumer连上来,相当于一个路由控制中心。 +- Broker启动,跟所有的NameServer保持长连接,定时发送心跳包。心跳包中包含当前Broker信息(IP+端口等)以及存储所有Topic信息。注册成功后,NameServer集群中就有Topic跟Broker的映射关系。 +- 收发消息前,先创建Topic,创建Topic时需要指定该Topic要存储在哪些Broker上,也可以在发送消息时自动创建Topic。 +- Producer发送消息,启动时先跟NameServer集群中的其中一台建立长连接,并从NameServer中获取当前发送的Topic存在哪些Broker上,轮询从队列列表中选择一个队列,然后与队列所在的Broker建立长连接从而向Broker发消息。 +- Consumer跟Producer类似,跟其中一台NameServer建立长连接,获取当前订阅Topic存在哪些Broker上,然后直接跟Broker建立连接通道,开始消费消息。 diff --git a/docs/cn/best_practice.md b/docs/cn/best_practice.md new file mode 100755 index 00000000000..ffa76b95377 --- /dev/null +++ b/docs/cn/best_practice.md @@ -0,0 +1,374 @@ +# 最佳实践 + + +## 1 生产者 + +### 1.1 发送消息注意事项 + +#### 1 Tags的使用 + +一个应用尽可能用一个Topic,而消息子类型则可以用tags来标识。tags可以由应用自由设置,只有生产者在发送消息设置了tags,消费方在订阅消息时才可以利用tags通过broker做消息过滤:message.setTags("TagA")。 + +#### 2 Keys的使用 + +每个消息在业务层面的唯一标识码要设置到keys字段,方便将来定位消息丢失问题。服务器会为每个消息创建索引(哈希索引),应用可以通过topic、key来查询这条消息内容,以及消息被谁消费。由于是哈希索引,请务必保证key尽可能唯一,这样可以避免潜在的哈希冲突。 + + +```java + // 订单Id + String orderId = "20034568923546"; + message.setKeys(orderId); +``` +#### 3 日志的打印 + +​消息发送成功或者失败要打印消息日志,务必要打印SendResult和key字段。send消息方法只要不抛异常,就代表发送成功。发送成功会有多个状态,在sendResult里定义。以下对每个状态进行说明: + +- **SEND_OK** + +消息发送成功。要注意的是消息发送成功也不意味着它是可靠的。要确保不会丢失任何消息,还应启用同步Master服务器或同步刷盘,即SYNC_MASTER或SYNC_FLUSH。 + + +- **FLUSH_DISK_TIMEOUT** + +消息发送成功但是服务器刷盘超时。此时消息已经进入服务器队列(内存),只有服务器宕机,消息才会丢失。消息存储配置参数中可以设置刷盘方式和同步刷盘时间长度,如果Broker服务器设置了刷盘方式为同步刷盘,即FlushDiskType=SYNC_FLUSH(默认为异步刷盘方式),当Broker服务器未在同步刷盘时间内(默认为5s)完成刷盘,则将返回该状态——刷盘超时。 + +- **FLUSH_SLAVE_TIMEOUT** + +消息发送成功,但是服务器同步到Slave时超时。此时消息已经进入服务器队列,只有服务器宕机,消息才会丢失。如果Broker服务器的角色是同步Master,即SYNC_MASTER(默认是异步Master即ASYNC_MASTER),并且从Broker服务器未在同步刷盘时间(默认为5秒)内完成与主服务器的同步,则将返回该状态——数据同步到Slave服务器超时。 + +- **SLAVE_NOT_AVAILABLE** + +消息发送成功,但是此时Slave不可用。如果Broker服务器的角色是同步Master,即SYNC_MASTER(默认是异步Master服务器即ASYNC_MASTER),但没有配置slave Broker服务器,则将返回该状态——无Slave服务器可用。 + + +### 1.2 消息发送失败处理方式 + +Producer的send方法本身支持内部重试,重试逻辑如下: + +- 至多重试2次(同步发送为2次,异步发送为0次)。 +- 如果发送失败,则轮转到下一个Broker。这个方法的总耗时时间不超过sendMsgTimeout设置的值,默认10s。 +- 如果本身向broker发送消息产生超时异常,就不会再重试。 + +以上策略也是在一定程度上保证了消息可以发送成功。如果业务对消息可靠性要求比较高,建议应用增加相应的重试逻辑:比如调用send同步方法发送失败时,则尝试将消息存储到db,然后由后台线程定时重试,确保消息一定到达Broker。 + +上述db重试方式为什么没有集成到MQ客户端内部做,而是要求应用自己去完成,主要基于以下几点考虑:首先,MQ的客户端设计为无状态模式,方便任意的水平扩展,且对机器资源的消耗仅仅是cpu、内存、网络。其次,如果MQ客户端内部集成一个KV存储模块,那么数据只有同步落盘才能较可靠,而同步落盘本身性能开销较大,所以通常会采用异步落盘,又由于应用关闭过程不受MQ运维人员控制,可能经常会发生 kill -9 这样暴力方式关闭,造成数据没有及时落盘而丢失。第三,Producer所在机器的可靠性较低,一般为虚拟机,不适合存储重要数据。综上,建议重试过程交由应用来控制。 + +### 1.3选择oneway形式发送 +通常消息的发送是这样一个过程: + +- 客户端发送请求到服务器 +- 服务器处理请求 +- 服务器向客户端返回应答 + +所以,一次消息发送的耗时时间是上述三个步骤的总和,而某些场景要求耗时非常短,但是对可靠性要求并不高,例如日志收集类应用,此类应用可以采用oneway形式调用,oneway形式只发送请求不等待应答,而发送请求在客户端实现层面仅仅是一个操作系统系统调用的开销,即将数据写入客户端的socket缓冲区,此过程耗时通常在微秒级。 + + +## 2 消费者 + +### 2.1 消费过程幂等 + +RocketMQ无法避免消息重复(Exactly-Once),所以如果业务对消费重复非常敏感,务必要在业务层面进行去重处理。可以借助关系数据库进行去重。首先需要确定消息的唯一键,可以是msgId,也可以是消息内容中的唯一标识字段,例如订单Id等。在消费之前判断唯一键是否在关系数据库中存在。如果不存在则插入,并消费,否则跳过。(实际过程要考虑原子性问题,判断是否存在可以尝试插入,如果报主键冲突,则插入失败,直接跳过) + +msgId一定是全局唯一标识符,但是实际使用中,可能会存在相同的消息有两个不同msgId的情况(消费者主动重发、因客户端重投机制导致的重复等),这种情况就需要使业务字段进行重复消费。 + +### 2.2 消费速度慢的处理方式 + +#### 1 提高消费并行度 + +绝大部分消息消费行为都属于 IO 密集型,即可能是操作数据库,或者调用 RPC,这类消费行为的消费速度在于后端数据库或者外系统的吞吐量,通过增加消费并行度,可以提高总的消费吞吐量,但是并行度增加到一定程度,反而会下降。所以,应用必须要设置合理的并行度。 如下有几种修改消费并行度的方法: + +- 同一个 ConsumerGroup 下,通过增加 Consumer 实例数量来提高并行度(需要注意的是超过订阅队列数的 Consumer 实例无效)。可以通过加机器,或者在已有机器启动多个进程的方式。 +- 提高单个 Consumer 的消费并行线程,通过修改参数 consumeThreadMin、consumeThreadMax实现。 + +#### 2 批量方式消费 + +某些业务流程如果支持批量方式消费,则可以很大程度上提高消费吞吐量,例如订单扣款类应用,一次处理一个订单耗时 1 s,一次处理 10 个订单可能也只耗时 2 s,这样即可大幅度提高消费的吞吐量,通过设置 consumer的 consumeMessageBatchMaxSize 返个参数,默认是 1,即一次只消费一条消息,例如设置为 N,那么每次消费的消息数小于等于 N。 + +#### 3 跳过非重要消息 + +发生消息堆积时,如果消费速度一直追不上发送速度,如果业务对数据要求不高的话,可以选择丢弃不重要的消息。例如,当某个队列的消息数堆积到100000条以上,则尝试丢弃部分或全部消息,这样就可以快速追上发送消息的速度。示例代码如下: + +```java + public ConsumeConcurrentlyStatus consumeMessage( + List msgs, + ConsumeConcurrentlyContext context) { + long offset = msgs.get(0).getQueueOffset(); + String maxOffset = + msgs.get(0).getProperty(Message.PROPERTY_MAX_OFFSET); + long diff = Long.parseLong(maxOffset) - offset; + if (diff > 100000) { + // TODO 消息堆积情况的特殊处理 + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + // TODO 正常消费过程 + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } +``` + + +#### 4 优化每条消息消费过程 + +举例如下,某条消息的消费过程如下: + +- 根据消息从 DB 查询【数据 1】 +- 根据消息从 DB 查询【数据 2】 +- 复杂的业务计算 +- 向 DB 插入【数据 3】 +- 向 DB 插入【数据 4】 + +这条消息的消费过程中有4次与 DB的 交互,如果按照每次 5ms 计算,那么总共耗时 20ms,假设业务计算耗时 5ms,那么总过耗时 25ms,所以如果能把 4 次 DB 交互优化为 2 次,那么总耗时就可以优化到 15ms,即总体性能提高了 40%。所以应用如果对时延敏感的话,可以把DB部署在SSD硬盘,相比于SCSI磁盘,前者的RT会小很多。 + +### 2.3 消费打印日志 + +如果消息量较少,建议在消费入口方法打印消息,消费耗时等,方便后续排查问题。 + + +```java + public ConsumeConcurrentlyStatus consumeMessage( + List msgs, + ConsumeConcurrentlyContext context) { + log.info("RECEIVE_MSG_BEGIN: " + msgs.toString()); + // TODO 正常消费过程 + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } +``` + +如果能打印每条消息消费耗时,那么在排查消费慢等线上问题时,会更方便。 + +### 2.4 其他消费建议 + +#### 1 关于消费者和订阅 + +​第一件需要注意的事情是,不同的消费者组可以独立的消费一些 topic,并且每个消费者组都有自己的消费偏移量,请确保同一组内的每个消费者订阅信息保持一致。 + +#### 2 关于有序消息 + +消费者将锁定每个消息队列,以确保他们被逐个消费,虽然这将会导致性能下降,但是当你关心消息顺序的时候会很有用。我们不建议抛出异常,你可以返回 ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT 作为替代。 + +#### 3 关于并发消费 + +顾名思义,消费者将并发消费这些消息,建议你使用它来获得良好性能,我们不建议抛出异常,你可以返回 ConsumeConcurrentlyStatus.RECONSUME_LATER 作为替代。 + +#### 4 关于消费状态Consume Status + +对于并发的消费监听器,你可以返回 RECONSUME_LATER 来通知消费者现在不能消费这条消息,并且希望可以稍后重新消费它。然后,你可以继续消费其他消息。对于有序的消息监听器,因为你关心它的顺序,所以不能跳过消息,但是你可以返回SUSPEND_CURRENT_QUEUE_A_MOMENT 告诉消费者等待片刻。 + +#### 5 关于Blocking + +不建议阻塞监听器,因为它会阻塞线程池,并最终可能会终止消费进程 + +#### 6 关于线程数设置 + +消费者使用 ThreadPoolExecutor 在内部对消息进行消费,所以你可以通过设置 setConsumeThreadMin 或 setConsumeThreadMax 来改变它。 + +#### 7 关于消费位点 + +当建立一个新的消费者组时,需要决定是否需要消费已经存在于 Broker 中的历史消息CONSUME_FROM_LAST_OFFSET 将会忽略历史消息,并消费之后生成的任何消息。CONSUME_FROM_FIRST_OFFSET 将会消费每个存在于 Broker 中的信息。你也可以使用 CONSUME_FROM_TIMESTAMP 来消费在指定时间戳后产生的消息。 + + + + + +## 3 Broker + +### 3.1 Broker 角色 +​ Broker 角色分为 ASYNC_MASTER(异步主机)、SYNC_MASTER(同步主机)以及SLAVE(从机)。如果对消息的可靠性要求比较严格,可以采用 SYNC_MASTER加SLAVE的部署方式。如果对消息可靠性要求不高,可以采用ASYNC_MASTER加SLAVE的部署方式。如果只是测试方便,则可以选择仅ASYNC_MASTER或仅SYNC_MASTER的部署方式。 +### 3.2 FlushDiskType +​ SYNC_FLUSH(同步刷新)相比于ASYNC_FLUSH(异步处理)会损失很多性能,但是也更可靠,所以需要根据实际的业务场景做好权衡。 +### 3.3 Broker 配置 + +| 参数名 | 默认值 | 说明 | +| -------------------------------- | ----------------------------- | ------------------------------------------------------------ | +| listenPort | 10911 | 接受客户端连接的监听端口 | +| namesrvAddr | null | nameServer 地址 | +| brokerIP1 | 网卡的 InetAddress | 当前 broker 监听的 IP | +| brokerIP2 | 跟 brokerIP1 一样 | 存在主从 broker 时,如果在 broker 主节点上配置了 brokerIP2 属性,broker 从节点会连接主节点配置的 brokerIP2 进行同步 | +| brokerName | null | broker 的名称 | +| brokerClusterName | DefaultCluster | 本 broker 所属的 Cluser 名称 | +| brokerId | 0 | broker id, 0 表示 master, 其他的正整数表示 slave | +| storePathCommitLog | $HOME/store/commitlog/ | 存储 commit log 的路径 | +| storePathConsumerQueue | $HOME/store/consumequeue/ | 存储 consume queue 的路径 | +| mapedFileSizeCommitLog | 1024 * 1024 * 1024(1G) | commit log 的映射文件大小 |​ +| deleteWhen | 04 | 在每天的什么时间删除已经超过文件保留时间的 commit log |​ +| fileReserverdTime | 72 | 以小时计算的文件保留时间 |​ +| brokerRole | ASYNC_MASTER | SYNC_MASTER/ASYNC_MASTER/SLAVE |​ +| flushDiskType | ASYNC_FLUSH | SYNC_FLUSH/ASYNC_FLUSH SYNC_FLUSH 模式下的 broker 保证在收到确认生产者之前将消息刷盘。ASYNC_FLUSH 模式下的 broker 则利用刷盘一组消息的模式,可以取得更好的性能。 |​ + +## 4 NameServer + +​RocketMQ 中,Name Servers 被设计用来做简单的路由管理。其职责包括: + +- Brokers 定期向每个名称服务器注册路由数据。 +- 名称服务器为客户端,包括生产者,消费者和命令行客户端提供最新的路由信息。 +​ +​ + +## 5 客户端配置 + +​ 相对于RocketMQ的Broker集群,生产者和消费者都是客户端。本小节主要描述生产者和消费者公共的行为配置。 + +### 5.1 客户端寻址方式 + +RocketMQ可以令客户端找到Name Server, 然后通过Name Server再找到Broker。如下所示有多种配置方式,优先级由高到低,高优先级会覆盖低优先级。 + +- 代码中指定Name Server地址,多个namesrv地址之间用分号分割 + +```java +producer.setNamesrvAddr("192.168.0.1:9876;192.168.0.2:9876"); + +consumer.setNamesrvAddr("192.168.0.1:9876;192.168.0.2:9876"); +``` +- Java启动参数中指定Name Server地址 + +```text +-Drocketmq.namesrv.addr=192.168.0.1:9876;192.168.0.2:9876 +``` +- 环境变量指定Name Server地址 + +```text +export NAMESRV_ADDR=192.168.0.1:9876;192.168.0.2:9876 +``` +- HTTP静态服务器寻址(默认) + +客户端启动后,会定时访问一个静态HTTP服务器,地址如下:,这个URL的返回内容如下: + +```text +192.168.0.1:9876;192.168.0.2:9876 +``` +客户端默认每隔2分钟访问一次这个HTTP服务器,并更新本地的Name Server地址。URL已经在代码中硬编码,可通过修改/etc/hosts文件来改变要访问的服务器,例如在/etc/hosts增加如下配置: +```text +10.232.22.67 jmenv.taobao.net +``` +推荐使用HTTP静态服务器寻址方式,好处是客户端部署简单,且Name Server集群可以热升级。 + +### 5.2 客户端配置 + +DefaultMQProducer、TransactionMQProducer、DefaultMQPushConsumer、DefaultMQPullConsumer都继承于ClientConfig类,ClientConfig为客户端的公共配置类。客户端的配置都是get、set形式,每个参数都可以用spring来配置,也可以在代码中配置,例如namesrvAddr这个参数可以这样配置,producer.setNamesrvAddr("192.168.0.1:9876"),其他参数同理。 + +#### 1 客户端的公共配置 + +| 参数名 | 默认值 | 说明 | +| ----------------------------- | ------- | ------------------------------------------------------------ | +| namesrvAddr | | Name Server地址列表,多个NameServer地址用分号隔开 | +| clientIP | 本机IP | 客户端本机IP地址,某些机器会发生无法识别客户端IP地址情况,需要应用在代码中强制指定 | +| instanceName | DEFAULT | 客户端实例名称,客户端创建的多个Producer、Consumer实际是共用一个内部实例(这个实例包含网络连接、线程资源等) | +| clientCallbackExecutorThreads | 4 | 通信层异步回调线程数 | +| pollNameServerInteval | 30000 | 轮询Name Server间隔时间,单位毫秒 | +| heartbeatBrokerInterval | 30000 | 向Broker发送心跳间隔时间,单位毫秒 | +| persistConsumerOffsetInterval | 5000 | 持久化Consumer消费进度间隔时间,单位毫秒 | + +#### 2 Producer配置 + +| 参数名 | 默认值 | 说明 | +| -------------------------------- | ---------------- | ------------------------------------------------------------ | +| producerGroup | DEFAULT_PRODUCER | Producer组名,多个Producer如果属于一个应用,发送同样的消息,则应该将它们归为同一组 | +| createTopicKey | TBW102 | 在发送消息时,自动创建服务器不存在的topic,需要指定Key,该Key可用于配置发送消息所在topic的默认路由。 | +| defaultTopicQueueNums | 4 | 在发送消息,自动创建服务器不存在的topic时,默认创建的队列数 | +| sendMsgTimeout | 10000 | 发送消息超时时间,单位毫秒 | +| compressMsgBodyOverHowmuch | 4096 | 消息Body超过多大开始压缩(Consumer收到消息会自动解压缩),单位字节 | +| retryAnotherBrokerWhenNotStoreOK | FALSE | 如果发送消息返回sendResult,但是sendStatus!=SEND_OK,是否重试发送 | +| retryTimesWhenSendFailed | 2 | 如果消息发送失败,最大重试次数,该参数只对同步发送模式起作用 | +| maxMessageSize | 4MB | 客户端限制的消息大小,超过报错,同时服务端也会限制,所以需要跟服务端配合使用。 | +| transactionCheckListener | | 事务消息回查监听器,如果发送事务消息,必须设置 | +| checkThreadPoolMinSize | 1 | Broker回查Producer事务状态时,线程池最小线程数 | +| checkThreadPoolMaxSize | 1 | Broker回查Producer事务状态时,线程池最大线程数 | +| checkRequestHoldMax | 2000 | Broker回查Producer事务状态时,Producer本地缓冲请求队列大小 | +| RPCHook | null | 该参数是在Producer创建时传入的,包含消息发送前的预处理和消息响应后的处理两个接口,用户可以在第一个接口中做一些安全控制或者其他操作。 | + +#### 3 PushConsumer配置 + +| 参数名 | 默认值 | 说明 | +| ---------------------------- | ----------------------------- | ------------------------------------------------------------ | +| consumerGroup | DEFAULT_CONSUMER | Consumer组名,多个Consumer如果属于一个应用,订阅同样的消息,且消费逻辑一致,则应该将它们归为同一组 | +| messageModel | CLUSTERING | 消费模型支持集群消费和广播消费两种 | +| consumeFromWhere | CONSUME_FROM_LAST_OFFSET | Consumer启动后,默认从上次消费的位置开始消费,这包含两种情况:一种是上次消费的位置未过期,则消费从上次中止的位置进行;一种是上次消费位置已经过期,则从当前队列第一条消息开始消费 | +| consumeTimestamp | 半个小时前 | 只有当consumeFromWhere值为CONSUME_FROM_TIMESTAMP时才起作用。 | +| allocateMessageQueueStrategy | AllocateMessageQueueAveragely | Rebalance算法实现策略 | +| subscription | | 订阅关系 | +| messageListener | | 消息监听器 | +| offsetStore | | 消费进度存储 | +| consumeThreadMin | 10 | 消费线程池最小线程数 | +| consumeThreadMax | 20 | 消费线程池最大线程数 | +| | | | +| consumeConcurrentlyMaxSpan | 2000 | 单队列并行消费允许的最大跨度 | +| pullThresholdForQueue | 1000 | 拉消息本地队列缓存消息最大数 | +| pullInterval | 0 | 拉消息间隔,由于是长轮询,所以为0,但是如果应用为了流控,也可以设置大于0的值,单位毫秒 | +| consumeMessageBatchMaxSize | 1 | 批量消费,一次消费多少条消息 | +| pullBatchSize | 32 | 批量拉消息,一次最多拉多少条 | + +#### 4 PullConsumer配置 + +| 参数名 | 默认值 | 说明 | +| -------------------------------- | ----------------------------- | ------------------------------------------------------------ | +| consumerGroup | DEFAULT_CONSUMER | Consumer组名,多个Consumer如果属于一个应用,订阅同样的消息,且消费逻辑一致,则应该将它们归为同一组 | +| brokerSuspendMaxTimeMillis | 20000 | 长轮询,Consumer拉消息请求在Broker挂起最长时间,单位毫秒 | +| consumerTimeoutMillisWhenSuspend | 30000 | 长轮询,Consumer拉消息请求在Broker挂起超过指定时间,客户端认为超时,单位毫秒 | +| consumerPullTimeoutMillis | 10000 | 非长轮询,拉消息超时时间,单位毫秒 | +| messageModel | BROADCASTING | 消息支持两种模式:集群消费和广播消费 | +| messageQueueListener | | 监听队列变化 | +| offsetStore | | 消费进度存储 | +| registerTopics | | 注册的topic集合 | +| allocateMessageQueueStrategy | AllocateMessageQueueAveragely | Rebalance算法实现策略 | + +#### 5 Message数据结构 + +| 字段名 | 默认值 | 说明 | +| -------------- | ------ | ------------------------------------------------------------ | +| Topic | null | 必填,消息所属topic的名称 | +| Body | null | 必填,消息体 | +| Tags | null | 选填,消息标签,方便服务器过滤使用。目前只支持每个消息设置一个tag | +| Keys | null | 选填,代表这条消息的业务关键词,服务器会根据keys创建哈希索引,设置后,可以在Console系统根据Topic、Keys来查询消息,由于是哈希索引,请尽可能保证key唯一,例如订单号,商品Id等。 | +| Flag | 0 | 选填,完全由应用来设置,RocketMQ不做干预 | +| DelayTimeLevel | 0 | 选填,消息延时级别,0表示不延时,大于0会延时特定的时间才会被消费 | +| WaitStoreMsgOK | TRUE | 选填,表示消息是否在服务器落盘后才返回应答。 | + +## 6 系统配置 + +本小节主要介绍系统(JVM/OS)相关的配置。 + +### 6.1 JVM选项 + +​ 推荐使用最新发布的JDK 1.8版本。通过设置相同的Xms和Xmx值来防止JVM调整堆大小以获得更好的性能。简单的JVM配置如下所示: +​ +​``` +​ +​-server -Xms8g -Xmx8g -Xmn4g +​ ``` +​ +​ +如果您不关心RocketMQ Broker的启动时间,还有一种更好的选择,就是通过“预触摸”Java堆以确保在JVM初始化期间每个页面都将被分配。那些不关心启动时间的人可以启用它: +​ -XX:+AlwaysPreTouch +禁用偏置锁定可能会减少JVM暂停, +​ -XX:-UseBiasedLocking +至于垃圾回收,建议使用带JDK 1.8的G1收集器。 + +```text +-XX:+UseG1GC -XX:G1HeapRegionSize=16m +-XX:G1ReservePercent=25 +-XX:InitiatingHeapOccupancyPercent=30 +``` + +​ 这些GC选项看起来有点激进,但事实证明它在我们的生产环境中具有良好的性能。另外不要把-XX:MaxGCPauseMillis的值设置太小,否则JVM将使用一个小的年轻代来实现这个目标,这将导致非常频繁的minor GC,所以建议使用rolling GC日志文件: + +```text +-XX:+UseGCLogFileRotation +-XX:NumberOfGCLogFiles=5 +-XX:GCLogFileSize=30m +``` + +如果写入GC文件会增加代理的延迟,可以考虑将GC日志文件重定向到内存文件系统: + +```text +-Xloggc:/dev/shm/mq_gc_%p.log123 +``` +### 6.2 Linux内核参数 + +​ os.sh脚本在bin文件夹中列出了许多内核参数,可以进行微小的更改然后用于生产用途。下面的参数需要注意,更多细节请参考/proc/sys/vm/*的[文档](https://www.kernel.org/doc/Documentation/sysctl/vm.txt) + +- **vm.extra_free_kbytes**,告诉VM在后台回收(kswapd)启动的阈值与直接回收(通过分配进程)的阈值之间保留额外的可用内存。RocketMQ使用此参数来避免内存分配中的长延迟。(与具体内核版本相关) +- **vm.min_free_kbytes**,如果将其设置为低于1024KB,将会巧妙的将系统破坏,并且系统在高负载下容易出现死锁。 +- **vm.max_map_count**,限制一个进程可能具有的最大内存映射区域数。RocketMQ将使用mmap加载CommitLog和ConsumeQueue,因此建议将为此参数设置较大的值。(agressiveness --> aggressiveness) +- **vm.swappiness**,定义内核交换内存页面的积极程度。较高的值会增加攻击性,较低的值会减少交换量。建议将值设置为10来避免交换延迟。 +- **File descriptor limits**,RocketMQ需要为文件(CommitLog和ConsumeQueue)和网络连接打开文件描述符。我们建议设置文件描述符的值为655350。 +- [Disk scheduler](https://access.redhat.com/documentation/en-US/Red_Hat_Enterprise_Linux/6/html/Performance_Tuning_Guide/ch06s04s02.html),RocketMQ建议使用I/O截止时间调度器,它试图为请求提供有保证的延迟。 +[]([]()) diff --git a/docs/cn/client/java/API_Reference_DefaultMQProducer.md b/docs/cn/client/java/API_Reference_DefaultMQProducer.md new file mode 100644 index 00000000000..c45f7d4e50c --- /dev/null +++ b/docs/cn/client/java/API_Reference_DefaultMQProducer.md @@ -0,0 +1,1088 @@ +## DefaultMQProducer +--- +### 类简介 + +`public class DefaultMQProducer +extends ClientConfig +implements MQProducer` + +>`DefaultMQProducer`类是应用用来投递消息的入口,开箱即用,可通过无参构造方法快速创建一个生产者。主要负责消息的发送,支持同步/异步/oneway的发送方式,这些发送方式均支持批量发送。可以通过该类提供的getter/setter方法,调整发送者的参数。`DefaultMQProducer`提供了多个send方法,每个send方法略有不同,在使用前务必详细了解其意图。下面给出一个生产者示例代码,[点击查看更多示例代码](https://github.com/apache/rocketmq/blob/master/example/src/main/java/org/apache/rocketmq/example/)。 + +``` java +public class Producer { + public static void main(String[] args) throws MQClientException { + // 创建指定分组名的生产者 + DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName"); + + // 启动生产者 + producer.start(); + + for (int i = 0; i < 128; i++) + try { + // 构建消息 + Message msg = new Message("TopicTest", + "TagA", + "OrderID188", + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + + // 同步发送 + SendResult sendResult = producer.send(msg); + + // 打印发送结果 + System.out.printf("%s%n", sendResult); + } catch (Exception e) { + e.printStackTrace(); + } + + producer.shutdown(); + } +} +``` + +**注意**:该类是线程安全的。在配置并启动完成后可在多个线程间安全共享。 + +### 字段摘要 +|类型|字段名称|描述| +|------|-------|-------| +|DefaultMQProducerImpl|defaultMQProducerImpl|生产者的内部默认实现| +|String|producerGroup|生产者分组| +|String|createTopicKey|在发送消息时,自动创建服务器不存在的topic| +|int|defaultTopicQueueNums|创建topic时默认的队列数量| +|int|sendMsgTimeout|发送消息的超时时间| +|int|compressMsgBodyOverHowmuch|压缩消息体的阈值| +|int|retryTimesWhenSendFailed|同步模式下内部尝试发送消息的最大次数| +|int|retryTimesWhenSendAsyncFailed|异步模式下内部尝试发送消息的最大次数| +|boolean|retryAnotherBrokerWhenNotStoreOK|是否在内部发送失败时重试另一个broker| +|int|maxMessageSize|消息的最大长度| +|TraceDispatcher|traceDispatcher|消息追踪器。使用rcpHook来追踪消息| + +### 构造方法摘要 + +|方法名称|方法描述| +|-------|------------| +|DefaultMQProducer()|由默认参数值创建一个生产者 | +|DefaultMQProducer(final String producerGroup)|使用指定的分组名创建一个生产者| +|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace)|使用指定的分组名创建一个生产者,并设置是否开启消息追踪| +|DefaultMQProducer(final String producerGroup, boolean enableMsgTrace, final String customizedTraceTopic)|使用指定的分组名创建一个生产者,并设置是否开启消息追踪及追踪topic的名称| +|DefaultMQProducer(RPCHook rpcHook)|使用指定的hook创建一个生产者| +|DefaultMQProducer(final String producerGroup, RPCHook rpcHook)|使用指定的分组名及自定义hook创建一个生产者| +|DefaultMQProducer(final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace,final String customizedTraceTopic)|使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息追踪及追踪topic的名称| + +### 使用方法摘要 + +|返回值|方法名称|方法描述| +|-------|-------|------------| +|void|createTopic(String key, String newTopic, int queueNum)|在broker上创建指定的topic| +|void|createTopic(String key, String newTopic, int queueNum, int topicSysFlag)|在broker上创建指定的topic| +|long|earliestMsgStoreTime(MessageQueue mq)|查询最早的消息存储时间| +|List|fetchPublishMessageQueues(String topic)|获取topic的消息队列| +|long|maxOffset(MessageQueue mq)|查询给定消息队列的最大offset| +|long|minOffset(MessageQueue mq)|查询给定消息队列的最小offset| +|QueryResult|queryMessage(String topic, String key, int maxNum, long begin, long end)|按关键字查询消息| +|long|searchOffset(MessageQueue mq, long timestamp)|查找指定时间的消息队列的物理offset| +|SendResult|send(Collection msgs)|同步批量发送消息| +|SendResult|send(Collection msgs, long timeout)|同步批量发送消息| +|SendResult|send(Collection msgs, MessageQueue messageQueue)|向指定的消息队列同步批量发送消息| +|SendResult|send(Collection msgs, MessageQueue messageQueue, long timeout)|向指定的消息队列同步批量发送消息,并指定超时时间| +|SendResult|send(Message msg)|同步单条发送消息| +|SendResult|send(Message msg, long timeout)|同步发送单条消息,并指定超时时间| +|SendResult|send(Message msg, MessageQueue mq)|向指定的消息队列同步发送单条消息| +|SendResult|send(Message msg, MessageQueue mq, long timeout)|向指定的消息队列同步单条发送消息,并指定超时时间| +|void|send(Message msg, MessageQueue mq, SendCallback sendCallback)|向指定的消息队列异步单条发送消息,并指定回调方法| +|void|send(Message msg, MessageQueue mq, SendCallback sendCallback, long timeout)|向指定的消息队列异步单条发送消息,并指定回调方法和超时时间| +|SendResult|send(Message msg, MessageQueueSelector selector, Object arg)|向消息队列同步单条发送消息,并指定发送队列选择器| +|SendResult|send(Message msg, MessageQueueSelector selector, Object arg, long timeout)|向消息队列同步单条发送消息,并指定发送队列选择器与超时时间| +|void|send(Message msg, MessageQueueSelector selector, Object arg, SendCallback sendCallback)|向指定的消息队列异步单条发送消息| +|void|send(Message msg, MessageQueueSelector selector, Object arg, SendCallback sendCallback, long timeout)|向指定的消息队列异步单条发送消息,并指定超时时间| +|void|send(Message msg, SendCallback sendCallback)|异步发送消息| +|void|send(Message msg, SendCallback sendCallback, long timeout)|异步发送消息,并指定回调方法和超时时间| +|TransactionSendResult|sendMessageInTransaction(Message msg, LocalTransactionExecuter tranExecuter, final Object arg)|发送事务消息,并指定本地执行事务实例| +|TransactionSendResult|sendMessageInTransaction(Message msg, Object arg)|发送事务消息| +|void|sendOneway(Message msg)|单向发送消息,不等待broker响应| +|void|sendOneway(Message msg, MessageQueue mq) |单向发送消息到指定队列,不等待broker响应| +|void|sendOneway(Message msg, MessageQueueSelector selector, Object arg)|单向发送消息到队列选择器的选中的队列,不等待broker响应| +|void|shutdown()|关闭当前生产者实例并释放相关资源| +|void|start()|启动生产者| +|MessageExt|viewMessage(String offsetMsgId)|根据给定的msgId查询消息| +|MessageExt|public MessageExt viewMessage(String topic, String msgId)|根据给定的msgId查询消息,并指定topic| + +### 字段详细信息 + +- [producerGroup](http://rocketmq.apache.org/docs/core-concept/) + + `private String producerGroup` + + 生产者的分组名称。相同的分组名称表明生产者实例在概念上归属于同一分组。这对事务消息十分重要,如果原始生产者在事务之后崩溃,那么broker可以联系同一生产者分组的不同生产者实例来提交或回滚事务。 + + 默认值:DEFAULT_PRODUCER + + 注意: 由数字、字母、下划线、横杠(-)、竖线(|)或百分号组成;不能为空;长度不能超过255。 + +- defaultMQProducerImpl + + `protected final transient DefaultMQProducerImpl defaultMQProducerImpl` + + 生产者的内部默认实现,在构造生产者时内部自动初始化,提供了大部分方法的内部实现。 + +- createTopicKey + + `private String createTopicKey = MixAll.AUTO_CREATE_TOPIC_KEY_TOPIC` + + 在发送消息时,自动创建服务器不存在的topic,需要指定Key,该Key可用于配置发送消息所在topic的默认路由。 + + 默认值:TBW102 + + 建议:测试或者demo使用,生产环境下不建议打开自动创建配置。 + +- defaultTopicQueueNums + + `private volatile int defaultTopicQueueNums = 4` + + 创建topic时默认的队列数量。 + + 默认值:4 + +- sendMsgTimeout + + `private int sendMsgTimeout = 3000` + + 发送消息时的超时时间。 + + 默认值:3000,单位:毫秒 + + 建议:不建议修改该值,该值应该与broker配置中的sendTimeout一致,发送超时,可临时修改该值,建议解决超时问题,提高broker集群的Tps。 + +- compressMsgBodyOverHowmuch + + `private int compressMsgBodyOverHowmuch = 1024 * 4` + + 压缩消息体阈值。大于4K的消息体将默认进行压缩。 + + 默认值:1024 * 4,单位:字节 + + 建议:可通过DefaultMQProducerImpl.setZipCompressLevel方法设置压缩率(默认为5,可选范围[0,9]);可通过DefaultMQProducerImpl.tryToCompressMessage方法测试出compressLevel与compressMsgBodyOverHowmuch最佳值。 + +- retryTimesWhenSendFailed + + `private int retryTimesWhenSendFailed = 2` + + 同步模式下,在返回发送失败之前,内部尝试重新发送消息的最大次数。 + + 默认值:2,即:默认情况下一条消息最多会被投递3次。 + + 注意:在极端情况下,这可能会导致消息的重复。 + +- retryTimesWhenSendAsyncFailed + + `private int retryTimesWhenSendAsyncFailed = 2` + + 异步模式下,在发送失败之前,内部尝试重新发送消息的最大次数。 + + 默认值:2,即:默认情况下一条消息最多会被投递3次。 + + 注意:在极端情况下,这可能会导致消息的重复。 + +- retryAnotherBrokerWhenNotStoreOK + + `private boolean retryAnotherBrokerWhenNotStoreOK = false` + + 同步模式下,消息保存失败时是否重试其他broker。 + + 默认值:false + + 注意:此配置关闭时,非投递时产生异常情况下,会忽略retryTimesWhenSendFailed配置。 + +- maxMessageSize + + `private int maxMessageSize = 1024 * 1024 * 4` + + 消息的最大大小。当消息题的字节数超过maxMessageSize就发送失败。 + + 默认值:1024 * 1024 * 4,单位:字节 + +- [traceDispatcher](https://github.com/apache/rocketmq/wiki/RIP-6-Message-Trace) + + `private TraceDispatcher traceDispatcher = null` + + 在开启消息追踪后,该类通过hook的方式把消息生产者,消息存储的broker和消费者消费消息的信息像链路一样记录下来。在构造生产者时根据构造入参enableMsgTrace来决定是否创建该对象。 + +### 构造方法详细信息 + +1. DefaultMQProducer + + `public DefaultMQProducer()` + + 创建一个新的生产者。 + +2. DefaultMQProducer + + `DefaultMQProducer(final String producerGroup)` + + 使用指定的分组名创建一个生产者。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 + +3. DefaultMQProducer + + `DefaultMQProducer(final String producerGroup, boolean enableMsgTrace)` + + 使用指定的分组名创建一个生产者,并设置是否开启消息追踪。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 + enableMsgTrace | boolean | 是 | false |是否开启消息追踪 + +4. DefaultMQProducer + + `DefaultMQProducer(final String producerGroup, boolean enableMsgTrace, final String customizedTraceTopic)` + + 使用指定的分组名创建一个生产者,并设置是否开启消息追踪及追踪topic的名称。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 + rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook + enableMsgTrace | boolean | 是 | false |是否开启消息追踪 + customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息跟踪topic的名称 + +5. DefaultMQProducer + + `DefaultMQProducer(RPCHook rpcHook)` + + 使用指定的hook创建一个生产者。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook + +6. DefaultMQProducer + + `DefaultMQProducer(final String producerGroup, RPCHook rpcHook)` + + 使用指定的分组名及自定义hook创建一个生产者。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 + rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook + +7. DefaultMQProducer + + `DefaultMQProducer(final String producerGroup, RPCHook rpcHook, boolean enableMsgTrace,final String customizedTraceTopic)` + + 使用指定的分组名及自定义hook创建一个生产者,并设置是否开启消息追踪及追踪topic的名称。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 缺省值 |描述 + ---|---|---|---|--- + producerGroup | String | 是 | DEFAULT_PRODUCER | 生产者的分组名称 + rpcHook | RPCHook | 否 | null |每个远程命令执行后会回调rpcHook + enableMsgTrace | boolean | 是 | false |是否开启消息追踪 + customizedTraceTopic | String | 否 | RMQ_SYS_TRACE_TOPIC | 消息跟踪topic的名称 + +### 使用方法详细信息 + +1. createTopic + + `public void createTopic(String key, String newTopic, int queueNum)` + + 在broker上创建一个topic。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 |值范围 | 说明 + ---|---|---|---|---|--- + key | String | 是 | | | 访问密钥。 + newTopic | String | 是 | | | 新建topic的名称。由数字、字母、下划线(_)、横杠(-)、竖线(|)或百分号(%)组成;
长度小于255;不能为TBW102或空。 + queueNum | int | 是 | 0 | (0, maxIntValue] | topic的队列数量。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - 生产者状态非Running;未找到broker等客户端异常。 + +2. createTopic + + `public void createTopic(String key, String newTopic, int queueNum, int topicSysFlag)` + + 在broker上创建一个topic。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 |值范围 | 说明 + ---|---|---|---|---|--- + key | String | 是 | | | 访问密钥。 + newTopic | String | 是 | | | 新建topic的名称。 + queueNum | int | 是 | 0 | (0, maxIntValue] | topic的队列数量。 + topicSysFlag | int | 是 | 0 | | 保留字段,暂未使用。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - 生产者状态非Running;未找到broker等客户端异常。 + +3. earliestMsgStoreTime + + `public long earliestMsgStoreTime(MessageQueue mq)` + + 查询最早的消息存储时间。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + mq | MessageQueue | 是 | | | 要查询的消息队列 + + - 返回值描述: + + 指定队列最早的消息存储时间。单位:毫秒。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常;线程中断等客户端异常。 + +4. fetchPublishMessageQueues + + `public List fetchPublishMessageQueues(String topic)` + + 获取topic的消息队列。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + topic | String | 是 | | | topic名称 + + - 返回值描述: + + 传入topic下的消息队列。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常;线程中断等客户端异常。 + +5. maxOffset + + `public long maxOffset(MessageQueue mq)` + + 查询消息队列的最大物理偏移量。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + mq | MessageQueue | 是 | | | 要查询的消息队列 + + - 返回值描述: + + 给定消息队列的最大物理偏移量。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常;线程中断等客户端异常。 + +6. minOffset + + `public long minOffset(MessageQueue mq)` + + 查询给定消息队列的最小物理偏移量。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + mq | MessageQueue | 是 | | | 要查询的消息队列 + + - 返回值描述: + + 给定消息队列的最小物理偏移量。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常;线程中断等客户端异常。 + +7. queryMessage + + `public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end)` + + 按关键字查询消息。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + topic | String | 是 | | | topic名称 + key | String | 否 | null | | 查找的关键字 + maxNum | int | 是 | | | 返回消息的最大数量 + begin | long | 是 | | | 开始时间戳,单位:毫秒 + end | long | 是 | | | 结束时间戳,单位:毫秒 + + - 返回值描述: + + 查询到的消息集合。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常等客户端异常客户端异常。
+ InterruptedException - 线程中断。 + +8. searchOffset + + `public long searchOffset(MessageQueue mq, long timestamp)` + + 查找指定时间的消息队列的物理偏移量。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + mq | MessageQueue | 是 | | | 要查询的消息队列。 + timestamp | long | 是 | | | 指定要查询时间的时间戳。单位:毫秒。 + + - 返回值描述: + + 指定时间的消息队列的物理偏移量。 + + - 异常描述: + + MQClientException - 生产者状态非Running;没有找到broker;broker返回失败;网络异常;线程中断等客户端异常。 + +9. send + + `public SendResult send(Collection msgs)` + + 同步批量发送消息。在返回发送失败之前,内部尝试重新发送消息的最大次数(参见*retryTimesWhenSendFailed*属性)。未明确指定发送队列,默认采取轮询策略发送。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msgs | Collection | 是 | | | 待发送的消息集合。集合内的消息必须属同一个topic。 + + - 返回值描述: + + 批量消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +10. send + + `public SendResult send(Collection msgs, long timeout)` + + 同步批量发送消息,如果在指定的超时时间内未完成消息投递,会抛出*RemotingTooMuchRequestException*。 + 在返回发送失败之前,内部尝试重新发送消息的最大次数(参见*retryTimesWhenSendFailed*属性)。未明确指定发送队列,默认采取轮询策略发送。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msgs | Collection | 是 | | | 待发送的消息集合。集合内的消息必须属同一个topic。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + + 批量消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +11. send + + `public SendResult send(Collection msgs, MessageQueue messageQueue)` + + 向给定队列同步批量发送消息。 + + 注意:指定队列意味着所有消息均为同一个topic。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msgs | Collection | 是 | | | 待发送的消息集合。集合内的消息必须属同一个topic。 + messageQueue | MessageQueue | 是 | | | 待投递的消息队列。指定队列意味着待投递消息均为同一个topic。 + + - 返回值描述: + + 批量消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +12. send + + `public SendResult send(Collection msgs, MessageQueue messageQueue, long timeout)` + + 向给定队列同步批量发送消息,如果在指定的超时时间内未完成消息投递,会抛出*RemotingTooMuchRequestException*。 + + 注意:指定队列意味着所有消息均为同一个topic。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msgs | Collection | 是 | | | 待发送的消息集合。集合内的消息必须属同一个topic。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + messageQueue | MessageQueue | 是 | | | 待投递的消息队列。指定队列意味着待投递消息均为同一个topic。 + + - 返回值描述: + + 批量消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +13. send + + `public SendResult send(Message msg)` + + 以同步模式发送消息,仅当发送过程完全完成时,此方法才会返回。 + 在返回发送失败之前,内部尝试重新发送消息的最大次数(参见*retryTimesWhenSendFailed*属性)。未明确指定发送队列,默认采取轮询策略发送。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +14. send + + `public SendResult send(Message msg, long timeout)` + + 以同步模式发送消息,如果在指定的超时时间内未完成消息投递,会抛出*RemotingTooMuchRequestException*。仅当发送过程完全完成时,此方法才会返回。 + 在返回发送失败之前,内部尝试重新发送消息的最大次数(参见*retryTimesWhenSendFailed*属性)。未明确指定发送队列,默认采取轮询策略发送。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +15. send + + `public SendResult send(Message msg, MessageQueue mq)` + + 向指定的消息队列同步发送单条消息。仅当发送过程完全完成时,此方法才会返回。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + mq | MessageQueue | 是 | | | 待投递的消息队列。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +16. send + + `public SendResult send(Message msg, MessageQueue mq, long timeout)` + + 向指定的消息队列同步发送单条消息,如果在指定的超时时间内未完成消息投递,会抛出*RemotingTooMuchRequestException*。仅当发送过程完全完成时,此方法才会返回。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + mq | MessageQueue | 是 | | | 待投递的消息队列。指定队列意味着待投递消息均为同一个topic。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +17. send + + `public void send(Message msg, MessageQueue mq, SendCallback sendCallback)` + + 向指定的消息队列异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + mq | MessageQueue | 是 | | | 待投递的消息队列。指定队列意味着待投递消息均为同一个topic。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +18. send + + `public void send(Message msg, MessageQueue mq, SendCallback sendCallback, long timeout)` + + 向指定的消息队列异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 若在指定时间内消息未发送成功,回调方法会收到*RemotingTooMuchRequestException*异常。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + mq | MessageQueue | 是 | | | 待投递的消息队列。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +19. send + + `public SendResult send(Message msg, MessageQueueSelector selector, Object arg)` + + 向通过`MessageQueueSelector`计算出的队列同步发送消息。 + + 可以通过自实现`MessageQueueSelector`接口,将某一类消息发送至固定的队列。比如:将同一个订单的状态变更消息投递至固定的队列。 + + 注意:此消息发送失败内部不会重试。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + selector | MessageQueueSelector | 是 | | | 队列选择器。 + arg | Object | 否 | | | 供队列选择器使用的参数对象。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +20. send + + `public SendResult send(Message msg, MessageQueueSelector selector, Object arg, long timeout)` + + 向通过`MessageQueueSelector`计算出的队列同步发送消息,并指定发送超时时间。 + + 可以通过自实现`MessageQueueSelector`接口,将某一类消息发送至固定的队列。比如:将同一个订单的状态变更消息投递至固定的队列。 + + 注意:此消息发送失败内部不会重试。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + selector | MessageQueueSelector | 是 | | | 队列选择器。 + arg | Object | 否 | | | 供队列选择器使用的参数对象。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + + 消息的发送结果,包含msgId,发送状态等信息。 + + - 异常描述: + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 发送线程中断。
+ RemotingTooMuchRequestException - 发送超时。 + +21. send + + `public void send(Message msg, MessageQueueSelector selector, Object arg, SendCallback sendCallback)` + + 向通过`MessageQueueSelector`计算出的队列异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + 可以通过自实现`MessageQueueSelector`接口,将某一类消息发送至固定的队列。比如:将同一个订单的状态变更消息投递至固定的队列。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + selector | MessageQueueSelector | 是 | | | 队列选择器。 + arg | Object | 否 | | | 供队列选择器使用的参数对象。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +22. send + + `public void send(Message msg, MessageQueueSelector selector, Object arg, SendCallback sendCallback, long timeout)` + + 向通过`MessageQueueSelector`计算出的队列异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + 可以通过自实现`MessageQueueSelector`接口,将某一类消息发送至固定的队列。比如:将同一个订单的状态变更消息投递至固定的队列。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + selector | MessageQueueSelector | 是 | | | 队列选择器。 + arg | Object | 否 | | | 供队列选择器使用的参数对象。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +23. send + + `public void send(Message msg, SendCallback sendCallback)` + + 异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +24. send + + `public void send(Message msg, SendCallback sendCallback, long timeout)` + + 异步发送单条消息,异步发送调用后直接返回,并在在发送成功或者异常时回调`sendCallback`,所以异步发送时`sendCallback`参数不能为null,否则在回调时会抛出`NullPointerException`。 + 异步发送时,在成功发送前,其内部会尝试重新发送消息的最大次数(参见*retryTimesWhenSendAsyncFailed*属性)。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + sendCallback | SendCallback | 是 | | | 回调接口的实现。 + timeout | long | 是 | 参见*sendMsgTimeout*属性 | | 发送超时时间,单位:毫秒。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +25. sendMessageInTransaction + + `public TransactionSendResult sendMessageInTransaction(Message msg, LocalTransactionExecuter tranExecuter, final Object arg)` + + 发送事务消息。该类不做默认实现,抛出`RuntimeException`异常。参见:`TransactionMQProducer`类。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待投递的事务消息 + tranExecuter | `LocalTransactionExecuter` | 是 | | | 本地事务执行器。该类*已过期*,将在5.0.0版本中移除。请勿使用该方法。 + arg | Object | 是 | | | 供本地事务执行程序使用的参数对象 + + - 返回值描述: + + 事务结果,参见:`LocalTransactionState`类。 + + - 异常描述: + + RuntimeException - 永远抛出该异常。 + +26. sendMessageInTransaction + + `public TransactionSendResult sendMessageInTransaction(Message msg, final Object arg)` + + 发送事务消息。该类不做默认实现,抛出`RuntimeException`异常。参见:`TransactionMQProducer`类。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待投递的事务消息 + arg | Object | 是 | | | 供本地事务执行程序使用的参数对象 + + - 返回值描述: + + 事务结果,参见:`LocalTransactionState`类。 + + - 异常描述: + + RuntimeException - 永远抛出该异常。 + +27. sendOneway + + `public void sendOneway(Message msg)` + + 以oneway形式发送消息,broker不会响应任何执行结果,和[UDP](https://en.wikipedia.org/wiki/User_Datagram_Protocol)类似。它具有最大的吞吐量但消息可能会丢失。 + + 可在消息量大,追求高吞吐量并允许消息丢失的情况下使用该方式。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待投递的消息 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +28. sendOneway + + `public void sendOneway(Message msg, MessageQueue mq)` + + 向指定队列以oneway形式发送消息,broker不会响应任何执行结果,和[UDP](https://en.wikipedia.org/wiki/User_Datagram_Protocol)类似。它具有最大的吞吐量但消息可能会丢失。 + + 可在消息量大,追求高吞吐量并允许消息丢失的情况下使用该方式。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待投递的消息 + mq | MessageQueue | 是 | | | 待投递的消息队列 + + - 返回值描述: + void + - 异常描述: + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +29. sendOneway + + `public void sendOneway(Message msg, MessageQueueSelector selector, Object arg)` + + 向通过`MessageQueueSelector`计算出的队列以oneway形式发送消息,broker不会响应任何执行结果,和[UDP](https://en.wikipedia.org/wiki/User_Datagram_Protocol)类似。它具有最大的吞吐量但消息可能会丢失。 + + 可在消息量大,追求高吞吐量并允许消息丢失的情况下使用该方式。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msg | Message | 是 | | | 待发送的消息。 + selector | MessageQueueSelector | 是 | | | 队列选择器。 + arg | Object | 否 | | | 供队列选择器使用的参数对象。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - broker不存在或未找到;namesrv地址为空;未找到topic的路由信息等客户端异常。
+ RemotingException - 网络异常。
+ InterruptedException - 发送线程中断。 + +30. shutdown + + `public void shutdown()` + + 关闭当前生产者实例并释放相关资源。 + + - 入参描述: + + 无。 + + - 返回值描述: + + void + + - 异常描述: + +31. start + + `public void start()` + + 启动生产者实例。在发送或查询消息之前必须调用此方法。它执行了许多内部初始化,比如:检查配置、与namesrv建立连接、启动一系列心跳等定时任务等。 + + - 入参描述: + + 无。 + + - 返回值描述: + + void + + - 异常描述: + + MQClientException - 初始化过程中出现失败。 + +32. viewMessage + + `public MessageExt viewMessage(String offsetMsgId)` + + 根据给定的msgId查询消息。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + offsetMsgId | String | 是 | | | offsetMsgId + + - 返回值描述: + + 返回`MessageExt`,包含:topic名称,消息题,消息ID,消费次数,生产者host等信息。 + + - 异常描述: + + RemotingException - 网络层发生错误。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 线程被中断。
+ MQClientException - 生产者状态非Running;msgId非法等。 + +33. viewMessage + + `public MessageExt viewMessage(String topic, String msgId)` + + 根据给定的msgId查询消息,并指定topic。 + + - 入参描述: + + 参数名 | 类型 | 是否必须 | 默认值 | 值范围 | 说明 + ---|---|---|---|---|--- + msgId | String | 是 | | | msgId + topic | String | 是 | | | topic名称 + + - 返回值描述: + + 返回`MessageExt`,包含:topic名称,消息题,消息ID,消费次数,生产者host等信息。 + + - 异常描述: + + RemotingException - 网络层发生错误。
+ MQBrokerException - broker发生错误。
+ InterruptedException - 线程被中断。
+ MQClientException - 生产者状态非Running;msgId非法等。 \ No newline at end of file diff --git a/docs/cn/concept.md b/docs/cn/concept.md index 38036f22f66..ef59050748a 100644 --- a/docs/cn/concept.md +++ b/docs/cn/concept.md @@ -1,9 +1,7 @@ # 基本概念 ## 消息模型(Message Model) - - -RocketMQ 消息模型如图1所示,主要由 Producer、Broker、Consumer 三部分组成,其中Producer 负责生产消息,Consumer 负责消费消息,Broker 负责存储消息。Broker 在实际部署过程中对应一台服务器,每个 Broker 可以存储多个Topic的消息,每个Topic的消息也可以分片存储于不同的 Broker。Message Queue 用于存储消息的物理地址,每个Topic中的消息地址存储于多个 Message Queue 中。ConsumerGroup 由多个Consumer 实例构成。 +RocketMQ主要由 Producer、Broker、Consumer 三部分组成,其中Producer 负责生产消息,Consumer 负责消费消息,Broker 负责存储消息。Broker 在实际部署过程中对应一台服务器,每个 Broker 可以存储多个Topic的消息,每个Topic的消息也可以分片存储于不同的 Broker。Message Queue 用于存储消息的物理地址,每个Topic中的消息地址存储于多个 Message Queue 中。ConsumerGroup 由多个Consumer 实例构成。 ## 消息生产者(Producer) 负责生产消息,一般由业务系统负责生产消息。一个消息生产者会把业务应用系统里产生的消息发送到broker服务器。RocketMQ提供多种发送方式,同步发送、异步发送、顺序发送、单向发送。同步和异步方式均需要Broker返回确认信息,单向发送不需要。 ## 消息消费者(Consumer) diff --git a/docs/cn/design.md b/docs/cn/design.md new file mode 100644 index 00000000000..5b11912bafe --- /dev/null +++ b/docs/cn/design.md @@ -0,0 +1,213 @@ + +## 设计(design) +### 1 消息存储 + +![](image/rocketmq_design_1.png) + +消息存储是RocketMQ中最为复杂和最为重要的一部分,本节将分别从RocketMQ的消息存储整体架构、PageCache与Mmap内存映射以及RocketMQ中两种不同的刷盘方式三方面来分别展开叙述。 + +#### 1.1 消息存储整体架构 +消息存储架构图中主要有下面三个跟消息存储相关的文件构成。 + +(1) CommitLog:消息主体以及元数据的存储主体,存储Producer端写入的消息主体内容,消息内容不是定长的。单个文件大小默认1G ,文件名长度为20位,左边补零,剩余为起始偏移量,比如00000000000000000000代表了第一个文件,起始偏移量为0,文件大小为1G=1073741824;当第一个文件写满了,第二个文件为00000000001073741824,起始偏移量为1073741824,以此类推。消息主要是顺序写入日志文件,当文件满了,写入下一个文件; + +(2) ConsumeQueue:消息消费队列,引入的目的主要是提高消息消费的性能,由于RocketMQ是基于主题topic的订阅模式,消息消费是针对主题进行的,如果要遍历commitlog文件中根据topic检索消息是非常低效的。Consumer即可根据ConsumeQueue来查找待消费的消息。其中,ConsumeQueue(逻辑消费队列)作为消费消息的索引,保存了指定Topic下的队列消息在CommitLog中的起始物理偏移量offset,消息大小size和消息Tag的HashCode值。consumequeue文件可以看成是基于topic的commitlog索引文件,故consumequeue文件夹的组织方式如下:topic/queue/file三层组织结构,具体存储路径为:$HOME/store/consumequeue/{topic}/{queueId}/{fileName}。同样consumequeue文件采取定长设计,每一个条目共20个字节,分别为8字节的commitlog物理偏移量、4字节的消息长度、8字节tag hashcode,单个文件由30W个条目组成,可以像数组一样随机访问每一个条目,每个ConsumeQueue文件大小约5.72M; + +(3) IndexFile:IndexFile(索引文件)提供了一种可以通过key或时间区间来查询消息的方法。Index文件的存储位置是:$HOME \store\index\${fileName},文件名fileName是以创建时的时间戳命名的,固定的单个IndexFile文件大小约为400M,一个IndexFile可以保存 2000W个索引,IndexFile的底层存储设计为在文件系统中实现HashMap结构,故rocketmq的索引文件其底层实现为hash索引。 + +在上面的RocketMQ的消息存储整体架构图中可以看出,RocketMQ采用的是混合型的存储结构,即为Broker单个实例下所有的队列共用一个日志数据文件(即为CommitLog)来存储。RocketMQ的混合型存储结构(多个Topic的消息实体内容都存储于一个CommitLog中)针对Producer和Consumer分别采用了数据和索引部分相分离的存储结构,Producer发送消息至Broker端,然后Broker端使用同步或者异步的方式对消息刷盘持久化,保存至CommitLog中。只要消息被刷盘持久化至磁盘文件CommitLog中,那么Producer发送的消息就不会丢失。正因为如此,Consumer也就肯定有机会去消费这条消息。当无法拉取到消息后,可以等下一次消息拉取,同时服务端也支持长轮询模式,如果一个消息拉取请求未拉取到消息,Broker允许等待30s的时间,只要这段时间内有新消息到达,将直接返回给消费端。这里,RocketMQ的具体做法是,使用Broker端的后台服务线程—ReputMessageService不停地分发请求并异步构建ConsumeQueue(逻辑消费队列)和IndexFile(索引文件)数据。 +#### 1.2 页缓存与内存映射 +页缓存(PageCache)是OS对文件的缓存,用于加速对文件的读写。一般来说,程序对文件进行顺序读写的速度几乎接近于内存的读写速度,主要原因就是由于OS使用PageCache机制对读写访问操作进行了性能优化,将一部分的内存用作PageCache。对于数据的写入,OS会先写入至Cache内,随后通过异步的方式由pdflush内核线程将Cache内的数据刷盘至物理磁盘上。对于数据的读取,如果一次读取文件时出现未命中PageCache的情况,OS从物理磁盘上访问读取文件的同时,会顺序对其他相邻块的数据文件进行预读取。 + +在RocketMQ中,ConsumeQueue逻辑消费队列存储的数据较少,并且是顺序读取,在page cache机制的预读取作用下,Consume Queue文件的读性能几乎接近读内存,即使在有消息堆积情况下也不会影响性能。而对于CommitLog消息存储的日志数据文件来说,读取消息内容时候会产生较多的随机访问读取,严重影响性能。如果选择合适的系统IO调度算法,比如设置调度算法为“Deadline”(此时块存储采用SSD的话),随机读的性能也会有所提升。 + +另外,RocketMQ主要通过MappedByteBuffer对文件进行读写操作。其中,利用了NIO中的FileChannel模型将磁盘上的物理文件直接映射到用户态的内存地址中(这种Mmap的方式减少了传统IO将磁盘文件数据在操作系统内核地址空间的缓冲区和用户应用程序地址空间的缓冲区之间来回进行拷贝的性能开销),将对文件的操作转化为直接对内存地址进行操作,从而极大地提高了文件的读写效率(正因为需要使用内存映射机制,故RocketMQ的文件存储都使用定长结构来存储,方便一次将整个文件映射至内存)。 +#### 1.3 消息刷盘 + +![](image/rocketmq_design_2.png) + +(1) 同步刷盘:如上图所示,只有在消息真正持久化至磁盘后RocketMQ的Broker端才会真正返回给Producer端一个成功的ACK响应。同步刷盘对MQ消息可靠性来说是一种不错的保障,但是性能上会有较大影响,一般适用于金融业务应用该模式较多。 + +(2) 异步刷盘:能够充分利用OS的PageCache的优势,只要消息写入PageCache即可将成功的ACK返回给Producer端。消息刷盘采用后台异步线程提交的方式进行,降低了读写延迟,提高了MQ的性能和吞吐量。 + +### 2 通信机制 +RocketMQ消息队列集群主要包括NameServe、Broker(Master/Slave)、Producer、Consumer4个角色,基本通讯流程如下: + +(1) Broker启动后需要完成一次将自己注册至NameServer的操作;随后每隔30s时间定时向NameServer上报Topic路由信息。 + +(2) 消息生产者Producer作为客户端发送消息时候,需要根据消息的Topic从本地缓存的TopicPublishInfoTable获取路由信息。如果没有则更新路由信息会从NameServer上重新拉取,同时Producer会默认每隔30s向NameServer拉取一次路由信息。 + +(3) 消息生产者Producer根据2)中获取的路由信息选择一个队列(MessageQueue)进行消息发送;Broker作为消息的接收者接收消息并落盘存储。 + +(4) 消息消费者Consumer根据2)中获取的路由信息,并再完成客户端的负载均衡后,选择其中的某一个或者某几个消息队列来拉取消息并进行消费。 + +从上面1)~3)中可以看出在消息生产者, Broker和NameServer之间都会发生通信(这里只说了MQ的部分通信),因此如何设计一个良好的网络通信模块在MQ中至关重要,它将决定RocketMQ集群整体的消息传输能力与最终的性能。 + +rocketmq-remoting 模块是 RocketMQ消息队列中负责网络通信的模块,它几乎被其他所有需要网络通信的模块(诸如rocketmq-client、rocketmq-broker、rocketmq-namesrv)所依赖和引用。为了实现客户端与服务器之间高效的数据请求与接收,RocketMQ消息队列自定义了通信协议并在Netty的基础之上扩展了通信模块。 +#### 2.1 Remoting通信类结构 + +![](image/rocketmq_design_3.png) + +#### 2.2 协议设计与编解码 +在Client和Server之间完成一次消息发送时,需要对发送的消息进行一个协议约定,因此就有必要自定义RocketMQ的消息协议。同时,为了高效地在网络中传输消息和对收到的消息读取,就需要对消息进行编解码。在RocketMQ中,RemotingCommand这个类在消息传输过程中对所有数据内容的封装,不但包含了所有的数据结构,还包含了编码解码操作。 + +Header字段 | 类型 | Request说明 | Response说明 +--- | --- | --- | --- | +code |int | 请求操作码,应答方根据不同的请求码进行不同的业务处理 | 应答响应码。0表示成功,非0则表示各种错误 +language | LanguageCode | 请求方实现的语言 | 应答方实现的语言 +version | int | 请求方程序的版本 | 应答方程序的版本 +opaque | int |相当于reqeustId,在同一个连接上的不同请求标识码,与响应消息中的相对应 | 应答不做修改直接返回 +flag | int | 区分是普通RPC还是onewayRPC得标志 | 区分是普通RPC还是onewayRPC得标志 +remark | String | 传输自定义文本信息 | 传输自定义文本信息 +extFields | HashMap | 请求自定义扩展信息 | 响应自定义扩展信息 + +![](image/rocketmq_design_4.png) + +可见传输内容主要可以分为以下4部分: + +(1) 消息长度:总长度,四个字节存储,占用一个int类型; + +(2) 序列化类型&消息头长度:同样占用一个int类型,第一个字节表示序列化类型,后面三个字节表示消息头长度; + +(3) 消息头数据:经过序列化后的消息头数据; + +(4) 消息主体数据:消息主体的二进制字节数据内容; + +#### 2.3 消息的通信方式和流程 +在RocketMQ消息队列中支持通信的方式主要有同步(sync)、异步(async)、单向(oneway) +三种。其中“单向”通信模式相对简单,一般用在发送心跳包场景下,无需关注其Response。这里,主要介绍RocketMQ的异步通信流程。 + +![](image/rocketmq_design_5.png) + +#### 2.4 Reactor多线程设计 +RocketMQ的RPC通信采用Netty组件作为底层通信库,同样也遵循了Reactor多线程模型,同时又在这之上做了一些扩展和优化。 + +![](image/rocketmq_design_6.png) + +上面的框图中可以大致了解RocketMQ中NettyRemotingServer的Reactor 多线程模型。一个 Reactor 主线程(eventLoopGroupBoss,即为上面的1)负责监听 TCP网络连接请求,建立好连接,创建SocketChannel,并注册到selector上。RocketMQ的源码中会自动根据OS的类型选择NIO和Epoll,也可以通过参数配置),然后监听真正的网络数据。拿到网络数据后,再丢给Worker线程池(eventLoopGroupSelector,即为上面的“N”,源码中默认设置为3),在真正执行业务逻辑之前需要进行SSL验证、编解码、空闲检查、网络连接管理,这些工作交给defaultEventExecutorGroup(即为上面的“M1”,源码中默认设置为8)去做。而处理业务操作放在业务线程池中执行,根据 RomotingCommand 的业务请求码code去processorTable这个本地缓存变量中找到对应的 processor,然后封装成task任务后,提交给对应的业务processor处理线程池来执行(sendMessageExecutor,以发送消息为例,即为上面的 “M2”)。从入口到业务逻辑的几个步骤中线程池一直再增加,这跟每一步逻辑复杂性相关,越复杂,需要的并发通道越宽。 + +线程数 | 线程名 | 线程具体说明 + --- | --- | --- +1 | NettyBoss_%d | Reactor 主线程 +N | NettyServerEPOLLSelector_%d_%d | Reactor 线程池 +M1 | NettyServerCodecThread_%d | Worker线程池 +M2 | RemotingExecutorThread_%d | 业务processor处理线程池 + +### 3 消息过滤 +RocketMQ分布式消息队列的消息过滤方式有别于其它MQ中间件,是在Consumer端订阅消息时再做消息过滤的。RocketMQ这么做是还是在于其Producer端写入消息和Consomer端订阅消息采用分离存储的机制来实现的,Consumer端订阅消息是需要通过ConsumeQueue这个消息消费的逻辑队列拿到一个索引,然后再从CommitLog里面读取真正的消息实体内容,所以说到底也是还绕不开其存储结构。其ConsumeQueue的存储结构如下,可以看到其中有8个字节存储的Message Tag的哈希值,基于Tag的消息过滤正式基于这个字段值的。 + +![](image/rocketmq_design_7.png) + +主要支持如下2种的过滤方式 +(1) Tag过滤方式:Consumer端在订阅消息时除了指定Topic还可以指定TAG,如果一个消息有多个TAG,可以用||分隔。其中,Consumer端会将这个订阅请求构建成一个 SubscriptionData,发送一个Pull消息的请求给Broker端。Broker端从RocketMQ的文件存储层—Store读取数据之前,会用这些数据先构建一个MessageFilter,然后传给Store。Store从 ConsumeQueue读取到一条记录后,会用它记录的消息tag hash值去做过滤,由于在服务端只是根据hashcode进行判断,无法精确对tag原始字符串进行过滤,故在消息消费端拉取到消息后,还需要对消息的原始tag字符串进行比对,如果不同,则丢弃该消息,不进行消息消费。 + +(2) SQL92的过滤方式:这种方式的大致做法和上面的Tag过滤方式一样,只是在Store层的具体过滤过程不太一样,真正的 SQL expression 的构建和执行由rocketmq-filter模块负责的。每次过滤都去执行SQL表达式会影响效率,所以RocketMQ使用了BloomFilter避免了每次都去执行。SQL92的表达式上下文为消息的属性。 + +### 4 负载均衡 +RocketMQ中的负载均衡都在Client端完成,具体来说的话,主要可以分为Producer端发送消息时候的负载均衡和Consumer端订阅消息的负载均衡。 +#### 4.1 Producer的负载均衡 +Producer端在发送消息的时候,会先根据Topic找到指定的TopicPublishInfo,在获取了TopicPublishInfo路由信息后,RocketMQ的客户端在默认方式下selectOneMessageQueue()方法会从TopicPublishInfo中的messageQueueList中选择一个队列(MessageQueue)进行发送消息。具体的容错策略均在MQFaultStrategy这个类中定义。这里有一个sendLatencyFaultEnable开关变量,如果开启,在随机递增取模的基础上,再过滤掉not available的Broker代理。所谓的"latencyFaultTolerance",是指对之前失败的,按一定的时间做退避。例如,如果上次请求的latency超过550Lms,就退避3000Lms;超过1000L,就退避60000L;如果关闭,采用随机递增取模的方式选择一个队列(MessageQueue)来发送消息,latencyFaultTolerance机制是实现消息发送高可用的核心关键所在。 +#### 4.2 Consumer的负载均衡 +在RocketMQ中,Consumer端的两种消费模式(Push/Pull)都是基于拉模式来获取消息的,而在Push模式只是对pull模式的一种封装,其本质实现为消息拉取线程在从服务器拉取到一批消息后,然后提交到消息消费线程池后,又“马不停蹄”的继续向服务器再次尝试拉取消息。如果未拉取到消息,则延迟一下又继续拉取。在两种基于拉模式的消费方式(Push/Pull)中,均需要Consumer端在知道从Broker端的哪一个消息队列—队列中去获取消息。因此,有必要在Consumer端来做负载均衡,即Broker端中多个MessageQueue分配给同一个ConsumerGroup中的哪些Consumer消费。 + +1、Consumer端的心跳包发送 + +在Consumer启动后,它就会通过定时任务不断地向RocketMQ集群中的所有Broker实例发送心跳包(其中包含了,消息消费分组名称、订阅关系集合、消息通信模式和客户端id的值等信息)。Broker端在收到Consumer的心跳消息后,会将它维护在ConsumerManager的本地缓存变量—consumerTable,同时并将封装后的客户端网络通道信息保存在本地缓存变量—channelInfoTable中,为之后做Consumer端的负载均衡提供可以依据的元数据信息。 + +2、Consumer端实现负载均衡的核心类—RebalanceImpl + +在Consumer实例的启动流程中的启动MQClientInstance实例部分,会完成负载均衡服务线程—RebalanceService的启动(每隔20s执行一次)。通过查看源码可以发现,RebalanceService线程的run()方法最终调用的是RebalanceImpl类的rebalanceByTopic()方法,该方法是实现Consumer端负载均衡的核心。这里,rebalanceByTopic()方法会根据消费者通信类型为“广播模式”还是“集群模式”做不同的逻辑处理。这里主要来看下集群模式下的主要处理流程: + +(1) 从rebalanceImpl实例的本地缓存变量—topicSubscribeInfoTable中,获取该Topic主题下的消息消费队列集合(mqSet); + +(2) 根据topic和consumerGroup为参数调用mQClientFactory.findConsumerIdList()方法向Broker端发送获取该消费组下消费者Id列表的RPC通信请求(Broker端基于前面Consumer端上报的心跳包数据而构建的consumerTable做出响应返回,业务请求码:GET_CONSUMER_LIST_BY_GROUP); + +(3) 先对Topic下的消息消费队列、消费者Id排序,然后用消息队列分配策略算法(默认为:消息队列的平均分配算法),计算出待拉取的消息队列。这里的平均分配算法,类似于分页的算法,将所有MessageQueue排好序类似于记录,将所有消费端Consumer排好序类似页数,并求出每一页需要包含的平均size和每个页面记录的范围range,最后遍历整个range而计算出当前Consumer端应该分配到的记录(这里即为:MessageQueue)。 + +![](image/rocketmq_design_8.png) + +(4) 然后,调用updateProcessQueueTableInRebalance()方法,具体的做法是,先将分配到的消息队列集合(mqSet)与processQueueTable做一个过滤比对。 + +![](image/rocketmq_design_9.png) + + + +- 上图中processQueueTable标注的红色部分,表示与分配到的消息队列集合mqSet互不包含。将这些队列设置Dropped属性为true,然后查看这些队列是否可以移除出processQueueTable缓存变量,这里具体执行removeUnnecessaryMessageQueue()方法,即每隔1s 查看是否可以获取当前消费处理队列的锁,拿到的话返回true。如果等待1s后,仍然拿不到当前消费处理队列的锁则返回false。如果返回true,则从processQueueTable缓存变量中移除对应的Entry; + +- 上图中processQueueTable的绿色部分,表示与分配到的消息队列集合mqSet的交集。判断该ProcessQueue是否已经过期了,在Pull模式的不用管,如果是Push模式的,设置Dropped属性为true,并且调用removeUnnecessaryMessageQueue()方法,像上面一样尝试移除Entry; + +最后,为过滤后的消息队列集合(mqSet)中的每个MessageQueue创建一个ProcessQueue对象并存入RebalanceImpl的processQueueTable队列中(其中调用RebalanceImpl实例的computePullFromWhere(MessageQueue mq)方法获取该MessageQueue对象的下一个进度消费值offset,随后填充至接下来要创建的pullRequest对象属性中),并创建拉取请求对象—pullRequest添加到拉取列表—pullRequestList中,最后执行dispatchPullRequest()方法,将Pull消息的请求对象PullRequest依次放入PullMessageService服务线程的阻塞队列pullRequestQueue中,待该服务线程取出后向Broker端发起Pull消息的请求。其中,可以重点对比下,RebalancePushImpl和RebalancePullImpl两个实现类的dispatchPullRequest()方法不同,RebalancePullImpl类里面的该方法为空,这样子也就回答了上一篇中最后的那道思考题了。 + +消息消费队列在同一消费组不同消费者之间的负载均衡,其核心设计理念是在一个消息消费队列在同一时间只允许被同一消费组内的一个消费者消费,一个消息消费者能同时消费多个消息队列。 + +### 5 事务消息 +Apache RocketMQ在4.3.0版中已经支持分布式事务消息,这里RocketMQ采用了2PC的思想来实现了提交事务消息,同时增加一个补偿逻辑来处理二阶段超时或者失败的消息,如下图所示。 + +![](image/rocketmq_design_10.png) + +#### 5.1 RocketMQ事务消息流程概要 +上图说明了事务消息的大致方案,其中分为两个流程:正常事务消息的发送及提交、事务消息的补偿流程。 + +1.事务消息发送及提交: + +(1) 发送消息(half消息)。 + +(2) 服务端响应消息写入结果。 + +(3) 根据发送结果执行本地事务(如果写入失败,此时half消息对业务不可见,本地逻辑不执行)。 + +(4) 根据本地事务状态执行Commit或者Rollback(Commit操作生成消息索引,消息对消费者可见) + +2.补偿流程: + +(1) 对没有Commit/Rollback的事务消息(pending状态的消息),从服务端发起一次“回查” + +(2) Producer收到回查消息,检查回查消息对应的本地事务的状态 + +(3) 根据本地事务状态,重新Commit或者Rollback + +其中,补偿阶段用于解决消息Commit或者Rollback发生超时或者失败的情况。 +#### 5.2 RocketMQ事务消息设计 +1.事务消息在一阶段对用户不可见 + +在RocketMQ事务消息的主要流程中,一阶段的消息如何对用户不可见。其中,事务消息相对普通消息最大的特点就是一阶段发送的消息对用户是不可见的。那么,如何做到写入消息但是对用户不可见呢?RocketMQ事务消息的做法是:如果消息是half消息,将备份原消息的主题与消息消费队列,然后改变主题为RMQ_SYS_TRANS_HALF_TOPIC。由于消费组未订阅该主题,故消费端无法消费half类型的消息,然后RocketMQ会开启一个定时任务,从Topic为RMQ_SYS_TRANS_HALF_TOPIC中拉取消息进行消费,根据生产者组获取一个服务提供者发送回查事务状态请求,根据事务状态来决定是提交或回滚消息。 + +在RocketMQ中,消息在服务端的存储结构如下,每条消息都会有对应的索引信息,Consumer通过ConsumeQueue这个二级索引来读取消息实体内容,其流程如下: + +![](image/rocketmq_design_11.png) + +RocketMQ的具体实现策略是:写入的如果事务消息,对消息的Topic和Queue等属性进行替换,同时将原来的Topic和Queue信息存储到消息的属性中,正因为消息主题被替换,故消息并不会转发到该原主题的消息消费队列,消费者无法感知消息的存在,不会消费。其实改变消息主题是RocketMQ的常用“套路”,回想一下延时消息的实现机制。 + +2.Commit和Rollback操作以及Op消息的引入 + +在完成一阶段写入一条对用户不可见的消息后,二阶段如果是Commit操作,则需要让消息对用户可见;如果是Rollback则需要撤销一阶段的消息。先说Rollback的情况。对于Rollback,本身一阶段的消息对用户是不可见的,其实不需要真正撤销消息(实际上RocketMQ也无法去真正的删除一条消息,因为是顺序写文件的)。但是区别于这条消息没有确定状态(Pending状态,事务悬而未决),需要一个操作来标识这条消息的最终状态。RocketMQ事务消息方案中引入了Op消息的概念,用Op消息标识事务消息已经确定的状态(Commit或者Rollback)。如果一条事务消息没有对应的Op消息,说明这个事务的状态还无法确定(可能是二阶段失败了)。引入Op消息后,事务消息无论是Commit或者Rollback都会记录一个Op操作。Commit相对于Rollback只是在写入Op消息前创建Half消息的索引。 + +3.Op消息的存储和对应关系 + +RocketMQ将Op消息写入到全局一个特定的Topic中通过源码中的方法—TransactionalMessageUtil.buildOpTopic();这个Topic是一个内部的Topic(像Half消息的Topic一样),不会被用户消费。Op消息的内容为对应的Half消息的存储的Offset,这样通过Op消息能索引到Half消息进行后续的回查操作。 + +![](image/rocketmq_design_12.png) + +4.Half消息的索引构建 + +在执行二阶段Commit操作时,需要构建出Half消息的索引。一阶段的Half消息由于是写到一个特殊的Topic,所以二阶段构建索引时需要读取出Half消息,并将Topic和Queue替换成真正的目标的Topic和Queue,之后通过一次普通消息的写入操作来生成一条对用户可见的消息。所以RocketMQ事务消息二阶段其实是利用了一阶段存储的消息的内容,在二阶段时恢复出一条完整的普通消息,然后走一遍消息写入流程。 + +5.如何处理二阶段失败的消息? + +如果在RocketMQ事务消息的二阶段过程中失败了,例如在做Commit操作时,出现网络问题导致Commit失败,那么需要通过一定的策略使这条消息最终被Commit。RocketMQ采用了一种补偿机制,称为“回查”。Broker端对未确定状态的消息发起回查,将消息发送到对应的Producer端(同一个Group的Producer),由Producer根据消息来检查本地事务的状态,进而执行Commit或者Rollback。Broker端通过对比Half消息和Op消息进行事务消息的回查并且推进CheckPoint(记录那些事务消息的状态是确定的)。 + +值得注意的是,rocketmq并不会无休止的的信息事务状态回查,默认回查15次,如果15次回查还是无法得知事务状态,rocketmq默认回滚该消息。 +### 6 消息查询 +RocketMQ支持按照下面两种维度(“按照Message Id查询消息”、“按照Message Key查询消息”)进行消息查询。 +#### 6.1 按照MessageId查询消息 +RocketMQ中的MessageId的长度总共有16字节,其中包含了消息存储主机地址(IP地址和端口),消息Commit Log offset。“按照MessageId查询消息”在RocketMQ中具体做法是:Client端从MessageId中解析出Broker的地址(IP地址和端口)和Commit Log的偏移地址后封装成一个RPC请求后通过Remoting通信层发送(业务请求码:VIEW_MESSAGE_BY_ID)。Broker端走的是QueryMessageProcessor,读取消息的过程用其中的 commitLog offset 和 size 去 commitLog 中找到真正的记录并解析成一个完整的消息返回。 +#### 6.2 按照Message Key查询消息 +“按照Message Key查询消息”,主要是基于RocketMQ的IndexFile索引文件来实现的。RocketMQ的索引文件逻辑结构,类似JDK中HashMap的实现。索引文件的具体结构如下: + +![](image/rocketmq_design_13.png) + +IndexFile索引文件为用户提供通过“按照Message Key查询消息”的消息索引查询服务,IndexFile文件的存储位置是:$HOME\store\index\${fileName},文件名fileName是以创建时的时间戳命名的,文件大小是固定的,等于40+500W\*4+2000W\*20= 420000040个字节大小。如果消息的properties中设置了UNIQ_KEY这个属性,就用 topic + “#” + UNIQ_KEY的value作为 key 来做写入操作。如果消息设置了KEYS属性(多个KEY以空格分隔),也会用 topic + “#” + KEY 来做索引。 + +其中的索引数据包含了Key Hash/CommitLog Offset/Timestamp/NextIndex offset 这四个字段,一共20 Byte。NextIndex offset 即前面读出来的 slotValue,如果有 hash冲突,就可以用这个字段将所有冲突的索引用链表的方式串起来了。Timestamp记录的是消息storeTimestamp之间的差,并不是一个绝对的时间。整个Index File的结构如图,40 Byte 的Header用于保存一些总的统计信息,4\*500W的 Slot Table并不保存真正的索引数据,而是保存每个槽位对应的单向链表的头。20\*2000W 是真正的索引数据,即一个 Index File 可以保存 2000W个索引。 + +“按照Message Key查询消息”的方式,RocketMQ的具体做法是,主要通过Broker端的QueryMessageProcessor业务处理器来查询,读取消息的过程就是用topic和key找到IndexFile索引文件中的一条记录,根据其中的commitLog offset从CommitLog文件中读取消息的实体内容。 diff --git a/docs/cn/dledger/deploy_guide.md b/docs/cn/dledger/deploy_guide.md new file mode 100644 index 00000000000..faebb96e598 --- /dev/null +++ b/docs/cn/dledger/deploy_guide.md @@ -0,0 +1,76 @@ +## 前言 +该文档主要介绍如何部署自动容灾切换的 RocketMQ-on-DLedger Group。 + +RocketMQ-on-DLedger Group 是指一组**相同名称**的 Broker,至少需要 3 个节点,通过 Raft 自动选举出一个 Leader,其余节点 作为 Follower,并在 Leader 和 Follower 之间复制数据以保证高可用。 +RocketMQ-on-DLedger Group 能自动容灾切换,并保证数据一致。 +RocketMQ-on-DLedger Group 是可以水平扩展的,也即可以部署任意多个 RocketMQ-on-DLedger Group 同时对外提供服务。 + +## 1. 新集群部署 + +#### 1.1 编写配置 +每个 RocketMQ-on-DLedger Group 至少准备三台机器(本文假设为 3)。 +编写 3 个配置文件,建议参考 conf/dledger 目录下的配置文件样例。 +关键配置介绍: + +| name | 含义 | 举例 | +| --- | --- | --- | +| enableDLegerCommitLog | 是否启动 DLedger  | true | +| dLegerGroup | DLedger Raft Group的名字,建议和 brokerName 保持一致 | RaftNode00 | +| dLegerPeers | DLedger Group 内各节点的端口信息,同一个 Group 内的各个节点配置必须要保证一致 | n0-127.0.0.1:40911;n1-127.0.0.1:40912;n2-127.0.0.1:40913 | +| dLegerSelfId | 节点 id, 必须属于 dLegerPeers 中的一个;同 Group 内各个节点要唯一 | n0 | +| sendMessageThreadPoolNums | 发送线程个数,建议配置成 Cpu 核数 | 16 | + +这里贴出 conf/dledger/broker-n0.conf 的配置举例。 + +``` +brokerClusterName = RaftCluster +brokerName=RaftNode00 +listenPort=30911 +namesrvAddr=127.0.0.1:9876 +storePathRootDir=/tmp/rmqstore/node00 +storePathCommitLog=/tmp/rmqstore/node00/commitlog +enableDLegerCommitLog=true +dLegerGroup=RaftNode00 +dLegerPeers=n0-127.0.0.1:40911;n1-127.0.0.1:40912;n2-127.0.0.1:40913 +## must be unique +dLegerSelfId=n0 +sendMessageThreadPoolNums=16 +``` + +### 1.2 启动 Broker + +与老版本的启动方式一致。 + +`nohup sh bin/mqbroker -c conf/dledger/xxx-n0.conf & ` +`nohup sh bin/mqbroker -c conf/dledger/xxx-n1.conf & ` +`nohup sh bin/mqbroker -c conf/dledger/xxx-n2.conf & ` + + +## 2. 旧集群升级 + +如果旧集群采用 Master 方式部署,则每个 Master 都需要转换成一个 RocketMQ-on-DLedger Group。 +如果旧集群采用 Master-Slave 方式部署,则每个 Master-Slave 组都需要转换成一个 RocketMQ-on-DLedger Group。 + +### 2.1 杀掉旧的 Broker + +可以通过 kill 命令来完成,也可以调用 `bin/mqshutdown broker`。 + +### 2.2 检查旧的 Commitlog + +RocketMQ-on-DLedger 组中的每个节点,可以兼容旧的 Commitlog ,但其 Raft 复制过程,只能针对新增加的消息。因此,为了避免出现异常,需要保证 旧的 Commitlog 是一致的。 +如果旧的集群是采用 Master-Slave 方式部署,有可能在shutdown时,其数据并不是一致的,建议通过md5sum 的方式,检查最近的最少 2 个 Commmitlog 文件,如果发现不一致,则通过拷贝的方式进行对齐。 + +虽然 RocketMQ-on-DLedger Group 也可以以 2 节点方式部署,但其会丧失容灾切换能力(2n + 1 原则,至少需要3个节点才能容忍其中 1 个宕机)。 +所以在对齐了 Master 和 Slave 的 Commitlog 之后,还需要准备第 3 台机器,并把旧的 Commitlog 从 Master 拷贝到 第 3 台机器(记得同时拷贝一下 config 文件夹)。 + +在 3 台机器准备好了之后,旧 Commitlog 文件也保证一致之后,就可以开始走下一步修改配置了。 + +### 2.3 修改配置 + +参考新集群部署。 + +### 2.4 重新启动 Broker + +参考新集群部署。 + + diff --git a/docs/cn/dledger/quick_start.md b/docs/cn/dledger/quick_start.md new file mode 100644 index 00000000000..3d1989a52cd --- /dev/null +++ b/docs/cn/dledger/quick_start.md @@ -0,0 +1,61 @@ +### 前言 +该文档主要介绍如何快速构建和部署基于 DLedger 的可以自动容灾切换的 RocketMQ 集群。 + +详细的新集群部署和旧集群升级指南请参考 [部署指南](deploy_guide.md)。 + +### 1. 源码构建 +构建分为两个部分,需要先构建 DLedger,然后 构建 RocketMQ + +#### 1.1 构建 DLedger + +`git clone https://github.com/openmessaging/openmessaging-storage-dledger.git` + +`cd openmessaging-storage-dledger` + +`mvn clean install -DskipTests` + +#### 1.2 构建 RocketMQ + +`git clone https://github.com/apache/rocketmq.git` + +`cd rocketmq` + +`git checkout -b store_with_dledger origin/store_with_dledger` + +`mvn -Prelease-all -DskipTests clean install -U` + +### 2. 快速部署 + +在构建成功后 + +`cd distribution/target/apache-rocketmq` + +`sh bin/dledger/fast-try.sh start` + +如果上面的步骤执行成功,可以通过 mqadmin 运维命令查看集群状态。 + +`sh bin/mqadmin clusterList -n 127.0.0.1:9876` + +顺利的话,会看到如下内容: + +![ClusterList](https://img.alicdn.com/5476e8b07b923/TB11Z.ZyCzqK1RjSZFLXXcn2XXa) + +(BID 为 0 的表示 Master,其余都是 Follower) + +启动成功,现在可以向集群收发消息,并进行容灾切换测试了。 + +关闭快速集群,可以执行: + +`sh bin/dledger/fast-try.sh stop` + +快速部署,默认配置在 conf/dledger 里面,默认的存储路径在 /tmp/rmqstore。 + + +### 3. 容灾切换 + +部署成功,杀掉 Leader 之后(在上面的例子中,杀掉端口 30931 所在的进程),等待约 10s 左右,用 clusterList 命令查看集群,就会发现 Leader 切换到另一个节点了。 + + + + + diff --git a/docs/cn/features.md b/docs/cn/features.md new file mode 100644 index 00000000000..4bfe94421e3 --- /dev/null +++ b/docs/cn/features.md @@ -0,0 +1,82 @@ +# 特性(features) +## 订阅与发布 +消息的发布是指某个生产者向某个topic发送消息;消息的订阅是指某个消费者关注了某个topic中带有某些tag的消息,进而从该topic消费数据。 +## 消息顺序 +消息有序指的是一类消息消费时,能按照发送的顺序来消费。例如:一个订单产生了三条消息分别是订单创建、订单付款、订单完成。消费时要按照这个顺序消费才能有意义,但是同时订单之间是可以并行消费的。RocketMQ可以严格的保证消息有序。 + +顺序消息分为全局顺序消息与分区顺序消息,全局顺序是指某个Topic下的所有消息都要保证顺序;部分顺序消息只要保证每一组消息被顺序消费即可。 +- 全局顺序 +对于指定的一个 Topic,所有消息按照严格的先入先出(FIFO)的顺序进行发布和消费。 +适用场景:性能要求不高,所有的消息严格按照 FIFO 原则进行消息发布和消费的场景 +- 分区顺序 +对于指定的一个 Topic,所有消息根据 sharding key 进行区块分区。 同一个分区内的消息按照严格的 FIFO 顺序进行发布和消费。 Sharding key 是顺序消息中用来区分不同分区的关键字段,和普通消息的 Key 是完全不同的概念。 +适用场景:性能要求高,以 sharding key 作为分区字段,在同一个区块中严格的按照 FIFO 原则进行消息发布和消费的场景。 +## 消息过滤 +RocketMQ的消费者可以根据Tag进行消息过滤,也支持自定义属性过滤。消息过滤目前是在Broker端实现的,优点是减少了对于Consumer无用消息的网络传输,缺点是增加了Broker的负担、而且实现相对复杂。 +## 消息可靠性 +RocketMQ支持消息的高可靠,影响消息可靠性的几种情况: +1) Broker正常关闭 +2) Broker异常Crash +3) OS Crash +4) 机器掉电,但是能立即恢复供电情况 +5) 机器无法开机(可能是cpu、主板、内存等关键设备损坏) +6) 磁盘设备损坏 + +1)、2)、3)、4) 四种情况都属于硬件资源可立即恢复情况,RocketMQ在这四种情况下能保证消息不丢,或者丢失少量数据(依赖刷盘方式是同步还是异步)。 + +5)、6)属于单点故障,且无法恢复,一旦发生,在此单点上的消息全部丢失。RocketMQ在这两种情况下,通过异步复制,可保证99%的消息不丢,但是仍然会有极少量的消息可能丢失。通过同步双写技术可以完全避免单点,同步双写势必会影响性能,适合对消息可靠性要求极高的场合,例如与Money相关的应用。注:RocketMQ从3.0版本开始支持同步双写。 + +## 至少一次 +至少一次(At least Once)指每个消息必须投递一次。Consumer先Pull消息到本地,消费完成后,才向服务器返回ack,如果没有消费一定不会ack消息,所以RocketMQ可以很好的支持此特性。 + +## 回溯消费 +回溯消费是指Consumer已经消费成功的消息,由于业务上需求需要重新消费,要支持此功能,Broker在向Consumer投递成功消息后,消息仍然需要保留。并且重新消费一般是按照时间维度,例如由于Consumer系统故障,恢复后需要重新消费1小时前的数据,那么Broker要提供一种机制,可以按照时间维度来回退消费进度。RocketMQ支持按照时间回溯消费,时间维度精确到毫秒。 + +## 事务消息 +RocketMQ事务消息(Transactional Message)是指应用本地事务和发送消息操作可以被定义到全局事务中,要么同时成功,要么同时失败。RocketMQ的事务消息提供类似 X/Open XA 的分布事务功能,通过事务消息能达到分布式事务的最终一致。 +## 定时消息 +定时消息(延迟队列)是指消息发送到broker后,不会立即被消费,等待特定时间投递给真正的topic。 +broker有配置项messageDelayLevel,默认值为“1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h”,18个level。可以配置自定义messageDelayLevel。注意,messageDelayLevel是broker的属性,不属于某个topic。发消息时,设置delayLevel等级即可:msg.setDelayLevel(level)。level有以下三种情况: + +- level == 0,消息为非延迟消息 +- 1<=level<=maxLevel,消息延迟特定时间,例如level==1,延迟1s +- level > maxLevel,则level== maxLevel,例如level==20,延迟2h + +定时消息会暂存在名为SCHEDULE_TOPIC_XXXX的topic中,并根据delayTimeLevel存入特定的queue,queueId = delayTimeLevel – 1,即一个queue只存相同延迟的消息,保证具有相同发送延迟的消息能够顺序消费。broker会调度地消费SCHEDULE_TOPIC_XXXX,将消息写入真实的topic。 + +需要注意的是,定时消息会在第一次写入和调度写入真实topic时都会计数,因此发送数量、tps都会变高。 + +## 消息重试 +Consumer消费消息失败后,要提供一种重试机制,令消息再消费一次。Consumer消费消息失败通常可以认为有以下几种情况: +- 由于消息本身的原因,例如反序列化失败,消息数据本身无法处理(例如话费充值,当前消息的手机号被注销,无法充值)等。这种错误通常需要跳过这条消息,再消费其它消息,而这条失败的消息即使立刻重试消费,99%也不成功,所以最好提供一种定时重试机制,即过10秒后再重试。 +- 由于依赖的下游应用服务不可用,例如db连接不可用,外系统网络不可达等。遇到这种错误,即使跳过当前失败的消息,消费其他消息同样也会报错。这种情况建议应用sleep 30s,再消费下一条消息,这样可以减轻Broker重试消息的压力。 + +RocketMQ会为每个消费组都设置一个Topic名称为“%RETRY%+consumerGroup”的重试队列(这里需要注意的是,这个Topic的重试队列是针对消费组,而不是针对每个Topic设置的),用于暂时保存因为各种异常而导致Consumer端无法消费的消息。考虑到异常恢复起来需要一些时间,会为重试队列设置多个重试级别,每个重试级别都有与之对应的重新投递延时,重试次数越多投递延时就越大。RocketMQ对于重试消息的处理是先保存至Topic名称为“SCHEDULE_TOPIC_XXXX”的延迟队列中,后台定时任务按照对应的时间进行Delay后重新保存至“%RETRY%+consumerGroup”的重试队列中。 +## 消息重投 +生产者在发送消息时,同步消息失败会重投,异步消息有重试,oneway没有任何保证。消息重投保证消息尽可能发送成功、不丢失,但可能会造成消息重复,消息重复在RocketMQ中是无法避免的问题。消息重复在一般情况下不会发生,当出现消息量大、网络抖动,消息重复就会是大概率事件。另外,生产者主动重发、consumer负载变化也会导致重复消息。如下方法可以设置消息重试策略: + +- retryTimesWhenSendFailed:同步发送失败重投次数,默认为2,因此生产者会最多尝试发送retryTimesWhenSendFailed + 1次。不会选择上次失败的broker,尝试向其他broker发送,最大程度保证消息不丢。超过重投次数,抛出异常,由客户端保证消息不丢。当出现RemotingException、MQClientException和部分MQBrokerException时会重投。 +- retryTimesWhenSendAsyncFailed:异步发送失败重试次数,异步重试不会选择其他broker,仅在同一个broker上做重试,不保证消息不丢。 +- retryAnotherBrokerWhenNotStoreOK:消息刷盘(主或备)超时或slave不可用(返回状态非SEND_OK),是否尝试发送到其他broker,默认false。十分重要消息可以开启。 +## 流量控制 +生产者流控,因为broker处理能力达到瓶颈;消费者流控,因为消费能力达到瓶颈。 + +生产者流控: +- commitLog文件被锁时间超过osPageCacheBusyTimeOutMills时,参数默认为1000ms,返回流控。 +- 如果开启transientStorePoolEnable == true,且broker为异步刷盘的主机,且transientStorePool中资源不足,拒绝当前send请求,返回流控。 +- broker每隔10ms检查send请求队列头部请求的等待时间,如果超过waitTimeMillsInSendQueue,默认200ms,拒绝当前send请求,返回流控。 +- broker通过拒绝send 请求方式实现流量控制。 + +注意,生产者流控,不会尝试消息重投。 + +消费者流控: +- 消费者本地缓存消息数超过pullThresholdForQueue时,默认1000。 +- 消费者本地缓存消息大小超过pullThresholdSizeForQueue时,默认100MB。 +- 消费者本地缓存消息跨度超过consumeConcurrentlyMaxSpan时,默认2000。 + +消费者流控的结果是降低拉取频率。 +## 死信队列 +死信队列用于处理无法被正常消费的消息。当一条消息初次消费失败,消息队列会自动进行消息重试;达到最大重试次数后,若消费依然失败,则表明消费者在正常情况下无法正确地消费该消息,此时,消息队列 不会立刻将消息丢弃,而是将其发送到该消费者对应的特殊队列中。 + +RocketMQ将这种正常情况下无法被消费的消息称为死信消息(Dead-Letter Message),将存储死信消息的特殊队列称为死信队列(Dead-Letter Queue)。在RocketMQ中,可以通过使用console控制台对死信队列中的消息进行重发来使得消费者实例再次进行消费。 + diff --git a/docs/cn/image/rocketmq_architecture_1.png b/docs/cn/image/rocketmq_architecture_1.png index 548c8752a84..b8120b27221 100644 Binary files a/docs/cn/image/rocketmq_architecture_1.png and b/docs/cn/image/rocketmq_architecture_1.png differ diff --git a/docs/cn/image/rocketmq_design_1.png b/docs/cn/image/rocketmq_design_1.png new file mode 100644 index 00000000000..8c719e115a1 Binary files /dev/null and b/docs/cn/image/rocketmq_design_1.png differ diff --git a/docs/cn/image/rocketmq_design_10.png b/docs/cn/image/rocketmq_design_10.png new file mode 100644 index 00000000000..df75da58775 Binary files /dev/null and b/docs/cn/image/rocketmq_design_10.png differ diff --git a/docs/cn/image/rocketmq_design_11.png b/docs/cn/image/rocketmq_design_11.png new file mode 100644 index 00000000000..e3dc741d49c Binary files /dev/null and b/docs/cn/image/rocketmq_design_11.png differ diff --git a/docs/cn/image/rocketmq_design_12.png b/docs/cn/image/rocketmq_design_12.png new file mode 100644 index 00000000000..bf95dbc4c80 Binary files /dev/null and b/docs/cn/image/rocketmq_design_12.png differ diff --git a/docs/cn/image/rocketmq_design_13.png b/docs/cn/image/rocketmq_design_13.png new file mode 100644 index 00000000000..32ba4551f53 Binary files /dev/null and b/docs/cn/image/rocketmq_design_13.png differ diff --git a/docs/cn/image/rocketmq_design_2.png b/docs/cn/image/rocketmq_design_2.png new file mode 100644 index 00000000000..1610ae0d934 Binary files /dev/null and b/docs/cn/image/rocketmq_design_2.png differ diff --git a/docs/cn/image/rocketmq_design_3.png b/docs/cn/image/rocketmq_design_3.png new file mode 100644 index 00000000000..a0796edb9cc Binary files /dev/null and b/docs/cn/image/rocketmq_design_3.png differ diff --git a/docs/cn/image/rocketmq_design_4.png b/docs/cn/image/rocketmq_design_4.png new file mode 100644 index 00000000000..bc8981855ec Binary files /dev/null and b/docs/cn/image/rocketmq_design_4.png differ diff --git a/docs/cn/image/rocketmq_design_5.png b/docs/cn/image/rocketmq_design_5.png new file mode 100644 index 00000000000..a52d31ebe30 Binary files /dev/null and b/docs/cn/image/rocketmq_design_5.png differ diff --git a/docs/cn/image/rocketmq_design_6.png b/docs/cn/image/rocketmq_design_6.png new file mode 100644 index 00000000000..8b675d8555b Binary files /dev/null and b/docs/cn/image/rocketmq_design_6.png differ diff --git a/docs/cn/image/rocketmq_design_7.png b/docs/cn/image/rocketmq_design_7.png new file mode 100644 index 00000000000..b0faa86c29c Binary files /dev/null and b/docs/cn/image/rocketmq_design_7.png differ diff --git a/docs/cn/image/rocketmq_design_8.png b/docs/cn/image/rocketmq_design_8.png new file mode 100644 index 00000000000..ab4a1fb68fd Binary files /dev/null and b/docs/cn/image/rocketmq_design_8.png differ diff --git a/docs/cn/image/rocketmq_design_9.png b/docs/cn/image/rocketmq_design_9.png new file mode 100644 index 00000000000..4af041641f5 Binary files /dev/null and b/docs/cn/image/rocketmq_design_9.png differ diff --git a/docs/cn/msg_trace/user_guide.md b/docs/cn/msg_trace/user_guide.md new file mode 100644 index 00000000000..0320e16e006 --- /dev/null +++ b/docs/cn/msg_trace/user_guide.md @@ -0,0 +1,106 @@ +# 消息轨迹 +## 前言 +该文档主要介绍如何快速部署和使用支持消息轨迹特性的RocketMQ 集群。 + +## 1. 消息轨迹数据关键属性 +| Producer端| Consumer端 | Broker端 | +| --- | --- | --- | +| 生产实例信息 | 消费实例信息 | 消息的Topic | +| 发送消息时间 | 投递时间,投递轮次  | 消息存储位置 | +| 消息是否发送成功 | 消息是否消费成功 | 消息的Key值 | +| 发送耗时 | 消费耗时 | 消息的Tag值 | + +## 2. 支持消息轨迹集群部署 + +### 2.1 Broker端配置文件 +这里贴出Broker端开启消息轨迹特性的properties配置文件内容: +``` +brokerClusterName=DefaultCluster +brokerName=broker-a +brokerId=0 +deleteWhen=04 +fileReservedTime=48 +brokerRole=ASYNC_MASTER +flushDiskType=ASYNC_FLUSH +storePathRootDir=/data/rocketmq/rootdir-a-m +storePathCommitLog=/data/rocketmq/commitlog-a-m +autoCreateSubscriptionGroup=true +## if msg tracing is open,the flag will be true +traceTopicEnable=true +listenPort=10911 +brokerIP1=XX.XX.XX.XX1 +namesrvAddr=XX.XX.XX.XX:9876 +``` + +### 2.2 普通模式 +RocketMQ集群中每一个Broker节点均用于存储Client端收集并发送过来的消息轨迹数据。因此,对于RocketMQ集群中的Broker节点数量并无要求和限制。 + +### 2.3 物理IO隔离模式 +对于消息轨迹数据量较大的场景,可以在RocketMQ集群中选择其中一个Broker节点专用于存储消息轨迹,使得用户普通的消息数据与消息轨迹数据的物理IO完全隔离,互不影响。在该模式下,RockeMQ集群中至少有两个Broker节点,其中一个Broker节点定义为存储消息轨迹数据的服务端。 + +### 2.4 启动开启消息轨迹的Broker +`nohup sh mqbroker -c ../conf/2m-noslave/broker-a.properties &` + +## 3. 保存消息轨迹的Topic定义 +RocketMQ的消息轨迹特性支持两种存储轨迹数据的方式: + +### 3.1 系统级的TraceTopic +在默认情况下,消息轨迹数据是存储于系统级的TraceTopic中(其名称为:**RMQ_SYS_TRACE_TOPIC**)。该Topic在Broker节点启动时,会自动创建出来(如上所叙,需要在Broker端的配置文件中将**traceTopicEnable**的开关变量设置为**true**)。 + +### 3.2 用户自定义的TraceTopic +如果用户不准备将消息轨迹的数据存储于系统级的默认TraceTopic,也可以自己定义并创建用户级的Topic来保存轨迹(即为创建普通的Topic用于保存消息轨迹数据)。下面一节会介绍Client客户端的接口如何支持用户自定义的TraceTopic。 + +## 4. 支持消息轨迹的Client客户端实践 +为了尽可能地减少用户业务系统使用RocketMQ消息轨迹特性的改造工作量,作者在设计时候采用对原来接口增加一个开关参数(**enableMsgTrace**)来实现消息轨迹是否开启;并新增一个自定义参(**customizedTraceTopic**)数来实现用户存储消息轨迹数据至自己创建的用户级Topic。 + +### 4.1 发送消息时开启消息轨迹 +``` + DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName",true); + producer.setNamesrvAddr("XX.XX.XX.XX1"); + producer.start(); + try { + { + Message msg = new Message("TopicTest", + "TagA", + "OrderID188", + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + SendResult sendResult = producer.send(msg); + System.out.printf("%s%n", sendResult); + } + + } catch (Exception e) { + e.printStackTrace(); + } +``` + +### 4.2 订阅消息时开启消息轨迹 +``` + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_JODIE_1",true); + consumer.subscribe("TopicTest", "*"); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + consumer.setConsumeTimestamp("20181109221800"); + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + System.out.printf("%s Receive New Messages: %s %n", Thread.currentThread().getName(), msgs); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + consumer.start(); + System.out.printf("Consumer Started.%n"); +``` + +### 4.3 支持自定义存储消息轨迹Topic +在上面的发送和订阅消息时候分别将DefaultMQProducer和DefaultMQPushConsumer实例的初始化修改为如下即可支持自定义存储消息轨迹Topic。 +``` + ##其中Topic_test11111需要用户自己预先创建,来保存消息轨迹; + DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName",true,"Topic_test11111"); + ...... + + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_JODIE_1",true,"Topic_test11111"); + ...... +``` + + + + diff --git a/docs/cn/operation.md b/docs/cn/operation.md new file mode 100644 index 00000000000..c00be933a7f --- /dev/null +++ b/docs/cn/operation.md @@ -0,0 +1,1410 @@ +## 运维管理(operation) + +### 1 集群搭建 + +#### 1.1 单Master模式 + +这种方式风险较大,一旦Broker重启或者宕机时,会导致整个服务不可用。不建议线上环境使用,可以用于本地测试。 + +##### 1)启动 NameServer + +```bash +### 首先启动Name Server +$ nohup sh mqnamesrv & + +### 验证Name Server 是否启动成功 +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +##### 2)启动 Broker + +```bash +### 启动Broker +$ nohup sh bin/mqbroker -n localhost:9876 & + +### 验证Name Server 是否启动成功,例如Broker的IP为:192.168.1.2,且名称为broker-a +$ tail -f ~/logs/rocketmqlogs/Broker.log +The broker[broker-a, 192.169.1.2:10911] boot success... +``` + +#### 1.2 多Master模式 + +一个集群无Slave,全是Master,例如2个Master或者3个Master,这种模式的优缺点如下: + +- 优点:配置简单,单个Master宕机或重启维护对应用无影响,在磁盘配置为RAID10时,即使机器宕机不可恢复情况下,由于RAID10磁盘非常可靠,消息也不会丢(异步刷盘丢失少量消息,同步刷盘一条不丢),性能最高; + +- 缺点:单台机器宕机期间,这台机器上未被消费的消息在机器恢复之前不可订阅,消息实时性会受到影响。 + +##### 1)启动NameServer + +NameServer需要先于Broker启动,且如果在生产环境使用,为了保证高可用,建议一般规模的集群启动3个NameServer,各节点的启动命令相同,如下: + +```bash +### 首先启动Name Server +$ nohup sh mqnamesrv & + +### 验证Name Server 是否启动成功 +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +##### 2)启动Broker集群 + +```bash +### 在机器A,启动第一个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-noslave/broker-a.properties & + +### 在机器B,启动第二个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-noslave/broker-b.properties & + +... +``` + +如上启动命令是在单个NameServer情况下使用的。对于多个NameServer的集群,Broker启动命令中`-n`后面的地址列表用分号隔开即可,例如 `192.168.1.1:9876;192.161.2:9876`。 + +#### 1.3 多Master多Slave模式-异步复制 + +每个Master配置一个Slave,有多对Master-Slave,HA采用异步复制方式,主备有短暂消息延迟(毫秒级),这种模式的优缺点如下: + +- 优点:即使磁盘损坏,消息丢失的非常少,且消息实时性不会受影响,同时Master宕机后,消费者仍然可以从Slave消费,而且此过程对应用透明,不需要人工干预,性能同多Master模式几乎一样; + +- 缺点:Master宕机,磁盘损坏情况下会丢失少量消息。 + +##### 1)启动NameServer + +```bash +### 首先启动Name Server +$ nohup sh mqnamesrv & + +### 验证Name Server 是否启动成功 +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +##### 2)启动Broker集群 + +```bash +### 在机器A,启动第一个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-a.properties & + +### 在机器B,启动第二个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-b.properties & + +### 在机器C,启动第一个Slave,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-a-s.properties & + +### 在机器D,启动第二个Slave,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-b-s.properties & +``` + +#### 1.4 多Master多Slave模式-同步双写 + +每个Master配置一个Slave,有多对Master-Slave,HA采用同步双写方式,即只有主备都写成功,才向应用返回成功,这种模式的优缺点如下: + +- 优点:数据与服务都无单点故障,Master宕机情况下,消息无延迟,服务可用性与数据可用性都非常高; + +- 缺点:性能比异步复制模式略低(大约低10%左右),发送单个消息的RT会略高,且目前版本在主节点宕机后,备机不能自动切换为主机。 + +##### 1)启动NameServer + +```bash +### 首先启动Name Server +$ nohup sh mqnamesrv & + +### 验证Name Server 是否启动成功 +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +##### 2)启动Broker集群 + +```bash +### 在机器A,启动第一个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-a.properties & + +### 在机器B,启动第二个Master,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-b.properties & + +### 在机器C,启动第一个Slave,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-a-s.properties & + +### 在机器D,启动第二个Slave,例如NameServer的IP为:192.168.1.1 +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-b-s.properties & +``` + +以上Broker与Slave配对是通过指定相同的BrokerName参数来配对,Master的BrokerId必须是0,Slave的BrokerId必须是大于0的数。另外一个Master下面可以挂载多个Slave,同一Master下的多个Slave通过指定不同的BrokerId来区分。$ROCKETMQ_HOME指的RocketMQ安装目录,需要用户自己设置此环境变量。 + +### 2 mqadmin管理工具 + +> 注意: +> +> 1. 执行命令方法:`./mqadmin {command} {args}` +> 2. 几乎所有命令都需要配置-n表示NameServer地址,格式为ip:port +> 3. 几乎所有命令都可以通过-h获取帮助 +> 4. 如果既有Broker地址(-b)配置项又有clusterName(-c)配置项,则优先以Broker地址执行命令,如果不配置Broker地址,则对集群中所有主机执行命令,只支持一个Broker地址。-b格式为ip:port,port默认是10911 +> 5. 在tools下可以看到很多命令,但并不是所有命令都能使用,只有在MQAdminStartup中初始化的命令才能使用,你也可以修改这个类,增加或自定义命令 +> 6. 由于版本更新问题,少部分命令可能未及时更新,遇到错误请直接阅读相关命令源码 + +#### 2.1 Topic相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
updateTopic创建更新Topic配置-bBroker 地址,表示 topic 所在 + Broker,只支持单台Broker,地址为ip:port
-ccluster 名称,表示 topic 所在集群(集群可通过 + clusterList 查询)
-h-打印帮助
-nNameServer服务地址,格式 ip:port
-p指定新topic的读写权限( W=2|R=4|WR=6 )
-r可读队列数(默认为 8)
-w可写队列数(默认为 8)
-ttopic 名称(名称只能使用字符 + ^[a-zA-Z0-9_-]+$ )
deleteTopic删除Topic-ccluster 名称,表示删除某集群下的某个 topic (集群 + 可通过 clusterList 查询)
-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic 名称(名称只能使用字符 + ^[a-zA-Z0-9_-]+$ )
topicList查看 Topic 列表信息-h打印帮助
-c不配置-c只返回topic列表,增加-c返回clusterName, + topic, consumerGroup信息,即topic的所属集群和订阅关系,没有参数
-nNameServer 服务地址,格式 ip:port
topicRoute查看 Topic 路由信息-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
topicStatus查看 Topic 消息队列offset-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
topicClusterList查看 Topic 所在集群列表-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
updateTopicPerm更新 Topic 读写权限-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
-bBroker 地址,表示 topic 所在 + Broker,只支持单台Broker,地址为ip:port
-p指定新 topic 的读写权限( W=2|R=4|WR=6 )
-ccluster 名称,表示 topic 所在集群(集群可通过 + clusterList 查询),-b优先,如果没有-b,则对集群中所有Broker执行命令
updateOrderConf从NameServer上创建、删除、获取特定命名空间的kv配置,目前还未启用-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic,键
-vorderConf,值
-mmethod,可选get、put、delete
allocateMQ以平均负载算法计算消费者列表负载消息队列的负载结果-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
-iipList,用逗号分隔,计算这些ip去负载Topic的消息队列
statsAll打印Topic订阅关系、TPS、积累量、24h读写总量等信息-h打印帮助
-nNameServer 服务地址,格式 ip:port
-a是否只打印活跃topic
-t指定topic
+ + + +#### 2.2 集群相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
clusterList查看集群信息,集群、BrokerName、BrokerId、TPS等信息-m打印更多信息 (增加打印出如下信息 #InTotalYest, + #OutTotalYest, #InTotalToday ,#OutTotalToday)
-h打印帮助
-nNameServer 服务地址,格式 ip:port
-i打印间隔,单位秒
clusterRT发送消息检测集群各Broker RT。消息发往${BrokerName} Topic。-aamount,每次探测的总数,RT = 总时间 / + amount
-s消息大小,单位B
-c探测哪个集群
-p是否打印格式化日志,以|分割,默认不打印
-h打印帮助
-m所属机房,打印使用
-i发送间隔,单位秒
-nNameServer 服务地址,格式 ip:port
+ + +#### 2.3 Broker相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
updateBrokerConfig更新 Broker 配置文件,会修改Broker.conf-bBroker 地址,格式为ip:port
-ccluster 名称
-kkey 值
-vvalue 值
-h打印帮助
-nNameServer 服务地址,格式 ip:port
brokerStatus查看 Broker 统计信息、运行状态(你想要的信息几乎都在里面)-bBroker 地址,地址为ip:port
-h打印帮助
-nNameServer 服务地址,格式 ip:port
brokerConsumeStatsBroker中各个消费者的消费情况,按Message Queue维度返回Consume + Offset,Broker Offset,Diff,TImestamp等信息-bBroker 地址,地址为ip:port
-t请求超时时间
-ldiff阈值,超过阈值才打印
-o是否为顺序topic,一般为false
-h打印帮助
-nNameServer 服务地址,格式 ip:port
getBrokerConfig获取Broker配置-bBroker 地址,地址为ip:port
-nNameServer 服务地址,格式 ip:port
wipeWritePerm从NameServer上清除 Broker写权限-bBroker 地址,地址为ip:port
-nNameServer 服务地址,格式 ip:port
-h打印帮助
cleanExpiredCQ清理Broker上过期的Consume Queue,如果手动减少对列数可能产生过期队列-nNameServer 服务地址,格式 ip:port
-h打印帮助
-bBroker 地址,地址为ip:port
-c集群名称
cleanUnusedTopic清理Broker上不使用的Topic,从内存中释放Topic的Consume + Queue,如果手动删除Topic会产生不使用的Topic-nNameServer 服务地址,格式 ip:port
-h打印帮助
-bBroker 地址,地址为ip:port
-c集群名称
sendMsgStatus向Broker发消息,返回发送状态和RT-nNameServer 服务地址,格式 ip:port
-h打印帮助
-bBrokerName,注意不同于Broker地址
-s消息大小,单位B
-c发送次数
+ + +#### 2.4 消息相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
queryMsgById根据offsetMsgId查询msg,如果使用开源控制台,应使用offsetMsgId,此命令还有其他参数,具体作用请阅读QueryMsgByIdSubCommand。-imsgId
-h打印帮助
-nNameServer 服务地址,格式 ip:port
queryMsgByKey根据消息 Key 查询消息-kmsgKey
-tTopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
queryMsgByOffset根据 Offset 查询消息-bBroker 名称,(这里需要注意 + 填写的是 Broker 的名称,不是 Broker 的地址,Broker 名称可以在 clusterList 查到)
-iquery 队列 id
-ooffset 值
-ttopic 名称
-h打印帮助
-nNameServer 服务地址,格式 ip:port
queryMsgByUniqueKey根据msgId查询,msgId不同于offsetMsgId,区别详见常见运维问题。-g,-d配合使用,查到消息后尝试让特定的消费者消费消息并返回消费结果-h打印帮助
-nNameServer 服务地址,格式 ip:port
-iuniqe msg id
-gconsumerGroup
-dclientId
-ttopic名称
checkMsgSendRT检测向topic发消息的RT,功能类似clusterRT-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic名称
-a探测次数
-s消息大小
sendMessage发送一条消息,可以根据配置发往特定Message Queue,或普通发送。-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic名称
-pbody,消息体
-kkeys
-ctags
-bBrokerName
-iqueueId
consumeMessage消费消息。可以根据offset、开始&结束时间戳、消息队列消费消息,配置不同执行不同消费逻辑,详见ConsumeMessageCommand。-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic名称
-bBrokerName
-o从offset开始消费
-iqueueId
-g消费者分组
-s开始时间戳,格式详见-h
-d结束时间戳
-c消费多少条消息
printMsg从Broker消费消息并打印,可选时间段-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic名称
-c字符集,例如UTF-8
-ssubExpress,过滤表达式
-b开始时间戳,格式参见-h
-e结束时间戳
-d是否打印消息体
printMsgByQueue类似printMsg,但指定Message Queue-h打印帮助
-nNameServer 服务地址,格式 ip:port
-ttopic名称
-iqueueId
-aBrokerName
-c字符集,例如UTF-8
-ssubExpress,过滤表达式
-b开始时间戳,格式参见-h
-e结束时间戳
-p是否打印消息
-d是否打印消息体
-f是否统计tag数量并打印
resetOffsetByTime按时间戳重置offset,Broker和consumer都会重置-h打印帮助
-nNameServer 服务地址,格式 ip:port
-g消费者分组
-ttopic名称
-s重置为此时间戳对应的offset
-f是否强制重置,如果false,只支持回溯offset,如果true,不管时间戳对应offset与consumeOffset关系
-c是否重置c++客户端offset
+ + +#### 2.5 消费者、消费组相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
consumerProgress查看订阅组消费状态,可以查看具体的client IP的消息积累量-g消费者所属组名
-s是否打印client IP
-h打印帮助
-nNameServer 服务地址,格式 ip:port
consumerStatus查看消费者状态,包括同一个分组中是否都是相同的订阅,分析Process + Queue是否堆积,返回消费者jstack结果,内容较多,使用者参见ConsumerStatusSubCommand-h打印帮助
-nNameServer 服务地址,格式 ip:port
-gconsumer group
-iclientId
-s是否执行jstack
getConsumerStatus获取 Consumer 消费进度-g消费者所属组名
-t查询主题
-iConsumer 客户端 ip
-nNameServer 服务地址,格式 ip:port
-h打印帮助
updateSubGroup更新或创建订阅关系-nNameServer 服务地址,格式 ip:port
-h打印帮助
-bBroker地址
-c集群名称
-g消费者分组名称
-s分组是否允许消费
-m是否从最小offset开始消费
-d是否是广播模式
-q重试队列数量
-r最大重试次数
-i当slaveReadEnable开启时有效,且还未达到从slave消费时建议从哪个BrokerId消费,可以配置备机id,主动从备机消费
-w如果Broker建议从slave消费,配置决定从哪个slave消费,配置BrokerId,例如1
-a当消费者数量变化时是否通知其他消费者负载均衡
deleteSubGroup从Broker删除订阅关系-nNameServer 服务地址,格式 ip:port
-h打印帮助
-bBroker地址
-c集群名称
-g消费者分组名称
cloneGroupOffset在目标群组中使用源群组的offset-nNameServer 服务地址,格式 ip:port
-h打印帮助
-s源消费者组
-d目标消费者组
-ttopic名称
-o暂未使用
+ + + + +#### 2.6 连接相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
consumerConnec tion查询 Consumer 的网络连接-g消费者所属组名
-nNameServer 服务地址,格式 ip:port
-h打印帮助
producerConnec tion查询 Producer 的网络连接-g生产者所属组名
-t主题名称
-nNameServer 服务地址,格式 ip:port
-h打印帮助
+ + + + +#### 2.7 NameServer相关 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
updateKvConfig更新NameServer的kv配置,目前还未使用-s命名空间
-kkey
-vvalue
-nNameServer 服务地址,格式 ip:port
-h打印帮助
deleteKvConfig删除NameServer的kv配置-s命名空间
-kkey
-nNameServer 服务地址,格式 ip:port
-h打印帮助
getNamesrvConfig获取NameServer配置-nNameServer 服务地址,格式 ip:port
-h打印帮助
updateNamesrvConfig修改NameServer配置-nNameServer 服务地址,格式 ip:port
-h打印帮助
-kkey
-vvalue
+ + + + +#### 2.8 其他 + + + + + + + + + + + + + + + + + + + + + + +
名称含义命令选项说明
startMonitoring开启监控进程,监控消息误删、重试队列消息数等-nNameServer 服务地址,格式 ip:port
-h打印帮助
+ + +### 3 运维常见问题 + +#### 3.1 RocketMQ的mqadmin命令报错问题 + +> 问题描述:有时候在部署完RocketMQ集群后,尝试执行“mqadmin”一些运维命令,会出现下面的异常信息: +> +> ```java +> org.apache.rocketmq.remoting.exception.RemotingConnectException: connect to failed +> ``` + +解决方法:可以在部署RocketMQ集群的虚拟机上执行`export NAMESRV_ADDR=ip:9876`(ip指的是集群中部署NameServer组件的机器ip地址)命令之后再使用“mqadmin”的相关命令进行查询,即可得到结果。 + +#### 3.2 RocketMQ生产端和消费端版本不一致导致不能正常消费的问题 + +> 问题描述:同一个生产端发出消息,A消费端可消费,B消费端却无法消费,rocketMQ Console中出现: +> +> ```java +> Not found the consumer group consume stats, because return offset table is empty, maybe the consumer not consume any message的异常消息。 +> ``` + + 解决方案:RocketMQ 的jar包:rocketmq-client等包应该保持生产端,消费端使用相同的version。 + +#### 3.3 新增一个topic的消费组时,无法消费历史消息的问题 + +> 问题描述:当同一个topic的新增消费组启动时,消费的消息是当前的offset的消息,并未获取历史消息。 + +解决方案:rocketmq默认策略是从消息队列尾部,即跳过历史消息。如果想消费历史消息,则需要设置:`org.apache.rocketmq.client.consumer.DefaultMQPushConsumer#setConsumeFromWhere`。常用的有以下三种配置: + +- 默认配置,一个新的订阅组第一次启动从队列的最后位置开始消费,后续再启动接着上次消费的进度开始消费,即跳过历史消息; + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); +``` + +- 一个新的订阅组第一次启动从队列的最前位置开始消费,后续再启动接着上次消费的进度开始消费,即消费Broker未过期的历史消息; + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); +``` + +- 一个新的订阅组第一次启动从指定时间点开始消费,后续再启动接着上次消费的进度开始消费,和consumer.setConsumeTimestamp()配合使用,默认是半个小时以前; + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); +``` + +#### 3.4 如何开启从Slave读数据功能 + +在某些情况下,Consumer需要将消费位点重置到1-2天前,这时在内存有限的Master Broker上,CommitLog会承载比较重的IO压力,影响到该Broker的其它消息的读与写。可以开启`slaveReadEnable=true`,当Master Broker发现Consumer的消费位点与CommitLog的最新值的差值的容量超过该机器内存的百分比(`accessMessageInMemoryMaxRatio=40%`),会推荐Consumer从Slave Broker中去读取数据,降低Master Broker的IO。 + +#### 3.5 性能调优问题 + +异步刷盘建议使用自旋锁,同步刷盘建议使用重入锁,调整Broker配置项`useReentrantLockWhenPutMessage`,默认为false;异步刷盘建议开启`TransientStorePoolEnable`;建议关闭transferMsgByHeap,提高拉消息效率;同步刷盘建议适当增大`sendMessageThreadPoolNums`,具体配置需要经过压测。 + +#### 3.6 在RocketMQ中msgId和offsetMsgId的含义与区别 + +使用RocketMQ完成生产者客户端消息发送后,通常会看到如下日志打印信息: + +```java +SendResult [sendStatus=SEND_OK, msgId=0A42333A0DC818B4AAC246C290FD0000, offsetMsgId=0A42333A00002A9F000000000134F1F5, messageQueue=MessageQueue [topic=topicTest1, BrokerName=mac.local, queueId=3], queueOffset=4] +``` + +- msgId,对于客户端来说msgId是由客户端producer实例端生成的,具体来说,调用方法`MessageClientIDSetter.createUniqIDBuffer()`生成唯一的Id; +- offsetMsgId,offsetMsgId是由Broker服务端在写入消息时生成的(采用”IP地址+Port端口”与“CommitLog的物理偏移量地址”做了一个字符串拼接),其中offsetMsgId就是在RocketMQ控制台直接输入查询的那个messageId。 diff --git a/docs/en/CLITools.md b/docs/en/CLITools.md new file mode 100644 index 00000000000..909f002ff46 --- /dev/null +++ b/docs/en/CLITools.md @@ -0,0 +1,1220 @@ +# Instructions on the use of mqadmin Management tools + +Before introducing the mqadmin management tool, the following points need to be declared: + +- The way of executing a command is:./mqadmin {command} {args} +- Almost all commands need to attach the -n option to represent the nameServer address, formatted as ip:port; +- Almost all commands can get help information with the -h option; +- If the broker address -b option and clusterName -c option are both configured with specific values, the command execution will select the broker address specified by -b option. The value of the -b option can only be configured with a single address. The format is ip:port. The default port value is 10911. If the value of the -b option is not configured, the command will be applied to all brokers in the entire cluster. +- You can see many commands under tools, but not all commands can be used, only the commands initialized in MQAdminStartup can be used, you can also modify this class, add or customize commands; +- Due to the issue of version update, a small number of commands may not be updated in time, please read the related command source code to eliminate and resolve the error. + +## 1 Topic related command instructions + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
updateTopicCreate or update the configuration of topic-bThe -b option declares the specific address of the broker, indicating that the broker, in which the topic is located supports only a single broker and the address format is ip:port.
-cThe -c option declares the name of the cluster, which represents the cluster in which the current topic is located. (clusters are available through clusterList query)
-h-Print help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-pThe -p option is used to specify the read and write permission for the new topic (W=2 | R=4 | WR=6)
-rThe -r option declares the number of readable queues (default 8)
-wThe -w option declares the number of writable queues (default 8)
-tThe -t option declares the name of the topic (the name can only use characters^ [a-zA-Z0-9s -] + $)
deleteTopicDelete the topic command-cThe -c option specifies the name of the cluster, which means that one of the topic in the specified cluster is deleted (cluster names can be queried via clusterList)
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-tThe -t option declares the name of the topic (the name can only use characters^ [a-zA-Z0-9s -] + $)
topicListView topic list information-hPrint help information
-cIf the -c option is not configured, only the topic list is returned, and the addition of -c option returns additional information about the clusterName, topic, consumerGroup, that is, the cluster and subscription to which the topic belongs, and no other option need to be configured.
-nDeclare the service address of the nameServer, and the option format is ip:port
topicRouteTo view topic specific routing information-tUsed to specify the name of the topic
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
topicStatusThe location of the offset used to view the topic message queue-tUsed to specify the name of the topic
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
topicClusterListTo view the list of clusters to which topic belongs-tUsed to specify the name of the topic
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
updateTopicPermThis command is used to update read and write permissions for topic-tUsed to specify the name of the topic
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-bThe -b option declares the specific address of the broker, indicating that the broker, in which the topic is located supports only a single broker and the address format is ip:port.
-pThe -p option is used to specify the read and write permission for the new topic (W=2 | R=4 | WR=6)
-cUsed to specify the name of the cluster that represents the cluster in which the topic is located, which can be accessed through the clusterList query, but the -b parameter has a higher priority, and if no -b option related configuration is specified, the command is executed on all broker in the cluster
updateOrderConfThe key, value configuration that creates, deletes, and retrieves specific namespaces from nameServer is not yet enabled.-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-ttopic,key
-vorderConf,value
-mmethod,available values include get, put, delete
allocateMQComputing load result of load message queue in consumer list with average load algorithm-tUsed to specify the name of the topic
-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-iIpList, is separated by commas to calculate which message queues these ip unload topic
statsAllFor printing topic subscription, TPS, cumulative amount, 24 hours read and write total, etc.-hPrint help information
-nDeclare the service address of the nameServer, and the option format is ip:port
-aWhether to print only active topic
-tUsed to specify the name of the topic
+ + + + + + + +## 2 Cluster related command instructions + +#### + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
clusterListView cluster information, cluster, brokerName, brokerId, TPS, and so on-mPrint more information (add print to # InTotalYest, + #OutTotalYest, #InTotalToday ,#OutTotalToday)
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-iPrint interval,unit basis is seconds
clusterRTSend message to detect each broker RT of the cluster.the message send to ${BrokerName} Topic-aamount,total number per probe,RT = Total time/amount
-sMessage size,unit basis is B
-cWhich cluster to detect.
-pWhether to print the formatted log,split with "|", not printed by default
-hPrint help information
-mOwned computer room for printing
-iThe interval, in seconds, at which a message is sent.
-nService address used to specify nameServer and formatted as ip:port
+ + + + + +## 3 Broker related command instructions + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
updateBrokerConfigThe configuration information used to update the broker and the contents of the Broker.conf file are modified-bDeclare the address of the broker and format as ip:port
-cSpecify the name of the cluster
-kthe value of k
-vthe value of value
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
brokerStatusFor viewing broker related statistics and running status (almost all the information you want is inside)-bDeclare the address of the broker and format as ip:port
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
brokerConsumeStatsGet the consumption of each consumer in broker and return information such as consume Offset,broker Offset,diff,timestamp by message queue dimension-bDeclare the address of the broker and format as ip:port
-tConfigure the timeout of the request
-lConfigure the diff threshold beyond which to print
-oSpecifies whether the order topic, is typically false
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
getBrokerConfigGet configuration information for the broker-bDeclare the address of the broker and format as ip:port
-nService address used to specify nameServer and formatted as ip:port
wipeWritePermClear write permissions for broker from nameServer-bDeclare the address of the broker and format as ip:port
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
cleanExpiredCQClean up expired consume Queue on broker,An expired queue may be generated if the number of columns is reduced manually-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-bDeclare the address of the broker and format as ip:port
-cUsed to specify the name of the cluster
cleanUnusedTopicClean up unused topic on broker and release topic's consume Queue from memory,If the topic is removed manually, an unused topic will be generated-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-bDeclare the address of the broker and format as ip:port
-cUsed to specify the name of the cluster
sendMsgStatusSend a message to the broker and then return the send status and RT-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-bbrokerName,note that this is not broker's address
-sMessage size,the unit of account is B
-cNumber of messages sent
+ + + +## 4 Message related command instructions + +#### + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
queryMsgByIdQuery msg according to offsetMsgId. If you use open source console, you should use offsetMsgId. There are other parameters for this command. For details, please read QueryMsgByIdSubCommand. +-imsgId
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
queryMsgByKeyQuery messages based on message Key-kmsgKey
-tThe name of the topic
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
queryMsgByOffsetQuery messages based on Offset-bThe name of broker,(Note here: the name of broker is filled in, not the address of broker, and the broker name can be found in clusterList)
-iQueue id of the query
-oThe value of offset
-tThe name of the topic
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
queryMsgByUniqueKeyAccording to the msgId query, msgId is different from offsetMsgId. The specific differences can be found in common operational and maintenance problems. "-g" option and "-d" option are to be used together, and when you find the message, try to get a particular consumer to consume the message and return the result of the consumption.-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-iuniqe msg id
-gconsumerGroup
-dclientId
-tThe name of the topic
checkMsgSendRTDetect RT to send a message to topic, function similar to clusterRT-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-tThe name of the topic
-athe number of probes
-sThe size of message
sendMessageSend a message that can be sent, as configured, to a particular message Queue, or to a normal send.-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-tThe name of the topic
-pbody,message body
-kkeys
-ctags
-bbrokerName
-iqueueId
consumeMessageConsumer messages. You can consume messages based on offset, start timestamps, end timestamps, message queues, and configure different consumption logic for different execution, as detailed in ConsumeMessageCommand.-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-tThe name of the topic
-bbrokerName
-oStart consumption from offset
-iqueueId
-gGroup of consumers
-sSpecify a start timestamp in a format see -h
-dSpecify a end timestamp
-cSpecify how many messages to consume
printMsgConsume messages from broker and print them, optional time periods-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-tThe name of the topic
-cCharacter set,for example UTF-8
-ssubExpress,filter expression
-bSpecify a start timestamp in a format see -h
-eSpecify the end timestamp
-dWhether to print the message body
printMsgByQueueSimilar to printMsg, but specifying message queue-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-tThe name of the topic
-iqueueId
-abrokerName
-cCharacter set,for example UTF-8
-ssubExpress,filter expression
-bSpecify a start timestamp in a format see -h
-eSpecify the end timestamp
-pWhether to print a message
-dWhether to print the message body
-fWhether to count the number of tags and print +
resetOffsetByTimeReset both offset,broker and consumer by timestamp-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-gGroup of consumers
-tThe name of the topic
-sResets the offset corresponding to this timestamp
-fWhether to force a reset, if set to false, only supports backtracking offset, if it is true, regardless of the relationship between offset and consume Offset with the timestamp
-cWhether to reset the C++ client offset
+ + + +## 5 Consumer and Consumer Group related command instructions + +#### + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
consumerProgressTo view the subscriber consumption status, you can see the amount of message accumulation for a specific client IP-gThe group name of consumer
-sWhether to print client IP
-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
consumerStatusSee the consumer status, including whether the same subscription is in the same group, analyze whether the process queue is stacked, return the consumer jstack results, more content, and see ConsumerStatusSubCommand for the user-hPrint help information
-nService address used to specify nameServer and formatted as ip:port
-gconsumer group
-iclientId
-sWhether to execute jstack
getConsumerStatusGet Consumer consumption progress-gthe group name of consumer
-tQuery topic
-iIp address of consumer client
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
updateSubGroupUpdate or create a subscription-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-bthe address of broker
-cThe name of cluster
-gThe group name of consumer
-sWhether the group is allowed to consume
-mWhether to start consumption from the minimum offset
-dIs it a broadcast mode
-qThe Number of retry queues
-rMaximum number of retries
-iWhen the slaveReadEnable is on and which brokerId consumption is recommended for consumption from slave, the brokerid of slave, can be configured to consume from the slave actively
-wIf broker recommends consumption from slave, configuration determines which slave consumption to consume from, and configure a specific brokerId, such as 1
-aWhether to notify other consumers of load balancing when the number of consumers changes
deleteSubGroupRemove subscriptions from broker-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-bthe address of broker
-cThe name of cluster
-gThe group name of consumer
cloneGroupOffsetUse the offset of the source group in the target group +-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-sSource consumer group
-dTarget consumer group
-tThe name of topic
-oNot used yet
+ + + + +## 6 Connection related command instructions + +#### + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
consumerConnec tionQuery the network connection of consumer-gThe group name of consumer
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
producerConnec tionQuery the network connection of producer-gthe group name of producer
-tThe name of topic
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
+ + + + +## 7 NameServer related command instructions + +#### + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
updateKvConfigUpdate the kv configuration of nameServer, which is not currently used-sSpecify a specific namespace
-kkey
-vvalue
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
deleteKvConfig Delete the kv configuration of nameServer-sSpecify a specific namespace
-kkey
-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
getNamesrvConfigGet the configuration of the nameServer-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
updateNamesrvConfigModifying the configuration of nameServer-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
-kThe value of key
-vThe value of value
+ + + + +## 8 Other relevant command notes + +#### + + + + + + + + + + + + + + + + + + + + + + +
NameMeaningCommand optionExplain
startMonitoringUsed to start the monitoring process, monitor message deletion, retry queue messages, etc.-nService address used to specify nameServer and formatted as ip:port
-hPrint help information
+ + diff --git a/docs/en/Concept.md b/docs/en/Concept.md new file mode 100644 index 00000000000..490b98c1567 --- /dev/null +++ b/docs/en/Concept.md @@ -0,0 +1,41 @@ +# Basic Concept + +## 1 Message Model + +RocketMQ message model is mainly composed of Producer, Broker and Consumer. The producer is responsible for producing messages and the consumer is for consuming messages, while the broker stores messages. +The broker is an independent server during actual deployment, and each broker can store messages from multiple topics. Even messages from the same topic can be stored in the different brokers by sharding strategy. +The message queue is used to store physical offsets of messages, and the message addresses are stored in seperate queues. The consumer group consists of multiple consumer instances. +## 2 Producer +The Producer is responsible for producing messages, typically by business systems. It sends messages generated by the systems to brokers. RocketMQ provides multiple paradigms of sending: synchronous, asynchronous, sequential and one-way. Both synchronous and asynchronous methods require the confirmation information return from the Broker, but one-way method does not require it. +## 3 Consumer +The Consumer is responsible for consuming messages, typically the background system is responsible for asynchronous consumption. The consumer pulls messages from brokers and feeds them into application. From the perspective of user, two types of consumers are provided: pull consumer and push consumer. +## 4 Topic +The Topic refers to a collection of one kind of message. Each topic contains several messages and one message can only belong to one topic. The topic is the basic unit of RocketMQ for message subscription. +## 5 Broker Server +As the role of the transfer station, the Broker Server stores and forwards messages. In RocketMQ, the broker server is responsible for receiving messages sent from producers, storing them and preparing to handle pull requests. It also stores the related message meta data, including consumer groups, consuming progress, topics, queues info and so on. +## 6 Name Server +The Name Server serves as the provider of routing service. The producer or the consumer can find the list of broker IP addresses for each topic through name server. Multiple name servers can be deployed in one cluster, but they are independent of each other and do not exchange information. +## 7 Pull Consumer +A type of Consumer, the application pulls messages from brokers by actively invoking the consumer pull message method, and the application has the advantages of controlling the timing and frequency of pulling messages. Once the batch of messages is pulled, user application will initiate consuming process. +## 8 Push Consumer +A type of Consumer. Under this high real-time performance mode, it will push the message to the consumer actively when the Broker receives the data. +## 9 Producer Group +A collection of the same type of Producer, which sends the same type of messages with consistent logic. If a transaction message is sent and the original producer crashes after sending, the broker server will contact other producers in the same producer group to commit or rollback the transactional message. +## 10 Consumer Group +A collection of the same type of Consumer, which sends the same type of messages with consistent logic. The consumer group makes load-balance and fault-tolerance super easy in terms of message consuming. +Warning: consumer instances of one consumer group must have exactly the same topic subscription(s). + +RocketMQ supports two types of consumption mode:Clustering and Broadcasting. +## 11 Consumption Mode - Clustering +Under the Clustering mode, all the messages from one topic will be delivered to all the consumers instances averagely as much as possible. That is, one message can be consumed by only one consumer instance. +## 12 Consumption Mode - Broadcasting +Under the Broadcasting mode, each consumer instance of the same consumer group receives every message published to the corresponding topic. +## 13 Normal Ordered Message +Under the Normal Ordered Message mode, the messages received by consumers from the same ConsumeQueue are sequential, but the messages received from the different message queues may be non-sequential. +## 14 Strictly Ordered Message +Under the Strictly Ordered Message mode, all messages received by the consumers from the same topic are sequential as the order they are stored. +## 15 Message +The physical carrier of information transmitted by a messaging system, the smallest unit of production and consumption data, each message must belong to one topic. +Each Message in RocketMQ has a unique message id and can carry a key used to store business-related value. The system has the function to query messages by its id or key. +## 16 Tag +Flags set for messages to distinguish different types of messages under the same topic, functioning as a "sub-topic". Messages from the same business unit can set different tags under the same topic in terms of different business purposes. The tag can effectively maintain the clarity and consistency of the code and optimize the query system provided by RocketMQ. The consumer can realize different "sub-topic" by using tag in order to achieve better expansibility. diff --git a/docs/en/Configuration_Client.md b/docs/en/Configuration_Client.md new file mode 100644 index 00000000000..4b6d2fe39a5 --- /dev/null +++ b/docs/en/Configuration_Client.md @@ -0,0 +1,119 @@ +## Client Configuration + + Relative to RocketMQ's Broker cluster, producers and consumers are client. In this section, it mainly describes the common behavior configuration of producers and consumers. +​ +### 1 Client Addressing mode + +```RocketMQ``` can let client find the ```Name Server```, and then find the ```Broker```by the ```Name Server```. Followings show a variety of configurations, and priority level from highly to lower, the highly priority configurations can override the lower priority configurations. + +- Specified ```Name Server``` address in the code, and multiple ```Name Server``` addresses are separated by semicolons + +```java +producer.setNamesrvAddr("192.168.0.1:9876;192.168.0.2:9876"); + +consumer.setNamesrvAddr("192.168.0.1:9876;192.168.0.2:9876"); +``` +- Specified ```Name Server``` address in the Java setup parameters + +```text +-Drocketmq.namesrv.addr=192.168.0.1:9876;192.168.0.2:9876 +``` +- Specified ```Name Server``` address in the envionment variables + +```text +export NAMESRV_ADDR=192.168.0.1:9876;192.168.0.2:9876 +``` +- HTTP static server addressing(default) + +After client started, it will access a http static server address, as: , this URL return the following contents: + +```text +192.168.0.1:9876;192.168.0.2:9876 +``` +By default, the client accesses the HTTP server every 2 minutes, and update the local Name Server address.The URL is hardcoded in the code, you can change the target server by updating ```/etc/hosts``` file, such as add following configuration at the ```/etc/hosts```: +```text +10.232.22.67 jmenv.taobao.net +``` +HTTP static server addressing is recommended, because it is simple client deployment, and the Name Server cluster can be upgraded hot. + +### 2 Client Configuration + +```DefaultMQProducer```,```TransactionMQProducer```,```DefaultMQPushConsumer```,```DefaultMQPullConsumer``` all extends the ```ClientConfig``` Class, ```ClientConfig``` as the client common configuration class. Client configuration style like getXXX,setXXX, each of the parameters can config by spring and also config their in the code. Such as the ```namesrvAddr``` parameter: ```producer.setNamesrvAddr("192.168.0.1:9876")```, same with the other parameters. + +#### 2.1 Client Common Configuration + +| Pamater Name | Default Value | Description | +| ----------------------------- | ------- | ------------------------------------------------------------ | +| namesrvAddr | | Name Server address list, multiple NameServer addresses are separated by semicolons | +| clientIP | local IP | Client local ip address, some machines will fail to recognize the client IP address, which needs to be enforced in the code | +| instanceName | DEFAULT | Name of the client instance, Multiple producers and consumers created by the client actually share one internal instance (this instance contains network connection, thread resources, etc.). | +| clientCallbackExecutorThreads | 4 | Number of communication layer asynchronous callback threads | +| pollNameServerInteval | 30000 | Polling the Name Server interval in milliseconds | +| heartbeatBrokerInterval | 30000 | The heartbeat interval, in milliseconds, is sent to the Broker | +| persistConsumerOffsetInterval | 5000 | The persistent Consumer consumes the progress interval in milliseconds | + +#### 2.2 Producer Configuration + +| Pamater Name | Default Value | Description | +| -------------------------------- | ---------------- | ------------------------------------------------------------ | +| producerGroup | DEFAULT_PRODUCER | The name of the Producer group. If multiple producers belong to one application and send the same message, they should be grouped into the same group | +| createTopicKey | TBW102 | When a message is sent, topics that do not exist on the server are automatically created and a Key is specified that can be used to configure the default route to the topic where the message is sent.| +| defaultTopicQueueNums | 4 | The number of default queue when sending messages and auto created topic which not exists the server| +| sendMsgTimeout | 10000 | Timeout time of sending message in milliseconds | +| compressMsgBodyOverHowmuch | 4096 | The message Body begins to compress beyond the size(the Consumer gets the message automatically unzipped.), unit of byte| +| retryAnotherBrokerWhenNotStoreOK | FALSE | If send message and return sendResult but sendStatus!=SEND_OK, Whether to resend | +| retryTimesWhenSendFailed | 2 | If send message failed, maximum number of retries, this parameter only works for synchronous send mode| +| maxMessageSize | 4MB | Client limit message size, over it may error. Server also limit so need to work with server | +| transactionCheckListener | | The transaction message looks back to the listener, if you want send transaction message, you must setup this +| checkThreadPoolMinSize | 1 | Minimum of thread in thread pool when Broker look back Producer transaction status | +| checkThreadPoolMaxSize | 1 | Maximum of thread in thread pool when Broker look back Producer transaction status | +| checkRequestHoldMax | 2000 | Producer local buffer request queue size when Broker look back Producer transaction status | +| RPCHook | null | This parameter is passed in when the Producer is creating, including the pre-processing before the message sending and the processing after the message response. The user can do some security control or other operations in the first interface.| + +#### 2.3 PushConsumer Configuration + +| Pamater Name | Default Value | Description | +| ---------------------------- | ----------------------------- | ------------------------------------------------------------ | +| consumerGroup | DEFAULT_CONSUMER | Consumer group name. If multi Consumer belong to an application, subscribe the same message and consume logic as the same, they should be gathered together | +| messageModel | CLUSTERING | Message support two mode: cluster consumption and broadcast consumption | +| consumeFromWhere | CONSUME_FROM_LAST_OFFSET | After Consumer started, default consumption from last location, it include two situation: One is last consumption location is not expired, and consumption start at last location; The other is last location expired, start consumption at current queue's first message | +| consumeTimestamp | Half an hour ago | Only consumeFromWhere=CONSUME_FROM_TIMESTAMP, this can work | +| allocateMessageQueueStrategy | AllocateMessageQueueAveragely | Implements strategy of Rebalance algorithms | +| subscription | | subscription relation | +| messageListener | | message listener | +| offsetStore | | Consumption progress store | +| consumeThreadMin | 10 | Minimum of thread in consumption thread pool | +| consumeThreadMax | 20 | Maximum of thread in consumption thread pool | +| | | | +| consumeConcurrentlyMaxSpan | 2000 | Maximum span allowed for single queue parallel consumption | +| pullThresholdForQueue | 1000 | Pull message local queue cache maximum number of messages | +| pullInterval | 0 | Pull message interval, because long polling it is 0, but for flow control, you can set value which greater than 0 in milliseconds | +| consumeMessageBatchMaxSize | 1 | Batch consume message | +| pullBatchSize | 32 | Batch pull message | + +#### 2.4 PullConsumer Configuration + +| Pamater Name | Default Value | Description | +| -------------------------------- | ----------------------------- | ------------------------------------------------------------ | +| consumerGroup | DEFAULT_CONSUMER | Consumer group name. If multi Consumer belong to an application, subscribe the same message and consume logic as the same, they should be gathered together | +| brokerSuspendMaxTimeMillis | 20000 | Long polling, Consumer pull message request suspended for the longest time in the Broker in milliseconds | +| consumerTimeoutMillisWhenSuspend | 30000 | Long polling, Consumer pull message request suspend in the Broker over this time value, client think timeout. Unit is milliseconds | +| consumerPullTimeoutMillis | 10000 | Not long polling, timeout time of pull message in milliseconds | +| messageModel | BROADCASTING | Message support two mode: cluster consumption and broadcast consumption | +| messageQueueListener | | Listening changing of queue | +| offsetStore | | Consumption schedule store | +| registerTopics | | Collection of registered topics | +| allocateMessageQueueStrategy | AllocateMessageQueueAveragely | Implements strategy about Rebalance algorithm | + +#### 2.5 Message Data Structure + +| Field Name | Default Value | Description | +| -------------- | ------ | ------------------------------------------------------------ | +| Topic | null | Required, the name of the topic to which the message belongs | +| Body | null | Required, message body | +| Tags | null | Optional, message tag, convenient for server filtering. Currently only one tag per message is supported | +| Keys | null | Optional, represent this message's business keys, server create hash indexes based keys. After setting, you can find message by ```Topics```,```Keys``` in Console system. Because of hash indexes, please make key as unique as possible, such as order number, goods Id and so on.| +| Flag | 0 | Optional, it is entirely up to the application, and RocketMQ does not intervene | +| DelayTimeLevel | 0 | Optional, message delay level, 0 represent no delay, greater tan 0 can consume | +| WaitStoreMsgOK | TRUE | Optional, indicates whether the message is not answered until the server is down. | + diff --git a/docs/en/Configuration_System.md b/docs/en/Configuration_System.md new file mode 100644 index 00000000000..cdf9a980184 --- /dev/null +++ b/docs/en/Configuration_System.md @@ -0,0 +1,67 @@ +# The system configuration + +This section focuses on the configuration of the system (JVM/OS) + +## **1 JVM Options** ## + +The latest released version of JDK 1.8 is recommended. Set the same Xms and Xmx value to prevent the JVM from resizing the heap for better performance. A simple JVM configuration is as follows: + + -server -Xms8g -Xmx8g -Xmn4g + +If you don’t care about the boot time of RocketMQ broker, pre-touch the Java heap to make sure that every page will be allocated during JVM initialization is a better choice. Those who don’t care about the boot time can enable it: + + -XX:+AlwaysPreTouch + +Disable biased locking maybe reduce JVM pauses: + + -XX:-UseBiasedLocking + +As for garbage collection, G1 collector with JDK 1.8 is recommended: + + -XX:+UseG1GC -XX:G1HeapRegionSize=16m + -XX:G1ReservePercent=25 + -XX:InitiatingHeapOccupancyPercent=30 + +These GC options looks a little aggressive, but it’s proved to have good performance in our production environment + +Don’t set a too small value for -XX:MaxGCPauseMillis, otherwise JVM will use a small young generation to achieve this goal which will cause very frequent minor GC.So use rolling GC log file is recommended: + + -XX:+UseGCLogFileRotation + -XX:NumberOfGCLogFiles=5 + -XX:GCLogFileSize=30m + +If write GC file will increase latency of broker, consider redirect GC log file to a memory file system: + + -Xloggc:/dev/shm/mq_gc_%p.log123 + +## 2 Linux Kernel Parameters ## + +There is a os.sh script that lists a lot of kernel parameters in folder bin which can be used for production use with minor changes. Below parameters need attention, and more details please refer to documentation for /proc/sys/vm/*. + + + + +- **vm.extra_free_kbytes**, tells the VM to keep extra free memory between the threshold where background reclaim (kswapd) kicks in, and the threshold where direct reclaim (by allocating processes) kicks in. RocketMQ uses this parameter to avoid high latency in memory allocation. (It is specific to the kernel version) + + + +- **vm.min_free_kbytes**, if you set this to lower than 1024KB, your system will become subtly broken, and prone to deadlock under high loads. + + + + + +- **vm.max_map_count**, limits the maximum number of memory map areas a process may have. RocketMQ will use mmap to load CommitLog and ConsumeQueue, so set a bigger value for this parameter is recommended. + + + +- **vm.swappiness**, define how aggressive the kernel will swap memory pages. Higher values will increase agressiveness, lower values decrease the amount of swap. 10 is recommended for this value to avoid swap latency. + + + +- **File descriptor limits**, RocketMQ needs open file descriptors for files(CommitLog and ConsumeQueue) and network connections. We recommend set 655350 for file descriptors. + + + +- **Disk scheduler**, the deadline I/O scheduler is recommended for RocketMQ, which attempts to provide a guaranteed latency for requests. + diff --git a/docs/en/Deployment.md b/docs/en/Deployment.md new file mode 100644 index 00000000000..b0a27327fbe --- /dev/null +++ b/docs/en/Deployment.md @@ -0,0 +1,157 @@ +# Installation Guides + +### 1 Single Master mode + +This is the simplest but also the riskiest mode, that makes the entire service unavailable once the broker restarts or goes down. Production environments are not recommended, but can be used for local testing and development. Here are the steps to build. + +**1)Start NameServer** + +```shell +### Start Name Server first +$ nohup sh mqnamesrv & + +### Then verify that the Name Server starts successfully +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +We can see 'The Name Server boot success.. ' in namesrv.log that indicates the NameServer has been started successfully. + +**2)Start Broker** + +```shell +### Also start broker first +$ nohup sh bin/mqbroker -n localhost:9876 & + +### Then verify that the broker is started successfully, for example, the IP of broker is 192.168.1.2 and the name is broker-a +$ tail -f ~/logs/rocketmqlogs/Broker.log +The broker[broker-a,192.169.1.2:10911] boot success... +``` + +We can see 'The broker[brokerName,ip:port] boot success..' in Broker.log that indicates the broker has been started successfully. + +### 2 Multiple Master mode + +Multiple master mode means a mode with all master nodes(such as 2 or 3 master nodes) and no slave node. The advantages and disadvantages of this mode are as follows: + +- Advantages: + 1. Simple configuration. + 2. Outage or restart(for maintenance) of one master node has no impact on the application. + 3. When the disk is configured as RAID10, messages are not lost because the RAID10 disk is very reliable, even if the machine is not recoverable (In the case of asynchronous flush disk mode of the message, a small number of messages are lost; If the brush mode of a message is synchronous, no message will be lost). + 4. In this mode, the performance is the highest. +- Disadvantages: + 1. During a single machine outage, messages that are not consumed on this machine are not subscribed to until the machine recovers, and message real-time is affected. + +The starting steps for multiple master mode are as follows: + +**1)Start NameServer** + +```shell +### Start Name Server first +$ nohup sh mqnamesrv & + +### Then verify that the Name Server starts successfully +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +**2)Start the Broker cluster** + +```shell +### For example, starting the first Master on machine A, assuming that the configured NameServer IP is: 192.168.1.1. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-noslave/broker-a.properties & + +### Then starting the second Master on machine B, assuming that the configured NameServer IP is: 192.168.1.1. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-noslave/broker-b.properties & + +... +``` + +The boot command shown above is used in the case of a single NameServer. For clusters of multiple NameServer, the address list after the -n argument in the broker boot command is separated by semicolons, for example, 192.168.1.1: 9876;192.161.2: 9876. + +### 3 Multiple Master And Multiple Slave Mode-Asynchronous replication + +Each master node configures more thran one slave nodes, with multiple pairs of master-slave.HA uses asynchronous replication, with a short message delay (millisecond) between master node and slave node.The advantages and disadvantages of this mode are as follows: + +- Advantages: + 1. Even if the disk is corrupted, very few messages will be lost and the real-time performance of the message will not be affected. + 2. At the same time, when master node is down, consumers can still consume messages from slave node, and the process is transparent to the application itself and does not require human intervention. + 3. Performance is almost as high as multiple master mode. +- Disadvantages: + 1. A small number of messages will be lost when master node is down and the disk is corrupted. + +The starting steps for multiple master and multiple slave mode are as follows: + +**1)Start NameServer** + +```shell +### Start Name Server first +$ nohup sh mqnamesrv & + +### Then verify that the Name Server starts successfully +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +**2)Start the Broker cluster** + +```shell +### For example, starting the first Master on machine A, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-a.properties & + +### Then starting the second Master on machine B, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-b.properties & + +### Then starting the first Slave on machine C, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-a-s.properties & + +### Last starting the second Slave on machine D, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-async/broker-b-s.properties & +``` + +The above shows a startup command for 2M-2S-Async mode, similar to other nM-nS-Async modes. + +### 4 Multiple Master And Multiple Slave Mode-Synchronous dual write + +In this mode, multiple slave node are configured for each master node and there are multiple pairs of Master-Slave.HA uses synchronous double-write, that is, the success response will be returned to the application only when the message is successfully written into the master node and replicated to more than one slave node. + +The advantages and disadvantages of this model are as follows: + +- Advantages: + 1. Neither the data nor the service has a single point of failure. + 2. In the case of master node shutdown, the message is also undelayed. + 3. Service availability and data availability are very high; +- Disadvantages: + 1. The performance in this mode is slightly lower than in asynchronous replication mode (about 10% lower). + 2. The RT sending a single message is slightly higher, and the current version, the slave node cannot automatically switch to the master after the master node is down. + +The starting steps are as follows: + +**1)Start NameServer** + +```shell +### Start Name Server first +$ nohup sh mqnamesrv & + +### Then verify that the Name Server starts successfully +$ tail -f ~/logs/rocketmqlogs/namesrv.log +The Name Server boot success... +``` + +**2)Start the Broker cluster** + +```shell +### For example, starting the first Master on machine A, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-a.properties & + +### Then starting the second Master on machine B, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-b.properties & + +### Then starting the first Slave on machine C, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-a-s.properties & + +### Last starting the second Slave on machine D, assuming that the configured NameServer IP is: 192.168.1.1 and port is 9876. +$ nohup sh mqbroker -n 192.168.1.1:9876 -c $ROCKETMQ_HOME/conf/2m-2s-sync/broker-b-s.properties & +``` + +The above Master and Slave are paired by specifying the same config named "brokerName", the "brokerId" of the master node must be 0, and the "brokerId" of the slave node must be greater than 0. \ No newline at end of file diff --git a/docs/en/Design_Filter.md b/docs/en/Design_Filter.md new file mode 100644 index 00000000000..28eb59948ec --- /dev/null +++ b/docs/en/Design_Filter.md @@ -0,0 +1,10 @@ +# Message Filter +RocketMQ - a distributed message queue, is different with all other MQ middleware, on the way of filtering messages. It's do the filter when the messages are subscribed via consumer side.RocketMQ do it lies in the separate storage mechanism that Producer side writing messages and Consomer subscribe messages, Consumer side will get an index from a logical message queue ConsumeQueue when subscribing, then read message entity from CommitLog using the index. So in the end, it is still impossible to get around its storage structure.The storage structure of ConsumeQueue is as follows, and there is a 8-byte Message Tag hashcode, The message filter based on Tag value is just used this Message Tag hash-code. + +![](images/rocketmq_design_7.png) + +The RocketMQ has two mainly filter types: + +* Tag filtering: Consumer can specify not only the message topic but also the message tag values, when subscribing. Multiple tag values need to be separated by '||'. When consumer subscribing a message, it builds the subscription request into a `SubscriptionData` object and sends a pull message request to the Broker side. Before the Broker side reads data from the RocketMQ file storage layer - Store, it will construct a `MessageFilter` using the `SubscriptionData` object and then pass it to the Store. Store get a record from `ConsumeQueue`, and it will filter the message by the saved tag hashcode, it is unable to filter the messages exactly in the server side because of only the hashcode will be used when filtering, Therefore, after the Consumer pulls the message, it also needs to compare the original tag string of the message. If the original tag string is not same with the expected, the message will be ignored. + +* SQL92 filtering: This filter behavior is almost same with the above `Tag filtering` method. The only difference is on the way how Store works. The rocketmq-filter module is responsible for the construction and execution of the real SQL expression. Executing an SQL expression every time a filter is executed affects efficiency, so RocketMQ uses BloomFilter to avoid doing it every time. The expression context of SQL92 is a property of the message. diff --git a/docs/en/Design_LoadBlancing.md b/docs/en/Design_LoadBlancing.md new file mode 100644 index 00000000000..b93b6ea86f6 --- /dev/null +++ b/docs/en/Design_LoadBlancing.md @@ -0,0 +1,42 @@ +## 4 Load Balancing +Load balancing in RocketMQ is accomplished on Client side. Specifically, it can be divided into load balancing at Producer side when sending messages and load balancing at Constumer side when subscribing messages. + +### 4.1 Producer Load Balancing +When the Producer sends a message, it will first find the specified TopicPublishInfo according to Topic. After getting the routing information of TopicPublishInfo, the RocketMQ client will select a queue (MessageQueue) from the messageQueue List in TopicPublishInfo to send the message by default.Specific fault-tolerant strategies are defined in the MQFaultStrategy class. +Here is a sendLatencyFaultEnable switch variable, which, if turned on, filters out the Broker agent of not available on the basis of randomly gradually increasing modular arithmetic selection. The so-called "latencyFault Tolerance" refers to a certain period of time to avoid previous failures. For example, if the latency of the last request exceeds 550 Lms, it will evade 3000 Lms; if it exceeds 1000L, it will evade 60000 L; if it is closed, it will choose a queue (MessageQueue) to send messages by randomly gradually increasing modular arithmetic, and the latencyFault Tolerance mechanism is the key to achieve high availability of message sending. + +### 4.2 Consumer Load Balancing +In RocketMQ, the two consumption modes (Push/Pull) on the Consumer side are both based on the pull mode to get the message, while in the Push mode it is only a kind of encapsulation of the pull mode, which is essentially implemented as the message pulling thread after pulling a batch of messages from the server. After submitting to the message consuming thread pool, it continues to try again to pull the message to the server. If the message is not pulled, the pull is delayed and continues. In both pull mode based consumption patterns (Push/Pull), the Consumer needs to know which message queue - queue from the Broker side to get the message. Therefore, it is necessary to do load balancing on the Consumer side, that is, which Consumer consumption is allocated to the same ConsumerGroup by more than one MessageQueue on the Broker side. + + 1, Heartbeat Packet Sending on Consumer side +After Consumer is started, it continuously sends heartbeat packets to all Broker instances in the RocketMQ cluster via timing task (which contains the message consumption group name, subscription relationship collection,Message communication mode and the value of the client id,etc). After receiving the heartbeat message from Consumer, Broker side maintains it in Consumer Manager's local caching variable—consumerTable, At the same time, the encapsulated client network channel information is stored in the local caching variable—channelInfoTable, which can provide metadata information for the later load balancing of Consumer. +2,Core Class for Load Balancing on Consumer side—RebalanceImpl +Starting the MQClientInstance instance in the startup process of the Consumer instance will complete the start of the load balancing service thread-RebalanceService (executed every 20 s). By looking at the source code, we can find that the run () method of the RebalanceService thread calls the rebalanceByTopic () method of the RebalanceImpl class, which is the core of the Consumer end load balancing. Here, rebalanceByTopic () method will do different logical processing depending on whether the consumer communication type is "broadcast mode" or "cluster mode". Here we mainly look at the main processing flow in cluster mode: +(1) Get the message consumption queue set (mqSet) under the Topic from the local cache variable—topicSubscribeInfoTable of the rebalanceImpl instance. +(2) Call mQClientFactory. findConsumerIdList () method to send a RPC communication request to Broker side to obtain the consumer Id list under the consumer group based on the parameters of topic and consumer group (consumer table constructed by Broker side based on the heartbeat data reported by the front consumer side responds and returns, business request code: GET_CONSUMER_LIST_BY_GROUP); +(3) First, the message consumption queue and the consumer Id under Topic are sorted, then the message queue to be pulled is calculated by using the message queue allocation strategy algorithm (default: the average allocation algorithm of the message queue). The average allocation algorithm here is similar to the paging algorithm. It ranks all MessageQueues like records. It ranks all consumers like pages. It calculates the average size of each page and the range of each page record. Finally, it traverses the whole range and calculates the records that the current consumer should allocate to (MessageQueue here). +![Image text](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_8.png) +(4) Then, the updateProcessQueueTableInRebalance () method is invoked, which first compares the allocated message queue set (mqSet) with processQueueTable for filtering. +![Image text](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_9.png) + + - The red part of the processQueueTable annotation in the figure above + indicates that it is not included with the assigned message queue set + mqSet. Set the Dropped attribute to true for these queues, and then + check whether these queues can remove the processQueueTable cache + variable or not. The removeUnnecessaryMessageQueue () method is + executed here, that is, check every 1s to see if the locks of the + current consumption processing queue can be retrieved and return true + if they are retrieved. If the lock of the current consumer processing + queue is still not available after waiting for 1s, it returns false. + If true is returned, the corresponding Entry is removed from the + processQueueTable cache variable. + - The green section in processQueueTable above represents the + intersection with the assigned message queue set mqSet. Determine + whether the ProcessQueue has expired, regardless of Pull mode, if it + is Push mode, set the Dropped attribute to true, and call the + removeUnnecessaryMessageQueue () method to try to remove Entry as + above; + +Finally, a ProcessQueue object is created for each MessageQueue in the filtered message queue set (mqSet) and stored in the processQueueTable queue of RebalanceImpl (where the computePullFromWhere (MessageQueue mq) method of the RebalanceImpl instance is invoked to obtain the next progress consumption value offset of the MessageQueue object, which is then populated into the attribute of pullRequest object to be created next time.), and create pull request object—pullRequest to add to pull list—pullRequestList, and finally execute dispatchPullRequest () method. PullRequest object of Pull message is put into the blocking queue pullRequestQueue of PullMessageService service thread in turn, and the request of Pull message is sent to Broker end after the service thread takes out. Among them, we can focus on the contrast, RebalancePushImpl and RebalancePullImpl two implementation classes dispatchPullRequest () method is different, the method in RebalancePullImpl class is empty, thus answering the last question in the previous article. + +The core design idea of message consumption queue is that a message consumption queue can only be consumed by one consumer in the same consumer group at the same time, and a message consumer can consume multiple message queues at the same time. diff --git a/docs/en/Design_Query.md b/docs/en/Design_Query.md new file mode 100644 index 00000000000..6988948d844 --- /dev/null +++ b/docs/en/Design_Query.md @@ -0,0 +1,17 @@ +# Message Queries + +RocketMQ supports message queries by two dimensions, which are "Query Message by Message Id" and "Query Message by Message Key". + +## 1. Query Message by Message Id +The MessageId in RocketMQ has a total length of 16 bytes, including the broker address (IP address and port) and CommitLog offset. In RocketMQ, the specific approach is that the Client resolves the Broker's address (IP address and port) and the CommitLog's offset address from the MessageId. Then both of them are encapsulated into an RPC request, and finally it will be sent through the communication layer (business request code: VIEW_MESSAGE_BY_ID). The Broker reads a message by using the CommitLog offset and size to find the real message in the CommitLog and then return, which is how QueryMessageProcessor works. + +## 2. Query Message by Message Id +"Query Messages by Message Key" is mainly based on RocketMQ's IndexFile. The logical structure of the IndexFile is similar to the implementation of HashMap in JDK. The specific structure of the IndexFile is as follows: + +![](images/rocketmq_design_message_query.png) + +The IndexFile provides the user with the querying service by “Querying Messages by Message Key”. The IndexFile is stored in $HOME\store\index${fileName}, and the file name is named after the timestamp at the time of creation. The file size is fixed, which is 420,000,040 bytes (40+5million\*4+20million\*20). If the UNIQ_KEY is set in the properties of the message, then the "topic + ‘#’ + UNIQ_KEY" will be used as the index. Likewise, if the KEYS is set in the properties of the message (multiple KEYs should be separated by spaces), then the "topic + ‘#’ + KEY" will be used as the index. + +The index data contains four fields, Key Hash, CommitLog offset, Timestamp and NextIndex offset, for a total of 20 Bytes. The NextIndex offset of the index data will point to the previous index data if the Key Hash of the index data is the same as that of the previous index data. If a hash conflict occurs, then the NextIndex offset can be used as the field to string all conflicting indexes in a linked list. What the Timestamp records is the time difference between two storeTimestamps, instead of a specific time. The structure of the entire IndexFile is shown in the graph. The Header is used to store some general statistics, which needs 40 bytes. The Slot Table of 4\*5million bytes does not save the real index data, but saves the header of the singly linked list corresponding to each slot. The Index Linked List of 20\*20million is the real index data, that is, an Index File can hold 20million indexes. + +The specific method of "Query Message by Message Key" is that the topic and message key are used to find the record in the IndexFile, and then read the message from the file of CommitLog according to the CommitLog offset in this record. \ No newline at end of file diff --git a/docs/en/Design_Remoting.md b/docs/en/Design_Remoting.md new file mode 100644 index 00000000000..b90413ec9a9 --- /dev/null +++ b/docs/en/Design_Remoting.md @@ -0,0 +1,50 @@ +## 2 Communication Mechanism +RocketMQ message queue cluster mainly includes four roles: NameServer, Broker (Master/Slave), Producer and Consumer. The basic communication process is as follows: +(1) After Broker start-up, it needs to complete one operation: register itself to NameServer, and then report Topic routing information to NameServer at regular intervals of 30 seconds. +(2) When message Producer sends a message as a client, it needs to obtain routing information from the local cache TopicPublishInfoTable according to the Topic of the message. If not, it will be retrieved from NameServer and update to local cache, at the same time, Producer will retrieve routing information from NameServer every 30 seconds by default. +(3) Message producer Producer chooses a queue to send the message according to the routing information obtained in 2); Broker receives the message and records it in disk as the receiver of the message. +(4) After message consumer Consumer get the routing information according to 2) and complete the load balancing of the client, then select one or several message queues to pull messages and consume them. + +From 1) ~ 3 above, we can see that both Producer, Broker and NameServer communicate with each other(only part of MQ communication is mentioned here), so how to design a good network communication module is very important in MQ. It will determine the overall messaging capability and final performance of the RocketMQ cluster. + +rocketmq-remoting module is the module responsible for network communication in RocketMQ message queue. It is relied on and referenced by almost all other modules (such as rocketmq-client,rocketmq-broker,rocketmq-namesrv) that need network communication. In order to realize the efficient data request and reception between the client and the server, the RocketMQ message queue defines the communication protocol and extends the communication module on the basis of Netty. + +### 2.1 Remoting Communication Class Structure +![](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_3.png) +### 2.2 Protocol Design and Codec +When a message is sent between Client and Server, a protocol convention is needed for the message sent, so it is necessary to customize the message protocol of RocketMQ. At the same time, in order to efficiently transmit messages and read the received messages, it is necessary to encode and decode the messages. In RocketMQ, the RemotingCommand class encapsulates all data content in the process of message transmission, which includes not only all data structures, but also encoding and decoding operations. + +Header field | Type | Request desc | Response desc +--- | --- | --- | --- | +code |int | Request code. answering business processing is different according to different requests code | Response code. 0 means success, and non-zero means errors. +language | LanguageCode | Language implemented by the requester | Language implemented by the responder +version | int | Version of Request Equation | Version of Response Equation +opaque | int |Equivalent to reqeustId, the different request identification codes on the same connection correspond to those in the response message| The response returns directly without modification +flag | int | Sign, used to distinguish between ordinary RPC or oneway RPC | Sign, used to distinguish between ordinary RPC or oneway RPC +remark | String | Transfer custom text information | Transfer custom text information +extFields | HashMap | Request custom extension information| Response custom extension information +![](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_4.png) +From the above figure, the transport content can be divided into four parts: + + (1) Message length: total length, four bytes of storage, occupying an int type; + +(2) Serialization type header length: occupying an int type. The first byte represents the serialization type, and the last three bytes represent the header length; + +(3) Header data: serialized header data; + +(4) Message body data: binary byte data content of message body; +#### 2.3 Message Communication Mode and Procedure +There are three main ways to support communication in RocketMQ message queue: synchronous (sync), asynchronous (async), one-way (oneway). The "one-way" communication mode is relatively simple and is generally used in sending heartbeat packets without paying attention to its Response. Here, mainly introduce the asynchronous communication flow of RocketMQ. +![](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_5.png) +#### 2.4 Reactor Multithread Design +The RPC communication of RocketMQ uses Netty component as the underlying communication library, and also follows the Reactor multithread model. At the same time, some extensions and optimizations are made on it. +![](https://github.com/apache/rocketmq/raw/develop/docs/cn/image/rocketmq_design_6.png) +Above block diagram can roughly understand the Reactor multi-thread model of NettyRemotingServer in RocketMQ. A Reactor main thread (eventLoopGroupBoss, is 1 above) is responsible for listening to TCP network connection requests, establishing connections, creating SocketChannel, and registering on selector. The source code of RocketMQ automatically selects NIO and Epoll according to the type of OS. Then listen to real network data. After you get the network data, you throw it to the Worker thread pool (eventLoopGroupSelector, is the "N" above, the default is 3 in the source code). You need to do SSL verification, codec, idle check, network connection management before you really execute the business logic. These tasks to defaultEventExecutorGroup (that is, "M1" above, the default set to 8 in the source code) to do. The processing business operations are executed in the business thread pool. According to the RomotingCommand business request code, the corresponding processor is found in the processorTable local cache variable and encapsulated into the task, and then submitted to the corresponding business processor processing thread pool for execution (sendMessageExecutor,). Take sending a message, for example, the "M2" above. The thread pool continues to increase in several steps from entry to business logic, which is related to the complexity of each step. The more complex the thread pool is, the wider the concurrent channel is required. +Number of thread | Name of thread | Desc of thread + --- | --- | --- +1 | NettyBoss_%d | Reactor Main thread +N | NettyServerEPOLLSelector_%d_%d | Reactor thread pool +M1 | NettyServerCodecThread_%d | Worker thread pool +M2 | RemotingExecutorThread_%d | bussiness processor thread pool + + diff --git a/docs/en/Design_Store.md b/docs/en/Design_Store.md new file mode 100644 index 00000000000..d0d1c90aaa4 --- /dev/null +++ b/docs/en/Design_Store.md @@ -0,0 +1,39 @@ +# Message Storage + + +![](images/rocketmq_storage_arch.png) + +Message storage is the most complicated and important part of RocketMQ. This section will describe the three aspects of RocketMQ: +* Message storage architecture +* PageCache and memory mapping +* RocketMQ's two different disk flushing methods. + +## 1 Message Storage Architecture + + +The message storage architecture diagram consists of 3 files related to message storage: `CommitLog` file, `ConsumeQueue` file, and `IndexFile`. + + +* `CommitLog`:The `CommitLog` file stores message body and metadata sent by producer, and the message content is not fixed length. The default size of one `CommitLog` file is 1G, the length of the file name is 20 digits, the left side is zero padded, and the remaining is the starting offset. For example, `00000000000000000000` represents the first file, the starting offset is 0, and the file size is 1G=1073741824, when the first `CommitLog` file is full, the second `CommitLog` file is `00000000001073741824`, the starting offset is 1073741824, and so on. The message is mainly appended to the log file sequentially. When one `CommitLog` file is full, the next will be written. +* `ConsumeQueue`: The `ConsumeQueue` is used to improve the performance of message consumption. Since RocketMQ uses topic-based subscription mode, message consumption is specific to the topic. Traversing the commitlog file to retrieve messages of one topic is very inefficient. The consumer can find the messages to be consumed according to the `ConsumeQueue`. The `ConsumeQueue`(logic consume queue) as an index of the consuming message stores the starting physical offset `offset` in `CommitLog` of the specified topic, the message size `size` and the hash code of the message tag. The `ConsumeQueue` file can be regarded as a topic-based `CommitLog` index file, so the consumequeue folder is organized as follows: `topic/queue/file` three-layer organization structure, the specific storage path is `$HOME/store/consumequeue/{topic}/{queueId }/{fileName}`. The consumequeue file uses a fixed-length design, each entry occupies 20 bytes, which is an 8-byte commitlog physical offset, a 4-byte message length, and an 8-byte tag hashcode. One consumequeue file consists of 30W entries, each entry can be randomly accessed like an array, each `ConsumeQueue` file's size is about 5.72M. +* `IndexFile`: The `IndexFile` provides a way to query messages by key or time interval. The path of the `IndexFile` is `$HOME/store/index/${fileName}`, the file name `fileName` is named after the timestamp when it was created. One IndexFile's size is about 400M, and it can store 2000W indexes. The underlying storage of `IndexFile` is designed to implement the `HashMap` structure in the file system, so RocketMQ's index file is a hash index. + + +From the above architecture of the RocketMQ message storage, we can see RocketMQ uses a hybrid storage structure, that is, all the queues in an instance of the broker share a single log file `CommitLog` to store messages. RocketMQ's hybrid storage structure(messages of multiple topics are stored in one CommitLog) uses a separate storage structure for the data and index parts for Producer and Consumer respectively. The Producer sends the message to the Broker, then the Broker persists the message to the CommitLog file synchronously or asynchronously. As long as the message is persisted to the CommitLog on the disk, the message sent by the Producer will not be lost. Because of this, Consumer will definitely have the opportunity to consume this message. When no message can be pulled, the consumer can wait for the next pull. And the server also supports the long polling mode: if a pull request pulls no messages, the Broker can wait for 30 seconds, as long as new message arrives in this interval, it will be returned directly to the consumer. Here, RocketMQ's specific approach is using Broker's background service thread `ReputMessageService` to continuously dispatch requests and asynchronously build ConsumeQueue (Logical Queue) and IndexFile data. + +## 2 PageCache and Memory Map + +PageCache is a cache of files by the operating system to speed up the reading and writing of files. In general, the speed of sequential read and write files is almost the same as the speed of read and write memory. The main reason is that the OS uses a portion of the memory as PageCache to optimize the performance of the read and write operations. For data writing, the OS will first write to the Cache, and then the `pdflush` kernel thread asynchronously flush the data in the Cache to the physical disk. For data reading, if it can not hit the page cache when reading a file at a time, the OS will read the file from the physical disk and prefetch the data files of other neighboring blocks sequentially. + +In RocketMQ, the logic consumption queue `ConsumeQueue` stores less data and is read sequentially. With the help of prefetch of the page cache mechanism, the read performance of the `ConsumeQueue` file is almost close to the memory read, even in the case of message accumulation, it does not affect performance. But for the log data file `CommitLog`, it will generate many random access reads when reading the message content, which seriously affects the performance. If you choose the appropriate IO scheduling algorithm, such as setting the IO scheduling algorithm to "Deadline" (when the block storage uses SSD), the performance of random reads will also be improved. + + +In addition, RocketMQ mainly reads and writes files through `MappedByteBuffer`. `MappedByteBuffer` uses the `FileChannel` model in NIO to directly map the physical files on the disk to the memory address in user space (`Mmap` method reduces the performance overhead of traditional IO copying disk file data back and forth between the buffer in kernel space and the buffer in user space), it converts the file operation into direct memory address manipulation, which greatly improves the efficiency of reading and writing files (Because of the need to use the memory mapping mechanism, RocketMQ's file storage is fixed-length, making it easy to map the entire file to memory at a time). + +## 3 Message Disk Flush + +![](images/rocketmq_storage_flush.png) + + +* synchronous flush: As shown above, the RocketMQ's Broker will return a successful `ACK` response to the Producer after the message is truly persisted to disk. Synchronous flushing is a good guarantee for the reliability of MQ messages, but it will have a big impact on performance. Generally, it is suitable for financial business applications. +* asynchronous flush: Asynchronous flushing can take full advantage of the PageCache of the OS, as long as the message is written to the PageCache, the successful `ACK` can be returned to the Producer. The message flushing is performed by the background asynchronous thread, which reduces the read and write delay and improves the performance and throughput of the MQ. diff --git a/docs/en/Design_Trancation.md b/docs/en/Design_Trancation.md new file mode 100644 index 00000000000..e5f10ed75de --- /dev/null +++ b/docs/en/Design_Trancation.md @@ -0,0 +1,51 @@ +# Transaction Message +## 1 Transaction Message +Apache RocketMQ supports distributed transaction message from version 4.3.0. RocketMQ implements transaction message by using the protocol of 2PC(two-phase commit), in addition adding a compensation logic to handle timeout-case or failure-case of commit-phase, as shown below. + +![](../cn/image/rocketmq_design_10.png) + +### 1.1 The Process of RocketMQ Transaction Message +The picture above shows the overall architecture of transaction message, including the sending of message(commit-request phase), the sending of commit/rollback(commit phase) and the compensation process. + +1. The sending of message and Commit/Rollback. + (1) Sending the message(named Half message in RocketMQ) + (2) The server responds the writing result(success or failure) of Half message. + (3) Handle local transaction according to the result(local transaction won't be executed when the result is failure). + (4) Sending Commit/Rollback to broker according to the result of local transaction(Commit will generate message index and make the message visible to consumers). + +2. Compensation process + (1) For a transaction message without a Commit/Rollback (means the message in the pending status), a "back-check" request is initiated from the broker. + (2) The Producer receives the "back-check" request and checks the status of the local transaction corresponding to the "back-check" message. + (3) Redo Commit or Rollback based on local transaction status. +The compensation phase is used to resolve the timeout or failure case of the message Commit or Rollback. + +### 1.2 The design of RocketMQ Transaction Message +1. Transaction message is invisible to users in first phase(commit-request phase) + + Upon on the main process of transaction message, the message of first phase is invisible to the user. This is also the biggest difference from normal message. So how do we write the message while making it invisible to the user? And below is the solution of RocketMQ: if the message is a Half message, the topic and queueId of the original message will be backed up, and then changes the topic to RMQ_SYS_TRANS_HALF_TOPIC. Since the consumer group does not subscribe to the topic, the consumer cannot consume the Half message. Then RocketMQ starts a timing task, pulls the message for RMQ_SYS_TRANS_HALF_TOPIC, obtains a channel according to producer group and sends a back-check to query local transaction status, and decide whether to submit or roll back the message according to the status. + + In RocketMQ, the storage structure of the message in the broker is as follows. Each message has corresponding index information. The Consumer reads the content of the message through the secondary index of the ConsumeQueue. The flow is as follows: + +![](../cn/image/rocketmq_design_11.png) + + The specific implementation strategy of RocketMQ is: if the transaction message is written, topic and queueId of the message are replaced, and the original topic and queueId are stored in the properties of the message. Because the replace of the topic, the message will not be forwarded to the Consumer Queue of the original topic, and the consumer cannot perceive the existence of the message and will not consume it. In fact, changing the topic is the conventional method of RocketMQ(just recall the implementation mechanism of the delay message). + +2. Commit/Rollback operation and introduction of Op message + + After finishing writing a message that is invisible to the user in the first phase, here comes two cases in the second phase. One is Commit operation, after which the message needs to be visible to the user; the other one is Rollback operation, after which the first phase message(Half message) needs to be revoked. For the case of Rollback, since first-phase message itself is invisible to the user, there is no need to actually revoke the message (in fact, RocketMQ can't actually delete a message because it is a sequential-write file). But still some operation needs to be done to identity the final status of the message, to differ it from pending status message. To do this, the concept of "Op message" is introduced, which means the message has a certain status(Commit or Rollback). If a transaction message does not have a corresponding Op message, the status of the transaction is still undetermined (probably the second-phase failed). By introducing the Op message, the RocketMQ records an Op message for every Half message regardless it is Commit or Rollback. The only difference between Commit and Rollback is that when it comes to Commit, the index of the Half message is created before the Op message is written. + +3. How Op message stored and the correspondence between Op message and Half message + + RocketMQ writes the Op message to a specific system topic(RMQ_SYS_TRANS_OP_HALF_TOPIC) which will be created via the method - TransactionMessageUtil.buildOpTopic(); this topic is an internal Topic (like the topic of RMQ_SYS_TRANS_HALF_TOPIC) and will not be consumed by the user. The content of the Op message is the physical offset of the corresponding Half message. Through the Op message we can index to the Half message for subsequent check-back operation. + +![](../cn/image/rocketmq_design_12.png) + +4. Index construction of Half messages + + When performing Commit operation of the second phase, the index of the Half message needs to be built. Since the Half message is written to a special topic(RMQ_SYS_TRANS_HALF_TOPIC) in the first phase of 2PC, so it needs to be read out from the special topic when building index, and replace the topic and queueId with the real target topic and queueId, and then write through a normal message that is visible to the user. Therefore, in conclusion, the second phase recovers a complete normal message using the content of the Half message stored in the first phase, and then goes through the message-writing process. + +5. How to handle the message failed in the second phase? + + If commit/rollback phase fails, for example, a network problem causes the Commit to fail when you do Commit. Then certain strategy is required to make sure the message finally commit. RocketMQ uses a compensation mechanism called "back-check". The broker initiates a back-check request for the message in pending status, and sends the request to the corresponding producer side (the same producer group as the producer group who sent the Half message). The producer checks the status of local transaction and redo Commit or Rollback. The broker performs the back-check by comparing the RMQ_SYS_TRANS_HALF_TOPIC messages and the RMQ_SYS_TRANS_OP_HALF_TOPIC messages and advances the checkpoint(recording those transaction messages that the status are certain). + + RocketMQ does not back-check the status of transaction messages endlessly. The default time is 15. If the transaction status is still unknown after 15 times, RocketMQ will roll back the message by default. diff --git a/docs/en/Example_Batch.md b/docs/en/Example_Batch.md new file mode 100644 index 00000000000..d7199ca7c78 --- /dev/null +++ b/docs/en/Example_Batch.md @@ -0,0 +1,76 @@ +# Batch Message Sample +------ +Sending messages in batch improves performance of delivering small messages. Messages of the same batch should have: same topic, same waitStoreMsgOK and no schedule support. You can send messages up to 4MiB at a time, but if you need to send a larger message, it is recommended to divide the larger messages into multiple small messages of no more than 1MiB. +### 1 Send Batch Messages +If you just send messages of no more than 4MiB at a time, it is easy to use batch: +```java +String topic = "BatchTest"; +List messages = new ArrayList<>(); +messages.add(new Message(topic, "TagA", "OrderID001", "Hello world 0".getBytes())); +messages.add(new Message(topic, "TagA", "OrderID002", "Hello world 1".getBytes())); +messages.add(new Message(topic, "TagA", "OrderID003", "Hello world 2".getBytes())); +try { + producer.send(messages); +} catch (Exception e) { + e.printStackTrace(); + //handle the error +} +``` +### 2 Split into Lists +The complexity only grow when you send large batch and you may not sure if it exceeds the size limit (4MiB). At this time, you’d better split the lists: +```java +public class ListSplitter implements Iterator> { + private final int SIZE_LIMIT = 1000 * 1000; + private final List messages; + private int currIndex; + public ListSplitter(List messages) { + this.messages = messages; + } + @Override public boolean hasNext() { + return currIndex < messages.size(); + } + @Override public List next() { + int nextIndex = currIndex; + int totalSize = 0; + for (; nextIndex < messages.size(); nextIndex++) { + Message message = messages.get(nextIndex); + int tmpSize = message.getTopic().length() + message.getBody().length; + Map properties = message.getProperties(); + for (Map.Entry entry : properties.entrySet()) { + tmpSize += entry.getKey().length() + entry.getValue().length(); + } + tmpSize = tmpSize + 20; //for log overhead + if (tmpSize > SIZE_LIMIT) { + //it is unexpected that single message exceeds the SIZE_LIMIT + //here just let it go, otherwise it will block the splitting process + if (nextIndex - currIndex == 0) { + //if the next sublist has no element, add this one and then break, otherwise just break + nextIndex++; + } + break; + } + if (tmpSize + totalSize > SIZE_LIMIT) { + break; + } else { + totalSize += tmpSize; + } + + } + List subList = messages.subList(currIndex, nextIndex); + currIndex = nextIndex; + return subList; + } +} + +// then you could split the large list into small ones: +ListSplitter splitter = new ListSplitter(messages); +while (splitter.hasNext()) { + try { + List listItem = splitter.next(); + producer.send(listItem); + } catch (Exception e) { + e.printStackTrace(); + // handle the error + } +} +``` \ No newline at end of file diff --git a/docs/en/Example_Delay.md b/docs/en/Example_Delay.md new file mode 100644 index 00000000000..fe41aed92ba --- /dev/null +++ b/docs/en/Example_Delay.md @@ -0,0 +1,85 @@ +# Schedule example + +### 1 Start consumer to wait for incoming subscribed messages + +```java +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; +import java.util.List; + +public class ScheduledMessageConsumer { + + public static void main(String[] args) throws Exception { + // Instantiate message consumer + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("ExampleConsumer"); + // Subscribe topics + consumer.subscribe("TestTopic", "*"); + // Register message listener + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List messages, ConsumeConcurrentlyContext context) { + for (MessageExt message : messages) { + // Print approximate delay time period + System.out.println("Receive message[msgId=" + message.getMsgId() + "] " + + (System.currentTimeMillis() - message.getStoreTimestamp()) + "ms later"); + } + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + // Launch consumer + consumer.start(); + } +} +``` + +### 2 Send scheduled messages + +```java +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.common.message.Message; + +public class ScheduledMessageProducer { + + public static void main(String[] args) throws Exception { + // Instantiate a producer to send scheduled messages + DefaultMQProducer producer = new DefaultMQProducer("ExampleProducerGroup"); + // Launch producer + producer.start(); + int totalMessagesToSend = 100; + for (int i = 0; i < totalMessagesToSend; i++) { + Message message = new Message("TestTopic", ("Hello scheduled message " + i).getBytes()); + // This message will be delivered to consumer 10 seconds later. + message.setDelayTimeLevel(3); + // Send the message + producer.send(message); + } + + // Shutdown producer after use. + producer.shutdown(); + } + +} +``` + +### 3 Verification + +You should see messages are consumed about 10 seconds later than their storing time. + +### 4 Use scenarios for scheduled messages + +For example, in e-commerce, if an order is submitted, a delay message can be sent, and the status of the order can be checked after 1 hour. If the order is still unpaid, the order can be cancelled and the inventory released. + +### 5 Restrictions on the use of scheduled messages + +```java +// org/apache/rocketmq/store/config/MessageStoreConfig.java + +private String messageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h"; +``` + +Nowadays RocketMq does not support any time delay. It needs to set several fixed delay levels, which correspond to level 1 to 18 from 1s to 2h. Message consumption failure will enter the delay message queue. Message sending time is related to the set delay level and the number of retries. + + See `SendMessageProcessor.java` diff --git a/docs/en/Example_Filter.md b/docs/en/Example_Filter.md new file mode 100644 index 00000000000..caf34cb7d50 --- /dev/null +++ b/docs/en/Example_Filter.md @@ -0,0 +1,86 @@ +# Filter Example +---------- + +In most cases, tag is a simple and useful design to select messages you want. For example: + +```java +DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_EXAMPLE"); +consumer.subscribe("TOPIC", "TAGA || TAGB || TAGC"); +``` + +The consumer will recieve messages that contains TAGA or TAGB or TAGC. But the limitation is that one message only can have one tag, and this may not work for sophisticated scenarios. In this case, you can use SQL expression to filter out messages. +SQL feature could do some calculation through the properties you put in when sending messages. Under the grammars defined by RocketMQ, you can implement some interesting logic. Here is an example: + +``` +------------ +| message | +|----------| a > 5 AND b = 'abc' +| a = 10 | --------------------> Gotten +| b = 'abc'| +| c = true | +------------ +------------ +| message | +|----------| a > 5 AND b = 'abc' +| a = 1 | --------------------> Missed +| b = 'abc'| +| c = true | +------------ +``` + +## 1 Grammars +RocketMQ only defines some basic grammars to support this feature. You could also extend it easily. + +- Numeric comparison, like **>**, **>=**, **<**, **<=**, **BETWEEN**, **=**; +- Character comparison, like **=**, **<>**, **IN**; +- **IS NULL** or **IS NOT NULL**; +- Logical **AND**, **OR**, **NOT**; + +Constant types are: + +- Numeric, like **123, 3.1415**; +- Character, like **‘abc’**, must be made with single quotes; +- **NULL**, special constant; +- Boolean, **TRUE** or **FALSE**; + +## 2 Usage constraints +Only push consumer could select messages by SQL92. The interface is: +``` +public void subscribe(finalString topic, final MessageSelector messageSelector) +``` + +## 3 Producer example +You can put properties in message through method putUserProperty when sending. + +```java +DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); +producer.start(); +Message msg = new Message("TopicTest", + tag, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) +); +// Set some properties. +msg.putUserProperty("a", String.valueOf(i)); +SendResult sendResult = producer.send(msg); + +producer.shutdown(); + +``` + +## 4 Consumer example +Use `MessageSelector.bySql` to select messages through SQL when consuming. + + +```java +DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_4"); +// only subsribe messages have property a, also a >=0 and a <= 3 +consumer.subscribe("TopicTest", MessageSelector.bySql("a between 0 and 3"); +consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } +}); +consumer.start(); + +``` diff --git a/docs/en/Example_Orderly.md b/docs/en/Example_Orderly.md new file mode 100644 index 00000000000..e9cde37491e --- /dev/null +++ b/docs/en/Example_Orderly.md @@ -0,0 +1,233 @@ +# Example for Ordered Messages + +RocketMQ provides ordered messages using FIFO order. All related messages need to be sent into the same message queue in an orderly manner. + +The following demonstrates ordered messages by ensuring order of create, pay, send and finish steps of sales order process. + +## 1 produce ordered messages + +```java +package org.apache.rocketmq.example.order2 + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.MessageQueueSelector; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageQueue; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +/* +* ordered messages producer +*/ +public class Producer { + + public static void main(String[] args) throws Exception { + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + producer.setNamesrvAddr("127.0.0.1:9876"); + producer.start(); + String[] tags = new String[]{"TagA", "TagC", "TagD"}; + // sales orders list + List orderList = new Producer().buildOrders(); + + Date date = new Date(); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + String dateStr = sdf.format(date); + + for (int i = 0; i < 10; i++) { + // generate message timestamp + String body = dateStr + " Hello RocketMQ " + orderList.get(i); + Message msg = new Message("TopicTest", tags[i % tags.length], "KEY" + i, body.getBytes()); + + SendResult sendResult = producer.send(msg, new MessageQueueSelector() { + @Override + public MessageQueue select(List mqs, Message msg, Object arg) { + Long id = (Long) arg; //message queue is selected by #salesOrderID + long index = id % mqs.size(); + return mqs.get((int) index); + } + }, orderList.get(i).getOrderId()); + + System.out.println(String.format("SendResult status:%s, queueId:%d, body:%s", + sendResult.getSendStatus(), + sendResult.getMessageQueue().getQueueId(), + body)); + } + + producer.shutdown(); + } + + /** + * each sales order step + */ + private static class OrderStep { + private long orderId; + private String desc; + + public long getOrderId() { + return orderId; + } + + public void setOrderId(long orderId) { + this.orderId = orderId; + } + + public String getDesc() { + return desc; + } + + public void setDesc(String desc) { + this.desc = desc; + } + + @Override + public String toString() { + return "OrderStep{" + + "orderId=" + orderId + + ", desc='" + desc + '\'' + + '}'; + } + } + + /** + * to generate ten OrderStep objects for three sales orders: + * #SalesOrder "15103111039L": create, pay, send, finish; + * #SalesOrder "15103111065L": create, pay, finish; + * #SalesOrder "15103117235L": create, pay, finish; + */ + private List buildOrders() { + + List orderList = new ArrayList(); + + //create sales order with orderid="15103111039L" + OrderStep orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("create"); + orderList.add(orderDemo); + + //create sales order with orderid="15103111065L" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("create"); + orderList.add(orderDemo); + + //pay sales order #"15103111039L" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("pay"); + orderList.add(orderDemo); + + //create sales order with orderid="15103117235L" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("create"); + orderList.add(orderDemo); + + //pay sales order #"15103111065L" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("pay"); + orderList.add(orderDemo); + + //pay sales order #"15103117235L" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("pay"); + orderList.add(orderDemo); + + //mark sales order #"15103111065L" as "finish" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111065L); + orderDemo.setDesc("finish"); + orderList.add(orderDemo); + + //mark mark sales order #"15103111039L" as "send" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("send"); + orderList.add(orderDemo); + + ////mark sales order #"15103117235L" as "finish" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103117235L); + orderDemo.setDesc("finish"); + orderList.add(orderDemo); + + //mark sales order #"15103111039L" as "finish" + orderDemo = new OrderStep(); + orderDemo.setOrderId(15103111039L); + orderDemo.setDesc("finish"); + orderList.add(orderDemo); + + return orderList; + } +} + +``` + +## 2 Consume ordered messages + +```java + +package org.apache.rocketmq.example.order2; + +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.apache.rocketmq.common.message.MessageExt; + +import java.util.List; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +/** + * consume messages in order + */ +public class ConsumerInOrder { + + public static void main(String[] args) throws Exception { + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_3"); + consumer.setNamesrvAddr("127.0.0.1:9876"); + /** + * when the consumer is first run, the start point of message queue where it can get messages will be set. + * or if it is restarted, it will continue from the last place to get messages. + */ + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + + consumer.subscribe("TopicTest", "TagA || TagC || TagD"); + + consumer.registerMessageListener(new MessageListenerOrderly() { + + Random random = new Random(); + + @Override + public ConsumeOrderlyStatus consumeMessage(List msgs, ConsumeOrderlyContext context) { + context.setAutoCommit(true); + for (MessageExt msg : msgs) { + // one consumer for each message queue, and messages order are kept in a single message queue. + System.out.println("consumeThread=" + Thread.currentThread().getName() + "queueId=" + msg.getQueueId() + ", content:" + new String(msg.getBody())); + } + + try { + TimeUnit.SECONDS.sleep(random.nextInt(10)); + } catch (Exception e) { + e.printStackTrace(); + } + return ConsumeOrderlyStatus.SUCCESS; + } + }); + + consumer.start(); + + System.out.println("Consumer Started."); + } +} + +``` + + diff --git a/docs/en/Example_Simple.md b/docs/en/Example_Simple.md new file mode 100644 index 00000000000..0ce4924cc78 --- /dev/null +++ b/docs/en/Example_Simple.md @@ -0,0 +1,136 @@ +# Basic Sample +------ +Two functions below are provided in the basic sample: +* The RocketMQ can be utilized to send messages in three ways: reliable synchronous, reliable asynchronous, and one-way transmission. The first two message types are reliable because there is a response whether they were sent successfully. +* The RocketMQ can be utilized to consume messages. +### 1 Add Dependency +maven: +``` java + + org.apache.rocketmq + rocketmq-client + 4.3.0 + +``` +gradle: +``` java +compile 'org.apache.rocketmq:rocketmq-client:4.3.0' +``` +### 2 Send Messages +##### 2.1 Use Producer to Send Synchronous Messages +Reliable synchronous transmission is used in extensive scenes, such as important notification messages, SMS notification. +``` java +public class SyncProducer { + public static void main(String[] args) throws Exception { + // Instantiate with a producer group name + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // Specify name server addresses + producer.setNamesrvAddr("localhost:9876"); + // Launch the producer instance + producer.start(); + for (int i = 0; i < 100; i++) { + // Create a message instance with specifying topic, tag and message body + Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); + // Send message to one of brokers + SendResult sendResult = producer.send(msg); + // Check whether the message has been delivered by the callback of sendResult + System.out.printf("%s%n", sendResult); + } + // Shut down once the producer instance is not longer in use + producer.shutdown(); + } +} +``` +##### 2.2 Send Asynchronous Messages +Asynchronous transmission is generally used in response time sensitive business scenarios. It means that it is unable for the sender to wait the response of the Broker too long. +``` java +public class AsyncProducer { + public static void main(String[] args) throws Exception { + // Instantiate with a producer group name + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // Specify name server addresses + producer.setNamesrvAddr("localhost:9876"); + // Launch the producer instance + producer.start(); + producer.setRetryTimesWhenSendAsyncFailed(0); + for (int i = 0; i < 100; i++) { + final int index = i; + // Create a message instance with specifying topic, tag and message body + Message msg = new Message("TopicTest", + "TagA", + "OrderID188", + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + // SendCallback: receive the callback of the asynchronous return result. + producer.send(msg, new SendCallback() { + @Override + public void onSuccess(SendResult sendResult) { + System.out.printf("%-10d OK %s %n", index, + sendResult.getMsgId()); + } + @Override + public void onException(Throwable e) { + System.out.printf("%-10d Exception %s %n", index, e); + e.printStackTrace(); + } + }); + } + // Shut down once the producer instance is not longer in use + producer.shutdown(); + } +} +``` +##### 2.3 Send Messages in One-way Mode +One-way transmission is used for cases requiring moderate reliability, such as log collection. +``` java +public class OnewayProducer { + public static void main(String[] args) throws Exception{ + // Instantiate with a producer group name + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + // Specify name server addresses + producer.setNamesrvAddr("localhost:9876"); + // Launch the producer instance + producer.start(); + for (int i = 0; i < 100; i++) { + // Create a message instance with specifying topic, tag and message body + Message msg = new Message("TopicTest" /* Topic */, + "TagA" /* Tag */, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) /* Message body */ + ); + // Send in one-way mode, no return result + producer.sendOneway(msg); + } + // Shut down once the producer instance is not longer in use + producer.shutdown(); + } +} +``` +### 3 Consume Messages +``` java +public class Consumer { + public static void main(String[] args) throws InterruptedException, MQClientException { + // Instantiate with specified consumer group name + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name"); + + // Specify name server addresses + consumer.setNamesrvAddr("localhost:9876"); + + // Subscribe one or more topics and tags for finding those messages need to be consumed + consumer.subscribe("TopicTest", "*"); + // Register callback to execute on arrival of messages fetched from brokers + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + System.out.printf("%s Receive New Messages: %s %n", Thread.currentThread().getName(), msgs); + // Mark the message that have been consumed successfully + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + // Launch the consumer instance + consumer.start(); + System.out.printf("Consumer Started.%n"); + } +} +``` \ No newline at end of file diff --git a/docs/en/Example_Transaction.md b/docs/en/Example_Transaction.md new file mode 100644 index 00000000000..9dc700e375e --- /dev/null +++ b/docs/en/Example_Transaction.md @@ -0,0 +1,96 @@ +# Transaction Message Example + +## 1 Transaction message status +There are three states for transaction message: +- TransactionStatus.CommitTransaction: commit transaction, it means that allow consumers to consume this message. +- TransactionStatus.RollbackTransaction: rollback transaction, it means that the message will be deleted and not allowed to consume. +- TransactionStatus.Unknown: intermediate state, it means that MQ is needed to check back to determine the status. + +## 2 Send transactional message example + +### 2.1 Create the transactional producer +Use ```TransactionMQProducer```class to create producer client, and specify a unique ```ProducerGroup```, and you can set up a custom thread pool to process check requests. After executing the local transaction, you need to reply to MQ according to the execution result, and the reply status is described in the above section. +```java +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; +import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; +import org.apache.rocketmq.common.message.MessageExt; +import java.util.List; +public class TransactionProducer { + public static void main(String[] args) throws MQClientException, InterruptedException { + TransactionListener transactionListener = new TransactionListenerImpl(); + TransactionMQProducer producer = new TransactionMQProducer("please_rename_unique_group_name"); + ExecutorService executorService = new ThreadPoolExecutor(2, 5, 100, TimeUnit.SECONDS, new ArrayBlockingQueue(2000), new ThreadFactory() { + @Override + public Thread newThread(Runnable r) { + Thread thread = new Thread(r); + thread.setName("client-transaction-msg-check-thread"); + return thread; + } + }); + producer.setExecutorService(executorService); + producer.setTransactionListener(transactionListener); + producer.start(); + String[] tags = new String[] {"TagA", "TagB", "TagC", "TagD", "TagE"}; + for (int i = 0; i < 10; i++) { + try { + Message msg = + new Message("TopicTest1234", tags[i % tags.length], "KEY" + i, + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET)); + SendResult sendResult = producer.sendMessageInTransaction(msg, null); + System.out.printf("%s%n", sendResult); + Thread.sleep(10); + } catch (MQClientException | UnsupportedEncodingException e) { + e.printStackTrace(); + } + } + for (int i = 0; i < 100000; i++) { + Thread.sleep(1000); + } + producer.shutdown(); + } +} +``` + +### 2.2 Implement the TransactionListener interface +The ```executeLocalTransaction``` method is used to execute local transaction when send half message succeed. It returns one of three transaction status mentioned in the previous section. + +The ```checkLocalTransaction``` method is used to check the local transaction status and respond to MQ check requests. It also returns one of three transaction status mentioned in the previous section. +```java +public class TransactionListenerImpl implements TransactionListener { + private AtomicInteger transactionIndex = new AtomicInteger(0); + private ConcurrentHashMap localTrans = new ConcurrentHashMap<>(); + @Override + public LocalTransactionState executeLocalTransaction(Message msg, Object arg) { + int value = transactionIndex.getAndIncrement(); + int status = value % 3; + localTrans.put(msg.getTransactionId(), status); + return LocalTransactionState.UNKNOW; + } + @Override + public LocalTransactionState checkLocalTransaction(MessageExt msg) { + Integer status = localTrans.get(msg.getTransactionId()); + if (null != status) { + switch (status) { + case 0: + return LocalTransactionState.UNKNOW; + case 1: + return LocalTransactionState.COMMIT_MESSAGE; + case 2: + return LocalTransactionState.ROLLBACK_MESSAGE; + } + } + return LocalTransactionState.COMMIT_MESSAGE; + } +} +``` + +## 3 Usage Constraint +1. Messages of the transactional have no schedule and batch support. +2. In order to avoid a single message being checked too many times and lead to half queue message accumulation, we limited the number of checks for a single message to 15 times by default, but users can change this limit by change the ```transactionCheckMax``` parameter in the configuration of the broker, if one message has been checked over ```transactionCheckMax``` times, broker will discard this message and print an error log at the same time by default. Users can change this behavior by override the ```AbstractTransactionCheckListener``` class. +3. A transactional message will be checked after a certain period of time that determined by parameter ```transactionTimeout``` in the configuration of the broker. And users also can change this limit by set user property “CHECK_IMMUNITY_TIME_IN_SECONDS” when sending transactional message, this parameter takes precedence over the “transactionMsgTimeout” parameter. +4. A transactional message maybe checked or consumed more than once. +5. Committed message reput to the user’s target topic may fail. Currently, it depends on the log record. High availability is ensured by the high availability mechanism of RocketMQ itself. If you want to ensure that the transactional message isn’t lost and the transaction integrity is guaranteed, it is recommended to use synchronous double write. mechanism. +6. Producer IDs of transactional messages cannot be shared with producer IDs of other types of messages. Unlike other types of message, transactional messages allow backward queries. MQ Server query clients by their Producer IDs. + diff --git a/docs/en/FAQ.md b/docs/en/FAQ.md new file mode 100644 index 00000000000..a08315cb153 --- /dev/null +++ b/docs/en/FAQ.md @@ -0,0 +1,109 @@ +# Frequently Asked Questions + +The following questions are frequently asked with regard to the RocketMQ project in general. + +## 1 General + +1. Why did we create rocketmq project instead of selecting other products? + + Please refer to [Why RocketMQ](http://rocketmq.apache.org/docs/motivation) + +2. Do I have to install other softeware, such as zookeeper, to use RocketMQ? + + No. RocketMQ can run independently. + +## 2 Usage + +### 1. Where does the newly created Consumer ID start consuming messages? + + 1) If the topic sends a message within three days, then the consumer start consuming messages from the first message saved in the server. + + 2) If the topic sends a message three days ago, the consumer starts to consume messages from the latest message in the server, in other words, starting from the tail of message queue. + + 3) If such consumer is rebooted, then it starts to consume messages from the last consumption location. + +### 2. How to reconsume message when consumption fails? + + 1) Cluster consumption pattern, The consumer business logic code returns Action.ReconsumerLater, NULL, or throws an exception, if a message failed to be consumed, it will retry for up to 16 times, after that, the message would be descarded. + + 2) Broadcast consumption patternThe broadcaset consumption still ensures that a message is consumered at least once, but no resend option is provided. + +### 3. How to query the failed message if there is a consumption failure? + + 1) Using topic query by time, you can query messages within a period of time. + + 2) Using Topic and Message Id to accurately query the message. + + 3) Using Topic and Message Key accurately query a class of messages with the same Message Key. + +### 4. Are messages delivered exactly once? + +RocketMQ ensures that all messages are delivered at least once. In most cases, the messages are not repeated. + +### 5. How to add a new broker? + + 1) Start up a new broker and register it to the same list of name servers. + + 2) By default, only internal system topics and consumer groups are created automatically. If you would like to have your business topic and consumer groups on the new node, please replicate them from the existing broker. Admin tool and command lines are provided to handle this. + +## 3 Configuration related + +The following answers are all default values and can be modified by configuration. + +### 1. How long are the messages saved on the server? + +Stored messages will be saved for up to 3 days, and messages that are not consumed for more than 3 days will be deleted. + +### 2. What is the size limit for message Body? + +Generally 256KB. + +### 3. How to set the number of consumer threads? + +When you start Consumer, set a ConsumeThreadNums property, example is as follows: +``` +consumer.setConsumeThreadMin(20); +consumer.setConsumeThreadMax(20); +``` + +## 4 Errors + +### 1. If you start a producer or consumer failed and the error message is producer group or consumer repeat. + +Reason:Using the same Producer /Consumer Group to launch multiple instances of Producer/Consumer in the same JVM may cause the client fail to start. + +Solution: Make sure that a JVM corresponding to one Producer /Consumer Group starts only with one Producer/Consumer instance. + +### 2. Consumer failed to start loading json file in broadcast mode. + +Reason: Fastjson version is too low to allow the broadcast consumer to load local offsets.json, causing the consumer boot failure. Damaged fastjson file can also cause the same problem. + +Solution: Fastjson version has to be upgraded to rocketmq client dependent version to ensure that the local offsets.json can be loaded. By default offsets.json file is in /home/{user}/.rocketmq_offsets. Or check the integrity of fastjson. + +### 3. What is the impact of a broker crash. + + 1) Master crashes + +Messages can no longer be sent to this broker set, but if you have another broker set available, messages can still be sent given the topic is present. Messages can still be consumed from slaves. + + 2) Some slave crash + +As long as there is another working slave, there will be no impact on sending messages. There will also be no impact on consuming messages except when the consumer group is set to consume from this slave preferably. By default, comsumer group consumes from master. + + 3) All slaves crash + +There will be no impact on sending messages to master, but, if the master is SYNC_MASTER, producer will get a SLAVE_NOT_AVAILABLE indicating that the message is not sent to any slaves. There will also be no impact on consuming messages except that if the consumer group is set to consume from slave preferably. By default, comsumer group consumes from master. + +### 4. Producer complains “No Topic Route Info”, how to diagnose? + +This happens when you are trying to send messages to a topic whose routing info is not available to the producer. + + 1) Make sure that the producer can connect to a name server and is capable of fetching routing meta info from it. + + 2) Make sure that name servers do contain routing meta info of the topic. You may query the routing meta info from name server through topicRoute using admin tools or web console. + + 3) Make sure that your brokers are sending heartbeats to the same list of name servers your producer is connecting to. + + 4) Make sure that the topic’s permssion is 6(rw-), or at least 2(-w-). + +If you can’t find this topic, create it on a broker via admin tools command updateTopic or web console. diff --git a/docs/en/Feature.md b/docs/en/Feature.md new file mode 100644 index 00000000000..806d6482913 --- /dev/null +++ b/docs/en/Feature.md @@ -0,0 +1,86 @@ +# Features +## 1 Subscribe and Publish +Message publication refers to that a producer sends messages to a topic; Message subscription means a consumer follows a topic with certain tags and then consumes data from that topic. + +## 2 Message Ordering +Message ordering refers to that a group of messages can be consumed orderly as they are published. For example, an order generates three messages: order creation, order payment, and order completion. It only makes sense to consume them in their generated order, but orders can be consumed in parallel at the same time. RocketMQ can strictly guarantee these messages are in order. + +Orderly message are divided into global orderly message and partitioned orderly message. Global order means that all messages under a certain topic must be in order, partitioned order only requires each group of messages are consumed orderly. +- Global message ordering: +For a given Topic, all messages are published and consumed in strict first-in-first-out (FIFO) order. +Applicable scenario: the performance requirement is not high, and all messages are published and consumed according to FIFO principle strictly. +- Partitioned message ordering: +For a given Topic, all messages are partitioned according to sharding key. Messages within the same partition are published and consumed in strict FIFO order. Sharding key is the key field to distinguish message's partition, which is a completely different concept from the key of ordinary messages. +Applicable scenario: high performance requirement, with sharding key as the partition field, messages within the same partition are published and consumed according to FIFO principle strictly. + +## 3 Message Filter +Consumers of RocketMQ can filter messages based on tags as well as support for user-defined attribute filtering. Message filter is currently implemented on the Broker side, with the advantage of reducing the network transmission of useless messages for Consumer and the disadvantage of increasing the burden on the Broker and relatively complex implementation. + +## 4 Message Reliability +RocketMQ supports high reliability of messages in several situations: +1 Broker shutdown normally +2 Broker abnormal crash +3 OS Crash +4 The machine is out of power, but it can be recovered immediately +5 The machine cannot be started up (the CPU, motherboard, memory and other key equipment may be damaged) +6 Disk equipment damaged + +In the four cases of 1), 2), 3), and 4) where the hardware resource can be recovered immediately, RocketMQ guarantees that the message will not be lost or a small amount of data will be lost (depending on whether the flush disk type is synchronous or asynchronous) + +5) and 6) are single point of failure and cannot be recovered. Once it happens, all messages on the single point will be lost. In both cases, RocketMQ ensures that 99% of the messages are not lost through asynchronous replication, but a very few number of messages may still be lost. Synchronous double write mode can completely avoid single point of failure, which will surely affect the performance and suitable for the occasion of high demand for message reliability, such as money related applications. Note: RocketMQ supports synchronous double writes since version 3.0. + +## 5 At Least Once +At least Once refers to that every message will be delivered at least once. RocketMQ supports this feature because the Consumer pulls the message locally and does not send an ack back to the server until it has consumed it. + +## 6 Backtracking Consumption +Backtracking consumption refers to that the Consumer has consumed the message successfully, but the business needs to consume again. To support this function, the message still needs to be retained after the Broker sends the message to the Consumer successfully. The re-consumption is normally based on time dimension. For example, after the recovery of the Consumer system failure, the data one hour ago needs to be re-consumed, then the Broker needs to provide a mechanism to reverse the consumption progress according to the time dimension. RocketMQ supports backtracking consumption by time trace, with the time dimension down to milliseconds. + +## 7 Transactional Message +RocketMQ transactional message refers to the fact that the application of a local transaction and the sending of a Message operation can be defined in a global transaction which means both succeed or fail simultaneously. RocketMQ transactional message provides distributed transaction functionality similar to X/Open XA, enabling the ultimate consistency of distributed transactions through transactional message. + +## 8 Scheduled Message +Scheduled message(delay queue) refers to that messages are not consumed immediately after they are sent to the broker, but waiting to be delivered to the real topic after a specific time. +The broker has a configuration item, messageDelayLevel, with default values “1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h”, 18 levels. Users can configure a custom messageDelayLevel. Note that messageDelayLevel is a broker's property rather than a topic's. When sending a message, just set the delayLevel level: msg.setDelayLevel(level). There are three types of levels: + +- level == 0, The message is not a delayed message +- 1<=level<=maxLevel, Message delay specific time, such as level==1, delay for 1s +- level > maxLevel, than level== maxLevel, such as level==20, delay for 2h + +Scheduled messages are temporarily saved in a topic named SCHEDULE_TOPIC_XXXX, and saved in a specific queue according to delayTimeLevel, queueId = delayTimeLevel - 1, that is, only messages with the same delay are saved in a queue, ensuring that messages with the same sending delay can be consumed orderly. The broker consumes SCHEDULE_TOPIC_XXXX on schedule and writes messages to the real topic. + +Note that Scheduled messages are counted both the first time they are written and the time they are scheduled to be written to the real topic, so both the send number and the TPS are increased. + +## 9 Message Retry +When the Consumer fails to consume the message, a retry mechanism is needed to make the message to be consumed again. Consumer's consume failure can usually be classified as follows: +- due to the reasons of the message itself, such as deserialization failure, the message data itself cannot be processed (for example, the phone number of the current message is cancelled and cannot be charged), etc. This kind of error usually requires skipping this message and consuming others since immediately retry would be failed 99%, so it is better to provide a timed retry mechanism that retries after 10 seconds. +- due to the reasons of dependent downstream application services are not available, such as db connection is not usable, perimeter network is not unreachable, etc. When this kind of error is encountered, consuming other messages will also result in an error even if the current failed message is skipped. In this case, it is recommended to sleep for 30s before consuming the next message, which will reduce the pressure on the broker to retry the message. + +RocketMQ will set up a retry queue named “%RETRY%+consumerGroup” for each consumer group(Note that the retry queue for this topic is for consumer groups, not for each topic) to temporarily save messages cannot be consumed by customer due to all kinds of reasons. Considering that it takes some time for the exception to recover, multiple retry levels are set for the retry queue, and each retry level has a corresponding re-deliver delay. The more retries, the greater the deliver delay. RocketMQ first save retry messages to the delay queue which topic is named “SCHEDULE_TOPIC_XXXX”, then background schedule task will save the messages to “%RETRY%+consumerGroup” retry queue according to their corresponding delay. + +## 10 Message Resend +When a producer sends a message, the synchronous message will be resent if fails, the asynchronous message will retry and oneway message is without any guarantee. Message resend ensures that messages are sent successfully and without lost as much as possible, but it can lead to message duplication, which is an unavoidable problem in RocketMQ. Under normal circumstances, message duplication will not occur, but when there is a large number of messages and network jitter, message duplication will be a high-probability event. In addition, producer initiative messages resend and the consumer load changes will also result in duplicate messages. The message retry policy can be set as follows: + +- retryTimesWhenSendFailed: Synchronous message retry times when send failed, default value is 2, so the producer will try to send retryTimesWhenSendFailed + 1 times at most. To ensure that the message is not lost, producer will try sending the message to another broker instead of selecting the broker that failed last time. An exception will be thrown if it reaches the retry limit, and the client should guarantee that the message will not be lost. Messages will resend when RemotingException, MQClientException, and partial MQBrokerException occur. +- retryTimesWhenSendAsyncFailed: Asynchronous message retry times when send failed, asynchronous retry sends message to the same broker instead of selecting another one and does not guarantee that the message wont lost. +- retryAnotherBrokerWhenNotStoreOK: Message flush disk (master or slave) timeout or slave not available (return status is not SEND_OK), whether to try to send to another broker, default value is false. Very important messages can set to true. + +## 11 Flow Control +Producer flow control, because broker processing capacity reaches a bottleneck; Consumer flow control, because the consumption capacity reaches a bottleneck. + +Producer flow control: +- When commitLog file locked time exceeds osPageCacheBusyTimeOutMills, default value of osPageCacheBusyTimeOutMills is 1000 ms, then return flow control. +- If transientStorePoolEnable == true, and the broker is asynchronous flush disk type, and resources are insufficient in the transientStorePool, reject the current send request and return flow control. +- The broker checks the head request wait time of the send request queue every 10ms. If the wait time exceeds waitTimeMillsInSendQueue, which default value is 200ms, the current send request is rejected and the flow control is returned. +- The broker implements flow control by rejecting send requests. + +Consumer flow control: +- When consumer local cache messages number exceeds pullThresholdForQueue, default value is 1000. +- When consumer local cache messages size exceeds pullThresholdSizeForQueue, default value is 100MB. +- When consumer local cache messages span exceeds consumeConcurrentlyMaxSpan, default value is 2000. + +The result of consumer flow control is to reduce the pull frequency. + +## 12 Dead Letter Queue +Dead letter queue is used to deal messages that cannot be consumed normally. When a message is consumed failed at first time, the message queue will automatically resend the message. If the consumption still fails after the maximum number retry, it indicates that the consumer cannot properly consume the message under normal circumstances. At this time, the message queue will not immediately abandon the message, but send it to the special queue corresponding to the consumer. + +RocketMQ defines the messages that could not be consumed under normal circumstances as Dead-Letter Messages, and the special queue in which the Dead-Letter Messages are saved as Dead-Letter Queues. In RocketMQ, the consumer instance can consume again by resending messages in the Dead-Letter Queue using console. \ No newline at end of file diff --git a/docs/en/Operations_Broker.md b/docs/en/Operations_Broker.md new file mode 100644 index 00000000000..2ec9f03763e --- /dev/null +++ b/docs/en/Operations_Broker.md @@ -0,0 +1,23 @@ +# 3 Broker + +## 3.1 Broker Role +Broker Role is ASYNC_MASTER, SYNC_MASTER or SLAVE. If you cannot tolerate message missing, we suggest you deploy SYNC_MASTER and attach a SLAVE to it. If you feel ok about missing, but you want the Broker to be always available, you may deploy ASYNC_MASTER with SLAVE. If you just want to make it easy, you may only need a ASYNC_MASTER without SLAVE. +## 3.2 FlushDiskType +ASYNC_FLUSH is recommended, for SYNC_FLUSH is expensive and will cause too much performance loss. If you want reliability, we recommend you use SYNC_MASTER with SLAVE. +## 3.3 Broker Configuration +| Parameter name | Default | Description | +| -------------------------------- | ----------------------------- | ------------------------------------------------------------ | +| listenPort | 10911 | listen port for client | +| namesrvAddr | null | name server address | +| brokerIP1 | InetAddress for network interface | Should be configured if having multiple addresses | +| brokerIP2 | InetAddress for network interface | If configured for the Master broker in the Master/Slave cluster, slave broker will connect to this port for data synchronization | +| brokerName | null | broker name | +| brokerClusterName | DefaultCluster | this broker belongs to which cluster | +| brokerId | 0 | broker id, 0 means master, positive integers mean slave | +| storePathCommitLog | $HOME/store/commitlog/ | file path for commit log | +| storePathConsumerQueue | $HOME/store/consumequeue/ | file path for consume queue | +| mapedFileSizeCommitLog | 1024 * 1024 * 1024(1G) | mapped file size for commit log |​ +| deleteWhen | 04 | When to delete the commitlog which is out of the reserve time |​ +| fileReserverdTime | 72 | The number of hours to keep a commitlog before deleting it |​ +| brokerRole | ASYNC_MASTER | SYNC_MASTER/ASYNC_MASTER/SLAVE |​ +| flushDiskType | ASYNC_FLUSH | {SYNC_FLUSH/ASYNC_FLUSH}. Broker of SYNC_FLUSH mode flushes each message onto disk before acknowledging producer. Broker of ASYNC_FLUSH mode, on the other hand, takes advantage of group-committing, achieving better performance. |​ \ No newline at end of file diff --git a/docs/en/Operations_Consumer.md b/docs/en/Operations_Consumer.md new file mode 100644 index 00000000000..84047dfc929 --- /dev/null +++ b/docs/en/Operations_Consumer.md @@ -0,0 +1,106 @@ +## Consumer + +---- + +### 1 Consumption process idempotent + +RocketMQ cannot avoid Exactly-Once, so if the business is very sensitive to consumption repetition, it is important to perform deduplication at the business level. Deduplication can be done with a relational database. First, you need to determine the unique key of the message, which can be either msgId or a unique identifier field in the message content, such as the order Id. Determine if a unique key exists in the relational database before consumption. If it does not exist, insert it and consume it, otherwise skip it. (The actual process should consider the atomic problem, determine whether there is an attempt to insert, if the primary key conflicts, the insertion fails, skip directly) + +### 2 Slow message processing + +#### 2.1 Increase consumption parallelism + +Most messages consumption behaviors are IO-intensive, That is, it may be to operate the database, or call RPC. The consumption speed of such consumption behavior lies in the throughput of the back-end database or the external system. By increasing the consumption parallelism, the total consumption throughput can be increased, but the degree of parallelism is increased to a certain extent. Instead it will fall.Therefore, the application must set a reasonable degree of parallelism. There are several ways to modify the degree of parallelism of consumption as follows: + +* Under the same ConsumerGroup, increase the degree of parallelism by increasing the number of Consumer instances (note that the Consumer instance that exceeds the number of subscription queues is invalid). Can be done by adding machines, or by starting multiple processes on an existing machine. +* Improve the consumption parallel thread of a single Consumer by modifying the parameters consumeThreadMin and consumeThreadMax. + +#### 2.2 Batch mode consumption + +Some business processes can increase consumption throughput to a large extent if they support batch mode consumption. For example, order deduction application, it takes 1s to process one order at a time, and it takes only 2s to process 10 orders at a time. In this way, the throughput of consumption can be greatly improved. By setting the consumer's consumeMessageBatchMaxSize to return a parameter, the default is 1, that is, only one message is consumed at a time, for example, set to N, then the number of messages consumed each time is less than or equal to N. + +#### 2.3 Skip non-critical messages + +When a message is accumulated, if the consumption speed cannot keep up with the transmission speed, if the service does not require high data, you can choose to discard the unimportant message. For example, when the number of messages in a queue is more than 100,000 , try to discard some or all of the messages, so that you can quickly catch up with the speed of sending messages. The sample code is as follows: + +```java +public ConsumeConcurrentlyStatus consumeMessage( + List msgs, + ConsumeConcurrentlyContext context){ + long offest = msgs.get(0).getQueueOffset(); + String maxOffset = + msgs.get(0).getProperty(Message.PROPERTY_MAX_OFFSET); + long diff = Long.parseLong(maxOffset) - offset; + if(diff > 100000){ + //TODO Special handling of message accumulation + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + //TODO Normal consumption process + return ConcumeConcurrentlyStatus.CONSUME_SUCCESS; +} +``` + +#### 2.4 Optimize each message consumption process + +For example, the consumption process of a message is as follows: + +* Query from DB according to the message [data 1] +* Query from DB according to the message [data 2] +* Complex business calculations +* Insert [Data 3] into the DB +* Insert [Data 4] into the DB + +There are 4 interactions with the DB in the consumption process of this message. If it is calculated by 5ms each time, it takes a total of 20ms. If the business calculation takes 5ms, then the total time is 25ms, So if you can optimize 4 DB interactions to 2 times, the total time can be optimized to 15ms, which means the overall performance is increased by 40%. Therefore, if the application is sensitive to delay, the DB can be deployed on the SSD hard disk. Compared with the SCSI disk, the former RT will be much smaller. + +### 3 Print Log + +If the amount of messages is small, it is recommended to print the message in the consumption entry method, consume time, etc., to facilitate subsequent troubleshooting. + +```java +public ConsumeConcurrentlyStatus consumeMessage( + List msgs, + ConsumeConcurrentlyContext context){ + log.info("RECEIVE_MSG_BEGIN: " + msgs.toString()); + //TODO Normal consumption process + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; +} +``` + +If you can print the time spent on each message, it will be more convenient when troubleshooting online problems such as slow consumption. + +### 4 Other consumption suggestions + +#### 4.1、Consumer Group and Subscriptions + +The first thing you should be aware of is that different Consumer Group can consume the same topic independently, and each of them will have their own consuming offsets. Please make sure each Consumer within the same Group to subscribe the same topics. + +#### 4.2、Orderly + +The Consumer will lock each MessageQueue to make sure it is consumed one by one in order. This will cause a performance loss, but it is useful when you care about the order of the messages. It is not recommended to throw exceptions, you can return ConsumeOrderlyStatus.SUSPEND_CURRENT_QUEUE_A_MOMENT instead. + +#### 4.3、Concurrently + +As the name tells, the Consumer will consume the messages concurrently. It is recommended to use this for good performance. It is not recommended to throw exceptions, you can return ConsumeConcurrentlyStatus.RECONSUME_LATER instead. + +#### 4.4、Consume Status + +For MessageListenerConcurrently, you can return RECONSUME_LATER to tell the consumer that you can not consume it right now and want to reconsume it later. Then you can continue to consume other messages. For MessageListenerOrderly, because you care about the order, you can not jump over the message, but you can return SUSPEND_CURRENT_QUEUE_A_MOMENT to tell the consumer to wait for a moment. + +#### 4.5、Blocking + +It is not recommend to block the Listener, because it will block the thread pool, and eventually may stop the consuming process. + +#### 4.6、Thread Number + +The consumer use a ThreadPoolExecutor to process consuming internally, so you can change it by setting setConsumeThreadMin or setConsumeThreadMax. + +#### 4.7、ConsumeFromWhere + +When a new Consumer Group is established, it will need to decide whether it needs to consume the historical messages which had already existed in the Broker. CONSUME_FROM_LAST_OFFSET will ignore the historical messages, and consume anything produced after that. CONSUME_FROM_FIRST_OFFSET will consume every message existed in the Broker. You can also use CONSUME_FROM_TIMESTAMP to consume messages produced after the specified timestamp. + + + + + + + diff --git a/docs/en/Operations_Producer.md b/docs/en/Operations_Producer.md new file mode 100644 index 00000000000..68ef4a69210 --- /dev/null +++ b/docs/en/Operations_Producer.md @@ -0,0 +1,44 @@ +### Producer +---- +##### 1 Message Sending Tips +###### 1.1 The Use of Tags +One application instance should use one topic as much as possible and the subtype of messages can be marked by tags. Tag provides extra flexibility to users. In the consume subscribing process, the messages filtering can only be handled by using tags when the tags are specified in the message sending process: `message.setTags("TagA")`. +###### 1.2 The Use of Keys +A business key can be set in one message and it will be easier to look up the message on a broker server to diagnose issues during development. Each message will be created index(hash index) by server, instance can query the content of this message by topic and key and who consumes the message.Because of the hash index, make sure that the key should be unique in order to avoid potential hash index conflict. +``` java +// Order Id +String orderId = "20034568923546"; +message.setKeys(orderId); +``` +###### 1.3 The Log Print +When sending a message,no matter success or fail, a message log must be printed which contains SendResult and Key. It is assumed that we will always get SEND_OK if no exception is thrown. Below is a list of descriptions about each status: +* SEND_OK + +SEND_OK means sending message successfully. SEND_OK does not mean it is reliable. To make sure no message would be lost, you should also enable SYNC_MASTER or SYNC_FLUSH. +* FLUSH_DISK_TIMEOUT + +FLUSH_DISK_TIMEOUT means sending message successfully but the Broker flushing the disk with timeout. In this kind of condition, the Broker has saved this message in memory, this message will be lost only if the Broker was down. The FlushDiskType and SyncFlushTimeout could be specified in MessageStoreConfig. If the Broker set MessageStoreConfig’s FlushDiskType=SYNC_FLUSH(default is ASYNC_FLUSH), and the Broker doesn’t finish flushing the disk within MessageStoreConfig’s syncFlushTimeout(default is 5 secs), you will get this status. +* FLUSH_SLAVE_TIMEOUT + +FLUSH_SLAVE_TIMEOUT means sending messages successfully but the slave Broker does not finish synchronizing with the master. If the Broker’s role is SYNC_MASTER(default is ASYNC_MASTER), and the slave Broker doesn’t finish synchronizing with the master within the MessageStoreConfig’s syncFlushTimeout(default is 5 secs), you will get this status. +* SLAVE_NOT_AVAILABLE + +SLAVE_NOT_AVAILABLE means sending messages successfully but no slave Broker configured. If the Broker’s role is SYNC_MASTER(default is ASYNC_MASTER), but no slave Broker is configured, you will get this status. + +##### 2 Operations on Message Sending failed +The send method of Producer can be retried, the retry process is illustrated below: +* The method will retry at most 2 times(2 times in synchronous mode, 0 times in asynchronous mode). +* If sending failed, it will turn to the next Broker. This strategy will be executed when the total costing time is less then sendMsgTimeout(default is 10 seconds). +* The retry method will be terminated if timeout exception was thrown when sending messages to Broker. + +The strategy above could make sure message sending successfully to a certain degree. Some more retry strategies, such as we could try to save the message to database if calling the send synchronous method failed and then retry by background thread's timed tasks, which will make sure the message is sent to Broker,could be improved if asking for high reliability business requirement. + +The reasons why the retry strategy using database have not integrated by the RocketMQ client will be explained below: Firstly, the design mode of the RocketMQ client is stateless mode. It means that the client is designed to be horizontally scalable at each level and the consumption of the client to physical resources is only CPU, memory and network. Then, if a key-value memory module is integrated by the client itself, the Asyn-Saving strategy will be utilized in consideration of the high resource consumption of the Syn-Saving strategy. However, given that operations staff does not manage the client shutoff, some special commands, such as kill -9, may be used which will lead to the lost of message because of no saving in time. Furthermore, the physical resource running Producer is not appropriate to save some significant data because of low reliability. Above all, the retry process should be controlled by program itself. + +##### 3 Send Messages in One-way Mode +The message sending is usually a process like below: +* Client sends request to sever. +* Sever handles request +* Sever returns response to client + +The total costing time of sending one message is the sum of costing time of three steps above. Some situations demand that total costing time must be in a quite low level, however, do not take reliable performance into consideration, such as log collection. This kind of application could be called in one-way mode, which means client sends request but not wait for response. In this kind of mode, the cost of sending request is only a call of system operation which means one operation writing data to client socket buffer. Generally, the time cost of this process will be controlled n microseconds level. diff --git a/docs/en/Operations_Trace.md b/docs/en/Operations_Trace.md new file mode 100644 index 00000000000..5ed9c8d0863 --- /dev/null +++ b/docs/en/Operations_Trace.md @@ -0,0 +1,104 @@ +# Message Trace + +## 1 Key Attributes of Message Trace Data + +| Producer | Consumer | Broker | +| ---------------- | ----------------- | ------------ | +| production instance information | consumption instance information | message Topic | +| send message time | post time, post round | message storage location | +| whether the message was sent successfully | Whether the message was successfully consumed | The Key of the message | +| Time spent sending | Time spent consuming | Tag of the message | + +## 2 Support for Message Trace Cluster Deployment + +### 2.1 Broker Configuration Fille + +The properties profile content of the Broker side enabled message trace feature is pasted here: + +``` +brokerClusterName=DefaultCluster +brokerName=broker-a +brokerId=0 +deleteWhen=04 +fileReservedTime=48 +brokerRole=ASYNC_MASTER +flushDiskType=ASYNC_FLUSH +storePathRootDir=/data/rocketmq/rootdir-a-m +storePathCommitLog=/data/rocketmq/commitlog-a-m +autoCreateSubscriptionGroup=true +## if msg tracing is open,the flag will be true +traceTopicEnable=true +listenPort=10911 +brokerIP1=XX.XX.XX.XX1 +namesrvAddr=XX.XX.XX.XX:9876 +``` + +### 2.2 Normal Mode +Each Broker node in the RocketMQ cluster is used to store message trace data collected and sent from the Client.Therefore, there are no requirements or restrictions on the number of Broker nodes in the RocketMQ cluster. + +### 2.3 Physical IO Isolation Mode +For scenarios with large amount of trace message data , one of the Broker nodes in the RocketMQ cluster can be selected to store the trace message , so that the common message data of the user and the physical IO of the trace message data are completely isolated from each other.In this mode, there are at least two Broker nodes in the RockeMQ cluster, one of which is defined as the server on which message trace data is stored. + +### 2.4 Start the Broker that Starts the MessageTrace +`nohup sh mqbroker -c ../conf/2m-noslave/broker-a.properties &` + +## 3 Save the Topic Definition of Message Trace +RocketMQ's message trace feature supports two ways to store trace data: + +### 3.1 System-level TraceTopic +By default, message track data is stored in the system-level TraceTopic(names:**RMQ_SYS_TRACE_TOPIC**)。This Topic is automatically created when the Broker node is started(As described above, the switch variable **traceTopicEnable** needs to be set to **true** in the Broker configuration file)。 + +### 3.2 Custom TraceTopic +If the user is not prepared to store the message track data in the system-level default TraceTopic, you can also define and create a user-level Topic to save the track (that is, to create a regular Topic to save the message track data)。The following section introduces how the Client interface supports the user-defined TraceTopic. + +## 4 Client Practices that Support Message Trace +In order to reduce as much as possible the transformation work of RocketMQ message trace feature used in the user service system, the author added a switch parameter (**enableMsgTrace**) to the original interface in the design to realize whether the message trace is opened or not. + +### 4.1 Opening the Message Trace when Sending the Message +``` + DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName",true); + producer.setNamesrvAddr("XX.XX.XX.XX1"); + producer.start(); + try { + { + Message msg = new Message("TopicTest", + "TagA", + "OrderID188", + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + SendResult sendResult = producer.send(msg); + System.out.printf("%s%n", sendResult); + } + + } catch (Exception e) { + e.printStackTrace(); + } +``` + +### 4.2 Opening Message Trace whenSubscribing to a Message +``` + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_JODIE_1",true); + consumer.subscribe("TopicTest", "*"); + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + consumer.setConsumeTimestamp("20181109221800"); + consumer.registerMessageListener(new MessageListenerConcurrently() { + @Override + public ConsumeConcurrentlyStatus consumeMessage(List msgs, ConsumeConcurrentlyContext context) { + System.out.printf("%s Receive New Messages: %s %n", Thread.currentThread().getName(), msgs); + return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; + } + }); + consumer.start(); + System.out.printf("Consumer Started.%n"); +``` + +### 4.3 Support for Custom Storage Message Trace Topic +The initialization of `DefaultMQProducer` and `DefaultMQPushConsumer` instances can be changed to support the custom storage message trace Topic as follows when sending and subscriving messages above. + +``` + ##Where Topic_test11111 needs to be pre-created by the user to save the message trace; + DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName",true,"Topic_test11111"); + ...... + + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("CID_JODIE_1",true,"Topic_test11111"); + ...... +``` \ No newline at end of file diff --git a/docs/en/Troubleshoopting.md b/docs/en/Troubleshoopting.md new file mode 100644 index 00000000000..02a245ed04f --- /dev/null +++ b/docs/en/Troubleshoopting.md @@ -0,0 +1,76 @@ +# Operation FAQ + +## 1 RocketMQ's mqadmin command error. + +> Problem: Sometimes after deploying the RocketMQ cluster, when you try to execute some commands of "mqadmin", the following exception will appear: +> +> ```java +> org.apache.rocketmq.remoting.exception.RemotingConnectException: connect to failed +> ``` + +Solution: Execute `export NAMESRV_ADDR=ip:9876` (ip refers to the address of NameServer deployed in the cluster) on the VM that deploys the RocketMQ cluster.Then you will execute commands of "mqadmin" successfully. + +## 2 The inconsistent version of RocketMQ between the producer and consumer leads to the problem that message can't be consumed normally. + +> Problem: The same producer sends a message, consumer A can consume, but consumer B can't consume, and the RocketMQ Console appears: +> +> ```java +> Not found the consumer group consume stats, because return offset table is empty, maybe the consumer not consume any message +> ``` + +Solution: The jar package of RocketMQ, such as rocketmq-client, should be the same version on the consumer and producer. + +## 3 When adding a new topic consumer group, historical messages can't be consumed. + +> Problem: When a new consumer group of the same topic is started, the consumed message is the current offset message, and the historical message is not obtained. + +Solution: The default policy of rocketmq is to start from the end of the message queue and skip the historical message. If you want to consume historical message, you need to set: + +```java +org.apache.rocketmq.client.consumer.DefaultMQPushConsumer#setConsumeFromWhere +``` + +There are three common configurations: + +- By default, a new subscription group starts to consume from the end of the queue for the first time, and then restarts and continue to consume from the last consume position, that is, to skip the historical message. + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); +``` + +- A new subscription group starts to consume from the head of the queue for the first time, and then restarts and continue to consume from the last consume position, that is, to consume the historical message that is not expired on Broker. + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); +``` + +- A new subscription group starts to consume from the specified time point for the first time, and then restarts and continue to consume from the last consume position. It is used together with `consumer.setConsumeTimestamp()`. The default is half an hour ago. + +```java +consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_TIMESTAMP); +``` + +## 4 How to enable reading data from Slave + +In some cases, the Consumer needs to reset the consume position to 1-2 days ago. At this time, on the Master Broker with limited memory, the CommitLog will carry a relatively heavy IO pressure, affecting the reading and writing of other messages on that Broker. You can enable `slaveReadEnable=true`. When Master Broker finds that the difference between the Consumer's consume position and the latest value of CommitLog exceeds the percentage of machine's memory (`accessMessageInMemoryMaxRatio=40%`), it will recommend Consumer to read from Slave Broker and relieve Master Broker's IO. + +## 5 Performance + +Asynchronous flush disk is recommended to use spin lock. + +Synchronous flush disk is recommended to use reentrant lock. Adjust the Broker configuration item `useReentrantLockWhenPutMessage`, and the default value is false. + +Asynchronous flush disk is recommended to open `TransientStorePoolEnable` and close `transferMsgByHeap` to improve the efficiency of pulling message; + +Synchronous flush disk is recommended to increase the `sendMessageThreadPoolNums` appropriately. The specific configuration needs to be tested. + +## 6 The meaning and difference between msgId and offsetMsgId in RocketMQ + +After sending message with RocketMQ, you will usually see the following log: + +```java +SendResult [sendStatus=SEND_OK, msgId=0A42333A0DC818B4AAC246C290FD0000, offsetMsgId=0A42333A00002A9F000000000134F1F5, messageQueue=MessageQueue [topic=topicTest1, BrokerName=mac.local, queueId=3], queueOffset=4] +``` + +- msgId,for the client, the msgId is generated by the producer instance. Specifically, the method `MessageClientIDSetter.createUniqIDBuffer()` is called to generate a unique Id. +- offsetMsgId, offsetMsgId is generated by the Broker server when writing a message ( string concating "IP address + port" and "CommitLog's physical offset address"), and offsetMsgId is the messageId used to query in the RocketMQ console. diff --git a/docs/en/acl/Operations_ACL.md b/docs/en/acl/Operations_ACL.md new file mode 100644 index 00000000000..b29f63e3d97 --- /dev/null +++ b/docs/en/acl/Operations_ACL.md @@ -0,0 +1,76 @@ +# Access control list +## Overview +This document focuses on how to quickly deploy and use a RocketMQ cluster that supports the privilege control feature. + +## 1. Access control features +Access Control (ACL) mainly provides Topic resource level user access control for RocketMQ.If you want to enable RocketMQ permission control, you can inject the AccessKey and SecretKey signatures through the RPCHook on the Client side.And then, the corresponding permission control attributes (including Topic access rights, IP whitelist and AccessKey and SecretKey signature) are set in the configuration file of distribution/conf/plain_acl.yml.The Broker side will check the permissions owned by the AccessKey, and if the verification fails, an exception is thrown; +The source code about ACL on the Client side can be find in **org.apache.rocketmq.example.simple.AclClient.java** + +## 2. Access control definition and attribute values +### 2.1 Access control definition +The definition of Topic resource access control for RocketMQ is mainly as shown in the following table. + +| Permission | explanation | +| --- | --- | +| DENY | permission deny | +| ANY | PUB or SUB permission | +| PUB | Publishing permission | +| SUB | Subscription permission | + +### 2.2 Main properties +| key | value | explanation | +| --- | --- | --- | +| globalWhiteRemoteAddresses | string |Global IP whitelist,example:
\*;
192.168.\*.\*;
192.168.0.1 | +| accessKey | string | Access Key | +| secretKey | string | Secret Key | +| whiteRemoteAddress | string | User IP whitelist,example:
\*;
192.168.\*.\*;
192.168.0.1 | +| admin | true;false | Whether an administrator account | +| defaultTopicPerm | DENY;PUB;SUB;PUB\|SUB | Default Topic permission | +| defaultGroupPerm | DENY;PUB;SUB;PUB\|SUB | Default ConsumerGroup permission | +| topicPerms | topic=permission | Topic only permission | +| groupPerms | group=permission | ConsumerGroup only permission | + +For details, please refer to the **distribution/conf/plain_acl.yml** configuration file. + +## 3. Cluster deployment with permission control +After defining the permission attribute in the **distribution/conf/plain_acl.yml** configuration file as described above, open the **aclEnable** switch variable to enable the ACL feature of the RocketMQ cluster.The configuration file of the ACL feature enabled on the broker is as follows: +```properties +brokerClusterName=DefaultCluster +brokerName=broker-a +brokerId=0 +deleteWhen=04 +fileReservedTime=48 +brokerRole=ASYNC_MASTER +flushDiskType=ASYNC_FLUSH +storePathRootDir=/data/rocketmq/rootdir-a-m +storePathCommitLog=/data/rocketmq/commitlog-a-m +autoCreateSubscriptionGroup=true +## if acl is open,the flag will be true +aclEnable=true +listenPort=10911 +brokerIP1=XX.XX.XX.XX1 +namesrvAddr=XX.XX.XX.XX:9876 +``` +## 4. Main process of access control +The main ACL process is divided into two parts, including privilege resolution and privilege check. + +### 4.1 Privilege resolution +The Broker side parses the client's RequestCommand request and obtains the attribute field that needs to be authenticated. +main attributes: + (1) AccessKey:Similar to the user name, on behalf of the user entity, the permission data corresponds to it; + (2) Signature:The client obtains the string according to the signature of the SecretKey, and the server uses the SecretKey to perform signature verification. + +### 4.2 Privilege check +The check logic of the right side of the broker is mainly divided into the following steps: + (1) Check if the global IP whitelist is hit; if yes, the check passes; otherwise, go to step (2); + (2) Check if the user IP whitelist is hit; if yes, the check passes; otherwise, go to step (3); + (3) Check the signature, if the verification fails, throw an exception; if the verification passes, go to step (4); + (4) Check the permissions required by the user request and the permissions owned by the user; if not, throw an exception; + + +The verification of the required permissions of the user requires attention to the following points: + (1) Special requests such as UPDATE_AND_CREATE_TOPIC can only be operated by the admin account; + (2) For a resource, if there is explicit configuration permission, the configured permission is used; if there is no explicit configuration permission, the default permission is adopted; + +## 5. Hot loading modified Access control +The default implementation of RocketrMQ's permission control store is based on the yml configuration file. Users can dynamically modify the properties defined by the permission control without restarting the Broker service node. diff --git a/docs/en/architecture.md b/docs/en/architecture.md new file mode 100644 index 00000000000..0e362663e88 --- /dev/null +++ b/docs/en/architecture.md @@ -0,0 +1,46 @@ +# Architecture design + +## Technology Architecture +![](image/rocketmq_architecture_1.png) + +The RocketMQ architecture is divided into four parts, as shown in the figure above: + + +- Producer:The role of message publishing supports distributed cluster mode deployment. Producer selects the corresponding Broker cluster queue for message delivery through MQ's load balancing module. The delivery process supports fast failure and low latency. + +- Consumer:The role of message consumption supports distributed cluster deployment. Support push, pull two modes to consume messages. It also supports cluster mode and broadcast mode consumption, and it provides a real-time message subscription mechanism to meet the needs of most users. + +- NameServer:NameServer is a very simple Topic routing registry with a role similar to ZooKeeper in Dubbo, which supports dynamic registration and discovery of Broker. It mainly includes two functions: Broker management, NameServer accepts the registration information of the Broker cluster and saves it as the basic data of the routing information. Then provide a heartbeat detection mechanism to check whether the broker is still alive; routing information management, each NameServer will save the entire routing information about the Broker cluster and the queue information for the client query. Then the Producer and Conumser can know the routing information of the entire Broker cluster through the NameServer, so as to deliver and consume the message. The NameServer is usually deployed in a cluster mode, and each instance does not communicate with each other. Broker registers its own routing information with each NameServer, so each NameServer instance stores a complete routing information. When a NameServer is offline for some reason, the Broker can still synchronize its routing information with other NameServers. The Producer and Consumer can still dynamically sense the information of the Broker's routing. + +- BrokerServer:Broker is responsible for the storage, delivery and query of messages and high availability guarantees. In order to achieve these functions, Broker includes the following important sub-modules. +1. Remoting Module:The entire broker entity handles requests from the clients side. +2. Client Manager:Topic subscription information for managing the client (Producer/Consumer) and maintaining the Consumer +3. Store Service:Provides a convenient and simple API interface for handling message storage to physical hard disks and query functions. +4. HA Service:Highly available service that provides data synchronization between Master Broker and Slave Broker. +5. Index Service:The message delivered to the Broker is indexed according to a specific Message key to provide a quick query of the message. + +![](image/rocketmq_architecture_2.png) + +## Deployment architecture + + +![](image/rocketmq_architecture_3.png) + + +### RocketMQ Network deployment features + +- NameServer is an almost stateless node that can be deployed in a cluster without any information synchronization between nodes. + +- The broker deployment is relatively complex. The Broker is divided into the Master and the Slave. One Master can correspond to multiple Slaves. However, one Slave can only correspond to one Master. The correspondence between the Master and the Slave is defined by specifying the same BrokerName and different BrokerId. The BrokerId is 0. Indicates Master, non-zero means Slave. The Master can also deploy multiple. Each broker establishes a long connection with all nodes in the NameServer cluster, and periodically registers Topic information to all NameServers. Note: The current RocketMQ version supports a Master Multi Slave on the deployment architecture, but only the slave server with BrokerId=1 will participate in the read load of the message. + +- The Producer establishes a long connection with one of the nodes in the NameServer cluster (randomly selected), periodically obtains Topic routing information from the NameServer, and establishes a long connection to the Master that provides the Topic service, and periodically sends a heartbeat to the Master. Producer is completely stateless and can be deployed in a cluster. + +- The Consumer establishes a long connection with one of the nodes in the NameServer cluster (randomly selected), periodically obtains Topic routing information from the NameServer, and establishes a long connection to the Master and Slave that provides the Topic service, and periodically sends heartbeats to the Master and Slave. The Consumer can subscribe to the message from the Master or subscribe to the message from the Slave. When the consumer pulls the message to the Master, the Master server will generate a read according to the distance between the offset and the maximum offset. I/O), and whether the server is readable or not, the next time it is recommended to pull from the Master or Slave. + +Describe the cluster workflow in conjunction with the deployment architecture diagram: + +- Start the NameServer, listen to the port after the NameServer, and wait for the Broker, Producer, and Consumer to connect, which is equivalent to a routing control center. +- The Broker starts, keeps a long connection with all NameServers, and sends heartbeat packets periodically. The heartbeat packet contains the current broker information (IP+ port, etc.) and stores all Topic information. After the registration is successful, there is a mapping relationship between Topic and Broker in the NameServer cluster. +- Before sending and receiving a message, create a Topic. When creating a Topic, you need to specify which Brokers the Topic should be stored on, or you can automatically create a Topic when sending a message. +- Producer sends a message. When starting, it first establishes a long connection with one of the NameServer clusters, and obtains from the NameServer which Brokers are currently sent by the Topic. Polling selects a queue from the queue list and then establishes with the broker where the queue is located. Long connection to send a message to the broker. +- The Consumer is similar to the Producer. It establishes a long connection with one of the NameServers, obtains which Brokers the current Topic exists on, and then directly establishes a connection channel with the Broker to start consuming messages. diff --git a/docs/en/best_practice.md b/docs/en/best_practice.md new file mode 100755 index 00000000000..0ee7387ee4b --- /dev/null +++ b/docs/en/best_practice.md @@ -0,0 +1,29 @@ +# Best practices + +## 1 Producer + +## 2 Consumer + +## 3 Broker + +### 3.1 Broker Role + +### 3.2 FlushDiskType + +### 3.3 Broker Configuration +| Parameter name | Default | Description | +| -------------------------------- | ----------------------------- | ------------------------------------------------------------ | +| listenPort | 10911 | listen port for client | +| namesrvAddr | null | name server address | +| brokerIP1 | InetAddress for network interface | Should be configured if having multiple addresses | +| brokerIP2 | InetAddress for network interface | If configured for the Master broker in the Master/Slave cluster, slave broker will connect to this port for data synchronization | +| brokerName | null | broker name | +| brokerClusterName | DefaultCluster | this broker belongs to which cluster | +| brokerId | 0 | broker id, 0 means master, positive integers mean slave | +| storePathCommitLog | $HOME/store/commitlog/ | file path for commit log | +| storePathConsumerQueue | $HOME/store/consumequeue/ | file path for consume queue | +| mapedFileSizeCommitLog | 1024 * 1024 * 1024(1G) | mapped file size for commit log |​ +| deleteWhen | 04 | When to delete the commitlog which is out of the reserve time |​ +| fileReserverdTime | 72 | The number of hours to keep a commitlog before deleting it |​ +| brokerRole | ASYNC_MASTER | SYNC_MASTER/ASYNC_MASTER/SLAVE |​ +| flushDiskType | ASYNC_FLUSH | {SYNC_FLUSH/ASYNC_FLUSH}. Broker of SYNC_FLUSH mode flushes each message onto disk before acknowledging producer. Broker of ASYNC_FLUSH mode, on the other hand, takes advantage of group-committing, achieving better performance. |​ \ No newline at end of file diff --git a/docs/en/design.md b/docs/en/design.md new file mode 100644 index 00000000000..4bd788aa99e --- /dev/null +++ b/docs/en/design.md @@ -0,0 +1,110 @@ + +## Design +### 1 Message Store + +![](../cn/image/rocketmq_design_1.png) + + +#### 1.1 The Architecure of Message Store + +#### 1.2 PageCache and Memory-Map(Mmap) + +#### 1.3 Message Flush + +![](../cn/image/rocketmq_design_2.png) + + +### 2 Communication Mechanism + +#### 2.1 The class diagram of Remoting module + +![](../cn/image/rocketmq_design_3.png) + +#### 2.2 The design of protocol and encode/decode + +![](../cn/image/rocketmq_design_4.png) + + +#### 2.3 The three ways and process of message communication + +![](../cn/image/rocketmq_design_5.png) + +#### 2.4 The multi-thread design of Reactor + +![](../cn/image/rocketmq_design_6.png) + + +### 3 Message Filter + +![](../cn/image/rocketmq_design_7.png) + +### 4 LoadBalancing + +#### 4.1 The loadBalance of Producer + +#### 4.2 The loadBalance of Consumer + +![](../cn/image/rocketmq_design_8.png) + + +![](../cn/image/rocketmq_design_9.png) + + + +### 5 Transactional Message +Apache RocketMQ supports distributed transactional message from version 4.3.0. RocketMQ implements transactional message by using the protocol of 2PC(two-phase commit), in addition adding a compensation logic to handle timeout-case or failure-case of commit-phase, as shown below. + +![](../cn/image/rocketmq_design_10.png) + +#### 5.1 The Process of RocketMQ Transactional Message +The picture above shows the overall architecture of transactional message, including the sending of message(commit-request phase), the sending of commit/rollback(commit phase) and the compensation process. + +1. The sending of message and Commit/Rollback. + (1) Sending the message(named Half message in RocketMQ) + (2) The server responds the writing result(success or failure) of Half message. + (3) Handle local transaction according to the result(local transaction won't be executed when the result is failure). + (4) Sending Commit/Rollback to broker according to the result of local transaction(Commit will generate message index and make the message visible to consumers). + +2. Compensation process + (1) For a transactional message without a Commit/Rollback (means the message in the pending status), a "back-check" request is initiated from the broker. + (2) The Producer receives the "back-check" request and checks the status of the local transaction corresponding to the "back-check" message. + (3) Redo Commit or Rollback based on local transaction status. +The compensation phase is used to resolve the timeout or failure case of the message Commit or Rollback. + +#### 5.2 The design of RocketMQ Transactional Message +1. Transactional message is invisible to users in first phase(commit-request phase) + + Upon on the main process of transactional message, the message of first phase is invisible to the user. This is also the biggest difference from normal message. So how do we write the message while making it invisible to the user? And below is the solution of RocketMQ: if the message is a Half message, the topic and queueId of the original message will be backed up, and then changes the topic to RMQ_SYS_TRANS_HALF_TOPIC. Since the consumer group does not subscribe to the topic, the consumer cannot consume the Half message. Then RocketMQ starts a timing task, pulls the message for RMQ_SYS_TRANS_HALF_TOPIC, obtains a channel according to producer group and sends a back-check to query local transaction status, and decide whether to submit or roll back the message according to the status. + + In RocketMQ, the storage structure of the message in the broker is as follows. Each message has corresponding index information. The Consumer reads the content of the message through the secondary index of the ConsumeQueue. The flow is as follows: + +![](../cn/image/rocketmq_design_11.png) + + The specific implementation strategy of RocketMQ is: if the transactional message is written, topic and queueId of the message are replaced, and the original topic and queueId are stored in the properties of the message. Because the replace of the topic, the message will not be forwarded to the Consumer Queue of the original topic, and the consumer cannot perceive the existence of the message and will not consume it. In fact, changing the topic is the conventional method of RocketMQ(just recall the implementation mechanism of the delay message). + +2. Commit/Rollback operation and introduction of Op message + + After finishing writing a message that is invisible to the user in the first phase, here comes two cases in the second phase. One is Commit operation, after which the message needs to be visible to the user; the other one is Rollback operation, after which the first phase message(Half message) needs to be revoked. For the case of Rollback, since first-phase message itself is invisible to the user, there is no need to actually revoke the message (in fact, RocketMQ can't actually delete a message because it is a sequential-write file). But still some operation needs to be done to identity the final status of the message, to differ it from pending status message. To do this, the concept of "Op message" is introduced, which means the message has a certain status(Commit or Rollback). If a transactional message does not have a corresponding Op message, the status of the transaction is still undetermined (probably the second-phase failed). By introducing the Op message, the RocketMQ records an Op message for every Half message regardless it is Commit or Rollback. The only difference between Commit and Rollback is that when it comes to Commit, the index of the Half message is created before the Op message is written. + +3. How Op message stored and the correspondence between Op message and Half message + + RocketMQ writes the Op message to a specific system topic(RMQ_SYS_TRANS_OP_HALF_TOPIC) which will be created via the method - TransactionalMessageUtil.buildOpTopic(); this topic is an internal Topic (like the topic of RMQ_SYS_TRANS_HALF_TOPIC) and will not be consumed by the user. The content of the Op message is the physical offset of the corresponding Half message. Through the Op message we can index to the Half message for subsequent check-back operation. + +![](../cn/image/rocketmq_design_12.png) + +4. Index construction of Half messages + + When performing Commit operation of the second phase, the index of the Half message needs to be built. Since the Half message is written to a special topic(RMQ_SYS_TRANS_HALF_TOPIC) in the first phase of 2PC, so it needs to be read out from the special topic when building index, and replace the topic and queueId with the real target topic and queueId, and then write through a normal message that is visible to the user. Therefore, in conclusion, the second phase recovers a complete normal message using the content of the Half message stored in the first phase, and then goes through the message-writing process. + +5. How to handle the message failed in the second phase? + + If commit/rollback phase fails, for example, a network problem causes the Commit to fail when you do Commit. Then certain strategy is required to make sure the message finally commit. RocketMQ uses a compensation mechanism called "back-check". The broker initiates a back-check request for the message in pending status, and sends the request to the corresponding producer side (the same producer group as the producer group who sent the Half message). The producer checks the status of local transaction and redo Commit or Rollback. The broker performs the back-check by comparing the RMQ_SYS_TRANS_HALF_TOPIC messages and the RMQ_SYS_TRANS_OP_HALF_TOPIC messages and advances the checkpoint(recording those transactional messages that the status are certain). + + RocketMQ does not back-check the status of transactional messages endlessly. The default time is 15. If the transaction status is still unknown after 15 times, RocketMQ will roll back the message by default. +### 6 Message Query + +#### 6.1 Query messages by messageId + +#### 6.2 Query messages by message key + +![](../cn/image/rocketmq_design_13.png) diff --git a/docs/en/image/rocketmq_architecture_1.png b/docs/en/image/rocketmq_architecture_1.png new file mode 100644 index 00000000000..548c8752a84 Binary files /dev/null and b/docs/en/image/rocketmq_architecture_1.png differ diff --git a/docs/en/image/rocketmq_architecture_2.png b/docs/en/image/rocketmq_architecture_2.png new file mode 100644 index 00000000000..b2ab8d34c6c Binary files /dev/null and b/docs/en/image/rocketmq_architecture_2.png differ diff --git a/docs/en/image/rocketmq_architecture_3.png b/docs/en/image/rocketmq_architecture_3.png new file mode 100644 index 00000000000..aa74ed7ac15 Binary files /dev/null and b/docs/en/image/rocketmq_architecture_3.png differ diff --git a/docs/en/images/rocketmq_design_7.png b/docs/en/images/rocketmq_design_7.png new file mode 100644 index 00000000000..b0faa86c29c Binary files /dev/null and b/docs/en/images/rocketmq_design_7.png differ diff --git a/docs/en/images/rocketmq_design_message_query.png b/docs/en/images/rocketmq_design_message_query.png new file mode 100644 index 00000000000..f5ca945cafc Binary files /dev/null and b/docs/en/images/rocketmq_design_message_query.png differ diff --git a/docs/en/images/rocketmq_storage_arch.png b/docs/en/images/rocketmq_storage_arch.png new file mode 100644 index 00000000000..8c719e115a1 Binary files /dev/null and b/docs/en/images/rocketmq_storage_arch.png differ diff --git a/docs/en/images/rocketmq_storage_flush.png b/docs/en/images/rocketmq_storage_flush.png new file mode 100644 index 00000000000..1610ae0d934 Binary files /dev/null and b/docs/en/images/rocketmq_storage_flush.png differ diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterConsumer.java similarity index 74% rename from example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java rename to example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterConsumer.java index c41c9c14c3c..8dd6d20eb79 100644 --- a/example/src/main/java/org/apache/rocketmq/example/filter/SqlConsumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterConsumer.java @@ -17,28 +17,24 @@ package org.apache.rocketmq.example.filter; +import java.util.List; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.MessageSelector; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; -import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.message.MessageExt; -import java.util.List; +public class SqlFilterConsumer { + + public static void main(String[] args) throws Exception { -public class SqlConsumer { + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name"); - public static void main(String[] args) { - DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name_4"); - try { - consumer.subscribe("TopicTest", - MessageSelector.bySql("(TAGS is not null and TAGS in ('TagA', 'TagB'))" + - "and (a is not null and a between 0 3)")); - } catch (MQClientException e) { - e.printStackTrace(); - return; - } + // Don't forget to set enablePropertyFilter=true in broker + consumer.subscribe("SqlFilterTest", + MessageSelector.bySql("(TAGS is not null and TAGS in ('TagA', 'TagB'))" + + "and (a is not null and a between 0 and 3)")); consumer.registerMessageListener(new MessageListenerConcurrently() { @@ -50,12 +46,7 @@ public ConsumeConcurrentlyStatus consumeMessage(List msgs, } }); - try { - consumer.start(); - } catch (MQClientException e) { - e.printStackTrace(); - return; - } + consumer.start(); System.out.printf("Consumer Started.%n"); } } diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/Producer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterProducer.java similarity index 59% rename from example/src/main/java/org/apache/rocketmq/example/filter/Producer.java rename to example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterProducer.java index 2a0da6546c8..001827068a0 100644 --- a/example/src/main/java/org/apache/rocketmq/example/filter/Producer.java +++ b/example/src/main/java/org/apache/rocketmq/example/filter/SqlFilterProducer.java @@ -14,33 +14,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.rocketmq.example.filter; -import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.remoting.common.RemotingHelper; -public class Producer { - public static void main(String[] args) throws MQClientException, InterruptedException { - DefaultMQProducer producer = new DefaultMQProducer("ProducerGroupName"); +public class SqlFilterProducer { + + public static void main(String[] args) throws Exception { + + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + producer.start(); - try { - for (int i = 0; i < 6000000; i++) { - Message msg = new Message("TopicFilter7", - "TagA", - "OrderID001", - "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); - - msg.putUserProperty("SequenceId", String.valueOf(i)); - SendResult sendResult = producer.send(msg); - System.out.printf("%s%n", sendResult); - } - } catch (Exception e) { - e.printStackTrace(); + String[] tags = new String[] {"TagA", "TagB", "TagC"}; + + for (int i = 0; i < 10; i++) { + Message msg = new Message("SqlFilterTest", + tags[i % tags.length], + ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) + ); + msg.putUserProperty("a", String.valueOf(i)); + + SendResult sendResult = producer.send(msg); + System.out.printf("%s%n", sendResult); } + producer.shutdown(); } } diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java b/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java deleted file mode 100644 index 3f3a0e65215..00000000000 --- a/example/src/main/java/org/apache/rocketmq/example/filter/SqlProducer.java +++ /dev/null @@ -1,67 +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.example.filter; - -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.client.producer.DefaultMQProducer; -import org.apache.rocketmq.client.producer.SendResult; -import org.apache.rocketmq.common.message.Message; -import org.apache.rocketmq.remoting.common.RemotingHelper; - -public class SqlProducer { - - public static void main(String[] args) { - DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); - try { - producer.start(); - } catch (MQClientException e) { - e.printStackTrace(); - return; - } - - for (int i = 0; i < 10; i++) { - try { - String tag; - int div = i % 3; - if (div == 0) { - tag = "TagA"; - } else if (div == 1) { - tag = "TagB"; - } else { - tag = "TagC"; - } - Message msg = new Message("TopicTest", - tag, - ("Hello RocketMQ " + i).getBytes(RemotingHelper.DEFAULT_CHARSET) - ); - msg.putUserProperty("a", String.valueOf(i)); - - SendResult sendResult = producer.send(msg); - System.out.printf("%s%n", sendResult); - } catch (Exception e) { - e.printStackTrace(); - try { - Thread.sleep(1000); - } catch (InterruptedException e1) { - e1.printStackTrace(); - } - } - } - producer.shutdown(); - } -} diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/Consumer.java b/example/src/main/java/org/apache/rocketmq/example/filter/TagFilterConsumer.java similarity index 81% rename from example/src/main/java/org/apache/rocketmq/example/filter/Consumer.java rename to example/src/main/java/org/apache/rocketmq/example/filter/TagFilterConsumer.java index bb491ac40af..ba3723cdfef 100644 --- a/example/src/main/java/org/apache/rocketmq/example/filter/Consumer.java +++ b/example/src/main/java/org/apache/rocketmq/example/filter/TagFilterConsumer.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.example.filter; -import java.io.File; import java.io.IOException; import java.util.List; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; @@ -24,20 +23,15 @@ import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.message.MessageExt; -public class Consumer { +public class TagFilterConsumer { public static void main(String[] args) throws InterruptedException, MQClientException, IOException { - DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("ConsumerGroupNamecc4"); - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - File classFile = new File(classLoader.getResource("MessageFilterImpl.java").getFile()); + DefaultMQPushConsumer consumer = new DefaultMQPushConsumer("please_rename_unique_group_name"); - String filterCode = MixAll.file2String(classFile); - consumer.subscribe("TopicTest", "org.apache.rocketmq.example.filter.MessageFilterImpl", - filterCode); + consumer.subscribe("TagFilterTest", "TagA || TagC"); consumer.registerMessageListener(new MessageListenerConcurrently() { diff --git a/example/src/main/java/org/apache/rocketmq/example/filter/TagFilterProducer.java b/example/src/main/java/org/apache/rocketmq/example/filter/TagFilterProducer.java new file mode 100644 index 00000000000..b0a9e2dd023 --- /dev/null +++ b/example/src/main/java/org/apache/rocketmq/example/filter/TagFilterProducer.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.example.filter; + +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.remoting.common.RemotingHelper; + +public class TagFilterProducer { + + public static void main(String[] args) throws Exception { + + DefaultMQProducer producer = new DefaultMQProducer("please_rename_unique_group_name"); + producer.start(); + + String[] tags = new String[] {"TagA", "TagB", "TagC"}; + + for (int i = 0; i < 60; i++) { + Message msg = new Message("TagFilterTest", + tags[i % tags.length], + "Hello world".getBytes(RemotingHelper.DEFAULT_CHARSET)); + + SendResult sendResult = producer.send(msg); + System.out.printf("%s%n", sendResult); + } + + producer.shutdown(); + } +} 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 00962ef246e..ecd057a29ac 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 @@ -128,6 +128,17 @@ public RegisterBrokerResult registerBroker( brokerData = new BrokerData(clusterName, brokerName, new HashMap()); this.brokerAddrTable.put(brokerName, brokerData); } + Map brokerAddrsMap = brokerData.getBrokerAddrs(); + //Switch slave to master: first remove <1, IP:PORT> in namesrv, then add <0, IP:PORT> + //The same IP:PORT must only have one record in brokerAddrTable + Iterator> it = brokerAddrsMap.entrySet().iterator(); + while (it.hasNext()) { + Entry item = it.next(); + if (null != brokerAddr && brokerAddr.equals(item.getValue()) && brokerId != item.getKey()) { + it.remove(); + } + } + String oldAddr = brokerData.getBrokerAddrs().put(brokerId, brokerAddr); registerFirst = registerFirst || (null == oldAddr); diff --git a/pom.xml b/pom.xml index ec8949cf5ae..7a349369c0e 100644 --- a/pom.xml +++ b/pom.xml @@ -100,8 +100,8 @@ false true - 1.7 - 1.7 + 1.8 + 1.8 jacoco ${project.basedir}/../test/target/jacoco-it.exec @@ -259,6 +259,7 @@ */target/** */*.iml docs/** + localbin/** diff --git a/store/pom.xml b/store/pom.xml index 727a198c887..a1aaf39a419 100644 --- a/store/pom.xml +++ b/store/pom.xml @@ -28,6 +28,21 @@ rocketmq-store ${project.version} + + io.openmessaging.storage + dledger + 0.1 + + + org.apache.rocketmq + rocketmq-remoting + + + org.slf4j + slf4j-log4j12 + + + ${project.groupId} rocketmq-common 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 dbb27cd744c..5e86b8a85fc 100644 --- a/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java +++ b/store/src/main/java/org/apache/rocketmq/store/AllocateMappedFileService.java @@ -120,16 +120,9 @@ public String getServiceName() { return AllocateMappedFileService.class.getSimpleName(); } + @Override public void shutdown() { - this.stopped = true; - this.thread.interrupt(); - - try { - this.thread.join(this.getJointime()); - } catch (InterruptedException e) { - log.error("Interrupted", e); - } - + super.shutdown(true); for (AllocateRequest req : this.requestTable.values()) { if (req.mappedFile != null) { log.info("delete pre allocated maped file, {}", req.mappedFile.getFileName()); 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 03b1151643f..cbcc1a7b728 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -45,11 +45,11 @@ public class CommitLog { // Message's MAGIC CODE daa320a7 public final static int MESSAGE_MAGIC_CODE = -626843481; - private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + protected static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); // End of file empty MAGIC CODE cbd43194 - private final static int BLANK_MAGIC_CODE = -875286124; - private final MappedFileQueue mappedFileQueue; - private final DefaultMessageStore defaultMessageStore; + protected final static int BLANK_MAGIC_CODE = -875286124; + protected final MappedFileQueue mappedFileQueue; + protected final DefaultMessageStore defaultMessageStore; private final FlushCommitLogService flushCommitLogService; //If TransientStorePool enabled, we must flush message to FileChannel at fixed periods @@ -57,11 +57,11 @@ public class CommitLog { private final AppendMessageCallback appendMessageCallback; private final ThreadLocal batchEncoderThreadLocal; - private HashMap topicQueueTable = new HashMap(1024); - private volatile long confirmOffset = -1L; + protected HashMap topicQueueTable = new HashMap(1024); + protected volatile long confirmOffset = -1L; private volatile long beginTimeInLock = 0; - private final PutMessageLock putMessageLock; + protected final PutMessageLock putMessageLock; public CommitLog(final DefaultMessageStore defaultMessageStore) { this.mappedFileQueue = new MappedFileQueue(defaultMessageStore.getMessageStoreConfig().getStorePathCommitLog(), @@ -212,6 +212,12 @@ else if (!dispatchRequest.isSuccess()) { log.warn("maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, processOffset); this.defaultMessageStore.truncateDirtyLogicFiles(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(); } } @@ -366,7 +372,7 @@ public DispatchRequest checkMessageAndReturnSize(java.nio.ByteBuffer byteBuffer, return new DispatchRequest(-1, false /* success */); } - private static int calMsgLength(int bodyLength, int topicLength, int propertiesLength) { + protected static int calMsgLength(int bodyLength, int topicLength, int propertiesLength) { final int msgLen = 4 //TOTALSIZE + 4 //MAGICCODE + 4 //BODYCRC @@ -396,6 +402,7 @@ public void setConfirmOffset(long phyOffset) { this.confirmOffset = phyOffset; } + @Deprecated public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) { // recover by the minimum time stamp boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); @@ -456,7 +463,7 @@ else if (size == 0) { } } } else { - log.info("recover physics file end, " + mappedFile.getFileName()); + log.info("recover physics file end, " + mappedFile.getFileName() + " pos=" + byteBuffer.position()); break; } } @@ -474,6 +481,7 @@ else if (size == 0) { } // 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(); 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 08c7f999069..a63d3746b30 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/ConsumeQueue.java @@ -106,7 +106,7 @@ public void recover() { if (offset >= 0 && size > 0) { mappedFileOffset = i + CQ_STORE_UNIT_SIZE; - this.maxPhysicOffset = offset; + this.maxPhysicOffset = offset + size; if (isExtAddr(tagsCode)) { maxExtAddr = tagsCode; } @@ -224,7 +224,7 @@ public void truncateDirtyLogicFiles(long phyOffet) { int logicFileSize = this.mappedFileSize; - this.maxPhysicOffset = phyOffet - 1; + this.maxPhysicOffset = phyOffet; long maxExtAddr = 1; while (true) { MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); @@ -249,7 +249,7 @@ public void truncateDirtyLogicFiles(long phyOffet) { mappedFile.setWrotePosition(pos); mappedFile.setCommittedPosition(pos); mappedFile.setFlushedPosition(pos); - this.maxPhysicOffset = offset; + this.maxPhysicOffset = offset + size; // This maybe not take effect, when not every consume queue has extend file. if (isExtAddr(tagsCode)) { maxExtAddr = tagsCode; @@ -267,7 +267,7 @@ public void truncateDirtyLogicFiles(long phyOffet) { mappedFile.setWrotePosition(pos); mappedFile.setCommittedPosition(pos); mappedFile.setFlushedPosition(pos); - this.maxPhysicOffset = offset; + this.maxPhysicOffset = offset + size; if (isExtAddr(tagsCode)) { maxExtAddr = tagsCode; } @@ -348,7 +348,7 @@ public void correctMinOffset(long phyMinOffset) { long tagsCode = result.getByteBuffer().getLong(); if (offsetPy >= phyMinOffset) { - this.minLogicOffset = result.getMappedFile().getFileFromOffset() + i; + this.minLogicOffset = mappedFile.getFileFromOffset() + i; log.info("Compute logical min offset: {}, topic: {}, queueId: {}", this.getMinOffsetInQueue(), this.topic, this.queueId); // This maybe not take effect, when not every consume queue has extend file. @@ -420,7 +420,8 @@ public void putMessagePositionInfoWrapper(DispatchRequest request) { private boolean putMessagePositionInfo(final long offset, final int size, final long tagsCode, final long cqOffset) { - if (offset <= this.maxPhysicOffset) { + if (offset + size <= this.maxPhysicOffset) { + log.warn("Maybe try to build consume queue repeatedly maxPhysicOffset={} phyOffset={}", maxPhysicOffset, offset); return true; } @@ -464,7 +465,7 @@ private boolean putMessagePositionInfo(final long offset, final int size, final ); } } - this.maxPhysicOffset = offset; + this.maxPhysicOffset = offset + size; return mappedFile.appendMessage(this.byteBufferIndex.array()); } return false; 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 e0aef4f3745..6cec58d8616 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -42,24 +42,23 @@ import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.logging.InternalLogger; -import org.apache.rocketmq.logging.InternalLoggerFactory; 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.running.RunningStats; import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.config.StorePathConfigHelper; +import org.apache.rocketmq.store.dledger.DLedgerCommitLog; import org.apache.rocketmq.store.ha.HAService; import org.apache.rocketmq.store.index.IndexService; import org.apache.rocketmq.store.index.QueryOffsetResult; import org.apache.rocketmq.store.schedule.ScheduleMessageService; import org.apache.rocketmq.store.stats.BrokerStatsManager; -import static org.apache.rocketmq.store.config.BrokerRole.SLAVE; - public class DefaultMessageStore implements MessageStore { private static final InternalLogger log = InternalLoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); @@ -119,7 +118,11 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br this.messageStoreConfig = messageStoreConfig; this.brokerStatsManager = brokerStatsManager; this.allocateMappedFileService = new AllocateMappedFileService(this); - this.commitLog = new CommitLog(this); + if (messageStoreConfig.isEnableDLegerCommitLog()) { + this.commitLog = new DLedgerCommitLog(this); + } else { + this.commitLog = new CommitLog(this); + } this.consumeQueueTable = new ConcurrentHashMap<>(32); this.flushConsumeQueueService = new FlushConsumeQueueService(); @@ -127,8 +130,11 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br this.cleanConsumeQueueService = new CleanConsumeQueueService(); this.storeStatsService = new StoreStatsService(); this.indexService = new IndexService(this); - this.haService = new HAService(this); - + if (!messageStoreConfig.isEnableDLegerCommitLog()) { + this.haService = new HAService(this); + } else { + this.haService = null; + } this.reputMessageService = new ReputMessageService(); this.scheduleMessageService = new ScheduleMessageService(this); @@ -216,29 +222,71 @@ public void start() throws Exception { lockFile.getChannel().write(ByteBuffer.wrap("lock".getBytes())); lockFile.getChannel().force(true); - - this.flushConsumeQueueService.start(); - this.commitLog.start(); - this.storeStatsService.start(); - - if (this.scheduleMessageService != null && SLAVE != messageStoreConfig.getBrokerRole()) { - this.scheduleMessageService.start(); + { + /** + * 1. Make sure the fast-forward messages to be truncated during the recovering according to the max physical offset of the commitlog; + * 2. DLedger committedPos may be missing, so the maxPhysicalPosInLogicQueue maybe bigger that maxOffset returned by DLedgerCommitLog, just let it go; + * 3. Calculate the reput offset according to the consume queue; + * 4. Make sure the fall-behind messages to be dispatched before starting the commitlog, especially when the broker role are automatically changed. + */ + long maxPhysicalPosInLogicQueue = commitLog.getMinOffset(); + for (ConcurrentMap maps : this.consumeQueueTable.values()) { + for (ConsumeQueue logic : maps.values()) { + if (logic.getMaxPhysicOffset() > maxPhysicalPosInLogicQueue) { + maxPhysicalPosInLogicQueue = logic.getMaxPhysicOffset(); + } + } + } + if (maxPhysicalPosInLogicQueue < 0) { + maxPhysicalPosInLogicQueue = 0; + } + if (maxPhysicalPosInLogicQueue < this.commitLog.getMinOffset()) { + maxPhysicalPosInLogicQueue = this.commitLog.getMinOffset(); + /** + * This happens in following conditions: + * 1. If someone removes all the consumequeue files or the disk get damaged. + * 2. Launch a new broker, and copy the commitlog from other brokers. + * + * All the conditions has the same in common that the maxPhysicalPosInLogicQueue should be 0. + * If the maxPhysicalPosInLogicQueue is gt 0, there maybe something wrong. + */ + log.warn("[TooSmallCqOffset] maxPhysicalPosInLogicQueue={} clMinOffset={}", maxPhysicalPosInLogicQueue, this.commitLog.getMinOffset()); + } + log.info("[SetReputOffset] maxPhysicalPosInLogicQueue={} clMinOffset={} clMaxOffset={} clConfirmedOffset={}", + maxPhysicalPosInLogicQueue, this.commitLog.getMinOffset(), this.commitLog.getMaxOffset(), this.commitLog.getConfirmOffset()); + this.reputMessageService.setReputFromOffset(maxPhysicalPosInLogicQueue); + this.reputMessageService.start(); + + /** + * 1. Finish dispatching the messages fall behind, then to start other services. + * 2. DLedger committedPos may be missing, so here just require dispatchBehindBytes <= 0 + */ + while (true) { + if (dispatchBehindBytes() <= 0) { + break; + } + Thread.sleep(1000); + log.info("Try to finish doing reput the messages fall behind during the starting, reputOffset={} maxOffset={} behind={}", this.reputMessageService.getReputFromOffset(), this.getMaxPhyOffset(), this.dispatchBehindBytes()); + } + this.recoverTopicQueueTable(); } - if (this.getMessageStoreConfig().isDuplicationEnable()) { - this.reputMessageService.setReputFromOffset(this.commitLog.getConfirmOffset()); - } else { - this.reputMessageService.setReputFromOffset(this.commitLog.getMaxOffset()); + if (!messageStoreConfig.isEnableDLegerCommitLog()) { + this.haService.start(); + this.handleScheduleMessageService(messageStoreConfig.getBrokerRole()); } - this.reputMessageService.start(); - this.haService.start(); + this.flushConsumeQueueService.start(); + this.commitLog.start(); + this.storeStatsService.start(); this.createTempFile(); this.addScheduleTask(); this.shutdown = false; } + + public void shutdown() { if (!this.shutdown) { this.shutdown = true; @@ -255,8 +303,9 @@ public void shutdown() { if (this.scheduleMessageService != null) { this.scheduleMessageService.shutdown(); } - - this.haService.shutdown(); + if (this.haService != null) { + this.haService.shutdown(); + } this.storeStatsService.shutdown(); this.indexService.shutdown(); @@ -1039,6 +1088,7 @@ public boolean checkInDiskByConsumeOffset(final String topic, final int queueId, return false; } + @Override public long dispatchBehindBytes() { return this.reputMessageService.behind(); } @@ -1320,7 +1370,7 @@ private long recoverConsumeQueue() { return maxPhysicOffset; } - private void recoverTopicQueueTable() { + public void recoverTopicQueueTable() { HashMap table = new HashMap(1024); long minPhyOffset = this.commitLog.getMinOffset(); for (ConcurrentMap maps : this.consumeQueueTable.values()) { @@ -1382,6 +1432,18 @@ public BrokerStatsManager getBrokerStatsManager() { return brokerStatsManager; } + @Override + public void handleScheduleMessageService(final BrokerRole brokerRole) { + if (this.scheduleMessageService != null) { + if (brokerRole == BrokerRole.SLAVE) { + this.scheduleMessageService.shutdown(); + } else { + this.scheduleMessageService.start(); + } + } + + } + public int remainTransientStoreBufferNumbs() { return this.transientStorePool.remainBufferNumbs(); } @@ -1748,6 +1810,11 @@ private boolean isCommitLogAvailable() { } private void doReput() { + if (this.reputFromOffset < DefaultMessageStore.this.commitLog.getMinOffset()) { + log.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(); + } for (boolean doNext = true; this.isCommitLogAvailable() && doNext; ) { if (DefaultMessageStore.this.getMessageStoreConfig().isDuplicationEnable() @@ -1763,7 +1830,7 @@ private void doReput() { for (int readSize = 0; readSize < result.getSize() && doNext; ) { DispatchRequest dispatchRequest = DefaultMessageStore.this.commitLog.checkMessageAndReturnSize(result.getByteBuffer(), false, false); - int size = dispatchRequest.getMsgSize(); + int size = dispatchRequest.getBufferSize() == -1 ? dispatchRequest.getMsgSize() : dispatchRequest.getBufferSize(); if (dispatchRequest.isSuccess()) { if (size > 0) { @@ -1797,12 +1864,10 @@ private void doReput() { this.reputFromOffset += size; } else { doNext = false; - if (DefaultMessageStore.this.brokerConfig.getBrokerId() == MixAll.MASTER_ID) { - log.error("[BUG]the master dispatch message to consume queue error, COMMITLOG OFFSET: {}", - this.reputFromOffset); + log.error("[BUG]dispatch message to consume queue error, COMMITLOG OFFSET: {}", + this.reputFromOffset); - this.reputFromOffset += result.getSize() - readSize; - } + this.reputFromOffset += result.getSize() - readSize; } } } diff --git a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java index 819bb948c75..89d47ced5ba 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java +++ b/store/src/main/java/org/apache/rocketmq/store/DispatchRequest.java @@ -22,7 +22,7 @@ public class DispatchRequest { private final String topic; private final int queueId; private final long commitLogOffset; - private final int msgSize; + private int msgSize; private final long tagsCode; private final long storeTimestamp; private final long consumeQueueOffset; @@ -35,6 +35,8 @@ public class DispatchRequest { private final Map propertiesMap; private byte[] bitMap; + private int bufferSize = -1;//the buffer size maybe larger than the msg size if the message is wrapped by something + public DispatchRequest( final String topic, final int queueId, @@ -156,4 +158,16 @@ public byte[] getBitMap() { public void setBitMap(byte[] bitMap) { this.bitMap = bitMap; } + + public void setMsgSize(int msgSize) { + this.msgSize = msgSize; + } + + public int getBufferSize() { + return bufferSize; + } + + public void setBufferSize(int bufferSize) { + this.bufferSize = bufferSize; + } } 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 86de3d2c1de..cc145921cef 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -154,8 +154,8 @@ public boolean load() { if (file.length() != this.mappedFileSize) { log.warn(file + "\t" + file.length() - + " length not matched message store config value, ignore it"); - return true; + + " length not matched message store config value, please check it manually"); + return false; } try { 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 0f9b4f0ae6e..5a046ff181f 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -21,6 +21,7 @@ import java.util.Set; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBatch; +import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.stats.BrokerStatsManager; /** @@ -366,4 +367,10 @@ QueryMessageResult queryMessage(final String topic, final String key, final int * @return BrokerStatsManager. */ BrokerStatsManager getBrokerStatsManager(); + + /** + * handle + * @param brokerRole + */ + void handleScheduleMessageService(BrokerRole brokerRole); } diff --git a/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java b/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java index 7a17114c8f3..1f826fedd89 100644 --- a/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java +++ b/store/src/main/java/org/apache/rocketmq/store/SelectMappedBufferResult.java @@ -48,9 +48,9 @@ public void setSize(final int s) { this.byteBuffer.limit(this.size); } - public MappedFile getMappedFile() { + /* public MappedFile getMappedFile() { return mappedFile; - } + }*/ // @Override // protected void finalize() { 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 8d60321ed8f..cb17345c3a1 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 @@ -143,6 +143,11 @@ public class MessageStoreConfig { private int transientStorePoolSize = 5; private boolean fastFailIfNoBufferInStorePool = false; + private boolean enableDLegerCommitLog = false; + private String dLegerGroup; + private String dLegerPeers; + private String dLegerSelfId; + public boolean isDebugLockEnable() { return debugLockEnable; } @@ -666,4 +671,35 @@ public void setCommitCommitLogThoroughInterval(final int commitCommitLogThorough this.commitCommitLogThoroughInterval = commitCommitLogThoroughInterval; } + public String getdLegerGroup() { + return dLegerGroup; + } + + public void setdLegerGroup(String dLegerGroup) { + this.dLegerGroup = dLegerGroup; + } + + public String getdLegerPeers() { + return dLegerPeers; + } + + public void setdLegerPeers(String dLegerPeers) { + this.dLegerPeers = dLegerPeers; + } + + public String getdLegerSelfId() { + return dLegerSelfId; + } + + public void setdLegerSelfId(String dLegerSelfId) { + this.dLegerSelfId = dLegerSelfId; + } + + public boolean isEnableDLegerCommitLog() { + return enableDLegerCommitLog; + } + + public void setEnableDLegerCommitLog(boolean enableDLegerCommitLog) { + this.enableDLegerCommitLog = enableDLegerCommitLog; + } } 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 new file mode 100644 index 00000000000..a48c1bed8ba --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/dledger/DLedgerCommitLog.java @@ -0,0 +1,747 @@ +/* + * 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.dledger; + +import io.openmessaging.storage.dledger.AppendFuture; +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.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.nio.ByteBuffer; +import java.util.HashMap; +import java.util.concurrent.TimeUnit; +import org.apache.rocketmq.common.UtilAll; +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.MessageExtBatch; +import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.store.AppendMessageResult; +import org.apache.rocketmq.store.AppendMessageStatus; +import org.apache.rocketmq.store.CommitLog; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.MappedFile; +import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.StoreStatsService; +import org.apache.rocketmq.store.schedule.ScheduleMessageService; + +/** + * Store all metadata downtime for recovery, data protection reliability + */ +public class DLedgerCommitLog extends CommitLog { + private final DLedgerServer dLedgerServer; + private final DLedgerConfig dLedgerConfig; + private final DLedgerMmapFileStore dLedgerFileStore; + private final MmapFileList dLedgerFileList; + + //The id identifies the broker role, 0 means master, others means slave + private final int id; + + private final MessageSerializer messageSerializer; + private volatile long beginTimeInDledgerLock = 0; + + //This offset separate the old commitlog from dledger commitlog + private long dividedCommitlogOffset = -1; + + + private boolean isInrecoveringOldCommitlog = false; + + public DLedgerCommitLog(final DefaultMessageStore defaultMessageStore) { + super(defaultMessageStore); + dLedgerConfig = new DLedgerConfig(); + dLedgerConfig.setEnableDiskForceClean(defaultMessageStore.getMessageStoreConfig().isCleanFileForciblyEnable()); + dLedgerConfig.setStoreType(DLedgerConfig.FILE); + dLedgerConfig.setSelfId(defaultMessageStore.getMessageStoreConfig().getdLegerSelfId()); + dLedgerConfig.setGroup(defaultMessageStore.getMessageStoreConfig().getdLegerGroup()); + dLedgerConfig.setPeers(defaultMessageStore.getMessageStoreConfig().getdLegerPeers()); + dLedgerConfig.setStoreBaseDir(defaultMessageStore.getMessageStoreConfig().getStorePathRootDir()); + dLedgerConfig.setMappedFileSizeForEntryData(defaultMessageStore.getMessageStoreConfig().getMapedFileSizeCommitLog()); + dLedgerConfig.setDeleteWhen(defaultMessageStore.getMessageStoreConfig().getDeleteWhen()); + dLedgerConfig.setFileReservedHours(defaultMessageStore.getMessageStoreConfig().getFileReservedTime() + 1); + id = Integer.valueOf(dLedgerConfig.getSelfId().substring(1)) + 1; + dLedgerServer = new DLedgerServer(dLedgerConfig); + dLedgerFileStore = (DLedgerMmapFileStore) dLedgerServer.getdLedgerStore(); + DLedgerMmapFileStore.AppendHook appendHook = (entry, buffer, bodyOffset) -> { + assert bodyOffset == DLedgerEntry.BODY_OFFSET; + buffer.position(buffer.position() + bodyOffset + MessageDecoder.PHY_POS_POSITION); + buffer.putLong(entry.getPos() + bodyOffset); + }; + dLedgerFileStore.addAppendHook(appendHook); + dLedgerFileList = dLedgerFileStore.getDataFileList(); + this.messageSerializer = new MessageSerializer(defaultMessageStore.getMessageStoreConfig().getMaxMessageSize()); + + } + + @Override + public boolean load() { + boolean result = super.load(); + if (!result) { + return false; + } + + return true; + } + + private void refreshConfig() { + dLedgerConfig.setEnableDiskForceClean(defaultMessageStore.getMessageStoreConfig().isCleanFileForciblyEnable()); + dLedgerConfig.setDeleteWhen(defaultMessageStore.getMessageStoreConfig().getDeleteWhen()); + dLedgerConfig.setFileReservedHours(defaultMessageStore.getMessageStoreConfig().getFileReservedTime() + 1); + } + + private void disableDeleteDledger() { + dLedgerConfig.setEnableDiskForceClean(false); + dLedgerConfig.setFileReservedHours(24 * 365 * 10); + } + + @Override + public void start() { + dLedgerServer.startup(); + } + + @Override + public void shutdown() { + dLedgerServer.shutdown(); + } + + @Override + public long flush() { + dLedgerFileStore.flush(); + return dLedgerFileList.getFlushedWhere(); + } + + @Override + public long getMaxOffset() { + if (dLedgerFileStore.getCommittedPos() > 0) { + return dLedgerFileStore.getCommittedPos(); + } + if (dLedgerFileList.getMinOffset() > 0) { + return dLedgerFileList.getMinOffset(); + } + return 0; + } + + @Override + public long getMinOffset() { + if (!mappedFileQueue.getMappedFiles().isEmpty()) { + return mappedFileQueue.getMinOffset(); + } + return dLedgerFileList.getMinOffset(); + } + + @Override + public long getConfirmOffset() { + return this.getMaxOffset(); + } + + @Override + public void setConfirmOffset(long phyOffset) { + log.warn("Should not set confirm offset {} for dleger commitlog", phyOffset); + } + + + + @Override + public long remainHowManyDataToCommit() { + return dLedgerFileList.remainHowManyDataToCommit(); + } + + @Override + public long remainHowManyDataToFlush() { + return dLedgerFileList.remainHowManyDataToFlush(); + } + + @Override + public int deleteExpiredFile( + final long expiredTime, + final int deleteFilesInterval, + final long intervalForcibly, + final boolean cleanImmediately + ) { + if (mappedFileQueue.getMappedFiles().isEmpty()) { + refreshConfig(); + //To prevent too much log in defaultMessageStore + return Integer.MAX_VALUE; + } else { + disableDeleteDledger(); + } + int count = super.deleteExpiredFile(expiredTime, deleteFilesInterval, intervalForcibly, cleanImmediately); + if (count > 0 || mappedFileQueue.getMappedFiles().size() != 1) { + return count; + } + //the old logic will keep the last file, here to delete it + MappedFile mappedFile = mappedFileQueue.getLastMappedFile(); + log.info("Try to delete the last old commitlog file {}", mappedFile.getFileName()); + long liveMaxTimestamp = mappedFile.getLastModifiedTimestamp() + expiredTime; + if (System.currentTimeMillis() >= liveMaxTimestamp || cleanImmediately) { + while (!mappedFile.destroy(10 * 1000)) { + DLedgerUtils.sleep(1000); + } + mappedFileQueue.getMappedFiles().remove(mappedFile); + } + return 1; + } + + + public SelectMappedBufferResult convertSbr(SelectMmapBufferResult sbr) { + if (sbr == null) { + return null; + } else { + return new DLedgerSelectMappedBufferResult(sbr); + } + + } + + public SelectMmapBufferResult truncate(SelectMmapBufferResult sbr) { + long committedPos = dLedgerFileStore.getCommittedPos(); + if (sbr == null || sbr.getStartOffset() == committedPos) { + return null; + } + if (sbr.getStartOffset() + sbr.getSize() <= committedPos) { + return sbr; + } else { + sbr.setSize((int) (committedPos - sbr.getStartOffset())); + return sbr; + } + } + + @Override + public SelectMappedBufferResult getData(final long offset) { + if (offset < dividedCommitlogOffset) { + return super.getData(offset); + } + return this.getData(offset, offset == 0); + } + + + @Override + public SelectMappedBufferResult getData(final long offset, final boolean returnFirstOnNotFound) { + if (offset < dividedCommitlogOffset) { + return super.getData(offset, returnFirstOnNotFound); + } + if (offset >= dLedgerFileStore.getCommittedPos()) { + return null; + } + int mappedFileSize = this.dLedgerServer.getdLedgerConfig().getMappedFileSizeForEntryData(); + MmapFile mappedFile = this.dLedgerFileList.findMappedFileByOffset(offset, returnFirstOnNotFound); + if (mappedFile != null) { + int pos = (int) (offset % mappedFileSize); + SelectMmapBufferResult sbr = mappedFile.selectMappedBuffer(pos); + return convertSbr(truncate(sbr)); + } + + return null; + } + + private void recover(long maxPhyOffsetOfConsumeQueue) { + dLedgerFileStore.load(); + if (dLedgerFileList.getMappedFiles().size() > 0) { + dLedgerFileStore.recover(); + dividedCommitlogOffset = dLedgerFileList.getFirstMappedFile().getFileFromOffset(); + MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); + if (mappedFile != null) { + disableDeleteDledger(); + } + long maxPhyOffset = dLedgerFileList.getMaxWrotePosition(); + // Clear ConsumeQueue redundant data + if (maxPhyOffsetOfConsumeQueue >= maxPhyOffset) { + log.warn("[TruncateCQ]maxPhyOffsetOfConsumeQueue({}) >= processOffset({}), truncate dirty logic files", maxPhyOffsetOfConsumeQueue, maxPhyOffset); + this.defaultMessageStore.truncateDirtyLogicFiles(maxPhyOffset); + } + return; + } + //Indicate that, it is the first time to load mixed commitlog, need to recover the old commitlog + isInrecoveringOldCommitlog = true; + //No need the abnormal recover + super.recoverNormally(maxPhyOffsetOfConsumeQueue); + isInrecoveringOldCommitlog = false; + MappedFile mappedFile = this.mappedFileQueue.getLastMappedFile(); + if (mappedFile == null) { + return; + } + ByteBuffer byteBuffer = mappedFile.sliceByteBuffer(); + byteBuffer.position(mappedFile.getWrotePosition()); + boolean needWriteMagicCode = true; + // 1 TOTAL SIZE + byteBuffer.getInt(); //size + int magicCode = byteBuffer.getInt(); + if (magicCode == CommitLog.BLANK_MAGIC_CODE) { + needWriteMagicCode = false; + } else { + log.info("Recover old commitlog found a illegal magic code={}", magicCode); + } + dLedgerConfig.setEnableDiskForceClean(false); + dividedCommitlogOffset = mappedFile.getFileFromOffset() + mappedFile.getFileSize(); + log.info("Recover old commitlog needWriteMagicCode={} pos={} file={} dividedCommitlogOffset={}", needWriteMagicCode, mappedFile.getFileFromOffset() + mappedFile.getWrotePosition(), mappedFile.getFileName(), dividedCommitlogOffset); + if (needWriteMagicCode) { + byteBuffer.position(mappedFile.getWrotePosition()); + byteBuffer.putInt(mappedFile.getFileSize() - mappedFile.getWrotePosition()); + byteBuffer.putInt(BLANK_MAGIC_CODE); + mappedFile.flush(0); + } + mappedFile.setWrotePosition(mappedFile.getFileSize()); + mappedFile.setCommittedPosition(mappedFile.getFileSize()); + mappedFile.setFlushedPosition(mappedFile.getFileSize()); + dLedgerFileList.getLastMappedFile(dividedCommitlogOffset); + log.info("Will set the initial commitlog offset={} for dledger", dividedCommitlogOffset); + } + + @Override + public void recoverNormally(long maxPhyOffsetOfConsumeQueue) { + recover(maxPhyOffsetOfConsumeQueue); + } + + @Override + public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) { + recover(maxPhyOffsetOfConsumeQueue); + } + + @Override + public DispatchRequest checkMessageAndReturnSize(ByteBuffer byteBuffer, final boolean checkCRC) { + return this.checkMessageAndReturnSize(byteBuffer, checkCRC, true); + } + + @Override + public DispatchRequest checkMessageAndReturnSize(ByteBuffer byteBuffer, final boolean checkCRC, + final boolean readBody) { + if (isInrecoveringOldCommitlog) { + return super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody); + } + try { + int bodyOffset = DLedgerEntry.BODY_OFFSET; + int pos = byteBuffer.position(); + int magic = byteBuffer.getInt(); + //In dledger, this field is size, it must be gt 0, so it could prevent collision + int magicOld = byteBuffer.getInt(); + if (magicOld == CommitLog.BLANK_MAGIC_CODE || magicOld == CommitLog.MESSAGE_MAGIC_CODE) { + byteBuffer.position(pos); + return super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody); + } + if (magic == MmapFileList.BLANK_MAGIC_CODE) { + return new DispatchRequest(0, true); + } + byteBuffer.position(pos + bodyOffset); + DispatchRequest dispatchRequest = super.checkMessageAndReturnSize(byteBuffer, checkCRC, readBody); + if (dispatchRequest.isSuccess()) { + dispatchRequest.setBufferSize(dispatchRequest.getMsgSize() + bodyOffset); + } else if (dispatchRequest.getMsgSize() > 0) { + dispatchRequest.setBufferSize(dispatchRequest.getMsgSize() + bodyOffset); + } + return dispatchRequest; + } catch (Throwable ignored) { + } + + return new DispatchRequest(-1, false /* success */); + } + + @Override + public boolean resetOffset(long offset) { + //currently, it seems resetOffset has no use + return false; + } + + @Override + public long getBeginTimeInLock() { + return beginTimeInDledgerLock; + } + + @Override + public PutMessageResult putMessage(final MessageExtBrokerInner msg) { + // Set the storage time + msg.setStoreTimestamp(System.currentTimeMillis()); + // Set the message body BODY CRC (consider the most appropriate setting + // on the client) + msg.setBodyCRC(UtilAll.crc32(msg.getBody())); + + StoreStatsService storeStatsService = this.defaultMessageStore.getStoreStatsService(); + + String topic = msg.getTopic(); + int queueId = msg.getQueueId(); + + //should be consistent with the old version + final int tranType = MessageSysFlag.getTransactionValue(msg.getSysFlag()); + if (tranType == MessageSysFlag.TRANSACTION_NOT_TYPE + || tranType == MessageSysFlag.TRANSACTION_COMMIT_TYPE) { + // Delay Delivery + if (msg.getDelayTimeLevel() > 0) { + if (msg.getDelayTimeLevel() > this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel()) { + msg.setDelayTimeLevel(this.defaultMessageStore.getScheduleMessageService().getMaxDelayLevel()); + } + + topic = ScheduleMessageService.SCHEDULE_TOPIC; + queueId = ScheduleMessageService.delayLevel2QueueId(msg.getDelayTimeLevel()); + + // Backup real topic, queueId + MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_TOPIC, msg.getTopic()); + MessageAccessor.putProperty(msg, MessageConst.PROPERTY_REAL_QUEUE_ID, String.valueOf(msg.getQueueId())); + msg.setPropertiesString(MessageDecoder.messageProperties2String(msg.getProperties())); + + msg.setTopic(topic); + msg.setQueueId(queueId); + } + } + + // Back to Results + AppendMessageResult appendResult; + AppendFuture dledgerFuture; + EncodeResult encodeResult; + + putMessageLock.lock(); //spin or ReentrantLock ,depending on store config + long eclipseTimeInLock; + long queueOffset; + try { + beginTimeInDledgerLock = this.defaultMessageStore.getSystemClock().now(); + encodeResult = this.messageSerializer.serialize(msg); + queueOffset = topicQueueTable.get(encodeResult.queueOffsetKey); + if (encodeResult.status != AppendMessageStatus.PUT_OK) { + return new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, new AppendMessageResult(encodeResult.status)); + } + AppendEntryRequest request = new AppendEntryRequest(); + request.setGroup(dLedgerConfig.getGroup()); + request.setRemoteId(dLedgerServer.getMemberState().getSelfId()); + request.setBody(encodeResult.data); + dledgerFuture = (AppendFuture) dLedgerServer.handleAppend(request); + if (dledgerFuture.getPos() == -1) { + return new PutMessageResult(PutMessageStatus.OS_PAGECACHE_BUSY, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)); + } + long wroteOffset = dledgerFuture.getPos() + DLedgerEntry.BODY_OFFSET; + ByteBuffer buffer = ByteBuffer.allocate(MessageDecoder.MSG_ID_LENGTH); + String msgId = MessageDecoder.createMessageId(buffer, msg.getStoreHostBytes(), wroteOffset); + eclipseTimeInLock = this.defaultMessageStore.getSystemClock().now() - beginTimeInDledgerLock; + appendResult = new AppendMessageResult(AppendMessageStatus.PUT_OK, wroteOffset, encodeResult.data.length, msgId, System.currentTimeMillis(), queueOffset, eclipseTimeInLock); + switch (tranType) { + case MessageSysFlag.TRANSACTION_PREPARED_TYPE: + case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: + break; + case MessageSysFlag.TRANSACTION_NOT_TYPE: + case MessageSysFlag.TRANSACTION_COMMIT_TYPE: + // The next update ConsumeQueue information + DLedgerCommitLog.this.topicQueueTable.put(encodeResult.queueOffsetKey, queueOffset + 1); + break; + default: + break; + } + } catch (Exception e) { + log.error("Put message error", e); + return new PutMessageResult(PutMessageStatus.UNKNOWN_ERROR, new AppendMessageResult(AppendMessageStatus.UNKNOWN_ERROR)); + } finally { + beginTimeInDledgerLock = 0; + putMessageLock.unlock(); + } + + if (eclipseTimeInLock > 500) { + log.warn("[NOTIFYME]putMessage in lock cost time(ms)={}, bodyLength={} AppendMessageResult={}", eclipseTimeInLock, msg.getBody().length, appendResult); + } + + PutMessageStatus putMessageStatus = PutMessageStatus.UNKNOWN_ERROR; + try { + AppendEntryResponse appendEntryResponse = dledgerFuture.get(3, TimeUnit.SECONDS); + switch (DLedgerResponseCode.valueOf(appendEntryResponse.getCode())) { + case SUCCESS: + putMessageStatus = PutMessageStatus.PUT_OK; + break; + case INCONSISTENT_LEADER: + case NOT_LEADER: + case LEADER_NOT_READY: + case DISK_FULL: + putMessageStatus = PutMessageStatus.SERVICE_NOT_AVAILABLE; + break; + case WAIT_QUORUM_ACK_TIMEOUT: + //Do not return flush_slave_timeout to the client, for the ons client will ignore it. + putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY; + break; + case LEADER_PENDING_FULL: + putMessageStatus = PutMessageStatus.OS_PAGECACHE_BUSY; + break; + } + } catch (Throwable t) { + log.error("Failed to get dledger append result", t); + } + + PutMessageResult putMessageResult = new PutMessageResult(putMessageStatus, appendResult); + if (putMessageStatus == PutMessageStatus.PUT_OK) { + // Statistics + storeStatsService.getSinglePutMessageTopicTimesTotal(msg.getTopic()).incrementAndGet(); + storeStatsService.getSinglePutMessageTopicSizeTotal(topic).addAndGet(appendResult.getWroteBytes()); + } + return putMessageResult; + } + + @Override + public PutMessageResult putMessages(final MessageExtBatch messageExtBatch) { + return new PutMessageResult(PutMessageStatus.MESSAGE_ILLEGAL, null); + } + + + + @Override + public SelectMappedBufferResult getMessage(final long offset, final int size) { + if (offset < dividedCommitlogOffset) { + return super.getMessage(offset, size); + } + int mappedFileSize = this.dLedgerServer.getdLedgerConfig().getMappedFileSizeForEntryData(); + MmapFile mappedFile = this.dLedgerFileList.findMappedFileByOffset(offset, offset == 0); + if (mappedFile != null) { + int pos = (int) (offset % mappedFileSize); + return convertSbr(mappedFile.selectMappedBuffer(pos, size)); + } + return null; + } + + @Override + public long rollNextFile(final long offset) { + int mappedFileSize = this.defaultMessageStore.getMessageStoreConfig().getMapedFileSizeCommitLog(); + return offset + mappedFileSize - offset % mappedFileSize; + } + + @Override + public HashMap getTopicQueueTable() { + return topicQueueTable; + } + + @Override + public void setTopicQueueTable(HashMap topicQueueTable) { + this.topicQueueTable = topicQueueTable; + } + + @Override + public void destroy() { + super.destroy(); + dLedgerFileList.destroy(); + } + + @Override + public boolean appendData(long startOffset, byte[] data) { + //the old ha service will invoke method, here to prevent it + return false; + } + + @Override + public void checkSelf() { + dLedgerFileList.checkSelf(); + } + + @Override + public long lockTimeMills() { + long diff = 0; + long begin = this.beginTimeInDledgerLock; + if (begin > 0) { + diff = this.defaultMessageStore.now() - begin; + } + + if (diff < 0) { + diff = 0; + } + + return diff; + } + + class EncodeResult { + private String queueOffsetKey; + private byte[] data; + private AppendMessageStatus status; + public EncodeResult(AppendMessageStatus status, byte[] data, String queueOffsetKey) { + this.data = data; + this.status = status; + this.queueOffsetKey = queueOffsetKey; + } + } + + class MessageSerializer { + // File at the end of the minimum fixed length empty + private static final int END_FILE_MIN_BLANK_LENGTH = 4 + 4; + private final ByteBuffer msgIdMemory; + // Store the message content + private final ByteBuffer msgStoreItemMemory; + // The maximum length of the message + private final int maxMessageSize; + // Build Message Key + private final StringBuilder keyBuilder = new StringBuilder(); + + private final StringBuilder msgIdBuilder = new StringBuilder(); + + private final ByteBuffer hostHolder = ByteBuffer.allocate(8); + + MessageSerializer(final int size) { + this.msgIdMemory = ByteBuffer.allocate(MessageDecoder.MSG_ID_LENGTH); + this.msgStoreItemMemory = ByteBuffer.allocate(size + END_FILE_MIN_BLANK_LENGTH); + this.maxMessageSize = size; + } + + public ByteBuffer getMsgStoreItemMemory() { + return msgStoreItemMemory; + } + + public EncodeResult serialize(final MessageExtBrokerInner msgInner) { + // STORETIMESTAMP + STOREHOSTADDRESS + OFFSET
+ + // PHY OFFSET + long wroteOffset = 0; + + this.resetByteBuffer(hostHolder, 8); + // Record ConsumeQueue information + keyBuilder.setLength(0); + keyBuilder.append(msgInner.getTopic()); + keyBuilder.append('-'); + keyBuilder.append(msgInner.getQueueId()); + String key = keyBuilder.toString(); + + Long queueOffset = DLedgerCommitLog.this.topicQueueTable.get(key); + if (null == queueOffset) { + queueOffset = 0L; + DLedgerCommitLog.this.topicQueueTable.put(key, queueOffset); + } + + // Transaction messages that require special handling + final int tranType = MessageSysFlag.getTransactionValue(msgInner.getSysFlag()); + switch (tranType) { + // Prepared and Rollback message is not consumed, will not enter the + // consumer queuec + case MessageSysFlag.TRANSACTION_PREPARED_TYPE: + case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: + queueOffset = 0L; + break; + case MessageSysFlag.TRANSACTION_NOT_TYPE: + case MessageSysFlag.TRANSACTION_COMMIT_TYPE: + default: + break; + } + + /** + * Serialize message + */ + final byte[] propertiesData = + msgInner.getPropertiesString() == null ? null : msgInner.getPropertiesString().getBytes(MessageDecoder.CHARSET_UTF8); + + final int propertiesLength = propertiesData == null ? 0 : propertiesData.length; + + if (propertiesLength > Short.MAX_VALUE) { + log.warn("putMessage message properties length too long. length={}", propertiesData.length); + return new EncodeResult(AppendMessageStatus.PROPERTIES_SIZE_EXCEEDED, null, key); + } + + final byte[] topicData = msgInner.getTopic().getBytes(MessageDecoder.CHARSET_UTF8); + final int topicLength = topicData.length; + + final int bodyLength = msgInner.getBody() == null ? 0 : msgInner.getBody().length; + + final int msgLen = calMsgLength(bodyLength, topicLength, propertiesLength); + + // Exceeds the maximum message + if (msgLen > this.maxMessageSize) { + DLedgerCommitLog.log.warn("message size exceeded, msg total size: " + msgLen + ", msg body size: " + bodyLength + + ", maxMessageSize: " + this.maxMessageSize); + return new EncodeResult(AppendMessageStatus.MESSAGE_SIZE_EXCEEDED, null, key); + } + // Initialization of storage space + this.resetByteBuffer(msgStoreItemMemory, msgLen); + // 1 TOTALSIZE + this.msgStoreItemMemory.putInt(msgLen); + // 2 MAGICCODE + this.msgStoreItemMemory.putInt(DLedgerCommitLog.MESSAGE_MAGIC_CODE); + // 3 BODYCRC + this.msgStoreItemMemory.putInt(msgInner.getBodyCRC()); + // 4 QUEUEID + this.msgStoreItemMemory.putInt(msgInner.getQueueId()); + // 5 FLAG + this.msgStoreItemMemory.putInt(msgInner.getFlag()); + // 6 QUEUEOFFSET + this.msgStoreItemMemory.putLong(queueOffset); + // 7 PHYSICALOFFSET + this.msgStoreItemMemory.putLong(wroteOffset); + // 8 SYSFLAG + this.msgStoreItemMemory.putInt(msgInner.getSysFlag()); + // 9 BORNTIMESTAMP + this.msgStoreItemMemory.putLong(msgInner.getBornTimestamp()); + // 10 BORNHOST + this.resetByteBuffer(hostHolder, 8); + this.msgStoreItemMemory.put(msgInner.getBornHostBytes(hostHolder)); + // 11 STORETIMESTAMP + this.msgStoreItemMemory.putLong(msgInner.getStoreTimestamp()); + // 12 STOREHOSTADDRESS + this.resetByteBuffer(hostHolder, 8); + this.msgStoreItemMemory.put(msgInner.getStoreHostBytes(hostHolder)); + //this.msgBatchMemory.put(msgInner.getStoreHostBytes()); + // 13 RECONSUMETIMES + this.msgStoreItemMemory.putInt(msgInner.getReconsumeTimes()); + // 14 Prepared Transaction Offset + this.msgStoreItemMemory.putLong(msgInner.getPreparedTransactionOffset()); + // 15 BODY + this.msgStoreItemMemory.putInt(bodyLength); + if (bodyLength > 0) { + this.msgStoreItemMemory.put(msgInner.getBody()); + } + // 16 TOPIC + this.msgStoreItemMemory.put((byte) topicLength); + this.msgStoreItemMemory.put(topicData); + // 17 PROPERTIES + this.msgStoreItemMemory.putShort((short) propertiesLength); + if (propertiesLength > 0) { + this.msgStoreItemMemory.put(propertiesData); + } + byte[] data = new byte[msgLen]; + this.msgStoreItemMemory.clear(); + this.msgStoreItemMemory.get(data); + return new EncodeResult(AppendMessageStatus.PUT_OK, data, key); + } + + private void resetByteBuffer(final ByteBuffer byteBuffer, final int limit) { + byteBuffer.flip(); + byteBuffer.limit(limit); + } + + } + + public static class DLedgerSelectMappedBufferResult extends SelectMappedBufferResult { + + private SelectMmapBufferResult sbr; + public DLedgerSelectMappedBufferResult(SelectMmapBufferResult sbr) { + super(sbr.getStartOffset(), sbr.getByteBuffer(), sbr.getSize(), null); + this.sbr = sbr; + } + + public synchronized void release() { + super.release(); + if (sbr != null) { + sbr.release(); + } + } + + } + + public DLedgerServer getdLedgerServer() { + return dLedgerServer; + } + + public int getId() { + return id; + } + + public long getDividedCommitlogOffset() { + return dividedCommitlogOffset; + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java index 251dc9971f8..c102881c1ca 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/HAConnection.java @@ -90,7 +90,7 @@ public ReadSocketService(final SocketChannel socketChannel) throws IOException { this.selector = RemotingUtil.openSelector(); this.socketChannel = socketChannel; this.socketChannel.register(this.selector, SelectionKey.OP_READ); - this.thread.setDaemon(true); + this.setDaemon(true); } @Override @@ -205,7 +205,7 @@ public WriteSocketService(final SocketChannel socketChannel) throws IOException this.selector = RemotingUtil.openSelector(); this.socketChannel = socketChannel; this.socketChannel.register(this.selector, SelectionKey.OP_WRITE); - this.thread.setDaemon(true); + this.setDaemon(true); } @Override diff --git a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java index e707463c795..50a48d4de9c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java +++ b/store/src/main/java/org/apache/rocketmq/store/schedule/ScheduleMessageService.java @@ -19,11 +19,11 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import java.util.Map.Entry; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.common.ConfigManager; import org.apache.rocketmq.common.TopicFilterType; import org.apache.rocketmq.common.constant.LoggerName; @@ -38,6 +38,7 @@ import org.apache.rocketmq.store.ConsumeQueueExt; import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.SelectMappedBufferResult; @@ -56,15 +57,15 @@ public class ScheduleMessageService extends ConfigManager { private final ConcurrentMap offsetTable = new ConcurrentHashMap(32); - - private final Timer timer = new Timer("ScheduleMessageTimerThread", true); - private final DefaultMessageStore defaultMessageStore; - + private final AtomicBoolean started = new AtomicBoolean(false); + private Timer timer; + private MessageStore writeMessageStore; private int maxDelayLevel; public ScheduleMessageService(final DefaultMessageStore defaultMessageStore) { this.defaultMessageStore = defaultMessageStore; + this.writeMessageStore = defaultMessageStore; } public static int queueId2DelayLevel(final int queueId) { @@ -75,10 +76,18 @@ public static int delayLevel2QueueId(final int delayLevel) { return delayLevel - 1; } + /** + * @param writeMessageStore + * the writeMessageStore to set + */ + public void setWriteMessageStore(MessageStore writeMessageStore) { + this.writeMessageStore = writeMessageStore; + } + public void buildRunningStats(HashMap stats) { - Iterator> it = this.offsetTable.entrySet().iterator(); + Iterator> it = this.offsetTable.entrySet().iterator(); while (it.hasNext()) { - Entry next = it.next(); + Map.Entry next = it.next(); int queueId = delayLevel2QueueId(next.getKey()); long delayOffset = next.getValue(); long maxOffset = this.defaultMessageStore.getMaxOffsetInQueue(SCHEDULE_TOPIC, queueId); @@ -102,35 +111,45 @@ public long computeDeliverTimestamp(final int delayLevel, final long storeTimest } public void start() { + if (started.compareAndSet(false, true)) { + this.timer = new Timer("ScheduleMessageTimerThread", true); + for (Map.Entry entry : this.delayLevelTable.entrySet()) { + Integer level = entry.getKey(); + Long timeDelay = entry.getValue(); + Long offset = this.offsetTable.get(level); + if (null == offset) { + offset = 0L; + } - for (Map.Entry entry : this.delayLevelTable.entrySet()) { - Integer level = entry.getKey(); - Long timeDelay = entry.getValue(); - Long offset = this.offsetTable.get(level); - if (null == offset) { - offset = 0L; - } - - if (timeDelay != null) { - this.timer.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME); + if (timeDelay != null) { + this.timer.schedule(new DeliverDelayedMessageTimerTask(level, offset), FIRST_DELAY_TIME); + } } - } - this.timer.scheduleAtFixedRate(new TimerTask() { + this.timer.scheduleAtFixedRate(new TimerTask() { - @Override - public void run() { - try { - ScheduleMessageService.this.persist(); - } catch (Throwable e) { - log.error("scheduleAtFixedRate flush exception", e); + @Override + public void run() { + try { + if (started.get()) ScheduleMessageService.this.persist(); + } catch (Throwable e) { + log.error("scheduleAtFixedRate flush exception", e); + } } - } - }, 10000, this.defaultMessageStore.getMessageStoreConfig().getFlushDelayOffsetInterval()); + }, 10000, this.defaultMessageStore.getMessageStoreConfig().getFlushDelayOffsetInterval()); + } } public void shutdown() { - this.timer.cancel(); + if (this.started.compareAndSet(true, false)) { + if (null != this.timer) + this.timer.cancel(); + } + + } + + public boolean isStarted() { + return started.get(); } public int getMaxDelayLevel() { @@ -214,7 +233,9 @@ public DeliverDelayedMessageTimerTask(int delayLevel, long offset) { @Override public void run() { try { - this.executeOnTimeup(); + if (isStarted()) { + this.executeOnTimeup(); + } } catch (Exception e) { // XXX: warn and notify me log.error("ScheduleMessageService, executeOnTimeup exception", e); @@ -285,7 +306,7 @@ public void executeOnTimeup() { try { MessageExtBrokerInner msgInner = this.messageTimeup(msgExt); PutMessageResult putMessageResult = - ScheduleMessageService.this.defaultMessageStore + ScheduleMessageService.this.writeMessageStore .putMessage(msgInner); if (putMessageResult != null diff --git a/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java new file mode 100644 index 00000000000..38178490be8 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/BatchPutMessageTest.java @@ -0,0 +1,172 @@ +/* + * 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 org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.Message; +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.store.config.FlushDiskType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.net.InetSocketAddress; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertTrue; + +public class BatchPutMessageTest { + + private MessageStore messageStore; + + public static final char NAME_VALUE_SEPARATOR = 1; + public static final char PROPERTY_SEPARATOR = 2; + public final static Charset CHARSET_UTF8 = Charset.forName("UTF-8"); + + @Before + public void init() throws Exception { + messageStore = buildMessageStore(); + boolean load = messageStore.load(); + assertTrue(load); + messageStore.start(); + } + + @After + public void destory() { + messageStore.shutdown(); + messageStore.destroy(); + + UtilAll.deleteFile(new File(System.getProperty("user.home") + File.separator + "putmessagesteststore")); + } + + private MessageStore buildMessageStore() throws Exception { + MessageStoreConfig messageStoreConfig = new MessageStoreConfig(); + messageStoreConfig.setMapedFileSizeCommitLog(1024 * 8); + messageStoreConfig.setMapedFileSizeConsumeQueue(1024 * 4); + messageStoreConfig.setMaxHashSlotNum(100); + messageStoreConfig.setMaxIndexNum(100 * 10); + messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); + messageStoreConfig.setFlushIntervalConsumeQueue(1); + messageStoreConfig.setStorePathRootDir(System.getProperty("user.home") + File.separator + "putmessagesteststore"); + messageStoreConfig.setStorePathCommitLog(System.getProperty("user.home") + File.separator + "putmessagesteststore" + File.separator + "commitlog"); + return new DefaultMessageStore(messageStoreConfig, new BrokerStatsManager("simpleTest"), new MyMessageArrivingListener(), new BrokerConfig()); + } + + @Test + public void testPutMessages() throws Exception { + List messages = new ArrayList<>(); + String topic = "batch-write-topic"; + int queue = 0; + int[] msgLengthArr = new int[11]; + msgLengthArr[0] = 0; + int j = 1; + for (int i = 0; i < 10; i++) { + Message msg = new Message(); + msg.setBody(("body" + i).getBytes()); + msg.setTopic(topic); + msg.setTags("TAG1"); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + messages.add(msg); + String properties = messageProperties2String(msg.getProperties()); + byte[] propertiesBytes = properties.getBytes(CHARSET_UTF8); + short propertiesLength = (short) propertiesBytes.length; + final byte[] topicData = msg.getTopic().getBytes(MessageDecoder.CHARSET_UTF8); + final int topicLength = topicData.length; + msgLengthArr[j] = calMsgLength(msg.getBody().length, topicLength, propertiesLength) + msgLengthArr[j - 1]; + j++; + } + byte[] batchMessageBody = MessageDecoder.encodeMessages(messages); + MessageExtBatch messageExtBatch = new MessageExtBatch(); + messageExtBatch.setTopic(topic); + messageExtBatch.setQueueId(queue); + messageExtBatch.setBody(batchMessageBody); + messageExtBatch.setBornTimestamp(System.currentTimeMillis()); + messageExtBatch.setStoreHost(new InetSocketAddress("127.0.0.1", 125)); + messageExtBatch.setBornHost(new InetSocketAddress("127.0.0.1", 126)); + + PutMessageResult putMessageResult = messageStore.putMessages(messageExtBatch); + assertThat(putMessageResult.isOk()).isTrue(); + + Thread.sleep(3 * 1000); + + for (long i = 0; i < 10; i++) { + MessageExt messageExt = messageStore.lookMessageByOffset(msgLengthArr[(int) i]); + assertThat(messageExt).isNotNull(); + GetMessageResult result = messageStore.getMessage("batch_write_group", topic, queue, i, 1024 * 1024, null); + assertThat(result).isNotNull(); + assertThat(result.getStatus()).isEqualTo(GetMessageStatus.FOUND); + result.release(); + } + + } + + private int calMsgLength(int bodyLength, int topicLength, int propertiesLength) { + final int msgLen = 4 //TOTALSIZE + + 4 //MAGICCODE + + 4 //BODYCRC + + 4 //QUEUEID + + 4 //FLAG + + 8 //QUEUEOFFSET + + 8 //PHYSICALOFFSET + + 4 //SYSFLAG + + 8 //BORNTIMESTAMP + + 8 //BORNHOST + + 8 //STORETIMESTAMP + + 8 //STOREHOSTADDRESS + + 4 //RECONSUMETIMES + + 8 //Prepared Transaction Offset + + 4 + (bodyLength > 0 ? bodyLength : 0) //BODY + + 1 + topicLength //TOPIC + + 2 + (propertiesLength > 0 ? propertiesLength : 0) //propertiesLength + + 0; + return msgLen; + } + + public String messageProperties2String(Map properties) { + StringBuilder sb = new StringBuilder(); + if (properties != null) { + for (final Map.Entry entry : properties.entrySet()) { + final String name = entry.getKey(); + final String value = entry.getValue(); + + sb.append(name); + sb.append(NAME_VALUE_SEPARATOR); + sb.append(value); + sb.append(PROPERTY_SEPARATOR); + } + } + return sb.toString(); + } + + private class MyMessageArrivingListener implements MessageArrivingListener { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, + byte[] filterBitMap, Map properties) { + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java index 7e01b8513c9..470d70b9534 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ConsumeQueueTest.java @@ -30,9 +30,10 @@ import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.stats.BrokerStatsManager; -import static org.assertj.core.api.Assertions.assertThat; import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + public class ConsumeQueueTest { private static final String msg = "Once, there was a chance for me!"; @@ -212,6 +213,7 @@ public void dispatch(DispatchRequest request) { try { try { putMsg(master); + Thread.sleep(3000L);//wait ConsumeQueue create success. } catch (Exception e) { e.printStackTrace(); assertThat(Boolean.FALSE).isTrue(); diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java new file mode 100644 index 00000000000..b987cd385d9 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreCleanFilesTest.java @@ -0,0 +1,387 @@ +/* + * 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 org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.store.config.FlushDiskType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.lang.reflect.Field; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.Calendar; +import java.util.Map; +import java.util.UUID; + +import static org.apache.rocketmq.common.message.MessageDecoder.CHARSET_UTF8; +import static org.apache.rocketmq.store.ConsumeQueue.CQ_STORE_UNIT_SIZE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test case for DefaultMessageStore.CleanCommitLogService and DefaultMessageStore.CleanConsumeQueueService + */ +public class DefaultMessageStoreCleanFilesTest { + private DefaultMessageStore messageStore; + private DefaultMessageStore.CleanCommitLogService cleanCommitLogService; + private DefaultMessageStore.CleanConsumeQueueService cleanConsumeQueueService; + + private SocketAddress bornHost; + private SocketAddress storeHost; + + private String topic = "test"; + private int queueId = 0; + private int fileCountCommitLog = 55; + // exactly one message per CommitLog file. + private int msgCount = fileCountCommitLog; + private int mappedFileSize = 128; + private int fileReservedTime = 1; + + @Before + public void init() throws Exception { + storeHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123); + bornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0); + } + + @Test + public void testDeleteExpiredFilesByTimeUp() throws Exception { + String deleteWhen = Calendar.getInstance().get(Calendar.HOUR_OF_DAY) + ""; + // the max value of diskMaxUsedSpaceRatio + int diskMaxUsedSpaceRatio = 99; + // used to ensure that automatic file deletion is not triggered + double diskSpaceCleanForciblyRatio = 0.999D; + initMessageStore(deleteWhen, diskMaxUsedSpaceRatio, diskSpaceCleanForciblyRatio); + + // build and put 55 messages, exactly one message per CommitLog file. + buildAndPutMessagesToMessageStore(msgCount); + + // undo comment out the code below, if want to debug this case rather than just run it. + // Thread.sleep(1000 * 60 + 100); + + MappedFileQueue commitLogQueue = getMappedFileQueueCommitLog(); + assertEquals(fileCountCommitLog, commitLogQueue.getMappedFiles().size()); + + int fileCountConsumeQueue = getFileCountConsumeQueue(); + MappedFileQueue consumeQueue = getMappedFileQueueConsumeQueue(); + assertEquals(fileCountConsumeQueue, consumeQueue.getMappedFiles().size()); + + int expireFileCount = 15; + expireFiles(commitLogQueue, expireFileCount); + + // magic code 10 reference to MappedFileQueue#DELETE_FILES_BATCH_MAX + for (int a = 1, fileCount = expireFileCount; a <= (int) Math.ceil((double) expireFileCount / 10); a++, fileCount -= 10) { + cleanCommitLogService.run(); + cleanConsumeQueueService.run(); + + int expectDeletedCount = fileCount >= 10 ? a * 10 : ((a - 1) * 10 + fileCount); + assertEquals(fileCountCommitLog - expectDeletedCount, commitLogQueue.getMappedFiles().size()); + + int msgCountPerFile = getMsgCountPerConsumeQueueMappedFile(); + int expectDeleteCountConsumeQueue = (int) Math.floor((double) expectDeletedCount / msgCountPerFile); + assertEquals(fileCountConsumeQueue - expectDeleteCountConsumeQueue, consumeQueue.getMappedFiles().size()); + } + } + + @Test + public void testDeleteExpiredFilesBySpaceFull() throws Exception { + String deleteWhen = "04"; + // the min value of diskMaxUsedSpaceRatio. + int diskMaxUsedSpaceRatio = 1; + // used to ensure that automatic file deletion is not triggered + double diskSpaceCleanForciblyRatio = 0.999D; + initMessageStore(deleteWhen, diskMaxUsedSpaceRatio, diskSpaceCleanForciblyRatio); + + // build and put 55 messages, exactly one message per CommitLog file. + buildAndPutMessagesToMessageStore(msgCount); + + // undo comment out the code below, if want to debug this case rather than just run it. + // Thread.sleep(1000 * 60 + 100); + + MappedFileQueue commitLogQueue = getMappedFileQueueCommitLog(); + assertEquals(fileCountCommitLog, commitLogQueue.getMappedFiles().size()); + + int fileCountConsumeQueue = getFileCountConsumeQueue(); + MappedFileQueue consumeQueue = getMappedFileQueueConsumeQueue(); + assertEquals(fileCountConsumeQueue, consumeQueue.getMappedFiles().size()); + + int expireFileCount = 15; + expireFiles(commitLogQueue, expireFileCount); + + // magic code 10 reference to MappedFileQueue#DELETE_FILES_BATCH_MAX + for (int a = 1, fileCount = expireFileCount; a <= (int) Math.ceil((double) expireFileCount / 10); a++, fileCount -= 10) { + cleanCommitLogService.run(); + cleanConsumeQueueService.run(); + + int expectDeletedCount = fileCount >= 10 ? a * 10 : ((a - 1) * 10 + fileCount); + assertEquals(fileCountCommitLog - expectDeletedCount, commitLogQueue.getMappedFiles().size()); + + int msgCountPerFile = getMsgCountPerConsumeQueueMappedFile(); + int expectDeleteCountConsumeQueue = (int) Math.floor((double) expectDeletedCount / msgCountPerFile); + assertEquals(fileCountConsumeQueue - expectDeleteCountConsumeQueue, consumeQueue.getMappedFiles().size()); + } + } + + @Test + public void testDeleteFilesImmediatelyBySpaceFull() throws Exception { + String deleteWhen = "04"; + // the min value of diskMaxUsedSpaceRatio. + int diskMaxUsedSpaceRatio = 1; + // make sure to trigger the automatic file deletion feature + double diskSpaceCleanForciblyRatio = 0.01D; + initMessageStore(deleteWhen, diskMaxUsedSpaceRatio, diskSpaceCleanForciblyRatio); + + // build and put 55 messages, exactly one message per CommitLog file. + buildAndPutMessagesToMessageStore(msgCount); + + // undo comment out the code below, if want to debug this case rather than just run it. + // Thread.sleep(1000 * 60 + 100); + + MappedFileQueue commitLogQueue = getMappedFileQueueCommitLog(); + assertEquals(fileCountCommitLog, commitLogQueue.getMappedFiles().size()); + + int fileCountConsumeQueue = getFileCountConsumeQueue(); + MappedFileQueue consumeQueue = getMappedFileQueueConsumeQueue(); + assertEquals(fileCountConsumeQueue, consumeQueue.getMappedFiles().size()); + + // In this case, there is no need to expire the files. + // int expireFileCount = 15; + // expireFiles(commitLogQueue, expireFileCount); + + // magic code 10 reference to MappedFileQueue#DELETE_FILES_BATCH_MAX + for (int a = 1, fileCount = fileCountCommitLog; + a <= (int) Math.ceil((double) fileCountCommitLog / 10) && fileCount >= 10; + a++, fileCount -= 10) { + cleanCommitLogService.run(); + cleanConsumeQueueService.run(); + + assertEquals(fileCountCommitLog - 10 * a, commitLogQueue.getMappedFiles().size()); + + int msgCountPerFile = getMsgCountPerConsumeQueueMappedFile(); + int expectDeleteCountConsumeQueue = (int) Math.floor((double) (a * 10) / msgCountPerFile); + assertEquals(fileCountConsumeQueue - expectDeleteCountConsumeQueue, consumeQueue.getMappedFiles().size()); + } + } + + @Test + public void testDeleteExpiredFilesManually() throws Exception { + String deleteWhen = "04"; + // the max value of diskMaxUsedSpaceRatio + int diskMaxUsedSpaceRatio = 99; + // used to ensure that automatic file deletion is not triggered + double diskSpaceCleanForciblyRatio = 0.999D; + initMessageStore(deleteWhen, diskMaxUsedSpaceRatio, diskSpaceCleanForciblyRatio); + + messageStore.executeDeleteFilesManually(); + + // build and put 55 messages, exactly one message per CommitLog file. + buildAndPutMessagesToMessageStore(msgCount); + + // undo comment out the code below, if want to debug this case rather than just run it. + // Thread.sleep(1000 * 60 + 100); + + MappedFileQueue commitLogQueue = getMappedFileQueueCommitLog(); + assertEquals(fileCountCommitLog, commitLogQueue.getMappedFiles().size()); + + int fileCountConsumeQueue = getFileCountConsumeQueue(); + MappedFileQueue consumeQueue = getMappedFileQueueConsumeQueue(); + assertEquals(fileCountConsumeQueue, consumeQueue.getMappedFiles().size()); + + int expireFileCount = 15; + expireFiles(commitLogQueue, expireFileCount); + + // magic code 10 reference to MappedFileQueue#DELETE_FILES_BATCH_MAX + for (int a = 1, fileCount = expireFileCount; a <= (int) Math.ceil((double) expireFileCount / 10); a++, fileCount -= 10) { + cleanCommitLogService.run(); + cleanConsumeQueueService.run(); + + int expectDeletedCount = fileCount >= 10 ? a * 10 : ((a - 1) * 10 + fileCount); + assertEquals(fileCountCommitLog - expectDeletedCount, commitLogQueue.getMappedFiles().size()); + + int msgCountPerFile = getMsgCountPerConsumeQueueMappedFile(); + int expectDeleteCountConsumeQueue = (int) Math.floor((double) expectDeletedCount / msgCountPerFile); + assertEquals(fileCountConsumeQueue - expectDeleteCountConsumeQueue, consumeQueue.getMappedFiles().size()); + } + } + + private DefaultMessageStore.CleanCommitLogService getCleanCommitLogService(double diskSpaceCleanForciblyRatio) + throws Exception { + Field serviceField = messageStore.getClass().getDeclaredField("cleanCommitLogService"); + serviceField.setAccessible(true); + DefaultMessageStore.CleanCommitLogService cleanCommitLogService = + (DefaultMessageStore.CleanCommitLogService) serviceField.get(messageStore); + serviceField.setAccessible(false); + + Field warningLevelRatioField = cleanCommitLogService.getClass().getDeclaredField("diskSpaceWarningLevelRatio"); + warningLevelRatioField.setAccessible(true); + warningLevelRatioField.set(cleanCommitLogService, diskSpaceCleanForciblyRatio); + warningLevelRatioField.setAccessible(false); + + Field cleanForciblyRatioField = cleanCommitLogService.getClass().getDeclaredField("diskSpaceCleanForciblyRatio"); + cleanForciblyRatioField.setAccessible(true); + cleanForciblyRatioField.set(cleanCommitLogService, diskSpaceCleanForciblyRatio); + cleanForciblyRatioField.setAccessible(false); + return cleanCommitLogService; + } + + private DefaultMessageStore.CleanConsumeQueueService getCleanConsumeQueueService() + throws Exception { + Field serviceField = messageStore.getClass().getDeclaredField("cleanConsumeQueueService"); + serviceField.setAccessible(true); + DefaultMessageStore.CleanConsumeQueueService cleanConsumeQueueService = + (DefaultMessageStore.CleanConsumeQueueService) serviceField.get(messageStore); + serviceField.setAccessible(false); + return cleanConsumeQueueService; + } + + private MappedFileQueue getMappedFileQueueConsumeQueue() + throws Exception { + ConsumeQueue consumeQueue = messageStore.getConsumeQueueTable().get(topic).get(queueId); + Field queueField = consumeQueue.getClass().getDeclaredField("mappedFileQueue"); + queueField.setAccessible(true); + MappedFileQueue fileQueue = (MappedFileQueue) queueField.get(consumeQueue); + queueField.setAccessible(false); + return fileQueue; + } + + private MappedFileQueue getMappedFileQueueCommitLog() throws Exception { + CommitLog commitLog = messageStore.getCommitLog(); + Field queueField = commitLog.getClass().getDeclaredField("mappedFileQueue"); + queueField.setAccessible(true); + MappedFileQueue fileQueue = (MappedFileQueue) queueField.get(commitLog); + queueField.setAccessible(false); + return fileQueue; + } + + private int getFileCountConsumeQueue() { + int countPerFile = getMsgCountPerConsumeQueueMappedFile(); + double fileCount = (double) msgCount / countPerFile; + return (int) Math.ceil(fileCount); + } + + private int getMsgCountPerConsumeQueueMappedFile() { + int size = messageStore.getMessageStoreConfig().getMapedFileSizeConsumeQueue(); + return size / CQ_STORE_UNIT_SIZE;// 7 in this case + } + + private void buildAndPutMessagesToMessageStore(int msgCount) throws Exception { + int msgLen = topic.getBytes(CHARSET_UTF8).length + 91; + int commitLogEndFileMinBlankLength = 4 + 4; + int singleMsgBodyLen = mappedFileSize - msgLen - commitLogEndFileMinBlankLength; + + for (int i = 0; i < msgCount; i++) { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic(topic); + msg.setBody(new byte[singleMsgBodyLen]); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + msg.setQueueId(queueId); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(storeHost); + msg.setBornHost(bornHost); + PutMessageResult result = messageStore.putMessage(msg); + assertTrue(result != null && result.isOk()); + } + + StoreTestUtil.waitCommitLogReput(messageStore); + StoreTestUtil.flushConsumeQueue(messageStore); + StoreTestUtil.flushConsumeIndex(messageStore); + } + + private void expireFiles(MappedFileQueue commitLogQueue, int expireCount) { + for (int i = 0; i < commitLogQueue.getMappedFiles().size(); i++) { + MappedFile mappedFile = commitLogQueue.getMappedFiles().get(i); + int reservedTime = fileReservedTime * 60 * 60 * 1000; + if (i < expireCount) { + boolean modified = mappedFile.getFile().setLastModified(System.currentTimeMillis() - reservedTime * 2); + assertTrue(modified); + } + } + } + + private void initMessageStore(String deleteWhen, int diskMaxUsedSpaceRatio, double diskSpaceCleanForciblyRatio) throws Exception { + MessageStoreConfig messageStoreConfig = new MessageStoreConfigForTest(); + messageStoreConfig.setMapedFileSizeCommitLog(mappedFileSize); + messageStoreConfig.setMapedFileSizeConsumeQueue(mappedFileSize); + messageStoreConfig.setMaxHashSlotNum(10000); + messageStoreConfig.setMaxIndexNum(100 * 100); + messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); + messageStoreConfig.setFlushIntervalConsumeQueue(1); + + // Invalidate DefaultMessageStore`s scheduled task of cleaning expired files. + // work with the code 'Thread.sleep(1000 * 60 + 100)' behind. + messageStoreConfig.setCleanResourceInterval(Integer.MAX_VALUE); + + messageStoreConfig.setFileReservedTime(fileReservedTime); + messageStoreConfig.setDeleteWhen(deleteWhen); + messageStoreConfig.setDiskMaxUsedSpaceRatio(diskMaxUsedSpaceRatio); + + String storePathRootDir = System.getProperty("user.home") + File.separator + + "DefaultMessageStoreCleanFilesTest-" + UUID.randomUUID(); + String storePathCommitLog = storePathRootDir + File.separator + "commitlog"; + messageStoreConfig.setStorePathRootDir(storePathRootDir); + messageStoreConfig.setStorePathCommitLog(storePathCommitLog); + + messageStore = new DefaultMessageStore(messageStoreConfig, + new BrokerStatsManager("test"), new MyMessageArrivingListener(), new BrokerConfig()); + + cleanCommitLogService = getCleanCommitLogService(diskSpaceCleanForciblyRatio); + cleanConsumeQueueService = getCleanConsumeQueueService(); + + assertTrue(messageStore.load()); + messageStore.start(); + } + + private class MyMessageArrivingListener implements MessageArrivingListener { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, + byte[] filterBitMap, Map properties) { + } + } + + @After + public void destroy() { + messageStore.shutdown(); + messageStore.destroy(); + + MessageStoreConfig messageStoreConfig = messageStore.getMessageStoreConfig(); + File file = new File(messageStoreConfig.getStorePathRootDir()); + UtilAll.deleteFile(file); + } + + private class MessageStoreConfigForTest extends MessageStoreConfig { + @Override + public int getDiskMaxUsedSpaceRatio() { + try { + Field diskMaxUsedSpaceRatioField = this.getClass().getSuperclass().getDeclaredField("diskMaxUsedSpaceRatio"); + diskMaxUsedSpaceRatioField.setAccessible(true); + int ratio = (int) diskMaxUsedSpaceRatioField.get(this); + diskMaxUsedSpaceRatioField.setAccessible(false); + return ratio; + } catch (Exception ignored) { + } + return super.getDiskMaxUsedSpaceRatio(); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java index 66f7f5dc8f5..78457dbfe8a 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreShutDownTest.java @@ -42,9 +42,9 @@ public class DefaultMessageStoreShutDownTest { public void init() throws Exception { messageStore = spy(buildMessageStore()); boolean load = messageStore.load(); - when(messageStore.dispatchBehindBytes()).thenReturn(100L); assertTrue(load); messageStore.start(); + when(messageStore.dispatchBehindBytes()).thenReturn(100L); } @Test 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 57b6999c437..d0f729340bd 100644 --- a/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/DefaultMessageStoreTest.java @@ -19,28 +19,37 @@ 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.nio.ByteBuffer; import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; import java.nio.channels.OverlappingFileLockException; import java.util.Map; +import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.store.config.FlushDiskType; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.config.StorePathConfigHelper; -import org.junit.After; import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.After; import org.junit.Before; +import org.junit.Ignore; 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.assertTrue; +import static org.mockito.Mockito.mock; +@RunWith(MockitoJUnitRunner.class) public class DefaultMessageStoreTest { private final String StoreMessage = "Once, there was a chance for me!"; private int QUEUE_TOTAL = 100; @@ -62,7 +71,7 @@ public void init() throws Exception { } @Test(expected = OverlappingFileLockException.class) - public void test_repate_restart() throws Exception { + public void test_repeat_restart() throws Exception { QUEUE_TOTAL = 1; MessageBody = StoreMessage.getBytes(); @@ -86,7 +95,7 @@ public void test_repate_restart() throws Exception { } @After - public void destory() { + public void destroy() { messageStore.shutdown(); messageStore.destroy(); @@ -115,6 +124,8 @@ public void testWriteAndRead() { messageStore.putMessage(buildMessage()); } + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + for (long i = 0; i < totalMsgs; i++) { GetMessageResult result = messageStore.getMessage("GROUP_A", "TOPIC_A", 0, i, 1024 * 1024, null); assertThat(result).isNotNull(); @@ -123,12 +134,282 @@ public void testWriteAndRead() { verifyThatMasterIsFunctional(totalMsgs, messageStore); } - private MessageExtBrokerInner buildMessage() { + @Test + public void should_look_message_successfully_when_offset_is_first() { + 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 should_look_message_successfully_when_offset_is_last() { + 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 should_look_message_failed_and_return_null_when_offset_is_out_of_bound() { + 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 should_get_consume_queue_offset_successfully_when_incomming_by_timestamp() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + String topic = "FooBar"; + AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + + ConsumeQueue consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId); + for (AppendMessageResult appendMessageResult : appendMessageResults) { + long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp()); + SelectMappedBufferResult indexBuffer = consumeQueue.getIndexBuffer(offset); + assertThat(indexBuffer.getByteBuffer().getLong()).isEqualTo(appendMessageResult.getWroteOffset()); + assertThat(indexBuffer.getByteBuffer().getInt()).isEqualTo(appendMessageResult.getWroteBytes()); + indexBuffer.release(); + } + } + + @Test + public void should_get_consume_queue_offset_successfully_when_timestamp_is_skewing() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + String topic = "FooBar"; + AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + int skewing = 2; + + ConsumeQueue consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId); + for (AppendMessageResult appendMessageResult : appendMessageResults) { + long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() + skewing); + long offset2 = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() - skewing); + SelectMappedBufferResult indexBuffer = consumeQueue.getIndexBuffer(offset); + SelectMappedBufferResult indexBuffer2 = consumeQueue.getIndexBuffer(offset2); + assertThat(indexBuffer.getByteBuffer().getLong()).isEqualTo(appendMessageResult.getWroteOffset()); + assertThat(indexBuffer.getByteBuffer().getInt()).isEqualTo(appendMessageResult.getWroteBytes()); + assertThat(indexBuffer2.getByteBuffer().getLong()).isEqualTo(appendMessageResult.getWroteOffset()); + assertThat(indexBuffer2.getByteBuffer().getInt()).isEqualTo(appendMessageResult.getWroteBytes()); + indexBuffer.release(); + indexBuffer2.release(); + } + } + + @Test + public void should_get_min_of_max_consume_queue_offset_when_timestamp_s_skewing_is_large() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + String topic = "FooBar"; + AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, true); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + int skewing = 20000; + + ConsumeQueue consumeQueue = getDefaultMessageStore().findConsumeQueue(topic, queueId); + for (AppendMessageResult appendMessageResult : appendMessageResults) { + long offset = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() + skewing); + long offset2 = messageStore.getOffsetInQueueByTime(topic, queueId, appendMessageResult.getStoreTimestamp() - skewing); + SelectMappedBufferResult indexBuffer = consumeQueue.getIndexBuffer(offset); + SelectMappedBufferResult indexBuffer2 = consumeQueue.getIndexBuffer(offset2); + assertThat(indexBuffer.getByteBuffer().getLong()).isEqualTo(appendMessageResults[totalCount - 1].getWroteOffset()); + assertThat(indexBuffer.getByteBuffer().getInt()).isEqualTo(appendMessageResults[totalCount - 1].getWroteBytes()); + assertThat(indexBuffer2.getByteBuffer().getLong()).isEqualTo(appendMessageResults[0].getWroteOffset()); + assertThat(indexBuffer2.getByteBuffer().getInt()).isEqualTo(appendMessageResults[0].getWroteBytes()); + + indexBuffer.release(); + indexBuffer2.release(); + } + } + + @Test + public void should_return_zero_when_consume_queue_not_found() throws InterruptedException { + 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((DefaultMessageStore) messageStore); + + long offset = messageStore.getOffsetInQueueByTime(topic, wrongQueueId, appendMessageResults[0].getStoreTimestamp()); + + assertThat(offset).isEqualTo(0); + } + + @Test + public void should_return_negative_one_when_invoke_getMessageStoreTimeStamp_if_consume_queue_not_found() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + int wrongQueueId = 1; + String topic = "FooBar"; + putMessages(totalCount, topic, queueId, false); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + + long messageStoreTimeStamp = messageStore.getMessageStoreTimeStamp(topic, wrongQueueId, 0); + + assertThat(messageStoreTimeStamp).isEqualTo(-1); + } + + @Test + public void should_return_negative_one_when_invoke_getMessageStoreTimeStamp_if_consumeQueueOffset_not_exist() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + int wrongQueueId = 1; + String topic = "FooBar"; + putMessages(totalCount, topic, queueId, true); + //Thread.sleep(10); + + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + + long messageStoreTimeStamp = messageStore.getMessageStoreTimeStamp(topic, wrongQueueId, -1); + + assertThat(messageStoreTimeStamp).isEqualTo(-1); + } + + + @Test + public void should_get_message_store_timestamp_successfully_when_incomming_by_topic_queueId_and_consumeQueueOffset() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + String topic = "FooBar"; + AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, false); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + + ConsumeQueue 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 should_return_negative_one_when_invoke_getStoreTime_if_incomming_param_is_null() { + long storeTime = getStoreTime(null); + + assertThat(storeTime).isEqualTo(-1); + } + + @Test + public void should_get_store_time_successfully_when_invoke_getStoreTime_if_everything_is_ok() throws InterruptedException { + final int totalCount = 10; + int queueId = 0; + String topic = "FooBar"; + AppendMessageResult[] appendMessageResults = putMessages(totalCount, topic, queueId, false); + //Thread.sleep(10); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + ConsumeQueue consumeQueue = messageStore.getConsumeQueue(topic, queueId); + + for (int i = 0; i < totalCount; i++) { + SelectMappedBufferResult indexBuffer = consumeQueue.getIndexBuffer(i); + long storeTime = getStoreTime(indexBuffer); + assertThat(storeTime).isEqualTo(appendMessageResults[i].getStoreTimestamp()); + indexBuffer.release(); + } + } + + @Test + public void should_return_negative_one_when_invoke_getStoreTime_if_phyOffset_is_less_than_commitLog_s_minOffset() { + long phyOffset = -10; + int size = 138; + ByteBuffer byteBuffer = ByteBuffer.allocate(100); + byteBuffer.putLong(phyOffset); + byteBuffer.putInt(size); + byteBuffer.flip(); + MappedFile mappedFile = mock(MappedFile.class); + SelectMappedBufferResult result = new SelectMappedBufferResult(0, byteBuffer, size, mappedFile); + + long storeTime = getStoreTime(result); + result.release(); + + assertThat(storeTime).isEqualTo(-1); + } + + private DefaultMessageStore getDefaultMessageStore() { + return (DefaultMessageStore)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 = buildMessage(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(SelectMappedBufferResult result) { + try { + Method getStoreTime = getDefaultMessageStore().getClass().getDeclaredMethod("getStoreTime", SelectMappedBufferResult.class); + getStoreTime.setAccessible(true); + return (long)getStoreTime.invoke(getDefaultMessageStore(), result); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new RuntimeException(e); + } + } + + private MessageExtBrokerInner buildMessage(byte[] messageBody, String topic) { MessageExtBrokerInner msg = new MessageExtBrokerInner(); - msg.setTopic("FooBar"); + msg.setTopic(topic); msg.setTags("TAG1"); msg.setKeys("Hello"); - msg.setBody(MessageBody); + msg.setBody(messageBody); msg.setKeys(String.valueOf(System.currentTimeMillis())); msg.setQueueId(Math.abs(QueueId.getAndIncrement()) % QUEUE_TOTAL); msg.setSysFlag(0); @@ -138,11 +419,17 @@ private MessageExtBrokerInner buildMessage() { return msg; } + private MessageExtBrokerInner buildMessage() { + return buildMessage(MessageBody, "FooBar"); + } + private void verifyThatMasterIsFunctional(long totalMsgs, MessageStore master) { for (long i = 0; i < totalMsgs; i++) { master.putMessage(buildMessage()); } + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + for (long i = 0; i < totalMsgs; i++) { GetMessageResult result = master.getMessage("GROUP_A", "TOPIC_A", 0, i, 1024 * 1024, null); assertThat(result).isNotNull(); @@ -163,16 +450,21 @@ public void testPullSize() throws Exception { } // wait for consume queue build // the sleep time should be great than consume queue flush interval - Thread.sleep(100); + //Thread.sleep(100); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) 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 @@ -186,7 +478,9 @@ public void testRecover() throws Exception { messageStore.putMessage(messageExtBrokerInner); } - Thread.sleep(100);//wait for build consumer queue + // Thread.sleep(100);//wait for build consumer queue + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); + long maxPhyOffset = messageStore.getMaxPhyOffset(); long maxCqOffset = messageStore.getMaxOffsetInQueue(topic, 0); @@ -206,7 +500,8 @@ public void testRecover() throws Exception { messageExtBrokerInner.setQueueId(0); messageStore.putMessage(messageExtBrokerInner); } - Thread.sleep(100); + //Thread.sleep(100); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); long secondLastPhyOffset = messageStore.getMaxPhyOffset(); long secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0); @@ -235,7 +530,8 @@ public void testRecover() throws Exception { messageExtBrokerInner.setQueueId(0); messageStore.putMessage(messageExtBrokerInner); } - Thread.sleep(100); + //Thread.sleep(100); + StoreTestUtil.waitCommitLogReput((DefaultMessageStore) messageStore); secondLastPhyOffset = messageStore.getMaxPhyOffset(); secondLastCqOffset = messageStore.getMaxOffsetInQueue(topic, 0); diff --git a/store/src/test/java/org/apache/rocketmq/store/HATest.java b/store/src/test/java/org/apache/rocketmq/store/HATest.java new file mode 100644 index 00000000000..83efcc1915e --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/HATest.java @@ -0,0 +1,182 @@ +/** + * 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 org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.UtilAll; +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.ha.HAService; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * HATest + * + */ +public class HATest { + private final String StoreMessage = "Once, there was a chance for me!"; + private int QUEUE_TOTAL = 100; + private AtomicInteger QueueId = new AtomicInteger(0); + private SocketAddress BornHost; + private SocketAddress StoreHost; + private byte[] MessageBody; + + private MessageStore messageStore; + private MessageStore slaveMessageStore; + private MessageStoreConfig masterMessageStoreConfig; + private MessageStoreConfig slaveStoreConfig; + private BrokerStatsManager brokerStatsManager = new BrokerStatsManager("simpleTest"); + private String storePathRootParentDir = System.getProperty("user.home") + File.separator + + UUID.randomUUID().toString().replace("-", ""); + private String storePathRootDir = storePathRootParentDir + File.separator + "store"; + @Before + public void init() throws Exception { + StoreHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123); + BornHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0); + masterMessageStoreConfig = new MessageStoreConfig(); + masterMessageStoreConfig.setBrokerRole(BrokerRole.SYNC_MASTER); + masterMessageStoreConfig.setStorePathRootDir(storePathRootDir+File.separator+"master"); + masterMessageStoreConfig.setStorePathCommitLog(storePathRootDir+File.separator+"master"+ File.separator+"commitlog"); + buildMessageStoreConfig(masterMessageStoreConfig); + slaveStoreConfig = new MessageStoreConfig(); + slaveStoreConfig.setBrokerRole(BrokerRole.SLAVE); + slaveStoreConfig.setStorePathRootDir(storePathRootDir+File.separator+"slave"); + slaveStoreConfig.setStorePathCommitLog(storePathRootDir+File.separator+"slave"+ File.separator+"commitlog"); + slaveStoreConfig.setHaListenPort(10943); + buildMessageStoreConfig(slaveStoreConfig); + messageStore = buildMessageStore(masterMessageStoreConfig,0L); + slaveMessageStore = buildMessageStore(slaveStoreConfig,1L); + boolean load = messageStore.load(); + boolean slaveLoad = slaveMessageStore.load(); + slaveMessageStore.updateHaMasterAddress("127.0.0.1:10912"); + assertTrue(load); + assertTrue(slaveLoad); + messageStore.start(); + slaveMessageStore.start(); + Thread.sleep(6000L);//because the haClient will wait 5s after the first connectMaster failed,sleep 6s + } + + @Test + public void testHandleHA() { + long totalMsgs = 10; + QUEUE_TOTAL = 1; + MessageBody = StoreMessage.getBytes(); + for (long i = 0; i < totalMsgs; i++) { + messageStore.putMessage(buildMessage()); + } + + for (int i = 0; i < 100 && isCommitLogAvailable((DefaultMessageStore) messageStore); i++) { + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + } + } + + for (int i = 0; i < 100 && isCommitLogAvailable((DefaultMessageStore) slaveMessageStore); i++) { + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + } + } + + for (long i = 0; i < totalMsgs; i++) { + GetMessageResult result = slaveMessageStore.getMessage("GROUP_A", "FooBar", 0, i, 1024 * 1024, null); + assertThat(result).isNotNull(); + assertTrue(GetMessageStatus.FOUND.equals(result.getStatus())); + result.release(); + } + } + + @After + public void destroy() throws Exception{ + Thread.sleep(5000L); + slaveMessageStore.shutdown(); + slaveMessageStore.destroy(); + messageStore.shutdown(); + messageStore.destroy(); + File file = new File(storePathRootParentDir); + UtilAll.deleteFile(file); + } + + private MessageStore buildMessageStore(MessageStoreConfig messageStoreConfig,long brokerId) throws Exception { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setBrokerId(brokerId); + return new DefaultMessageStore(messageStoreConfig, brokerStatsManager, null, brokerConfig); + } + + private void buildMessageStoreConfig(MessageStoreConfig messageStoreConfig){ + messageStoreConfig.setMapedFileSizeCommitLog(1024 * 1024 * 10); + messageStoreConfig.setMapedFileSizeConsumeQueue(1024 * 1024 * 10); + messageStoreConfig.setMaxHashSlotNum(10000); + messageStoreConfig.setMaxIndexNum(100 * 100); + messageStoreConfig.setFlushDiskType(FlushDiskType.SYNC_FLUSH); + messageStoreConfig.setFlushIntervalConsumeQueue(1); + } + + private MessageExtBrokerInner buildMessage() { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic("FooBar"); + msg.setTags("TAG1"); + msg.setBody(MessageBody); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + msg.setQueueId(Math.abs(QueueId.getAndIncrement()) % QUEUE_TOTAL); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(StoreHost); + msg.setBornHost(BornHost); + return msg; + } + + private boolean isCommitLogAvailable(DefaultMessageStore store) { + try { + + Field serviceField = store.getClass().getDeclaredField("reputMessageService"); + serviceField.setAccessible(true); + DefaultMessageStore.ReputMessageService reputService = + (DefaultMessageStore.ReputMessageService) serviceField.get(store); + + Method method = DefaultMessageStore.ReputMessageService.class.getDeclaredMethod("isCommitLogAvailable"); + method.setAccessible(true); + return (boolean) method.invoke(reputService); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | NoSuchFieldException e ) { + throw new RuntimeException(e); + } + } + +} diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java b/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java new file mode 100644 index 00000000000..0dcb3b254c0 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/StoreTestBase.java @@ -0,0 +1,93 @@ +/* + * 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.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.store.config.StorePathConfigHelper; +import org.junit.After; + +public class StoreTestBase { + + private int QUEUE_TOTAL = 100; + private AtomicInteger QueueId = new AtomicInteger(0); + private SocketAddress BornHost = new InetSocketAddress("127.0.0.1", 8123); + private SocketAddress StoreHost = BornHost; + private byte[] MessageBody = new byte[1024]; + + protected Set baseDirs = new HashSet<>(); + + private static AtomicInteger port = new AtomicInteger(30000); + + public static synchronized int nextPort() { + return port.addAndGet(5); + } + + protected MessageExtBrokerInner buildMessage() { + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic("StoreTest"); + msg.setTags("TAG1"); + msg.setKeys("Hello"); + msg.setBody(MessageBody); + msg.setKeys(String.valueOf(System.currentTimeMillis())); + msg.setQueueId(Math.abs(QueueId.getAndIncrement()) % QUEUE_TOTAL); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(StoreHost); + msg.setBornHost(BornHost); + return msg; + } + + public static String createBaseDir() { + String baseDir = System.getProperty("user.home") + File.separator + "unitteststore" + File.separator + UUID.randomUUID(); + final File file = new File(baseDir); + if (file.exists()) { + System.exit(1); + } + return baseDir; + } + + public static boolean makeSureFileExists(String fileName) throws Exception { + File file = new File(fileName); + MappedFile.ensureDirOK(file.getParent()); + return file.createNewFile(); + } + + + public static void deleteFile(String fileName) { + deleteFile(new File(fileName)); + } + + public static void deleteFile(File file) { + UtilAll.deleteFile(file); + } + + @After + public void clear() { + for (String baseDir : baseDirs) { + deleteFile(baseDir); + } + } + +} diff --git a/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java new file mode 100644 index 00000000000..59809c27cb3 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/StoreTestUtil.java @@ -0,0 +1,89 @@ +/* + * 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 org.apache.rocketmq.logging.InternalLogger; +import org.apache.rocketmq.logging.InternalLoggerFactory; +import org.apache.rocketmq.store.index.IndexFile; +import org.apache.rocketmq.store.index.IndexService; + +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; + + +public class StoreTestUtil { + + private static final InternalLogger log = InternalLoggerFactory.getLogger(StoreTestUtil.class); + + public static boolean isCommitLogAvailable(DefaultMessageStore store) { + try { + + Field serviceField = store.getClass().getDeclaredField("reputMessageService"); + serviceField.setAccessible(true); + DefaultMessageStore.ReputMessageService reputService = + (DefaultMessageStore.ReputMessageService) serviceField.get(store); + + Method method = DefaultMessageStore.ReputMessageService.class.getDeclaredMethod("isCommitLogAvailable"); + method.setAccessible(true); + return (boolean) method.invoke(reputService); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | NoSuchFieldException e) { + throw new RuntimeException(e); + } + } + + public static void flushConsumeQueue(DefaultMessageStore store) throws Exception { + Field field = store.getClass().getDeclaredField("flushConsumeQueueService"); + field.setAccessible(true); + DefaultMessageStore.FlushConsumeQueueService flushService = (DefaultMessageStore.FlushConsumeQueueService) field.get(store); + + final int RETRY_TIMES_OVER = 3; + Method method = DefaultMessageStore.FlushConsumeQueueService.class.getDeclaredMethod("doFlush", int.class); + method.setAccessible(true); + method.invoke(flushService, RETRY_TIMES_OVER); + } + + + public static void waitCommitLogReput(DefaultMessageStore store) { + for (int i = 0; i < 500 && isCommitLogAvailable(store); i++) { + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + } + } + + if (isCommitLogAvailable(store)) { + log.warn("isCommitLogAvailable expected false ,but true"); + } + } + + + public static void flushConsumeIndex(DefaultMessageStore store) throws NoSuchFieldException, Exception { + Field field = store.getClass().getDeclaredField("indexService"); + field.setAccessible(true); + IndexService indexService = (IndexService) field.get(store); + + Field field2 = indexService.getClass().getDeclaredField("indexFileList"); + field2.setAccessible(true); + ArrayList indexFileList = (ArrayList) field2.get(indexService); + + for (IndexFile f : indexFileList) { + indexService.flush(f); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java new file mode 100644 index 00000000000..d0829d122a1 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/DLedgerCommitlogTest.java @@ -0,0 +1,213 @@ +/* + * 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.dledger; + +import io.openmessaging.storage.dledger.DLedgerServer; +import io.openmessaging.storage.dledger.store.file.DLedgerMmapFileStore; +import io.openmessaging.storage.dledger.store.file.MmapFileList; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.GetMessageStatus; +import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.junit.Assert; +import org.junit.Test; + +public class DLedgerCommitlogTest extends MessageStoreTestBase { + + + @Test + public void testTruncateCQ() throws Exception { + String base = createBaseDir(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + { + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, false, 0); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog(); + DLedgerServer dLedgerServer = dLedgerCommitLog.getdLedgerServer(); + DLedgerMmapFileStore dLedgerMmapFileStore = (DLedgerMmapFileStore) dLedgerServer.getdLedgerStore(); + MmapFileList mmapFileList = dLedgerMmapFileStore.getDataFileList(); + Thread.sleep(2000); + doPutMessages(messageStore, topic, 0, 2000, 0); + Thread.sleep(100); + Assert.assertEquals(24, mmapFileList.getMappedFiles().size()); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(2000, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + doGetMessages(messageStore, topic, 0, 2000, 0); + messageStore.shutdown(); + } + + { + //Abnormal recover, left some commitlogs + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 4); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog(); + DLedgerServer dLedgerServer = dLedgerCommitLog.getdLedgerServer(); + DLedgerMmapFileStore dLedgerMmapFileStore = (DLedgerMmapFileStore) dLedgerServer.getdLedgerStore(); + MmapFileList mmapFileList = dLedgerMmapFileStore.getDataFileList(); + Thread.sleep(1000); + Assert.assertEquals(20, mmapFileList.getMappedFiles().size()); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1700, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + doGetMessages(messageStore, topic, 0, 1700, 0); + messageStore.shutdown(); + } + { + //Abnormal recover, left none commitlogs + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 20); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) messageStore.getCommitLog(); + DLedgerServer dLedgerServer = dLedgerCommitLog.getdLedgerServer(); + DLedgerMmapFileStore dLedgerMmapFileStore = (DLedgerMmapFileStore) dLedgerServer.getdLedgerStore(); + MmapFileList mmapFileList = dLedgerMmapFileStore.getDataFileList(); + Thread.sleep(1000); + Assert.assertEquals(0, mmapFileList.getMappedFiles().size()); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + messageStore.shutdown(); + } + } + + + + @Test + public void testRecover() throws Exception { + String base = createBaseDir(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + String topic = UUID.randomUUID().toString(); + { + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, false, 0); + Thread.sleep(1000); + doPutMessages(messageStore, topic, 0, 1000, 0); + Thread.sleep(100); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + doGetMessages(messageStore, topic, 0, 1000, 0); + messageStore.shutdown(); + } + + { + //normal recover + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, false, 0); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + doGetMessages(messageStore, topic, 0, 1000, 0); + messageStore.shutdown(); + } + + { + //abnormal recover + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 0); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + doGetMessages(messageStore, topic, 0, 1000, 0); + messageStore.shutdown(); + } + } + + + + @Test + public void testPutAndGetMessage() throws Exception { + String base = createBaseDir(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + DefaultMessageStore messageStore = createDledgerMessageStore(base, group, "n0", peers, null, false, 0); + Thread.sleep(1000); + String topic = UUID.randomUUID().toString(); + + List results = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + MessageExtBrokerInner msgInner = buildMessage(); + msgInner.setTopic(topic); + msgInner.setQueueId(0); + PutMessageResult putMessageResult = messageStore.putMessage(msgInner); + results.add(putMessageResult); + Assert.assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + Assert.assertEquals(i, putMessageResult.getAppendMessageResult().getLogicsOffset()); + } + Thread.sleep(100); + Assert.assertEquals(0, messageStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(10, messageStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, messageStore.dispatchBehindBytes()); + GetMessageResult getMessageResult = messageStore.getMessage("group", topic, 0, 0, 32, null); + Assert.assertEquals(GetMessageStatus.FOUND, getMessageResult.getStatus()); + + Assert.assertEquals(10, getMessageResult.getMessageBufferList().size()); + Assert.assertEquals(10, getMessageResult.getMessageMapedList().size()); + + for (int i = 0; i < results.size(); i++) { + ByteBuffer buffer = getMessageResult.getMessageBufferList().get(i); + MessageExt messageExt = MessageDecoder.decode(buffer); + Assert.assertEquals(i, messageExt.getQueueOffset()); + Assert.assertEquals(results.get(i).getAppendMessageResult().getMsgId(), messageExt.getMsgId()); + Assert.assertEquals(results.get(i).getAppendMessageResult().getWroteOffset(), messageExt.getCommitLogOffset()); + } + messageStore.destroy(); + messageStore.shutdown(); + } + + + @Test + public void testCommittedPos() throws Exception { + String peers = String.format("n0-localhost:%d;n1-localhost:%d", nextPort(), nextPort()); + String group = UUID.randomUUID().toString(); + DefaultMessageStore leaderStore = createDledgerMessageStore(createBaseDir(), group,"n0", peers, "n0", false, 0); + + String topic = UUID.randomUUID().toString(); + MessageExtBrokerInner msgInner = buildMessage(); + msgInner.setTopic(topic); + msgInner.setQueueId(0); + PutMessageResult putMessageResult = leaderStore.putMessage(msgInner); + Assert.assertEquals(PutMessageStatus.OS_PAGECACHE_BUSY, putMessageResult.getPutMessageStatus()); + + Thread.sleep(1000); + + Assert.assertEquals(0, leaderStore.getCommitLog().getMaxOffset()); + Assert.assertEquals(0, leaderStore.getMaxOffsetInQueue(topic, 0)); + + + DefaultMessageStore followerStore = createDledgerMessageStore(createBaseDir(), group,"n1", peers, "n0", false, 0); + Thread.sleep(2000); + + Assert.assertEquals(1, leaderStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(1, followerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertTrue(leaderStore.getCommitLog().getMaxOffset() > 0); + + + leaderStore.destroy(); + followerStore.destroy(); + + leaderStore.shutdown(); + followerStore.shutdown(); + } + + +} diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java new file mode 100644 index 00000000000..e45178aaee9 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MessageStoreTestBase.java @@ -0,0 +1,143 @@ +/* + * 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.dledger; + +import io.openmessaging.storage.dledger.DLedgerConfig; +import io.openmessaging.storage.dledger.DLedgerServer; +import java.io.File; +import java.util.Arrays; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.GetMessageResult; +import org.apache.rocketmq.store.MessageExtBrokerInner; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.StoreTestBase; +import org.apache.rocketmq.store.config.FlushDiskType; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.config.StorePathConfigHelper; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.Assert; + +public class MessageStoreTestBase extends StoreTestBase { + + protected DefaultMessageStore createDledgerMessageStore(String base, String group, String selfId, String peers, String leaderId, boolean createAbort, int deleteFileNum) throws Exception { + System.setProperty("dledger.disk.ratio.check", "0.95"); + System.setProperty("dledger.disk.ratio.clean", "0.95"); + baseDirs.add(base); + MessageStoreConfig storeConfig = new MessageStoreConfig(); + storeConfig.setMapedFileSizeCommitLog(1024 * 100); + storeConfig.setMapedFileSizeConsumeQueue(1024); + storeConfig.setMaxHashSlotNum(100); + storeConfig.setMaxIndexNum(100 * 10); + storeConfig.setStorePathRootDir(base); + storeConfig.setStorePathCommitLog(base + File.separator + "commitlog"); + storeConfig.setFlushDiskType(FlushDiskType.ASYNC_FLUSH); + + storeConfig.setEnableDLegerCommitLog(true); + storeConfig.setdLegerGroup(group); + storeConfig.setdLegerPeers(peers); + storeConfig.setdLegerSelfId(selfId); + DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("DLedgerCommitlogTest"), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + + }, new BrokerConfig()); + DLedgerServer dLegerServer = ((DLedgerCommitLog) defaultMessageStore.getCommitLog()).getdLedgerServer(); + if (leaderId != null) { + dLegerServer.getdLedgerConfig().setEnableLeaderElector(false); + if (selfId.equals(leaderId)) { + dLegerServer.getMemberState().changeToLeader(-1); + } else { + dLegerServer.getMemberState().changeToFollower(-1, leaderId); + } + + } + if (createAbort) { + String fileName = StorePathConfigHelper.getAbortFile(storeConfig.getStorePathRootDir()); + makeSureFileExists(fileName); + } + if (deleteFileNum > 0) { + DLedgerConfig config = dLegerServer.getdLedgerConfig(); + if (deleteFileNum > 0) { + File dir = new File(config.getDataStorePath()); + File[] files = dir.listFiles(); + if (files != null) { + Arrays.sort(files); + for (int i = files.length - 1; i >= 0; i--) { + File file = files[i]; + file.delete(); + if (files.length - i >= deleteFileNum) { + break; + } + } + } + } + } + Assert.assertTrue(defaultMessageStore.load()); + defaultMessageStore.start(); + return defaultMessageStore; + } + + + protected DefaultMessageStore createMessageStore(String base, boolean createAbort) throws Exception { + baseDirs.add(base); + MessageStoreConfig storeConfig = new MessageStoreConfig(); + storeConfig.setMapedFileSizeCommitLog(1024 * 100); + storeConfig.setMapedFileSizeConsumeQueue(1024); + storeConfig.setMaxHashSlotNum(100); + storeConfig.setMaxIndexNum(100 * 10); + storeConfig.setStorePathRootDir(base); + storeConfig.setStorePathCommitLog(base + File.separator + "commitlog"); + storeConfig.setFlushDiskType(FlushDiskType.ASYNC_FLUSH); + DefaultMessageStore defaultMessageStore = new DefaultMessageStore(storeConfig, new BrokerStatsManager("CommitlogTest"), (topic, queueId, logicOffset, tagsCode, msgStoreTime, filterBitMap, properties) -> { + + }, new BrokerConfig()); + + if (createAbort) { + String fileName = StorePathConfigHelper.getAbortFile(storeConfig.getStorePathRootDir()); + makeSureFileExists(fileName); + } + Assert.assertTrue(defaultMessageStore.load()); + defaultMessageStore.start(); + return defaultMessageStore; + } + + protected void doPutMessages(MessageStore messageStore, String topic, int queueId, int num, long beginLogicsOffset) { + for (int i = 0; i < num; i++) { + MessageExtBrokerInner msgInner = buildMessage(); + msgInner.setTopic(topic); + msgInner.setQueueId(queueId); + PutMessageResult putMessageResult = messageStore.putMessage(msgInner); + Assert.assertEquals(PutMessageStatus.PUT_OK, putMessageResult.getPutMessageStatus()); + Assert.assertEquals(beginLogicsOffset + i, putMessageResult.getAppendMessageResult().getLogicsOffset()); + } + } + + protected void doGetMessages(MessageStore messageStore, String topic, int queueId, int num, long beginLogicsOffset) { + for (int i = 0; i < num; i++) { + GetMessageResult getMessageResult = messageStore.getMessage("group", topic, queueId, beginLogicsOffset + i, 3, null); + Assert.assertNotNull(getMessageResult); + Assert.assertTrue(!getMessageResult.getMessageBufferList().isEmpty()); + MessageExt messageExt = MessageDecoder.decode(getMessageResult.getMessageBufferList().get(0)); + Assert.assertEquals(beginLogicsOffset + i, messageExt.getQueueOffset()); + getMessageResult.release(); + } + } + +} diff --git a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java new file mode 100644 index 00000000000..a5242a35105 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java @@ -0,0 +1,193 @@ +/* + * 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.dledger; + +import java.util.UUID; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.StoreTestBase; +import org.apache.rocketmq.store.config.StorePathConfigHelper; +import org.junit.Assert; +import org.junit.Test; + +public class MixCommitlogTest extends MessageStoreTestBase { + + + + @Test + public void testFallBehindCQ() throws Exception { + String base = createBaseDir(); + String topic = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + { + DefaultMessageStore originalStore = createMessageStore(base, false); + doPutMessages(originalStore, topic, 0, 1000, 0); + Assert.assertEquals(11, originalStore.getMaxPhyOffset()/originalStore.getMessageStoreConfig().getMapedFileSizeCommitLog()); + Thread.sleep(500); + Assert.assertEquals(0, originalStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, originalStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, originalStore.dispatchBehindBytes()); + doGetMessages(originalStore, topic, 0, 1000, 0); + originalStore.shutdown(); + } + //delete the cq files + { + StoreTestBase.deleteFile(StorePathConfigHelper.getStorePathConsumeQueue(base)); + } + { + DefaultMessageStore dledgerStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 0); + Thread.sleep(2000); + Assert.assertEquals(0, dledgerStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, dledgerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, dledgerStore.dispatchBehindBytes()); + doGetMessages(dledgerStore, topic, 0, 1000, 0); + doPutMessages(dledgerStore, topic, 0, 1000, 1000); + Thread.sleep(500); + Assert.assertEquals(0, dledgerStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(2000, dledgerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, dledgerStore.dispatchBehindBytes()); + doGetMessages(dledgerStore, topic, 0, 2000, 0); + dledgerStore.shutdown(); + } + } + + + + @Test + public void testPutAndGet() throws Exception { + String base = createBaseDir(); + String topic = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + + long dividedOffset; + { + DefaultMessageStore originalStore = createMessageStore(base, false); + doPutMessages(originalStore, topic, 0, 1000, 0); + Thread.sleep(500); + Assert.assertEquals(0, originalStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, originalStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, originalStore.dispatchBehindBytes()); + dividedOffset = originalStore.getCommitLog().getMaxOffset(); + dividedOffset = dividedOffset - dividedOffset % originalStore.getMessageStoreConfig().getMapedFileSizeCommitLog() + originalStore.getMessageStoreConfig().getMapedFileSizeCommitLog(); + doGetMessages(originalStore, topic, 0, 1000, 0); + originalStore.shutdown(); + } + { + DefaultMessageStore recoverOriginalStore = createMessageStore(base, true); + Thread.sleep(500); + Assert.assertEquals(0, recoverOriginalStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, recoverOriginalStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, recoverOriginalStore.dispatchBehindBytes()); + doGetMessages(recoverOriginalStore, topic, 0, 1000, 0); + recoverOriginalStore.shutdown(); + } + { + DefaultMessageStore dledgerStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 0); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) dledgerStore.getCommitLog(); + Assert.assertFalse(dLedgerCommitLog.getdLedgerServer().getdLedgerConfig().isEnableDiskForceClean()); + Assert.assertEquals(dividedOffset, dLedgerCommitLog.getDividedCommitlogOffset()); + Assert.assertEquals(0, dledgerStore.dispatchBehindBytes()); + Assert.assertEquals(dividedOffset, dLedgerCommitLog.getMaxOffset()); + Thread.sleep(2000); + doPutMessages(dledgerStore, topic, 0, 1000, 1000); + Thread.sleep(500); + Assert.assertEquals(0, dledgerStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(2000, dledgerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, dledgerStore.dispatchBehindBytes()); + doGetMessages(dledgerStore, topic, 0, 2000, 0); + dledgerStore.shutdown(); + } + { + DefaultMessageStore recoverDledgerStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 0); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) recoverDledgerStore.getCommitLog(); + Assert.assertFalse(dLedgerCommitLog.getdLedgerServer().getdLedgerConfig().isEnableDiskForceClean()); + Assert.assertEquals(dividedOffset, dLedgerCommitLog.getDividedCommitlogOffset()); + Thread.sleep(2000); + doPutMessages(recoverDledgerStore, topic, 0, 1000, 2000); + Thread.sleep(500); + Assert.assertEquals(0, recoverDledgerStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(3000, recoverDledgerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, recoverDledgerStore.dispatchBehindBytes()); + doGetMessages(recoverDledgerStore, topic, 0, 3000, 0); + recoverDledgerStore.shutdown(); + } + } + + @Test + public void testDeleteExpiredFiles() throws Exception { + String base = createBaseDir(); + String topic = UUID.randomUUID().toString(); + String peers = String.format("n0-localhost:%d", nextPort()); + String group = UUID.randomUUID().toString(); + + long dividedOffset; + { + DefaultMessageStore originalStore = createMessageStore(base, false); + doPutMessages(originalStore, topic, 0, 1000, 0); + Thread.sleep(500); + Assert.assertEquals(0, originalStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(1000, originalStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, originalStore.dispatchBehindBytes()); + dividedOffset = originalStore.getCommitLog().getMaxOffset(); + dividedOffset = dividedOffset - dividedOffset % originalStore.getMessageStoreConfig().getMapedFileSizeCommitLog() + originalStore.getMessageStoreConfig().getMapedFileSizeCommitLog(); + originalStore.shutdown(); + } + long maxPhysicalOffset; + { + DefaultMessageStore dledgerStore = createDledgerMessageStore(base, group, "n0", peers, null, true, 0); + DLedgerCommitLog dLedgerCommitLog = (DLedgerCommitLog) dledgerStore.getCommitLog(); + Assert.assertTrue(dledgerStore.getMessageStoreConfig().isCleanFileForciblyEnable()); + Assert.assertFalse(dLedgerCommitLog.getdLedgerServer().getdLedgerConfig().isEnableDiskForceClean()); + Assert.assertEquals(dividedOffset, dLedgerCommitLog.getDividedCommitlogOffset()); + Thread.sleep(2000); + doPutMessages(dledgerStore, topic, 0, 1000, 1000); + Thread.sleep(500); + Assert.assertEquals(0, dledgerStore.getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(2000, dledgerStore.getMaxOffsetInQueue(topic, 0)); + Assert.assertEquals(0, dledgerStore.dispatchBehindBytes()); + Assert.assertEquals(0, dledgerStore.getMinPhyOffset()); + maxPhysicalOffset = dledgerStore.getMaxPhyOffset(); + Assert.assertTrue(maxPhysicalOffset > 0); + + doGetMessages(dledgerStore, topic, 0, 2000, 0); + + for (int i = 0; i < 100; i++) { + dledgerStore.getCommitLog().deleteExpiredFile(System.currentTimeMillis(), 0, 0, true); + } + Assert.assertEquals(dividedOffset, dledgerStore.getMinPhyOffset()); + Assert.assertEquals(maxPhysicalOffset, dledgerStore.getMaxPhyOffset()); + for (int i = 0; i < 100; i++) { + Assert.assertEquals(Integer.MAX_VALUE, dledgerStore.getCommitLog().deleteExpiredFile(System.currentTimeMillis(), 0, 0, true)); + } + Assert.assertEquals(dividedOffset, dledgerStore.getMinPhyOffset()); + Assert.assertEquals(maxPhysicalOffset, dledgerStore.getMaxPhyOffset()); + + Assert.assertTrue(dledgerStore.getMessageStoreConfig().isCleanFileForciblyEnable()); + Assert.assertTrue(dLedgerCommitLog.getdLedgerServer().getdLedgerConfig().isEnableDiskForceClean()); + + //Test fresh + dledgerStore.getMessageStoreConfig().setCleanFileForciblyEnable(false); + for (int i = 0; i < 100; i++) { + Assert.assertEquals(Integer.MAX_VALUE, dledgerStore.getCommitLog().deleteExpiredFile(System.currentTimeMillis(), 0, 0, true)); + } + Assert.assertFalse(dLedgerCommitLog.getdLedgerServer().getdLedgerConfig().isEnableDiskForceClean()); + doGetMessages(dledgerStore, topic, 0, 1000, 1000); + dledgerStore.shutdown(); + } + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java b/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java new file mode 100644 index 00000000000..befbefd28d6 --- /dev/null +++ b/store/src/test/java/org/apache/rocketmq/store/schedule/ScheduleMessageServiceTest.java @@ -0,0 +1,224 @@ +/* + * 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.schedule; + +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.store.*; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.assertj.core.api.Assertions.assertThat; + + +public class ScheduleMessageServiceTest { + + + /** + * t + * defaultMessageDelayLevel = "1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h 2h" + */ + String testMessageDelayLevel = "5s 8s"; + /** + * choose delay level + */ + int delayLevel = 2; + + private static final String storePath = System.getProperty("user.home") + File.separator + "schedule_test#" + UUID.randomUUID(); + private static final int commitLogFileSize = 1024; + private static final int cqFileSize = 10; + private static final int cqExtFileSize = 10 * (ConsumeQueueExt.CqExtUnit.MIN_EXT_UNIT_SIZE + 64); + + private static SocketAddress bornHost; + private static SocketAddress storeHost; + DefaultMessageStore messageStore; + MessageStoreConfig messageStoreConfig; + BrokerConfig brokerConfig; + ScheduleMessageService scheduleMessageService; + + static String sendMessage = " ------- schedule message test -------"; + static String topic = "schedule_topic_test"; + static String messageGroup = "delayGroupTest"; + + + static { + try { + bornHost = new InetSocketAddress(InetAddress.getLocalHost(), 8123); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + try { + storeHost = new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 0); + } catch (UnknownHostException e) { + e.printStackTrace(); + } + } + + + @Before + public void init() throws Exception { + messageStoreConfig = new MessageStoreConfig(); + messageStoreConfig.setMessageDelayLevel(testMessageDelayLevel); + messageStoreConfig.setMapedFileSizeCommitLog(commitLogFileSize); + messageStoreConfig.setMapedFileSizeConsumeQueue(cqFileSize); + messageStoreConfig.setMappedFileSizeConsumeQueueExt(cqExtFileSize); + messageStoreConfig.setMessageIndexEnable(false); + messageStoreConfig.setEnableConsumeQueueExt(true); + messageStoreConfig.setStorePathRootDir(storePath); + messageStoreConfig.setStorePathCommitLog(storePath + File.separator + "commitlog"); + + brokerConfig = new BrokerConfig(); + BrokerStatsManager manager = new BrokerStatsManager(brokerConfig.getBrokerClusterName()); + messageStore = new DefaultMessageStore(messageStoreConfig, manager, new MyMessageArrivingListener(), new BrokerConfig()); + + assertThat(messageStore.load()).isTrue(); + + messageStore.start(); + scheduleMessageService = messageStore.getScheduleMessageService(); + } + + + @Test + public void deliverDelayedMessageTimerTaskTest() throws Exception { + MessageExtBrokerInner msg = buildMessage(); + int realQueueId = msg.getQueueId(); + // set delayLevel,and send delay message + msg.setDelayTimeLevel(delayLevel); + PutMessageResult result = messageStore.putMessage(msg); + assertThat(result.isOk()).isTrue(); + + // make sure consumerQueue offset = commitLog offset + StoreTestUtil.waitCommitLogReput(messageStore); + + // consumer message + int delayQueueId = ScheduleMessageService.delayLevel2QueueId(delayLevel); + assertThat(delayQueueId).isEqualTo(delayLevel - 1); + + Long offset = result.getAppendMessageResult().getLogicsOffset(); + + // now, no message in queue,must wait > delayTime + GetMessageResult messageResult = getMessage(realQueueId, offset); + assertThat(messageResult.getStatus()).isEqualTo(GetMessageStatus.NO_MESSAGE_IN_QUEUE); + + // timer run maybe delay, then consumer message again + // and wait offsetTable + TimeUnit.SECONDS.sleep(10); + scheduleMessageService.buildRunningStats(new HashMap()); + + messageResult = getMessage(realQueueId, offset); + // now,found the message + assertThat(messageResult.getStatus()).isEqualTo(GetMessageStatus.FOUND); + + + // get the message body + ByteBuffer byteBuffer = ByteBuffer.allocate(messageResult.getBufferTotalSize()); + List byteBufferList = messageResult.getMessageBufferList(); + for (ByteBuffer bb : byteBufferList) { + byteBuffer.put(bb); + } + + // warp and decode the message + byteBuffer = ByteBuffer.wrap(byteBuffer.array()); + List msgList = MessageDecoder.decodes(byteBuffer); + String retryMsg = new String(msgList.get(0).getBody()); + assertThat(sendMessage).isEqualTo(retryMsg); + + // method will wait 10s,so I run it by myself + scheduleMessageService.persist(); + + // add mapFile release + messageResult.release(); + + } + + /** + * add some [error/no use] code test + */ + @Test + public void otherTest() { + // the method no use ,why need ? + int queueId = ScheduleMessageService.queueId2DelayLevel(delayLevel); + assertThat(queueId).isEqualTo(delayLevel + 1); + + // error delayLevelTest + Long time = scheduleMessageService.computeDeliverTimestamp(999, 0); + assertThat(time).isEqualTo(1000); + + // just decode + scheduleMessageService.decode(new DelayOffsetSerializeWrapper().toJson()); + } + + + private GetMessageResult getMessage(int queueId, Long offset) { + return messageStore.getMessage(messageGroup, topic, + queueId, offset, 1, null); + + } + + + @After + public void shutdown() throws InterruptedException { + messageStore.shutdown(); + messageStore.destroy(); + File file = new File(messageStoreConfig.getStorePathRootDir()); + UtilAll.deleteFile(file); + } + + + public MessageExtBrokerInner buildMessage() { + + byte[] msgBody = sendMessage.getBytes(); + MessageExtBrokerInner msg = new MessageExtBrokerInner(); + msg.setTopic(topic); + msg.setTags("schedule_tag"); + msg.setKeys("schedule_key"); + msg.setBody(msgBody); + msg.setSysFlag(0); + msg.setBornTimestamp(System.currentTimeMillis()); + msg.setStoreHost(storeHost); + msg.setBornHost(bornHost); + return msg; + } + + + private class MyMessageArrivingListener implements MessageArrivingListener { + @Override + public void arriving(String topic, int queueId, long logicOffset, long tagsCode, long msgStoreTime, + byte[] filterBitMap, Map properties) { + } + } + + +} diff --git a/store/src/test/resources/logback-test.xml b/store/src/test/resources/logback-test.xml index 875b6715ac9..a033816ddad 100644 --- a/store/src/test/resources/logback-test.xml +++ b/store/src/test/resources/logback-test.xml @@ -28,7 +28,7 @@ - + diff --git a/test/src/main/java/org/apache/rocketmq/test/factory/ConsumerFactory.java b/test/src/main/java/org/apache/rocketmq/test/factory/ConsumerFactory.java index 0f94f3606c5..48508462668 100644 --- a/test/src/main/java/org/apache/rocketmq/test/factory/ConsumerFactory.java +++ b/test/src/main/java/org/apache/rocketmq/test/factory/ConsumerFactory.java @@ -17,6 +17,8 @@ package org.apache.rocketmq.test.factory; +import java.util.UUID; +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; import org.apache.rocketmq.client.consumer.MessageSelector; import org.apache.rocketmq.test.client.rmq.RMQBroadCastConsumer; import org.apache.rocketmq.test.client.rmq.RMQNormalConsumer; @@ -60,4 +62,12 @@ public static RMQSqlConsumer getRMQSqlConsumer(String nsAddr, String consumerGro consumer.start(); return consumer; } + + public static DefaultMQPullConsumer getRMQPullConsumer(String nsAddr, String consumerGroup) throws Exception { + DefaultMQPullConsumer defaultMQPullConsumer = new DefaultMQPullConsumer(consumerGroup); + defaultMQPullConsumer.setInstanceName(UUID.randomUUID().toString()); + defaultMQPullConsumer.setNamesrvAddr(nsAddr); + defaultMQPullConsumer.start(); + return defaultMQPullConsumer; + } } diff --git a/test/src/main/java/org/apache/rocketmq/test/factory/ProducerFactory.java b/test/src/main/java/org/apache/rocketmq/test/factory/ProducerFactory.java index 66767cc9f0d..76e6e09d0bf 100644 --- a/test/src/main/java/org/apache/rocketmq/test/factory/ProducerFactory.java +++ b/test/src/main/java/org/apache/rocketmq/test/factory/ProducerFactory.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.test.factory; +import java.util.UUID; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.producer.DefaultMQProducer; import org.apache.rocketmq.test.util.RandomUtil; @@ -25,6 +26,7 @@ public class ProducerFactory { public static DefaultMQProducer getRMQProducer(String ns) { DefaultMQProducer producer = new DefaultMQProducer(RandomUtil.getStringByUUID()); + producer.setInstanceName(UUID.randomUUID().toString()); producer.setNamesrvAddr(ns); try { producer.start(); diff --git a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java index bd151d0561e..8863ee3e52d 100644 --- a/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java +++ b/test/src/main/java/org/apache/rocketmq/test/util/MQAdmin.java @@ -19,6 +19,7 @@ import java.util.HashMap; import java.util.Set; +import java.util.UUID; import org.apache.log4j.Logger; import org.apache.rocketmq.common.admin.TopicStatsTable; import org.apache.rocketmq.common.protocol.body.ClusterInfo; @@ -40,6 +41,7 @@ public static boolean createTopic(String nameSrvAddr, String clusterName, String int queueNum, int waitTimeSec) { boolean createResult = false; DefaultMQAdminExt mqAdminExt = new DefaultMQAdminExt(); + mqAdminExt.setInstanceName(UUID.randomUUID().toString()); mqAdminExt.setNamesrvAddr(nameSrvAddr); try { mqAdminExt.start(); diff --git a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java index 780bd47506d..45c6750dfdf 100644 --- a/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java +++ b/test/src/test/java/org/apache/rocketmq/test/base/BaseConf.java @@ -36,13 +36,13 @@ import org.apache.rocketmq.test.util.MQRandomUtils; public class BaseConf { - protected static String nsAddr; + public static String nsAddr; protected static String broker1Name; protected static String broker2Name; protected static String clusterName; protected static int brokerNum; protected static int waitTime = 5; - protected static int consumeTime = 5 * 60 * 1000; + protected static int consumeTime = 2 * 60 * 1000; protected static NamesrvController namesrvController; protected static BrokerController brokerController1; protected static BrokerController brokerController2; diff --git a/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java b/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java index 9dcb2d8b755..0eacd584fad 100644 --- a/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java +++ b/test/src/test/java/org/apache/rocketmq/test/base/IntegrationTestBase.java @@ -47,9 +47,14 @@ public class IntegrationTestBase { protected static final List BROKER_CONTROLLERS = new ArrayList<>(); protected static final List NAMESRV_CONTROLLERS = new ArrayList<>(); protected static int topicCreateTime = 30 * 1000; - protected static final int COMMIT_LOG_SIZE = 1024 * 1024 * 256; + protected static final int COMMIT_LOG_SIZE = 1024 * 1024 * 100; protected static final int INDEX_NUM = 1000; + private static final AtomicInteger port = new AtomicInteger(40000); + + public static synchronized int nextPort() { + return port.addAndGet(random.nextInt(10) + 10); + } protected static Random random = new Random(); static { @@ -87,7 +92,7 @@ public void run() { } - private static String createBaseDir() { + public static String createBaseDir() { String baseDir = System.getProperty("user.home") + SEP + "unitteststore-" + UUID.randomUUID(); final File file = new File(baseDir); if (file.exists()) { @@ -105,14 +110,14 @@ public static NamesrvController createAndStartNamesrv() { namesrvConfig.setKvConfigPath(baseDir + SEP + "namesrv" + SEP + "kvConfig.json"); namesrvConfig.setConfigStorePath(baseDir + SEP + "namesrv" + SEP + "namesrv.properties"); - nameServerNettyServerConfig.setListenPort(9000 + random.nextInt(1000)); + nameServerNettyServerConfig.setListenPort(nextPort()); NamesrvController namesrvController = new NamesrvController(namesrvConfig, nameServerNettyServerConfig); try { Assert.assertTrue(namesrvController.initialize()); logger.info("Name Server Start:{}", nameServerNettyServerConfig.getListenPort()); namesrvController.start(); } catch (Exception e) { - logger.info("Name Server start failed"); + logger.info("Name Server start failed", e); System.exit(1); } NAMESRV_CONTROLLERS.add(namesrvController); @@ -123,8 +128,6 @@ public static NamesrvController createAndStartNamesrv() { public static BrokerController createAndStartBroker(String nsAddr) { String baseDir = createBaseDir(); BrokerConfig brokerConfig = new BrokerConfig(); - NettyServerConfig nettyServerConfig = new NettyServerConfig(); - NettyClientConfig nettyClientConfig = new NettyClientConfig(); MessageStoreConfig storeConfig = new MessageStoreConfig(); brokerConfig.setBrokerName(BROKER_NAME_PREFIX + BROKER_INDEX.getAndIncrement()); brokerConfig.setBrokerIP1("127.0.0.1"); @@ -132,18 +135,25 @@ public static BrokerController createAndStartBroker(String nsAddr) { brokerConfig.setEnablePropertyFilter(true); storeConfig.setStorePathRootDir(baseDir); storeConfig.setStorePathCommitLog(baseDir + SEP + "commitlog"); - storeConfig.setHaListenPort(8000 + random.nextInt(1000)); storeConfig.setMapedFileSizeCommitLog(COMMIT_LOG_SIZE); storeConfig.setMaxIndexNum(INDEX_NUM); storeConfig.setMaxHashSlotNum(INDEX_NUM * 4); - nettyServerConfig.setListenPort(10000 + random.nextInt(1000)); + return createAndStartBroker(storeConfig, brokerConfig); + + } + + public static BrokerController createAndStartBroker(MessageStoreConfig storeConfig, BrokerConfig brokerConfig) { + NettyServerConfig nettyServerConfig = new NettyServerConfig(); + NettyClientConfig nettyClientConfig = new NettyClientConfig(); + nettyServerConfig.setListenPort(nextPort()); + storeConfig.setHaListenPort(nextPort()); BrokerController brokerController = new BrokerController(brokerConfig, nettyServerConfig, nettyClientConfig, storeConfig); try { Assert.assertTrue(brokerController.initialize()); logger.info("Broker Start name:{} addr:{}", brokerConfig.getBrokerName(), brokerController.getBrokerAddr()); brokerController.start(); - } catch (Exception e) { - logger.info("Broker start failed"); + } catch (Throwable t) { + logger.error("Broker start failed, will exit", t); System.exit(1); } BROKER_CONTROLLERS.add(brokerController); @@ -179,15 +189,7 @@ public static void deleteFile(File file) { if (!file.exists()) { return; } - if (file.isFile()) { - file.delete(); - } else if (file.isDirectory()) { - File[] files = file.listFiles(); - for (File file1 : files) { - deleteFile(file1); - } - file.delete(); - } + UtilAll.deleteFile(file); } } diff --git a/test/src/test/java/org/apache/rocketmq/test/base/dledger/DLedgerProduceAndConsumeIT.java b/test/src/test/java/org/apache/rocketmq/test/base/dledger/DLedgerProduceAndConsumeIT.java new file mode 100644 index 00000000000..e2d9db1983e --- /dev/null +++ b/test/src/test/java/org/apache/rocketmq/test/base/dledger/DLedgerProduceAndConsumeIT.java @@ -0,0 +1,121 @@ +/* + * 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.base.dledger; + +import java.util.UUID; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer; +import org.apache.rocketmq.client.consumer.PullResult; +import org.apache.rocketmq.client.consumer.PullStatus; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.client.producer.SendStatus; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.store.config.BrokerRole; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.test.base.BaseConf; +import org.apache.rocketmq.test.base.IntegrationTestBase; +import org.apache.rocketmq.test.factory.ConsumerFactory; +import org.apache.rocketmq.test.factory.ProducerFactory; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import static org.apache.rocketmq.test.base.IntegrationTestBase.nextPort; +import static sun.util.locale.BaseLocale.SEP; + +public class DLedgerProduceAndConsumeIT { + + public BrokerConfig buildBrokerConfig(String cluster, String brokerName) { + BrokerConfig brokerConfig = new BrokerConfig(); + brokerConfig.setBrokerClusterName(cluster); + brokerConfig.setBrokerName(brokerName); + brokerConfig.setBrokerIP1("127.0.0.1"); + brokerConfig.setNamesrvAddr(BaseConf.nsAddr); + return brokerConfig; + } + + public MessageStoreConfig buildStoreConfig(String brokerName, String peers, String selfId) { + MessageStoreConfig storeConfig = new MessageStoreConfig(); + String baseDir = IntegrationTestBase.createBaseDir(); + storeConfig.setStorePathRootDir(baseDir); + storeConfig.setStorePathCommitLog(baseDir + SEP + "commitlog"); + storeConfig.setHaListenPort(nextPort()); + storeConfig.setMapedFileSizeCommitLog(10 * 1024 * 1024); + storeConfig.setEnableDLegerCommitLog(true); + storeConfig.setdLegerGroup(brokerName); + storeConfig.setdLegerSelfId(selfId); + storeConfig.setdLegerPeers(peers); + return storeConfig; + } + + @Test + public void testProduceAndConsume() throws Exception { + String cluster = UUID.randomUUID().toString(); + String brokerName = UUID.randomUUID().toString(); + String selfId = "n0"; + String peers = String.format("n0-localhost:%d", nextPort()); + BrokerConfig brokerConfig = buildBrokerConfig(cluster, brokerName); + MessageStoreConfig storeConfig = buildStoreConfig(brokerName, peers, selfId); + BrokerController brokerController = IntegrationTestBase.createAndStartBroker(storeConfig, brokerConfig); + Thread.sleep(3000); + + Assert.assertEquals(BrokerRole.SYNC_MASTER, storeConfig.getBrokerRole()); + + + String topic = UUID.randomUUID().toString(); + String consumerGroup = UUID.randomUUID().toString(); + IntegrationTestBase.initTopic(topic, BaseConf.nsAddr, cluster, 1); + DefaultMQProducer producer = ProducerFactory.getRMQProducer(BaseConf.nsAddr); + DefaultMQPullConsumer consumer = ConsumerFactory.getRMQPullConsumer(BaseConf.nsAddr, consumerGroup); + + for (int i = 0; i < 10; i++) { + Message message = new Message(); + message.setTopic(topic); + message.setBody(("Hello" + i).getBytes()); + SendResult sendResult = producer.send(message); + Assert.assertEquals(SendStatus.SEND_OK, sendResult.getSendStatus()); + Assert.assertEquals(0, sendResult.getMessageQueue().getQueueId()); + Assert.assertEquals(brokerName, sendResult.getMessageQueue().getBrokerName()); + Assert.assertEquals(i, sendResult.getQueueOffset()); + Assert.assertNotNull(sendResult.getMsgId()); + Assert.assertNotNull(sendResult.getOffsetMsgId()); + } + + Thread.sleep(500); + Assert.assertEquals(0, brokerController.getMessageStore().getMinOffsetInQueue(topic, 0)); + Assert.assertEquals(10, brokerController.getMessageStore().getMaxOffsetInQueue(topic, 0)); + + MessageQueue messageQueue = new MessageQueue(topic, brokerName, 0); + PullResult pullResult= consumer.pull(messageQueue, "*", 0, 32); + Assert.assertEquals(PullStatus.FOUND, pullResult.getPullStatus()); + Assert.assertEquals(10, pullResult.getMsgFoundList().size()); + + for (int i = 0; i < 10; i++) { + MessageExt messageExt = pullResult.getMsgFoundList().get(i); + Assert.assertEquals(i, messageExt.getQueueOffset()); + Assert.assertArrayEquals(("Hello" + i).getBytes(), messageExt.getBody()); + } + + producer.shutdown(); + consumer.shutdown(); + brokerController.shutdown(); + } +} diff --git a/test/src/test/java/org/apache/rocketmq/test/client/consumer/broadcast/order/OrderMsgBroadCastIT.java b/test/src/test/java/org/apache/rocketmq/test/client/consumer/broadcast/order/OrderMsgBroadCastIT.java index 9294c3fd934..1d7ea2033ac 100644 --- a/test/src/test/java/org/apache/rocketmq/test/client/consumer/broadcast/order/OrderMsgBroadCastIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/client/consumer/broadcast/order/OrderMsgBroadCastIT.java @@ -29,10 +29,15 @@ import org.apache.rocketmq.test.util.VerifyUtils; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import static com.google.common.truth.Truth.assertThat; +/** + * Currently, dose not support the ordered broadcast message + */ +@Ignore public class OrderMsgBroadCastIT extends BaseBroadCastIT { private static Logger logger = Logger.getLogger(OrderMsgBroadCastIT.class); private RMQNormalProducer producer = null; diff --git a/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java b/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java index 8cb0f41c1c0..b97b0637b04 100644 --- a/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java +++ b/test/src/test/java/org/apache/rocketmq/test/delay/NormalMsgDelayIT.java @@ -51,7 +51,8 @@ public void tearDown() { } @Test - public void testDelayLevell() { + public void testDelayLevel1() throws Exception { + Thread.sleep(3000); int delayLevel = 1; List delayMsgs = MQMessageFactory.getDelayMsg(topic, delayLevel, msgSize); producer.send(delayMsgs); 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 571f0519b63..c33ae52dbd3 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 @@ -19,6 +19,7 @@ import java.util.Set; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionGroup; import org.apache.commons.cli.Options; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.sysflag.TopicSysFlag; @@ -43,13 +44,16 @@ public String commandDesc() { @Override public Options buildCommandlineOptions(Options options) { + OptionGroup optionGroup = new OptionGroup(); + Option opt = new Option("b", "brokerAddr", true, "create topic to which broker"); - opt.setRequired(false); - options.addOption(opt); + optionGroup.addOption(opt); opt = new Option("c", "clusterName", true, "create topic to which cluster"); - opt.setRequired(false); - options.addOption(opt); + optionGroup.addOption(opt); + + optionGroup.setRequired(true); + options.addOptionGroup(optionGroup); opt = new Option("t", "topic", true, "topic name"); opt.setRequired(true); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java new file mode 100644 index 00000000000..af4deace0a9 --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/namesrv/UpdateKvConfigCommandTest.java @@ -0,0 +1,77 @@ +/* + * 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.namesrv; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.apache.rocketmq.client.ClientConfig; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.MQClientManager; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExtImpl; +import org.apache.rocketmq.tools.command.SubCommandException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; + +import java.lang.reflect.Field; + +import static org.mockito.Mockito.mock; + +public class UpdateKvConfigCommandTest { + private static DefaultMQAdminExt defaultMQAdminExt; + private static DefaultMQAdminExtImpl defaultMQAdminExtImpl; + private static MQClientInstance mqClientInstance = MQClientManager.getInstance().getAndCreateMQClientInstance(new ClientConfig()); + private static MQClientAPIImpl mQClientAPIImpl; + + @BeforeClass + public static void init() throws NoSuchFieldException, IllegalAccessException { + mQClientAPIImpl = mock(MQClientAPIImpl.class); + defaultMQAdminExt = new DefaultMQAdminExt(); + defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, 1000); + + Field field = DefaultMQAdminExtImpl.class.getDeclaredField("mqClientInstance"); + field.setAccessible(true); + field.set(defaultMQAdminExtImpl, mqClientInstance); + field = MQClientInstance.class.getDeclaredField("mQClientAPIImpl"); + field.setAccessible(true); + field.set(mqClientInstance, mQClientAPIImpl); + field = DefaultMQAdminExt.class.getDeclaredField("defaultMQAdminExtImpl"); + field.setAccessible(true); + field.set(defaultMQAdminExt, defaultMQAdminExtImpl); + } + + @AfterClass + public static void terminate() { + defaultMQAdminExt.shutdown(); + } + + @Ignore + @Test + public void testExecute() throws SubCommandException { + UpdateKvConfigCommand cmd = new UpdateKvConfigCommand(); + Options options = ServerUtil.buildCommandlineOptions(new Options()); + String[] subargs = new String[]{"-s namespace", "-k topicname", "-v unit_test"}; + final CommandLine commandLine = + ServerUtil.parseCmdLine("mqadmin " + cmd.commandName() + cmd.commandDesc(), subargs, cmd.buildCommandlineOptions(options), new PosixParser()); + cmd.execute(commandLine, options, null); + } +}