From 471c7fa3d5a195cfde28f5f5572d5831f8edf451 Mon Sep 17 00:00:00 2001 From: ShannonDing Date: Wed, 5 Feb 2020 18:14:11 +0800 Subject: [PATCH 1/7] interface implement --- example/common.h | 39 +- include/DefaultMQProducer.h | 106 ++- include/DefaultMQPullConsumer.h | 65 +- include/DefaultMQPushConsumer.h | 139 ++-- include/MQClient.h | 240 +----- include/TransactionMQProducer.h | 31 +- src/common/Arg_helper.cpp | 2 +- src/common/AsyncArg.h | 72 +- src/common/AsyncCallbackWrap.h | 2 +- src/common/ByteOrder.h | 32 +- src/common/ClientRPCHook.cpp | 2 +- src/common/ClientRPCHook.h | 2 +- src/common/CommunicationMode.h | 2 +- .../{MQClient.cpp => DefaultMQClient.cpp} | 82 +- src/common/FilterAPI.h | 2 +- src/common/InputStream.cpp | 32 +- src/common/InputStream.h | 32 +- src/common/MemoryInputStream.cpp | 32 +- src/common/MemoryInputStream.h | 32 +- src/common/MemoryOutputStream.cpp | 32 +- src/common/MemoryOutputStream.h | 32 +- src/common/MessageSysFlag.cpp | 2 +- src/common/MessageSysFlag.h | 2 +- src/common/OutputStream.cpp | 32 +- src/common/OutputStream.h | 32 +- src/common/PermName.cpp | 2 +- src/common/PermName.h | 2 +- src/common/PullSysFlag.cpp | 2 +- src/common/PullSysFlag.h | 2 +- src/common/ServiceState.h | 2 +- src/common/SubscriptionGroupConfig.h | 2 +- src/common/TopAddressing.cpp | 2 +- src/common/TopicConfig.cpp | 2 +- src/common/TopicConfig.h | 2 +- src/common/TopicFilterType.h | 2 +- src/common/UtilAll.cpp | 2 +- src/common/UtilAll.h | 2 +- src/common/Validators.cpp | 2 +- src/common/Validators.h | 2 +- src/common/VirtualEnvUtil.h | 2 +- src/common/big_endian.cpp | 2 +- src/common/big_endian.h | 2 +- src/common/dataBlock.cpp | 32 +- src/common/dataBlock.h | 32 +- src/common/sync_http_client.cpp | 6 +- src/common/sync_http_client.h | 2 +- src/common/url.cpp | 2 +- src/common/url.h | 2 +- src/consumer/ConsumeMsgService.h | 5 +- ...umer.cpp => DefaultMQPullConsumerImpl.cpp} | 70 +- src/consumer/DefaultMQPullConsumerImpl.h | 148 ++++ ...umer.cpp => DefaultMQPushConsumerImpl.cpp} | 126 +-- src/consumer/DefaultMQPushConsumerImpl.h | 165 ++++ src/consumer/Rebalance.cpp | 10 +- src/extern/CProducer.cpp | 1 + {include => src/include}/BatchMessage.h | 0 src/include/DefaultMQClient.h | 194 +++++ {include => src/include}/MQConsumer.h | 136 ++-- {include => src/include}/MQProducer.h | 122 +-- {include => src/include}/QueryResult.h | 84 +- {include => src/include}/SendMessageHook.h | 96 +-- src/log/Logging.h | 2 +- src/message/BatchMessage.cpp | 2 +- src/producer/DefaultMQProducer.cpp | 728 ++++-------------- src/producer/DefaultMQProducerImpl.cpp | 635 +++++++++++++++ src/producer/DefaultMQProducerImpl.h | 118 +++ ...ucer.cpp => TransactionMQProducerImpl.cpp} | 24 +- src/producer/TransactionMQProducerImpl.h | 74 ++ 68 files changed, 2318 insertions(+), 1610 deletions(-) rename src/common/{MQClient.cpp => DefaultMQClient.cpp} (64%) rename src/consumer/{DefaultMQPullConsumer.cpp => DefaultMQPullConsumerImpl.cpp} (83%) create mode 100644 src/consumer/DefaultMQPullConsumerImpl.h rename src/consumer/{DefaultMQPushConsumer.cpp => DefaultMQPushConsumerImpl.cpp} (88%) create mode 100644 src/consumer/DefaultMQPushConsumerImpl.h rename {include => src/include}/BatchMessage.h (100%) create mode 100644 src/include/DefaultMQClient.h rename {include => src/include}/MQConsumer.h (95%) rename {include => src/include}/MQProducer.h (95%) rename {include => src/include}/QueryResult.h (97%) rename {include => src/include}/SendMessageHook.h (87%) create mode 100644 src/producer/DefaultMQProducerImpl.cpp create mode 100644 src/producer/DefaultMQProducerImpl.h rename src/producer/{TransactionMQProducer.cpp => TransactionMQProducerImpl.cpp} (91%) create mode 100644 src/producer/TransactionMQProducerImpl.h diff --git a/example/common.h b/example/common.h index 5b674c37e..1c8fc9ec9 100644 --- a/example/common.h +++ b/example/common.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_ #define ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_ @@ -99,15 +99,6 @@ class TpsReportService { boost::atomic tps_count_; }; -/* -static void PrintResult(rocketmq::SendResult* result) { - std::cout << "sendresult = " << result->getSendStatus() - << ", msgid = " << result->getMsgId() - << ", queueOffset = " << result->getQueueOffset() << "," - << result->getMessageQueue().toString() << endl; -} -*/ - void PrintPullResult(rocketmq::PullResult* result) { std::cout << result->toString() << std::endl; if (result->pullStatus == rocketmq::FOUND) { diff --git a/include/DefaultMQProducer.h b/include/DefaultMQProducer.h index 630e76553..a96a59baa 100644 --- a/include/DefaultMQProducer.h +++ b/include/DefaultMQProducer.h @@ -18,25 +18,25 @@ #ifndef __DEFAULTMQPRODUCER_H__ #define __DEFAULTMQPRODUCER_H__ -#include "BatchMessage.h" +#include "AsyncCallback.h" +#include "MQClient.h" #include "MQMessageQueue.h" -#include "MQProducer.h" +#include "MQSelector.h" #include "RocketMQClient.h" #include "SendResult.h" +#include "SessionCredentials.h" namespace rocketmq { +class DefaultMQProducerImpl; //& msgs); - bool dealWithNameSpace(); + // log configuration interface, default LOG_LEVEL is LOG_LEVEL_INFO, default + // log file num is 3, each log size is 100M + void setLogLevel(elogLevel inputLevel); + elogLevel getLogLevel(); + void setLogFileSizeAndNum(int fileNum, long perFileSize); // perFileSize is MB unit + + /** set TcpTransport pull thread num, which dermine the num of threads to + * distribute network data, + * 1. its default value is CPU num, it must be setted before producer/consumer + * start, minimum value is CPU num; + * 2. this pullThread num must be tested on your environment to find the best + * value for RT of sendMsg or delay time of consume msg before you change it; + * 3. producer and consumer need different pullThread num, if set this num, + * producer and consumer must set different instanceName. + **/ + void setTcpTransportPullThreadNum(int num); + const int getTcpTransportPullThreadNum() const; + + /** timeout of tcp connect, it is same meaning for both producer and consumer; + * 1. default value is 3000ms + * 2. input parameter could only be milliSecond, suggestion value is + * 1000-3000ms; + **/ + void setTcpTransportConnectTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportConnectTimeout() const; + + /** timeout of tryLock tcpTransport before sendMsg/pullMsg, if timeout, + * returns NULL + * 1. paremeter unit is ms, default value is 3000ms, the minimun value is 1000ms + * suggestion value is 3000ms; + * 2. if configured with value smaller than 1000ms, the tryLockTimeout value + * will be setted to 1000ms + **/ + void setTcpTransportTryLockTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportTryLockTimeout() const; + + void setUnitName(std::string unitName); + const std::string& getUnitName() const; + + void setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel); + const SessionCredentials& getSessionCredentials() const; private: - int m_sendMsgTimeout; - int m_compressMsgBodyOverHowmuch; - int m_maxMessageSize; // #include -#include "MQConsumer.h" +#include "AsyncCallback.h" +#include "ConsumeType.h" +#include "MQClient.h" +#include "MQMessage.h" +#include "MQMessageExt.h" #include "MQMessageQueue.h" #include "MQueueListener.h" +#include "PullResult.h" #include "RocketMQClient.h" namespace rocketmq { -class Rebalance; class SubscriptionData; -class OffsetStore; -class PullAPIWrapper; class ConsumerRunningInfo; //& mqs); virtual void doRebalance(); virtual void persistConsumerOffset(); @@ -54,8 +75,6 @@ class ROCKETMQCLIENT_API DefaultMQPullConsumer : public MQConsumer { virtual void getSubscriptions(std::vector&); virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset); virtual void removeConsumeOffset(const MQMessageQueue& mq); - virtual bool producePullMsgTask(boost::weak_ptr pullRequest); - virtual Rebalance* getRebalance() const; // m_registerTopics; - - MQueueListener* m_pMessageQueueListener; - OffsetStore* m_pOffsetStore; - Rebalance* m_pRebalance; - PullAPIWrapper* m_pPullAPIWrapper; }; // -#include -#include -#include -#include -#include #include #include "AsyncCallback.h" -#include "MQConsumer.h" +#include "ConsumeType.h" +#include "MQClient.h" #include "MQMessageListener.h" #include "MQMessageQueue.h" namespace rocketmq { -class Rebalance; -class SubscriptionData; -class OffsetStore; -class PullAPIWrapper; -class PullRequest; -class ConsumeMsgService; -class TaskQueue; -class TaskThread; -class AsyncPullCallback; -class ConsumerRunningInfo; //&); virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset); virtual void removeConsumeOffset(const MQMessageQueue& mq); - virtual PullResult pull(const MQMessageQueue& mq, const std::string& subExpression, int64 offset, int maxNums) { - return PullResult(); - } - virtual void pull(const MQMessageQueue& mq, - const std::string& subExpression, - int64 offset, - int maxNums, - PullCallback* pPullCallback) {} - virtual ConsumerRunningInfo* getConsumerRunningInfo(); - //); - virtual bool producePullMsgTaskLater(boost::weak_ptr, int millis); - static void static_triggerNextPullRequest(void* context, - boost::asio::deadline_timer* t, - boost::weak_ptr); - void triggerNextPullRequest(boost::asio::deadline_timer* t, boost::weak_ptr); - void runPullMsgQueue(TaskQueue* pTaskQueue); - void pullMessage(boost::weak_ptr pullrequest); - void pullMessageAsync(boost::weak_ptr pullrequest); - void setAsyncPull(bool asyncFlag); - AsyncPullCallback* getAsyncPullCallBack(boost::weak_ptr, MQMessageQueue msgQueue); - void shutdownAsyncPullCallBack(); - /* for orderly consume, set the pull num of message size by each pullMsg, default value is 1; @@ -128,37 +85,63 @@ class ROCKETMQCLIENT_API DefaultMQPushConsumer : public MQConsumer { void setMaxCacheMsgSizePerQueue(int maxCacheSize); int getMaxCacheMsgSizePerQueue() const; - private: - void checkConfig(); - void copySubscription(); - void updateTopicSubscribeInfoWhenSubscriptionChanged(); - bool dealWithNameSpace(); - - private: - uint64_t m_startTime; - ConsumeFromWhere m_consumeFromWhere; - std::map m_subTopics; - int m_consumeThreadCount; - OffsetStore* m_pOffsetStore; - Rebalance* m_pRebalance; - PullAPIWrapper* m_pPullAPIWrapper; - ConsumeMsgService* m_consumerService; - MQMessageListener* m_pMessageListener; - int m_consumeMessageBatchMaxSize; - int m_maxMsgCacheSize; - int m_maxReconsumeTimes = -1; - boost::asio::io_service m_async_ioService; - boost::scoped_ptr m_async_service_thread; - - typedef std::map PullMAP; - PullMAP m_PullCallback; - bool m_asyncPull; - int m_asyncPullTimeout; - int m_pullMsgThreadPoolNum; - - private: - TaskQueue* m_pullmsgQueue; - std::unique_ptr m_pullmsgThread; + MessageModel getMessageModel() const; + void setMessageModel(MessageModel messageModel); + const std::string& getNamesrvAddr() const; + void setNamesrvAddr(const std::string& namesrvAddr); + const std::string& getNamesrvDomain() const; + void setNamesrvDomain(const std::string& namesrvDomain); + const std::string& getInstanceName() const; + void setInstanceName(const std::string& instanceName); + // nameSpace + const std::string& getNameSpace() const; + void setNameSpace(const std::string& nameSpace); + const std::string& getGroupName() const; + void setGroupName(const std::string& groupname); + + // log configuration interface, default LOG_LEVEL is LOG_LEVEL_INFO, default + // log file num is 3, each log size is 100M + void setLogLevel(elogLevel inputLevel); + elogLevel getLogLevel(); + void setLogFileSizeAndNum(int fileNum, long perFileSize); // perFileSize is MB unit + + /** set TcpTransport pull thread num, which dermine the num of threads to + * distribute network data, + * 1. its default value is CPU num, it must be setted before producer/consumer + * start, minimum value is CPU num; + * 2. this pullThread num must be tested on your environment to find the best + * value for RT of sendMsg or delay time of consume msg before you change it; + * 3. producer and consumer need different pullThread num, if set this num, + * producer and consumer must set different instanceName. + **/ + void setTcpTransportPullThreadNum(int num); + const int getTcpTransportPullThreadNum() const; + + /** timeout of tcp connect, it is same meaning for both producer and consumer; + * 1. default value is 3000ms + * 2. input parameter could only be milliSecond, suggestion value is + * 1000-3000ms; + **/ + void setTcpTransportConnectTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportConnectTimeout() const; + + /** timeout of tryLock tcpTransport before sendMsg/pullMsg, if timeout, + * returns NULL + * 1. paremeter unit is ms, default value is 3000ms, the minimun value is 1000ms + * suggestion value is 3000ms; + * 2. if configured with value smaller than 1000ms, the tryLockTimeout value + * will be setted to 1000ms + **/ + void setTcpTransportTryLockTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportTryLockTimeout() const; + + void setUnitName(std::string unitName); + const std::string& getUnitName() const; + + void setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel); + const SessionCredentials& getSessionCredentials() const; }; // -#include -#include -#include -#include -#include -#include "MQMessageExt.h" -#include "MQMessageQueue.h" -#include "QueryResult.h" -#include "RocketMQClient.h" -#include "SessionCredentials.h" - -namespace rocketmq { -class MQClientFactory; -// getTopicMessageQueueInfo(const std::string& topic); - - // log configuration interface, default LOG_LEVEL is LOG_LEVEL_INFO, default - // log file num is 3, each log size is 100M - void setLogLevel(elogLevel inputLevel); - elogLevel getLogLevel(); - void setLogFileSizeAndNum(int fileNum, long perFileSize); // perFileSize is MB unit - - /** set TcpTransport pull thread num, which dermine the num of threads to - distribute network data, - 1. its default value is CPU num, it must be setted before producer/consumer - start, minimum value is CPU num; - 2. this pullThread num must be tested on your environment to find the best - value for RT of sendMsg or delay time of consume msg before you change it; - 3. producer and consumer need different pullThread num, if set this num, - producer and consumer must set different instanceName. - 4. configuration suggestion: - 1>. minimum RT of sendMsg: - pullThreadNum = brokerNum*2 - **/ - void setTcpTransportPullThreadNum(int num); - const int getTcpTransportPullThreadNum() const; - - /** timeout of tcp connect, it is same meaning for both producer and consumer; - 1. default value is 3000ms - 2. input parameter could only be milliSecond, suggestion value is - 1000-3000ms; - **/ - void setTcpTransportConnectTimeout(uint64_t timeout); // ms - const uint64_t getTcpTransportConnectTimeout() const; - - /** timeout of tryLock tcpTransport before sendMsg/pullMsg, if timeout, - returns NULL - 1. paremeter unit is ms, default value is 3000ms, the minimun value is - 1000ms - suggestion value is 3000ms; - 2. if configured with value smaller than 1000ms, the tryLockTimeout value - will be setted to 1000ms - **/ - void setTcpTransportTryLockTimeout(uint64_t timeout); // ms - const uint64_t getTcpTransportTryLockTimeout() const; - - void setUnitName(std::string unitName); - const std::string& getUnitName(); - - void setSessionCredentials(const std::string& input_accessKey, - const std::string& input_secretKey, - const std::string& input_onsChannel); - const SessionCredentials& getSessionCredentials() const; - - protected: - virtual void start(); - virtual void shutdown(); - MQClientFactory* getFactory() const; - virtual bool isServiceStateOk(); - - protected: - std::string m_namesrvAddr; - std::string m_namesrvDomain; - std::string m_instanceName; - std::string m_nameSpace; - std::string m_GroupName; - MQClientFactory* m_clientFactory; - int m_serviceState; - int m_pullThreadNum; - uint64_t m_tcpConnectTimeout; // ms - uint64_t m_tcpTransportTryLockTimeout; // s - - std::string m_unitName; - SessionCredentials m_SessionCredentials; -}; -// -#include -#include -#include -#include #include #include #include "DefaultMQProducer.h" @@ -34,13 +29,12 @@ namespace rocketmq { class ROCKETMQCLIENT_API TransactionMQProducer : public DefaultMQProducer { public: - TransactionMQProducer(const std::string& producerGroup) - : DefaultMQProducer(producerGroup), m_thread_num(1), m_ioServiceWork(m_ioService) {} + TransactionMQProducer(const std::string& producerGroup); virtual ~TransactionMQProducer() {} void start(); void shutdown(); - std::shared_ptr getTransactionListener() { return m_transactionListener; } - void setTransactionListener(TransactionListener* listener) { m_transactionListener.reset(listener); } + std::shared_ptr getTransactionListener(); + void setTransactionListener(TransactionListener* listener); TransactionSendResult sendMessageInTransaction(MQMessage& msg, void* arg); void checkTransactionState(const std::string& addr, const MQMessageExt& message, @@ -49,25 +43,6 @@ class ROCKETMQCLIENT_API TransactionMQProducer : public DefaultMQProducer { const std::string& msgId, const std::string& transactionId, const std::string& offsetMsgId); - - private: - void initTransactionEnv(); - void destroyTransactionEnv(); - void endTransaction(SendResult& sendResult, LocalTransactionState& localTransactionState); - void checkTransactionStateImpl(const std::string& addr, - const MQMessageExt& message, - long tranStateTableOffset, - long commitLogOffset, - const std::string& msgId, - const std::string& transactionId, - const std::string& offsetMsgId); - - private: - std::shared_ptr m_transactionListener; - int m_thread_num; - boost::thread_group m_threadpool; - boost::asio::io_service m_ioService; - boost::asio::io_service::work m_ioServiceWork; }; } // namespace rocketmq diff --git a/src/common/Arg_helper.cpp b/src/common/Arg_helper.cpp index 61ac54936..934954e74 100644 --- a/src/common/Arg_helper.cpp +++ b/src/common/Arg_helper.cpp @@ -64,4 +64,4 @@ string Arg_helper::get_option_value(string opt_) const { } //(destBytes)[1] = (uint8)(value >> 8); static_cast(destBytes)[2] = (uint8)value; } -} +} // namespace rocketmq #endif // BYTEORDER_H_INCLUDED diff --git a/src/common/ClientRPCHook.cpp b/src/common/ClientRPCHook.cpp index cd216d10b..fc013dada 100644 --- a/src/common/ClientRPCHook.cpp +++ b/src/common/ClientRPCHook.cpp @@ -72,4 +72,4 @@ void ClientRPCHook::doBeforeRequest(const string& remoteAddr, RemotingCommand& r LOG_ERROR("signature for request failed"); } } -} +} // namespace rocketmq diff --git a/src/common/ClientRPCHook.h b/src/common/ClientRPCHook.h index 72e2f7619..52fef3022 100644 --- a/src/common/ClientRPCHook.h +++ b/src/common/ClientRPCHook.h @@ -41,5 +41,5 @@ class ClientRPCHook : public RPCHook { virtual void doAfterResponse(RemotingCommand& request, RemotingCommand& response) {} }; -} +} // namespace rocketmq #endif diff --git a/src/common/CommunicationMode.h b/src/common/CommunicationMode.h index 6ba947ab6..b2cb60150 100644 --- a/src/common/CommunicationMode.h +++ b/src/common/CommunicationMode.h @@ -23,5 +23,5 @@ namespace rocketmq { enum CommunicationMode { ComMode_SYNC, ComMode_ASYNC, ComMode_ONEWAY }; //createTopic(key, newTopic, queueNum, m_SessionCredentials); } catch (MQException& e) { @@ -104,31 +104,31 @@ void MQClient::createTopic(const string& key, const string& newTopic, int queueN } } -int64 MQClient::earliestMsgStoreTime(const MQMessageQueue& mq) { +int64 DefaultMQClient::earliestMsgStoreTime(const MQMessageQueue& mq) { return getFactory()->earliestMsgStoreTime(mq, m_SessionCredentials); } -QueryResult MQClient::queryMessage(const string& topic, const string& key, int maxNum, int64 begin, int64 end) { +QueryResult DefaultMQClient::queryMessage(const string& topic, const string& key, int maxNum, int64 begin, int64 end) { return getFactory()->queryMessage(topic, key, maxNum, begin, end, m_SessionCredentials); } -int64 MQClient::minOffset(const MQMessageQueue& mq) { +int64 DefaultMQClient::minOffset(const MQMessageQueue& mq) { return getFactory()->minOffset(mq, m_SessionCredentials); } -int64 MQClient::maxOffset(const MQMessageQueue& mq) { +int64 DefaultMQClient::maxOffset(const MQMessageQueue& mq) { return getFactory()->maxOffset(mq, m_SessionCredentials); } -int64 MQClient::searchOffset(const MQMessageQueue& mq, uint64_t timestamp) { +int64 DefaultMQClient::searchOffset(const MQMessageQueue& mq, uint64_t timestamp) { return getFactory()->searchOffset(mq, timestamp, m_SessionCredentials); } -MQMessageExt* MQClient::viewMessage(const string& msgId) { +MQMessageExt* DefaultMQClient::viewMessage(const string& msgId) { return getFactory()->viewMessage(msgId, m_SessionCredentials); } -vector MQClient::getTopicMessageQueueInfo(const string& topic) { +vector DefaultMQClient::getTopicMessageQueueInfo(const string& topic) { boost::weak_ptr weak_topicPublishInfo( getFactory()->tryToFindTopicPublishInfo(topic, m_SessionCredentials)); boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); @@ -138,7 +138,7 @@ vector MQClient::getTopicMessageQueueInfo(const string& topic) { THROW_MQEXCEPTION(MQClientException, "could not find MessageQueue Info of topic: [" + topic + "].", -1); } -void MQClient::start() { +void DefaultMQClient::start() { if (getFactory() == NULL) { m_clientFactory = MQClientManager::getInstance()->getMQClientFactory( getMQClientId(), m_pullThreadNum, m_tcpConnectTimeout, m_tcpTransportTryLockTimeout, m_unitName); @@ -149,74 +149,74 @@ void MQClient::start() { getGroupName().c_str(), getMQClientId().c_str(), getInstanceName().c_str(), getNamesrvAddr().c_str()); } -void MQClient::shutdown() { +void DefaultMQClient::shutdown() { m_clientFactory->shutdown(); m_clientFactory = NULL; } -MQClientFactory* MQClient::getFactory() const { +MQClientFactory* DefaultMQClient::getFactory() const { return m_clientFactory; } -bool MQClient::isServiceStateOk() { +bool DefaultMQClient::isServiceStateOk() { return m_serviceState == RUNNING; } -void MQClient::setLogLevel(elogLevel inputLevel) { +void DefaultMQClient::setLogLevel(elogLevel inputLevel) { ALOG_ADAPTER->setLogLevel(inputLevel); } -elogLevel MQClient::getLogLevel() { +elogLevel DefaultMQClient::getLogLevel() { return ALOG_ADAPTER->getLogLevel(); } -void MQClient::setLogFileSizeAndNum(int fileNum, long perFileSize) { +void DefaultMQClient::setLogFileSizeAndNum(int fileNum, long perFileSize) { ALOG_ADAPTER->setLogFileNumAndSize(fileNum, perFileSize); } -void MQClient::setTcpTransportPullThreadNum(int num) { +void DefaultMQClient::setTcpTransportPullThreadNum(int num) { if (num > m_pullThreadNum) { m_pullThreadNum = num; } } -const int MQClient::getTcpTransportPullThreadNum() const { +const int DefaultMQClient::getTcpTransportPullThreadNum() const { return m_pullThreadNum; } -void MQClient::setTcpTransportConnectTimeout(uint64_t timeout) { +void DefaultMQClient::setTcpTransportConnectTimeout(uint64_t timeout) { m_tcpConnectTimeout = timeout; } -const uint64_t MQClient::getTcpTransportConnectTimeout() const { +const uint64_t DefaultMQClient::getTcpTransportConnectTimeout() const { return m_tcpConnectTimeout; } -void MQClient::setTcpTransportTryLockTimeout(uint64_t timeout) { +void DefaultMQClient::setTcpTransportTryLockTimeout(uint64_t timeout) { if (timeout < 1000) { timeout = 1000; } m_tcpTransportTryLockTimeout = timeout / 1000; } -const uint64_t MQClient::getTcpTransportTryLockTimeout() const { +const uint64_t DefaultMQClient::getTcpTransportTryLockTimeout() const { return m_tcpTransportTryLockTimeout; } -void MQClient::setUnitName(string unitName) { +void DefaultMQClient::setUnitName(string unitName) { m_unitName = unitName; } -const string& MQClient::getUnitName() { +const string& DefaultMQClient::getUnitName() const { return m_unitName; } -void MQClient::setSessionCredentials(const string& input_accessKey, - const string& input_secretKey, - const string& input_onsChannel) { +void DefaultMQClient::setSessionCredentials(const string& input_accessKey, + const string& input_secretKey, + const string& input_onsChannel) { m_SessionCredentials.setAccessKey(input_accessKey); m_SessionCredentials.setSecretKey(input_secretKey); m_SessionCredentials.setAuthChannel(input_onsChannel); } -const SessionCredentials& MQClient::getSessionCredentials() const { +const SessionCredentials& DefaultMQClient::getSessionCredentials() const { return m_SessionCredentials; } diff --git a/src/common/FilterAPI.h b/src/common/FilterAPI.h index d632bc232..77255ba6f 100644 --- a/src/common/FilterAPI.h +++ b/src/common/FilterAPI.h @@ -57,5 +57,5 @@ class FilterAPI { }; // #include "MemoryOutputStream.h" @@ -108,4 +108,4 @@ void InputStream::skipNextBytes(int64 numBytesToSkip) { std::free(temp); } } -} +} // namespace rocketmq diff --git a/src/common/InputStream.h b/src/common/InputStream.h index 8c6ddb51a..8cf1b8482 100644 --- a/src/common/InputStream.h +++ b/src/common/InputStream.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef INPUTSTREAM_H_INCLUDED #define INPUTSTREAM_H_INCLUDED @@ -190,5 +190,5 @@ class ROCKETMQCLIENT_API InputStream { //============================================================================== InputStream() {} }; -} +} // namespace rocketmq #endif // INPUTSTREAM_H_INCLUDED diff --git a/src/common/MemoryInputStream.cpp b/src/common/MemoryInputStream.cpp index 3d6599246..b857c446e 100644 --- a/src/common/MemoryInputStream.cpp +++ b/src/common/MemoryInputStream.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include "MemoryInputStream.h" namespace rocketmq { @@ -72,4 +72,4 @@ bool MemoryInputStream::setPosition(const int64 pos) { int64 MemoryInputStream::getPosition() { return (int64)position; } -} +} // namespace rocketmq diff --git a/src/common/MemoryInputStream.h b/src/common/MemoryInputStream.h index 53584b9a3..2d3a365b3 100644 --- a/src/common/MemoryInputStream.h +++ b/src/common/MemoryInputStream.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef MEMORYINPUTSTREAM_H_INCLUDED #define MEMORYINPUTSTREAM_H_INCLUDED @@ -92,5 +92,5 @@ class ROCKETMQCLIENT_API MemoryInputStream : public InputStream { void createInternalCopy(); }; -} +} // namespace rocketmq #endif diff --git a/src/common/MemoryOutputStream.cpp b/src/common/MemoryOutputStream.cpp index 9578c8a35..d7366f534 100644 --- a/src/common/MemoryOutputStream.cpp +++ b/src/common/MemoryOutputStream.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include "MemoryOutputStream.h" namespace rocketmq { @@ -153,4 +153,4 @@ OutputStream& operator<<(OutputStream& stream, const MemoryOutputStream& streamT return stream; } -} +} // namespace rocketmq diff --git a/src/common/MemoryOutputStream.h b/src/common/MemoryOutputStream.h index 4b39879b9..cc3245a5e 100644 --- a/src/common/MemoryOutputStream.h +++ b/src/common/MemoryOutputStream.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef MEMORYOUTPUTSTREAM_H_INCLUDED #define MEMORYOUTPUTSTREAM_H_INCLUDED @@ -126,5 +126,5 @@ class ROCKETMQCLIENT_API MemoryOutputStream : public OutputStream { /** Copies all the data that has been written to a MemoryOutputStream into * another stream. */ OutputStream& operator<<(OutputStream& stream, const MemoryOutputStream& streamToRead); -} +} // namespace rocketmq #endif // MEMORYOUTPUTSTREAM_H_INCLUDED diff --git a/src/common/MessageSysFlag.cpp b/src/common/MessageSysFlag.cpp index 50a28b476..732f1d5c6 100644 --- a/src/common/MessageSysFlag.cpp +++ b/src/common/MessageSysFlag.cpp @@ -34,4 +34,4 @@ int MessageSysFlag::resetTransactionValue(int flag, int type) { } // #include "big_endian.h" @@ -101,4 +101,4 @@ int64 OutputStream::writeFromInputStream(InputStream& source, int64 numBytesToWr return numWritten; } -} +} // namespace rocketmq diff --git a/src/common/OutputStream.h b/src/common/OutputStream.h index 3792169d9..6dd459ddb 100644 --- a/src/common/OutputStream.h +++ b/src/common/OutputStream.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef OUTPUTSTREAM_H_INCLUDED #define OUTPUTSTREAM_H_INCLUDED @@ -143,6 +143,6 @@ class ROCKETMQCLIENT_API OutputStream { */ virtual int64 writeFromInputStream(InputStream& source, int64 maxNumBytesToWrite); }; -} +} // namespace rocketmq #endif // OUTPUTSTREAM_H_INCLUDED diff --git a/src/common/PermName.cpp b/src/common/PermName.cpp index bd36bfc05..7484cc675 100644 --- a/src/common/PermName.cpp +++ b/src/common/PermName.cpp @@ -54,4 +54,4 @@ string PermName::perm2String(int perm) { } // @@ -204,4 +204,4 @@ void MemoryBlock::copyTo(void* const dst, int offset, int num) const { if (num > 0) memcpy(d, data + offset, num); } -} +} // namespace rocketmq diff --git a/src/common/dataBlock.h b/src/common/dataBlock.h index 4419af31d..7472927cd 100644 --- a/src/common/dataBlock.h +++ b/src/common/dataBlock.h @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #ifndef __DATABLOCK_H__ #define __DATABLOCK_H__ @@ -203,6 +203,6 @@ class ROCKETMQCLIENT_API MemoryBlock { int size; char* data; }; -} +} // namespace rocketmq #endif diff --git a/src/common/sync_http_client.cpp b/src/common/sync_http_client.cpp index 78f0292c6..52137b395 100644 --- a/src/common/sync_http_client.cpp +++ b/src/common/sync_http_client.cpp @@ -26,9 +26,9 @@ #include "Logging.h" #include "url.h" -using boost::lambda::var; -using boost::asio::ip::tcp; using boost::asio::deadline_timer; +using boost::asio::ip::tcp; +using boost::lambda::var; namespace { void check_deadline(deadline_timer* deadline, tcp::socket* socket, const boost::system::error_code& ec) { @@ -148,4 +148,4 @@ bool SyncfetchNsAddr(const Url& url_s, std::string& body) { return ret; } -} // end of namespace ons +} // namespace rocketmq diff --git a/src/common/sync_http_client.h b/src/common/sync_http_client.h index 983660595..1bf55fa18 100644 --- a/src/common/sync_http_client.h +++ b/src/common/sync_http_client.h @@ -24,6 +24,6 @@ class Url; extern bool SyncfetchNsAddr(const Url& url_s, std::string& body); -} // namespace ons +} // namespace rocketmq #endif // ROCKETMQ_CLIENT4CPP__SYNC_HTTP_CLIENT_H_ diff --git a/src/common/url.cpp b/src/common/url.cpp index f8d915528..d63203721 100644 --- a/src/common/url.cpp +++ b/src/common/url.cpp @@ -62,4 +62,4 @@ void Url::parse(const std::string& url_s) { query_.assign(query_i, url_s.end()); } -} // namespace ons +} // namespace rocketmq diff --git a/src/common/url.h b/src/common/url.h index 0fbcb85fe..62d86a232 100644 --- a/src/common/url.h +++ b/src/common/url.h @@ -34,5 +34,5 @@ class Url { std::string path_; std::string query_; }; -} +} // namespace rocketmq #endif // ROCKETMQ_CLIENT4CPP_URL_HH_ diff --git a/src/consumer/ConsumeMsgService.h b/src/consumer/ConsumeMsgService.h index a6c720617..db9b27a36 100644 --- a/src/consumer/ConsumeMsgService.h +++ b/src/consumer/ConsumeMsgService.h @@ -27,9 +27,10 @@ #include "Logging.h" #include "MQMessageListener.h" #include "PullRequest.h" - +#include "MQConsumer.h" +#include "DefaultMQPushConsumerImpl.h" namespace rocketmq { -class MQConsumer; +//class MQConsumer; //unregisterConsumer(this); getFactory()->shutdown(); @@ -142,11 +142,11 @@ void DefaultMQPullConsumer::shutdown() { } } -bool DefaultMQPullConsumer::sendMessageBack(MQMessageExt& msg, int delayLevel, string& brokerName) { +bool DefaultMQPullConsumerImpl::sendMessageBack(MQMessageExt& msg, int delayLevel, string& brokerName) { return true; } -void DefaultMQPullConsumer::fetchSubscribeMessageQueues(const string& topic, vector& mqs) { +void DefaultMQPullConsumerImpl::fetchSubscribeMessageQueues(const string& topic, vector& mqs) { mqs.clear(); try { const string localTopic = NameSpaceUtil::withNameSpace(topic, getNameSpace()); @@ -156,23 +156,23 @@ void DefaultMQPullConsumer::fetchSubscribeMessageQueues(const string& topic, vec } } -void DefaultMQPullConsumer::updateTopicSubscribeInfo(const string& topic, vector& info) {} +void DefaultMQPullConsumerImpl::updateTopicSubscribeInfo(const string& topic, vector& info) {} -void DefaultMQPullConsumer::registerMessageQueueListener(const string& topic, MQueueListener* pListener) { +void DefaultMQPullConsumerImpl::registerMessageQueueListener(const string& topic, MQueueListener* pListener) { m_registerTopics.insert(topic); if (pListener) { m_pMessageQueueListener = pListener; } } -PullResult DefaultMQPullConsumer::pull(const MQMessageQueue& mq, +PullResult DefaultMQPullConsumerImpl::pull(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums) { return pullSyncImpl(mq, subExpression, offset, maxNums, false); } -void DefaultMQPullConsumer::pull(const MQMessageQueue& mq, +void DefaultMQPullConsumerImpl::pull(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums, @@ -180,14 +180,14 @@ void DefaultMQPullConsumer::pull(const MQMessageQueue& mq, pullAsyncImpl(mq, subExpression, offset, maxNums, false, pPullCallback); } -PullResult DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq, +PullResult DefaultMQPullConsumerImpl::pullBlockIfNotFound(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums) { return pullSyncImpl(mq, subExpression, offset, maxNums, true); } -void DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq, +void DefaultMQPullConsumerImpl::pullBlockIfNotFound(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums, @@ -195,7 +195,7 @@ void DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq, pullAsyncImpl(mq, subExpression, offset, maxNums, true, pPullCallback); } -PullResult DefaultMQPullConsumer::pullSyncImpl(const MQMessageQueue& mq, +PullResult DefaultMQPullConsumerImpl::pullSyncImpl(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums, @@ -240,7 +240,7 @@ PullResult DefaultMQPullConsumer::pullSyncImpl(const MQMessageQueue& mq, return PullResult(BROKER_TIMEOUT); } -void DefaultMQPullConsumer::pullAsyncImpl(const MQMessageQueue& mq, +void DefaultMQPullConsumerImpl::pullAsyncImpl(const MQMessageQueue& mq, const string& subExpression, int64 offset, int maxNums, @@ -289,7 +289,7 @@ void DefaultMQPullConsumer::pullAsyncImpl(const MQMessageQueue& mq, } } -void DefaultMQPullConsumer::subscriptionAutomatically(const string& topic) { +void DefaultMQPullConsumerImpl::subscriptionAutomatically(const string& topic) { SubscriptionData* pSdata = m_pRebalance->getSubscriptionData(topic); if (pSdata == NULL) { unique_ptr subscriptionData(FilterAPI::buildSubscriptionData(topic, SUB_ALL)); @@ -297,19 +297,19 @@ void DefaultMQPullConsumer::subscriptionAutomatically(const string& topic) { } } -void DefaultMQPullConsumer::updateConsumeOffset(const MQMessageQueue& mq, int64 offset) { +void DefaultMQPullConsumerImpl::updateConsumeOffset(const MQMessageQueue& mq, int64 offset) { m_pOffsetStore->updateOffset(mq, offset); } -void DefaultMQPullConsumer::removeConsumeOffset(const MQMessageQueue& mq) { +void DefaultMQPullConsumerImpl::removeConsumeOffset(const MQMessageQueue& mq) { m_pOffsetStore->removeOffset(mq); } -int64 DefaultMQPullConsumer::fetchConsumeOffset(const MQMessageQueue& mq, bool fromStore) { +int64 DefaultMQPullConsumerImpl::fetchConsumeOffset(const MQMessageQueue& mq, bool fromStore) { return m_pOffsetStore->readOffset(mq, fromStore ? READ_FROM_STORE : MEMORY_FIRST_THEN_STORE, getSessionCredentials()); } -void DefaultMQPullConsumer::persistConsumerOffset() { +void DefaultMQPullConsumerImpl::persistConsumerOffset() { /*As do not execute rebalance for pullConsumer now, requestTable is always empty map requestTable = @@ -326,17 +326,17 @@ void DefaultMQPullConsumer::persistConsumerOffset() { m_pOffsetStore->persistAll(mqs);*/ } -void DefaultMQPullConsumer::persistConsumerOffsetByResetOffset() {} +void DefaultMQPullConsumerImpl::persistConsumerOffsetByResetOffset() {} -void DefaultMQPullConsumer::persistConsumerOffset4PullConsumer(const MQMessageQueue& mq) { +void DefaultMQPullConsumerImpl::persistConsumerOffset4PullConsumer(const MQMessageQueue& mq) { if (isServiceStateOk()) { m_pOffsetStore->persist(mq, getSessionCredentials()); } } -void DefaultMQPullConsumer::fetchMessageQueuesInBalance(const string& topic, vector mqs) {} +void DefaultMQPullConsumerImpl::fetchMessageQueuesInBalance(const string& topic, vector mqs) {} -void DefaultMQPullConsumer::checkConfig() { +void DefaultMQPullConsumerImpl::checkConfig() { string groupname = getGroupName(); // check consumerGroup Validators::checkGroup(groupname); @@ -351,9 +351,9 @@ void DefaultMQPullConsumer::checkConfig() { } } -void DefaultMQPullConsumer::doRebalance() {} +void DefaultMQPullConsumerImpl::doRebalance() {} -void DefaultMQPullConsumer::copySubscription() { +void DefaultMQPullConsumerImpl::copySubscription() { set::iterator it = m_registerTopics.begin(); for (; it != m_registerTopics.end(); ++it) { unique_ptr subscriptionData(FilterAPI::buildSubscriptionData((*it), SUB_ALL)); @@ -361,15 +361,15 @@ void DefaultMQPullConsumer::copySubscription() { } } -ConsumeType DefaultMQPullConsumer::getConsumeType() { +ConsumeType DefaultMQPullConsumerImpl::getConsumeType() { return CONSUME_ACTIVELY; } -ConsumeFromWhere DefaultMQPullConsumer::getConsumeFromWhere() { +ConsumeFromWhere DefaultMQPullConsumerImpl::getConsumeFromWhere() { return CONSUME_FROM_LAST_OFFSET; } -void DefaultMQPullConsumer::getSubscriptions(vector& result) { +void DefaultMQPullConsumerImpl::getSubscriptions(vector& result) { set::iterator it = m_registerTopics.begin(); for (; it != m_registerTopics.end(); ++it) { SubscriptionData ms(*it, SUB_ALL); @@ -377,15 +377,15 @@ void DefaultMQPullConsumer::getSubscriptions(vector& result) { } } -bool DefaultMQPullConsumer::producePullMsgTask(boost::weak_ptr pullRequest) { +bool DefaultMQPullConsumerImpl::producePullMsgTask(boost::weak_ptr pullRequest) { return true; } -Rebalance* DefaultMQPullConsumer::getRebalance() const { +Rebalance* DefaultMQPullConsumerImpl::getRebalance() const { return NULL; } // we should deal with name space before producer start. -bool DefaultMQPullConsumer::dealWithNameSpace() { +bool DefaultMQPullConsumerImpl::dealWithNameSpace() { string ns = getNameSpace(); if (ns.empty()) { string nsAddr = getNamesrvAddr(); diff --git a/src/consumer/DefaultMQPullConsumerImpl.h b/src/consumer/DefaultMQPullConsumerImpl.h new file mode 100644 index 000000000..18f26d90e --- /dev/null +++ b/src/consumer/DefaultMQPullConsumerImpl.h @@ -0,0 +1,148 @@ +/* + * 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. + */ + +#ifndef __DEFAULTMQPULLCONSUMERIMPL_H__ +#define __DEFAULTMQPULLCONSUMERIMPL_H__ + +#include +#include +#include "MQConsumer.h" +#include "MQMessageQueue.h" +#include "MQueueListener.h" +#include "RocketMQClient.h" + +namespace rocketmq { +class Rebalance; +class SubscriptionData; +class OffsetStore; +class PullAPIWrapper; +class ConsumerRunningInfo; +//& mqs); + virtual void doRebalance(); + virtual void persistConsumerOffset(); + virtual void persistConsumerOffsetByResetOffset(); + virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info); + virtual ConsumeType getConsumeType(); + virtual ConsumeFromWhere getConsumeFromWhere(); + virtual void getSubscriptions(std::vector&); + virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset); + virtual void removeConsumeOffset(const MQMessageQueue& mq); + virtual bool producePullMsgTask(boost::weak_ptr pullRequest); + virtual Rebalance* getRebalance() const; + // mqs); + + // temp persist consumer offset interface, only valid with + // RemoteBrokerOffsetStore, updateConsumeOffset should be called before. + void persistConsumerOffset4PullConsumer(const MQMessageQueue& mq); + + private: + void checkConfig(); + void copySubscription(); + bool dealWithNameSpace(); + + PullResult pullSyncImpl(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + bool block); + + void pullAsyncImpl(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + bool block, + PullCallback* pPullCallback); + + void subscriptionAutomatically(const std::string& topic); + + private: + std::set m_registerTopics; + + MQueueListener* m_pMessageQueueListener; + OffsetStore* m_pOffsetStore; + Rebalance* m_pRebalance; + PullAPIWrapper* m_pPullAPIWrapper; +}; +// request) + AsyncPullCallback(DefaultMQPushConsumerImpl* pushConsumer, boost::weak_ptr request) : m_callbackOwner(pushConsumer), m_pullRequest(request), m_bShutdown(false) {} virtual ~AsyncPullCallback() { m_callbackOwner = NULL; } @@ -185,7 +185,7 @@ class AsyncPullCallback : public PullCallback { void setPullRequest(boost::weak_ptr& pullRequest) { m_pullRequest = pullRequest; } private: - DefaultMQPushConsumer* m_callbackOwner; + DefaultMQPushConsumerImpl* m_callbackOwner; boost::weak_ptr m_pullRequest; bool m_bShutdown; }; @@ -193,7 +193,7 @@ class AsyncPullCallback : public PullCallback { //findBrokerAddressInPublish(brokerName); @@ -267,7 +267,7 @@ bool DefaultMQPushConsumer::sendMessageBack(MQMessageExt& msg, int delayLevel, s return true; } -void DefaultMQPushConsumer::fetchSubscribeMessageQueues(const string& topic, vector& mqs) { +void DefaultMQPushConsumerImpl::fetchSubscribeMessageQueues(const string& topic, vector& mqs) { mqs.clear(); try { getFactory()->fetchSubscribeMessageQueues(topic, mqs, getSessionCredentials()); @@ -276,7 +276,7 @@ void DefaultMQPushConsumer::fetchSubscribeMessageQueues(const string& topic, vec } } -void DefaultMQPushConsumer::doRebalance() { +void DefaultMQPushConsumerImpl::doRebalance() { if (isServiceStateOk()) { try { m_pRebalance->doRebalance(); @@ -286,19 +286,19 @@ void DefaultMQPushConsumer::doRebalance() { } } -void DefaultMQPushConsumer::persistConsumerOffset() { +void DefaultMQPushConsumerImpl::persistConsumerOffset() { if (isServiceStateOk()) { m_pRebalance->persistConsumerOffset(); } } -void DefaultMQPushConsumer::persistConsumerOffsetByResetOffset() { +void DefaultMQPushConsumerImpl::persistConsumerOffsetByResetOffset() { if (isServiceStateOk()) { m_pRebalance->persistConsumerOffsetByResetOffset(); } } -void DefaultMQPushConsumer::start() { +void DefaultMQPushConsumerImpl::start() { #ifndef WIN32 /* Ignore the SIGPIPE */ struct sigaction sa; @@ -312,8 +312,8 @@ void DefaultMQPushConsumer::start() { switch (m_serviceState) { case CREATE_JUST: { m_serviceState = START_FAILED; - MQClient::start(); - LOG_INFO("DefaultMQPushConsumer:%s start", m_GroupName.c_str()); + DefaultMQClient::start(); + LOG_INFO("DefaultMQPushConsumerImpl:%s start", m_GroupName.c_str()); //rebalanceImmediately(); } -void DefaultMQPushConsumer::shutdown() { +void DefaultMQPushConsumerImpl::shutdown() { switch (m_serviceState) { case RUNNING: { - LOG_INFO("DefaultMQPushConsumer shutdown"); + LOG_INFO("DefaultMQPushConsumerImpl shutdown"); m_async_ioService.stop(); m_async_service_thread->interrupt(); m_async_service_thread->join(); @@ -419,36 +419,36 @@ void DefaultMQPushConsumer::shutdown() { } } -void DefaultMQPushConsumer::registerMessageListener(MQMessageListener* pMessageListener) { +void DefaultMQPushConsumerImpl::registerMessageListener(MQMessageListener* pMessageListener) { if (NULL != pMessageListener) { m_pMessageListener = pMessageListener; } } -MessageListenerType DefaultMQPushConsumer::getMessageListenerType() { +MessageListenerType DefaultMQPushConsumerImpl::getMessageListenerType() { if (NULL != m_pMessageListener) { return m_pMessageListener->getMessageListenerType(); } return messageListenerDefaultly; } -ConsumeMsgService* DefaultMQPushConsumer::getConsumerMsgService() const { +ConsumeMsgService* DefaultMQPushConsumerImpl::getConsumerMsgService() const { return m_consumerService; } -OffsetStore* DefaultMQPushConsumer::getOffsetStore() const { +OffsetStore* DefaultMQPushConsumerImpl::getOffsetStore() const { return m_pOffsetStore; } -Rebalance* DefaultMQPushConsumer::getRebalance() const { +Rebalance* DefaultMQPushConsumerImpl::getRebalance() const { return m_pRebalance; } -void DefaultMQPushConsumer::subscribe(const string& topic, const string& subExpression) { +void DefaultMQPushConsumerImpl::subscribe(const string& topic, const string& subExpression) { m_subTopics[topic] = subExpression; } -void DefaultMQPushConsumer::checkConfig() { +void DefaultMQPushConsumerImpl::checkConfig() { string groupname = getGroupName(); // check consumerGroup Validators::checkGroup(groupname); @@ -467,7 +467,7 @@ void DefaultMQPushConsumer::checkConfig() { } } -void DefaultMQPushConsumer::copySubscription() { +void DefaultMQPushConsumerImpl::copySubscription() { map::iterator it = m_subTopics.begin(); for (; it != m_subTopics.end(); ++it) { LOG_INFO("buildSubscriptionData,:%s,%s", it->first.c_str(), it->second.c_str()); @@ -493,11 +493,11 @@ void DefaultMQPushConsumer::copySubscription() { } } -void DefaultMQPushConsumer::updateTopicSubscribeInfo(const string& topic, vector& info) { +void DefaultMQPushConsumerImpl::updateTopicSubscribeInfo(const string& topic, vector& info) { m_pRebalance->setTopicSubscribeInfo(topic, info); } -void DefaultMQPushConsumer::updateTopicSubscribeInfoWhenSubscriptionChanged() { +void DefaultMQPushConsumerImpl::updateTopicSubscribeInfoWhenSubscriptionChanged() { map& subTable = m_pRebalance->getSubscriptionInner(); map::iterator it = subTable.begin(); for (; it != subTable.end(); ++it) { @@ -508,19 +508,19 @@ void DefaultMQPushConsumer::updateTopicSubscribeInfoWhenSubscriptionChanged() { } } -ConsumeType DefaultMQPushConsumer::getConsumeType() { +ConsumeType DefaultMQPushConsumerImpl::getConsumeType() { return CONSUME_PASSIVELY; } -ConsumeFromWhere DefaultMQPushConsumer::getConsumeFromWhere() { +ConsumeFromWhere DefaultMQPushConsumerImpl::getConsumeFromWhere() { return m_consumeFromWhere; } -void DefaultMQPushConsumer::setConsumeFromWhere(ConsumeFromWhere consumeFromWhere) { +void DefaultMQPushConsumerImpl::setConsumeFromWhere(ConsumeFromWhere consumeFromWhere) { m_consumeFromWhere = consumeFromWhere; } -void DefaultMQPushConsumer::getSubscriptions(vector& result) { +void DefaultMQPushConsumerImpl::getSubscriptions(vector& result) { map& subTable = m_pRebalance->getSubscriptionInner(); map::iterator it = subTable.begin(); for (; it != subTable.end(); ++it) { @@ -528,7 +528,7 @@ void DefaultMQPushConsumer::getSubscriptions(vector& result) { } } -void DefaultMQPushConsumer::updateConsumeOffset(const MQMessageQueue& mq, int64 offset) { +void DefaultMQPushConsumerImpl::updateConsumeOffset(const MQMessageQueue& mq, int64 offset) { if (offset >= 0) { m_pOffsetStore->updateOffset(mq, offset); } else { @@ -536,24 +536,24 @@ void DefaultMQPushConsumer::updateConsumeOffset(const MQMessageQueue& mq, int64 } } -void DefaultMQPushConsumer::removeConsumeOffset(const MQMessageQueue& mq) { +void DefaultMQPushConsumerImpl::removeConsumeOffset(const MQMessageQueue& mq) { m_pOffsetStore->removeOffset(mq); } -void DefaultMQPushConsumer::static_triggerNextPullRequest(void* context, +void DefaultMQPushConsumerImpl::static_triggerNextPullRequest(void* context, boost::asio::deadline_timer* t, boost::weak_ptr pullRequest) { if (pullRequest.expired()) { LOG_WARN("Pull request has been released before."); return; } - DefaultMQPushConsumer* pDefaultMQPushConsumer = (DefaultMQPushConsumer*)context; - if (pDefaultMQPushConsumer) { - pDefaultMQPushConsumer->triggerNextPullRequest(t, pullRequest); + DefaultMQPushConsumerImpl* pDefaultMQPushConsumerImpl = (DefaultMQPushConsumerImpl*)context; + if (pDefaultMQPushConsumerImpl) { + pDefaultMQPushConsumerImpl->triggerNextPullRequest(t, pullRequest); } } -void DefaultMQPushConsumer::triggerNextPullRequest(boost::asio::deadline_timer* t, +void DefaultMQPushConsumerImpl::triggerNextPullRequest(boost::asio::deadline_timer* t, boost::weak_ptr pullRequest) { // delete first to avoild memleak deleteAndZero(t); @@ -565,7 +565,7 @@ void DefaultMQPushConsumer::triggerNextPullRequest(boost::asio::deadline_timer* producePullMsgTask(request); } -bool DefaultMQPushConsumer::producePullMsgTaskLater(boost::weak_ptr pullRequest, int millis) { +bool DefaultMQPushConsumerImpl::producePullMsgTaskLater(boost::weak_ptr pullRequest, int millis) { boost::shared_ptr request = pullRequest.lock(); if (!request) { LOG_INFO("Pull request is invalid. Maybe it is dropped before."); @@ -578,7 +578,7 @@ bool DefaultMQPushConsumer::producePullMsgTaskLater(boost::weak_ptr if (m_pullmsgQueue->bTaskQueueStatusOK() && isServiceStateOk()) { boost::asio::deadline_timer* t = new boost::asio::deadline_timer(m_async_ioService, boost::posix_time::milliseconds(millis)); - t->async_wait(boost::bind(&(DefaultMQPushConsumer::static_triggerNextPullRequest), this, t, request)); + t->async_wait(boost::bind(&(DefaultMQPushConsumerImpl::static_triggerNextPullRequest), this, t, request)); LOG_INFO("Produce Pull request [%s] Later and Sleep [%d]ms.", (request->m_messageQueue).toString().c_str(), millis); return true; } else { @@ -588,7 +588,7 @@ bool DefaultMQPushConsumer::producePullMsgTaskLater(boost::weak_ptr } } -bool DefaultMQPushConsumer::producePullMsgTask(boost::weak_ptr pullRequest) { +bool DefaultMQPushConsumerImpl::producePullMsgTask(boost::weak_ptr pullRequest) { boost::shared_ptr request = pullRequest.lock(); if (!request) { LOG_WARN("Pull request has been released."); @@ -600,9 +600,9 @@ bool DefaultMQPushConsumer::producePullMsgTask(boost::weak_ptr pull } if (m_pullmsgQueue->bTaskQueueStatusOK() && isServiceStateOk()) { if (m_asyncPull) { - m_pullmsgQueue->produce(TaskBinder::gen(&DefaultMQPushConsumer::pullMessageAsync, this, request)); + m_pullmsgQueue->produce(TaskBinder::gen(&DefaultMQPushConsumerImpl::pullMessageAsync, this, request)); } else { - m_pullmsgQueue->produce(TaskBinder::gen(&DefaultMQPushConsumer::pullMessage, this, request)); + m_pullmsgQueue->produce(TaskBinder::gen(&DefaultMQPushConsumerImpl::pullMessage, this, request)); } } else { LOG_WARN("produce PullRequest of mq:%s failed", request->m_messageQueue.toString().c_str()); @@ -611,11 +611,11 @@ bool DefaultMQPushConsumer::producePullMsgTask(boost::weak_ptr pull return true; } -void DefaultMQPushConsumer::runPullMsgQueue(TaskQueue* pTaskQueue) { +void DefaultMQPushConsumerImpl::runPullMsgQueue(TaskQueue* pTaskQueue) { pTaskQueue->run(); } -void DefaultMQPushConsumer::pullMessage(boost::weak_ptr pullRequest) { +void DefaultMQPushConsumerImpl::pullMessage(boost::weak_ptr pullRequest) { boost::shared_ptr request = pullRequest.lock(); if (!request) { LOG_ERROR("Pull request is released, return"); @@ -771,7 +771,7 @@ void DefaultMQPushConsumer::pullMessage(boost::weak_ptr pullRequest } } -AsyncPullCallback* DefaultMQPushConsumer::getAsyncPullCallBack(boost::weak_ptr pullRequest, +AsyncPullCallback* DefaultMQPushConsumerImpl::getAsyncPullCallBack(boost::weak_ptr pullRequest, MQMessageQueue msgQueue) { boost::shared_ptr request = pullRequest.lock(); if (!request) { @@ -794,7 +794,7 @@ AsyncPullCallback* DefaultMQPushConsumer::getAsyncPullCallBack(boost::weak_ptr

lock(m_asyncCallbackLock); if (m_asyncPull) { PullMAP::iterator it = m_PullCallback.begin(); @@ -808,7 +808,7 @@ void DefaultMQPushConsumer::shutdownAsyncPullCallBack() { } } -void DefaultMQPushConsumer::pullMessageAsync(boost::weak_ptr pullRequest) { +void DefaultMQPushConsumerImpl::pullMessageAsync(boost::weak_ptr pullRequest) { boost::shared_ptr request = pullRequest.lock(); if (!request) { LOG_ERROR("Pull request is released, return"); @@ -904,7 +904,7 @@ void DefaultMQPushConsumer::pullMessageAsync(boost::weak_ptr pullRe } } -void DefaultMQPushConsumer::setAsyncPull(bool asyncFlag) { +void DefaultMQPushConsumerImpl::setAsyncPull(bool asyncFlag) { if (asyncFlag) { LOG_INFO("set pushConsumer:%s to async default pull mode", getGroupName().c_str()); } else { @@ -913,7 +913,7 @@ void DefaultMQPushConsumer::setAsyncPull(bool asyncFlag) { m_asyncPull = asyncFlag; } -void DefaultMQPushConsumer::setConsumeThreadCount(int threadCount) { +void DefaultMQPushConsumerImpl::setConsumeThreadCount(int threadCount) { if (threadCount > 0) { m_consumeThreadCount = threadCount; } else { @@ -921,10 +921,10 @@ void DefaultMQPushConsumer::setConsumeThreadCount(int threadCount) { } } -int DefaultMQPushConsumer::getConsumeThreadCount() const { +int DefaultMQPushConsumerImpl::getConsumeThreadCount() const { return m_consumeThreadCount; } -void DefaultMQPushConsumer::setMaxReconsumeTimes(int maxReconsumeTimes) { +void DefaultMQPushConsumerImpl::setMaxReconsumeTimes(int maxReconsumeTimes) { if (maxReconsumeTimes > 0) { m_maxReconsumeTimes = maxReconsumeTimes; } else { @@ -932,7 +932,7 @@ void DefaultMQPushConsumer::setMaxReconsumeTimes(int maxReconsumeTimes) { } } -int DefaultMQPushConsumer::getMaxReconsumeTimes() const { +int DefaultMQPushConsumerImpl::getMaxReconsumeTimes() const { if (m_maxReconsumeTimes >= 0) { return m_maxReconsumeTimes; } @@ -940,35 +940,35 @@ int DefaultMQPushConsumer::getMaxReconsumeTimes() const { return 16; } -void DefaultMQPushConsumer::setPullMsgThreadPoolCount(int threadCount) { +void DefaultMQPushConsumerImpl::setPullMsgThreadPoolCount(int threadCount) { m_pullMsgThreadPoolNum = threadCount; } -int DefaultMQPushConsumer::getPullMsgThreadPoolCount() const { +int DefaultMQPushConsumerImpl::getPullMsgThreadPoolCount() const { return m_pullMsgThreadPoolNum; } -int DefaultMQPushConsumer::getConsumeMessageBatchMaxSize() const { +int DefaultMQPushConsumerImpl::getConsumeMessageBatchMaxSize() const { return m_consumeMessageBatchMaxSize; } -void DefaultMQPushConsumer::setConsumeMessageBatchMaxSize(int consumeMessageBatchMaxSize) { +void DefaultMQPushConsumerImpl::setConsumeMessageBatchMaxSize(int consumeMessageBatchMaxSize) { if (consumeMessageBatchMaxSize >= 1) m_consumeMessageBatchMaxSize = consumeMessageBatchMaxSize; } -void DefaultMQPushConsumer::setMaxCacheMsgSizePerQueue(int maxCacheSize) { +void DefaultMQPushConsumerImpl::setMaxCacheMsgSizePerQueue(int maxCacheSize) { if (maxCacheSize > 0 && maxCacheSize < 65535) { LOG_INFO("set maxCacheSize to:%d for consumer:%s", maxCacheSize, getGroupName().c_str()); m_maxMsgCacheSize = maxCacheSize; } } -int DefaultMQPushConsumer::getMaxCacheMsgSizePerQueue() const { +int DefaultMQPushConsumerImpl::getMaxCacheMsgSizePerQueue() const { return m_maxMsgCacheSize; } -ConsumerRunningInfo* DefaultMQPushConsumer::getConsumerRunningInfo() { +ConsumerRunningInfo* DefaultMQPushConsumerImpl::getConsumerRunningInfo() { auto* info = new ConsumerRunningInfo(); if (m_consumerService->getConsumeMsgSerivceListenerType() == messageListenerOrderly) { info->setProperty(ConsumerRunningInfo::PROP_CONSUME_ORDERLY, "true"); @@ -1005,7 +1005,7 @@ ConsumerRunningInfo* DefaultMQPushConsumer::getConsumerRunningInfo() { return info; } // we should deal with name space before producer start. -bool DefaultMQPushConsumer::dealWithNameSpace() { +bool DefaultMQPushConsumerImpl::dealWithNameSpace() { string ns = getNameSpace(); if (ns.empty()) { string nsAddr = getNamesrvAddr(); diff --git a/src/consumer/DefaultMQPushConsumerImpl.h b/src/consumer/DefaultMQPushConsumerImpl.h new file mode 100644 index 000000000..0a75b56da --- /dev/null +++ b/src/consumer/DefaultMQPushConsumerImpl.h @@ -0,0 +1,165 @@ +/* + * 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. + */ + +#ifndef __DEFAULTMQPUSHCONSUMERIMPL_H__ +#define __DEFAULTMQPUSHCONSUMERIMPL_H__ + +#include +#include +#include +#include +#include +#include +#include +#include "AsyncCallback.h" +#include "MQConsumer.h" +#include "MQMessageListener.h" +#include "MQMessageQueue.h" + +namespace rocketmq { + +class Rebalance; +class SubscriptionData; +class OffsetStore; +class PullAPIWrapper; +class PullRequest; +class ConsumeMsgService; +class TaskQueue; +class TaskThread; +class AsyncPullCallback; +class ConsumerRunningInfo; +//& mqs); + virtual void doRebalance(); + virtual void persistConsumerOffset(); + virtual void persistConsumerOffsetByResetOffset(); + virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info); + virtual ConsumeType getConsumeType(); + virtual ConsumeFromWhere getConsumeFromWhere(); + void setConsumeFromWhere(ConsumeFromWhere consumeFromWhere); + virtual void getSubscriptions(std::vector&); + virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset); + virtual void removeConsumeOffset(const MQMessageQueue& mq); + virtual PullResult pull(const MQMessageQueue& mq, const std::string& subExpression, int64 offset, int maxNums) { + return PullResult(); + } + virtual void pull(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + PullCallback* pPullCallback) {} + virtual ConsumerRunningInfo* getConsumerRunningInfo(); + //); + virtual bool producePullMsgTaskLater(boost::weak_ptr, int millis); + static void static_triggerNextPullRequest(void* context, + boost::asio::deadline_timer* t, + boost::weak_ptr); + void triggerNextPullRequest(boost::asio::deadline_timer* t, boost::weak_ptr); + void runPullMsgQueue(TaskQueue* pTaskQueue); + void pullMessage(boost::weak_ptr pullrequest); + void pullMessageAsync(boost::weak_ptr pullrequest); + void setAsyncPull(bool asyncFlag); + AsyncPullCallback* getAsyncPullCallBack(boost::weak_ptr, MQMessageQueue msgQueue); + void shutdownAsyncPullCallBack(); + + /* + for orderly consume, set the pull num of message size by each pullMsg, + default value is 1; + */ + void setConsumeMessageBatchMaxSize(int consumeMessageBatchMaxSize); + int getConsumeMessageBatchMaxSize() const; + + /* + set consuming thread count, default value is cpu cores + */ + void setConsumeThreadCount(int threadCount); + int getConsumeThreadCount() const; + void setMaxReconsumeTimes(int maxReconsumeTimes); + int getMaxReconsumeTimes() const; + + /* + set pullMsg thread count, default value is cpu cores + */ + void setPullMsgThreadPoolCount(int threadCount); + int getPullMsgThreadPoolCount() const; + + /* + set max cache msg size perQueue in memory if consumer could not consume msgs + immediately + default maxCacheMsgSize perQueue is 1000, set range is:1~65535 + */ + void setMaxCacheMsgSizePerQueue(int maxCacheSize); + int getMaxCacheMsgSizePerQueue() const; + + private: + void checkConfig(); + void copySubscription(); + void updateTopicSubscribeInfoWhenSubscriptionChanged(); + bool dealWithNameSpace(); + + private: + uint64_t m_startTime; + ConsumeFromWhere m_consumeFromWhere; + std::map m_subTopics; + int m_consumeThreadCount; + OffsetStore* m_pOffsetStore; + Rebalance* m_pRebalance; + PullAPIWrapper* m_pPullAPIWrapper; + ConsumeMsgService* m_consumerService; + MQMessageListener* m_pMessageListener; + int m_consumeMessageBatchMaxSize; + int m_maxMsgCacheSize; + int m_maxReconsumeTimes = -1; + boost::asio::io_service m_async_ioService; + boost::scoped_ptr m_async_service_thread; + + typedef std::map PullMAP; + PullMAP m_PullCallback; + bool m_asyncPull; + int m_asyncPullTimeout; + int m_pullMsgThreadPoolNum; + + private: + TaskQueue* m_pullmsgQueue; + std::unique_ptr m_pullmsgThread; +}; +//(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = static_cast(m_pConsumer); OffsetStore* pOffsetStore = pConsumer->getOffsetStore(); vector mqs; { @@ -174,7 +174,7 @@ void Rebalance::persistConsumerOffset() { } void Rebalance::persistConsumerOffsetByResetOffset() { - DefaultMQPushConsumer* pConsumer = static_cast(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = static_cast(m_pConsumer); OffsetStore* pOffsetStore = pConsumer->getOffsetStore(); vector mqs; { @@ -552,7 +552,7 @@ bool RebalancePush::updateRequestTableInRebalance(const string& topic, vector(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = dynamic_cast(m_pConsumer); if (!pConsumer) { LOG_ERROR("Cast consumer pointer to DefaultMQPushConsumer pointer failed when computePullFromWhere %s", mq.toString().c_str()); @@ -639,7 +639,7 @@ void RebalancePush::messageQueueChanged(const string& topic, void RebalancePush::removeUnnecessaryMessageQueue(const MQMessageQueue& mq) { // DefaultMQPushConsumer *pConsumer = static_cast(m_pConsumer); - DefaultMQPushConsumer* pConsumer = dynamic_cast(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = dynamic_cast(m_pConsumer); if (!pConsumer) { LOG_ERROR("Cast MQConsumer* to DefaultMQPushConsumer* failed when remove %s", mq.toString().c_str()); return; diff --git a/src/extern/CProducer.cpp b/src/extern/CProducer.cpp index 397d77235..94f1bdf34 100644 --- a/src/extern/CProducer.cpp +++ b/src/extern/CProducer.cpp @@ -31,6 +31,7 @@ #include "TransactionListener.h" #include "TransactionMQProducer.h" #include "TransactionSendResult.h" +#include "MQClient.h" #include "UtilAll.h" #ifdef __cplusplus diff --git a/include/BatchMessage.h b/src/include/BatchMessage.h similarity index 100% rename from include/BatchMessage.h rename to src/include/BatchMessage.h diff --git a/src/include/DefaultMQClient.h b/src/include/DefaultMQClient.h new file mode 100644 index 000000000..b8d8ba1bc --- /dev/null +++ b/src/include/DefaultMQClient.h @@ -0,0 +1,194 @@ +/* + * 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. + */ + +#ifndef __DEFAULTMQADMIN_H__ +#define __DEFAULTMQADMIN_H__ +#include +#include +#include +#include +#include +#include +#include "MQMessageExt.h" +#include "MQMessageQueue.h" +#include "QueryResult.h" +#include "RocketMQClient.h" +#include "SessionCredentials.h" +#include "MQClient.h" + +namespace rocketmq { +class MQClientFactory; +// getTopicMessageQueueInfo(const std::string& topic); + + // log configuration interface, default LOG_LEVEL is LOG_LEVEL_INFO, default + // log file num is 3, each log size is 100M + void setLogLevel(elogLevel inputLevel); + elogLevel getLogLevel(); + void setLogFileSizeAndNum(int fileNum, long perFileSize); // perFileSize is MB unit + + /** set TcpTransport pull thread num, which dermine the num of threads to + distribute network data, + 1. its default value is CPU num, it must be setted before producer/consumer + start, minimum value is CPU num; + 2. this pullThread num must be tested on your environment to find the best + value for RT of sendMsg or delay time of consume msg before you change it; + 3. producer and consumer need different pullThread num, if set this num, + producer and consumer must set different instanceName. + 4. configuration suggestion: + 1>. minimum RT of sendMsg: + pullThreadNum = brokerNum*2 + **/ + void setTcpTransportPullThreadNum(int num); + const int getTcpTransportPullThreadNum() const; + + /** timeout of tcp connect, it is same meaning for both producer and consumer; + 1. default value is 3000ms + 2. input parameter could only be milliSecond, suggestion value is + 1000-3000ms; + **/ + void setTcpTransportConnectTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportConnectTimeout() const; + + /** timeout of tryLock tcpTransport before sendMsg/pullMsg, if timeout, + returns NULL + 1. paremeter unit is ms, default value is 3000ms, the minimun value is + 1000ms + suggestion value is 3000ms; + 2. if configured with value smaller than 1000ms, the tryLockTimeout value + will be setted to 1000ms + **/ + void setTcpTransportTryLockTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportTryLockTimeout() const; + + void setUnitName(std::string unitName); + const std::string& getUnitName() const; + + void setSessionCredentials(const std::string& input_accessKey, + const std::string& input_secretKey, + const std::string& input_onsChannel); + const SessionCredentials& getSessionCredentials() const; + + protected: + virtual void start(); + virtual void shutdown(); + MQClientFactory* getFactory() const; + virtual bool isServiceStateOk(); + + protected: + std::string m_namesrvAddr; + std::string m_namesrvDomain; + std::string m_instanceName; + std::string m_nameSpace; + std::string m_GroupName; + MQClientFactory* m_clientFactory; + int m_serviceState; + int m_pullThreadNum; + uint64_t m_tcpConnectTimeout; // ms + uint64_t m_tcpTransportTryLockTimeout; // s + + std::string m_unitName; + SessionCredentials m_SessionCredentials; +}; +// -#include "AsyncCallback.h" -#include "ConsumeType.h" -#include "MQClient.h" -#include "RocketMQClient.h" - -namespace rocketmq { -class SubscriptionData; -class PullRequest; -class Rebalance; -class ConsumerRunningInfo; -//& mqs) = 0; - virtual void doRebalance() = 0; - virtual void persistConsumerOffset() = 0; - virtual void persistConsumerOffsetByResetOffset() = 0; - virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info) = 0; - virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset) = 0; - virtual void removeConsumeOffset(const MQMessageQueue& mq) = 0; - virtual ConsumeType getConsumeType() = 0; - virtual ConsumeFromWhere getConsumeFromWhere() = 0; - virtual void getSubscriptions(std::vector&) = 0; - virtual bool producePullMsgTask(boost::weak_ptr) = 0; - virtual Rebalance* getRebalance() const = 0; - virtual PullResult pull(const MQMessageQueue& mq, const std::string& subExpression, int64 offset, int maxNums) = 0; - virtual void pull(const MQMessageQueue& mq, - const std::string& subExpression, - int64 offset, - int maxNums, - PullCallback* pPullCallback) = 0; - virtual ConsumerRunningInfo* getConsumerRunningInfo() = 0; - - public: - MessageModel getMessageModel() const { return m_messageModel; } - void setMessageModel(MessageModel messageModel) { m_messageModel = messageModel; } - bool isUseNameSpaceMode() const { return m_useNameSpaceMode; } - - protected: - MessageModel m_messageModel; - bool m_useNameSpaceMode = false; -}; - -// +#include "AsyncCallback.h" +#include "ConsumeType.h" +#include "DefaultMQClient.h" +#include "RocketMQClient.h" + +namespace rocketmq { +class SubscriptionData; +class PullRequest; +class Rebalance; +class ConsumerRunningInfo; +//& mqs) = 0; + virtual void doRebalance() = 0; + virtual void persistConsumerOffset() = 0; + virtual void persistConsumerOffsetByResetOffset() = 0; + virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info) = 0; + virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset) = 0; + virtual void removeConsumeOffset(const MQMessageQueue& mq) = 0; + virtual ConsumeType getConsumeType() = 0; + virtual ConsumeFromWhere getConsumeFromWhere() = 0; + virtual void getSubscriptions(std::vector&) = 0; + virtual bool producePullMsgTask(boost::weak_ptr) = 0; + virtual Rebalance* getRebalance() const = 0; + virtual PullResult pull(const MQMessageQueue& mq, const std::string& subExpression, int64 offset, int maxNums) = 0; + virtual void pull(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + PullCallback* pPullCallback) = 0; + virtual ConsumerRunningInfo* getConsumerRunningInfo() = 0; + + public: + MessageModel getMessageModel() const { return m_messageModel; } + void setMessageModel(MessageModel messageModel) { m_messageModel = messageModel; } + bool isUseNameSpaceMode() const { return m_useNameSpaceMode; } + + protected: + MessageModel m_messageModel; + bool m_useNameSpaceMode = false; +}; + +//& msgs) = 0; - virtual SendResult send(std::vector& msgs, const MQMessageQueue& mq) = 0; - virtual void sendOneway(MQMessage& msg, bool bSelectActiveBroker = false) = 0; - virtual void sendOneway(MQMessage& msg, const MQMessageQueue& mq) = 0; - virtual void sendOneway(MQMessage& msg, MessageQueueSelector* selector, void* arg) = 0; -}; -//& msgs) = 0; + virtual SendResult send(std::vector& msgs, const MQMessageQueue& mq) = 0; + virtual void sendOneway(MQMessage& msg, bool bSelectActiveBroker = false) = 0; + virtual void sendOneway(MQMessage& msg, const MQMessageQueue& mq) = 0; + virtual void sendOneway(MQMessage& msg, MessageQueueSelector* selector, void* arg) = 0; +}; +//& messageList) { - m_indexLastUpdateTimestamp = indexLastUpdateTimestamp; - m_messageList = messageList; - } - - uint64 getIndexLastUpdateTimestamp() { return m_indexLastUpdateTimestamp; } - - std::vector& getMessageList() { return m_messageList; } - - private: - uint64 m_indexLastUpdateTimestamp; - std::vector m_messageList; -}; -//& messageList) { + m_indexLastUpdateTimestamp = indexLastUpdateTimestamp; + m_messageList = messageList; + } + + uint64 getIndexLastUpdateTimestamp() { return m_indexLastUpdateTimestamp; } + + std::vector& getMessageList() { return m_messageList; } + + private: + uint64 m_indexLastUpdateTimestamp; + std::vector m_messageList; +}; +// #include #include -#include "MQClient.h" +#include "include/DefaultMQClient.h" namespace logging = boost::log; namespace src = boost::log::sources; diff --git a/src/message/BatchMessage.cpp b/src/message/BatchMessage.cpp index 3a2481bd3..9bcca1cf0 100644 --- a/src/message/BatchMessage.cpp +++ b/src/message/BatchMessage.cpp @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "BatchMessage.h" +#include "include/BatchMessage.h" #include "MQDecoder.h" #include "StringIdMaker.h" diff --git a/src/producer/DefaultMQProducer.cpp b/src/producer/DefaultMQProducer.cpp index ee81a7b8d..9a6d41496 100644 --- a/src/producer/DefaultMQProducer.cpp +++ b/src/producer/DefaultMQProducer.cpp @@ -1,635 +1,207 @@ -/* - * 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. - */ - #include "DefaultMQProducer.h" -#include -#include - -#include "BatchMessage.h" -#include "CommandHeader.h" -#include "CommunicationMode.h" -#include "Logging.h" -#include "MQClientAPIImpl.h" -#include "MQClientException.h" -#include "MQClientFactory.h" -#include "MQClientManager.h" -#include "MQDecoder.h" -#include "MQProtos.h" -#include "MessageAccessor.h" -#include "NameSpaceUtil.h" -#include "StringIdMaker.h" -#include "TopicPublishInfo.h" -#include "Validators.h" +#include "DefaultMQProducerImpl.h" namespace rocketmq { -//registerProducer(this); - if (!registerOK) { - m_serviceState = CREATE_JUST; - THROW_MQEXCEPTION( - MQClientException, - "The producer group[" + getGroupName() + "] has been created before, specify another name please.", -1); - } - - getFactory()->start(); - getFactory()->sendHeartbeatToAllBroker(); - m_serviceState = RUNNING; - break; - } - case RUNNING: - case START_FAILED: - case SHUTDOWN_ALREADY: - break; - default: - break; - } + impl->start(); } void DefaultMQProducer::shutdown() { - switch (m_serviceState) { - case RUNNING: { - LOG_INFO("DefaultMQProducer shutdown"); - getFactory()->unregisterProducer(this); - getFactory()->shutdown(); - m_serviceState = SHUTDOWN_ALREADY; - break; - } - case SHUTDOWN_ALREADY: - case CREATE_JUST: - break; - default: - break; - } + impl->shutdown(); } -SendResult DefaultMQProducer::send(MQMessage& msg, bool bSelectActiveBroker) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - try { - return sendDefaultImpl(msg, ComMode_SYNC, NULL, bSelectActiveBroker); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return SendResult(); +// start mqclient set +const std::string& DefaultMQProducer::getNamesrvAddr() const { + return impl->getNamesrvAddr(); } -void DefaultMQProducer::send(MQMessage& msg, SendCallback* pSendCallback, bool bSelectActiveBroker) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - try { - sendDefaultImpl(msg, ComMode_ASYNC, pSendCallback, bSelectActiveBroker); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } +void DefaultMQProducer::setNamesrvAddr(const std::string& namesrvAddr) { + impl->setNamesrvAddr(namesrvAddr); } -SendResult DefaultMQProducer::send(std::vector& msgs) { - SendResult result; - try { - BatchMessage batchMessage = buildBatchMessage(msgs); - result = sendDefaultImpl(batchMessage, ComMode_SYNC, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return result; +const std::string& DefaultMQProducer::getNamesrvDomain() const { + return impl->getNamesrvDomain(); } -SendResult DefaultMQProducer::send(std::vector& msgs, const MQMessageQueue& mq) { - SendResult result; - try { - BatchMessage batchMessage = buildBatchMessage(msgs); - result = sendKernelImpl(batchMessage, mq, ComMode_SYNC, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return result; -} - -BatchMessage DefaultMQProducer::buildBatchMessage(std::vector& msgs) { - if (msgs.size() < 1) { - THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -1); - } - BatchMessage batchMessage; - bool firstFlag = true; - string topic; - bool waitStoreMsgOK = false; - for (auto& msg : msgs) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - if (firstFlag) { - topic = msg.getTopic(); - waitStoreMsgOK = msg.isWaitStoreMsgOK(); - firstFlag = false; - - if (UtilAll::startsWith_retry(topic)) { - THROW_MQEXCEPTION(MQClientException, "Retry Group is not supported for batching", -1); - } - } else { - if (msg.getDelayTimeLevel() > 0) { - THROW_MQEXCEPTION(MQClientException, "TimeDelayLevel in not supported for batching", -1); - } - if (msg.getTopic() != topic) { - THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -1); - } - if (msg.isWaitStoreMsgOK() != waitStoreMsgOK) { - THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -2); - } - } - } - batchMessage.setBody(BatchMessage::encode(msgs)); - batchMessage.setTopic(topic); - batchMessage.setWaitStoreMsgOK(waitStoreMsgOK); - return batchMessage; +void DefaultMQProducer::setNamesrvDomain(const std::string& namesrvDomain) { + impl->setNamesrvDomain(namesrvDomain); +} +void DefaultMQProducer::setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel) { + impl->setSessionCredentials(accessKey, secretKey, accessChannel); } -SendResult DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - if (msg.getTopic() != mq.getTopic()) { - LOG_WARN("message's topic not equal mq's topic"); - } - try { - return sendKernelImpl(msg, mq, ComMode_SYNC, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return SendResult(); +const SessionCredentials& DefaultMQProducer::getSessionCredentials() const { + return impl->getSessionCredentials(); +} +const std::string& DefaultMQProducer::getInstanceName() const { + return impl->getInstanceName(); } -void DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq, SendCallback* pSendCallback) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - if (msg.getTopic() != mq.getTopic()) { - LOG_WARN("message's topic not equal mq's topic"); - } - try { - sendKernelImpl(msg, mq, ComMode_ASYNC, pSendCallback); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } +void DefaultMQProducer::setInstanceName(const std::string& instanceName) { + impl->setInstanceName(instanceName); } -void DefaultMQProducer::sendOneway(MQMessage& msg, bool bSelectActiveBroker) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - try { - sendDefaultImpl(msg, ComMode_ONEWAY, NULL, bSelectActiveBroker); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } +const std::string& DefaultMQProducer::getNameSpace() const { + return impl->getNameSpace(); } -void DefaultMQProducer::sendOneway(MQMessage& msg, const MQMessageQueue& mq) { - Validators::checkMessage(msg, getMaxMessageSize()); - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - if (msg.getTopic() != mq.getTopic()) { - LOG_WARN("message's topic not equal mq's topic"); - } - try { - sendKernelImpl(msg, mq, ComMode_ONEWAY, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } -} - -SendResult DefaultMQProducer::send(MQMessage& msg, MessageQueueSelector* pSelector, void* arg) { - try { - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - return sendSelectImpl(msg, pSelector, arg, ComMode_SYNC, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return SendResult(); +void DefaultMQProducer::setNameSpace(const std::string& nameSpace) { + impl->setNameSpace(nameSpace); +} +const std::string& DefaultMQProducer::getGroupName() const { + return impl->getGroupName(); } -SendResult DefaultMQProducer::send(MQMessage& msg, - MessageQueueSelector* pSelector, - void* arg, - int autoRetryTimes, - bool bActiveBroker) { - try { - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - return sendAutoRetrySelectImpl(msg, pSelector, arg, ComMode_SYNC, NULL, autoRetryTimes, bActiveBroker); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } - return SendResult(); -} - -void DefaultMQProducer::send(MQMessage& msg, MessageQueueSelector* pSelector, void* arg, SendCallback* pSendCallback) { - try { - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - sendSelectImpl(msg, pSelector, arg, ComMode_ASYNC, pSendCallback); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } -} - -void DefaultMQProducer::sendOneway(MQMessage& msg, MessageQueueSelector* pSelector, void* arg) { - try { - if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { - MessageAccessor::withNameSpace(msg, getNameSpace()); - } - sendSelectImpl(msg, pSelector, arg, ComMode_ONEWAY, NULL); - } catch (MQException& e) { - LOG_ERROR(e.what()); - throw e; - } +void DefaultMQProducer::setGroupName(const std::string& groupName) { + impl->setGroupName(groupName); +} +void DefaultMQProducer::setSendMsgTimeout(int sendMsgTimeout) { + impl->setSendMsgTimeout(sendMsgTimeout); } int DefaultMQProducer::getSendMsgTimeout() const { - return m_sendMsgTimeout; + return impl->getSendMsgTimeout(); } -void DefaultMQProducer::setSendMsgTimeout(int sendMsgTimeout) { - m_sendMsgTimeout = sendMsgTimeout; +void DefaultMQProducer::setRetryTimes(int times) { + impl->setRetryTimes(times); +} + +int DefaultMQProducer::getRetryTimes() const { + return impl->getRetryTimes(); } int DefaultMQProducer::getCompressMsgBodyOverHowmuch() const { - return m_compressMsgBodyOverHowmuch; + return impl->getCompressMsgBodyOverHowmuch(); } -void DefaultMQProducer::setCompressMsgBodyOverHowmuch(int compressMsgBodyOverHowmuch) { - m_compressMsgBodyOverHowmuch = compressMsgBodyOverHowmuch; +void DefaultMQProducer::setCompressMsgBodyOverHowmuch(int compressMsgBodyThreshold) { + impl->setCompressMsgBodyOverHowmuch(compressMsgBodyThreshold); } -int DefaultMQProducer::getMaxMessageSize() const { - return m_maxMessageSize; +int DefaultMQProducer::getCompressLevel() const { + return impl->getCompressLevel(); +} + +void DefaultMQProducer::setCompressLevel(int compressLevel) { + impl->setCompressLevel(compressLevel); } void DefaultMQProducer::setMaxMessageSize(int maxMessageSize) { - m_maxMessageSize = maxMessageSize; + impl->setMaxMessageSize(maxMessageSize); } -int DefaultMQProducer::getCompressLevel() const { - return m_compressLevel; +int DefaultMQProducer::getMaxMessageSize() const { + return impl->getMaxMessageSize(); } -void DefaultMQProducer::setCompressLevel(int compressLevel) { - assert((compressLevel >= 0 && compressLevel <= 9) || compressLevel == -1); - - m_compressLevel = compressLevel; -} - -// weak_topicPublishInfo( - getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); - boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); - if (topicPublishInfo) { - if (times == 1) { - mq_index = topicPublishInfo->getWhichQueue(); - } else { - mq_index++; - } - - SendResult sendResult; - MQMessageQueue mq; - if (bActiveMQ) - mq = topicPublishInfo->selectOneActiveMessageQueue(lastmq, mq_index); - else - mq = topicPublishInfo->selectOneMessageQueue(lastmq, mq_index); - - lastmq = mq; - if (mq.getQueueId() == -1) { - // THROW_MQEXCEPTION(MQClientException, "the MQMessageQueue is - // invalide", -1); - continue; - } - - try { - LOG_DEBUG("send to mq:%s", mq.toString().data()); - sendResult = sendKernelImpl(msg, mq, communicationMode, pSendCallback); - switch (communicationMode) { - case ComMode_ASYNC: - return sendResult; - case ComMode_ONEWAY: - return sendResult; - case ComMode_SYNC: - if (sendResult.getSendStatus() != SEND_OK) { - if (bActiveMQ) { - topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); - } - continue; - } - return sendResult; - default: - break; - } - } catch (...) { - LOG_ERROR("send failed of times:%d,brokerName:%s", times, mq.getBrokerName().c_str()); - if (bActiveMQ) { - topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); - } - continue; - } - } // end of for - LOG_WARN("Retry many times, still failed"); - } - string info = "No route info of this topic: " + msg.getTopic(); - THROW_MQEXCEPTION(MQClientException, info, -1); -} - -SendResult DefaultMQProducer::sendKernelImpl(MQMessage& msg, - const MQMessageQueue& mq, - int communicationMode, - SendCallback* sendCallback) { - string brokerAddr = getFactory()->findBrokerAddressInPublish(mq.getBrokerName()); - - if (brokerAddr.empty()) { - getFactory()->tryToFindTopicPublishInfo(mq.getTopic(), getSessionCredentials()); - brokerAddr = getFactory()->findBrokerAddressInPublish(mq.getBrokerName()); - } - - if (!brokerAddr.empty()) { - try { - bool isBatchMsg = std::type_index(typeid(msg)) == std::type_index(typeid(BatchMessage)); - // msgId is produced by client, offsetMsgId produced by broker. (same with java sdk) - if (!isBatchMsg) { - string unique_id = StringIdMaker::getInstance().createUniqID(); - msg.setProperty(MQMessage::PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, unique_id); - - // batch does not support compressing right now, - tryToCompressMessage(msg); - } - - LOG_DEBUG("produce before:%s to %s", msg.toString().c_str(), mq.toString().c_str()); - - SendMessageRequestHeader* requestHeader = new SendMessageRequestHeader(); - requestHeader->producerGroup = getGroupName(); - requestHeader->topic = (msg.getTopic()); - requestHeader->defaultTopic = DEFAULT_TOPIC; - requestHeader->defaultTopicQueueNums = 4; - requestHeader->queueId = (mq.getQueueId()); - requestHeader->sysFlag = (msg.getSysFlag()); - requestHeader->bornTimestamp = UtilAll::currentTimeMillis(); - requestHeader->flag = (msg.getFlag()); - requestHeader->consumeRetryTimes = 16; - requestHeader->batch = isBatchMsg; - requestHeader->properties = (MQDecoder::messageProperties2String(msg.getProperties())); - - return getFactory()->getMQClientAPIImpl()->sendMessage(brokerAddr, mq.getBrokerName(), msg, requestHeader, - getSendMsgTimeout(), getRetryTimes4Async(), - communicationMode, sendCallback, getSessionCredentials()); - } catch (MQException& e) { - throw e; - } - } - THROW_MQEXCEPTION(MQClientException, "The broker[" + mq.getBrokerName() + "] not exist", -1); -} - -SendResult DefaultMQProducer::sendSelectImpl(MQMessage& msg, - MessageQueueSelector* pSelector, - void* pArg, - int communicationMode, - SendCallback* sendCallback) { - Validators::checkMessage(msg, getMaxMessageSize()); - - boost::weak_ptr weak_topicPublishInfo( - getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); - boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); - if (topicPublishInfo) //&& topicPublishInfo->ok()) - { - MQMessageQueue mq = pSelector->select(topicPublishInfo->getMessageQueueList(), msg, pArg); - return sendKernelImpl(msg, mq, communicationMode, sendCallback); - } - THROW_MQEXCEPTION(MQClientException, "No route info for this topic", -1); -} - -SendResult DefaultMQProducer::sendAutoRetrySelectImpl(MQMessage& msg, - MessageQueueSelector* pSelector, - void* pArg, - int communicationMode, - SendCallback* pSendCallback, - int autoRetryTimes, - bool bActiveMQ) { - Validators::checkMessage(msg, getMaxMessageSize()); - - MQMessageQueue lastmq; - MQMessageQueue mq; - int mq_index = 0; - for (int times = 1; times <= autoRetryTimes + 1; times++) { - boost::weak_ptr weak_topicPublishInfo( - getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); - boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); - if (topicPublishInfo) { - SendResult sendResult; - if (times == 1) { - // always send to selected MQ firstly, evenif bActiveMQ was setted to true - mq = pSelector->select(topicPublishInfo->getMessageQueueList(), msg, pArg); - lastmq = mq; - } else { - LOG_INFO("sendAutoRetrySelectImpl with times:%d", times); - std::vector mqs(topicPublishInfo->getMessageQueueList()); - for (size_t i = 0; i < mqs.size(); i++) { - if (mqs[i] == lastmq) - mq_index = i; - } - if (bActiveMQ) - mq = topicPublishInfo->selectOneActiveMessageQueue(lastmq, mq_index); - else - mq = topicPublishInfo->selectOneMessageQueue(lastmq, mq_index); - lastmq = mq; - if (mq.getQueueId() == -1) { - // THROW_MQEXCEPTION(MQClientException, "the MQMessageQueue is - // invalide", -1); - continue; - } - } - - try { - LOG_DEBUG("send to broker:%s", mq.toString().c_str()); - sendResult = sendKernelImpl(msg, mq, communicationMode, pSendCallback); - switch (communicationMode) { - case ComMode_ASYNC: - return sendResult; - case ComMode_ONEWAY: - return sendResult; - case ComMode_SYNC: - if (sendResult.getSendStatus() != SEND_OK) { - if (bActiveMQ) { - topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); - } - continue; - } - return sendResult; - default: - break; - } - } catch (...) { - LOG_ERROR("send failed of times:%d,mq:%s", times, mq.toString().c_str()); - if (bActiveMQ) { - topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); - } - continue; - } - } // end of for - LOG_WARN("Retry many times, still failed"); - } - THROW_MQEXCEPTION(MQClientException, "No route info of this topic, ", -1); -} - -bool DefaultMQProducer::tryToCompressMessage(MQMessage& msg) { - int sysFlag = msg.getSysFlag(); - if ((sysFlag & MessageSysFlag::CompressedFlag) == MessageSysFlag::CompressedFlag) { - return true; - } - - string body = msg.getBody(); - if ((int)body.length() >= getCompressMsgBodyOverHowmuch()) { - string outBody; - if (UtilAll::deflate(body, outBody, getCompressLevel())) { - msg.setBody(outBody); - msg.setSysFlag(sysFlag | MessageSysFlag::CompressedFlag); - return true; - } - } - - return false; +void DefaultMQProducer::setRetryTimes4Async(int times) { + impl->setRetryTimes4Async(times); } -int DefaultMQProducer::getRetryTimes() const { - return m_retryTimes; +int DefaultMQProducer::getRetryTimes4Async() const { + return impl->getRetryTimes4Async(); } -void DefaultMQProducer::setRetryTimes(int times) { - if (times <= 0) { - LOG_WARN("set retry times illegal, use default value:5"); - return; - } +void DefaultMQProducer::setLogLevel(elogLevel inputLevel) { + impl->setLogLevel(inputLevel); +} - if (times > 15) { - LOG_WARN("set retry times illegal, use max value:15"); - m_retryTimes = 15; - return; - } - LOG_WARN("set retry times to:%d", times); - m_retryTimes = times; +elogLevel DefaultMQProducer::getLogLevel() { + return impl->getLogLevel(); +} +void DefaultMQProducer::setLogFileSizeAndNum(int fileNum, long perFileSize) { + impl->setLogFileSizeAndNum(fileNum, perFileSize); } -int DefaultMQProducer::getRetryTimes4Async() const { - return m_retryTimes4Async; +void DefaultMQProducer::setTcpTransportPullThreadNum(int num) { + impl->setTcpTransportPullThreadNum(num); +} +const int DefaultMQProducer::getTcpTransportPullThreadNum() const { + return impl->getTcpTransportPullThreadNum(); } -void DefaultMQProducer::setRetryTimes4Async(int times) { - if (times <= 0) { - LOG_WARN("set retry times illegal, use default value:1"); - m_retryTimes4Async = 1; - return; - } - - if (times > 15) { - LOG_WARN("set retry times illegal, use max value:15"); - m_retryTimes4Async = 15; - return; - } - LOG_INFO("set retry times to:%d", times); - m_retryTimes4Async = times; -} - -// we should deal with name space before producer start. -bool DefaultMQProducer::dealWithNameSpace() { - string ns = getNameSpace(); - if (ns.empty()) { - string nsAddr = getNamesrvAddr(); - if (!NameSpaceUtil::checkNameSpaceExistInNameServer(nsAddr)) { - return true; - } - ns = NameSpaceUtil::getNameSpaceFromNsURL(nsAddr); - // reset namespace - setNameSpace(ns); - } - // reset group name - if (!NameSpaceUtil::hasNameSpace(getGroupName(), ns)) { - string fullGID = NameSpaceUtil::withNameSpace(getGroupName(), ns); - setGroupName(fullGID); - } - return true; -} -//setTcpTransportConnectTimeout(timeout); +} +const uint64_t DefaultMQProducer::getTcpTransportConnectTimeout() const { + return impl->getTcpTransportConnectTimeout(); +} +void DefaultMQProducer::setTcpTransportTryLockTimeout(uint64_t timeout) { + impl->setTcpTransportTryLockTimeout(timeout); +} +const uint64_t DefaultMQProducer::getTcpTransportTryLockTimeout() const { + return impl->getTcpTransportTryLockTimeout(); +} + +void DefaultMQProducer::setUnitName(std::string unitName) { + impl->setUnitName(unitName); +} +const std::string& DefaultMQProducer::getUnitName() const { + return impl->getUnitName(); +} + +SendResult DefaultMQProducer::send(MQMessage& msg, bool bSelectActiveBroker) { + return impl->send(msg, bSelectActiveBroker); +} + +SendResult DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq) { + return impl->send(msg, mq); +} + +SendResult DefaultMQProducer::send(MQMessage& msg, MessageQueueSelector* selector, void* arg) { + return impl->send(msg, selector, arg); +} + +SendResult DefaultMQProducer::send(MQMessage& msg, + MessageQueueSelector* selector, + void* arg, + int autoRetryTimes, + bool bActiveBroker) { + return impl->send(msg, selector, arg, autoRetryTimes, bActiveBroker); +} + +SendResult DefaultMQProducer::send(std::vector& msgs) { + return impl->send(msgs); +} + +SendResult DefaultMQProducer::send(std::vector& msgs, const MQMessageQueue& mq) { + return impl->send(msgs, mq); +} + +void DefaultMQProducer::send(MQMessage& msg, SendCallback* pSendCallback, bool bSelectActiveBroker) { + impl->send(msg, pSendCallback, bSelectActiveBroker); +} + +void DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq, SendCallback* pSendCallback) { + impl->send(msg, mq, pSendCallback); +} + +void DefaultMQProducer::send(MQMessage& msg, MessageQueueSelector* selector, void* arg, SendCallback* pSendCallback) { + impl->send(msg, selector, arg, pSendCallback); +} + +void DefaultMQProducer::sendOneway(MQMessage& msg, bool bSelectActiveBroker) { + impl->sendOneway(msg, bSelectActiveBroker); +} + +void DefaultMQProducer::sendOneway(MQMessage& msg, const MQMessageQueue& mq) { + impl->sendOneway(msg, mq); +} + +void DefaultMQProducer::sendOneway(MQMessage& msg, MessageQueueSelector* selector, void* arg) { + impl->sendOneway(msg, selector, arg); +} + +} // namespace rocketmq \ No newline at end of file diff --git a/src/producer/DefaultMQProducerImpl.cpp b/src/producer/DefaultMQProducerImpl.cpp new file mode 100644 index 000000000..9a1f14509 --- /dev/null +++ b/src/producer/DefaultMQProducerImpl.cpp @@ -0,0 +1,635 @@ +/* + * 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. + */ + +#include "DefaultMQProducerImpl.h" + +#include +#include + +#include "BatchMessage.h" +#include "CommandHeader.h" +#include "CommunicationMode.h" +#include "Logging.h" +#include "MQClientAPIImpl.h" +#include "MQClientException.h" +#include "MQClientFactory.h" +#include "MQClientManager.h" +#include "MQDecoder.h" +#include "MQProtos.h" +#include "MessageAccessor.h" +#include "NameSpaceUtil.h" +#include "StringIdMaker.h" +#include "TopicPublishInfo.h" +#include "Validators.h" + +namespace rocketmq { + +//registerProducer(this); + if (!registerOK) { + m_serviceState = CREATE_JUST; + THROW_MQEXCEPTION( + MQClientException, + "The producer group[" + getGroupName() + "] has been created before, specify another name please.", -1); + } + + getFactory()->start(); + getFactory()->sendHeartbeatToAllBroker(); + m_serviceState = RUNNING; + break; + } + case RUNNING: + case START_FAILED: + case SHUTDOWN_ALREADY: + break; + default: + break; + } +} + +void DefaultMQProducerImpl::shutdown() { + switch (m_serviceState) { + case RUNNING: { + LOG_INFO("DefaultMQProducerImpl shutdown"); + getFactory()->unregisterProducer(this); + getFactory()->shutdown(); + m_serviceState = SHUTDOWN_ALREADY; + break; + } + case SHUTDOWN_ALREADY: + case CREATE_JUST: + break; + default: + break; + } +} + +SendResult DefaultMQProducerImpl::send(MQMessage& msg, bool bSelectActiveBroker) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + try { + return sendDefaultImpl(msg, ComMode_SYNC, NULL, bSelectActiveBroker); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return SendResult(); +} + +void DefaultMQProducerImpl::send(MQMessage& msg, SendCallback* pSendCallback, bool bSelectActiveBroker) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + try { + sendDefaultImpl(msg, ComMode_ASYNC, pSendCallback, bSelectActiveBroker); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +SendResult DefaultMQProducerImpl::send(std::vector& msgs) { + SendResult result; + try { + BatchMessage batchMessage = buildBatchMessage(msgs); + result = sendDefaultImpl(batchMessage, ComMode_SYNC, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return result; +} + +SendResult DefaultMQProducerImpl::send(std::vector& msgs, const MQMessageQueue& mq) { + SendResult result; + try { + BatchMessage batchMessage = buildBatchMessage(msgs); + result = sendKernelImpl(batchMessage, mq, ComMode_SYNC, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return result; +} + +BatchMessage DefaultMQProducerImpl::buildBatchMessage(std::vector& msgs) { + if (msgs.size() < 1) { + THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -1); + } + BatchMessage batchMessage; + bool firstFlag = true; + string topic; + bool waitStoreMsgOK = false; + for (auto& msg : msgs) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + if (firstFlag) { + topic = msg.getTopic(); + waitStoreMsgOK = msg.isWaitStoreMsgOK(); + firstFlag = false; + + if (UtilAll::startsWith_retry(topic)) { + THROW_MQEXCEPTION(MQClientException, "Retry Group is not supported for batching", -1); + } + } else { + if (msg.getDelayTimeLevel() > 0) { + THROW_MQEXCEPTION(MQClientException, "TimeDelayLevel in not supported for batching", -1); + } + if (msg.getTopic() != topic) { + THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -1); + } + if (msg.isWaitStoreMsgOK() != waitStoreMsgOK) { + THROW_MQEXCEPTION(MQClientException, "msgs need one message at least", -2); + } + } + } + batchMessage.setBody(BatchMessage::encode(msgs)); + batchMessage.setTopic(topic); + batchMessage.setWaitStoreMsgOK(waitStoreMsgOK); + return batchMessage; +} + +SendResult DefaultMQProducerImpl::send(MQMessage& msg, const MQMessageQueue& mq) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + if (msg.getTopic() != mq.getTopic()) { + LOG_WARN("message's topic not equal mq's topic"); + } + try { + return sendKernelImpl(msg, mq, ComMode_SYNC, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return SendResult(); +} + +void DefaultMQProducerImpl::send(MQMessage& msg, const MQMessageQueue& mq, SendCallback* pSendCallback) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + if (msg.getTopic() != mq.getTopic()) { + LOG_WARN("message's topic not equal mq's topic"); + } + try { + sendKernelImpl(msg, mq, ComMode_ASYNC, pSendCallback); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +void DefaultMQProducerImpl::sendOneway(MQMessage& msg, bool bSelectActiveBroker) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + try { + sendDefaultImpl(msg, ComMode_ONEWAY, NULL, bSelectActiveBroker); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +void DefaultMQProducerImpl::sendOneway(MQMessage& msg, const MQMessageQueue& mq) { + Validators::checkMessage(msg, getMaxMessageSize()); + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + if (msg.getTopic() != mq.getTopic()) { + LOG_WARN("message's topic not equal mq's topic"); + } + try { + sendKernelImpl(msg, mq, ComMode_ONEWAY, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +SendResult DefaultMQProducerImpl::send(MQMessage& msg, MessageQueueSelector* pSelector, void* arg) { + try { + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + return sendSelectImpl(msg, pSelector, arg, ComMode_SYNC, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return SendResult(); +} + +SendResult DefaultMQProducerImpl::send(MQMessage& msg, + MessageQueueSelector* pSelector, + void* arg, + int autoRetryTimes, + bool bActiveBroker) { + try { + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + return sendAutoRetrySelectImpl(msg, pSelector, arg, ComMode_SYNC, NULL, autoRetryTimes, bActiveBroker); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } + return SendResult(); +} + +void DefaultMQProducerImpl::send(MQMessage& msg, MessageQueueSelector* pSelector, void* arg, SendCallback* pSendCallback) { + try { + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + sendSelectImpl(msg, pSelector, arg, ComMode_ASYNC, pSendCallback); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +void DefaultMQProducerImpl::sendOneway(MQMessage& msg, MessageQueueSelector* pSelector, void* arg) { + try { + if (!NameSpaceUtil::hasNameSpace(msg.getTopic(), getNameSpace())) { + MessageAccessor::withNameSpace(msg, getNameSpace()); + } + sendSelectImpl(msg, pSelector, arg, ComMode_ONEWAY, NULL); + } catch (MQException& e) { + LOG_ERROR(e.what()); + throw e; + } +} + +int DefaultMQProducerImpl::getSendMsgTimeout() const { + return m_sendMsgTimeout; +} + +void DefaultMQProducerImpl::setSendMsgTimeout(int sendMsgTimeout) { + m_sendMsgTimeout = sendMsgTimeout; +} + +int DefaultMQProducerImpl::getCompressMsgBodyOverHowmuch() const { + return m_compressMsgBodyOverHowmuch; +} + +void DefaultMQProducerImpl::setCompressMsgBodyOverHowmuch(int compressMsgBodyOverHowmuch) { + m_compressMsgBodyOverHowmuch = compressMsgBodyOverHowmuch; +} + +int DefaultMQProducerImpl::getMaxMessageSize() const { + return m_maxMessageSize; +} + +void DefaultMQProducerImpl::setMaxMessageSize(int maxMessageSize) { + m_maxMessageSize = maxMessageSize; +} + +int DefaultMQProducerImpl::getCompressLevel() const { + return m_compressLevel; +} + +void DefaultMQProducerImpl::setCompressLevel(int compressLevel) { + assert((compressLevel >= 0 && compressLevel <= 9) || compressLevel == -1); + + m_compressLevel = compressLevel; +} + +// weak_topicPublishInfo( + getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); + boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); + if (topicPublishInfo) { + if (times == 1) { + mq_index = topicPublishInfo->getWhichQueue(); + } else { + mq_index++; + } + + SendResult sendResult; + MQMessageQueue mq; + if (bActiveMQ) + mq = topicPublishInfo->selectOneActiveMessageQueue(lastmq, mq_index); + else + mq = topicPublishInfo->selectOneMessageQueue(lastmq, mq_index); + + lastmq = mq; + if (mq.getQueueId() == -1) { + // THROW_MQEXCEPTION(MQClientException, "the MQMessageQueue is + // invalide", -1); + continue; + } + + try { + LOG_DEBUG("send to mq:%s", mq.toString().data()); + sendResult = sendKernelImpl(msg, mq, communicationMode, pSendCallback); + switch (communicationMode) { + case ComMode_ASYNC: + return sendResult; + case ComMode_ONEWAY: + return sendResult; + case ComMode_SYNC: + if (sendResult.getSendStatus() != SEND_OK) { + if (bActiveMQ) { + topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); + } + continue; + } + return sendResult; + default: + break; + } + } catch (...) { + LOG_ERROR("send failed of times:%d,brokerName:%s", times, mq.getBrokerName().c_str()); + if (bActiveMQ) { + topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); + } + continue; + } + } // end of for + LOG_WARN("Retry many times, still failed"); + } + string info = "No route info of this topic: " + msg.getTopic(); + THROW_MQEXCEPTION(MQClientException, info, -1); +} + +SendResult DefaultMQProducerImpl::sendKernelImpl(MQMessage& msg, + const MQMessageQueue& mq, + int communicationMode, + SendCallback* sendCallback) { + string brokerAddr = getFactory()->findBrokerAddressInPublish(mq.getBrokerName()); + + if (brokerAddr.empty()) { + getFactory()->tryToFindTopicPublishInfo(mq.getTopic(), getSessionCredentials()); + brokerAddr = getFactory()->findBrokerAddressInPublish(mq.getBrokerName()); + } + + if (!brokerAddr.empty()) { + try { + bool isBatchMsg = std::type_index(typeid(msg)) == std::type_index(typeid(BatchMessage)); + // msgId is produced by client, offsetMsgId produced by broker. (same with java sdk) + if (!isBatchMsg) { + string unique_id = StringIdMaker::getInstance().createUniqID(); + msg.setProperty(MQMessage::PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, unique_id); + + // batch does not support compressing right now, + tryToCompressMessage(msg); + } + + LOG_DEBUG("produce before:%s to %s", msg.toString().c_str(), mq.toString().c_str()); + + SendMessageRequestHeader* requestHeader = new SendMessageRequestHeader(); + requestHeader->producerGroup = getGroupName(); + requestHeader->topic = (msg.getTopic()); + requestHeader->defaultTopic = DEFAULT_TOPIC; + requestHeader->defaultTopicQueueNums = 4; + requestHeader->queueId = (mq.getQueueId()); + requestHeader->sysFlag = (msg.getSysFlag()); + requestHeader->bornTimestamp = UtilAll::currentTimeMillis(); + requestHeader->flag = (msg.getFlag()); + requestHeader->consumeRetryTimes = 16; + requestHeader->batch = isBatchMsg; + requestHeader->properties = (MQDecoder::messageProperties2String(msg.getProperties())); + + return getFactory()->getMQClientAPIImpl()->sendMessage(brokerAddr, mq.getBrokerName(), msg, requestHeader, + getSendMsgTimeout(), getRetryTimes4Async(), + communicationMode, sendCallback, getSessionCredentials()); + } catch (MQException& e) { + throw e; + } + } + THROW_MQEXCEPTION(MQClientException, "The broker[" + mq.getBrokerName() + "] not exist", -1); +} + +SendResult DefaultMQProducerImpl::sendSelectImpl(MQMessage& msg, + MessageQueueSelector* pSelector, + void* pArg, + int communicationMode, + SendCallback* sendCallback) { + Validators::checkMessage(msg, getMaxMessageSize()); + + boost::weak_ptr weak_topicPublishInfo( + getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); + boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); + if (topicPublishInfo) //&& topicPublishInfo->ok()) + { + MQMessageQueue mq = pSelector->select(topicPublishInfo->getMessageQueueList(), msg, pArg); + return sendKernelImpl(msg, mq, communicationMode, sendCallback); + } + THROW_MQEXCEPTION(MQClientException, "No route info for this topic", -1); +} + +SendResult DefaultMQProducerImpl::sendAutoRetrySelectImpl(MQMessage& msg, + MessageQueueSelector* pSelector, + void* pArg, + int communicationMode, + SendCallback* pSendCallback, + int autoRetryTimes, + bool bActiveMQ) { + Validators::checkMessage(msg, getMaxMessageSize()); + + MQMessageQueue lastmq; + MQMessageQueue mq; + int mq_index = 0; + for (int times = 1; times <= autoRetryTimes + 1; times++) { + boost::weak_ptr weak_topicPublishInfo( + getFactory()->tryToFindTopicPublishInfo(msg.getTopic(), getSessionCredentials())); + boost::shared_ptr topicPublishInfo(weak_topicPublishInfo.lock()); + if (topicPublishInfo) { + SendResult sendResult; + if (times == 1) { + // always send to selected MQ firstly, evenif bActiveMQ was setted to true + mq = pSelector->select(topicPublishInfo->getMessageQueueList(), msg, pArg); + lastmq = mq; + } else { + LOG_INFO("sendAutoRetrySelectImpl with times:%d", times); + std::vector mqs(topicPublishInfo->getMessageQueueList()); + for (size_t i = 0; i < mqs.size(); i++) { + if (mqs[i] == lastmq) + mq_index = i; + } + if (bActiveMQ) + mq = topicPublishInfo->selectOneActiveMessageQueue(lastmq, mq_index); + else + mq = topicPublishInfo->selectOneMessageQueue(lastmq, mq_index); + lastmq = mq; + if (mq.getQueueId() == -1) { + // THROW_MQEXCEPTION(MQClientException, "the MQMessageQueue is + // invalide", -1); + continue; + } + } + + try { + LOG_DEBUG("send to broker:%s", mq.toString().c_str()); + sendResult = sendKernelImpl(msg, mq, communicationMode, pSendCallback); + switch (communicationMode) { + case ComMode_ASYNC: + return sendResult; + case ComMode_ONEWAY: + return sendResult; + case ComMode_SYNC: + if (sendResult.getSendStatus() != SEND_OK) { + if (bActiveMQ) { + topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); + } + continue; + } + return sendResult; + default: + break; + } + } catch (...) { + LOG_ERROR("send failed of times:%d,mq:%s", times, mq.toString().c_str()); + if (bActiveMQ) { + topicPublishInfo->updateNonServiceMessageQueue(mq, getSendMsgTimeout()); + } + continue; + } + } // end of for + LOG_WARN("Retry many times, still failed"); + } + THROW_MQEXCEPTION(MQClientException, "No route info of this topic, ", -1); +} + +bool DefaultMQProducerImpl::tryToCompressMessage(MQMessage& msg) { + int sysFlag = msg.getSysFlag(); + if ((sysFlag & MessageSysFlag::CompressedFlag) == MessageSysFlag::CompressedFlag) { + return true; + } + + string body = msg.getBody(); + if ((int)body.length() >= getCompressMsgBodyOverHowmuch()) { + string outBody; + if (UtilAll::deflate(body, outBody, getCompressLevel())) { + msg.setBody(outBody); + msg.setSysFlag(sysFlag | MessageSysFlag::CompressedFlag); + return true; + } + } + + return false; +} + +int DefaultMQProducerImpl::getRetryTimes() const { + return m_retryTimes; +} + +void DefaultMQProducerImpl::setRetryTimes(int times) { + if (times <= 0) { + LOG_WARN("set retry times illegal, use default value:5"); + return; + } + + if (times > 15) { + LOG_WARN("set retry times illegal, use max value:15"); + m_retryTimes = 15; + return; + } + LOG_WARN("set retry times to:%d", times); + m_retryTimes = times; +} + +int DefaultMQProducerImpl::getRetryTimes4Async() const { + return m_retryTimes4Async; +} + +void DefaultMQProducerImpl::setRetryTimes4Async(int times) { + if (times <= 0) { + LOG_WARN("set retry times illegal, use default value:1"); + m_retryTimes4Async = 1; + return; + } + + if (times > 15) { + LOG_WARN("set retry times illegal, use max value:15"); + m_retryTimes4Async = 15; + return; + } + LOG_INFO("set retry times to:%d", times); + m_retryTimes4Async = times; +} + +// we should deal with name space before producer start. +bool DefaultMQProducerImpl::dealWithNameSpace() { + string ns = getNameSpace(); + if (ns.empty()) { + string nsAddr = getNamesrvAddr(); + if (!NameSpaceUtil::checkNameSpaceExistInNameServer(nsAddr)) { + return true; + } + ns = NameSpaceUtil::getNameSpaceFromNsURL(nsAddr); + // reset namespace + setNameSpace(ns); + } + // reset group name + if (!NameSpaceUtil::hasNameSpace(getGroupName(), ns)) { + string fullGID = NameSpaceUtil::withNameSpace(getGroupName(), ns); + setGroupName(fullGID); + } + return true; +} +//& msgs); + virtual SendResult send(std::vector& msgs, const MQMessageQueue& mq); + virtual void send(MQMessage& msg, SendCallback* pSendCallback, bool bSelectActiveBroker = false); + virtual void send(MQMessage& msg, const MQMessageQueue& mq, SendCallback* pSendCallback); + virtual void send(MQMessage& msg, MessageQueueSelector* selector, void* arg, SendCallback* pSendCallback); + virtual void sendOneway(MQMessage& msg, bool bSelectActiveBroker = false); + virtual void sendOneway(MQMessage& msg, const MQMessageQueue& mq); + virtual void sendOneway(MQMessage& msg, MessageQueueSelector* selector, void* arg); + //& msgs); + bool dealWithNameSpace(); + + private: + int m_sendMsgTimeout; + int m_compressMsgBodyOverHowmuch; + int m_maxMessageSize; // #include "CommandHeader.h" #include "Logging.h" @@ -28,18 +28,18 @@ using namespace std; namespace rocketmq { -void TransactionMQProducer::initTransactionEnv() { +void TransactionMQProducerImpl::initTransactionEnv() { for (int i = 0; i < m_thread_num; ++i) { m_threadpool.create_thread(boost::bind(&boost::asio::io_service::run, &m_ioService)); } } -void TransactionMQProducer::destroyTransactionEnv() { +void TransactionMQProducerImpl::destroyTransactionEnv() { m_ioService.stop(); m_threadpool.join_all(); } -TransactionSendResult TransactionMQProducer::sendMessageInTransaction(MQMessage& msg, void* arg) { +TransactionSendResult TransactionMQProducerImpl::sendMessageInTransaction(MQMessage& msg, void* arg) { if (!m_transactionListener) { THROW_MQEXCEPTION(MQClientException, "transactionListener is null", -1); } @@ -98,7 +98,7 @@ TransactionSendResult TransactionMQProducer::sendMessageInTransaction(MQMessage& return transactionSendResult; } -void TransactionMQProducer::endTransaction(SendResult& sendResult, LocalTransactionState& localTransactionState) { +void TransactionMQProducerImpl::endTransaction(SendResult& sendResult, LocalTransactionState& localTransactionState) { MQMessageId id; if (sendResult.getOffsetMsgId() != "") { id = MQDecoder::decodeMessageId(sendResult.getOffsetMsgId()); @@ -130,7 +130,7 @@ void TransactionMQProducer::endTransaction(SendResult& sendResult, LocalTransact getFactory()->endTransactionOneway(sendResult.getMessageQueue(), requestHeader, getSessionCredentials()); } -void TransactionMQProducer::checkTransactionState(const std::string& addr, +void TransactionMQProducerImpl::checkTransactionState(const std::string& addr, const MQMessageExt& message, long tranStateTableOffset, long commitLogOffset, @@ -143,11 +143,11 @@ void TransactionMQProducer::checkTransactionState(const std::string& addr, THROW_MQEXCEPTION(MQClientException, "checkTransactionState, transactionListener null", -1); } - m_ioService.post(boost::bind(&TransactionMQProducer::checkTransactionStateImpl, this, addr, message, + m_ioService.post(boost::bind(&TransactionMQProducerImpl::checkTransactionStateImpl, this, addr, message, tranStateTableOffset, commitLogOffset, msgId, transactionId, offsetMsgId)); } -void TransactionMQProducer::checkTransactionStateImpl(const std::string& addr, +void TransactionMQProducerImpl::checkTransactionStateImpl(const std::string& addr, const MQMessageExt& message, long tranStateTableOffset, long commitLogOffset, @@ -203,13 +203,13 @@ void TransactionMQProducer::checkTransactionStateImpl(const std::string& addr, } } -void TransactionMQProducer::start() { +void TransactionMQProducerImpl::start() { initTransactionEnv(); - DefaultMQProducer::start(); + DefaultMQProducerImpl::start(); } -void TransactionMQProducer::shutdown() { - DefaultMQProducer::shutdown(); +void TransactionMQProducerImpl::shutdown() { + DefaultMQProducerImpl::shutdown(); destroyTransactionEnv(); } diff --git a/src/producer/TransactionMQProducerImpl.h b/src/producer/TransactionMQProducerImpl.h new file mode 100644 index 000000000..da68c06cf --- /dev/null +++ b/src/producer/TransactionMQProducerImpl.h @@ -0,0 +1,74 @@ +/* + * 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. + */ + +#ifndef __TRANSACTIONMQPRODUCERIMPL_H__ +#define __TRANSACTIONMQPRODUCERIMPL_H__ + +#include +#include +#include +#include +#include +#include +#include +#include "DefaultMQProducerImpl.h" +#include "MQMessageExt.h" +#include "TransactionListener.h" +#include "TransactionSendResult.h" + +namespace rocketmq { + +class TransactionMQProducerImpl : public DefaultMQProducerImpl { + public: + TransactionMQProducerImpl(const std::string& producerGroup) + : DefaultMQProducerImpl(producerGroup), m_thread_num(1), m_ioServiceWork(m_ioService) {} + virtual ~TransactionMQProducerImpl() {} + void start(); + void shutdown(); + std::shared_ptr getTransactionListener() { return m_transactionListener; } + void setTransactionListener(TransactionListener* listener) { m_transactionListener.reset(listener); } + TransactionSendResult sendMessageInTransaction(MQMessage& msg, void* arg); + void checkTransactionState(const std::string& addr, + const MQMessageExt& message, + long tranStateTableOffset, + long commitLogOffset, + const std::string& msgId, + const std::string& transactionId, + const std::string& offsetMsgId); + + private: + void initTransactionEnv(); + void destroyTransactionEnv(); + void endTransaction(SendResult& sendResult, LocalTransactionState& localTransactionState); + void checkTransactionStateImpl(const std::string& addr, + const MQMessageExt& message, + long tranStateTableOffset, + long commitLogOffset, + const std::string& msgId, + const std::string& transactionId, + const std::string& offsetMsgId); + + private: + std::shared_ptr m_transactionListener; + int m_thread_num; + boost::thread_group m_threadpool; + boost::asio::io_service m_ioService; + boost::asio::io_service::work m_ioServiceWork; +}; +} // namespace rocketmq + +#endif From 521b31624dc4bba5a178cc80bb0594e6a00abf87 Mon Sep 17 00:00:00 2001 From: ShannonDing Date: Thu, 6 Feb 2020 14:02:18 +0800 Subject: [PATCH 2/7] format code style --- example/AsyncPushConsumer.cpp | 30 +++++++------- example/BatchProducer.cpp | 32 +++++++-------- example/CAsyncProducer.c | 34 ++++++++-------- example/COrderlyAsyncProducer.c | 34 ++++++++-------- example/OrderProducer.cpp | 30 +++++++------- example/OrderlyPushConsumer.cpp | 30 +++++++------- example/PullConsumeMessage.c | 36 ++++++++--------- example/PullConsumer.cpp | 30 +++++++------- example/PushConsumeMessage.c | 34 ++++++++-------- example/PushConsumer.cpp | 30 +++++++------- example/PushConsumerOrderly.c | 32 +++++++-------- example/README.md | 5 +-- example/SendDelayMsg.cpp | 32 +++++++-------- example/SyncProducer.cpp | 32 +++++++-------- src/consumer/ConsumeMsgService.h | 6 +-- src/consumer/DefaultMQPullConsumerImpl.cpp | 46 +++++++++++----------- src/consumer/DefaultMQPullConsumerImpl.h | 4 +- src/consumer/DefaultMQPushConsumerImpl.cpp | 8 ++-- src/consumer/DefaultMQPushConsumerImpl.h | 4 +- src/consumer/Rebalance.cpp | 8 ++-- src/extern/CMessageExt.cpp | 2 +- src/extern/CProducer.cpp | 2 +- src/include/DefaultMQClient.h | 5 +-- src/include/MQConsumer.h | 2 +- src/include/MQProducer.h | 2 +- src/include/SendMessageHook.h | 8 ++-- 26 files changed, 254 insertions(+), 264 deletions(-) diff --git a/example/AsyncPushConsumer.cpp b/example/AsyncPushConsumer.cpp index 414f96739..7b644af64 100644 --- a/example/AsyncPushConsumer.cpp +++ b/example/AsyncPushConsumer.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include #include diff --git a/example/BatchProducer.cpp b/example/BatchProducer.cpp index d889ea91f..97a3abddb 100644 --- a/example/BatchProducer.cpp +++ b/example/BatchProducer.cpp @@ -1,26 +1,24 @@ /* -* 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. -*/ + * 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. + */ #include #include #include #include #include -#include -#include #include #include #include diff --git a/example/CAsyncProducer.c b/example/CAsyncProducer.c index 2d2af61d5..9837c1ff5 100644 --- a/example/CAsyncProducer.c +++ b/example/CAsyncProducer.c @@ -1,30 +1,30 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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. + */ #include -#include "CProducer.h" #include "CCommon.h" #include "CMessage.h" +#include "CProducer.h" #include "CSendResult.h" #ifdef _WIN32 #include #else -#include #include +#include #endif void thread_sleep(unsigned milliseconds) { diff --git a/example/COrderlyAsyncProducer.c b/example/COrderlyAsyncProducer.c index 48a822cec..b48fb5485 100644 --- a/example/COrderlyAsyncProducer.c +++ b/example/COrderlyAsyncProducer.c @@ -1,30 +1,30 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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. + */ #include -#include "CProducer.h" #include "CCommon.h" #include "CMessage.h" +#include "CProducer.h" #include "CSendResult.h" #ifdef _WIN32 #include #else -#include #include +#include #endif void thread_sleep(unsigned milliseconds) { diff --git a/example/OrderProducer.cpp b/example/OrderProducer.cpp index 613add4b8..010fd1de9 100644 --- a/example/OrderProducer.cpp +++ b/example/OrderProducer.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include #include #include diff --git a/example/OrderlyPushConsumer.cpp b/example/OrderlyPushConsumer.cpp index c8a9eaa10..a6d82ff82 100644 --- a/example/OrderlyPushConsumer.cpp +++ b/example/OrderlyPushConsumer.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include #include diff --git a/example/PullConsumeMessage.c b/example/PullConsumeMessage.c index 0868d08d0..f9b9bc217 100644 --- a/example/PullConsumeMessage.c +++ b/example/PullConsumeMessage.c @@ -1,31 +1,31 @@ /* -* 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. -*/ + * 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. + */ #include -#include "CPullConsumer.h" #include "CCommon.h" #include "CMessageExt.h" -#include "CPullResult.h" #include "CMessageQueue.h" +#include "CPullConsumer.h" +#include "CPullResult.h" #ifdef _WIN32 #include #else -#include #include +#include #endif void thread_sleep(unsigned milliseconds) { diff --git a/example/PullConsumer.cpp b/example/PullConsumer.cpp index 52e9094d6..2955c1075 100644 --- a/example/PullConsumer.cpp +++ b/example/PullConsumer.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include #include diff --git a/example/PushConsumeMessage.c b/example/PushConsumeMessage.c index 2cb9c3f49..85ea28c0a 100644 --- a/example/PushConsumeMessage.c +++ b/example/PushConsumeMessage.c @@ -1,29 +1,29 @@ /* -* 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. -*/ + * 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. + */ #include -#include "CPushConsumer.h" #include "CCommon.h" #include "CMessageExt.h" +#include "CPushConsumer.h" #ifdef _WIN32 #include #else -#include #include +#include #endif void thread_sleep(unsigned milliseconds) { diff --git a/example/PushConsumer.cpp b/example/PushConsumer.cpp index f0d4926cf..0b69ca5b9 100644 --- a/example/PushConsumer.cpp +++ b/example/PushConsumer.cpp @@ -1,19 +1,19 @@ /* -* 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. -*/ + * 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. + */ #include #include diff --git a/example/PushConsumerOrderly.c b/example/PushConsumerOrderly.c index 4b2c2bbbd..9601799a7 100644 --- a/example/PushConsumerOrderly.c +++ b/example/PushConsumerOrderly.c @@ -1,26 +1,26 @@ /* -* 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. -*/ + * 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. + */ #ifndef WIN32 #include #endif #include -#include "CPushConsumer.h" #include "CCommon.h" #include "CMessageExt.h" +#include "CPushConsumer.h" int doConsumeMessage(struct CPushConsumer* consumer, CMessageExt* msgExt) { printf("Hello,doConsumeMessage by Application!\n"); diff --git a/example/README.md b/example/README.md index 91965d012..378a4ead3 100644 --- a/example/README.md +++ b/example/README.md @@ -1,4 +1 @@ -1. AsyncProducer -2. OrderlyProducer -3. SyncProducer - +1. AsyncProducer 2. OrderlyProducer 3. SyncProducer diff --git a/example/SendDelayMsg.cpp b/example/SendDelayMsg.cpp index 248642780..860f5b33d 100644 --- a/example/SendDelayMsg.cpp +++ b/example/SendDelayMsg.cpp @@ -1,27 +1,25 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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. + */ #include #include #include #include #include -#include -#include #include #include #include diff --git a/example/SyncProducer.cpp b/example/SyncProducer.cpp index 0edb6dc8e..e63dcfefd 100644 --- a/example/SyncProducer.cpp +++ b/example/SyncProducer.cpp @@ -1,27 +1,25 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * 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. + */ #include #include #include #include #include -#include -#include #include #include #include diff --git a/src/consumer/ConsumeMsgService.h b/src/consumer/ConsumeMsgService.h index db9b27a36..8745bae1d 100644 --- a/src/consumer/ConsumeMsgService.h +++ b/src/consumer/ConsumeMsgService.h @@ -24,13 +24,13 @@ #include #include #include +#include "DefaultMQPushConsumerImpl.h" #include "Logging.h" +#include "MQConsumer.h" #include "MQMessageListener.h" #include "PullRequest.h" -#include "MQConsumer.h" -#include "DefaultMQPushConsumerImpl.h" namespace rocketmq { -//class MQConsumer; +// class MQConsumer; // pullRequest) { + boost::asio::deadline_timer* t, + boost::weak_ptr pullRequest) { if (pullRequest.expired()) { LOG_WARN("Pull request has been released before."); return; @@ -554,7 +554,7 @@ void DefaultMQPushConsumerImpl::static_triggerNextPullRequest(void* context, } void DefaultMQPushConsumerImpl::triggerNextPullRequest(boost::asio::deadline_timer* t, - boost::weak_ptr pullRequest) { + boost::weak_ptr pullRequest) { // delete first to avoild memleak deleteAndZero(t); boost::shared_ptr request = pullRequest.lock(); @@ -772,7 +772,7 @@ void DefaultMQPushConsumerImpl::pullMessage(boost::weak_ptr pullReq } AsyncPullCallback* DefaultMQPushConsumerImpl::getAsyncPullCallBack(boost::weak_ptr pullRequest, - MQMessageQueue msgQueue) { + MQMessageQueue msgQueue) { boost::shared_ptr request = pullRequest.lock(); if (!request) { return NULL; diff --git a/src/consumer/DefaultMQPushConsumerImpl.h b/src/consumer/DefaultMQPushConsumerImpl.h index 0a75b56da..f90b6cd2c 100644 --- a/src/consumer/DefaultMQPushConsumerImpl.h +++ b/src/consumer/DefaultMQPushConsumerImpl.h @@ -43,9 +43,9 @@ class TaskThread; class AsyncPullCallback; class ConsumerRunningInfo; //(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = static_cast(m_pConsumer); OffsetStore* pOffsetStore = pConsumer->getOffsetStore(); vector mqs; { @@ -174,7 +174,7 @@ void Rebalance::persistConsumerOffset() { } void Rebalance::persistConsumerOffsetByResetOffset() { - DefaultMQPushConsumerImpl* pConsumer = static_cast(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = static_cast(m_pConsumer); OffsetStore* pOffsetStore = pConsumer->getOffsetStore(); vector mqs; { @@ -552,7 +552,7 @@ bool RebalancePush::updateRequestTableInRebalance(const string& topic, vector(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = dynamic_cast(m_pConsumer); if (!pConsumer) { LOG_ERROR("Cast consumer pointer to DefaultMQPushConsumer pointer failed when computePullFromWhere %s", mq.toString().c_str()); @@ -639,7 +639,7 @@ void RebalancePush::messageQueueChanged(const string& topic, void RebalancePush::removeUnnecessaryMessageQueue(const MQMessageQueue& mq) { // DefaultMQPushConsumer *pConsumer = static_cast(m_pConsumer); - DefaultMQPushConsumerImpl* pConsumer = dynamic_cast(m_pConsumer); + DefaultMQPushConsumerImpl* pConsumer = dynamic_cast(m_pConsumer); if (!pConsumer) { LOG_ERROR("Cast MQConsumer* to DefaultMQPushConsumer* failed when remove %s", mq.toString().c_str()); return; diff --git a/src/extern/CMessageExt.cpp b/src/extern/CMessageExt.cpp index 03a5e2dcd..a013b843e 100644 --- a/src/extern/CMessageExt.cpp +++ b/src/extern/CMessageExt.cpp @@ -15,9 +15,9 @@ * limitations under the License. */ -#include "MQMessageExt.h" #include "CMessageExt.h" #include "CCommon.h" +#include "MQMessageExt.h" #ifdef __cplusplus extern "C" { diff --git a/src/extern/CProducer.cpp b/src/extern/CProducer.cpp index 94f1bdf34..e230257b6 100644 --- a/src/extern/CProducer.cpp +++ b/src/extern/CProducer.cpp @@ -27,11 +27,11 @@ #include "CMessage.h" #include "CSendResult.h" #include "DefaultMQProducer.h" +#include "MQClient.h" #include "MQClientErrorContainer.h" #include "TransactionListener.h" #include "TransactionMQProducer.h" #include "TransactionSendResult.h" -#include "MQClient.h" #include "UtilAll.h" #ifdef __cplusplus diff --git a/src/include/DefaultMQClient.h b/src/include/DefaultMQClient.h index b8d8ba1bc..0ae4934a0 100644 --- a/src/include/DefaultMQClient.h +++ b/src/include/DefaultMQClient.h @@ -23,19 +23,18 @@ #include #include #include +#include "MQClient.h" #include "MQMessageExt.h" #include "MQMessageQueue.h" #include "QueryResult.h" #include "RocketMQClient.h" #include "SessionCredentials.h" -#include "MQClient.h" namespace rocketmq { class MQClientFactory; // Date: Thu, 6 Feb 2020 14:56:04 +0800 Subject: [PATCH 3/7] add impl for pull and push consumer --- include/DefaultMQPullConsumer.h | 9 +- include/DefaultMQPushConsumer.h | 20 +-- src/consumer/DefaultMQPullConsumer.cpp | 176 ++++++++++++++++++++++ src/consumer/DefaultMQPushConsumer.cpp | 196 +++++++++++++++++++++++++ 4 files changed, 384 insertions(+), 17 deletions(-) create mode 100644 src/consumer/DefaultMQPullConsumer.cpp create mode 100644 src/consumer/DefaultMQPushConsumer.cpp diff --git a/include/DefaultMQPullConsumer.h b/include/DefaultMQPullConsumer.h index 9291e0fc9..024c072de 100644 --- a/include/DefaultMQPullConsumer.h +++ b/include/DefaultMQPullConsumer.h @@ -32,7 +32,7 @@ namespace rocketmq { class SubscriptionData; -class ConsumerRunningInfo; +class DefaultMQPullConsumerImpl; //& mqs); - virtual void doRebalance(); virtual void persistConsumerOffset(); virtual void persistConsumerOffsetByResetOffset(); virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info); @@ -124,8 +124,6 @@ class ROCKETMQCLIENT_API DefaultMQPullConsumer { int maxNums, PullCallback* pPullCallback); - virtual ConsumerRunningInfo* getConsumerRunningInfo() { return NULL; } - int64 fetchConsumeOffset(const MQMessageQueue& mq, bool fromStore); void fetchMessageQueuesInBalance(const std::string& topic, std::vector mqs); @@ -133,6 +131,9 @@ class ROCKETMQCLIENT_API DefaultMQPullConsumer { // temp persist consumer offset interface, only valid with // RemoteBrokerOffsetStore, updateConsumeOffset should be called before. void persistConsumerOffset4PullConsumer(const MQMessageQueue& mq); + +private: + DefaultMQPullConsumerImpl* impl; }; //& mqs); - virtual void doRebalance(); - virtual void persistConsumerOffset(); - virtual void persistConsumerOffsetByResetOffset(); - virtual void updateTopicSubscribeInfo(const std::string& topic, std::vector& info); - virtual ConsumeType getConsumeType(); + +virtual ConsumeType getConsumeType(); virtual ConsumeFromWhere getConsumeFromWhere(); void setConsumeFromWhere(ConsumeFromWhere consumeFromWhere); - virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset); - virtual void removeConsumeOffset(const MQMessageQueue& mq); void registerMessageListener(MQMessageListener* pMessageListener); MessageListenerType getMessageListenerType(); @@ -142,6 +133,9 @@ class ROCKETMQCLIENT_API DefaultMQPushConsumer { const std::string& secretKey, const std::string& accessChannel); const SessionCredentials& getSessionCredentials() const; + +private: + DefaultMQPushConsumerImpl * impl; }; //start(); +} + +void DefaultMQPullConsumer::shutdown() { + impl->shutdown(); +} + +// start mqclient set +const std::string& DefaultMQPullConsumer::getNamesrvAddr() const { + return impl->getNamesrvAddr(); +} + +void DefaultMQPullConsumer::setNamesrvAddr(const std::string& namesrvAddr) { + impl->setNamesrvAddr(namesrvAddr); +} + +const std::string& DefaultMQPullConsumer::getNamesrvDomain() const { + return impl->getNamesrvDomain(); +} + +void DefaultMQPullConsumer::setNamesrvDomain(const std::string& namesrvDomain) { + impl->setNamesrvDomain(namesrvDomain); +} +void DefaultMQPullConsumer::setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel) { + impl->setSessionCredentials(accessKey, secretKey, accessChannel); +} + +const SessionCredentials& DefaultMQPullConsumer::getSessionCredentials() const { + return impl->getSessionCredentials(); +} +const std::string& DefaultMQPullConsumer::getInstanceName() const { + return impl->getInstanceName(); +} + +void DefaultMQPullConsumer::setInstanceName(const std::string& instanceName) { + impl->setInstanceName(instanceName); +} + +const std::string& DefaultMQPullConsumer::getNameSpace() const { + return impl->getNameSpace(); +} + +void DefaultMQPullConsumer::setNameSpace(const std::string& nameSpace) { + impl->setNameSpace(nameSpace); +} +const std::string& DefaultMQPullConsumer::getGroupName() const { + return impl->getGroupName(); +} + +void DefaultMQPullConsumer::setGroupName(const std::string& groupName) { + impl->setGroupName(groupName); +} + +void DefaultMQPullConsumer::setLogLevel(elogLevel inputLevel) { + impl->setLogLevel(inputLevel); +} + +elogLevel DefaultMQPullConsumer::getLogLevel() { + return impl->getLogLevel(); +} +void DefaultMQPullConsumer::setLogFileSizeAndNum(int fileNum, long perFileSize) { + impl->setLogFileSizeAndNum(fileNum, perFileSize); +} + +// void DefaultMQPullConsumer::setUnitName(std::string unitName) { +// impl->setUnitName(unitName); +// } +// const std::string& DefaultMQPullConsumer::getUnitName() const { +// return impl->getUnitName(); +// } + +void DefaultMQPullConsumer::fetchSubscribeMessageQueues(const std::string& topic, std::vector& mqs) { + impl->fetchSubscribeMessageQueues(topic, mqs); +} + +void DefaultMQPullConsumer::persistConsumerOffset() { + impl->persistConsumerOffset(); +} +void DefaultMQPullConsumer::persistConsumerOffsetByResetOffset() { + impl->persistConsumerOffsetByResetOffset(); +} +void DefaultMQPullConsumer::updateTopicSubscribeInfo(const std::string& topic, std::vector& info) { + impl->updateTopicSubscribeInfo(topic, info); +} +ConsumeType DefaultMQPullConsumer::getConsumeType() { + return impl->getConsumeType(); +} +ConsumeFromWhere DefaultMQPullConsumer::getConsumeFromWhere() { + return impl->getConsumeFromWhere(); +} +void DefaultMQPullConsumer::getSubscriptions(std::vector& subData) { + impl->getSubscriptions(subData); +} +void DefaultMQPullConsumer::updateConsumeOffset(const MQMessageQueue& mq, int64 offset) { + impl->updateConsumeOffset(mq, offset); +} +void DefaultMQPullConsumer::removeConsumeOffset(const MQMessageQueue& mq) { + impl->removeConsumeOffset(mq); +} + +void DefaultMQPullConsumer::registerMessageQueueListener(const std::string& topic, MQueueListener* pListener) { + impl->registerMessageQueueListener(topic, pListener); +} +PullResult DefaultMQPullConsumer::pull(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums) { + return impl->pull(mq, subExpression, offset, maxNums); +} +void DefaultMQPullConsumer::pull(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + PullCallback* pPullCallback) { + impl->pull(mq, subExpression, offset, maxNums, pPullCallback); +} + +PullResult DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums) { + return impl->pullBlockIfNotFound(mq, subExpression, offset, maxNums); +} +void DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq, + const std::string& subExpression, + int64 offset, + int maxNums, + PullCallback* pPullCallback) { + impl->pullBlockIfNotFound(mq, subExpression, offset, maxNums, pPullCallback); +} + +int64 DefaultMQPullConsumer::fetchConsumeOffset(const MQMessageQueue& mq, bool fromStore) { + return impl->fetchConsumeOffset(mq, fromStore); +} + +void DefaultMQPullConsumer::fetchMessageQueuesInBalance(const std::string& topic, std::vector mqs) { + impl->fetchMessageQueuesInBalance(topic, mqs); +} +void DefaultMQPullConsumer::persistConsumerOffset4PullConsumer(const MQMessageQueue& mq) { + // impl->persistConsumerOffsetByResetOffset(mq); +} + +//start(); +} + +void DefaultMQPushConsumer::shutdown() { + impl->shutdown(); +} + +ConsumeType DefaultMQPushConsumer::getConsumeType() { + return impl->getConsumeType(); +} +ConsumeFromWhere DefaultMQPushConsumer::getConsumeFromWhere() { + return impl->getConsumeFromWhere(); +} +void DefaultMQPushConsumer::setConsumeFromWhere(ConsumeFromWhere consumeFromWhere) { + impl->setConsumeFromWhere(consumeFromWhere); +} + +void DefaultMQPushConsumer::registerMessageListener(MQMessageListener* pMessageListener) { + impl->registerMessageListener(pMessageListener); +} +MessageListenerType DefaultMQPushConsumer::getMessageListenerType() { + return impl->getMessageListenerType(); +} +void DefaultMQPushConsumer::subscribe(const std::string& topic, const std::string& subExpression) { + impl->subscribe(topic, subExpression); +} + +void DefaultMQPushConsumer::setConsumeMessageBatchMaxSize(int consumeMessageBatchMaxSize) { + impl->setConsumeMessageBatchMaxSize(consumeMessageBatchMaxSize); +} +int DefaultMQPushConsumer::getConsumeMessageBatchMaxSize() const { + return impl->getConsumeMessageBatchMaxSize(); +} + +/* + set consuming thread count, default value is cpu cores +*/ +void DefaultMQPushConsumer::setConsumeThreadCount(int threadCount) { + impl->setConsumeThreadCount(threadCount); +} +int DefaultMQPushConsumer::getConsumeThreadCount() const { + impl->getConsumeThreadCount(); +} +void DefaultMQPushConsumer::setMaxReconsumeTimes(int maxReconsumeTimes) { + impl->setMaxReconsumeTimes(maxReconsumeTimes); +} +int DefaultMQPushConsumer::getMaxReconsumeTimes() const { + return impl->getMaxReconsumeTimes(); +} + +/* + set pullMsg thread count, default value is cpu cores +*/ +void DefaultMQPushConsumer::setPullMsgThreadPoolCount(int threadCount) { + impl->setPullMsgThreadPoolCount(threadCount); +} +int DefaultMQPushConsumer::getPullMsgThreadPoolCount() const { + return impl->getPullMsgThreadPoolCount(); +} + +/* + set max cache msg size perQueue in memory if consumer could not consume msgs + immediately + default maxCacheMsgSize perQueue is 1000, set range is:1~65535 +*/ +void DefaultMQPushConsumer::setMaxCacheMsgSizePerQueue(int maxCacheSize) { + impl->setMaxCacheMsgSizePerQueue(maxCacheSize); +} +int DefaultMQPushConsumer::getMaxCacheMsgSizePerQueue() const { + impl->getMaxCacheMsgSizePerQueue(); +} + +MessageModel DefaultMQPushConsumer::getMessageModel() const { + return impl->getMessageModel(); +} +void DefaultMQPushConsumer::setMessageModel(MessageModel messageModel) { + impl->setMessageModel(messageModel); +} + +const std::string& DefaultMQPushConsumer::getNamesrvAddr() const { + return impl->getNamesrvAddr(); +} + +void DefaultMQPushConsumer::setNamesrvAddr(const std::string& namesrvAddr) { + impl->setNamesrvAddr(namesrvAddr); +} + +const std::string& DefaultMQPushConsumer::getNamesrvDomain() const { + return impl->getNamesrvDomain(); +} + +void DefaultMQPushConsumer::setNamesrvDomain(const std::string& namesrvDomain) { + impl->setNamesrvDomain(namesrvDomain); +} +void DefaultMQPushConsumer::setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel) { + impl->setSessionCredentials(accessKey, secretKey, accessChannel); +} + +const SessionCredentials& DefaultMQPushConsumer::getSessionCredentials() const { + return impl->getSessionCredentials(); +} +const std::string& DefaultMQPushConsumer::getInstanceName() const { + return impl->getInstanceName(); +} + +void DefaultMQPushConsumer::setInstanceName(const std::string& instanceName) { + impl->setInstanceName(instanceName); +} + +const std::string& DefaultMQPushConsumer::getNameSpace() const { + return impl->getNameSpace(); +} + +void DefaultMQPushConsumer::setNameSpace(const std::string& nameSpace) { + impl->setNameSpace(nameSpace); +} +const std::string& DefaultMQPushConsumer::getGroupName() const { + return impl->getGroupName(); +} + +void DefaultMQPushConsumer::setGroupName(const std::string& groupName) { + impl->setGroupName(groupName); +} + +void DefaultMQPushConsumer::setLogLevel(elogLevel inputLevel) { + impl->setLogLevel(inputLevel); +} + +elogLevel DefaultMQPushConsumer::getLogLevel() { + return impl->getLogLevel(); +} +void DefaultMQPushConsumer::setLogFileSizeAndNum(int fileNum, long perFileSize) { + impl->setLogFileSizeAndNum(fileNum, perFileSize); +} + +void DefaultMQPushConsumer::setUnitName(std::string unitName) { + impl->setUnitName(unitName); +} +const std::string& DefaultMQPushConsumer::getUnitName() const { + return impl->getUnitName(); +} + +void DefaultMQPushConsumer::setTcpTransportPullThreadNum(int num) { + impl->setTcpTransportPullThreadNum(num); +} +const int DefaultMQPushConsumer::getTcpTransportPullThreadNum() const { + return impl->getTcpTransportPullThreadNum(); +} + +void DefaultMQPushConsumer::setTcpTransportConnectTimeout(uint64_t timeout) { + impl->setTcpTransportConnectTimeout(timeout); +} +const uint64_t DefaultMQPushConsumer::getTcpTransportConnectTimeout() const { + return impl->getTcpTransportConnectTimeout(); +} +void DefaultMQPushConsumer::setTcpTransportTryLockTimeout(uint64_t timeout) { + impl->setTcpTransportTryLockTimeout(timeout); +} +const uint64_t DefaultMQPushConsumer::getTcpTransportTryLockTimeout() const { + return impl->getTcpTransportTryLockTimeout(); +} + +// Date: Thu, 6 Feb 2020 16:38:12 +0800 Subject: [PATCH 4/7] Add impl for transaction producer --- include/DefaultMQPushConsumer.h | 11 +- include/TransactionMQProducer.h | 72 +++++++++++- src/consumer/DefaultMQPushConsumer.cpp | 4 +- src/producer/TransactionMQProducer.cpp | 155 +++++++++++++++++++++++++ 4 files changed, 231 insertions(+), 11 deletions(-) create mode 100644 src/producer/TransactionMQProducer.cpp diff --git a/include/DefaultMQPushConsumer.h b/include/DefaultMQPushConsumer.h index 8bf116800..cd4e5992c 100644 --- a/include/DefaultMQPushConsumer.h +++ b/include/DefaultMQPushConsumer.h @@ -26,10 +26,9 @@ #include "MQMessageQueue.h" namespace rocketmq { - class DefaultMQPushConsumerImpl; +class DefaultMQPushConsumerImpl; // #include -#include "DefaultMQProducer.h" +#include "MQClient.h" +#include "MQMessage.h" #include "MQMessageExt.h" #include "TransactionListener.h" #include "TransactionSendResult.h" namespace rocketmq { - -class ROCKETMQCLIENT_API TransactionMQProducer : public DefaultMQProducer { +class TransactionMQProducerImpl; +class ROCKETMQCLIENT_API TransactionMQProducer { public: TransactionMQProducer(const std::string& producerGroup); - virtual ~TransactionMQProducer() {} + virtual ~TransactionMQProducer(); void start(); void shutdown(); + const std::string& getNamesrvAddr() const; + void setNamesrvAddr(const std::string& namesrvAddr); + const std::string& getNamesrvDomain() const; + void setNamesrvDomain(const std::string& namesrvDomain); + const std::string& getInstanceName() const; + void setInstanceName(const std::string& instanceName); + // nameSpace + const std::string& getNameSpace() const; + void setNameSpace(const std::string& nameSpace); + const std::string& getGroupName() const; + void setGroupName(const std::string& groupname); + void setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel); + const SessionCredentials& getSessionCredentials() const; + + void setUnitName(std::string unitName); + const std::string& getUnitName() const; + + int getSendMsgTimeout() const; + void setSendMsgTimeout(int sendMsgTimeout); + void setTcpTransportPullThreadNum(int num); + const int getTcpTransportPullThreadNum() const; + + /** timeout of tcp connect, it is same meaning for both producer and consumer; + * 1. default value is 3000ms + * 2. input parameter could only be milliSecond, suggestion value is + * 1000-3000ms; + **/ + void setTcpTransportConnectTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportConnectTimeout() const; + + /** timeout of tryLock tcpTransport before sendMsg/pullMsg, if timeout, + * returns NULL + * 1. paremeter unit is ms, default value is 3000ms, the minimun value is 1000ms + * suggestion value is 3000ms; + * 2. if configured with value smaller than 1000ms, the tryLockTimeout value + * will be setted to 1000ms + **/ + void setTcpTransportTryLockTimeout(uint64_t timeout); // ms + const uint64_t getTcpTransportTryLockTimeout() const; + + /* + * if msgBody size is large than m_compressMsgBodyOverHowmuch + * rocketmq cpp will compress msgBody according to compressLevel + */ + int getCompressMsgBodyOverHowmuch() const; + void setCompressMsgBodyOverHowmuch(int compressMsgBodyOverHowmuch); + int getCompressLevel() const; + void setCompressLevel(int compressLevel); + + // if msgbody size larger than maxMsgBodySize, exception will be throwed + int getMaxMessageSize() const; + void setMaxMessageSize(int maxMessageSize); + // log configuration interface, default LOG_LEVEL is LOG_LEVEL_INFO, default + // log file num is 3, each log size is 100M + void setLogLevel(elogLevel inputLevel); + elogLevel getLogLevel(); + void setLogFileSizeAndNum(int fileNum, long perFileSize); // perFileSize is MB unit + std::shared_ptr getTransactionListener(); void setTransactionListener(TransactionListener* listener); TransactionSendResult sendMessageInTransaction(MQMessage& msg, void* arg); @@ -43,6 +104,9 @@ class ROCKETMQCLIENT_API TransactionMQProducer : public DefaultMQProducer { const std::string& msgId, const std::string& transactionId, const std::string& offsetMsgId); + + private: + TransactionMQProducerImpl* impl; }; } // namespace rocketmq diff --git a/src/consumer/DefaultMQPushConsumer.cpp b/src/consumer/DefaultMQPushConsumer.cpp index 7e5261bad..bb4841e8d 100644 --- a/src/consumer/DefaultMQPushConsumer.cpp +++ b/src/consumer/DefaultMQPushConsumer.cpp @@ -191,6 +191,8 @@ void DefaultMQPushConsumer::setTcpTransportTryLockTimeout(uint64_t timeout) { const uint64_t DefaultMQPushConsumer::getTcpTransportTryLockTimeout() const { return impl->getTcpTransportTryLockTimeout(); } - +void DefaultMQPushConsumer::setAsyncPull(bool asyncFlag) { + impl->setAsyncPull(asyncFlag); +} //start(); +} + +void TransactionMQProducer::shutdown() { + impl->shutdown(); +} + +// start mqclient set +const std::string& TransactionMQProducer::getNamesrvAddr() const { + return impl->getNamesrvAddr(); +} + +void TransactionMQProducer::setNamesrvAddr(const std::string& namesrvAddr) { + impl->setNamesrvAddr(namesrvAddr); +} + +const std::string& TransactionMQProducer::getNamesrvDomain() const { + return impl->getNamesrvDomain(); +} + +void TransactionMQProducer::setNamesrvDomain(const std::string& namesrvDomain) { + impl->setNamesrvDomain(namesrvDomain); +} +void TransactionMQProducer::setSessionCredentials(const std::string& accessKey, + const std::string& secretKey, + const std::string& accessChannel) { + impl->setSessionCredentials(accessKey, secretKey, accessChannel); +} + +const SessionCredentials& TransactionMQProducer::getSessionCredentials() const { + return impl->getSessionCredentials(); +} +const std::string& TransactionMQProducer::getInstanceName() const { + return impl->getInstanceName(); +} + +void TransactionMQProducer::setInstanceName(const std::string& instanceName) { + impl->setInstanceName(instanceName); +} + +const std::string& TransactionMQProducer::getNameSpace() const { + return impl->getNameSpace(); +} + +void TransactionMQProducer::setNameSpace(const std::string& nameSpace) { + impl->setNameSpace(nameSpace); +} +const std::string& TransactionMQProducer::getGroupName() const { + return impl->getGroupName(); +} + +void TransactionMQProducer::setGroupName(const std::string& groupName) { + impl->setGroupName(groupName); +} +void TransactionMQProducer::setSendMsgTimeout(int sendMsgTimeout) { + impl->setSendMsgTimeout(sendMsgTimeout); +} + +int TransactionMQProducer::getSendMsgTimeout() const { + return impl->getSendMsgTimeout(); +} +void TransactionMQProducer::setTcpTransportPullThreadNum(int num) { + impl->setTcpTransportPullThreadNum(num); +} +const int TransactionMQProducer::getTcpTransportPullThreadNum() const { + return impl->getTcpTransportPullThreadNum(); +} + +void TransactionMQProducer::setTcpTransportConnectTimeout(uint64_t timeout) { + impl->setTcpTransportConnectTimeout(timeout); +} +const uint64_t TransactionMQProducer::getTcpTransportConnectTimeout() const { + return impl->getTcpTransportConnectTimeout(); +} +void TransactionMQProducer::setTcpTransportTryLockTimeout(uint64_t timeout) { + impl->setTcpTransportTryLockTimeout(timeout); +} +const uint64_t TransactionMQProducer::getTcpTransportTryLockTimeout() const { + return impl->getTcpTransportTryLockTimeout(); +} +int TransactionMQProducer::getCompressMsgBodyOverHowmuch() const { + return impl->getCompressMsgBodyOverHowmuch(); +} + +void TransactionMQProducer::setCompressMsgBodyOverHowmuch(int compressMsgBodyThreshold) { + impl->setCompressMsgBodyOverHowmuch(compressMsgBodyThreshold); +} + +int TransactionMQProducer::getCompressLevel() const { + return impl->getCompressLevel(); +} + +void TransactionMQProducer::setCompressLevel(int compressLevel) { + impl->setCompressLevel(compressLevel); +} + +void TransactionMQProducer::setMaxMessageSize(int maxMessageSize) { + impl->setMaxMessageSize(maxMessageSize); +} + +int TransactionMQProducer::getMaxMessageSize() const { + return impl->getMaxMessageSize(); +} + +void TransactionMQProducer::setLogLevel(elogLevel inputLevel) { + impl->setLogLevel(inputLevel); +} + +elogLevel TransactionMQProducer::getLogLevel() { + return impl->getLogLevel(); +} +void TransactionMQProducer::setLogFileSizeAndNum(int fileNum, long perFileSize) { + impl->setLogFileSizeAndNum(fileNum, perFileSize); +} + +void TransactionMQProducer::setUnitName(std::string unitName) { + impl->setUnitName(unitName); +} +const std::string& TransactionMQProducer::getUnitName() const { + return impl->getUnitName(); +} + +std::shared_ptr TransactionMQProducer::getTransactionListener() { + return impl->getTransactionListener(); +} +void TransactionMQProducer::setTransactionListener(TransactionListener* listener) { + impl->setTransactionListener(listener); +} +TransactionSendResult TransactionMQProducer::sendMessageInTransaction(MQMessage& msg, void* arg) { + return impl->sendMessageInTransaction(msg, arg); +} +void TransactionMQProducer::checkTransactionState(const std::string& addr, + const MQMessageExt& message, + long tranStateTableOffset, + long commitLogOffset, + const std::string& msgId, + const std::string& transactionId, + const std::string& offsetMsgId) { + impl->checkTransactionState(addr, message, tranStateTableOffset, commitLogOffset, msgId, transactionId, offsetMsgId); +} +} // namespace rocketmq \ No newline at end of file From 9b2ec2ff5b56dc2bbe9be858034882dc187659fd Mon Sep 17 00:00:00 2001 From: ShannonDing Date: Thu, 6 Feb 2020 17:15:30 +0800 Subject: [PATCH 5/7] modify example witout boost --- example/AsyncProducer.cpp | 2 +- example/AsyncPushConsumer.cpp | 3 --- example/BatchProducer.cpp | 2 +- example/OrderProducer.cpp | 5 ++--- example/OrderlyPushConsumer.cpp | 4 ++-- example/PullConsumer.cpp | 1 - example/SyncProducer.cpp | 2 +- example/common.h | 27 +++++++++++++------------- src/consumer/DefaultMQPushConsumer.cpp | 4 ++-- 9 files changed, 23 insertions(+), 27 deletions(-) diff --git a/example/AsyncProducer.cpp b/example/AsyncProducer.cpp index 279cea3c5..5be900c6e 100644 --- a/example/AsyncProducer.cpp +++ b/example/AsyncProducer.cpp @@ -29,7 +29,7 @@ using namespace rocketmq; -boost::atomic g_quit; +std::atomic g_quit; std::mutex g_mtx; std::condition_variable g_finished; SendCallback* g_callback = NULL; diff --git a/example/AsyncPushConsumer.cpp b/example/AsyncPushConsumer.cpp index 7b644af64..671c2e474 100644 --- a/example/AsyncPushConsumer.cpp +++ b/example/AsyncPushConsumer.cpp @@ -20,9 +20,6 @@ #include #include #include -#include -#include -#include #include #include "common.h" diff --git a/example/BatchProducer.cpp b/example/BatchProducer.cpp index 97a3abddb..18f33cbf8 100644 --- a/example/BatchProducer.cpp +++ b/example/BatchProducer.cpp @@ -27,7 +27,7 @@ using namespace rocketmq; using namespace std; -boost::atomic g_quit; +std::atomic g_quit; std::mutex g_mtx; std::condition_variable g_finished; TpsReportService g_tps; diff --git a/example/OrderProducer.cpp b/example/OrderProducer.cpp index 010fd1de9..402e916d9 100644 --- a/example/OrderProducer.cpp +++ b/example/OrderProducer.cpp @@ -14,14 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include + #include #include #include #include #include -#include #include #include "common.h" @@ -30,7 +29,7 @@ using namespace rocketmq; std::condition_variable g_finished; std::mutex g_mtx; -boost::atomic g_quit(false); +std::atomic g_quit(false); class SelectMessageQueueByHash : public MessageQueueSelector { public: diff --git a/example/OrderlyPushConsumer.cpp b/example/OrderlyPushConsumer.cpp index a6d82ff82..174e2b3a9 100644 --- a/example/OrderlyPushConsumer.cpp +++ b/example/OrderlyPushConsumer.cpp @@ -31,8 +31,8 @@ using namespace rocketmq; std::condition_variable g_finished; std::mutex g_mtx; -boost::atomic g_consumedCount(0); -boost::atomic g_quit(false); +std::atomic g_consumedCount(0); +std::atomic g_quit(false); TpsReportService g_tps; class MyMsgListener : public MessageListenerOrderly { diff --git a/example/PullConsumer.cpp b/example/PullConsumer.cpp index 2955c1075..e85a82188 100644 --- a/example/PullConsumer.cpp +++ b/example/PullConsumer.cpp @@ -16,7 +16,6 @@ */ #include -#include #include #include diff --git a/example/SyncProducer.cpp b/example/SyncProducer.cpp index e63dcfefd..5d66da259 100644 --- a/example/SyncProducer.cpp +++ b/example/SyncProducer.cpp @@ -28,7 +28,7 @@ using namespace rocketmq; -boost::atomic g_quit; +std::atomic g_quit; std::mutex g_mtx; std::condition_variable g_finished; TpsReportService g_tps; diff --git a/example/common.h b/example/common.h index 1c8fc9ec9..e3f1b0937 100644 --- a/example/common.h +++ b/example/common.h @@ -17,28 +17,29 @@ #ifndef ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_ #define ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_ -#include -#include -#include -#include +#include +#include #include +#include #include #include #include - +#ifndef WIN32 +#include "unistd.h" +#endif #include "Arg_helper.h" #include "DefaultMQProducer.h" #include "DefaultMQPullConsumer.h" #include "DefaultMQPushConsumer.h" using namespace std; -boost::atomic g_msgCount(1); +std::atomic g_msgCount(1); class RocketmqSendAndConsumerArgs { public: RocketmqSendAndConsumerArgs() : body("msgbody for test"), - thread_count(boost::thread::hardware_concurrency()), + thread_count(std::thread::hardware_concurrency()), broadcasting(false), syncpush(false), SelectUnactiveBroker(false), @@ -69,7 +70,7 @@ class TpsReportService { std::cout << "tps_thread_ is null" << std::endl; return; } - tps_thread_.reset(new boost::thread(boost::bind(&TpsReportService::TpsReport, this))); + tps_thread_.reset(new std::thread(std::bind(&TpsReportService::TpsReport, this))); } ~TpsReportService() { @@ -86,17 +87,17 @@ class TpsReportService { void TpsReport() { while (!quit_flag_.load()) { - boost::this_thread::sleep_for(tps_interval_); + std::this_thread::sleep_for(tps_interval_); std::cout << "tps: " << tps_count_.load() << std::endl; tps_count_.store(0); } } private: - boost::chrono::seconds tps_interval_; - boost::shared_ptr tps_thread_; - boost::atomic quit_flag_; - boost::atomic tps_count_; + std::chrono::seconds tps_interval_; + std::shared_ptr tps_thread_; + std::atomic quit_flag_; + std::atomic tps_count_; }; void PrintPullResult(rocketmq::PullResult* result) { diff --git a/src/consumer/DefaultMQPushConsumer.cpp b/src/consumer/DefaultMQPushConsumer.cpp index bb4841e8d..cede0c9a8 100644 --- a/src/consumer/DefaultMQPushConsumer.cpp +++ b/src/consumer/DefaultMQPushConsumer.cpp @@ -69,7 +69,7 @@ void DefaultMQPushConsumer::setConsumeThreadCount(int threadCount) { impl->setConsumeThreadCount(threadCount); } int DefaultMQPushConsumer::getConsumeThreadCount() const { - impl->getConsumeThreadCount(); + return impl->getConsumeThreadCount(); } void DefaultMQPushConsumer::setMaxReconsumeTimes(int maxReconsumeTimes) { impl->setMaxReconsumeTimes(maxReconsumeTimes); @@ -97,7 +97,7 @@ void DefaultMQPushConsumer::setMaxCacheMsgSizePerQueue(int maxCacheSize) { impl->setMaxCacheMsgSizePerQueue(maxCacheSize); } int DefaultMQPushConsumer::getMaxCacheMsgSizePerQueue() const { - impl->getMaxCacheMsgSizePerQueue(); + return impl->getMaxCacheMsgSizePerQueue(); } MessageModel DefaultMQPushConsumer::getMessageModel() const { From 520cc4877e1e58e380dce7b443278d58317b8940 Mon Sep 17 00:00:00 2001 From: ShannonDing Date: Thu, 6 Feb 2020 18:50:08 +0800 Subject: [PATCH 6/7] format code style --- include/DefaultMQPullConsumer.h | 6 +-- src/producer/DefaultMQProducerImpl.cpp | 45 ++++++++++++---------- src/producer/DefaultMQProducerImpl.h | 4 +- src/producer/TransactionMQProducerImpl.cpp | 26 ++++++------- 4 files changed, 42 insertions(+), 39 deletions(-) diff --git a/include/DefaultMQPullConsumer.h b/include/DefaultMQPullConsumer.h index 024c072de..552bbab36 100644 --- a/include/DefaultMQPullConsumer.h +++ b/include/DefaultMQPullConsumer.h @@ -64,7 +64,7 @@ class ROCKETMQCLIENT_API DefaultMQPullConsumer { void setSessionCredentials(const std::string& accessKey, const std::string& secretKey, const std::string& accessChannel); - const SessionCredentials& getSessionCredentials() const; + const SessionCredentials& getSessionCredentials() const; //& mqs); virtual void persistConsumerOffset(); @@ -132,8 +132,8 @@ class ROCKETMQCLIENT_API DefaultMQPullConsumer { // RemoteBrokerOffsetStore, updateConsumeOffset should be called before. void persistConsumerOffset4PullConsumer(const MQMessageQueue& mq); -private: - DefaultMQPullConsumerImpl* impl; + private: + DefaultMQPullConsumerImpl* impl; }; //findBrokerAddressInPublish(mq.getBrokerName()); if (brokerAddr.empty()) { @@ -464,10 +467,10 @@ SendResult DefaultMQProducerImpl::sendKernelImpl(MQMessage& msg, } SendResult DefaultMQProducerImpl::sendSelectImpl(MQMessage& msg, - MessageQueueSelector* pSelector, - void* pArg, - int communicationMode, - SendCallback* sendCallback) { + MessageQueueSelector* pSelector, + void* pArg, + int communicationMode, + SendCallback* sendCallback) { Validators::checkMessage(msg, getMaxMessageSize()); boost::weak_ptr weak_topicPublishInfo( @@ -482,12 +485,12 @@ SendResult DefaultMQProducerImpl::sendSelectImpl(MQMessage& msg, } SendResult DefaultMQProducerImpl::sendAutoRetrySelectImpl(MQMessage& msg, - MessageQueueSelector* pSelector, - void* pArg, - int communicationMode, - SendCallback* pSendCallback, - int autoRetryTimes, - bool bActiveMQ) { + MessageQueueSelector* pSelector, + void* pArg, + int communicationMode, + SendCallback* pSendCallback, + int autoRetryTimes, + bool bActiveMQ) { Validators::checkMessage(msg, getMaxMessageSize()); MQMessageQueue lastmq; diff --git a/src/producer/DefaultMQProducerImpl.h b/src/producer/DefaultMQProducerImpl.h index f87156306..c4d2393ba 100644 --- a/src/producer/DefaultMQProducerImpl.h +++ b/src/producer/DefaultMQProducerImpl.h @@ -26,9 +26,9 @@ namespace rocketmq { // Date: Thu, 6 Feb 2020 19:58:27 +0800 Subject: [PATCH 7/7] format code style --- src/producer/TransactionMQProducerImpl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/producer/TransactionMQProducerImpl.h b/src/producer/TransactionMQProducerImpl.h index da68c06cf..603318b2f 100644 --- a/src/producer/TransactionMQProducerImpl.h +++ b/src/producer/TransactionMQProducerImpl.h @@ -32,9 +32,9 @@ namespace rocketmq { -class TransactionMQProducerImpl : public DefaultMQProducerImpl { +class TransactionMQProducerImpl : public DefaultMQProducerImpl { public: - TransactionMQProducerImpl(const std::string& producerGroup) + TransactionMQProducerImpl(const std::string& producerGroup) : DefaultMQProducerImpl(producerGroup), m_thread_num(1), m_ioServiceWork(m_ioService) {} virtual ~TransactionMQProducerImpl() {} void start();