You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by li...@apache.org on 2017/09/04 05:04:47 UTC

[01/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Repository: incubator-rocketmq-externals
Updated Branches:
  refs/heads/master 343ab198e -> 70ce5c770


http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/TcpRemotingClient.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/TcpRemotingClient.cpp b/rocketmq-cpp/src/transport/TcpRemotingClient.cpp
new file mode 100755
index 0000000..f74d529
--- /dev/null
+++ b/rocketmq-cpp/src/transport/TcpRemotingClient.cpp
@@ -0,0 +1,728 @@
+/*
+ * 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 "TcpRemotingClient.h"
+#include <stddef.h>
+#include <sys/prctl.h>
+#include "Logging.h"
+#include "MemoryOutputStream.h"
+#include "TopAddressing.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+//<!************************************************************************
+TcpRemotingClient::TcpRemotingClient(int pullThreadNum,
+                                     uint64_t tcpConnectTimeout,
+                                     uint64_t tcpTransportTryLockTimeout)
+    : m_pullThreadNum(pullThreadNum),
+      m_tcpConnectTimeout(tcpConnectTimeout),
+      m_tcpTransportTryLockTimeout(tcpTransportTryLockTimeout),
+      m_namesrvIndex(0),
+      m_ioServiceWork(m_ioService) {
+  string taskName = UtilAll::getProcessName();
+  prctl(PR_SET_NAME, "networkTP", 0, 0, 0);
+  for (int i = 0; i != pullThreadNum; ++i) {
+    m_threadpool.create_thread(
+        boost::bind(&boost::asio::io_service::run, &m_ioService));
+  }
+  prctl(PR_SET_NAME, taskName.c_str(), 0, 0, 0);
+
+  LOG_INFO(
+      "m_tcpConnectTimeout:%ju, m_tcpTransportTryLockTimeout:%ju, "
+      "m_pullThreadNum:%d",
+      m_tcpConnectTimeout, m_tcpTransportTryLockTimeout, m_pullThreadNum);
+  m_async_service_thread.reset(new boost::thread(
+      boost::bind(&TcpRemotingClient::boost_asio_work, this)));
+}
+
+void TcpRemotingClient::boost_asio_work() {
+  LOG_INFO("TcpRemotingClient::boost asio async service runing");
+  boost::asio::io_service::work work(m_async_ioService);  // avoid async io
+                                                          // service stops after
+                                                          // first timer timeout
+                                                          // callback
+  m_async_ioService.run();
+}
+
+TcpRemotingClient::~TcpRemotingClient() {
+  m_tcpTable.clear();
+  m_futureTable.clear();
+  m_asyncFutureTable.clear();
+  m_namesrvAddrList.clear();
+  removeAllTimerCallback();
+}
+
+void TcpRemotingClient::stopAllTcpTransportThread() {
+  LOG_DEBUG("TcpRemotingClient::stopAllTcpTransportThread Begin");
+  m_async_ioService.stop();
+  m_async_service_thread->interrupt();
+  m_async_service_thread->join();
+  removeAllTimerCallback();
+
+  {
+    TcpMap::iterator it = m_tcpTable.begin();
+    for (; it != m_tcpTable.end(); ++it) {
+      it->second->disconnect(it->first);
+    }
+    m_tcpTable.clear();
+  }
+
+  m_ioService.stop();
+  m_threadpool.join_all();
+
+  {
+    boost::lock_guard<boost::mutex> lock(m_futureTableMutex);
+    for (ResMap::iterator it = m_futureTable.begin(); it != m_futureTable.end();
+         ++it) {
+      if (it->second) it->second->releaseThreadCondition();
+    }
+  }
+  LOG_DEBUG("TcpRemotingClient::stopAllTcpTransportThread End");
+}
+
+void TcpRemotingClient::updateNameServerAddressList(const string& addrs) {
+  if (!addrs.empty()) {
+    boost::unique_lock<boost::timed_mutex> lock(m_namesrvlock,
+                                                boost::try_to_lock);
+    if (!lock.owns_lock()) {
+      if (!lock.timed_lock(boost::get_system_time() +
+                           boost::posix_time::seconds(10))) {
+        LOG_ERROR("updateNameServerAddressList get timed_mutex timeout");
+        return;
+      }
+    }
+    // clear first;
+    m_namesrvAddrList.clear();
+
+    vector<string> out;
+    UtilAll::Split(out, addrs, ";");
+    for (size_t i = 0; i < out.size(); i++) {
+      string addr = out[i];
+      UtilAll::Trim(addr);
+
+      string hostName;
+      short portNumber;
+      if (UtilAll::SplitURL(addr, hostName, portNumber)) {
+        LOG_INFO("update Namesrv:%s", addr.c_str());
+        m_namesrvAddrList.push_back(addr);
+      }
+    }
+    out.clear();
+  }
+}
+
+bool TcpRemotingClient::invokeHeartBeat(const string& addr,
+                                        RemotingCommand& request) {
+  boost::shared_ptr<TcpTransport> pTcp = GetTransport(addr, true);
+  if (pTcp != NULL) {
+    int code = request.getCode();
+    int opaque = request.getOpaque();
+    boost::shared_ptr<ResponseFuture> responseFuture(
+        new ResponseFuture(code, opaque, this, 3000, false, NULL));
+    addResponseFuture(opaque, responseFuture);
+    // LOG_INFO("invokeHeartbeat success, addr:%s, code:%d, opaque:%d,
+    // timeoutms:%d", addr.c_str(), code, opaque, 3000);
+
+    if (SendCommand(pTcp, request)) {
+      responseFuture->setSendRequestOK(true);
+      unique_ptr<RemotingCommand> pRsp(responseFuture->waitResponse(3000));
+      if (pRsp == NULL) {
+        LOG_ERROR(
+            "wait response timeout of heartbeat, so closeTransport of addr:%s",
+            addr.c_str());
+        CloseTransport(addr, pTcp);
+        return false;
+      } else if (pRsp->getCode() == SUCCESS_VALUE) {
+        return true;
+      } else {
+        LOG_WARN("get error response:%d of heartbeat to addr:%s",
+                 pRsp->getCode(), addr.c_str());
+        return false;
+      }
+    } else {
+      CloseTransport(addr, pTcp);
+    }
+  }
+  return false;
+}
+
+RemotingCommand* TcpRemotingClient::invokeSync(const string& addr,
+                                               RemotingCommand& request,
+                                               int timeoutMillis /* = 3000 */) {
+  boost::shared_ptr<TcpTransport> pTcp = GetTransport(addr, true);
+  if (pTcp != NULL) {
+    int code = request.getCode();
+    int opaque = request.getOpaque();
+    boost::shared_ptr<ResponseFuture> responseFuture(
+        new ResponseFuture(code, opaque, this, timeoutMillis, false, NULL));
+    addResponseFuture(opaque, responseFuture);
+
+    if (SendCommand(pTcp, request)) {
+      // LOG_INFO("invokeSync success, addr:%s, code:%d, opaque:%d,
+      // timeoutms:%d", addr.c_str(), code, opaque, timeoutMillis);
+      responseFuture->setSendRequestOK(true);
+      RemotingCommand* pRsp = responseFuture->waitResponse(timeoutMillis);
+      if (pRsp == NULL) {
+        if (code != GET_CONSUMER_LIST_BY_GROUP) {
+          LOG_WARN(
+              "wait response timeout or get NULL response of code:%d, so "
+              "closeTransport of addr:%s",
+              code, addr.c_str());
+          CloseTransport(addr, pTcp);
+        }
+        // avoid responseFuture leak;
+        findAndDeleteResponseFuture(opaque);
+        return NULL;
+      } else {
+        return pRsp;
+      }
+    } else {
+      // avoid responseFuture leak;
+      findAndDeleteResponseFuture(opaque);
+      CloseTransport(addr, pTcp);
+    }
+  }
+  return NULL;
+}
+
+bool TcpRemotingClient::invokeAsync(const string& addr,
+                                    RemotingCommand& request,
+                                    AsyncCallbackWrap* cbw,
+                                    int64 timeoutMilliseconds) {
+  boost::shared_ptr<TcpTransport> pTcp = GetTransport(addr, true);
+  if (pTcp != NULL) {
+    //<!not delete, for callback to delete;
+    int code = request.getCode();
+    int opaque = request.getOpaque();
+    boost::shared_ptr<ResponseFuture> responseFuture(
+        new ResponseFuture(code, opaque, this, timeoutMilliseconds, true, cbw));
+    addAsyncResponseFuture(opaque, responseFuture);
+    if (cbw) {
+      boost::asio::deadline_timer* t = new boost::asio::deadline_timer(
+          m_async_ioService,
+          boost::posix_time::milliseconds(timeoutMilliseconds));
+      addTimerCallback(t, opaque);
+      boost::system::error_code e;
+      t->async_wait(
+          boost::bind(&TcpRemotingClient::handleAsyncPullForResponseTimeout,
+                      this, e, opaque));
+    }
+
+    if (SendCommand(pTcp, request))  // Even if send failed, asyncTimerThread
+                                     // will trigger next pull request or report
+                                     // send msg failed
+    {
+      LOG_DEBUG("invokeAsync success, addr:%s, code:%d, opaque:%d",
+                addr.c_str(), code, opaque);
+      responseFuture->setSendRequestOK(true);
+    }
+    return true;
+  }
+  LOG_ERROR("invokeAsync failed of addr:%s", addr.c_str());
+  return false;
+}
+
+void TcpRemotingClient::invokeOneway(const string& addr,
+                                     RemotingCommand& request) {
+  //<!not need callback;
+  boost::shared_ptr<TcpTransport> pTcp = GetTransport(addr, true);
+  if (pTcp != NULL) {
+    request.markOnewayRPC();
+    LOG_DEBUG("invokeOneway success, addr:%s, code:%d", addr.c_str(),
+              request.getCode());
+    SendCommand(pTcp, request);
+  }
+}
+
+boost::shared_ptr<TcpTransport> TcpRemotingClient::GetTransport(
+    const string& addr, bool needRespons) {
+  if (addr.empty()) return CreateNameserverTransport(needRespons);
+
+  return CreateTransport(addr, needRespons);
+}
+
+boost::shared_ptr<TcpTransport> TcpRemotingClient::CreateTransport(
+    const string& addr, bool needRespons) {
+  boost::shared_ptr<TcpTransport> tts;
+  {
+    // try get m_tcpLock util m_tcpTransportTryLockTimeout to avoid blocking
+    // long
+    // time, if could not get m_tcpLock, return NULL
+    bool bGetMutex = false;
+    boost::unique_lock<boost::timed_mutex> lock(m_tcpLock, boost::try_to_lock);
+    if (!lock.owns_lock()) {
+      if (!lock.timed_lock(
+              boost::get_system_time() +
+              boost::posix_time::seconds(m_tcpTransportTryLockTimeout))) {
+        LOG_ERROR("GetTransport of:%s get timed_mutex timeout", addr.c_str());
+        boost::shared_ptr<TcpTransport> pTcp;
+        return pTcp;
+      } else {
+        bGetMutex = true;
+      }
+    } else {
+      bGetMutex = true;
+    }
+    if (bGetMutex) {
+      if (m_tcpTable.find(addr) != m_tcpTable.end()) {
+        boost::weak_ptr<TcpTransport> weakPtcp(m_tcpTable[addr]);
+        boost::shared_ptr<TcpTransport> tcp = weakPtcp.lock();
+        if (tcp) {
+          tcpConnectStatus connectStatus = tcp->getTcpConnectStatus();
+          if (connectStatus == e_connectWaitResponse) {
+            boost::shared_ptr<TcpTransport> pTcp;
+            return pTcp;
+          } else if (connectStatus == e_connectFail) {
+            LOG_ERROR("tcpTransport with server disconnected, erase server:%s",
+                      addr.c_str());
+            tcp->disconnect(
+                addr);  // avoid coredump when connection with broker was broken
+            m_tcpTable.erase(addr);
+          } else if (connectStatus == e_connectSuccess) {
+            return tcp;
+          } else {
+            LOG_ERROR(
+                "go to fault state, erase:%s from tcpMap, and reconnect "
+                "it",
+                addr.c_str());
+            m_tcpTable.erase(addr);
+          }
+        }
+      }
+
+      //<!callback;
+      READ_CALLBACK callback =
+          needRespons ? &TcpRemotingClient::static_messageReceived : NULL;
+
+      tts.reset(new TcpTransport(this, callback));
+      tcpConnectStatus connectStatus = tts->connect(addr, m_tcpConnectTimeout);
+      if (connectStatus != e_connectWaitResponse) {
+        LOG_WARN("can not connect to :%s", addr.c_str());
+        tts->disconnect(addr);
+        boost::shared_ptr<TcpTransport> pTcp;
+        return pTcp;
+      } else {
+        m_tcpTable[addr] = tts;  // even if connecting failed finally, this
+                                 // server transport will be erased by next
+                                 // CreateTransport
+      }
+    } else {
+      LOG_WARN("get tcpTransport mutex failed :%s", addr.c_str());
+      boost::shared_ptr<TcpTransport> pTcp;
+      return pTcp;
+    }
+  }
+
+  tcpConnectStatus connectStatus =
+      tts->waitTcpConnectEvent(m_tcpConnectTimeout);
+  if (connectStatus != e_connectSuccess) {
+    LOG_WARN("can not connect to server:%s", addr.c_str());
+    tts->disconnect(addr);
+    boost::shared_ptr<TcpTransport> pTcp;
+    return pTcp;
+  } else {
+    LOG_INFO("connect server with addr:%s success", addr.c_str());
+    return tts;
+  }
+}
+
+boost::shared_ptr<TcpTransport> TcpRemotingClient::CreateNameserverTransport(
+    bool needRespons) {
+  // m_namesrvLock was added to avoid operation of nameServer was blocked by
+  // m_tcpLock, it was used by single Thread mostly, so no performance impact
+  // try get m_tcpLock util m_tcpTransportTryLockTimeout to avoid blocking long
+  // time, if could not get m_namesrvlock, return NULL
+  bool bGetMutex = false;
+  boost::unique_lock<boost::timed_mutex> lock(m_namesrvlock,
+                                              boost::try_to_lock);
+  if (!lock.owns_lock()) {
+    if (!lock.timed_lock(
+            boost::get_system_time() +
+            boost::posix_time::seconds(m_tcpTransportTryLockTimeout))) {
+      LOG_ERROR("CreateNameserverTransport get timed_mutex timeout");
+      boost::shared_ptr<TcpTransport> pTcp;
+      return pTcp;
+    } else {
+      bGetMutex = true;
+    }
+  } else {
+    bGetMutex = true;
+  }
+
+  if (bGetMutex) {
+    if (!m_namesrvAddrChoosed.empty()) {
+      boost::shared_ptr<TcpTransport> pTcp =
+          GetTransport(m_namesrvAddrChoosed, true);
+      if (pTcp)
+        return pTcp;
+      else
+        m_namesrvAddrChoosed.clear();
+    }
+
+    vector<string>::iterator itp = m_namesrvAddrList.begin();
+    for (; itp != m_namesrvAddrList.end(); ++itp) {
+      unsigned int index = m_namesrvIndex % m_namesrvAddrList.size();
+      if (m_namesrvIndex == numeric_limits<unsigned int>::max())
+        m_namesrvIndex = 0;
+      m_namesrvIndex++;
+      LOG_INFO("namesrvIndex is:%d, index:%d, namesrvaddrlist size:%zu",
+               m_namesrvIndex, index, m_namesrvAddrList.size());
+      boost::shared_ptr<TcpTransport> pTcp =
+          GetTransport(m_namesrvAddrList[index], true);
+      if (pTcp) {
+        m_namesrvAddrChoosed = m_namesrvAddrList[index];
+        return pTcp;
+      }
+    }
+    boost::shared_ptr<TcpTransport> pTcp;
+    return pTcp;
+  } else {
+    LOG_WARN("get nameServer tcpTransport mutex failed");
+    boost::shared_ptr<TcpTransport> pTcp;
+    return pTcp;
+  }
+}
+
+void TcpRemotingClient::CloseTransport(const string& addr,
+                                       boost::shared_ptr<TcpTransport> pTcp) {
+  if (addr.empty()) {
+    return CloseNameServerTransport(pTcp);
+  }
+
+  bool bGetMutex = false;
+  boost::unique_lock<boost::timed_mutex> lock(m_tcpLock, boost::try_to_lock);
+  if (!lock.owns_lock()) {
+    if (!lock.timed_lock(
+            boost::get_system_time() +
+            boost::posix_time::seconds(m_tcpTransportTryLockTimeout))) {
+      LOG_ERROR("CloseTransport of:%s get timed_mutex timeout", addr.c_str());
+      return;
+    } else {
+      bGetMutex = true;
+    }
+  } else {
+    bGetMutex = true;
+  }
+  LOG_ERROR("CloseTransport of:%s", addr.c_str());
+  if (bGetMutex) {
+    bool removeItemFromTable = true;
+    if (m_tcpTable.find(addr) != m_tcpTable.end()) {
+      if (m_tcpTable[addr]->getStartTime() != pTcp->getStartTime()) {
+        LOG_INFO(
+            "tcpTransport with addr:%s has been closed before, and has been "
+            "created again, nothing to do",
+            addr.c_str());
+        removeItemFromTable = false;
+      }
+    } else {
+      LOG_INFO(
+          "tcpTransport with addr:%s had been removed from tcpTable before",
+          addr.c_str());
+      removeItemFromTable = false;
+    }
+
+    if (removeItemFromTable == true) {
+      LOG_WARN("closeTransport: disconnect broker:%s with state:%d",
+               addr.c_str(), m_tcpTable[addr]->getTcpConnectStatus());
+      if (m_tcpTable[addr]->getTcpConnectStatus() == e_connectSuccess)
+        m_tcpTable[addr]->disconnect(
+            addr);  // avoid coredump when connection with server was broken
+      LOG_WARN("closeTransport: erase broker: %s", addr.c_str());
+      m_tcpTable.erase(addr);
+    }
+  } else {
+    LOG_WARN("CloseTransport::get tcpTransport mutex failed:%s", addr.c_str());
+    return;
+  }
+  LOG_ERROR("CloseTransport of:%s end", addr.c_str());
+}
+
+void TcpRemotingClient::CloseNameServerTransport(
+    boost::shared_ptr<TcpTransport> pTcp) {
+  bool bGetMutex = false;
+  boost::unique_lock<boost::timed_mutex> lock(m_namesrvlock,
+                                              boost::try_to_lock);
+  if (!lock.owns_lock()) {
+    if (!lock.timed_lock(
+            boost::get_system_time() +
+            boost::posix_time::seconds(m_tcpTransportTryLockTimeout))) {
+      LOG_ERROR("CreateNameserverTransport get timed_mutex timeout");
+      return;
+    } else {
+      bGetMutex = true;
+    }
+  } else {
+    bGetMutex = true;
+  }
+  if (bGetMutex) {
+    string addr = m_namesrvAddrChoosed;
+    bool removeItemFromTable = true;
+    if (m_tcpTable.find(addr) != m_tcpTable.end()) {
+      if (m_tcpTable[addr]->getStartTime() != pTcp->getStartTime()) {
+        LOG_INFO(
+            "tcpTransport with addr:%s has been closed before, and has been "
+            "created again, nothing to do",
+            addr.c_str());
+        removeItemFromTable = false;
+      }
+    } else {
+      LOG_INFO(
+          "tcpTransport with addr:%s had been removed from tcpTable before",
+          addr.c_str());
+      removeItemFromTable = false;
+    }
+
+    if (removeItemFromTable == true) {
+      m_tcpTable[addr]->disconnect(
+          addr);  // avoid coredump when connection with server was broken
+      LOG_WARN("closeTransport: erase broker: %s", addr.c_str());
+      m_tcpTable.erase(addr);
+      m_namesrvAddrChoosed.clear();
+    }
+  } else {
+    LOG_WARN("CloseNameServerTransport::get tcpTransport mutex failed:%s",
+             m_namesrvAddrChoosed.c_str());
+    return;
+  }
+}
+
+bool TcpRemotingClient::SendCommand(boost::shared_ptr<TcpTransport> pTts,
+                                    RemotingCommand& msg) {
+  const MemoryBlock* phead = msg.GetHead();
+  const MemoryBlock* pbody = msg.GetBody();
+
+  unique_ptr<MemoryOutputStream> result(new MemoryOutputStream(1024));
+  if (phead->getData()) {
+    result->write(phead->getData(), phead->getSize());
+  }
+  if (pbody->getData()) {
+    result->write(pbody->getData(), pbody->getSize());
+  }
+  const char* pData = static_cast<const char*>(result->getData());
+  int len = result->getDataSize();
+  return pTts->sendMessage(pData, len);
+}
+
+void TcpRemotingClient::static_messageReceived(void* context,
+                                               const MemoryBlock& mem,
+                                               const string& addr) {
+  TcpRemotingClient* pTcpRemotingClient = (TcpRemotingClient*)context;
+  if (pTcpRemotingClient) pTcpRemotingClient->messageReceived(mem, addr);
+}
+
+void TcpRemotingClient::messageReceived(const MemoryBlock& mem,
+                                        const string& addr) {
+  m_ioService.post(
+      boost::bind(&TcpRemotingClient::ProcessData, this, mem, addr));
+}
+
+void TcpRemotingClient::ProcessData(const MemoryBlock& mem,
+                                    const string& addr) {
+  RemotingCommand* pRespondCmd = NULL;
+  try {
+    pRespondCmd = RemotingCommand::Decode(mem);
+  } catch (...) {
+    LOG_ERROR("processData_error");
+    return;
+  }
+
+  int opaque = pRespondCmd->getOpaque();
+
+  //<!process self;
+  if (pRespondCmd->isResponseType()) {
+    boost::shared_ptr<ResponseFuture> pFuture(
+        findAndDeleteAsyncResponseFuture(opaque));
+    if (!pFuture) {
+      pFuture = findAndDeleteResponseFuture(opaque);
+      if (pFuture) {
+        if (pFuture->getSyncResponseFlag()) {
+          LOG_WARN("waitResponse already timeout of opaque:%d", opaque);
+          deleteAndZero(pRespondCmd);
+          return;
+        }
+        LOG_DEBUG("find_response opaque:%d", opaque);
+      } else {
+        LOG_DEBUG("responseFuture was deleted by timeout of opaque:%d", opaque);
+        deleteAndZero(pRespondCmd);
+        return;
+      }
+    }
+    processResponseCommand(pRespondCmd, pFuture);
+  } else {
+    processRequestCommand(pRespondCmd, addr);
+  }
+}
+
+void TcpRemotingClient::processResponseCommand(
+    RemotingCommand* pCmd, boost::shared_ptr<ResponseFuture> pfuture) {
+  int code = pfuture->getRequestCode();
+  int opaque = pCmd->getOpaque();
+  LOG_DEBUG("processResponseCommand, code:%d,opaque:%d", code, opaque);
+  pCmd->SetExtHeader(code);  // set head , for response use
+
+  pfuture->setResponse(pCmd);
+
+  if (pfuture->getASyncFlag()) {
+    if (!pfuture->getAsyncResponseFlag()) {
+      pfuture->setAsyncResponseFlag();
+      pfuture->setAsyncCallBackStatus(asyncCallBackStatus_response);
+      pfuture->executeInvokeCallback();
+      cancelTimerCallback(opaque);
+    }
+  }
+}
+
+void TcpRemotingClient::processRequestCommand(RemotingCommand* pCmd,
+                                              const string& addr) {
+  unique_ptr<RemotingCommand> pRequestCommand(pCmd);
+  int requestCode = pRequestCommand->getCode();
+  if (m_requestTable.find(requestCode) == m_requestTable.end()) {
+    LOG_ERROR("can_not_find request:%d processor", requestCode);
+  } else {
+    unique_ptr<RemotingCommand> pResponse(
+        m_requestTable[requestCode]->processRequest(addr,
+                                                    pRequestCommand.get()));
+    if (!pRequestCommand->isOnewayRPC()) {
+      if (pResponse) {
+        pResponse->setOpaque(pRequestCommand->getOpaque());
+        pResponse->markResponseType();
+        pResponse->Encode();
+
+        invokeOneway(addr, *pResponse);
+      }
+    }
+  }
+}
+
+void TcpRemotingClient::addResponseFuture(
+    int opaque, boost::shared_ptr<ResponseFuture> pfuture) {
+  boost::lock_guard<boost::mutex> lock(m_futureTableMutex);
+  m_futureTable[opaque] = pfuture;
+}
+
+// Note: after call this function, shared_ptr of m_syncFutureTable[opaque] will
+// be erased, so caller must ensure the life cycle of returned shared_ptr;
+boost::shared_ptr<ResponseFuture>
+TcpRemotingClient::findAndDeleteResponseFuture(int opaque) {
+  boost::lock_guard<boost::mutex> lock(m_futureTableMutex);
+  boost::shared_ptr<ResponseFuture> pResponseFuture;
+  if (m_futureTable.find(opaque) != m_futureTable.end()) {
+    pResponseFuture = m_futureTable[opaque];
+    m_futureTable.erase(opaque);
+  }
+  return pResponseFuture;
+}
+
+void TcpRemotingClient::handleAsyncPullForResponseTimeout(
+    const boost::system::error_code& e, int opaque) {
+  if (e == boost::asio::error::operation_aborted) {
+    return;
+  }
+
+  boost::shared_ptr<ResponseFuture> pFuture(
+      findAndDeleteAsyncResponseFuture(opaque));
+  if (pFuture && pFuture->getASyncFlag() && (pFuture->getAsyncCallbackWrap())) {
+    if ((pFuture->getAsyncResponseFlag() !=
+         true))  // if no response received, then check timeout or not
+    {
+      LOG_ERROR("no response got for opaque:%d", opaque);
+      pFuture->setAsyncCallBackStatus(asyncCallBackStatus_timeout);
+      pFuture->executeInvokeCallbackException();
+    }
+  }
+
+  eraseTimerCallback(opaque);
+}
+
+void TcpRemotingClient::addAsyncResponseFuture(
+    int opaque, boost::shared_ptr<ResponseFuture> pfuture) {
+  boost::lock_guard<boost::mutex> lock(m_asyncFutureLock);
+  m_asyncFutureTable[opaque] = pfuture;
+}
+
+// Note: after call this function, shared_ptr of m_asyncFutureTable[opaque] will
+// be erased, so caller must ensure the life cycle of returned shared_ptr;
+boost::shared_ptr<ResponseFuture>
+TcpRemotingClient::findAndDeleteAsyncResponseFuture(int opaque) {
+  boost::lock_guard<boost::mutex> lock(m_asyncFutureLock);
+  boost::shared_ptr<ResponseFuture> pResponseFuture;
+  if (m_asyncFutureTable.find(opaque) != m_asyncFutureTable.end()) {
+    pResponseFuture = m_asyncFutureTable[opaque];
+    m_asyncFutureTable.erase(opaque);
+  }
+
+  return pResponseFuture;
+}
+
+void TcpRemotingClient::registerProcessor(
+    MQRequestCode requestCode,
+    ClientRemotingProcessor* clientRemotingProcessor) {
+  if (m_requestTable.find(requestCode) != m_requestTable.end())
+    m_requestTable.erase(requestCode);
+  m_requestTable[requestCode] = clientRemotingProcessor;
+}
+
+void TcpRemotingClient::addTimerCallback(boost::asio::deadline_timer* t,
+                                         int opaque) {
+  boost::lock_guard<boost::mutex> lock(m_timerMapMutex);
+  if (m_async_timer_map.find(opaque) != m_async_timer_map.end()) {
+    // AGENT_INFO("addTimerCallback:erase timerCallback opaque:%lld", opaque);
+    boost::asio::deadline_timer* old_t = m_async_timer_map[opaque];
+    old_t->cancel();
+    delete old_t;
+    old_t = NULL;
+    m_async_timer_map.erase(opaque);
+  }
+  m_async_timer_map[opaque] = t;
+}
+
+void TcpRemotingClient::eraseTimerCallback(int opaque) {
+  boost::lock_guard<boost::mutex> lock(m_timerMapMutex);
+  if (m_async_timer_map.find(opaque) != m_async_timer_map.end()) {
+    boost::asio::deadline_timer* t = m_async_timer_map[opaque];
+    delete t;
+    t = NULL;
+    m_async_timer_map.erase(opaque);
+  }
+}
+
+void TcpRemotingClient::cancelTimerCallback(int opaque) {
+  boost::lock_guard<boost::mutex> lock(m_timerMapMutex);
+  if (m_async_timer_map.find(opaque) != m_async_timer_map.end()) {
+    // AGENT_INFO("cancel timerCallback opaque:%lld", opaque);
+    boost::asio::deadline_timer* t = m_async_timer_map[opaque];
+    t->cancel();
+    delete t;
+    t = NULL;
+    m_async_timer_map.erase(opaque);
+  }
+}
+
+void TcpRemotingClient::removeAllTimerCallback() {
+  boost::lock_guard<boost::mutex> lock(m_timerMapMutex);
+  for (asyncTimerMap::iterator it = m_async_timer_map.begin();
+       it != m_async_timer_map.end(); ++it) {
+    boost::asio::deadline_timer* t = it->second;
+    t->cancel();
+    delete t;
+    t = NULL;
+  }
+  m_async_timer_map.clear();
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/TcpRemotingClient.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/TcpRemotingClient.h b/rocketmq-cpp/src/transport/TcpRemotingClient.h
new file mode 100755
index 0000000..832b49a
--- /dev/null
+++ b/rocketmq-cpp/src/transport/TcpRemotingClient.h
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __TCPREMOTINGCLIENT_H__
+#define __TCPREMOTINGCLIENT_H__
+
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/weak_ptr.hpp>
+#include <map>
+#include "ClientRemotingProcessor.h"
+#include "RemotingCommand.h"
+#include "ResponseFuture.h"
+#include "SocketUtil.h"
+#include "TcpTransport.h"
+
+namespace rocketmq {
+//<!************************************************************************
+
+class TcpRemotingClient {
+ public:
+  TcpRemotingClient(int pullThreadNum, uint64_t tcpConnectTimeout,
+                    uint64_t tcpTransportTryLockTimeout);
+  virtual ~TcpRemotingClient();
+  void stopAllTcpTransportThread();
+  void updateNameServerAddressList(const string& addrs);
+
+  //<!delete outsite;
+  RemotingCommand* invokeSync(const string& addr, RemotingCommand& request,
+                              int timeoutMillis = 3000);
+
+  bool invokeHeartBeat(const string& addr, RemotingCommand& request);
+
+  bool invokeAsync(const string& addr, RemotingCommand& request,
+                   AsyncCallbackWrap* cbw, int64 timeoutMilliseconds);
+
+  void invokeOneway(const string& addr, RemotingCommand& request);
+
+  void ProcessData(const MemoryBlock& mem, const string& addr);
+
+  void registerProcessor(MQRequestCode requestCode,
+                         ClientRemotingProcessor* clientRemotingProcessor);
+
+  void boost_asio_work();
+  void handleAsyncPullForResponseTimeout(const boost::system::error_code& e,
+                                         int opaque);
+
+ private:
+  static void static_messageReceived(void* context, const MemoryBlock& mem,
+                                     const string& addr);
+  void messageReceived(const MemoryBlock& mem, const string& addr);
+  boost::shared_ptr<TcpTransport> GetTransport(const string& addr,
+                                               bool needRespons);
+  boost::shared_ptr<TcpTransport> CreateTransport(const string& addr,
+                                                  bool needRespons);
+  boost::shared_ptr<TcpTransport> CreateNameserverTransport(bool needRespons);
+  void CloseTransport(const string& addr, boost::shared_ptr<TcpTransport> pTcp);
+  void CloseNameServerTransport(boost::shared_ptr<TcpTransport> pTcp);
+  bool SendCommand(boost::shared_ptr<TcpTransport> pTts, RemotingCommand& msg);
+  void processRequestCommand(RemotingCommand* pCmd, const string& addr);
+  void processResponseCommand(RemotingCommand* pCmd,
+                              boost::shared_ptr<ResponseFuture> pfuture);
+
+  void addResponseFuture(int opaque, boost::shared_ptr<ResponseFuture> pfuture);
+  boost::shared_ptr<ResponseFuture> findAndDeleteResponseFuture(int opaque);
+
+  void addAsyncResponseFuture(int opaque,
+                              boost::shared_ptr<ResponseFuture> pfuture);
+  boost::shared_ptr<ResponseFuture> findAndDeleteAsyncResponseFuture(
+      int opaque);
+
+  void addTimerCallback(boost::asio::deadline_timer* t, int opaque);
+  void eraseTimerCallback(int opaque);
+  void cancelTimerCallback(int opaque);
+  void removeAllTimerCallback();
+
+ private:
+  typedef map<string, boost::shared_ptr<TcpTransport>> TcpMap;
+  typedef map<int, boost::shared_ptr<ResponseFuture>> ResMap;
+
+  typedef map<int, ClientRemotingProcessor*> RequestMap;
+  RequestMap m_requestTable;
+
+  boost::mutex m_futureTableMutex;
+  ResMap m_futureTable;  //<! id->future;
+
+  ResMap m_asyncFutureTable;
+  boost::mutex m_asyncFutureLock;
+
+  TcpMap m_tcpTable;  //<! ip->tcp;
+  boost::timed_mutex m_tcpLock;
+
+  // ThreadPool        m_threadpool;
+  int m_pullThreadNum;
+  uint64_t m_tcpConnectTimeout;           // ms
+  uint64_t m_tcpTransportTryLockTimeout;  // s
+
+  //<! Nameserver
+  boost::timed_mutex m_namesrvlock;
+  vector<string> m_namesrvAddrList;
+  string m_namesrvAddrChoosed;
+  unsigned int m_namesrvIndex;
+  boost::asio::io_service m_ioService;
+  boost::thread_group m_threadpool;
+  boost::asio::io_service::work m_ioServiceWork;
+
+  boost::asio::io_service m_async_ioService;
+  unique_ptr<boost::thread> m_async_service_thread;
+
+  typedef map<int, boost::asio::deadline_timer*> asyncTimerMap;
+  boost::mutex m_timerMapMutex;
+  asyncTimerMap m_async_timer_map;
+};
+
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/TcpTransport.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/TcpTransport.cpp b/rocketmq-cpp/src/transport/TcpTransport.cpp
new file mode 100755
index 0000000..82e9526
--- /dev/null
+++ b/rocketmq-cpp/src/transport/TcpTransport.cpp
@@ -0,0 +1,330 @@
+/*
+ * 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 "TcpTransport.h"
+#include <arpa/inet.h>  // for sockaddr_in and inet_ntoa...
+#include <netinet/tcp.h>
+#include <sys/socket.h>  // for socket(), bind(), and connect()...
+#include "Logging.h"
+#include "TcpRemotingClient.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+//<!************************************************************************
+TcpTransport::TcpTransport(TcpRemotingClient *pTcpRemointClient,
+                           READ_CALLBACK handle /* = NULL */)
+    : m_tcpConnectStatus(e_connectInit),
+      m_ReadDatathread(NULL),
+      m_readcallback(handle),
+      m_tcpRemotingClient(pTcpRemointClient) {
+  m_startTime = UtilAll::currentTimeMillis();
+  evthread_use_pthreads();
+  m_eventBase = NULL;
+  m_bufferEvent = NULL;
+}
+TcpTransport::~TcpTransport() {
+  m_readcallback = NULL;
+  m_bufferEvent = NULL;
+  m_eventBase = NULL;
+}
+
+tcpConnectStatus TcpTransport::connect(const string &strServerURL,
+                                       int timeOutMillisecs /* = 3000 */) {
+  string hostName;
+  short portNumber;
+  if (!UtilAll::SplitURL(strServerURL, hostName, portNumber)) {
+    return e_connectFail;
+  }
+
+  boost::lock_guard<boost::mutex> lock(m_socketLock);
+
+  struct sockaddr_in sin;
+  memset(&sin, 0, sizeof(sin));
+  sin.sin_family = AF_INET;
+  sin.sin_addr.s_addr = inet_addr(hostName.c_str());
+  sin.sin_port = htons(portNumber);
+
+  m_eventBase = event_base_new();
+  m_bufferEvent = bufferevent_socket_new(
+      m_eventBase, -1, BEV_OPT_CLOSE_ON_FREE | BEV_OPT_THREADSAFE);
+  bufferevent_setcb(m_bufferEvent, readNextMessageIntCallback, NULL, eventcb,
+                    this);
+  bufferevent_enable(m_bufferEvent, EV_READ | EV_WRITE);
+  bufferevent_setwatermark(m_bufferEvent, EV_READ, 4, 0);
+
+  setTcpConnectStatus(e_connectWaitResponse);
+  if (bufferevent_socket_connect(m_bufferEvent, (struct sockaddr *)&sin,
+                                 sizeof(sin)) < 0) {
+    LOG_INFO("connect to fd:%d failed", bufferevent_getfd(m_bufferEvent));
+    setTcpConnectStatus(e_connectFail);
+    freeBufferEvent();
+    return e_connectFail;
+  } else {
+    int fd = bufferevent_getfd(m_bufferEvent);
+    LOG_INFO("try to connect to fd:%d, addr:%s", fd, (hostName.c_str()));
+    /*struct timeval timeout;
+    timeout.tv_sec = timeOutMillisecs/1000;
+    timeout.tv_usec = 0;
+    struct event* evtimeout = evtimer_new(m_eventBase, timeoutcb, this);
+    evtimer_add(evtimeout, &timeout);*/
+    evthread_make_base_notifiable(m_eventBase);
+    m_ReadDatathread =
+        new boost::thread(boost::bind(&TcpTransport::runThread, this));
+    return e_connectWaitResponse;
+  }
+}
+
+void TcpTransport::setTcpConnectStatus(tcpConnectStatus connectStatus) {
+  m_tcpConnectStatus = connectStatus;
+}
+
+tcpConnectStatus TcpTransport::getTcpConnectStatus() {
+  return m_tcpConnectStatus;
+}
+
+tcpConnectStatus TcpTransport::waitTcpConnectEvent(int timeoutMillisecs) {
+  boost::unique_lock<boost::mutex> lk(m_connectEventLock);
+  if (!m_connectEvent.timed_wait(
+          lk, boost::posix_time::milliseconds(timeoutMillisecs))) {
+    LOG_INFO("connect timeout");
+  }
+  return getTcpConnectStatus();
+}
+
+void TcpTransport::setTcpConnectEvent(tcpConnectStatus connectStatus) {
+  tcpConnectStatus baseStatus(getTcpConnectStatus());
+  setTcpConnectStatus(connectStatus);
+  if (baseStatus == e_connectWaitResponse) {
+    LOG_INFO("received libevent callback event");
+    m_connectEvent.notify_all();
+  }
+}
+
+void TcpTransport::disconnect(const string &addr) {
+  boost::lock_guard<boost::mutex> lock(m_socketLock);
+  if (getTcpConnectStatus() != e_connectInit) {
+    clearBufferEventCallback();
+    LOG_INFO("disconnect:%s start", addr.c_str());
+    m_connectEvent.notify_all();
+    setTcpConnectStatus(e_connectInit);
+    if (m_ReadDatathread) {
+      m_ReadDatathread->interrupt();
+      exitBaseDispatch();
+      while (m_ReadDatathread->timed_join(boost::posix_time::seconds(1)) ==
+             false) {
+        LOG_WARN("join readDataThread fail, retry");
+        m_ReadDatathread->interrupt();
+        exitBaseDispatch();
+      }
+      delete m_ReadDatathread;
+      m_ReadDatathread = NULL;
+    }
+    freeBufferEvent();
+    LOG_INFO("disconnect:%s completely", addr.c_str());
+  }
+}
+
+void TcpTransport::clearBufferEventCallback() {
+  if (m_bufferEvent) {
+    // Bufferevents are internally reference-counted, so if the bufferevent has
+    // pending deferred callbacks when you free it, it won't be deleted until
+    // the callbacks are done.
+    // so just empty callback to avoid future callback by libevent
+    bufferevent_setcb(m_bufferEvent, NULL, NULL, NULL, NULL);
+  }
+}
+
+void TcpTransport::freeBufferEvent() {
+  if (m_bufferEvent) {
+    bufferevent_free(m_bufferEvent);
+  }
+  if (m_eventBase) {
+    event_base_free(m_eventBase);
+  }
+}
+void TcpTransport::exitBaseDispatch() {
+  if (m_eventBase) {
+    event_base_loopbreak(m_eventBase);
+    // event_base_loopexit(m_eventBase, NULL);  //Note: memory leak will be
+    // occured when timer callback was not done;
+  }
+}
+
+void TcpTransport::runThread() {
+  while (m_ReadDatathread) {
+    if (m_eventBase != NULL) {
+      event_base_dispatch(m_eventBase);
+      // event_base_loop(m_eventBase, EVLOOP_ONCE);//EVLOOP_NONBLOCK should not
+      // be used, as could not callback event immediatly
+    }
+    LOG_INFO("event_base_dispatch exit once");
+    boost::this_thread::sleep(boost::posix_time::milliseconds(1));
+    if (getTcpConnectStatus() != e_connectSuccess) return;
+  }
+}
+
+void TcpTransport::timeoutcb(evutil_socket_t fd, short what, void *arg) {
+  LOG_INFO("timeoutcb: received  event:%d on fd:%d", what, fd);
+  TcpTransport *tcpTrans = (TcpTransport *)arg;
+  if (tcpTrans->getTcpConnectStatus() != e_connectSuccess) {
+    LOG_INFO("timeoutcb: after connect time, tcp was not established on fd:%d",
+             fd);
+    tcpTrans->setTcpConnectStatus(e_connectFail);
+  } else {
+    LOG_INFO("timeoutcb: after connect time, tcp was established on fd:%d", fd);
+  }
+}
+
+void TcpTransport::eventcb(struct bufferevent *bev, short what, void *ctx) {
+  evutil_socket_t fd = bufferevent_getfd(bev);
+  TcpTransport *tcpTrans = (TcpTransport *)ctx;
+  LOG_INFO("eventcb: received event:%x on fd:%d", what, fd);
+  if (what & BEV_EVENT_CONNECTED) {
+    int val = 1;
+    setsockopt(fd, IPPROTO_TCP, TCP_NODELAY, (char *)&val, sizeof(val));
+    LOG_INFO("eventcb:connect to fd:%d successfully", fd);
+    tcpTrans->setTcpConnectEvent(e_connectSuccess);
+  } else if (what & (BEV_EVENT_ERROR | BEV_EVENT_EOF | BEV_EVENT_READING |
+                     BEV_EVENT_WRITING)) {
+    LOG_INFO("eventcb:rcv error event cb:%x on fd:%d", what, fd);
+    tcpTrans->setTcpConnectEvent(e_connectFail);
+    bufferevent_setcb(bev, NULL, NULL, NULL, NULL);
+    // bufferevent_disable(bev, EV_READ|EV_WRITE);
+    // bufferevent_free(bev);
+  } else {
+    LOG_ERROR("eventcb: received error event:%d on fd:%d", what, fd);
+  }
+}
+
+void TcpTransport::readNextMessageIntCallback(struct bufferevent *bev,
+                                              void *ctx) {
+  /* This callback is invoked when there is data to read on bev. */
+
+  // protocol:  <length> <header length> <header data> <body data>
+  //                    1                   2                       3 4
+  // rocketmq protocol contains 4 parts as following:
+  //     1��big endian 4 bytes int, its length is sum of 2,3 and 4
+  //     2��big endian 4 bytes int, its length is 3
+  //     3��use json to serialization data
+  //     4��application could self-defination binary data
+
+  struct evbuffer *input = bufferevent_get_input(bev);
+  while (1) {
+    struct evbuffer_iovec v[4];
+    int n = evbuffer_peek(input, 4, NULL, v, sizeof(v) / sizeof(v[0]));
+
+    int idx = 0;
+    char hdr[4];
+    char *p = hdr;
+    unsigned int needed = 4;
+
+    for (idx = 0; idx < n; idx++) {
+      if (needed) {
+        unsigned int tmp = needed < v[idx].iov_len ? needed : v[idx].iov_len;
+        memcpy(p, v[idx].iov_base, tmp);
+        p += tmp;
+        needed -= tmp;
+      } else {
+        break;
+      }
+    }
+
+    if (needed) {
+      LOG_DEBUG(" too little data received with sum = %d ", 4 - needed);
+      return;
+    }
+    uint32 totalLenOfOneMsg =
+        *(uint32 *)hdr;  // first 4 bytes, which indicates 1st part of protocol
+    uint32 bytesInMessage = ntohl(totalLenOfOneMsg);
+    LOG_DEBUG("fd:%d, totalLen:%zu, bytesInMessage:%d", bufferevent_getfd(bev),
+              v[0].iov_len, bytesInMessage);
+
+    uint32 len = evbuffer_get_length(input);
+    if (len >= bytesInMessage + 4) {
+      LOG_DEBUG("had received all data with len:%d from fd:%d", len,
+                bufferevent_getfd(bev));
+    } else {
+      LOG_DEBUG(
+          "didn't received whole bytesInMessage:%d, from fd:%d, totalLen:%d",
+          bytesInMessage, bufferevent_getfd(bev), len);
+      return;  // consider large data which was not received completely by now
+    }
+
+    if (bytesInMessage > 0) {
+      MemoryBlock messageData(bytesInMessage, true);
+      uint32 bytesRead = 0;
+      void *data = (void *)(messageData.getData() + bytesRead);
+      bufferevent_read(bev, data, 4);
+      bytesRead = bufferevent_read(bev, data, bytesInMessage);
+
+      TcpTransport *tcpTrans = (TcpTransport *)ctx;
+      tcpTrans->messageReceived(messageData);
+    }
+  }
+}
+
+bool TcpTransport::sendMessage(const char *pData, int len) {
+  boost::lock_guard<boost::mutex> lock(m_socketLock);
+  if (getTcpConnectStatus() != e_connectSuccess) {
+    return false;
+  }
+
+  int bytes_left = len;
+  int bytes_written = 0;
+  const char *ptr = pData;
+
+  /*NOTE:
+      1. do not need to consider large data which could not send by once, as
+     bufferevent could handle this case;
+  */
+  if (m_bufferEvent) {
+    bytes_written = bufferevent_write(m_bufferEvent, ptr, bytes_left);
+    if (bytes_written == 0)
+      return true;
+    else
+      return false;
+  }
+  return false;
+}
+
+void TcpTransport::messageReceived(const MemoryBlock &mem) {
+  if (m_readcallback) {
+    m_readcallback(m_tcpRemotingClient, mem, getPeerAddrAndPort());
+  }
+}
+
+const string TcpTransport::getPeerAddrAndPort() {
+  struct sockaddr_in broker;
+  socklen_t cLen = sizeof(broker);
+
+  // getsockname(m_socket->getRawSocketHandle(), (struct sockaddr*) &s, &sLen);
+  // // ! use connectSock here.
+  getpeername(bufferevent_getfd(m_bufferEvent), (struct sockaddr *)&broker,
+              &cLen);  // ! use connectSock here.
+  LOG_DEBUG("broker addr: %s, broker port: %d", inet_ntoa(broker.sin_addr),
+            ntohs(broker.sin_port));
+  string brokerAddr(inet_ntoa(broker.sin_addr));
+  brokerAddr.append(":");
+  string brokerPort(UtilAll::to_string(ntohs(broker.sin_port)));
+  brokerAddr.append(brokerPort);
+  LOG_DEBUG("brokerAddr:%s", brokerAddr.c_str());
+  return brokerAddr;
+}
+
+const uint64_t TcpTransport::getStartTime() const { return m_startTime; }
+
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/TcpTransport.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/TcpTransport.h b/rocketmq-cpp/src/transport/TcpTransport.h
new file mode 100755
index 0000000..7ba32dc
--- /dev/null
+++ b/rocketmq-cpp/src/transport/TcpTransport.h
@@ -0,0 +1,91 @@
+/*
+ * 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 __TCPTRANSPORT_H__
+#define __TCPTRANSPORT_H__
+
+#include <boost/atomic.hpp>
+#include <boost/thread/condition_variable.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include "dataBlock.h"
+
+extern "C" {
+#include "event2/buffer.h"
+#include "event2/bufferevent.h"
+#include "event2/event.h"
+#include "event2/thread.h"
+}
+
+namespace rocketmq {
+//<!***************************************************************************
+typedef enum {
+  e_connectInit = 0,
+  e_connectWaitResponse = 1,
+  e_connectSuccess = 2,
+  e_connectFail = 3
+} tcpConnectStatus;
+
+typedef void (*READ_CALLBACK)(void *context, const MemoryBlock &,
+                              const std::string &);
+class TcpRemotingClient;
+class TcpTransport {
+ public:
+  TcpTransport(TcpRemotingClient *pTcpRemointClient,
+               READ_CALLBACK handle = NULL);
+  virtual ~TcpTransport();
+
+  tcpConnectStatus connect(const std::string &strServerURL,
+                           int timeOutMillisecs = 3000);
+  void disconnect(const std::string &addr);
+  tcpConnectStatus waitTcpConnectEvent(int timeoutMillisecs = 3000);
+  void setTcpConnectStatus(tcpConnectStatus connectStatus);
+  tcpConnectStatus getTcpConnectStatus();
+  bool sendMessage(const char *pData, int len);
+  const std::string getPeerAddrAndPort();
+  const uint64_t getStartTime() const;
+
+ private:
+  void messageReceived(const MemoryBlock &mem);
+  static void readNextMessageIntCallback(struct bufferevent *bev, void *ctx);
+  static void eventcb(struct bufferevent *bev, short what, void *ctx);
+  static void timeoutcb(evutil_socket_t fd, short what, void *arg);
+  void runThread();
+  void clearBufferEventCallback();
+  void freeBufferEvent();
+  void exitBaseDispatch();
+  void setTcpConnectEvent(tcpConnectStatus connectStatus);
+
+ private:
+  uint64_t m_startTime;
+  boost::mutex m_socketLock;
+  struct event_base *m_eventBase;
+  struct bufferevent *m_bufferEvent;
+  boost::atomic<tcpConnectStatus> m_tcpConnectStatus;
+  boost::mutex m_connectEventLock;
+  boost::condition_variable_any m_connectEvent;
+  //<!read data thread
+  boost::thread *m_ReadDatathread;
+
+  //<! read data callback
+  READ_CALLBACK m_readcallback;
+  TcpRemotingClient *m_tcpRemotingClient;
+};
+
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif



[05/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/Rebalance.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/Rebalance.cpp b/rocketmq-cpp/src/consumer/Rebalance.cpp
new file mode 100755
index 0000000..a19e7a7
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/Rebalance.cpp
@@ -0,0 +1,677 @@
+/*
+ * 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 "Rebalance.h"
+#include "DefaultMQPushConsumer.h"
+#include "LockBatchBody.h"
+#include "Logging.h"
+#include "MQClientAPIImpl.h"
+#include "MQClientFactory.h"
+#include "OffsetStore.h"
+
+namespace rocketmq {
+//<!************************************************************************
+Rebalance::Rebalance(MQConsumer* consumer, MQClientFactory* pfactory)
+    : m_pConsumer(consumer), m_pClientFactory(pfactory) {
+  m_pAllocateMQStrategy = new AllocateMQAveragely();
+}
+
+Rebalance::~Rebalance() {
+  {
+    map<string, SubscriptionData*>::iterator it = m_subscriptionData.begin();
+    for (; it != m_subscriptionData.end(); ++it) deleteAndZero(it->second);
+    m_subscriptionData.clear();
+  }
+  {
+    MQ2PULLREQ::iterator it = m_requestQueueTable.begin();
+    for (; it != m_requestQueueTable.end(); ++it) {
+      delete it->second;
+      it->second = NULL;
+    }
+    m_requestQueueTable.clear();
+  }
+  m_topicSubscribeInfoTable.clear();
+  m_pConsumer = NULL;
+  m_pClientFactory = NULL;
+  deleteAndZero(m_pAllocateMQStrategy);
+}
+
+void Rebalance::doRebalance() {
+  LOG_DEBUG("start doRebalance");
+  try {
+    map<string, SubscriptionData*>::iterator it = m_subscriptionData.begin();
+    for (; it != m_subscriptionData.end(); ++it) {
+      string topic = (it->first);
+      LOG_INFO("current topic is:%s", topic.c_str());
+      //<!topic -> mqs
+      vector<MQMessageQueue> mqAll;
+      if (!getTopicSubscribeInfo(topic, mqAll)) {
+        continue;
+      }
+      if (mqAll.empty()) {
+        if (!UtilAll::startsWith_retry(topic))
+          THROW_MQEXCEPTION(MQClientException, "doRebalance the topic is empty",
+                            -1);
+      }
+
+      //<!msg model;
+      switch (m_pConsumer->getMessageModel()) {
+        case BROADCASTING: {
+          bool changed = updateRequestTableInRebalance(topic, mqAll);
+          if (changed) {
+            messageQueueChanged(topic, mqAll, mqAll);
+          }
+          break;
+        }
+        case CLUSTERING: {
+          vector<string> cidAll;
+          m_pClientFactory->findConsumerIds(
+              topic, m_pConsumer->getGroupName(), cidAll,
+              m_pConsumer->getSessionCredentials());
+
+          if (cidAll.empty()) {
+            /*remove the droping pullRequest changes for recovery consume fastly
+                from network broken
+                //drop all pullRequest
+                MQ2PULLREQ::iterator it = m_requestQueueTable.begin();
+                for (; it != m_requestQueueTable.end(); ++it)
+                {
+                    if(!(it->second->isDroped()))
+                    {
+                        MQMessageQueue mqtemp = it->first;
+                        it->second->setDroped(true);
+                        removeUnnecessaryMessageQueue(mqtemp);
+                        it->second->clearAllMsgs();//add clear operation to
+            avoid bad
+                state when dropped pullRequest returns normal
+                        LOG_INFO("find consumer failed, drop undropped mq:%s",
+                mqtemp.toString().c_str());
+                    }
+            }*/
+
+            THROW_MQEXCEPTION(MQClientException,
+                              "doRebalance the cidAll is empty", -1);
+          }
+          // log
+          for (int i = 0; i < (int)cidAll.size(); ++i) {
+            LOG_INFO("client id:%s of topic:%s", cidAll[i].c_str(),
+                     topic.c_str());
+          }
+          //<! sort;
+          sort(mqAll.begin(), mqAll.end());
+          sort(cidAll.begin(), cidAll.end());
+
+          //<! allocate;
+          vector<MQMessageQueue> allocateResult;
+          try {
+            m_pAllocateMQStrategy->allocate(m_pConsumer->getMQClientId(), mqAll,
+                                            cidAll, allocateResult);
+          } catch (MQException& e) {
+            THROW_MQEXCEPTION(MQClientException, "allocate error", -1);
+          }
+
+          // log
+          for (int i = 0; i < (int)allocateResult.size(); ++i) {
+            LOG_INFO("allocate mq:%s", allocateResult[i].toString().c_str());
+          }
+
+          //<!update local;
+          bool changed = updateRequestTableInRebalance(topic, allocateResult);
+          if (changed) {
+            messageQueueChanged(topic, mqAll, allocateResult);
+            break;
+          }
+        }
+        default:
+          break;
+      }
+    }
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+void Rebalance::persistConsumerOffset() {
+  DefaultMQPushConsumer* pConsumer =
+      static_cast<DefaultMQPushConsumer*>(m_pConsumer);
+  OffsetStore* pOffsetStore = pConsumer->getOffsetStore();
+  vector<MQMessageQueue> mqs;
+  {
+    boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+    MQ2PULLREQ::iterator it = m_requestQueueTable.begin();
+    for (; it != m_requestQueueTable.end(); ++it) {
+      if (it->second && (!it->second->isDroped())) {
+        mqs.push_back(it->first);
+      }
+    }
+  }
+
+  if (pConsumer->getMessageModel() == BROADCASTING) {
+    pOffsetStore->persistAll(mqs);
+  } else {
+    vector<MQMessageQueue>::iterator it2 = mqs.begin();
+    for (; it2 != mqs.end(); ++it2) {
+      pOffsetStore->persist(*it2, m_pConsumer->getSessionCredentials());
+    }
+  }
+}
+
+void Rebalance::persistConsumerOffsetByResetOffset() {
+  boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+  DefaultMQPushConsumer* pConsumer =
+      static_cast<DefaultMQPushConsumer*>(m_pConsumer);
+  OffsetStore* pOffsetStore = pConsumer->getOffsetStore();
+  vector<MQMessageQueue> mqs;
+  {
+    boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+    MQ2PULLREQ::iterator it = m_requestQueueTable.begin();
+    for (; it != m_requestQueueTable.end(); ++it) {
+      if (it->second) {  // even if it was dropped, also need update offset when
+                         // rcv resetOffset cmd
+        mqs.push_back(it->first);
+      }
+    }
+  }
+  vector<MQMessageQueue>::iterator it2 = mqs.begin();
+  for (; it2 != mqs.end(); ++it2) {
+    pOffsetStore->persist(*it2, m_pConsumer->getSessionCredentials());
+  }
+}
+
+SubscriptionData* Rebalance::getSubscriptionData(const string& topic) {
+  if (m_subscriptionData.find(topic) != m_subscriptionData.end()) {
+    return m_subscriptionData[topic];
+  }
+  return NULL;
+}
+
+map<string, SubscriptionData*>& Rebalance::getSubscriptionInner() {
+  return m_subscriptionData;
+}
+
+void Rebalance::setSubscriptionData(const string& topic,
+                                    SubscriptionData* pdata) {
+  if (pdata != NULL &&
+      m_subscriptionData.find(topic) == m_subscriptionData.end())
+    m_subscriptionData[topic] = pdata;
+}
+
+void Rebalance::setTopicSubscribeInfo(const string& topic,
+                                      vector<MQMessageQueue>& mqs) {
+  if (m_subscriptionData.find(topic) != m_subscriptionData.end()) {
+    {
+      boost::lock_guard<boost::mutex> lock(m_topicSubscribeInfoTableMutex);
+      if (m_topicSubscribeInfoTable.find(topic) !=
+          m_topicSubscribeInfoTable.end())
+        m_topicSubscribeInfoTable.erase(topic);
+      m_topicSubscribeInfoTable[topic] = mqs;
+    }
+    // log
+    vector<MQMessageQueue>::iterator it = mqs.begin();
+    for (; it != mqs.end(); ++it) {
+      LOG_DEBUG("topic [%s] has :%s", topic.c_str(), (*it).toString().c_str());
+    }
+  }
+}
+
+bool Rebalance::getTopicSubscribeInfo(const string& topic,
+                                      vector<MQMessageQueue>& mqs) {
+  boost::lock_guard<boost::mutex> lock(m_topicSubscribeInfoTableMutex);
+  if (m_topicSubscribeInfoTable.find(topic) !=
+      m_topicSubscribeInfoTable.end()) {
+    mqs = m_topicSubscribeInfoTable[topic];
+    return true;
+  }
+  return false;
+}
+
+void Rebalance::addPullRequest(MQMessageQueue mq, PullRequest* pPullRequest) {
+  boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+  m_requestQueueTable[mq] = pPullRequest;
+}
+
+PullRequest* Rebalance::getPullRequest(MQMessageQueue mq) {
+  boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+  if (m_requestQueueTable.find(mq) != m_requestQueueTable.end()) {
+    return m_requestQueueTable[mq];
+  }
+  return NULL;
+}
+
+map<MQMessageQueue, PullRequest*> Rebalance::getPullRequestTable() {
+  boost::lock_guard<boost::mutex> lock(m_requestTableMutex);
+  return m_requestQueueTable;
+}
+
+void Rebalance::unlockAll(bool oneway) {
+  map<string, vector<MQMessageQueue>*> brokerMqs;
+  MQ2PULLREQ requestQueueTable = getPullRequestTable();
+  for (MQ2PULLREQ::iterator it = requestQueueTable.begin();
+       it != requestQueueTable.end(); ++it) {
+    if (!(it->second->isDroped())) {
+      if (brokerMqs.find(it->first.getBrokerName()) == brokerMqs.end()) {
+        vector<MQMessageQueue>* mqs = new vector<MQMessageQueue>;
+        brokerMqs[it->first.getBrokerName()] = mqs;
+      } else {
+        brokerMqs[it->first.getBrokerName()]->push_back(it->first);
+      }
+    }
+  }
+  LOG_INFO("unLockAll %zu broker mqs", brokerMqs.size());
+  for (map<string, vector<MQMessageQueue>*>::iterator itb = brokerMqs.begin();
+       itb != brokerMqs.end(); ++itb) {
+    unique_ptr<FindBrokerResult> pFindBrokerResult(
+        m_pClientFactory->findBrokerAddressInSubscribe(itb->first, MASTER_ID,
+                                                       true));
+    unique_ptr<UnlockBatchRequestBody> unlockBatchRequest(
+        new UnlockBatchRequestBody());
+    vector<MQMessageQueue> mqs(*(itb->second));
+    unlockBatchRequest->setClientId(m_pConsumer->getMQClientId());
+    unlockBatchRequest->setConsumerGroup(m_pConsumer->getGroupName());
+    unlockBatchRequest->setMqSet(mqs);
+
+    try {
+      m_pClientFactory->getMQClientAPIImpl()->unlockBatchMQ(
+          pFindBrokerResult->brokerAddr, unlockBatchRequest.get(), 1000,
+          m_pConsumer->getSessionCredentials());
+      for (unsigned int i = 0; i != mqs.size(); ++i) {
+        PullRequest* pullreq = getPullRequest(mqs[i]);
+        if (pullreq) {
+          LOG_INFO("unlockBatchMQ success of mq:%s", mqs[i].toString().c_str());
+          pullreq->setLocked(true);
+        } else {
+          LOG_ERROR("unlockBatchMQ fails of mq:%s", mqs[i].toString().c_str());
+        }
+      }
+    } catch (MQException& e) {
+      LOG_ERROR("unlockBatchMQ fails");
+    }
+    deleteAndZero(itb->second);
+  }
+  brokerMqs.clear();
+}
+
+void Rebalance::unlock(MQMessageQueue mq) {
+  unique_ptr<FindBrokerResult> pFindBrokerResult(
+      m_pClientFactory->findBrokerAddressInSubscribe(mq.getBrokerName(),
+                                                     MASTER_ID, true));
+  unique_ptr<UnlockBatchRequestBody> unlockBatchRequest(
+      new UnlockBatchRequestBody());
+  vector<MQMessageQueue> mqs;
+  mqs.push_back(mq);
+  unlockBatchRequest->setClientId(m_pConsumer->getMQClientId());
+  unlockBatchRequest->setConsumerGroup(m_pConsumer->getGroupName());
+  unlockBatchRequest->setMqSet(mqs);
+
+  try {
+    m_pClientFactory->getMQClientAPIImpl()->unlockBatchMQ(
+        pFindBrokerResult->brokerAddr, unlockBatchRequest.get(), 1000,
+        m_pConsumer->getSessionCredentials());
+    for (unsigned int i = 0; i != mqs.size(); ++i) {
+      PullRequest* pullreq = getPullRequest(mqs[i]);
+      if (pullreq) {
+        LOG_INFO("unlock success of mq:%s", mqs[i].toString().c_str());
+        pullreq->setLocked(true);
+      } else {
+        LOG_ERROR("unlock fails of mq:%s", mqs[i].toString().c_str());
+      }
+    }
+  } catch (MQException& e) {
+    LOG_ERROR("unlock fails of mq:%s", mq.toString().c_str());
+  }
+}
+
+void Rebalance::lockAll() {
+  map<string, vector<MQMessageQueue>*> brokerMqs;
+  MQ2PULLREQ requestQueueTable = getPullRequestTable();
+  for (MQ2PULLREQ::iterator it = requestQueueTable.begin();
+       it != requestQueueTable.end(); ++it) {
+    if (!(it->second->isDroped())) {
+      string brokerKey = it->first.getBrokerName() + it->first.getTopic();
+      if (brokerMqs.find(brokerKey) == brokerMqs.end()) {
+        vector<MQMessageQueue>* mqs = new vector<MQMessageQueue>;
+        brokerMqs[brokerKey] = mqs;
+        brokerMqs[brokerKey]->push_back(it->first);
+      } else {
+        brokerMqs[brokerKey]->push_back(it->first);
+      }
+    }
+  }
+  LOG_INFO("LockAll %zu broker mqs", brokerMqs.size());
+  for (map<string, vector<MQMessageQueue>*>::iterator itb = brokerMqs.begin();
+       itb != brokerMqs.end(); ++itb) {
+    unique_ptr<FindBrokerResult> pFindBrokerResult(
+        m_pClientFactory->findBrokerAddressInSubscribe(
+            (*(itb->second))[0].getBrokerName(), MASTER_ID, true));
+    unique_ptr<LockBatchRequestBody> lockBatchRequest(
+        new LockBatchRequestBody());
+    lockBatchRequest->setClientId(m_pConsumer->getMQClientId());
+    lockBatchRequest->setConsumerGroup(m_pConsumer->getGroupName());
+    lockBatchRequest->setMqSet(*(itb->second));
+    LOG_INFO("try to lock:%zu mqs of broker:%s", itb->second->size(),
+             itb->first.c_str());
+    try {
+      vector<MQMessageQueue> messageQueues;
+      m_pClientFactory->getMQClientAPIImpl()->lockBatchMQ(
+          pFindBrokerResult->brokerAddr, lockBatchRequest.get(), messageQueues,
+          1000, m_pConsumer->getSessionCredentials());
+      for (unsigned int i = 0; i != messageQueues.size(); ++i) {
+        PullRequest* pullreq = getPullRequest(messageQueues[i]);
+        if (pullreq) {
+          LOG_INFO("lockBatchMQ success of mq:%s",
+                   messageQueues[i].toString().c_str());
+          pullreq->setLocked(true);
+          pullreq->setLastLockTimestamp(UtilAll::currentTimeMillis());
+        } else {
+          LOG_ERROR("lockBatchMQ fails of mq:%s",
+                    messageQueues[i].toString().c_str());
+        }
+      }
+      messageQueues.clear();
+    } catch (MQException& e) {
+      LOG_ERROR("lockBatchMQ fails");
+    }
+    deleteAndZero(itb->second);
+  }
+  brokerMqs.clear();
+}
+bool Rebalance::lock(MQMessageQueue mq) {
+  unique_ptr<FindBrokerResult> pFindBrokerResult(
+      m_pClientFactory->findBrokerAddressInSubscribe(mq.getBrokerName(),
+                                                     MASTER_ID, true));
+  unique_ptr<LockBatchRequestBody> lockBatchRequest(new LockBatchRequestBody());
+  lockBatchRequest->setClientId(m_pConsumer->getMQClientId());
+  lockBatchRequest->setConsumerGroup(m_pConsumer->getGroupName());
+  vector<MQMessageQueue> in_mqSet;
+  in_mqSet.push_back(mq);
+  lockBatchRequest->setMqSet(in_mqSet);
+  bool lockResult = false;
+
+  try {
+    vector<MQMessageQueue> messageQueues;
+    LOG_DEBUG("try to lock mq:%s", mq.toString().c_str());
+    m_pClientFactory->getMQClientAPIImpl()->lockBatchMQ(
+        pFindBrokerResult->brokerAddr, lockBatchRequest.get(), messageQueues,
+        1000, m_pConsumer->getSessionCredentials());
+    if (messageQueues.size() == 0) {
+      LOG_ERROR("lock mq on broker:%s failed",
+                pFindBrokerResult->brokerAddr.c_str());
+      return false;
+    }
+    for (unsigned int i = 0; i != messageQueues.size(); ++i) {
+      PullRequest* pullreq = getPullRequest(messageQueues[i]);
+      if (pullreq) {
+        LOG_INFO("lock success of mq:%s", messageQueues[i].toString().c_str());
+        pullreq->setLocked(true);
+        pullreq->setLastLockTimestamp(UtilAll::currentTimeMillis());
+        lockResult = true;
+      } else {
+        LOG_ERROR("lock fails of mq:%s", messageQueues[i].toString().c_str());
+      }
+    }
+    messageQueues.clear();
+    return lockResult;
+  } catch (MQException& e) {
+    LOG_ERROR("lock fails of mq:%s", mq.toString().c_str());
+    return false;
+  }
+}
+
+//<!************************************************************************
+RebalancePull::RebalancePull(MQConsumer* consumer, MQClientFactory* pfactory)
+    : Rebalance(consumer, pfactory) {}
+
+bool RebalancePull::updateRequestTableInRebalance(
+    const string& topic, vector<MQMessageQueue>& mqsSelf) {
+  return false;
+}
+
+int64 RebalancePull::computePullFromWhere(const MQMessageQueue& mq) {
+  return 0;
+}
+
+void RebalancePull::messageQueueChanged(const string& topic,
+                                        vector<MQMessageQueue>& mqAll,
+                                        vector<MQMessageQueue>& mqDivided) {}
+
+void RebalancePull::removeUnnecessaryMessageQueue(const MQMessageQueue& mq) {}
+
+//<!***************************************************************************
+RebalancePush::RebalancePush(MQConsumer* consumer, MQClientFactory* pfactory)
+    : Rebalance(consumer, pfactory) {}
+
+bool RebalancePush::updateRequestTableInRebalance(
+    const string& topic, vector<MQMessageQueue>& mqsSelf) {
+  LOG_DEBUG("updateRequestTableInRebalance Enter");
+  if (mqsSelf.empty()) {
+    LOG_WARN("allocated queue is empty for topic:%s", topic.c_str());
+  }
+
+  bool changed = false;
+
+  //<!remove
+  MQ2PULLREQ requestQueueTable(getPullRequestTable());
+  MQ2PULLREQ::iterator it = requestQueueTable.begin();
+  for (; it != requestQueueTable.end(); ++it) {
+    MQMessageQueue mqtemp = it->first;
+    if (mqtemp.getTopic().compare(topic) == 0) {
+      if (mqsSelf.empty() ||
+          (find(mqsSelf.begin(), mqsSelf.end(), mqtemp) == mqsSelf.end())) {
+        if (!(it->second->isDroped())) {
+          it->second->setDroped(true);
+          removeUnnecessaryMessageQueue(mqtemp);
+          it->second->clearAllMsgs();  // add clear operation to avoid bad state
+                                       // when dropped pullRequest returns
+                                       // normal
+          LOG_INFO("drop mq:%s", mqtemp.toString().c_str());
+        }
+        changed = true;
+      }
+    }
+  }
+
+  //<!add
+  vector<PullRequest*> pullrequestAdd;
+  DefaultMQPushConsumer* pConsumer =
+      static_cast<DefaultMQPushConsumer*>(m_pConsumer);
+  vector<MQMessageQueue>::iterator it2 = mqsSelf.begin();
+  for (; it2 != mqsSelf.end(); ++it2) {
+    PullRequest* pPullRequest(getPullRequest(*it2));
+    if (pPullRequest && pPullRequest->isDroped()) {
+      LOG_DEBUG(
+          "before resume the pull handle of this pullRequest, its mq is:%s, "
+          "its offset is:%lld",
+          (it2->toString()).c_str(), pPullRequest->getNextOffset());
+      pConsumer->getOffsetStore()->removeOffset(
+          *it2);  // remove dirty offset which maybe update to
+                  // OffsetStore::m_offsetTable by consuming After last
+                  // drop
+      int64 nextOffset = computePullFromWhere(*it2);
+      if (nextOffset >= 0) {
+        pPullRequest->setDroped(false);
+        pPullRequest->clearAllMsgs();  // avoid consume accumulation and consume
+                                       // dumplication issues
+        pPullRequest->setNextOffset(nextOffset);
+        pPullRequest->updateQueueMaxOffset(nextOffset);
+        LOG_INFO(
+            "after resume the pull handle of this pullRequest, its mq is:%s, "
+            "its offset is:%lld",
+            (it2->toString()).c_str(), pPullRequest->getNextOffset());
+        changed = true;
+        pConsumer->producePullMsgTask(pPullRequest);
+      } else {
+        LOG_ERROR(
+            "get fatel error QueryOffset of mq:%s, do not reconsume this queue",
+            (it2->toString()).c_str());
+      }
+    }
+
+    if (!pPullRequest) {
+      LOG_INFO("updateRequestTableInRebalance Doesn't find old mq");
+      PullRequest* pullRequest = new PullRequest(m_pConsumer->getGroupName());
+      pullRequest->m_messageQueue = *it2;
+
+      int64 nextOffset = computePullFromWhere(*it2);
+      if (nextOffset >= 0) {
+        pullRequest->setNextOffset(nextOffset);
+        pullRequest->clearAllMsgs();  // avoid consume accumulation and consume
+                                      // dumplication issues
+        changed = true;
+        //<! mq-> pq;
+        addPullRequest(*it2, pullRequest);
+        pullrequestAdd.push_back(pullRequest);
+        LOG_INFO("add mq:%s, request initiall offset:%lld",
+                 (*it2).toString().c_str(), nextOffset);
+      }
+    }
+  }
+
+  vector<PullRequest*>::iterator it3 = pullrequestAdd.begin();
+  for (; it3 != pullrequestAdd.end(); ++it3) {
+    LOG_DEBUG("start pull request");
+    pConsumer->producePullMsgTask(*it3);
+  }
+
+  LOG_DEBUG("updateRequestTableInRebalance exit");
+  return changed;
+}
+
+int64 RebalancePush::computePullFromWhere(const MQMessageQueue& mq) {
+  int64 result = -1;
+  DefaultMQPushConsumer* pConsumer =
+      static_cast<DefaultMQPushConsumer*>(m_pConsumer);
+  ConsumeFromWhere consumeFromWhere = pConsumer->getConsumeFromWhere();
+  OffsetStore* pOffsetStore = pConsumer->getOffsetStore();
+  switch (consumeFromWhere) {
+    case CONSUME_FROM_LAST_OFFSET: {
+      int64 lastOffset = pOffsetStore->readOffset(
+          mq, READ_FROM_STORE, m_pConsumer->getSessionCredentials());
+      if (lastOffset >= 0) {
+        LOG_INFO("CONSUME_FROM_LAST_OFFSET, lastOffset of mq:%s is:%lld",
+                 mq.toString().c_str(), lastOffset);
+        result = lastOffset;
+      }
+      else if (-1 == lastOffset) {
+        LOG_WARN("CONSUME_FROM_LAST_OFFSET, lastOffset of mq:%s is -1",
+                 mq.toString().c_str());
+        if (UtilAll::startsWith_retry(mq.getTopic())) {
+          LOG_INFO("CONSUME_FROM_LAST_OFFSET, lastOffset of mq:%s is 0",
+                   mq.toString().c_str());
+          result = 0;
+        }
+        else {
+          try {
+            result = pConsumer->maxOffset(mq);
+            LOG_INFO("CONSUME_FROM_LAST_OFFSET, maxOffset of mq:%s is:%lld",
+                     mq.toString().c_str(), result);
+          } catch (MQException& e) {
+            LOG_ERROR(
+                "CONSUME_FROM_LAST_OFFSET error, lastOffset  of mq:%s is -1",
+                mq.toString().c_str());
+            result = -1;
+          }
+        }
+      }
+      else {
+        LOG_ERROR("CONSUME_FROM_LAST_OFFSET error, lastOffset  of mq:%s is -1",
+                  mq.toString().c_str());
+        result = -1;
+      }
+      break;
+    }
+    case CONSUME_FROM_FIRST_OFFSET: {
+      int64 lastOffset = pOffsetStore->readOffset(
+          mq, READ_FROM_STORE, m_pConsumer->getSessionCredentials());
+      if (lastOffset >= 0) {
+        LOG_INFO("CONSUME_FROM_FIRST_OFFSET, lastOffset of mq:%s is:%lld",
+                 mq.toString().c_str(), lastOffset);
+        result = lastOffset;
+      } else if (-1 == lastOffset)
+      {
+        LOG_INFO("CONSUME_FROM_FIRST_OFFSET, lastOffset of mq:%s, return 0",
+                 mq.toString().c_str());
+        result = 0;
+      }
+      else {
+        LOG_ERROR("CONSUME_FROM_FIRST_OFFSET, lastOffset of mq:%s, return -1",
+                  mq.toString().c_str());
+        result = -1;
+      }
+      break;
+    }
+    case CONSUME_FROM_TIMESTAMP: {
+      int64 lastOffset = pOffsetStore->readOffset(
+          mq, READ_FROM_STORE, m_pConsumer->getSessionCredentials());
+      if (lastOffset >= 0) {
+        LOG_INFO("CONSUME_FROM_TIMESTAMP, lastOffset of mq:%s is:%lld",
+                 mq.toString().c_str(), lastOffset);
+        result = lastOffset;
+      }
+      else if (-1 == lastOffset) {
+        if (UtilAll::startsWith_retry(mq.getTopic())) {
+          try {
+            result = pConsumer->maxOffset(mq);
+            LOG_INFO("CONSUME_FROM_TIMESTAMP, maxOffset  of mq:%s is:%lld",
+                     mq.toString().c_str(), result);
+          } catch (MQException& e) {
+            LOG_ERROR(
+                "CONSUME_FROM_TIMESTAMP error, lastOffset  of mq:%s is -1",
+                mq.toString().c_str());
+            result = -1;
+          }
+        }
+        else {
+          try {
+          } catch (MQException& e) {
+            LOG_ERROR(
+                "CONSUME_FROM_TIMESTAMP error, lastOffset  of mq:%s, return 0",
+                mq.toString().c_str());
+            result = -1;
+          }
+        }
+      }
+      else {
+        LOG_ERROR(
+            "CONSUME_FROM_TIMESTAMP error, lastOffset  of mq:%s, return -1",
+            mq.toString().c_str());
+        result = -1;
+      }
+      break;
+    }
+    default:
+      break;
+  }
+  return result;
+}
+
+void RebalancePush::messageQueueChanged(const string& topic,
+                                        vector<MQMessageQueue>& mqAll,
+                                        vector<MQMessageQueue>& mqDivided) {}
+
+void RebalancePush::removeUnnecessaryMessageQueue(const MQMessageQueue& mq) {
+  DefaultMQPushConsumer* pConsumer =
+      static_cast<DefaultMQPushConsumer*>(m_pConsumer);
+  OffsetStore* pOffsetStore = pConsumer->getOffsetStore();
+
+  pOffsetStore->persist(mq, m_pConsumer->getSessionCredentials());
+  pOffsetStore->removeOffset(mq);
+  if (pConsumer->getMessageListenerType() == messageListenerOrderly) {
+    unlock(mq);
+  }
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/Rebalance.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/Rebalance.h b/rocketmq-cpp/src/consumer/Rebalance.h
new file mode 100755
index 0000000..42f8667
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/Rebalance.h
@@ -0,0 +1,123 @@
+/*
+ * 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 __REBALANCEIMPL_H__
+#define __REBALANCEIMPL_H__
+
+#include "AllocateMQStrategy.h"
+#include "ConsumeType.h"
+#include "MQConsumer.h"
+#include "MQMessageQueue.h"
+#include "PullRequest.h"
+#include "SubscriptionData.h"
+
+#include <boost/thread/mutex.hpp>
+
+namespace rocketmq {
+class MQClientFactory;
+//<!************************************************************************
+class Rebalance {
+ public:
+  Rebalance(MQConsumer*, MQClientFactory*);
+  virtual ~Rebalance();
+
+  virtual void messageQueueChanged(const string& topic,
+                                   vector<MQMessageQueue>& mqAll,
+                                   vector<MQMessageQueue>& mqDivided) = 0;
+
+  virtual void removeUnnecessaryMessageQueue(const MQMessageQueue& mq) = 0;
+
+  virtual int64 computePullFromWhere(const MQMessageQueue& mq) = 0;
+
+  virtual bool updateRequestTableInRebalance(
+      const string& topic, vector<MQMessageQueue>& mqsSelf) = 0;
+
+ public:
+  void doRebalance();
+  void persistConsumerOffset();
+  void persistConsumerOffsetByResetOffset();
+  //<!m_subscriptionInner;
+  SubscriptionData* getSubscriptionData(const string& topic);
+  void setSubscriptionData(const string& topic, SubscriptionData* pdata);
+
+  map<string, SubscriptionData*>& getSubscriptionInner();
+
+  //<!m_topicSubscribeInfoTable;
+  void setTopicSubscribeInfo(const string& topic, vector<MQMessageQueue>& mqs);
+  bool getTopicSubscribeInfo(const string& topic, vector<MQMessageQueue>& mqs);
+
+  void addPullRequest(MQMessageQueue mq, PullRequest* pPullRequest);
+  PullRequest* getPullRequest(MQMessageQueue mq);
+  map<MQMessageQueue, PullRequest*> getPullRequestTable();
+  void lockAll();
+  bool lock(MQMessageQueue mq);
+  void unlockAll(bool oneway = false);
+  void unlock(MQMessageQueue mq);
+
+ protected:
+  map<string, SubscriptionData*> m_subscriptionData;
+
+  boost::mutex m_topicSubscribeInfoTableMutex;
+  map<string, vector<MQMessageQueue>> m_topicSubscribeInfoTable;
+  typedef map<MQMessageQueue, PullRequest*> MQ2PULLREQ;
+  MQ2PULLREQ m_requestQueueTable;
+  boost::mutex m_requestTableMutex;
+
+  AllocateMQStrategy* m_pAllocateMQStrategy;
+  MQConsumer* m_pConsumer;
+  MQClientFactory* m_pClientFactory;
+};
+
+//<!************************************************************************
+class RebalancePull : public Rebalance {
+ public:
+  RebalancePull(MQConsumer*, MQClientFactory*);
+  virtual ~RebalancePull(){};
+
+  virtual void messageQueueChanged(const string& topic,
+                                   vector<MQMessageQueue>& mqAll,
+                                   vector<MQMessageQueue>& mqDivided);
+
+  virtual void removeUnnecessaryMessageQueue(const MQMessageQueue& mq);
+
+  virtual int64 computePullFromWhere(const MQMessageQueue& mq);
+
+  virtual bool updateRequestTableInRebalance(const string& topic,
+                                             vector<MQMessageQueue>& mqsSelf);
+};
+
+//<!***************************************************************************
+class RebalancePush : public Rebalance {
+ public:
+  RebalancePush(MQConsumer*, MQClientFactory*);
+  virtual ~RebalancePush(){};
+
+  virtual void messageQueueChanged(const string& topic,
+                                   vector<MQMessageQueue>& mqAll,
+                                   vector<MQMessageQueue>& mqDivided);
+
+  virtual void removeUnnecessaryMessageQueue(const MQMessageQueue& mq);
+
+  virtual int64 computePullFromWhere(const MQMessageQueue& mq);
+
+  virtual bool updateRequestTableInRebalance(const string& topic,
+                                             vector<MQMessageQueue>& mqsSelf);
+};
+
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/SubscriptionData.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/SubscriptionData.cpp b/rocketmq-cpp/src/consumer/SubscriptionData.cpp
new file mode 100755
index 0000000..9b20642
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/SubscriptionData.cpp
@@ -0,0 +1,120 @@
+/*
+ * 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 "SubscriptionData.h"
+#include <algorithm>
+#include <sstream>
+#include <vector>
+#include "UtilAll.h"
+#include "Logging.h"
+namespace rocketmq {
+//<!************************************************************************
+SubscriptionData::SubscriptionData() {
+  m_subVersion = UtilAll::currentTimeMillis();
+}
+
+SubscriptionData::SubscriptionData(const string& topic, const string& subString)
+    : m_topic(topic), m_subString(subString) {
+  m_subVersion = UtilAll::currentTimeMillis();
+}
+
+SubscriptionData::SubscriptionData(const SubscriptionData& other) {
+  m_subString = other.m_subString;
+  m_subVersion = other.m_subVersion;
+  m_tagSet = other.m_tagSet;
+  m_topic = other.m_topic;
+  m_codeSet = other.m_codeSet;
+}
+
+const string& SubscriptionData::getTopic() const { return m_topic; }
+
+const string& SubscriptionData::getSubString() const { return m_subString; }
+
+void SubscriptionData::setSubString(const string& sub) { m_subString = sub; }
+
+int64 SubscriptionData::getSubVersion() const { return m_subVersion; }
+
+void SubscriptionData::putTagsSet(const string& tag) {
+  m_tagSet.push_back(tag);
+}
+
+bool SubscriptionData::containTag(const string& tag) {
+  return std::find(m_tagSet.begin(), m_tagSet.end(), tag) != m_tagSet.end();
+}
+
+vector<string>& SubscriptionData::getTagsSet() { return m_tagSet; }
+
+bool SubscriptionData::operator==(const SubscriptionData& other) const {
+  if (!m_subString.compare(other.m_subString)) {
+    return false;
+  }
+  if (m_subVersion != other.m_subVersion) {
+    return false;
+  }
+  if (m_tagSet.size() != other.m_tagSet.size()) {
+    return false;
+  }
+  if (!m_topic.compare(other.m_topic)) {
+    return false;
+  }
+  return true;
+}
+
+bool SubscriptionData::operator<(const SubscriptionData& other) const {
+  int ret = m_topic.compare(other.m_topic);
+  if (ret < 0) {
+    return true;
+  } else if (ret == 0) {
+    ret = m_subString.compare(other.m_subString);
+    if (ret < 0) {
+      return true;
+    } else {
+      return false;
+    }
+  } else {
+    return false;
+  }
+}
+
+void SubscriptionData::putCodeSet(const string& tag) {
+  int value = atoi(tag.c_str());
+  m_codeSet.push_back(value);
+}
+
+Json::Value SubscriptionData::toJson() const {
+  Json::Value outJson;
+  outJson["subString"] = m_subString;
+  outJson["subVersion"] = UtilAll::to_string(m_subVersion);
+  outJson["topic"] = m_topic;
+
+  {
+    vector<string>::const_iterator it = m_tagSet.begin();
+    for (; it != m_tagSet.end(); it++) {
+      outJson["tagsSet"].append(*it);
+    }
+  }
+
+  {
+    vector<int>::const_iterator it = m_codeSet.begin();
+    for (; it != m_codeSet.end(); it++) {
+      outJson["codeSet"].append(*it);
+    }
+  }
+  return outJson;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/SubscriptionData.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/SubscriptionData.h b/rocketmq-cpp/src/consumer/SubscriptionData.h
new file mode 100755
index 0000000..89be74f
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/SubscriptionData.h
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __SUBSCRIPTIONDATA_H__
+#define __SUBSCRIPTIONDATA_H__
+
+#include <string>
+#include "UtilAll.h"
+#include "json/json.h"
+
+namespace rocketmq {
+//<!************************************************************************
+class SubscriptionData {
+ public:
+  SubscriptionData();
+  virtual ~SubscriptionData() {
+    m_tagSet.clear();
+    m_codeSet.clear();
+  }
+  SubscriptionData(const string& topic, const string& subString);
+  SubscriptionData(const SubscriptionData& other);
+
+  const string& getTopic() const;
+  const string& getSubString() const;
+  void setSubString(const string& sub);
+  int64 getSubVersion() const;
+
+  void putTagsSet(const string& tag);
+  bool containTag(const string& tag);
+  vector<string>& getTagsSet();
+
+  void putCodeSet(const string& tag);
+
+  bool operator==(const SubscriptionData& other) const;
+  bool operator<(const SubscriptionData& other) const;
+
+  Json::Value toJson() const;
+
+ private:
+  string m_topic;
+  string m_subString;
+  int64 m_subVersion;
+  vector<string> m_tagSet;
+  vector<int> m_codeSet;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/dllmain.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/dllmain.cpp b/rocketmq-cpp/src/dllmain.cpp
new file mode 100755
index 0000000..72f61fd
--- /dev/null
+++ b/rocketmq-cpp/src/dllmain.cpp
@@ -0,0 +1,34 @@
+/*
+ * 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 <stdio.h>
+#include "windows.h"
+
+BOOL APIENTRY DllMain(HMODULE hModule, DWORD ul_reason_for_call,
+                      LPVOID lpReserved) {
+  switch (ul_reason_for_call) {
+    case DLL_PROCESS_ATTACH:
+      break;
+    case DLL_THREAD_ATTACH:
+      break;
+    case DLL_THREAD_DETACH:
+      break;
+    case DLL_PROCESS_DETACH:
+      break;
+  }
+  return TRUE;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/log/Logging.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/log/Logging.cpp b/rocketmq-cpp/src/log/Logging.cpp
new file mode 100644
index 0000000..9ac812f
--- /dev/null
+++ b/rocketmq-cpp/src/log/Logging.cpp
@@ -0,0 +1,96 @@
+#include "Logging.h"
+#include <boost/date_time/gregorian/gregorian.hpp>
+#include "UtilAll.h"
+#define BOOST_DATE_TIME_SOURCE
+
+namespace rocketmq {
+
+logAdapter::~logAdapter() { logging::core::get()->remove_all_sinks(); }
+
+logAdapter& logAdapter::getLogInstance() {
+  static logAdapter alogInstance;
+  return alogInstance;
+}
+
+logAdapter::logAdapter() : m_logLevel(eLOG_LEVEL_INFO) {
+  string homeDir(UtilAll::getHomeDirectory());
+  homeDir.append("/logs/metaq-client4cpp/");
+  m_logFile += homeDir;
+  std::string fileName =
+      UtilAll::to_string(getpid()) + "_" + "rocketmq-cpp.log.%N";
+  m_logFile += fileName;
+
+  // boost::log::expressions::attr<
+  // boost::log::attributes::current_thread_id::value_type>("ThreadID");
+  boost::log::register_simple_formatter_factory<
+      boost::log::trivial::severity_level, char>("Severity");
+  m_logSink = logging::add_file_log(
+      keywords::file_name = m_logFile,
+      keywords::rotation_size = 10 * 1024 * 1024,
+      keywords::time_based_rotation =
+          sinks::file::rotation_at_time_point(0, 0, 0),
+      keywords::format = "[%TimeStamp%](%Severity%):%Message%",
+      keywords::min_free_space = 300 * 1024 * 1024, keywords::target = homeDir,
+      keywords::max_size = 20 * 1024 * 1024,  // max keep 3 log file defaultly
+      keywords::auto_flush = true);
+  logging::core::get()->set_filter(logging::trivial::severity >=
+                                   logging::trivial::info);
+
+  logging::add_common_attributes();
+}
+
+void logAdapter::setLogLevel(elogLevel logLevel) {
+  m_logLevel = logLevel;
+  switch (logLevel) {
+    case eLOG_LEVEL_DISABLE:
+      logging::core::get()->set_filter(logging::trivial::severity >
+                                       logging::trivial::fatal);
+
+      break;
+    case eLOG_LEVEL_FATAL:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::fatal);
+      break;
+    case eLOG_LEVEL_ERROR:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::error);
+
+      break;
+    case eLOG_LEVEL_WARN:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::warning);
+
+      break;
+    case eLOG_LEVEL_INFO:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::info);
+
+      break;
+    case eLOG_LEVEL_DEBUG:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::debug);
+
+      break;
+    case eLOG_LEVEL_TRACE:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::trace);
+
+      break;
+    default:
+      logging::core::get()->set_filter(logging::trivial::severity >=
+                                       logging::trivial::info);
+
+      break;
+  }
+}
+
+elogLevel logAdapter::getLogLevel() { return m_logLevel; }
+
+void logAdapter::setLogFileNumAndSize(int logNum, int sizeOfPerFile) {
+  string homeDir(UtilAll::getHomeDirectory());
+  homeDir.append("/logs/metaq-client4cpp/");
+  m_logSink->locked_backend()->set_file_collector(sinks::file::make_collector(
+      keywords::target = homeDir,
+      keywords::max_size = logNum * sizeOfPerFile * 1024 * 1024));
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/log/Logging.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/log/Logging.h b/rocketmq-cpp/src/log/Logging.h
new file mode 100644
index 0000000..98e9659
--- /dev/null
+++ b/rocketmq-cpp/src/log/Logging.h
@@ -0,0 +1,75 @@
+#ifndef _ALOG_ADAPTER_H_
+#define _ALOG_ADAPTER_H_
+
+#include <boost/date_time/posix_time/posix_time_types.hpp>
+#include <boost/log/core.hpp>
+#include <boost/log/expressions.hpp>
+#include <boost/log/sinks/text_file_backend.hpp>
+#include <boost/log/sources/record_ostream.hpp>
+#include <boost/log/sources/severity_logger.hpp>
+#include <boost/log/trivial.hpp>
+#include <boost/log/utility/manipulators/add_value.hpp>
+#include <boost/log/utility/setup/common_attributes.hpp>
+#include <boost/log/utility/setup/file.hpp>
+#include <boost/scoped_array.hpp>
+#include <boost/shared_ptr.hpp>
+#include "MQClient.h"
+
+namespace logging = boost::log;
+namespace src = boost::log::sources;
+namespace sinks = boost::log::sinks;
+namespace expr = boost::log::expressions;
+namespace keywords = boost::log::keywords;
+using namespace boost::log::trivial;
+namespace rocketmq {
+
+class logAdapter {
+ public:
+  ~logAdapter();
+  static logAdapter& getLogInstance();
+  void setLogLevel(elogLevel logLevel);
+  elogLevel getLogLevel();
+  void setLogFileNumAndSize(int logNum, int sizeOfPerFile);
+  src::severity_logger<boost::log::trivial::severity_level>&
+  getSeverityLogger() {
+    return m_severityLogger;
+  }
+
+ private:
+  logAdapter();
+  elogLevel m_logLevel;
+  std::string m_logFile;
+  src::severity_logger<boost::log::trivial::severity_level> m_severityLogger;
+  typedef sinks::synchronous_sink<sinks::text_file_backend> logSink_t;
+  boost::shared_ptr<logSink_t> m_logSink;
+};
+
+#define ALOG_ADAPTER logAdapter::getLogInstance()
+
+#define AGENT_LOGGER ALOG_ADAPTER.getSeverityLogger()
+
+class LogUtil {
+ public:
+  static void VLogError(boost::log::trivial::severity_level level,
+                        const char* format, ...) {
+    va_list arg_ptr;
+    va_start(arg_ptr, format);
+    boost::scoped_array<char> formattedString(new char[1024]);
+    vsnprintf(formattedString.get(), 1024, format, arg_ptr);
+    BOOST_LOG_SEV(AGENT_LOGGER, level) << formattedString.get();
+    va_end(arg_ptr);
+  }
+};
+
+#define LOG_FATAL(format, args...) \
+  LogUtil::VLogError(boost::log::trivial::fatal, format, ##args)
+#define LOG_ERROR(format, args...) \
+  LogUtil::VLogError(boost::log::trivial::error, format, ##args)
+#define LOG_WARN(format, args...) \
+  LogUtil::VLogError(boost::log::trivial::warning, format, ##args)
+#define LOG_INFO(format, args...) \
+  LogUtil::VLogError(boost::log::trivial::info, format, ##args)
+#define LOG_DEBUG(format, args...) \
+  LogUtil::VLogError(boost::log::trivial::debug, format, ##args)
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQDecoder.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQDecoder.cpp b/rocketmq-cpp/src/message/MQDecoder.cpp
new file mode 100755
index 0000000..4dde1f5
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQDecoder.cpp
@@ -0,0 +1,254 @@
+/*
+ * 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 "MQDecoder.h"
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sstream>
+#include "Logging.h"
+#include "MemoryOutputStream.h"
+#include "MessageSysFlag.h"
+#include "UtilAll.h"
+namespace rocketmq {
+//<!***************************************************************************
+const int MQDecoder::MSG_ID_LENGTH = 8 + 8;
+
+const char MQDecoder::NAME_VALUE_SEPARATOR = 1;
+const char MQDecoder::PROPERTY_SEPARATOR = 2;
+
+int MQDecoder::MessageMagicCodePostion = 4;
+int MQDecoder::MessageFlagPostion = 16;
+int MQDecoder::MessagePhysicOffsetPostion = 28;
+int MQDecoder::MessageStoreTimestampPostion = 56;
+//<!***************************************************************************
+string MQDecoder::createMessageId(sockaddr addr, int64 offset) {
+  int host, port;
+  socketAddress2IPPort(addr, host, port);
+
+  MemoryOutputStream outputmen(MSG_ID_LENGTH);
+  outputmen.writeIntBigEndian(host);
+  outputmen.writeIntBigEndian(port);
+  outputmen.writeInt64BigEndian(offset);
+
+  const char* bytes = static_cast<const char*>(outputmen.getData());
+  int len = outputmen.getDataSize();
+
+  return UtilAll::bytes2string(bytes, len);
+}
+
+MQMessageId MQDecoder::decodeMessageId(const string& msgId) {
+
+  string ipstr = msgId.substr(0, 8);
+  string portstr = msgId.substr(8, 8);
+  string offsetstr = msgId.substr(16);
+
+  char* end;
+  int ipint = strtoul(ipstr.c_str(), &end, 16);
+  int portint = strtoul(portstr.c_str(), &end, 16);
+
+  int64 offset = UtilAll::hexstr2ull(offsetstr.c_str());
+
+  offset = n2hll(offset);
+
+  portint = ntohl(portint);
+  short port = portint;
+
+  struct sockaddr_in sa;
+  sa.sin_family = AF_INET;
+  sa.sin_port = htons(port);
+  sa.sin_addr.s_addr = ipint;
+
+  sockaddr addr;
+  memcpy(&addr, &sa, sizeof(sockaddr));
+
+  MQMessageId id(addr, offset);
+
+  return id;
+}
+
+MQMessageExt* MQDecoder::decode(MemoryInputStream& byteBuffer) {
+  return decode(byteBuffer, true);
+}
+
+MQMessageExt* MQDecoder::decode(MemoryInputStream& byteBuffer, bool readBody) {
+  MQMessageExt* msgExt = new MQMessageExt();
+
+  // 1 TOTALSIZE
+  int storeSize = byteBuffer.readIntBigEndian();
+  msgExt->setStoreSize(storeSize);
+
+  // 2 MAGICCODE sizeof(int)
+  byteBuffer.skipNextBytes(sizeof(int));
+
+  // 3 BODYCRC
+  int bodyCRC = byteBuffer.readIntBigEndian();
+  msgExt->setBodyCRC(bodyCRC);
+
+  // 4 QUEUEID
+  int queueId = byteBuffer.readIntBigEndian();
+  msgExt->setQueueId(queueId);
+
+  // 5 FLAG
+  int flag = byteBuffer.readIntBigEndian();
+  msgExt->setFlag(flag);
+
+  // 6 QUEUEOFFSET
+  int64 queueOffset = byteBuffer.readInt64BigEndian();
+  msgExt->setQueueOffset(queueOffset);
+
+  // 7 PHYSICALOFFSET
+  int64 physicOffset = byteBuffer.readInt64BigEndian();
+  msgExt->setCommitLogOffset(physicOffset);
+
+  // 8 SYSFLAG
+  int sysFlag = byteBuffer.readIntBigEndian();
+  msgExt->setSysFlag(sysFlag);
+
+  // 9 BORNTIMESTAMP
+  int64 bornTimeStamp = byteBuffer.readInt64BigEndian();
+  msgExt->setBornTimestamp(bornTimeStamp);
+
+  // 10 BORNHOST
+  int bornHost = byteBuffer.readIntBigEndian();
+  int port = byteBuffer.readIntBigEndian();
+  sockaddr bornAddr = IPPort2socketAddress(bornHost, port);
+  msgExt->setBornHost(bornAddr);
+
+  // 11 STORETIMESTAMP
+  int64 storeTimestamp = byteBuffer.readInt64BigEndian();
+  msgExt->setStoreTimestamp(storeTimestamp);
+
+  // // 12 STOREHOST
+  int storeHost = byteBuffer.readIntBigEndian();
+  port = byteBuffer.readIntBigEndian();
+  sockaddr storeAddr = IPPort2socketAddress(storeHost, port);
+  msgExt->setStoreHost(storeAddr);
+
+  // 13 RECONSUMETIMES
+  int reconsumeTimes = byteBuffer.readIntBigEndian();
+  msgExt->setReconsumeTimes(reconsumeTimes);
+
+  // 14 Prepared Transaction Offset
+  int64 preparedTransactionOffset = byteBuffer.readInt64BigEndian();
+  msgExt->setPreparedTransactionOffset(preparedTransactionOffset);
+
+  // 15 BODY
+  int bodyLen = byteBuffer.readIntBigEndian();
+  if (bodyLen > 0) {
+    if (readBody) {
+      MemoryBlock block;
+      byteBuffer.readIntoMemoryBlock(block, bodyLen);
+
+      const char* const pBody = static_cast<const char*>(block.getData());
+      int len = block.getSize();
+      string msgbody(pBody, len);
+
+      // decompress body
+      if ((sysFlag & MessageSysFlag::CompressedFlag) ==
+          MessageSysFlag::CompressedFlag) {
+        string outbody;
+        if (UtilAll::inflate(msgbody, outbody)) {
+          msgExt->setBody(outbody);
+        }
+      } else {
+        msgExt->setBody(msgbody);
+      }
+    } else {
+      byteBuffer.skipNextBytes(bodyLen);
+    }
+  }
+
+  // 16 TOPIC
+  int topicLen = (int)byteBuffer.readByte();
+  MemoryBlock block;
+  byteBuffer.readIntoMemoryBlock(block, topicLen);
+  const char* const pTopic = static_cast<const char*>(block.getData());
+  topicLen = block.getSize();
+  msgExt->setTopic(pTopic, topicLen);
+
+  // 17 properties
+  short propertiesLen = byteBuffer.readShortBigEndian();
+  if (propertiesLen > 0) {
+    MemoryBlock block;
+    byteBuffer.readIntoMemoryBlock(block, propertiesLen);
+    const char* const pProperty = static_cast<const char*>(block.getData());
+    int len = block.getSize();
+    string propertiesString(pProperty, len);
+
+    map<string, string> propertiesMap;
+    string2messageProperties(propertiesString, propertiesMap);
+    msgExt->setProperties(propertiesMap);
+    propertiesMap.clear();
+  }
+
+  // 18 msg ID
+  string msgId = createMessageId(msgExt->getStoreHost(),
+                                 (int64)msgExt->getCommitLogOffset());
+  msgExt->setMsgId(msgId);
+
+  // LOG_INFO("get msgExt from remote server, its contents
+  // are:%s",msgExt->toString().c_str());
+  return msgExt;
+}
+
+void MQDecoder::decodes(const MemoryBlock* mem, vector<MQMessageExt>& mqvec) {
+  mqvec.clear();
+  decodes(mem, mqvec, true);
+}
+
+void MQDecoder::decodes(const MemoryBlock* mem, vector<MQMessageExt>& mqvec,
+                        bool readBody) {
+  MemoryInputStream rawInput(*mem, true);
+
+  while (rawInput.getNumBytesRemaining() > 0) {
+    unique_ptr<MQMessageExt> msg(decode(rawInput, readBody));
+    mqvec.push_back(*msg);
+  }
+}
+
+string MQDecoder::messageProperties2String(
+    const map<string, string>& properties) {
+  string os;
+  map<string, string>::const_iterator it = properties.begin();
+
+  for (; it != properties.end(); ++it) {
+    // os << it->first << NAME_VALUE_SEPARATOR << it->second <<
+    // PROPERTY_SEPARATOR;
+    os.append(it->first);
+    os += NAME_VALUE_SEPARATOR;
+    os.append(it->second);
+    os += PROPERTY_SEPARATOR;
+  }
+
+  return os;
+}
+
+void MQDecoder::string2messageProperties(const string& propertiesString,
+                                         map<string, string>& properties) {
+  vector<string> out;
+  UtilAll::Split(out, propertiesString, PROPERTY_SEPARATOR);
+
+  for (size_t i = 0; i < out.size(); i++) {
+    vector<string> outValue;
+    UtilAll::Split(outValue, out[i], NAME_VALUE_SEPARATOR);
+
+    if (outValue.size() == 2) {
+      properties[outValue[0]] = outValue[1];
+    }
+  }
+}
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQDecoder.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQDecoder.h b/rocketmq-cpp/src/message/MQDecoder.h
new file mode 100755
index 0000000..393e4c7
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQDecoder.h
@@ -0,0 +1,57 @@
+/*
+ * 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 __MESSAGEDECODER_H__
+#define __MESSAGEDECODER_H__
+
+#include "MQClientException.h"
+#include "MQMessageExt.h"
+#include "MQMessageId.h"
+#include "MemoryInputStream.h"
+#include "SocketUtil.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class MQDecoder {
+ public:
+  static string createMessageId(sockaddr addr, int64 offset);
+  static MQMessageId decodeMessageId(const string& msgId);
+
+  static void decodes(const MemoryBlock* mem, vector<MQMessageExt>& mqvec);
+
+  static void decodes(const MemoryBlock* mem, vector<MQMessageExt>& mqvec,
+                      bool readBody);
+
+  static string messageProperties2String(const map<string, string>& properties);
+  static void string2messageProperties(const string& propertiesString,
+                                       map<string, string>& properties);
+
+ private:
+  static MQMessageExt* decode(MemoryInputStream& byteBuffer);
+  static MQMessageExt* decode(MemoryInputStream& byteBuffer, bool readBody);
+
+ public:
+  static const char NAME_VALUE_SEPARATOR;
+  static const char PROPERTY_SEPARATOR;
+  static const int MSG_ID_LENGTH;
+  static int MessageMagicCodePostion;
+  static int MessageFlagPostion;
+  static int MessagePhysicOffsetPostion;
+  static int MessageStoreTimestampPostion;
+};
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQMessage.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQMessage.cpp b/rocketmq-cpp/src/message/MQMessage.cpp
new file mode 100755
index 0000000..db5487f
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQMessage.cpp
@@ -0,0 +1,197 @@
+/*
+ * 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 "MQMessage.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+
+const string MQMessage::PROPERTY_KEYS = "KEYS";
+const string MQMessage::PROPERTY_TAGS = "TAGS";
+const string MQMessage::PROPERTY_WAIT_STORE_MSG_OK = "WAIT";
+const string MQMessage::PROPERTY_DELAY_TIME_LEVEL = "DELAY";
+const string MQMessage::PROPERTY_RETRY_TOPIC = "RETRY_TOPIC";
+const string MQMessage::PROPERTY_REAL_TOPIC = "REAL_TOPIC";
+const string MQMessage::PROPERTY_REAL_QUEUE_ID = "REAL_QID";
+const string MQMessage::PROPERTY_TRANSACTION_PREPARED = "TRAN_MSG";
+const string MQMessage::PROPERTY_PRODUCER_GROUP = "PGROUP";
+const string MQMessage::PROPERTY_MIN_OFFSET = "MIN_OFFSET";
+const string MQMessage::PROPERTY_MAX_OFFSET = "MAX_OFFSET";
+const string MQMessage::KEY_SEPARATOR = " ";
+//<!************************************************************************
+MQMessage::MQMessage() { Init("", "", "", 0, "", true); }
+
+MQMessage::MQMessage(const string& topic, const string& body) {
+  Init(topic, "", "", 0, body, true);
+}
+
+MQMessage::MQMessage(const string& topic, const string& tags,
+                     const string& body) {
+  Init(topic, tags, "", 0, body, true);
+}
+
+MQMessage::MQMessage(const string& topic, const string& tags,
+                     const string& keys, const string& body) {
+  Init(topic, tags, keys, 0, body, true);
+}
+
+MQMessage::MQMessage(const string& topic, const string& tags,
+                     const string& keys, const int flag, const string& body,
+                     bool waitStoreMsgOK) {
+  Init(topic, tags, keys, flag, body, waitStoreMsgOK);
+}
+
+MQMessage::~MQMessage() { m_properties.clear(); }
+
+MQMessage::MQMessage(const MQMessage& other) {
+  m_body = other.m_body;
+  m_topic = other.m_topic;
+  m_flag = other.m_flag;
+  m_properties = other.m_properties;
+}
+
+MQMessage& MQMessage::operator=(const MQMessage& other) {
+  if (this != &other) {
+    m_body = other.m_body;
+    m_topic = other.m_topic;
+    m_flag = other.m_flag;
+    m_properties = other.m_properties;
+  }
+  return *this;
+}
+
+void MQMessage::setProperty(const string& name, const string& value) {
+  m_properties[name] = value;
+}
+
+string MQMessage::getProperty(const string& name) const {
+  map<string, string>::const_iterator it = m_properties.find(name);
+
+  return (it == m_properties.end()) ? "" : (*it).second;
+}
+
+string MQMessage::getTopic() const { return m_topic; }
+
+void MQMessage::setTopic(const string& topic) { m_topic = topic; }
+
+void MQMessage::setTopic(const char* body, int len) {
+  m_topic.clear();
+  m_topic.append(body, len);
+}
+
+string MQMessage::getTags() const { return getProperty(PROPERTY_TAGS); }
+
+void MQMessage::setTags(const string& tags) {
+  setProperty(PROPERTY_TAGS, tags);
+}
+
+string MQMessage::getKeys() const { return getProperty(PROPERTY_KEYS); }
+
+void MQMessage::setKeys(const string& keys) {
+  setProperty(PROPERTY_KEYS, keys);
+}
+
+void MQMessage::setKeys(const vector<string>& keys) {
+  if (keys.empty()) {
+    return;
+  }
+
+  vector<string>::const_iterator it = keys.begin();
+  string str;
+  str += *it;
+  it++;
+
+  for (; it != keys.end(); it++) {
+    str += KEY_SEPARATOR;
+    str += *it;
+  }
+
+  setKeys(str);
+}
+
+int MQMessage::getDelayTimeLevel() const {
+  string tmp = getProperty(PROPERTY_DELAY_TIME_LEVEL);
+  if (!tmp.empty()) {
+    return atoi(tmp.c_str());
+  }
+  return 0;
+}
+
+void MQMessage::setDelayTimeLevel(int level) {
+  char tmp[16];
+  sprintf(tmp, "%d", level);
+
+  setProperty(PROPERTY_DELAY_TIME_LEVEL, tmp);
+}
+
+bool MQMessage::isWaitStoreMsgOK() {
+  string tmp = getProperty(PROPERTY_WAIT_STORE_MSG_OK);
+  if (tmp.empty()) {
+    return true;
+  } else {
+    return (tmp == "true") ? true : false;
+  }
+}
+
+void MQMessage::setWaitStoreMsgOK(bool waitStoreMsgOK) {
+  if (waitStoreMsgOK) {
+    setProperty(PROPERTY_WAIT_STORE_MSG_OK, "true");
+  } else {
+    setProperty(PROPERTY_WAIT_STORE_MSG_OK, "false");
+  }
+}
+
+int MQMessage::getFlag() const { return m_flag; }
+
+void MQMessage::setFlag(int flag) { m_flag = flag; }
+
+string MQMessage::getBody() const { return m_body; }
+
+void MQMessage::setBody(const char* body, int len) {
+  m_body.clear();
+  m_body.append(body, len);
+}
+
+void MQMessage::setBody(const string &body) {
+  m_body.clear();
+  m_body.append(body);
+}
+
+map<string, string> MQMessage::getProperties() const { return m_properties; }
+
+void MQMessage::setProperties(map<string, string>& properties) {
+  m_properties = properties;
+}
+
+void MQMessage::Init(const string& topic, const string& tags,
+                     const string& keys, const int flag, const string& body,
+                     bool waitStoreMsgOK) {
+  m_topic = topic;
+  m_flag = flag;
+  m_body = body;
+
+  if (tags.length() > 0) {
+    setTags(tags);
+  }
+
+  if (keys.length() > 0) {
+    setKeys(keys);
+  }
+
+  setWaitStoreMsgOK(waitStoreMsgOK);
+}
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQMessageExt.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQMessageExt.cpp b/rocketmq-cpp/src/message/MQMessageExt.cpp
new file mode 100755
index 0000000..bfba42d
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQMessageExt.cpp
@@ -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.
+ */
+#include "MQMessageExt.h"
+#include "MessageSysFlag.h"
+#include "SocketUtil.h"
+#include "TopicFilterType.h"
+
+namespace rocketmq {
+//<!************************************************************************
+MQMessageExt::MQMessageExt()
+    : m_queueOffset(0),
+      m_commitLogOffset(0),
+      m_bornTimestamp(0),
+      m_storeTimestamp(0),
+      m_preparedTransactionOffset(0),
+      m_queueId(0),
+      m_storeSize(0),
+      m_sysFlag(0),
+      m_bodyCRC(0),
+      m_reconsumeTimes(3),
+      m_msgId("") {}
+
+MQMessageExt::MQMessageExt(int queueId, int64 bornTimestamp, sockaddr bornHost,
+                           int64 storeTimestamp, sockaddr storeHost,
+                           string msgId)
+    : m_queueOffset(0),
+      m_commitLogOffset(0),
+      m_bornTimestamp(bornTimestamp),
+      m_storeTimestamp(storeTimestamp),
+      m_preparedTransactionOffset(0),
+      m_queueId(queueId),
+      m_storeSize(0),
+      m_sysFlag(0),
+      m_bodyCRC(0),
+      m_reconsumeTimes(3),
+      m_bornHost(bornHost),
+      m_storeHost(storeHost),
+      m_msgId(msgId) {}
+
+MQMessageExt::~MQMessageExt() {}
+
+int MQMessageExt::getQueueId() const { return m_queueId; }
+
+void MQMessageExt::setQueueId(int queueId) { m_queueId = queueId; }
+
+int64 MQMessageExt::getBornTimestamp() const { return m_bornTimestamp; }
+
+void MQMessageExt::setBornTimestamp(int64 bornTimestamp) {
+  m_bornTimestamp = bornTimestamp;
+}
+
+sockaddr MQMessageExt::getBornHost() const { return m_bornHost; }
+
+string MQMessageExt::getBornHostString() const {
+  return socketAddress2String(m_bornHost);
+}
+
+string MQMessageExt::getBornHostNameString() const {
+  return getHostName(m_bornHost);
+}
+
+void MQMessageExt::setBornHost(const sockaddr& bornHost) {
+  m_bornHost = bornHost;
+}
+
+int64 MQMessageExt::getStoreTimestamp() const { return m_storeTimestamp; }
+
+void MQMessageExt::setStoreTimestamp(int64 storeTimestamp) {
+  m_storeTimestamp = storeTimestamp;
+}
+
+sockaddr MQMessageExt::getStoreHost() const { return m_storeHost; }
+
+string MQMessageExt::getStoreHostString() const {
+  return socketAddress2String(m_storeHost);
+}
+
+void MQMessageExt::setStoreHost(const sockaddr& storeHost) {
+  m_storeHost = storeHost;
+}
+
+const string& MQMessageExt::getMsgId() const { return m_msgId; }
+
+void MQMessageExt::setMsgId(const string& msgId) { m_msgId = msgId; }
+
+int MQMessageExt::getSysFlag() const { return m_sysFlag; }
+
+void MQMessageExt::setSysFlag(int sysFlag) { m_sysFlag = sysFlag; }
+
+int MQMessageExt::getBodyCRC() const { return m_bodyCRC; }
+
+void MQMessageExt::setBodyCRC(int bodyCRC) { m_bodyCRC = bodyCRC; }
+
+int64 MQMessageExt::getQueueOffset() const { return m_queueOffset; }
+
+void MQMessageExt::setQueueOffset(int64 queueOffset) {
+  m_queueOffset = queueOffset;
+}
+
+int64 MQMessageExt::getCommitLogOffset() const { return m_commitLogOffset; }
+
+void MQMessageExt::setCommitLogOffset(int64 physicOffset) {
+  m_commitLogOffset = physicOffset;
+}
+
+int MQMessageExt::getStoreSize() const { return m_storeSize; }
+
+void MQMessageExt::setStoreSize(int storeSize) { m_storeSize = storeSize; }
+
+int MQMessageExt::parseTopicFilterType(int sysFlag) {
+  if ((sysFlag & MessageSysFlag::MultiTagsFlag) ==
+      MessageSysFlag::MultiTagsFlag) {
+    return MULTI_TAG;
+  }
+  return SINGLE_TAG;
+}
+
+int MQMessageExt::getReconsumeTimes() const { return m_reconsumeTimes; }
+
+void MQMessageExt::setReconsumeTimes(int reconsumeTimes) {
+  m_reconsumeTimes = reconsumeTimes;
+}
+
+int64 MQMessageExt::getPreparedTransactionOffset() const {
+  return m_preparedTransactionOffset;
+}
+
+void MQMessageExt::setPreparedTransactionOffset(
+    int64 preparedTransactionOffset) {
+  m_preparedTransactionOffset = preparedTransactionOffset;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQMessageId.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQMessageId.h b/rocketmq-cpp/src/message/MQMessageId.h
new file mode 100755
index 0000000..366ac20
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQMessageId.h
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __MESSAGEID_H__
+#define __MESSAGEID_H__
+
+#include "SocketUtil.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class MQMessageId {
+ public:
+  MQMessageId(sockaddr address, int64 offset)
+      : m_address(address), m_offset(offset) {}
+
+  sockaddr getAddress() const { return m_address; }
+
+  void setAddress(sockaddr address) { m_address = address; }
+
+  int64 getOffset() const { return m_offset; }
+
+  void setOffset(int64 offset) { m_offset = offset; }
+
+ private:
+  sockaddr m_address;
+  int64 m_offset;
+};
+
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/message/MQMessageQueue.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/message/MQMessageQueue.cpp b/rocketmq-cpp/src/message/MQMessageQueue.cpp
new file mode 100755
index 0000000..60481e5
--- /dev/null
+++ b/rocketmq-cpp/src/message/MQMessageQueue.cpp
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "MQMessageQueue.h"
+
+namespace rocketmq {
+//<!************************************************************************
+MQMessageQueue::MQMessageQueue() {
+  m_queueId = -1;  // invalide mq
+  m_topic.clear();
+  m_brokerName.clear();
+}
+
+MQMessageQueue::MQMessageQueue(const std::string& topic, const std::string& brokerName,
+                               int queueId)
+    : m_topic(topic), m_brokerName(brokerName), m_queueId(queueId) {}
+
+MQMessageQueue::MQMessageQueue(const MQMessageQueue& other)
+    : m_topic(other.m_topic),
+      m_brokerName(other.m_brokerName),
+      m_queueId(other.m_queueId) {}
+
+MQMessageQueue& MQMessageQueue::operator=(const MQMessageQueue& other) {
+  if (this != &other) {
+    m_brokerName = other.m_brokerName;
+    m_topic = other.m_topic;
+    m_queueId = other.m_queueId;
+  }
+  return *this;
+}
+
+std::string MQMessageQueue::getTopic() const { return m_topic; }
+
+void MQMessageQueue::setTopic(const std::string& topic) { m_topic = topic; }
+
+std::string MQMessageQueue::getBrokerName() const { return m_brokerName; }
+
+void MQMessageQueue::setBrokerName(const std::string& brokerName) {
+  m_brokerName = brokerName;
+}
+
+int MQMessageQueue::getQueueId() const { return m_queueId; }
+
+void MQMessageQueue::setQueueId(int queueId) { m_queueId = queueId; }
+
+bool MQMessageQueue::operator==(const MQMessageQueue& mq) const {
+  if (this == &mq) {
+    return true;
+  }
+
+  if (m_brokerName != mq.m_brokerName) {
+    return false;
+  }
+
+  if (m_queueId != mq.m_queueId) {
+    return false;
+  }
+
+  if (m_topic != mq.m_topic) {
+    return false;
+  }
+
+  return true;
+}
+
+int MQMessageQueue::compareTo(const MQMessageQueue& mq) const {
+  int result = m_topic.compare(mq.m_topic);
+  if (result != 0) {
+    return result;
+  }
+
+  result = m_brokerName.compare(mq.m_brokerName);
+  if (result != 0) {
+    return result;
+  }
+
+  return m_queueId - mq.m_queueId;
+}
+
+bool MQMessageQueue::operator<(const MQMessageQueue& mq) const {
+  return compareTo(mq) < 0;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;


[08/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/FilterAPI.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/FilterAPI.h b/rocketmq-cpp/src/common/FilterAPI.h
new file mode 100755
index 0000000..c95f17e
--- /dev/null
+++ b/rocketmq-cpp/src/common/FilterAPI.h
@@ -0,0 +1,63 @@
+/*
+ * 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 __FILTERAPI_H__
+#define __FILTERAPI_H__
+
+#include <string>
+#include "MQClientException.h"
+#include "SubscriptionData.h"
+#include "UtilAll.h"
+namespace rocketmq {
+//<!***************************************************************************
+class FilterAPI {
+ public:
+  static SubscriptionData* buildSubscriptionData(const string topic,
+                                                 const string& subString) {
+    //<!delete in balance;
+    SubscriptionData* subscriptionData = new SubscriptionData(topic, subString);
+
+    if (subString.empty() || !subString.compare(SUB_ALL)) {
+      subscriptionData->setSubString(SUB_ALL);
+    } else {
+      vector<string> out;
+      UtilAll::Split(out, subString, "||");
+
+      if (out.empty()) {
+        THROW_MQEXCEPTION(MQClientException, "FilterAPI subString split error",
+                          -1);
+      }
+
+      for (size_t i = 0; i < out.size(); i++) {
+        string tag = out[i];
+        if (!tag.empty()) {
+          UtilAll::Trim(tag);
+          if (!tag.empty()) {
+            subscriptionData->putTagsSet(tag);
+            subscriptionData->putCodeSet(tag);
+          }
+        }
+      }
+    }
+
+    return subscriptionData;
+  }
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/InputStream.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/InputStream.cpp b/rocketmq-cpp/src/common/InputStream.cpp
new file mode 100644
index 0000000..4eb5587
--- /dev/null
+++ b/rocketmq-cpp/src/common/InputStream.cpp
@@ -0,0 +1,149 @@
+#include "InputStream.h"
+#include <algorithm>
+#include "MemoryOutputStream.h"
+
+namespace rocketmq {
+int64 InputStream::getNumBytesRemaining() {
+  int64 len = getTotalLength();
+
+  if (len >= 0) len -= getPosition();
+
+  return len;
+}
+
+char InputStream::readByte() {
+  char temp = 0;
+  read(&temp, 1);
+  return temp;
+}
+
+bool InputStream::readBool() { return readByte() != 0; }
+
+short InputStream::readShort() {
+  char temp[2];
+
+  if (read(temp, 2) == 2) return (short)ByteOrder::littleEndianShort(temp);
+
+  return 0;
+}
+
+short InputStream::readShortBigEndian() {
+  char temp[2];
+
+  if (read(temp, 2) == 2) return (short)ByteOrder::bigEndianShort(temp);
+
+  return 0;
+}
+
+int InputStream::readInt() {
+  char temp[4];
+
+  if (read(temp, 4) == 4) return (int)ByteOrder::littleEndianInt(temp);
+
+  return 0;
+}
+
+int InputStream::readIntBigEndian() {
+  char temp[4];
+
+  if (read(temp, 4) == 4) return (int)ByteOrder::bigEndianInt(temp);
+
+  return 0;
+}
+
+int InputStream::readCompressedInt() {
+  const uint8 sizeByte = (uint8)readByte();
+  if (sizeByte == 0) return 0;
+
+  const int numBytes = (sizeByte & 0x7f);
+  if (numBytes > 4) {
+    return 0;
+  }
+
+  char bytes[4] = {0, 0, 0, 0};
+  if (read(bytes, numBytes) != numBytes) return 0;
+
+  const int num = (int)ByteOrder::littleEndianInt(bytes);
+  return (sizeByte >> 7) ? -num : num;
+}
+
+int64 InputStream::readInt64() {
+  union {
+    uint8 asBytes[8];
+    uint64 asInt64;
+  } n;
+
+  if (read(n.asBytes, 8) == 8)
+    return (int64)ByteOrder::swapIfBigEndian(n.asInt64);
+
+  return 0;
+}
+
+int64 InputStream::readInt64BigEndian() {
+  union {
+    uint8 asBytes[8];
+    uint64 asInt64;
+  } n;
+
+  if (read(n.asBytes, 8) == 8)
+    return (int64)ByteOrder::swapIfLittleEndian(n.asInt64);
+
+  return 0;
+}
+
+float InputStream::readFloat() {
+  // the union below relies on these types being the same size...
+  union {
+    int32 asInt;
+    float asFloat;
+  } n;
+  n.asInt = (int32)readInt();
+  return n.asFloat;
+}
+
+float InputStream::readFloatBigEndian() {
+  union {
+    int32 asInt;
+    float asFloat;
+  } n;
+  n.asInt = (int32)readIntBigEndian();
+  return n.asFloat;
+}
+
+double InputStream::readDouble() {
+  union {
+    int64 asInt;
+    double asDouble;
+  } n;
+  n.asInt = readInt64();
+  return n.asDouble;
+}
+
+double InputStream::readDoubleBigEndian() {
+  union {
+    int64 asInt;
+    double asDouble;
+  } n;
+  n.asInt = readInt64BigEndian();
+  return n.asDouble;
+}
+
+size_t InputStream::readIntoMemoryBlock(MemoryBlock& block, ssize_t numBytes) {
+  MemoryOutputStream mo(block, true);
+  return (size_t)mo.writeFromInputStream(*this, numBytes);
+}
+
+//==============================================================================
+void InputStream::skipNextBytes(int64 numBytesToSkip) {
+  if (numBytesToSkip > 0) {
+    const int skipBufferSize = (int)std::min(numBytesToSkip, (int64)16384);
+    char* temp = static_cast<char*>(std::malloc(skipBufferSize * sizeof(char)));
+
+    while (numBytesToSkip > 0 && !isExhausted())
+      numBytesToSkip -=
+          read(temp, (int)std::min(numBytesToSkip, (int64)skipBufferSize));
+
+    std::free(temp);
+  }
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/InputStream.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/InputStream.h b/rocketmq-cpp/src/common/InputStream.h
new file mode 100644
index 0000000..c987be7
--- /dev/null
+++ b/rocketmq-cpp/src/common/InputStream.h
@@ -0,0 +1,241 @@
+#ifndef INPUTSTREAM_H_INCLUDED
+#define INPUTSTREAM_H_INCLUDED
+
+#include "ByteOrder.h"
+#include "dataBlock.h"
+//==============================================================================
+/** The base class for streams that read data.
+
+    Input and output streams are used throughout the library - subclasses can
+   override
+    some or all of the virtual functions to implement their behaviour.
+
+    @see OutputStream, MemoryInputStream, BufferedInputStream, FileInputStream
+*/
+namespace rocketmq {
+class ROCKETMQCLIENT_API InputStream {
+ public:
+  /** Destructor. */
+  virtual ~InputStream() {}
+
+  //==============================================================================
+  /** Returns the total number of bytes available for reading in this stream.
+
+      Note that this is the number of bytes available from the start of the
+      stream, not from the current position.
+
+      If the size of the stream isn't actually known, this will return -1.
+
+      @see getNumBytesRemaining
+  */
+  virtual int64 getTotalLength() = 0;
+
+  /** Returns the number of bytes available for reading, or a negative value if
+      the remaining length is not known.
+      @see getTotalLength
+  */
+  int64 getNumBytesRemaining();
+
+  /** Returns true if the stream has no more data to read. */
+  virtual bool isExhausted() = 0;
+
+  //==============================================================================
+  /** Reads some data from the stream into a memory buffer.
+
+      This is the only read method that subclasses actually need to implement,
+     as the
+      InputStream base class implements the other read methods in terms of this
+     one (although
+      it's often more efficient for subclasses to implement them directly).
+
+      @param destBuffer       the destination buffer for the data. This must not
+     be null.
+      @param maxBytesToRead   the maximum number of bytes to read - make sure
+     the
+                              memory block passed in is big enough to contain
+     this
+                              many bytes. This value must not be negative.
+
+      @returns    the actual number of bytes that were read, which may be less
+     than
+                  maxBytesToRead if the stream is exhausted before it gets that
+     far
+  */
+  virtual int read(void* destBuffer, int maxBytesToRead) = 0;
+
+  /** Reads a byte from the stream.
+      If the stream is exhausted, this will return zero.
+      @see OutputStream::writeByte
+  */
+  virtual char readByte();
+
+  /** Reads a boolean from the stream.
+      The bool is encoded as a single byte - non-zero for true, 0 for false.
+      If the stream is exhausted, this will return false.
+      @see OutputStream::writeBool
+  */
+  virtual bool readBool();
+
+  /** Reads two bytes from the stream as a little-endian 16-bit value.
+      If the next two bytes read are byte1 and byte2, this returns (byte1 |
+     (byte2 << 8)).
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeShort, readShortBigEndian
+  */
+  virtual short readShort();
+
+  /** Reads two bytes from the stream as a little-endian 16-bit value.
+      If the next two bytes read are byte1 and byte2, this returns (byte2 |
+     (byte1 << 8)).
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeShortBigEndian, readShort
+  */
+  virtual short readShortBigEndian();
+
+  /** Reads four bytes from the stream as a little-endian 32-bit value.
+
+      If the next four bytes are byte1 to byte4, this returns
+      (byte1 | (byte2 << 8) | (byte3 << 16) | (byte4 << 24)).
+
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+
+      @see OutputStream::writeInt, readIntBigEndian
+  */
+  virtual int readInt();
+
+  /** Reads four bytes from the stream as a big-endian 32-bit value.
+
+      If the next four bytes are byte1 to byte4, this returns
+      (byte4 | (byte3 << 8) | (byte2 << 16) | (byte1 << 24)).
+
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+
+      @see OutputStream::writeIntBigEndian, readInt
+  */
+  virtual int readIntBigEndian();
+
+  /** Reads eight bytes from the stream as a little-endian 64-bit value.
+
+      If the next eight bytes are byte1 to byte8, this returns
+      (byte1 | (byte2 << 8) | (byte3 << 16) | (byte4 << 24) | (byte5 << 32) |
+     (byte6 << 40) | (byte7 << 48) | (byte8 << 56)).
+
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+
+      @see OutputStream::writeInt64, readInt64BigEndian
+  */
+  virtual int64 readInt64();
+
+  /** Reads eight bytes from the stream as a big-endian 64-bit value.
+
+      If the next eight bytes are byte1 to byte8, this returns
+      (byte8 | (byte7 << 8) | (byte6 << 16) | (byte5 << 24) | (byte4 << 32) |
+     (byte3 << 40) | (byte2 << 48) | (byte1 << 56)).
+
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+
+      @see OutputStream::writeInt64BigEndian, readInt64
+  */
+  virtual int64 readInt64BigEndian();
+
+  /** Reads four bytes as a 32-bit floating point value.
+      The raw 32-bit encoding of the float is read from the stream as a
+     little-endian int.
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeFloat, readDouble
+  */
+  virtual float readFloat();
+
+  /** Reads four bytes as a 32-bit floating point value.
+      The raw 32-bit encoding of the float is read from the stream as a
+     big-endian int.
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeFloatBigEndian, readDoubleBigEndian
+  */
+  virtual float readFloatBigEndian();
+
+  /** Reads eight bytes as a 64-bit floating point value.
+      The raw 64-bit encoding of the double is read from the stream as a
+     little-endian int64.
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeDouble, readFloat
+  */
+  virtual double readDouble();
+
+  /** Reads eight bytes as a 64-bit floating point value.
+      The raw 64-bit encoding of the double is read from the stream as a
+     big-endian int64.
+      If the stream is exhausted partway through reading the bytes, this will
+     return zero.
+      @see OutputStream::writeDoubleBigEndian, readFloatBigEndian
+  */
+  virtual double readDoubleBigEndian();
+
+  /** Reads an encoded 32-bit number from the stream using a space-saving
+     compressed format.
+      For small values, this is more space-efficient than using readInt() and
+     OutputStream::writeInt()
+      The format used is: number of significant bytes + up to 4 bytes in
+     little-endian order.
+      @see OutputStream::writeCompressedInt()
+  */
+  virtual int readCompressedInt();
+
+  //==============================================================================whole
+  // stream and turn it into a string.
+  /** Reads from the stream and appends the data to a MemoryBlock.
+
+      @param destBlock            the block to append the data onto
+      @param maxNumBytesToRead    if this is a positive value, it sets a limit
+     to the number
+                                  of bytes that will be read - if it's negative,
+     data
+                                  will be read until the stream is exhausted.
+      @returns the number of bytes that were added to the memory block
+  */
+  virtual size_t readIntoMemoryBlock(MemoryBlock& destBlock,
+                                     ssize_t maxNumBytesToRead = -1);
+
+  //==============================================================================
+  /** Returns the offset of the next byte that will be read from the stream.
+      @see setPosition
+  */
+  virtual int64 getPosition() = 0;
+
+  /** Tries to move the current read position of the stream.
+
+      The position is an absolute number of bytes from the stream's start.
+
+      Some streams might not be able to do this, in which case they should do
+      nothing and return false. Others might be able to manage it by resetting
+      themselves and skipping to the correct position, although this is
+      obviously a bit slow.
+
+      @returns  true if the stream manages to reposition itself correctly
+      @see getPosition
+  */
+  virtual bool setPosition(int64 newPosition) = 0;
+
+  /** Reads and discards a number of bytes from the stream.
+
+      Some input streams might implement this efficiently, but the base
+      class will just keep reading data until the requisite number of bytes
+      have been done.
+  */
+  virtual void skipNextBytes(int64 numBytesToSkip);
+
+ protected:
+  //==============================================================================
+  InputStream() {}
+};
+}
+#endif  // INPUTSTREAM_H_INCLUDED

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MQClient.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MQClient.cpp b/rocketmq-cpp/src/common/MQClient.cpp
new file mode 100755
index 0000000..25d84de
--- /dev/null
+++ b/rocketmq-cpp/src/common/MQClient.cpp
@@ -0,0 +1,201 @@
+/*
+ * 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 "MQClient.h"
+#include "Logging.h"
+#include "MQClientFactory.h"
+#include "MQClientManager.h"
+#include "TopicPublishInfo.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+#define METAQCPP_VERSION "1.0.0"
+#define BUILD_DATE "08-08-2017"
+// display version: strings bin/librocketmq.so |grep VERSION
+const char *metaq_build_time =
+    "VERSION: " METAQCPP_VERSION ", BUILD DATE: " BUILD_DATE " ";
+
+//<!************************************************************************
+MQClient::MQClient() {
+  string NAMESRV_ADDR_ENV = "NAMESRV_ADDR";
+  if (const char *addr = getenv(NAMESRV_ADDR_ENV.c_str()))
+    m_namesrvAddr = addr;
+  else
+    m_namesrvAddr = "";
+
+  m_instanceName = "DEFAULT";
+  m_clientFactory = NULL;
+  m_serviceState = CREATE_JUST;
+  m_pullThreadNum = boost::thread::hardware_concurrency();
+  m_tcpConnectTimeout = 3000;        // 3s
+  m_tcpTransportTryLockTimeout = 3;  // 3s
+  m_unitName = "";
+}
+
+MQClient::~MQClient() {}
+
+string MQClient::getMQClientId() const {
+  string clientIP = UtilAll::getLocalAddress();
+  string processId = UtilAll::to_string(getpid());
+  return processId + "-" + clientIP + "@" + m_instanceName;
+}
+
+//<!groupName;
+const string &MQClient::getGroupName() const { return m_GroupName; }
+
+void MQClient::setGroupName(const string &groupname) {
+  m_GroupName = groupname;
+}
+
+const string &MQClient::getNamesrvAddr() const { return m_namesrvAddr; }
+
+void MQClient::setNamesrvAddr(const string &namesrvAddr) {
+  m_namesrvAddr = namesrvAddr;
+}
+
+const string &MQClient::getNamesrvDomain() const { return m_namesrvDomain; }
+
+void MQClient::setNamesrvDomain(const string &namesrvDomain) {
+  m_namesrvDomain = namesrvDomain;
+}
+
+const string &MQClient::getInstanceName() const { return m_instanceName; }
+
+void MQClient::setInstanceName(const string &instanceName) {
+  m_instanceName = instanceName;
+}
+
+void MQClient::createTopic(const string &key, const string &newTopic,
+                           int queueNum) {
+  try {
+    getFactory()->createTopic(key, newTopic, queueNum, m_SessionCredentials);
+  } catch (MQException &e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+int64 MQClient::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) {
+  return getFactory()->queryMessage(topic, key, maxNum, begin, end,
+                                    m_SessionCredentials);
+}
+
+int64 MQClient::minOffset(const MQMessageQueue &mq) {
+  return getFactory()->minOffset(mq, m_SessionCredentials);
+}
+
+int64 MQClient::maxOffset(const MQMessageQueue &mq) {
+  return getFactory()->maxOffset(mq, m_SessionCredentials);
+}
+
+int64 MQClient::searchOffset(const MQMessageQueue &mq, uint64_t timestamp) {
+  return getFactory()->searchOffset(mq, timestamp, m_SessionCredentials);
+}
+
+MQMessageExt *MQClient::viewMessage(const string &msgId) {
+  return getFactory()->viewMessage(msgId, m_SessionCredentials);
+}
+
+vector<MQMessageQueue> MQClient::getTopicMessageQueueInfo(const string &topic) {
+  boost::weak_ptr<TopicPublishInfo> weak_topicPublishInfo(
+      getFactory()->tryToFindTopicPublishInfo(topic, m_SessionCredentials));
+  boost::shared_ptr<TopicPublishInfo> topicPublishInfo(
+      weak_topicPublishInfo.lock());
+  if (topicPublishInfo) {
+    return topicPublishInfo->getMessageQueueList();
+  }
+  THROW_MQEXCEPTION(
+      MQClientException,
+      "could not find MessageQueue Info of topic: [" + topic + "].", -1);
+}
+
+void MQClient::start() {
+  if (getFactory() == NULL) {
+    m_clientFactory = MQClientManager::getInstance()->getMQClientFactory(
+        getMQClientId(), m_pullThreadNum, m_tcpConnectTimeout,
+        m_tcpTransportTryLockTimeout, m_unitName);
+  }
+  LOG_INFO(
+      "MQClient "
+      "start,groupname:%s,clientID:%s,instanceName:%s,nameserveraddr:%s",
+      getGroupName().c_str(), getMQClientId().c_str(),
+      getInstanceName().c_str(), getNamesrvAddr().c_str());
+}
+
+void MQClient::shutdown() { m_clientFactory = NULL; }
+
+MQClientFactory *MQClient::getFactory() const { return m_clientFactory; }
+
+bool MQClient::isServiceStateOk() { return m_serviceState == RUNNING; }
+
+void MQClient::setMetaqLogLevel(elogLevel inputLevel) {
+  ALOG_ADAPTER.setLogLevel(inputLevel);
+}
+
+void MQClient::setMetaqLogFileSizeAndNum(int fileNum, long perFileSize) {
+  ALOG_ADAPTER.setLogFileNumAndSize(fileNum, perFileSize);
+}
+
+void MQClient::setTcpTransportPullThreadNum(int num) {
+  if (num > m_pullThreadNum) {
+    m_pullThreadNum = num;
+  }
+}
+
+const int MQClient::getTcpTransportPullThreadNum() const {
+  return m_pullThreadNum;
+}
+
+void MQClient::setTcpTransportConnectTimeout(uint64_t timeout) {
+  m_tcpConnectTimeout = timeout;
+}
+const uint64_t MQClient::getTcpTransportConnectTimeout() const {
+  return m_tcpConnectTimeout;
+}
+
+void MQClient::setTcpTransportTryLockTimeout(uint64_t timeout) {
+  if (timeout < 1000) {
+    timeout = 1000;
+  }
+  m_tcpTransportTryLockTimeout = timeout / 1000;
+}
+const uint64_t MQClient::getTcpTransportTryLockTimeout() const {
+  return m_tcpTransportTryLockTimeout;
+}
+
+void MQClient::setUnitName(string unitName) { m_unitName = unitName; }
+const string &MQClient::getUnitName() { return m_unitName; }
+
+void MQClient::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 {
+  return m_SessionCredentials;
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MQVersion.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MQVersion.cpp b/rocketmq-cpp/src/common/MQVersion.cpp
new file mode 100755
index 0000000..015390c
--- /dev/null
+++ b/rocketmq-cpp/src/common/MQVersion.cpp
@@ -0,0 +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.
+ */
+#include "MQVersion.h"
+
+namespace rocketmq {
+int MQVersion::s_CurrentVersion = MQVersion::V3_1_8;
+
+//<!************************************************************************
+const char* MQVersion::getVersionDesc(int value) {
+  switch (value) {
+    // case V1_0_0:
+    // return "V1_0_0";
+  }
+  return "";
+}
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MQVersion.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MQVersion.h b/rocketmq-cpp/src/common/MQVersion.h
new file mode 100755
index 0000000..aa9e9cd
--- /dev/null
+++ b/rocketmq-cpp/src/common/MQVersion.h
@@ -0,0 +1,229 @@
+/*
+ * 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 __MQVERSION_H__
+#define __MQVERSION_H__
+
+#include <string>
+
+namespace rocketmq {
+//<!***************************************************************************
+class MQVersion {
+ public:
+  enum Version {
+    V3_0_0_SNAPSHOT,
+    V3_0_0_ALPHA1,
+    V3_0_0_BETA1,
+    V3_0_0_BETA2,
+    V3_0_0_BETA3,
+    V3_0_0_BETA4,
+    V3_0_0_BETA5,
+    V3_0_0_BETA6_SNAPSHOT,
+    V3_0_0_BETA6,
+    V3_0_0_BETA7_SNAPSHOT,
+    V3_0_0_BETA7,
+    V3_0_0_BETA8_SNAPSHOT,
+    V3_0_0_BETA8,
+    V3_0_0_BETA9_SNAPSHOT,
+    V3_0_0_BETA9,
+    V3_0_0_FINAL,
+    V3_0_1_SNAPSHOT,
+    V3_0_1,
+    V3_0_2_SNAPSHOT,
+    V3_0_2,
+    V3_0_3_SNAPSHOT,
+    V3_0_3,
+    V3_0_4_SNAPSHOT,
+    V3_0_4,
+    V3_0_5_SNAPSHOT,
+    V3_0_5,
+    V3_0_6_SNAPSHOT,
+    V3_0_6,
+    V3_0_7_SNAPSHOT,
+    V3_0_7,
+    V3_0_8_SNAPSHOT,
+    V3_0_8,
+    V3_0_9_SNAPSHOT,
+    V3_0_9,
+
+    V3_0_10_SNAPSHOT,
+    V3_0_10,
+
+    V3_0_11_SNAPSHOT,
+    V3_0_11,
+
+    V3_0_12_SNAPSHOT,
+    V3_0_12,
+
+    V3_0_13_SNAPSHOT,
+    V3_0_13,
+
+    V3_0_14_SNAPSHOT,
+    V3_0_14,
+
+    V3_0_15_SNAPSHOT,
+    V3_0_15,
+
+    V3_1_0_SNAPSHOT,
+    V3_1_0,
+
+    V3_1_1_SNAPSHOT,
+    V3_1_1,
+
+    V3_1_2_SNAPSHOT,
+    V3_1_2,
+
+    V3_1_3_SNAPSHOT,
+    V3_1_3,
+
+    V3_1_4_SNAPSHOT,
+    V3_1_4,
+
+    V3_1_5_SNAPSHOT,
+    V3_1_5,
+
+    V3_1_6_SNAPSHOT,
+    V3_1_6,
+
+    V3_1_7_SNAPSHOT,
+    V3_1_7,
+
+    V3_1_8_SNAPSHOT,
+    V3_1_8,
+
+    V3_1_9_SNAPSHOT,
+    V3_1_9,
+
+    V3_2_0_SNAPSHOT,
+    V3_2_0,
+
+    V3_2_1_SNAPSHOT,
+    V3_2_1,
+
+    V3_2_2_SNAPSHOT,
+    V3_2_2,
+
+    V3_2_3_SNAPSHOT,
+    V3_2_3,
+
+    V3_2_4_SNAPSHOT,
+    V3_2_4,
+
+    V3_2_5_SNAPSHOT,
+    V3_2_5,
+
+    V3_2_6_SNAPSHOT,
+    V3_2_6,
+
+    V3_2_7_SNAPSHOT,
+    V3_2_7,
+
+    V3_2_8_SNAPSHOT,
+    V3_2_8,
+
+    V3_2_9_SNAPSHOT,
+    V3_2_9,
+
+    V3_3_1_SNAPSHOT,
+    V3_3_1,
+
+    V3_3_2_SNAPSHOT,
+    V3_3_2,
+
+    V3_3_3_SNAPSHOT,
+    V3_3_3,
+
+    V3_3_4_SNAPSHOT,
+    V3_3_4,
+
+    V3_3_5_SNAPSHOT,
+    V3_3_5,
+
+    V3_3_6_SNAPSHOT,
+    V3_3_6,
+
+    V3_3_7_SNAPSHOT,
+    V3_3_7,
+
+    V3_3_8_SNAPSHOT,
+    V3_3_8,
+
+    V3_3_9_SNAPSHOT,
+    V3_3_9,
+
+    V3_4_1_SNAPSHOT,
+    V3_4_1,
+
+    V3_4_2_SNAPSHOT,
+    V3_4_2,
+
+    V3_4_3_SNAPSHOT,
+    V3_4_3,
+
+    V3_4_4_SNAPSHOT,
+    V3_4_4,
+
+    V3_4_5_SNAPSHOT,
+    V3_4_5,
+
+    V3_4_6_SNAPSHOT,
+    V3_4_6,
+
+    V3_4_7_SNAPSHOT,
+    V3_4_7,
+
+    V3_4_8_SNAPSHOT,
+    V3_4_8,
+
+    V3_4_9_SNAPSHOT,
+    V3_4_9,
+    V3_5_1_SNAPSHOT,
+    V3_5_1,
+
+    V3_5_2_SNAPSHOT,
+    V3_5_2,
+
+    V3_5_3_SNAPSHOT,
+    V3_5_3,
+
+    V3_5_4_SNAPSHOT,
+    V3_5_4,
+
+    V3_5_5_SNAPSHOT,
+    V3_5_5,
+
+    V3_5_6_SNAPSHOT,
+    V3_5_6,
+
+    V3_5_7_SNAPSHOT,
+    V3_5_7,
+
+    V3_5_8_SNAPSHOT,
+    V3_5_8,
+
+    V3_5_9_SNAPSHOT,
+    V3_5_9,
+  };
+
+  static const char* getVersionDesc(int value);
+
+ public:
+  static int s_CurrentVersion;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MemoryInputStream.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MemoryInputStream.cpp b/rocketmq-cpp/src/common/MemoryInputStream.cpp
new file mode 100644
index 0000000..bfbd772
--- /dev/null
+++ b/rocketmq-cpp/src/common/MemoryInputStream.cpp
@@ -0,0 +1,55 @@
+#include "MemoryInputStream.h"
+
+namespace rocketmq {
+MemoryInputStream::MemoryInputStream(const void* const sourceData,
+                                     const size_t sourceDataSize,
+                                     const bool keepInternalCopy)
+    : data(sourceData),
+      dataSize(sourceDataSize),
+      position(0),
+      internalCopy(NULL) {
+  if (keepInternalCopy) createInternalCopy();
+}
+
+MemoryInputStream::MemoryInputStream(const MemoryBlock& sourceData,
+                                     const bool keepInternalCopy)
+    : data(sourceData.getData()),
+      dataSize(sourceData.getSize()),
+      position(0),
+      internalCopy(NULL) {
+  if (keepInternalCopy) createInternalCopy();
+}
+
+void MemoryInputStream::createInternalCopy() {
+  std::free(internalCopy);
+  internalCopy = static_cast<char*>(std::malloc(dataSize));
+  memcpy(internalCopy, data, dataSize);
+  data = internalCopy;
+}
+
+MemoryInputStream::~MemoryInputStream() { std::free(internalCopy); }
+
+int64 MemoryInputStream::getTotalLength() { return (int64)dataSize; }
+
+int MemoryInputStream::read(void* const buffer, const int howMany) {
+  const int num = std::min(howMany, (int)(dataSize - position));
+  if (num <= 0) return 0;
+
+  memcpy(buffer, data + position, (size_t)num);
+  position += (unsigned int)num;
+  return num;
+}
+
+bool MemoryInputStream::isExhausted() { return position >= dataSize; }
+
+bool MemoryInputStream::setPosition(const int64 pos) {
+  if (pos < 0)
+    position = 0;
+  else
+    position = (int64)dataSize < pos ? (int64)dataSize : pos;
+
+  return true;
+}
+
+int64 MemoryInputStream::getPosition() { return (int64)position; }
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MemoryInputStream.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MemoryInputStream.h b/rocketmq-cpp/src/common/MemoryInputStream.h
new file mode 100644
index 0000000..a25f079
--- /dev/null
+++ b/rocketmq-cpp/src/common/MemoryInputStream.h
@@ -0,0 +1,80 @@
+#ifndef MEMORYINPUTSTREAM_H_INCLUDED
+#define MEMORYINPUTSTREAM_H_INCLUDED
+
+#include "InputStream.h"
+
+namespace rocketmq {
+//==============================================================================
+/**
+    Allows a block of data to be accessed as a stream.
+
+    This can either be used to refer to a shared block of memory, or can make
+   its
+    own internal copy of the data when the MemoryInputStream is created.
+*/
+class ROCKETMQCLIENT_API MemoryInputStream : public InputStream {
+ public:
+  //==============================================================================
+  /** Creates a MemoryInputStream.
+
+      @param sourceData               the block of data to use as the stream's
+     source
+      @param sourceDataSize           the number of bytes in the source data
+     block
+      @param keepInternalCopyOfData   if false, the stream will just keep a
+     pointer to
+                                      the source data, so this data shouldn't be
+     changed
+                                      for the lifetime of the stream; if this
+     parameter is
+                                      true, the stream will make its own copy of
+     the
+                                      data and use that.
+  */
+  MemoryInputStream(const void* sourceData, size_t sourceDataSize,
+                    bool keepInternalCopyOfData);
+
+  /** Creates a MemoryInputStream.
+
+      @param data                     a block of data to use as the stream's
+     source
+      @param keepInternalCopyOfData   if false, the stream will just keep a
+     reference to
+                                      the source data, so this data shouldn't be
+     changed
+                                      for the lifetime of the stream; if this
+     parameter is
+                                      true, the stream will make its own copy of
+     the
+                                      data and use that.
+  */
+  MemoryInputStream(const MemoryBlock& data, bool keepInternalCopyOfData);
+
+  /** Destructor. */
+  ~MemoryInputStream();
+
+  /** Returns a pointer to the source data block from which this stream is
+   * reading. */
+  const void* getData() const { return data; }
+
+  /** Returns the number of bytes of source data in the block from which this
+   * stream is reading. */
+  size_t getDataSize() const { return dataSize; }
+
+  //==============================================================================
+  int64 getPosition();
+  bool setPosition(int64 pos);
+  int64 getTotalLength();
+  bool isExhausted();
+  int read(void* destBuffer, int maxBytesToRead);
+
+ private:
+  //==============================================================================
+  const void* data;
+  size_t dataSize, position;
+  char* internalCopy;
+
+  void createInternalCopy();
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MemoryOutputStream.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MemoryOutputStream.cpp b/rocketmq-cpp/src/common/MemoryOutputStream.cpp
new file mode 100644
index 0000000..36d9f8c
--- /dev/null
+++ b/rocketmq-cpp/src/common/MemoryOutputStream.cpp
@@ -0,0 +1,148 @@
+#include "MemoryOutputStream.h"
+
+namespace rocketmq {
+MemoryOutputStream::MemoryOutputStream(const size_t initialSize)
+    : blockToUse(&internalBlock),
+      externalData(NULL),
+      position(0),
+      size(0),
+      availableSize(0) {
+  internalBlock.setSize(initialSize, false);
+}
+
+MemoryOutputStream::MemoryOutputStream(MemoryBlock& memoryBlockToWriteTo,
+                                       const bool appendToExistingBlockContent)
+    : blockToUse(&memoryBlockToWriteTo),
+      externalData(NULL),
+      position(0),
+      size(0),
+      availableSize(0) {
+  if (appendToExistingBlockContent)
+    position = size = memoryBlockToWriteTo.getSize();
+}
+
+MemoryOutputStream::MemoryOutputStream(void* destBuffer, size_t destBufferSize)
+    : blockToUse(NULL),
+      externalData(destBuffer),
+      position(0),
+      size(0),
+      availableSize(destBufferSize) {}
+
+MemoryOutputStream::~MemoryOutputStream() { trimExternalBlockSize(); }
+
+void MemoryOutputStream::flush() { trimExternalBlockSize(); }
+
+void MemoryOutputStream::trimExternalBlockSize() {
+  if (blockToUse != &internalBlock && blockToUse != NULL)
+    blockToUse->setSize(size, false);
+}
+
+void MemoryOutputStream::preallocate(const size_t bytesToPreallocate) {
+  if (blockToUse != NULL) blockToUse->ensureSize(bytesToPreallocate + 1);
+}
+
+void MemoryOutputStream::reset() {
+  position = 0;
+  size = 0;
+}
+
+char* MemoryOutputStream::prepareToWrite(size_t numBytes) {
+  size_t storageNeeded = position + numBytes;
+
+  char* data;
+
+  if (blockToUse != NULL) {
+    if (storageNeeded >= (unsigned int)(blockToUse->getSize()))
+      blockToUse->ensureSize(
+          (storageNeeded + std::min(storageNeeded / 2, (size_t)(1024 * 1024)) +
+           32) &
+          ~31u);
+
+    data = static_cast<char*>(blockToUse->getData());
+  } else {
+    if (storageNeeded > availableSize) return NULL;
+
+    data = static_cast<char*>(externalData);
+  }
+
+  char* const writePointer = data + position;
+  position += numBytes;
+  size = std::max(size, position);
+  return writePointer;
+}
+
+bool MemoryOutputStream::write(const void* const buffer, size_t howMany) {
+  if (howMany == 0) return true;
+
+  if (char* dest = prepareToWrite(howMany)) {
+    memcpy(dest, buffer, howMany);
+    return true;
+  }
+
+  return false;
+}
+
+bool MemoryOutputStream::writeRepeatedByte(uint8 byte, size_t howMany) {
+  if (howMany == 0) return true;
+
+  if (char* dest = prepareToWrite(howMany)) {
+    memset(dest, byte, howMany);
+    return true;
+  }
+
+  return false;
+}
+
+MemoryBlock MemoryOutputStream::getMemoryBlock() const {
+  return MemoryBlock(getData(), getDataSize());
+}
+
+const void* MemoryOutputStream::getData() const {
+  if (blockToUse == NULL) return externalData;
+
+  if ((unsigned int)blockToUse->getSize() > size)
+    static_cast<char*>(blockToUse->getData())[size] = 0;
+
+  return blockToUse->getData();
+}
+
+bool MemoryOutputStream::setPosition(int64 newPosition) {
+  if (newPosition <= (int64)size) {
+    // ok to seek backwards
+    if (newPosition < 0)
+      position = 0;
+    else
+      position = (int64)size < newPosition ? size : newPosition;
+    return true;
+  }
+
+  // can't move beyond the end of the stream..
+  return false;
+}
+
+int64 MemoryOutputStream::writeFromInputStream(InputStream& source,
+                                               int64 maxNumBytesToWrite) {
+  // before writing from an input, see if we can preallocate to make it more
+  // efficient..
+  int64 availableData = source.getTotalLength() - source.getPosition();
+
+  if (availableData > 0) {
+    if (maxNumBytesToWrite > availableData || maxNumBytesToWrite < 0)
+      maxNumBytesToWrite = availableData;
+
+    if (blockToUse != NULL)
+      preallocate(blockToUse->getSize() + (size_t)maxNumBytesToWrite);
+  }
+
+  return OutputStream::writeFromInputStream(source, maxNumBytesToWrite);
+}
+
+OutputStream& operator<<(OutputStream& stream,
+                         const MemoryOutputStream& streamToRead) {
+  const size_t dataSize = streamToRead.getDataSize();
+
+  if (dataSize > 0) stream.write(streamToRead.getData(), dataSize);
+
+  return stream;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MemoryOutputStream.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MemoryOutputStream.h b/rocketmq-cpp/src/common/MemoryOutputStream.h
new file mode 100644
index 0000000..53fe66c
--- /dev/null
+++ b/rocketmq-cpp/src/common/MemoryOutputStream.h
@@ -0,0 +1,115 @@
+#ifndef MEMORYOUTPUTSTREAM_H_INCLUDED
+#define MEMORYOUTPUTSTREAM_H_INCLUDED
+
+#include "OutputStream.h"
+
+namespace rocketmq {
+//==============================================================================
+/**
+    Writes data to an internal memory buffer, which grows as required.
+
+    The data that was written into the stream can then be accessed later as
+    a contiguous block of memory.
+*/
+class ROCKETMQCLIENT_API MemoryOutputStream : public OutputStream {
+ public:
+  //==============================================================================
+  /** Creates an empty memory stream, ready to be written into.
+      @param initialSize  the intial amount of capacity to allocate for writing
+     into
+  */
+  MemoryOutputStream(size_t initialSize = 256);
+
+  /** Creates a memory stream for writing into into a pre-existing MemoryBlock
+     object.
+
+      Note that the destination block will always be larger than the amount of
+     data
+      that has been written to the stream, because the MemoryOutputStream keeps
+     some
+      spare capactity at its end. To trim the block's size down to fit the
+     actual
+      data, call flush(), or delete the MemoryOutputStream.
+
+      @param memoryBlockToWriteTo             the block into which new data will
+     be written.
+      @param appendToExistingBlockContent     if this is true, the contents of
+     the block will be
+                                              kept, and new data will be
+     appended to it. If false,
+                                              the block will be cleared before
+     use
+  */
+  MemoryOutputStream(MemoryBlock& memoryBlockToWriteTo,
+                     bool appendToExistingBlockContent);
+
+  /** Creates a MemoryOutputStream that will write into a user-supplied,
+     fixed-size
+      block of memory.
+      When using this mode, the stream will write directly into this memory area
+     until
+      it's full, at which point write operations will fail.
+  */
+  MemoryOutputStream(void* destBuffer, size_t destBufferSize);
+
+  /** Destructor.
+      This will free any data that was written to it.
+  */
+  ~MemoryOutputStream();
+
+  //==============================================================================
+  /** Returns a pointer to the data that has been written to the stream.
+      @see getDataSize
+  */
+  const void* getData() const;
+
+  /** Returns the number of bytes of data that have been written to the stream.
+      @see getData
+  */
+  size_t getDataSize() const { return size; }
+
+  /** Resets the stream, clearing any data that has been written to it so far.
+   */
+  void reset();
+
+  /** Increases the internal storage capacity to be able to contain at least the
+     specified
+      amount of data without needing to be resized.
+  */
+  void preallocate(size_t bytesToPreallocate);
+
+  /** Returns a copy of the stream's data as a memory block. */
+  MemoryBlock getMemoryBlock() const;
+
+  //==============================================================================
+  /** If the stream is writing to a user-supplied MemoryBlock, this will trim
+     any excess
+      capacity off the block, so that its length matches the amount of actual
+     data that
+      has been written so far.
+  */
+  void flush();
+
+  bool write(const void*, size_t);
+  int64 getPosition() { return (int64)position; }
+  bool setPosition(int64);
+  int64 writeFromInputStream(InputStream&, int64 maxNumBytesToWrite);
+  bool writeRepeatedByte(uint8 byte, size_t numTimesToRepeat);
+
+ private:
+  //==============================================================================
+  MemoryBlock* const blockToUse;
+  MemoryBlock internalBlock;
+  void* externalData;
+  size_t position, size, availableSize;
+
+  void trimExternalBlockSize();
+  char* prepareToWrite(size_t);
+};
+
+/** Copies all the data that has been written to a MemoryOutputStream into
+ * another stream. */
+OutputStream& operator<<(OutputStream& stream,
+                         const MemoryOutputStream& streamToRead);
+}
+#endif  // MEMORYOUTPUTSTREAM_H_INCLUDED

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MessageSysFlag.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MessageSysFlag.cpp b/rocketmq-cpp/src/common/MessageSysFlag.cpp
new file mode 100755
index 0000000..50a28b4
--- /dev/null
+++ b/rocketmq-cpp/src/common/MessageSysFlag.cpp
@@ -0,0 +1,37 @@
+/*
+ * 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 "MessageSysFlag.h"
+
+namespace rocketmq {
+int MessageSysFlag::CompressedFlag = (0x1 << 0);
+int MessageSysFlag::MultiTagsFlag = (0x1 << 1);
+
+int MessageSysFlag::TransactionNotType = (0x0 << 2);
+int MessageSysFlag::TransactionPreparedType = (0x1 << 2);
+int MessageSysFlag::TransactionCommitType = (0x2 << 2);
+int MessageSysFlag::TransactionRollbackType = (0x3 << 2);
+
+int MessageSysFlag::getTransactionValue(int flag) {
+  return flag & TransactionRollbackType;
+}
+
+int MessageSysFlag::resetTransactionValue(int flag, int type) {
+  return (flag & (~TransactionRollbackType)) | type;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/MessageSysFlag.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/MessageSysFlag.h b/rocketmq-cpp/src/common/MessageSysFlag.h
new file mode 100755
index 0000000..d7f7993
--- /dev/null
+++ b/rocketmq-cpp/src/common/MessageSysFlag.h
@@ -0,0 +1,38 @@
+/*
+ * 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 __MESSAGESYSFLAG_H__
+#define __MESSAGESYSFLAG_H__
+
+namespace rocketmq {
+//<!************************************************************************
+class MessageSysFlag {
+ public:
+  static int getTransactionValue(int flag);
+  static int resetTransactionValue(int flag, int type);
+
+ public:
+  static int CompressedFlag;
+  static int MultiTagsFlag;
+  static int TransactionNotType;
+  static int TransactionPreparedType;
+  static int TransactionCommitType;
+  static int TransactionRollbackType;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/NamesrvConfig.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/NamesrvConfig.h b/rocketmq-cpp/src/common/NamesrvConfig.h
new file mode 100755
index 0000000..6c0259f
--- /dev/null
+++ b/rocketmq-cpp/src/common/NamesrvConfig.h
@@ -0,0 +1,56 @@
+/*
+ * 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 __NAMESRVCONFIG_H__
+#define __NAMESRVCONFIG_H__
+
+#include <stdlib.h>
+#include <string>
+namespace rocketmq {
+//<!***************************************************************************
+class NamesrvConfig {
+ public:
+  NamesrvConfig() {
+    m_kvConfigPath = "";
+
+    char* home = getenv(ROCKETMQ_HOME_ENV.c_str());
+    if (home) {
+      m_rocketmqHome = home;
+    } else {
+      m_rocketmqHome = "";
+    }
+  }
+
+  const string& getRocketmqHome() const { return m_rocketmqHome; }
+
+  void setRocketmqHome(const string& rocketmqHome) {
+    m_rocketmqHome = rocketmqHome;
+  }
+
+  const string& getKvConfigPath() const { return m_kvConfigPath; }
+
+  void setKvConfigPath(const string& kvConfigPath) {
+    m_kvConfigPath = kvConfigPath;
+  }
+
+ private:
+  string m_rocketmqHome;
+  string m_kvConfigPath;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/OutputStream.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/OutputStream.cpp b/rocketmq-cpp/src/common/OutputStream.cpp
new file mode 100644
index 0000000..59d8c0f
--- /dev/null
+++ b/rocketmq-cpp/src/common/OutputStream.cpp
@@ -0,0 +1,129 @@
+#include "OutputStream.h"
+#include <limits>
+
+namespace rocketmq {
+//==============================================================================
+OutputStream::OutputStream() {}
+
+OutputStream::~OutputStream() {}
+
+//==============================================================================
+bool OutputStream::writeBool(const bool b) {
+  return writeByte(b ? (char)1 : (char)0);
+}
+
+bool OutputStream::writeByte(char byte) { return write(&byte, 1); }
+
+bool OutputStream::writeRepeatedByte(uint8 byte, size_t numTimesToRepeat) {
+  for (size_t i = 0; i < numTimesToRepeat; ++i)
+    if (!writeByte((char)byte)) return false;
+
+  return true;
+}
+
+bool OutputStream::writeShort(short value) {
+  const unsigned short v = ByteOrder::swapIfBigEndian((unsigned short)value);
+  return write(&v, 2);
+}
+
+bool OutputStream::writeShortBigEndian(short value) {
+  const unsigned short v = ByteOrder::swapIfLittleEndian((unsigned short)value);
+  return write(&v, 2);
+}
+
+bool OutputStream::writeInt(int value) {
+  const unsigned int v = ByteOrder::swapIfBigEndian((unsigned int)value);
+  return write(&v, 4);
+}
+
+bool OutputStream::writeIntBigEndian(int value) {
+  const unsigned int v = ByteOrder::swapIfLittleEndian((unsigned int)value);
+  return write(&v, 4);
+}
+
+bool OutputStream::writeCompressedInt(int value) {
+  unsigned int un = (value < 0) ? (unsigned int)-value : (unsigned int)value;
+
+  uint8 data[5];
+  int num = 0;
+
+  while (un > 0) {
+    data[++num] = (uint8)un;
+    un >>= 8;
+  }
+
+  data[0] = (uint8)num;
+
+  if (value < 0) data[0] |= 0x80;
+
+  return write(data, (size_t)num + 1);
+}
+
+bool OutputStream::writeInt64(int64 value) {
+  const uint64 v = ByteOrder::swapIfBigEndian((uint64)value);
+  return write(&v, 8);
+}
+
+bool OutputStream::writeInt64BigEndian(int64 value) {
+  const uint64 v = ByteOrder::swapIfLittleEndian((uint64)value);
+  return write(&v, 8);
+}
+
+bool OutputStream::writeFloat(float value) {
+  union {
+    int asInt;
+    float asFloat;
+  } n;
+  n.asFloat = value;
+  return writeInt(n.asInt);
+}
+
+bool OutputStream::writeFloatBigEndian(float value) {
+  union {
+    int asInt;
+    float asFloat;
+  } n;
+  n.asFloat = value;
+  return writeIntBigEndian(n.asInt);
+}
+
+bool OutputStream::writeDouble(double value) {
+  union {
+    int64 asInt;
+    double asDouble;
+  } n;
+  n.asDouble = value;
+  return writeInt64(n.asInt);
+}
+
+bool OutputStream::writeDoubleBigEndian(double value) {
+  union {
+    int64 asInt;
+    double asDouble;
+  } n;
+  n.asDouble = value;
+  return writeInt64BigEndian(n.asInt);
+}
+
+int64 OutputStream::writeFromInputStream(InputStream& source,
+                                         int64 numBytesToWrite) {
+  if (numBytesToWrite < 0) numBytesToWrite = std::numeric_limits<int64>::max();
+
+  int64 numWritten = 0;
+
+  while (numBytesToWrite > 0) {
+    char buffer[8192];
+    const int num = source.read(
+        buffer, (int)std::min(numBytesToWrite, (int64)sizeof(buffer)));
+
+    if (num <= 0) break;
+
+    write(buffer, (size_t)num);
+
+    numBytesToWrite -= num;
+    numWritten += num;
+  }
+
+  return numWritten;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/OutputStream.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/OutputStream.h b/rocketmq-cpp/src/common/OutputStream.h
new file mode 100644
index 0000000..1fb2124
--- /dev/null
+++ b/rocketmq-cpp/src/common/OutputStream.h
@@ -0,0 +1,184 @@
+#ifndef OUTPUTSTREAM_H_INCLUDED
+#define OUTPUTSTREAM_H_INCLUDED
+
+#include "ByteOrder.h"
+#include "InputStream.h"
+namespace rocketmq {
+//==============================================================================
+/**
+    The base class for streams that write data to some kind of destination.
+
+    Input and output streams are used throughout the library - subclasses can
+   override
+    some or all of the virtual functions to implement their behaviour.
+
+    @see InputStream, MemoryOutputStream, FileOutputStream
+*/
+class ROCKETMQCLIENT_API OutputStream {
+ protected:
+  //==============================================================================
+  OutputStream();
+
+ public:
+  /** Destructor.
+
+      Some subclasses might want to do things like call flush() during their
+      destructors.
+  */
+  virtual ~OutputStream();
+
+  //==============================================================================
+  /** If the stream is using a buffer, this will ensure it gets written
+      out to the destination. */
+  virtual void flush() = 0;
+
+  /** Tries to move the stream's output position.
+
+      Not all streams will be able to seek to a new position - this will return
+      false if it fails to work.
+
+      @see getPosition
+  */
+  virtual bool setPosition(int64 newPosition) = 0;
+
+  /** Returns the stream's current position.
+
+      @see setPosition
+  */
+  virtual int64 getPosition() = 0;
+
+  //==============================================================================
+  /** Writes a block of data to the stream.
+
+      When creating a subclass of OutputStream, this is the only write method
+      that needs to be overloaded - the base class has methods for writing other
+      types of data which use this to do the work.
+
+      @param dataToWrite      the target buffer to receive the data. This must
+     not be null.
+      @param numberOfBytes    the number of bytes to write.
+      @returns false if the write operation fails for some reason
+  */
+  virtual bool write(const void* dataToWrite, size_t numberOfBytes) = 0;
+
+  //==============================================================================
+  /** Writes a single byte to the stream.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readByte
+  */
+  virtual bool writeByte(char byte);
+
+  /** Writes a boolean to the stream as a single byte.
+      This is encoded as a binary byte (not as text) with a value of 1 or 0.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readBool
+  */
+  virtual bool writeBool(bool boolValue);
+
+  /** Writes a 16-bit integer to the stream in a little-endian byte order.
+      This will write two bytes to the stream: (value & 0xff), then (value >>
+     8).
+      @returns false if the write operation fails for some reason
+      @see InputStream::readShort
+  */
+  virtual bool writeShort(short value);
+
+  /** Writes a 16-bit integer to the stream in a big-endian byte order.
+      This will write two bytes to the stream: (value >> 8), then (value &
+     0xff).
+      @returns false if the write operation fails for some reason
+      @see InputStream::readShortBigEndian
+  */
+  virtual bool writeShortBigEndian(short value);
+
+  /** Writes a 32-bit integer to the stream in a little-endian byte order.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readInt
+  */
+  virtual bool writeInt(int value);
+
+  /** Writes a 32-bit integer to the stream in a big-endian byte order.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readIntBigEndian
+  */
+  virtual bool writeIntBigEndian(int value);
+
+  /** Writes a 64-bit integer to the stream in a little-endian byte order.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readInt64
+  */
+  virtual bool writeInt64(int64 value);
+
+  /** Writes a 64-bit integer to the stream in a big-endian byte order.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readInt64BigEndian
+  */
+  virtual bool writeInt64BigEndian(int64 value);
+
+  /** Writes a 32-bit floating point value to the stream in a binary format.
+      The binary 32-bit encoding of the float is written as a little-endian int.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readFloat
+  */
+  virtual bool writeFloat(float value);
+
+  /** Writes a 32-bit floating point value to the stream in a binary format.
+      The binary 32-bit encoding of the float is written as a big-endian int.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readFloatBigEndian
+  */
+  virtual bool writeFloatBigEndian(float value);
+
+  /** Writes a 64-bit floating point value to the stream in a binary format.
+      The eight raw bytes of the double value are written out as a little-endian
+     64-bit int.
+      @returns false if the write operation fails for some reason
+      @see InputStream::readDouble
+  */
+  virtual bool writeDouble(double value);
+
+  /** Writes a 64-bit floating point value to the stream in a binary format.
+      The eight raw bytes of the double value are written out as a big-endian
+     64-bit int.
+      @see InputStream::readDoubleBigEndian
+      @returns false if the write operation fails for some reason
+  */
+  virtual bool writeDoubleBigEndian(double value);
+
+  /** Writes a byte to the output stream a given number of times.
+      @returns false if the write operation fails for some reason
+  */
+  virtual bool writeRepeatedByte(uint8 byte, size_t numTimesToRepeat);
+
+  /** Writes a condensed binary encoding of a 32-bit integer.
+
+      If you're storing a lot of integers which are unlikely to have very large
+     values,
+      this can save a lot of space, because values under 0xff will only take up
+     2 bytes,
+      under 0xffff only 3 bytes, etc.
+
+      The format used is: number of significant bytes + up to 4 bytes in
+     little-endian order.
+
+      @returns false if the write operation fails for some reason
+      @see InputStream::readCompressedInt
+  */
+  virtual bool writeCompressedInt(int value);
+
+  /** Reads data from an input stream and writes it to this stream.
+
+      @param source               the stream to read from
+      @param maxNumBytesToWrite   the number of bytes to read from the stream
+     (if this is
+                                  less than zero, it will keep reading until the
+     input
+                                  is exhausted)
+      @returns the number of bytes written
+  */
+  virtual int64 writeFromInputStream(InputStream& source,
+                                     int64 maxNumBytesToWrite);
+};
+}
+
+#endif  // OUTPUTSTREAM_H_INCLUDED

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/PermName.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/PermName.cpp b/rocketmq-cpp/src/common/PermName.cpp
new file mode 100755
index 0000000..7f168a4
--- /dev/null
+++ b/rocketmq-cpp/src/common/PermName.cpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "PermName.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+int PermName::PERM_PRIORITY = 0x1 << 3;
+int PermName::PERM_READ = 0x1 << 2;
+int PermName::PERM_WRITE = 0x1 << 1;
+int PermName::PERM_INHERIT = 0x1 << 0;
+
+bool PermName::isReadable(int perm) { return (perm & PERM_READ) == PERM_READ; }
+
+bool PermName::isWriteable(int perm) {
+  return (perm & PERM_WRITE) == PERM_WRITE;
+}
+
+bool PermName::isInherited(int perm) {
+  return (perm & PERM_INHERIT) == PERM_INHERIT;
+}
+
+string PermName::perm2String(int perm) {
+  string pm("---");
+  if (isReadable(perm)) {
+    pm.replace(0, 1, "R");
+  }
+
+  if (isWriteable(perm)) {
+    pm.replace(1, 2, "W");
+  }
+
+  if (isInherited(perm)) {
+    pm.replace(2, 3, "X");
+  }
+
+  return pm;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/PermName.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/PermName.h b/rocketmq-cpp/src/common/PermName.h
new file mode 100755
index 0000000..6556382
--- /dev/null
+++ b/rocketmq-cpp/src/common/PermName.h
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __PERMNAME_H__
+#define __PERMNAME_H__
+
+#include <string>
+
+namespace rocketmq {
+//<!***************************************************************************
+class PermName {
+ public:
+  static int PERM_PRIORITY;
+  static int PERM_READ;
+  static int PERM_WRITE;
+  static int PERM_INHERIT;
+
+  static bool isReadable(int perm);
+  static bool isWriteable(int perm);
+  static bool isInherited(int perm);
+  static std::string perm2String(int perm);
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/PullSysFlag.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/PullSysFlag.cpp b/rocketmq-cpp/src/common/PullSysFlag.cpp
new file mode 100755
index 0000000..6d68457
--- /dev/null
+++ b/rocketmq-cpp/src/common/PullSysFlag.cpp
@@ -0,0 +1,70 @@
+/*
+ * 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 "PullSysFlag.h"
+
+namespace rocketmq {
+//<!************************************************************************
+int PullSysFlag::FLAG_COMMIT_OFFSET = 0x1 << 0;
+int PullSysFlag::FLAG_SUSPEND = 0x1 << 1;
+int PullSysFlag::FLAG_SUBSCRIPTION = 0x1 << 2;
+int PullSysFlag::FLAG_CLASS_FILTER = 0x1 << 3;
+
+int PullSysFlag::buildSysFlag(bool commitOffset, bool suspend,
+                              bool subscription, bool classFilter) {
+  int flag = 0;
+
+  if (commitOffset) {
+    flag |= FLAG_COMMIT_OFFSET;
+  }
+
+  if (suspend) {
+    flag |= FLAG_SUSPEND;
+  }
+
+  if (subscription) {
+    flag |= FLAG_SUBSCRIPTION;
+  }
+
+  if (classFilter) {
+    flag |= FLAG_CLASS_FILTER;
+  }
+
+  return flag;
+}
+
+int PullSysFlag::clearCommitOffsetFlag(int sysFlag) {
+  return sysFlag & (~FLAG_COMMIT_OFFSET);
+}
+
+bool PullSysFlag::hasCommitOffsetFlag(int sysFlag) {
+  return (sysFlag & FLAG_COMMIT_OFFSET) == FLAG_COMMIT_OFFSET;
+}
+
+bool PullSysFlag::hasSuspendFlag(int sysFlag) {
+  return (sysFlag & FLAG_SUSPEND) == FLAG_SUSPEND;
+}
+
+bool PullSysFlag::hasSubscriptionFlag(int sysFlag) {
+  return (sysFlag & FLAG_SUBSCRIPTION) == FLAG_SUBSCRIPTION;
+}
+
+bool PullSysFlag::hasClassFilterFlag(int sysFlag) {
+  return (sysFlag & FLAG_CLASS_FILTER) == FLAG_CLASS_FILTER;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/PullSysFlag.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/PullSysFlag.h b/rocketmq-cpp/src/common/PullSysFlag.h
new file mode 100755
index 0000000..c809772
--- /dev/null
+++ b/rocketmq-cpp/src/common/PullSysFlag.h
@@ -0,0 +1,41 @@
+/*
+ * 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 __PULLSYSFLAG_H__
+#define __PULLSYSFLAG_H__
+namespace rocketmq {
+//<!************************************************************************
+class PullSysFlag {
+ public:
+  static int buildSysFlag(bool commitOffset, bool suspend, bool subscription,
+                          bool classFilter);
+
+  static int clearCommitOffsetFlag(int sysFlag);
+  static bool hasCommitOffsetFlag(int sysFlag);
+  static bool hasSuspendFlag(int sysFlag);
+  static bool hasSubscriptionFlag(int sysFlag);
+  static bool hasClassFilterFlag(int sysFlag);
+
+ private:
+  static int FLAG_COMMIT_OFFSET;
+  static int FLAG_SUSPEND;
+  static int FLAG_SUBSCRIPTION;
+  static int FLAG_CLASS_FILTER;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/ServiceState.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/ServiceState.h b/rocketmq-cpp/src/common/ServiceState.h
new file mode 100755
index 0000000..a8ae792
--- /dev/null
+++ b/rocketmq-cpp/src/common/ServiceState.h
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __SERVICESTATE_H__
+#define __SERVICESTATE_H__
+namespace rocketmq {
+//<!***************************************************************************
+enum ServiceState { CREATE_JUST, RUNNING, SHUTDOWN_ALREADY, START_FAILED };
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/SubscriptionGroupConfig.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/SubscriptionGroupConfig.h b/rocketmq-cpp/src/common/SubscriptionGroupConfig.h
new file mode 100755
index 0000000..9a54140
--- /dev/null
+++ b/rocketmq-cpp/src/common/SubscriptionGroupConfig.h
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __SUBSCRIPTIONGROUPCONFIG_H__
+#define __SUBSCRIPTIONGROUPCONFIG_H__
+
+#include <string>
+
+namespace rocketmq {
+//<!***************************************************************************
+class SubscriptionGroupConfig {
+ public:
+  SubscriptionGroupConfig(const string& groupName) {
+    this->groupName = groupName;
+    consumeEnable = true;
+    consumeFromMinEnable = true;
+    consumeBroadcastEnable = true;
+    retryQueueNums = 1;
+    retryMaxTimes = 5;
+    brokerId = MASTER_ID;
+    whichBrokerWhenConsumeSlowly = 1;
+  }
+
+  string groupName;
+  bool consumeEnable;
+  bool consumeFromMinEnable;
+  bool consumeBroadcastEnable;
+  int retryQueueNums;
+  int retryMaxTimes;
+  int brokerId;
+  int whichBrokerWhenConsumeSlowly;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/TopAddressing.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/TopAddressing.cpp b/rocketmq-cpp/src/common/TopAddressing.cpp
new file mode 100644
index 0000000..3a1742d
--- /dev/null
+++ b/rocketmq-cpp/src/common/TopAddressing.cpp
@@ -0,0 +1,108 @@
+/*
+ * 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 "TopAddressing.h"
+#include <arpa/inet.h>
+#include <netdb.h>
+#include <netinet/in.h>
+#include <sys/socket.h>
+#include <vector>
+#include "UtilAll.h"
+#include "sync_http_client.h"
+#include "url.h"
+
+namespace rocketmq {
+TopAddressing::TopAddressing(string unitName) : m_unitName(unitName) {}
+
+TopAddressing::~TopAddressing() {}
+
+int TopAddressing::IsIPAddr(const char* sValue) {
+  if (NULL == sValue) return -1;
+
+  while (*sValue != '\0') {
+    if ((*sValue < '0' || *sValue > '9') && (*sValue != '.')) return -1;
+    sValue++;
+  }
+  return 0;
+}
+
+void TopAddressing::updateNameServerAddressList(const string& adds) {
+  boost::lock_guard<boost::mutex> lock(m_addrLock);
+  vector<string> out;
+  UtilAll::Split(out, adds, ";");
+  if (out.size() > 0) m_addrs.clear();
+  for (size_t i = 0; i < out.size(); i++) {
+    string addr = out[i];
+    UtilAll::Trim(addr);
+
+    list<string>::iterator findit = find(m_addrs.begin(), m_addrs.end(), addr);
+    if (findit == m_addrs.end()) {
+      string hostName;
+      short portNumber;
+      if (UtilAll::SplitURL(addr, hostName, portNumber)) {
+        LOG_INFO("updateNameServerAddressList:%s", addr.c_str());
+        m_addrs.push_back(addr);
+      }
+    }
+  }
+}
+
+string TopAddressing::fetchNSAddr(const string& NSDomain) {
+  LOG_DEBUG("fetchNSAddr begin");
+  string nsAddr = NSDomain.empty() ? WS_ADDR : NSDomain;
+  if (!m_unitName.empty()) {
+    nsAddr = nsAddr + "-" + m_unitName + "?nofix=1";
+    LOG_INFO("NSAddr is:%s", nsAddr.c_str());
+  }
+
+  std::string tmp_nameservers;
+  std::string nameservers;
+  Url url_s(nsAddr);
+  LOG_INFO("fetchNSAddr protocol: %s, port: %s, host:%s, path:%s, ",
+           url_s.protocol_.c_str(), url_s.port_.c_str(), url_s.host_.c_str(),
+           url_s.path_.c_str());
+
+  bool ret = SyncfetchNsAddr(url_s, tmp_nameservers);
+  if (ret) {
+    nameservers = clearNewLine(tmp_nameservers);
+    if (nameservers.empty()) {
+      LOG_ERROR("fetchNSAddr with domain is empty");
+    } else {
+      updateNameServerAddressList(nameservers);
+    }
+  } else {
+    LOG_ERROR(
+        "fetchNSAddr with domain failed, connect failure or wrong response");
+  }
+
+  return nameservers;
+}
+
+string TopAddressing::clearNewLine(const string& str) {
+  string newString = str;
+  size_t index = newString.find("\r");
+  if (index != string::npos) {
+    return newString.substr(0, index);
+  }
+
+  index = newString.find("\n");
+  if (index != string::npos) {
+    return newString.substr(0, index);
+  }
+
+  return newString;
+}
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/TopAddressing.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/TopAddressing.h b/rocketmq-cpp/src/common/TopAddressing.h
new file mode 100755
index 0000000..a850023
--- /dev/null
+++ b/rocketmq-cpp/src/common/TopAddressing.h
@@ -0,0 +1,48 @@
+/*
+ * 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 __TOPADDRESSING_H__
+#define __TOPADDRESSING_H__
+
+#include <sys/time.h>
+#include <boost/thread/thread.hpp>
+#include <list>
+#include <map>
+#include <string>
+#include "Logging.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+class TopAddressing {
+ public:
+  TopAddressing(string unitName);
+  virtual ~TopAddressing();
+
+ public:
+  string fetchNSAddr(const string& NSDomain);
+
+ private:
+  string clearNewLine(const string& str);
+  void updateNameServerAddressList(const string& adds);
+  int IsIPAddr(const char* sValue);
+
+ private:
+  boost::mutex m_addrLock;
+  list<string> m_addrs;
+  string m_unitName;
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/TopicConfig.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/TopicConfig.cpp b/rocketmq-cpp/src/common/TopicConfig.cpp
new file mode 100755
index 0000000..e0e1b4d
--- /dev/null
+++ b/rocketmq-cpp/src/common/TopicConfig.cpp
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "TopicConfig.h"
+#include <stdlib.h>
+#include <sstream>
+#include "PermName.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+int TopicConfig::DefaultReadQueueNums = 16;
+int TopicConfig::DefaultWriteQueueNums = 16;
+string TopicConfig::SEPARATOR = " ";
+
+TopicConfig::TopicConfig()
+    : m_topicName(""),
+      m_readQueueNums(DefaultReadQueueNums),
+      m_writeQueueNums(DefaultWriteQueueNums),
+      m_perm(PermName::PERM_READ | PermName::PERM_WRITE),
+      m_topicFilterType(SINGLE_TAG) {}
+
+TopicConfig::TopicConfig(const string& topicName)
+    : m_topicName(topicName),
+      m_readQueueNums(DefaultReadQueueNums),
+      m_writeQueueNums(DefaultWriteQueueNums),
+      m_perm(PermName::PERM_READ | PermName::PERM_WRITE),
+      m_topicFilterType(SINGLE_TAG) {}
+
+TopicConfig::TopicConfig(const string& topicName, int readQueueNums,
+                         int writeQueueNums, int perm)
+    : m_topicName(topicName),
+      m_readQueueNums(readQueueNums),
+      m_writeQueueNums(writeQueueNums),
+      m_perm(perm),
+      m_topicFilterType(SINGLE_TAG) {}
+
+TopicConfig::~TopicConfig() {}
+
+string TopicConfig::encode() {
+  stringstream ss;
+  ss << m_topicName << SEPARATOR << m_readQueueNums << SEPARATOR
+     << m_writeQueueNums << SEPARATOR << m_perm << SEPARATOR
+     << m_topicFilterType;
+
+  return ss.str();
+}
+
+bool TopicConfig::decode(const string& in) {
+  stringstream ss(in);
+
+  ss >> m_topicName;
+  ss >> m_readQueueNums;
+  ss >> m_writeQueueNums;
+  ss >> m_perm;
+
+  int type;
+  ss >> type;
+  m_topicFilterType = (TopicFilterType)type;
+
+  return true;
+}
+
+const string& TopicConfig::getTopicName() { return m_topicName; }
+
+void TopicConfig::setTopicName(const string& topicName) {
+  m_topicName = topicName;
+}
+
+int TopicConfig::getReadQueueNums() { return m_readQueueNums; }
+
+void TopicConfig::setReadQueueNums(int readQueueNums) {
+  m_readQueueNums = readQueueNums;
+}
+
+int TopicConfig::getWriteQueueNums() { return m_writeQueueNums; }
+
+void TopicConfig::setWriteQueueNums(int writeQueueNums) {
+  m_writeQueueNums = writeQueueNums;
+}
+
+int TopicConfig::getPerm() { return m_perm; }
+
+void TopicConfig::setPerm(int perm) { m_perm = perm; }
+
+TopicFilterType TopicConfig::getTopicFilterType() { return m_topicFilterType; }
+
+void TopicConfig::setTopicFilterType(TopicFilterType topicFilterType) {
+  m_topicFilterType = topicFilterType;
+}
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/TopicConfig.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/TopicConfig.h b/rocketmq-cpp/src/common/TopicConfig.h
new file mode 100755
index 0000000..0e7c17a
--- /dev/null
+++ b/rocketmq-cpp/src/common/TopicConfig.h
@@ -0,0 +1,61 @@
+/*
+ * 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 __TOPICCONFIG_H__
+#define __TOPICCONFIG_H__
+
+#include <string>
+#include "TopicFilterType.h"
+#include "UtilAll.h"
+namespace rocketmq {
+//<!************************************************************************
+class TopicConfig {
+ public:
+  TopicConfig();
+  TopicConfig(const string& topicName);
+  TopicConfig(const string& topicName, int readQueueNums, int writeQueueNums,
+              int perm);
+  ~TopicConfig();
+
+  string encode();
+  bool decode(const string& in);
+  const string& getTopicName();
+  void setTopicName(const string& topicName);
+  int getReadQueueNums();
+  void setReadQueueNums(int readQueueNums);
+  int getWriteQueueNums();
+  void setWriteQueueNums(int writeQueueNums);
+  int getPerm();
+  void setPerm(int perm);
+  TopicFilterType getTopicFilterType();
+  void setTopicFilterType(TopicFilterType topicFilterType);
+
+ public:
+  static int DefaultReadQueueNums;
+  static int DefaultWriteQueueNums;
+
+ private:
+  static string SEPARATOR;
+
+  string m_topicName;
+  int m_readQueueNums;
+  int m_writeQueueNums;
+  int m_perm;
+  TopicFilterType m_topicFilterType;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/TopicFilterType.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/TopicFilterType.h b/rocketmq-cpp/src/common/TopicFilterType.h
new file mode 100755
index 0000000..9055003
--- /dev/null
+++ b/rocketmq-cpp/src/common/TopicFilterType.h
@@ -0,0 +1,34 @@
+/*
+ * 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 __TOPICFILTERTYPE_H__
+#define __TOPICFILTERTYPE_H__
+
+namespace rocketmq {
+//<!***************************************************************************
+enum TopicFilterType {
+  /**
+   * each msg could only have one tag
+   */
+  SINGLE_TAG,
+  /**
+   * not support now
+   */
+  MULTI_TAG
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif


[09/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/FastDelegate.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/FastDelegate.h b/rocketmq-cpp/src/common/FastDelegate.h
new file mode 100644
index 0000000..b2f5035
--- /dev/null
+++ b/rocketmq-cpp/src/common/FastDelegate.h
@@ -0,0 +1,2626 @@
+//                      FastDelegate.h
+//  Efficient delegates in C++ that generate only two lines of asm code!
+//  Documentation is found at http://www.codeproject.com/cpp/FastDelegate.asp
+//
+//                      - Don Clugston, Mar 2004.
+//      Major contributions were made by Jody Hagins.
+// History:
+// 24-Apr-04 1.0  * Submitted to CodeProject.
+// 28-Apr-04 1.1  * Prevent most unsafe uses of evil static function hack.
+//                * Improved syntax for horrible_cast (thanks Paul Bludov).
+//                * Tested on Metrowerks MWCC and Intel ICL (IA32)
+//                * Compiled, but not run, on Comeau C++ and Intel Itanium ICL.
+//  27-Jun-04 1.2 * Now works on Borland C++ Builder 5.5
+//                * Now works on /clr "managed C++" code on VC7, VC7.1
+//                * Comeau C++ now compiles without warnings.
+//                * Prevent the virtual inheritance case from being used on
+//                    VC6 and earlier, which generate incorrect code.
+//                * Improved warning and error messages. Non-standard hacks
+//                   now have compile-time checks to make them safer.
+//                * implicit_cast used instead of static_cast in many cases.
+//                * If calling a const member function, a const class pointer
+//                can be used.
+//                * MakeDelegate() global helper function added to simplify
+//                pass-by-value.
+//                * Added fastdelegate.clear()
+// 16-Jul-04 1.2.1* Workaround for gcc bug (const member function pointers in
+// templates)
+// 30-Oct-04 1.3  * Support for (non-void) return values.
+//                * No more workarounds in client code!
+//                   MSVC and Intel now use a clever hack invented by John
+//                   Dlugosz:
+//                   - The FASTDELEGATEDECLARE workaround is no longer
+//                   necessary.
+//                   - No more warning messages for VC6
+//                * Less use of macros. Error messages should be more
+//                comprehensible.
+//                * Added include guards
+//                * Added FastDelegate::empty() to test if invocation is safe
+//                (Thanks Neville Franks).
+//                * Now tested on VS 2005 Express Beta, PGI C++
+// 24-Dec-04 1.4  * Added DelegateMemento, to allow collections of disparate
+// delegates.
+//                * <,>,<=,>= comparison operators to allow storage in ordered
+//                containers.
+//                * Substantial reduction of code size, especially the 'Closure'
+//                class.
+//                * Standardised all the compiler-specific workarounds.
+//                * MFP conversion now works for CodePlay (but not yet supported
+//                in the full code).
+//                * Now compiles without warnings on _any_ supported compiler,
+//                including BCC 5.5.1
+//                * New syntax: FastDelegate< int (char *, double) >.
+// 14-Feb-05 1.4.1* Now treats =0 as equivalent to .clear(), ==0 as equivalent
+// to .empty(). (Thanks elfric).
+//                * Now tested on Intel ICL for AMD64, VS2005 Beta for AMD64 and
+//                Itanium.
+// 30-Mar-05 1.5  * Safebool idiom: "if (dg)" is now equivalent to "if
+// (!dg.empty())"
+//                * Fully supported by CodePlay VectorC
+//                * Bugfix for Metrowerks: empty() was buggy because a valid MFP
+//                can be 0 on MWCC!
+//                * More optimal assignment,== and != operators for static
+//                function pointers.
+
+#ifndef FASTDELEGATE_H
+#define FASTDELEGATE_H
+#if _MSC_VER > 1000
+#pragma once
+#endif  // _MSC_VER > 1000
+
+#include <memory.h>  // to allow <,> comparisons
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Configuration options
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// Uncomment the following #define for optimally-sized delegates.
+// In this case, the generated asm code is almost identical to the code you'd
+// get
+// if the compiler had native support for delegates.
+// It will not work on systems where sizeof(dataptr) < sizeof(codeptr).
+// Thus, it will not work for DOS compilers using the medium model.
+// It will also probably fail on some DSP systems.
+#define FASTDELEGATE_USESTATICFUNCTIONHACK
+
+// Uncomment the next line to allow function declarator syntax.
+// It is automatically enabled for those compilers where it is known to work.
+//#define FASTDELEGATE_ALLOW_FUNCTION_TYPE_SYNTAX
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Compiler identification for workarounds
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// Compiler identification. It's not easy to identify Visual C++ because
+// many vendors fraudulently define Microsoft's identifiers.
+#if defined(_MSC_VER) && !defined(__MWERKS__) && !defined(__VECTOR_C) && \
+    !defined(__ICL) && !defined(__BORLANDC__)
+#define FASTDLGT_ISMSVC
+
+#if (_MSC_VER < 1300)  // Many workarounds are required for VC6.
+#define FASTDLGT_VC6
+#pragma warning(disable : 4786)  // disable this ridiculous warning
+#endif
+
+#endif
+
+// Does the compiler uses Microsoft's member function pointer structure?
+// If so, it needs special treatment.
+// Metrowerks CodeWarrior, Intel, and CodePlay fraudulently define Microsoft's
+// identifier, _MSC_VER. We need to filter Metrowerks out.
+#if defined(_MSC_VER) && !defined(__MWERKS__)
+#define FASTDLGT_MICROSOFT_MFP
+
+#if !defined(__VECTOR_C)
+// CodePlay doesn't have the __single/multi/virtual_inheritance keywords
+#define FASTDLGT_HASINHERITANCE_KEYWORDS
+#endif
+#endif
+
+// Does it allow function declarator syntax? The following compilers are known
+// to work:
+#if defined(FASTDLGT_ISMSVC) && (_MSC_VER >= 1310)  // VC 7.1
+#define FASTDELEGATE_ALLOW_FUNCTION_TYPE_SYNTAX
+#endif
+
+// Gcc(2.95+), and versions of Digital Mars, Intel and Comeau in common use.
+#if defined(__DMC__) || defined(__GNUC__) || defined(__ICL) || defined(__COMO__)
+#define FASTDELEGATE_ALLOW_FUNCTION_TYPE_SYNTAX
+#endif
+
+// It works on Metrowerks MWCC 3.2.2. From boost.Config it should work on
+// earlier ones too.
+#if defined(__MWERKS__)
+#define FASTDELEGATE_ALLOW_FUNCTION_TYPE_SYNTAX
+#endif
+
+#ifdef __GNUC__  // Workaround GCC bug #8271
+// At present, GCC doesn't recognize constness of MFPs in templates
+#define FASTDELEGATE_GCC_BUG_8271
+#endif
+
+////////////////////////////////////////////////////////////////////////////////
+//                      General tricks used in this code
+//
+// (a) Error messages are generated by typdefing an array of negative size to
+//     generate compile-time errors.
+// (b) Warning messages on MSVC are generated by declaring unused variables, and
+//      enabling the "variable XXX is never used" warning.
+// (c) Unions are used in a few compiler-specific cases to perform illegal
+// casts.
+// (d) For Microsoft and Intel, when adjusting the 'this' pointer, it's cast to
+//     (char *) first to ensure that the correct number of *bytes* are added.
+//
+////////////////////////////////////////////////////////////////////////////////
+//                      Helper templates
+//
+////////////////////////////////////////////////////////////////////////////////
+
+namespace fastdelegate {
+namespace detail {  // we'll hide the implementation details in a nested
+                    // namespace.
+
+//      implicit_cast< >
+// I believe this was originally going to be in the C++ standard but
+// was left out by accident. It's even milder than static_cast.
+// I use it instead of static_cast<> to emphasize that I'm not doing
+// anything nasty.
+// Usage is identical to static_cast<>
+template <class OutputClass, class InputClass>
+inline OutputClass implicit_cast(InputClass input) {
+  return input;
+}
+
+//      horrible_cast< >
+// This is truly evil. It completely subverts C++'s type system, allowing you
+// to cast from any class to any other class. Technically, using a union
+// to perform the cast is undefined behaviour (even in C). But we can see if
+// it is OK by checking that the union is the same size as each of its members.
+// horrible_cast<> should only be used for compiler-specific workarounds.
+// Usage is identical to reinterpret_cast<>.
+
+// This union is declared outside the horrible_cast because BCC 5.5.1
+// can't inline a function with a nested class, and gives a warning.
+template <class OutputClass, class InputClass>
+union horrible_union {
+  OutputClass out;
+  InputClass in;
+};
+
+template <class OutputClass, class InputClass>
+inline OutputClass horrible_cast(const InputClass input) {
+  horrible_union<OutputClass, InputClass> u;
+  // Cause a compile-time error if in, out and u are not the same size.
+  // If the compile fails here, it means the compiler has peculiar
+  // unions which would prevent the cast from working.
+  // typedef int ERROR_CantUseHorrible_cast[sizeof(InputClass)==sizeof(u)
+  //  && sizeof(InputClass)==sizeof(OutputClass) ? 1 : -1];
+  u.in = input;
+  return u.out;
+}
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Workarounds
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// Backwards compatibility: This macro used to be necessary in the virtual
+// inheritance
+// case for Intel and Microsoft. Now it just forward-declares the class.
+#define FASTDELEGATEDECLARE(CLASSNAME) class CLASSNAME;
+
+// Prevent use of the static function hack with the DOS medium model.
+#ifdef __MEDIUM__
+#undef FASTDELEGATE_USESTATICFUNCTIONHACK
+#endif
+
+//          DefaultVoid - a workaround for 'void' templates in VC6.
+//
+//  (1) VC6 and earlier do not allow 'void' as a default template argument.
+//  (2) They also doesn't allow you to return 'void' from a function.
+//
+// Workaround for (1): Declare a dummy type 'DefaultVoid' which we use
+//   when we'd like to use 'void'. We convert it into 'void' and back
+//   using the templates DefaultVoidToVoid<> and VoidToDefaultVoid<>.
+// Workaround for (2): On VC6, the code for calling a void function is
+//   identical to the code for calling a non-void function in which the
+//   return value is never used, provided the return value is returned
+//   in the EAX register, rather than on the stack.
+//   This is true for most fundamental types such as int, enum, void *.
+//   Const void * is the safest option since it doesn't participate
+//   in any automatic conversions. But on a 16-bit compiler it might
+//   cause extra code to be generated, so we disable it for all compilers
+//   except for VC6 (and VC5).
+#ifdef FASTDLGT_VC6
+// VC6 workaround
+typedef const void* DefaultVoid;
+#else
+// On any other compiler, just use a normal void.
+typedef void DefaultVoid;
+#endif
+
+// Translate from 'DefaultVoid' to 'void'.
+// Everything else is unchanged
+template <class T>
+struct DefaultVoidToVoid {
+  typedef T type;
+};
+
+template <>
+struct DefaultVoidToVoid<DefaultVoid> {
+  typedef void type;
+};
+
+// Translate from 'void' into 'DefaultVoid'
+// Everything else is unchanged
+template <class T>
+struct VoidToDefaultVoid {
+  typedef T type;
+};
+
+template <>
+struct VoidToDefaultVoid<void> {
+  typedef DefaultVoid type;
+};
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Fast Delegates, part 1:
+//
+//      Conversion of member function pointer to a standard form
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// GenericClass is a fake class, ONLY used to provide a type.
+// It is vitally important that it is never defined, so that the compiler
+// doesn't
+// think it can optimize the invocation. For example, Borland generates simpler
+// code if it knows the class only uses single inheritance.
+
+// Compilers using Microsoft's structure need to be treated as a special case.
+#ifdef FASTDLGT_MICROSOFT_MFP
+
+#ifdef FASTDLGT_HASINHERITANCE_KEYWORDS
+// For Microsoft and Intel, we want to ensure that it's the most efficient type
+// of MFP
+// (4 bytes), even when the /vmg option is used. Declaring an empty class
+// would give 16 byte pointers in this case....
+class __single_inheritance GenericClass;
+#endif
+// ...but for Codeplay, an empty class *always* gives 4 byte pointers.
+// If compiled with the /clr option ("managed C++"), the JIT compiler thinks
+// it needs to load GenericClass before it can call any of its functions,
+// (compiles OK but crashes at runtime!), so we need to declare an
+// empty class to make it happy.
+// Codeplay and VC4 can't cope with the unknown_inheritance case either.
+class GenericClass {};
+#else
+class GenericClass;
+#endif
+
+// The size of a single inheritance member function pointer.
+const int SINGLE_MEMFUNCPTR_SIZE = sizeof(void (GenericClass::*)());
+
+//                      SimplifyMemFunc< >::Convert()
+//
+//  A template function that converts an arbitrary member function pointer into
+//  the
+//  simplest possible form of member function pointer, using a supplied 'this'
+//  pointer.
+//  According to the standard, this can be done legally with reinterpret_cast<>.
+//  For (non-standard) compilers which use member function pointers which vary
+//  in size
+//  depending on the class, we need to use  knowledge of the internal structure
+//  of a
+//  member function pointer, as used by the compiler. Template specialization is
+//  used
+//  to distinguish between the sizes. Because some compilers don't support
+//  partial
+//  template specialisation, I use full specialisation of a wrapper struct.
+
+// general case -- don't know how to convert it. Force a compile failure
+template <int N>
+struct SimplifyMemFunc {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+    // Unsupported member function type -- force a compile failure.
+    // (it's illegal to have a array with negative size).
+    // typedef char ERROR_Unsupported_member_function_pointer_on_this_compiler[N
+    // - 100];
+    return 0;
+  }
+};
+
+// For compilers where all member func ptrs are the same size, everything goes
+// here.
+// For non-standard compilers, only single_inheritance classes go here.
+template <>
+struct SimplifyMemFunc<SINGLE_MEMFUNCPTR_SIZE> {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+#if defined __DMC__
+    // Digital Mars doesn't allow you to cast between abitrary PMF's,
+    // even though the standard says you can. The 32-bit compiler lets you
+    // static_cast through an int, but the DOS compiler doesn't.
+    bound_func = horrible_cast<GenericMemFuncType>(function_to_bind);
+#else
+    bound_func = reinterpret_cast<GenericMemFuncType>(function_to_bind);
+#endif
+    return reinterpret_cast<GenericClass*>(pthis);
+  }
+};
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Fast Delegates, part 1b:
+//
+//                  Workarounds for Microsoft and Intel
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// Compilers with member function pointers which violate the standard (MSVC,
+// Intel, Codeplay),
+// need to be treated as a special case.
+#ifdef FASTDLGT_MICROSOFT_MFP
+
+// We use unions to perform horrible_casts. I would like to use #pragma
+// pack(push, 1)
+// at the start of each function for extra safety, but VC6 seems to ICE
+// intermittently if you do this inside a template.
+
+// __multiple_inheritance classes go here
+// Nasty hack for Microsoft and Intel (IA32 and Itanium)
+template <>
+struct SimplifyMemFunc<SINGLE_MEMFUNCPTR_SIZE + sizeof(int)> {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+    // We need to use a horrible_cast to do this conversion.
+    // In MSVC, a multiple inheritance member pointer is internally defined as:
+    union {
+      XFuncType func;
+      struct {
+        GenericMemFuncType funcaddress;  // points to the actual member function
+        int delta;  // #BYTES to be added to the 'this' pointer
+      } s;
+    } u;
+    // Check that the horrible_cast will work
+    typedef int ERROR_CantUsehorrible_cast
+        [sizeof(function_to_bind) == sizeof(u.s) ? 1 : -1];
+    u.func = function_to_bind;
+    bound_func = u.s.funcaddress;
+    return reinterpret_cast<GenericClass*>(reinterpret_cast<char*>(pthis) +
+                                           u.s.delta);
+  }
+};
+
+// virtual inheritance is a real nuisance. It's inefficient and complicated.
+// On MSVC and Intel, there isn't enough information in the pointer itself to
+// enable conversion to a closure pointer. Earlier versions of this code didn't
+// work for all cases, and generated a compile-time error instead.
+// But a very clever hack invented by John M. Dlugosz solves this problem.
+// My code is somewhat different to his: I have no asm code, and I make no
+// assumptions about the calling convention that is used.
+
+// In VC++ and ICL, a virtual_inheritance member pointer
+// is internally defined as:
+struct MicrosoftVirtualMFP {
+  void (GenericClass::*codeptr)();  // points to the actual member function
+  int delta;                        // #bytes to be added to the 'this' pointer
+  int vtable_index;                 // or 0 if no virtual inheritance
+};
+// The CRUCIAL feature of Microsoft/Intel MFPs which we exploit is that the
+// m_codeptr member is *always* called, regardless of the values of the other
+// members. (This is *not* true for other compilers, eg GCC, which obtain the
+// function address from the vtable if a virtual function is being called).
+// Dlugosz's trick is to make the codeptr point to a probe function which
+// returns the 'this' pointer that was used.
+
+// Define a generic class that uses virtual inheritance.
+// It has a trival member function that returns the value of the 'this' pointer.
+struct GenericVirtualClass : virtual public GenericClass {
+  typedef GenericVirtualClass* (GenericVirtualClass::*ProbePtrType)();
+  GenericVirtualClass* GetThis() { return this; }
+};
+
+// __virtual_inheritance classes go here
+template <>
+struct SimplifyMemFunc<SINGLE_MEMFUNCPTR_SIZE + 2 * sizeof(int)> {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+    union {
+      XFuncType func;
+      GenericClass* (X::*ProbeFunc)();
+      MicrosoftVirtualMFP s;
+    } u;
+    u.func = function_to_bind;
+    bound_func = reinterpret_cast<GenericMemFuncType>(u.s.codeptr);
+    union {
+      GenericVirtualClass::ProbePtrType virtfunc;
+      MicrosoftVirtualMFP s;
+    } u2;
+    // Check that the horrible_cast<>s will work
+    typedef int
+        ERROR_CantUsehorrible_cast[sizeof(function_to_bind) == sizeof(u.s) &&
+                                           sizeof(function_to_bind) ==
+                                               sizeof(u.ProbeFunc) &&
+                                           sizeof(u2.virtfunc) == sizeof(u2.s)
+                                       ? 1
+                                       : -1];
+    // Unfortunately, taking the address of a MF prevents it from being inlined,
+    // so
+    // this next line can't be completely optimised away by the compiler.
+    u2.virtfunc = &GenericVirtualClass::GetThis;
+    u.s.codeptr = u2.s.codeptr;
+    return (pthis->*u.ProbeFunc)();
+  }
+};
+
+#if (_MSC_VER < 1300)
+
+// Nasty hack for Microsoft Visual C++ 6.0
+// unknown_inheritance classes go here
+// There is a compiler bug in MSVC6 which generates incorrect code in this
+// case!!
+template <>
+struct SimplifyMemFunc<SINGLE_MEMFUNCPTR_SIZE + 3 * sizeof(int)> {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+    // There is an apalling but obscure compiler bug in MSVC6 and earlier:
+    // vtable_index and 'vtordisp' are always set to 0 in the
+    // unknown_inheritance case!
+    // This means that an incorrect function could be called!!!
+    // Compiling with the /vmg option leads to potentially incorrect code.
+    // This is probably the reason that the IDE has a user interface for
+    // specifying
+    // the /vmg option, but it is disabled -  you can only specify /vmg on
+    // the command line. In VC1.5 and earlier, the compiler would ICE if it ever
+    // encountered this situation.
+    // It is OK to use the /vmg option if /vmm or /vms is specified.
+
+    // Fortunately, the wrong function is only called in very obscure cases.
+    // It only occurs when a derived class overrides a virtual function declared
+    // in a virtual base class, and the member function
+    // points to the *Derived* version of that function. The problem can be
+    // completely averted in 100% of cases by using the *Base class* for the
+    // member fpointer. Ie, if you use the base class as an interface, you'll
+    // stay out of trouble.
+    // Occasionally, you might want to point directly to a derived class
+    // function
+    // that isn't an override of a base class. In this case, both vtable_index
+    // and 'vtordisp' are zero, but a virtual_inheritance pointer will be
+    // generated.
+    // We can generate correct code in this case. To prevent an incorrect call
+    // from
+    // ever being made, on MSVC6 we generate a warning, and call a function to
+    // make the program crash instantly.
+    typedef char ERROR_VC6CompilerBug[-100];
+    return 0;
+  }
+};
+
+#else
+
+// Nasty hack for Microsoft and Intel (IA32 and Itanium)
+// unknown_inheritance classes go here
+// This is probably the ugliest bit of code I've ever written. Look at the
+// casts!
+// There is a compiler bug in MSVC6 which prevents it from using this code.
+template <>
+struct SimplifyMemFunc<SINGLE_MEMFUNCPTR_SIZE + 3 * sizeof(int)> {
+  template <class X, class XFuncType, class GenericMemFuncType>
+  inline static GenericClass* Convert(X* pthis, XFuncType function_to_bind,
+                                      GenericMemFuncType& bound_func) {
+    // The member function pointer is 16 bytes long. We can't use a normal cast,
+    // but
+    // we can use a union to do the conversion.
+    union {
+      XFuncType func;
+      // In VC++ and ICL, an unknown_inheritance member pointer
+      // is internally defined as:
+      struct {
+        GenericMemFuncType
+            m_funcaddress;  // points to the actual member function
+        int delta;          // #bytes to be added to the 'this' pointer
+        int vtordisp;       // #bytes to add to 'this' to find the vtable
+        int vtable_index;   // or 0 if no virtual inheritance
+      } s;
+    } u;
+    // Check that the horrible_cast will work
+    typedef int
+        ERROR_CantUsehorrible_cast[sizeof(XFuncType) == sizeof(u.s) ? 1 : -1];
+    u.func = function_to_bind;
+    bound_func = u.s.funcaddress;
+    int virtual_delta = 0;
+    if (u.s.vtable_index) {  // Virtual inheritance is used
+      // First, get to the vtable.
+      // It is 'vtordisp' bytes from the start of the class.
+      const int* vtable = *reinterpret_cast<const int* const*>(
+          reinterpret_cast<const char*>(pthis) + u.s.vtordisp);
+
+      // 'vtable_index' tells us where in the table we should be looking.
+      virtual_delta =
+          u.s.vtordisp +
+          *reinterpret_cast<const int*>(reinterpret_cast<const char*>(vtable) +
+                                        u.s.vtable_index);
+    }
+    // The int at 'virtual_delta' gives us the amount to add to 'this'.
+    // Finally we can add the three components together. Phew!
+    return reinterpret_cast<GenericClass*>(reinterpret_cast<char*>(pthis) +
+                                           u.s.delta + virtual_delta);
+  };
+};
+#endif  // MSVC 7 and greater
+
+#endif  // MS/Intel hacks
+
+}  // namespace detail
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Fast Delegates, part 2:
+//
+//  Define the delegate storage, and cope with static functions
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// DelegateMemento -- an opaque structure which can hold an arbitary delegate.
+// It knows nothing about the calling convention or number of arguments used by
+// the function pointed to.
+// It supplies comparison operators so that it can be stored in STL collections.
+// It cannot be set to anything other than null, nor invoked directly:
+//   it must be converted to a specific delegate.
+
+// Implementation:
+// There are two possible implementations: the Safe method and the Evil method.
+//              DelegateMemento - Safe version
+//
+// This implementation is standard-compliant, but a bit tricky.
+// A static function pointer is stored inside the class.
+// Here are the valid values:
+// +-- Static pointer --+--pThis --+-- pMemFunc-+-- Meaning------+
+// |   0                |  0       |   0        | Empty          |
+// |   !=0              |(dontcare)|  Invoker   | Static function|
+// |   0                |  !=0     |  !=0*      | Method call    |
+// +--------------------+----------+------------+----------------+
+//  * For Metrowerks, this can be 0. (first virtual function in a
+//       single_inheritance class).
+// When stored stored inside a specific delegate, the 'dontcare' entries are
+// replaced
+// with a reference to the delegate itself. This complicates the = and ==
+// operators
+// for the delegate class.
+
+//              DelegateMemento - Evil version
+//
+// For compilers where data pointers are at least as big as code pointers, it is
+// possible to store the function pointer in the this pointer, using another
+// horrible_cast. In this case the DelegateMemento implementation is simple:
+// +--pThis --+-- pMemFunc-+-- Meaning---------------------+
+// |    0     |  0         | Empty                         |
+// |  !=0     |  !=0*      | Static function or method call|
+// +----------+------------+-------------------------------+
+//  * For Metrowerks, this can be 0. (first virtual function in a
+//       single_inheritance class).
+// Note that the Sun C++ and MSVC documentation explicitly state that they
+// support static_cast between void * and function pointers.
+
+class DelegateMemento {
+ protected:
+  // the data is protected, not private, because many
+  // compilers have problems with template friends.
+  typedef void (detail::GenericClass::*GenericMemFuncType)();  // arbitrary MFP.
+  detail::GenericClass* m_pthis;
+  GenericMemFuncType m_pFunction;
+
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+  typedef void (*GenericFuncPtr)();  // arbitrary code pointer
+  GenericFuncPtr m_pStaticFunction;
+#endif
+
+ public:
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+  DelegateMemento() : m_pthis(0), m_pFunction(0), m_pStaticFunction(0){};
+  void clear() {
+    m_pthis = 0;
+    m_pFunction = 0;
+    m_pStaticFunction = 0;
+  }
+#else
+  DelegateMemento() : m_pthis(0), m_pFunction(0){};
+  void clear() {
+    m_pthis = 0;
+    m_pFunction = 0;
+  }
+#endif
+ public:
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+  inline bool IsEqual(const DelegateMemento& x) const {
+    // We have to cope with the static function pointers as a special case
+    if (m_pFunction != x.m_pFunction) return false;
+    // the static function ptrs must either both be equal, or both be 0.
+    if (m_pStaticFunction != x.m_pStaticFunction) return false;
+    if (m_pStaticFunction != 0)
+      return m_pthis == x.m_pthis;
+    else
+      return true;
+  }
+#else  // Evil Method
+  inline bool IsEqual(const DelegateMemento& x) const {
+    return m_pthis == x.m_pthis && m_pFunction == x.m_pFunction;
+  }
+#endif
+  // Provide a strict weak ordering for DelegateMementos.
+  inline bool IsLess(const DelegateMemento& right) const {
+// deal with static function pointers first
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+    if (m_pStaticFunction != 0 || right.m_pStaticFunction != 0)
+      return m_pStaticFunction < right.m_pStaticFunction;
+#endif
+    if (m_pthis != right.m_pthis) return m_pthis < right.m_pthis;
+    // There are no ordering operators for member function pointers,
+    // but we can fake one by comparing each byte. The resulting ordering is
+    // arbitrary (and compiler-dependent), but it permits storage in ordered STL
+    // containers.
+    return memcmp(&m_pFunction, &right.m_pFunction, sizeof(m_pFunction)) < 0;
+  }
+  // BUGFIX (Mar 2005):
+  // We can't just compare m_pFunction because on Metrowerks,
+  // m_pFunction can be zero even if the delegate is not empty!
+  inline bool operator!() const {  // Is it bound to anything?
+    return m_pthis == 0 && m_pFunction == 0;
+  }
+  inline bool empty() const {  // Is it bound to anything?
+    return m_pthis == 0 && m_pFunction == 0;
+  }
+
+ public:
+  DelegateMemento& operator=(const DelegateMemento& right) {
+    SetMementoFrom(right);
+    return *this;
+  }
+  inline bool operator<(const DelegateMemento& right) { return IsLess(right); }
+  inline bool operator>(const DelegateMemento& right) {
+    return right.IsLess(*this);
+  }
+  DelegateMemento(const DelegateMemento& right)
+      : m_pthis(right.m_pthis),
+        m_pFunction(right.m_pFunction)
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+        ,
+        m_pStaticFunction(right.m_pStaticFunction)
+#endif
+  {
+  }
+
+ protected:
+  void SetMementoFrom(const DelegateMemento& right) {
+    m_pFunction = right.m_pFunction;
+    m_pthis = right.m_pthis;
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+    m_pStaticFunction = right.m_pStaticFunction;
+#endif
+  }
+};
+
+//                      ClosurePtr<>
+//
+// A private wrapper class that adds function signatures to DelegateMemento.
+// It's the class that does most of the actual work.
+// The signatures are specified by:
+// GenericMemFunc: must be a type of GenericClass member function pointer.
+// StaticFuncPtr:  must be a type of function pointer with the same signature
+//                 as GenericMemFunc.
+// UnvoidStaticFuncPtr: is the same as StaticFuncPtr, except on VC6
+//                 where it never returns void (returns DefaultVoid instead).
+
+// An outer class, FastDelegateN<>, handles the invoking and creates the
+// necessary typedefs.
+// This class does everything else.
+
+namespace detail {
+
+template <class GenericMemFunc, class StaticFuncPtr, class UnvoidStaticFuncPtr>
+class ClosurePtr : public DelegateMemento {
+ public:
+  // These functions are for setting the delegate to a member function.
+
+  // Here's the clever bit: we convert an arbitrary member function into a
+  // standard form. XMemFunc should be a member function of class X, but I can't
+  // enforce that here. It needs to be enforced by the wrapper class.
+  template <class X, class XMemFunc>
+  inline void bindmemfunc(X* pthis, XMemFunc function_to_bind) {
+    m_pthis = SimplifyMemFunc<sizeof(function_to_bind)>::Convert(
+        pthis, function_to_bind, m_pFunction);
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+    m_pStaticFunction = 0;
+#endif
+  }
+  // For const member functions, we only need a const class pointer.
+  // Since we know that the member function is const, it's safe to
+  // remove the const qualifier from the 'this' pointer with a const_cast.
+  // VC6 has problems if we just overload 'bindmemfunc', so we give it a
+  // different name.
+  template <class X, class XMemFunc>
+  inline void bindconstmemfunc(const X* pthis, XMemFunc function_to_bind) {
+    m_pthis = SimplifyMemFunc<sizeof(function_to_bind)>::Convert(
+        const_cast<X*>(pthis), function_to_bind, m_pFunction);
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+    m_pStaticFunction = 0;
+#endif
+  }
+#ifdef FASTDELEGATE_GCC_BUG_8271  // At present, GCC doesn't recognize constness
+                                  // of MFPs in templates
+  template <class X, class XMemFunc>
+  inline void bindmemfunc(const X* pthis, XMemFunc function_to_bind) {
+    bindconstmemfunc(pthis, function_to_bind);
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+    m_pStaticFunction = 0;
+#endif
+  }
+#endif
+  // These functions are required for invoking the stored function
+  inline GenericClass* GetClosureThis() const { return m_pthis; }
+  inline GenericMemFunc GetClosureMemPtr() const {
+    return reinterpret_cast<GenericMemFunc>(m_pFunction);
+  }
+
+// There are a few ways of dealing with static function pointers.
+// There's a standard-compliant, but tricky method.
+// There's also a straightforward hack, that won't work on DOS compilers using
+// the
+// medium memory model. It's so evil that I can't recommend it, but I've
+// implemented it anyway because it produces very nice asm code.
+
+#if !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+
+  //              ClosurePtr<> - Safe version
+  //
+  // This implementation is standard-compliant, but a bit tricky.
+  // I store the function pointer inside the class, and the delegate then
+  // points to itself. Whenever the delegate is copied, these self-references
+  // must be transformed, and this complicates the = and == operators.
+ public:
+  // The next two functions are for operator ==, =, and the copy constructor.
+  // We may need to convert the m_pthis pointers, so that
+  // they remain as self-references.
+  template <class DerivedClass>
+  inline void CopyFrom(DerivedClass* pParent, const DelegateMemento& x) {
+    SetMementoFrom(x);
+    if (m_pStaticFunction != 0) {
+      // transform self references...
+      m_pthis = reinterpret_cast<GenericClass*>(pParent);
+    }
+  }
+  // For static functions, the 'static_function_invoker' class in the parent
+  // will be called. The parent then needs to call GetStaticFunction() to find
+  // out
+  // the actual function to invoke.
+  template <class DerivedClass, class ParentInvokerSig>
+  inline void bindstaticfunc(DerivedClass* pParent,
+                             ParentInvokerSig static_function_invoker,
+                             StaticFuncPtr function_to_bind) {
+    if (function_to_bind == 0) {  // cope with assignment to 0
+      m_pFunction = 0;
+    } else {
+      bindmemfunc(pParent, static_function_invoker);
+    }
+    m_pStaticFunction = reinterpret_cast<GenericFuncPtr>(function_to_bind);
+  }
+  inline UnvoidStaticFuncPtr GetStaticFunction() const {
+    return reinterpret_cast<UnvoidStaticFuncPtr>(m_pStaticFunction);
+  }
+#else
+
+  //              ClosurePtr<> - Evil version
+  //
+  // For compilers where data pointers are at least as big as code pointers, it
+  // is
+  // possible to store the function pointer in the this pointer, using another
+  // horrible_cast. Invocation isn't any faster, but it saves 4 bytes, and
+  // speeds up comparison and assignment. If C++ provided direct language
+  // support
+  // for delegates, they would produce asm code that was almost identical to
+  // this.
+  // Note that the Sun C++ and MSVC documentation explicitly state that they
+  // support static_cast between void * and function pointers.
+
+  template <class DerivedClass>
+  inline void CopyFrom(DerivedClass* pParent, const DelegateMemento& right) {
+    SetMementoFrom(right);
+  }
+  // For static functions, the 'static_function_invoker' class in the parent
+  // will be called. The parent then needs to call GetStaticFunction() to find
+  // out
+  // the actual function to invoke.
+  // ******** EVIL, EVIL CODE! *******
+  template <class DerivedClass, class ParentInvokerSig>
+  inline void bindstaticfunc(DerivedClass* pParent,
+                             ParentInvokerSig static_function_invoker,
+                             StaticFuncPtr function_to_bind) {
+    if (function_to_bind == 0) {  // cope with assignment to 0
+      m_pFunction = 0;
+    } else {
+      // We'll be ignoring the 'this' pointer, but we need to make sure we pass
+      // a valid value to bindmemfunc().
+      bindmemfunc(pParent, static_function_invoker);
+    }
+
+    // WARNING! Evil hack. We store the function in the 'this' pointer!
+    // Ensure that there's a compilation failure if function pointers
+    // and data pointers have different sizes.
+    // If you get this error, you need to #undef
+    // FASTDELEGATE_USESTATICFUNCTIONHACK.
+    // typedef int ERROR_CantUseEvilMethod[sizeof(GenericClass*) ==
+    // sizeof(function_to_bind) ? 1 : -1];
+    m_pthis = horrible_cast<GenericClass*>(function_to_bind);
+    // MSVC, SunC++ and DMC accept the following (non-standard) code:
+    //      m_pthis = static_cast<GenericClass *>(static_cast<void
+    //      *>(function_to_bind));
+    // BCC32, Comeau and DMC accept this method. MSVC7.1 needs __int64 instead
+    // of long
+    //      m_pthis = reinterpret_cast<GenericClass
+    //      *>(reinterpret_cast<long>(function_to_bind));
+  }
+  // ******** EVIL, EVIL CODE! *******
+  // This function will be called with an invalid 'this' pointer!!
+  // We're just returning the 'this' pointer, converted into
+  // a function pointer!
+  inline UnvoidStaticFuncPtr GetStaticFunction() const {
+    // Ensure that there's a compilation failure if function pointers
+    // and data pointers have different sizes.
+    // If you get this error, you need to #undef
+    // FASTDELEGATE_USESTATICFUNCTIONHACK.
+    // typedef int ERROR_CantUseEvilMethod[sizeof(UnvoidStaticFuncPtr) ==
+    // sizeof(this) ? 1 : -1];
+    return horrible_cast<UnvoidStaticFuncPtr>(this);
+  }
+#endif  // !defined(FASTDELEGATE_USESTATICFUNCTIONHACK)
+
+  // Does the closure contain this static function?
+  inline bool IsEqualToStaticFuncPtr(StaticFuncPtr funcptr) {
+    if (funcptr == 0) return empty();
+    // For the Evil method, if it doesn't actually contain a static function,
+    // this will return an arbitrary
+    // value that is not equal to any valid function pointer.
+    else
+      return funcptr == reinterpret_cast<StaticFuncPtr>(GetStaticFunction());
+  }
+};
+
+}  // namespace detail
+
+////////////////////////////////////////////////////////////////////////////////
+//                      Fast Delegates, part 3:
+//
+//              Wrapper classes to ensure type safety
+//
+////////////////////////////////////////////////////////////////////////////////
+
+// Once we have the member function conversion templates, it's easy to make the
+// wrapper classes. So that they will work with as many compilers as possible,
+// the classes are of the form
+//   FastDelegate3<int, char *, double>
+// They can cope with any combination of parameters. The max number of
+// parameters
+// allowed is 8, but it is trivial to increase this limit.
+// Note that we need to treat const member functions seperately.
+// All this class does is to enforce type safety, and invoke the delegate with
+// the correct list of parameters.
+
+// Because of the weird rule about the class of derived member function
+// pointers,
+// you sometimes need to apply a downcast to the 'this' pointer.
+// This is the reason for the use of "implicit_cast<X*>(pthis)" in the code
+// below.
+// If CDerivedClass is derived from CBaseClass, but doesn't override
+// SimpleVirtualFunction,
+// without this trick you'd need to write:
+//      MyDelegate(static_cast<CBaseClass *>(&d),
+//      &CDerivedClass::SimpleVirtualFunction);
+// but with the trick you can write
+//      MyDelegate(&d, &CDerivedClass::SimpleVirtualFunction);
+
+// RetType is the type the compiler uses in compiling the template. For VC6,
+// it cannot be void. DesiredRetType is the real type which is returned from
+// all of the functions. It can be void.
+
+// Implicit conversion to "bool" is achieved using the safe_bool idiom,
+// using member data pointers (MDP). This allows "if (dg)..." syntax
+// Because some compilers (eg codeplay) don't have a unique value for a zero
+// MDP, an extra padding member is added to the SafeBool struct.
+// Some compilers (eg VC6) won't implicitly convert from 0 to an MDP, so
+// in that case the static function constructor is not made explicit; this
+// allows "if (dg==0) ..." to compile.
+
+// N=0
+template <class RetType = detail::DefaultVoid>
+class FastDelegate0 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)();
+  typedef RetType (*UnvoidStaticFunctionPtr)();
+  typedef RetType (detail::GenericClass::*GenericMemFn)();
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate0 type;
+
+  // Construction and comparison functions
+  FastDelegate0() { clear(); }
+  FastDelegate0(const FastDelegate0& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate0& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate0& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate0& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate0& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate0& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate0(Y* pthis, DesiredRetType (X::*function_to_bind)()) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)()) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate0(const Y* pthis, DesiredRetType (X::*function_to_bind)() const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)() const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate0(DesiredRetType (*function_to_bind)()) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)()) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)()) {
+    m_Closure.bindstaticfunc(this, &FastDelegate0::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()() const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))();
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction() const {
+    return (*(m_Closure.GetStaticFunction()))();
+  }
+};
+
+// N=1
+template <class Param1, class RetType = detail::DefaultVoid>
+class FastDelegate1 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate1 type;
+
+  // Construction and comparison functions
+  FastDelegate1() { clear(); }
+  FastDelegate1(const FastDelegate1& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate1& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate1& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate1& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate1& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate1& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate1(Y* pthis, DesiredRetType (X::*function_to_bind)(Param1 p1)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(Param1 p1)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate1(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate1(DesiredRetType (*function_to_bind)(Param1 p1)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate1::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(p1);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1) const {
+    return (*(m_Closure.GetStaticFunction()))(p1);
+  }
+};
+
+// N=2
+template <class Param1, class Param2, class RetType = detail::DefaultVoid>
+class FastDelegate2 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate2 type;
+
+  // Construction and comparison functions
+  FastDelegate2() { clear(); }
+  FastDelegate2(const FastDelegate2& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate2& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate2& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate2& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate2& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate2& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate2(Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(Param1 p1,
+                                                                   Param2 p2)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate2(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2)
+                    const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2)
+                       const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate2(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate2::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(p1,
+                                                                         p2);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2);
+  }
+};
+
+// N=3
+template <class Param1, class Param2, class Param3,
+          class RetType = detail::DefaultVoid>
+class FastDelegate3 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate3 type;
+
+  // Construction and comparison functions
+  FastDelegate3() { clear(); }
+  FastDelegate3(const FastDelegate3& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate3& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate3& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate3& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate3& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate3& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate3(Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate3(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate3(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                   Param3 p3)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                    Param3 p3)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate3::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2, Param3 p3) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(p1, p2,
+                                                                         p3);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2, Param3 p3) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2, p3);
+  }
+};
+
+// N=4
+template <class Param1, class Param2, class Param3, class Param4,
+          class RetType = detail::DefaultVoid>
+class FastDelegate4 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                              Param4 p4);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                             Param4 p4);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3, Param4 p4);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate4 type;
+
+  // Construction and comparison functions
+  FastDelegate4() { clear(); }
+  FastDelegate4(const FastDelegate4& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate4& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate4& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate4& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate4& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate4& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate4(Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                                 Param1 p1, Param2 p2, Param3 p3, Param4 p4)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate4(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4)
+                    const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4)
+                       const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate4(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                   Param3 p3, Param4 p4)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                    Param3 p3, Param4 p4)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate4::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2, Param3 p3, Param4 p4) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(
+        p1, p2, p3, p4);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2, Param3 p3,
+                               Param4 p4) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2, p3, p4);
+  }
+};
+
+// N=5
+template <class Param1, class Param2, class Param3, class Param4, class Param5,
+          class RetType = detail::DefaultVoid>
+class FastDelegate5 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                              Param4 p4, Param5 p5);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                             Param4 p4, Param5 p5);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3, Param4 p4,
+                                                        Param5 p5);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate5 type;
+
+  // Construction and comparison functions
+  FastDelegate5() { clear(); }
+  FastDelegate5(const FastDelegate5& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate5& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate5& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate5& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate5& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate5& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate5(Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4,
+                                                         Param5 p5)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate5(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4,
+                                                         Param5 p5) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate5(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                   Param3 p3, Param4 p4,
+                                                   Param5 p5)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                    Param3 p3, Param4 p4,
+                                                    Param5 p5)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate5::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                     Param5 p5) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(
+        p1, p2, p3, p4, p5);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                               Param5 p5) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2, p3, p4, p5);
+  }
+};
+
+// N=6
+template <class Param1, class Param2, class Param3, class Param4, class Param5,
+          class Param6, class RetType = detail::DefaultVoid>
+class FastDelegate6 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                              Param4 p4, Param5 p5, Param6 p6);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                             Param4 p4, Param5 p5, Param6 p6);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3, Param4 p4,
+                                                        Param5 p5, Param6 p6);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate6 type;
+
+  // Construction and comparison functions
+  FastDelegate6() { clear(); }
+  FastDelegate6(const FastDelegate6& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate6& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate6& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate6& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate6& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate6& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate6(Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5, Param6 p6)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                                 Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                                 Param5 p5, Param6 p6)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate6(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5, Param6 p6)
+                    const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4,
+                                                         Param5 p5, Param6 p6)
+                       const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate6(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                   Param3 p3, Param4 p4,
+                                                   Param5 p5, Param6 p6)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                    Param3 p3, Param4 p4,
+                                                    Param5 p5, Param6 p6)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5, Param6 p6)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate6::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2, Param3 p3, Param4 p4, Param5 p5,
+                     Param6 p6) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(
+        p1, p2, p3, p4, p5, p6);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                               Param5 p5, Param6 p6) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2, p3, p4, p5, p6);
+  }
+};
+
+// N=7
+template <class Param1, class Param2, class Param3, class Param4, class Param5,
+          class Param6, class Param7, class RetType = detail::DefaultVoid>
+class FastDelegate7 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                              Param4 p4, Param5 p5, Param6 p6,
+                                              Param7 p7);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                             Param4 p4, Param5 p5, Param6 p6,
+                                             Param7 p7);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3, Param4 p4,
+                                                        Param5 p5, Param6 p6,
+                                                        Param7 p7);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate7 type;
+
+  // Construction and comparison functions
+  FastDelegate7() { clear(); }
+  FastDelegate7(const FastDelegate7& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate7& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate7& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate7& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate7& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate7& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate7(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                              Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                              Param5 p5, Param6 p6, Param7 p7)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                                 Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                                 Param5 p5, Param6 p6, Param7 p7)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate7(const Y* pthis, DesiredRetType (X::*function_to_bind)(
+                                    Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                                    Param5 p5, Param6 p6, Param7 p7) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4,
+                                                         Param5 p5, Param6 p6,
+                                                         Param7 p7) const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member function call.
+  // This constructor also provides implicit conversion
+  FastDelegate7(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                   Param3 p3, Param4 p4,
+                                                   Param5 p5, Param6 p6,
+                                                   Param7 p7)) {
+    bind(function_to_bind);
+  }
+  // for efficiency, prevent creation of a temporary
+  void operator=(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                    Param3 p3, Param4 p4,
+                                                    Param5 p5, Param6 p6,
+                                                    Param7 p7)) {
+    bind(function_to_bind);
+  }
+  inline void bind(DesiredRetType (*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5, Param6 p6,
+                                                      Param7 p7)) {
+    m_Closure.bindstaticfunc(this, &FastDelegate7::InvokeStaticFunction,
+                             function_to_bind);
+  }
+  // Invoke the delegate
+  RetType operator()(Param1 p1, Param2 p2, Param3 p3, Param4 p4, Param5 p5,
+                     Param6 p6, Param7 p7) const {
+    return (m_Closure.GetClosureThis()->*(m_Closure.GetClosureMemPtr()))(
+        p1, p2, p3, p4, p5, p6, p7);
+  }
+  // Implicit conversion to "bool" using the safe_bool idiom
+ private:
+  typedef struct SafeBoolStruct {
+    int a_data_pointer_to_this_is_0_on_buggy_compilers;
+    StaticFunctionPtr m_nonzero;
+  } UselessTypedef;
+  typedef StaticFunctionPtr SafeBoolStruct::*unspecified_bool_type;
+
+ public:
+  operator unspecified_bool_type() const {
+    return empty() ? 0 : &SafeBoolStruct::m_nonzero;
+  }
+  // necessary to allow ==0 to work despite the safe_bool idiom
+  inline bool operator==(StaticFunctionPtr funcptr) {
+    return m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!=(StaticFunctionPtr funcptr) {
+    return !m_Closure.IsEqualToStaticFuncPtr(funcptr);
+  }
+  inline bool operator!() const {  // Is it bound to anything?
+    return !m_Closure;
+  }
+  inline bool empty() const { return !m_Closure; }
+  void clear() { m_Closure.clear(); }
+  // Conversion to and from the DelegateMemento storage class
+  const DelegateMemento& GetMemento() { return m_Closure; }
+  void SetMemento(const DelegateMemento& any) { m_Closure.CopyFrom(this, any); }
+
+ private:  // Invoker for static functions
+  RetType InvokeStaticFunction(Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                               Param5 p5, Param6 p6, Param7 p7) const {
+    return (*(m_Closure.GetStaticFunction()))(p1, p2, p3, p4, p5, p6, p7);
+  }
+};
+
+// N=8
+template <class Param1, class Param2, class Param3, class Param4, class Param5,
+          class Param6, class Param7, class Param8,
+          class RetType = detail::DefaultVoid>
+class FastDelegate8 {
+ private:
+  typedef typename detail::DefaultVoidToVoid<RetType>::type DesiredRetType;
+  typedef DesiredRetType (*StaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                              Param4 p4, Param5 p5, Param6 p6,
+                                              Param7 p7, Param8 p8);
+  typedef RetType (*UnvoidStaticFunctionPtr)(Param1 p1, Param2 p2, Param3 p3,
+                                             Param4 p4, Param5 p5, Param6 p6,
+                                             Param7 p7, Param8 p8);
+  typedef RetType (detail::GenericClass::*GenericMemFn)(Param1 p1, Param2 p2,
+                                                        Param3 p3, Param4 p4,
+                                                        Param5 p5, Param6 p6,
+                                                        Param7 p7, Param8 p8);
+  typedef detail::ClosurePtr<GenericMemFn, StaticFunctionPtr,
+                             UnvoidStaticFunctionPtr>
+      ClosureType;
+  ClosureType m_Closure;
+
+ public:
+  // Typedefs to aid generic programming
+  typedef FastDelegate8 type;
+
+  // Construction and comparison functions
+  FastDelegate8() { clear(); }
+  FastDelegate8(const FastDelegate8& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  void operator=(const FastDelegate8& x) {
+    m_Closure.CopyFrom(this, x.m_Closure);
+  }
+  bool operator==(const FastDelegate8& x) const {
+    return m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator!=(const FastDelegate8& x) const {
+    return !m_Closure.IsEqual(x.m_Closure);
+  }
+  bool operator<(const FastDelegate8& x) const {
+    return m_Closure.IsLess(x.m_Closure);
+  }
+  bool operator>(const FastDelegate8& x) const {
+    return x.m_Closure.IsLess(m_Closure);
+  }
+  // Binding to non-const member functions
+  template <class X, class Y>
+  FastDelegate8(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                              Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                              Param5 p5, Param6 p6, Param7 p7, Param8 p8)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(Y* pthis, DesiredRetType (X::*function_to_bind)(
+                                 Param1 p1, Param2 p2, Param3 p3, Param4 p4,
+                                 Param5 p5, Param6 p6, Param7 p7, Param8 p8)) {
+    m_Closure.bindmemfunc(detail::implicit_cast<X*>(pthis), function_to_bind);
+  }
+  // Binding to const member functions.
+  template <class X, class Y>
+  FastDelegate8(const Y* pthis,
+                DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                      Param3 p3, Param4 p4,
+                                                      Param5 p5, Param6 p6,
+                                                      Param7 p7, Param8 p8)
+                    const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  template <class X, class Y>
+  inline void bind(const Y* pthis,
+                   DesiredRetType (X::*function_to_bind)(Param1 p1, Param2 p2,
+                                                         Param3 p3, Param4 p4,
+                                                         Param5 p5, Param6 p6,
+                                                         Param7 p7, Param8 p8)
+                       const) {
+    m_Closure.bindconstmemfunc(detail::implicit_cast<const X*>(pthis),
+                               function_to_bind);
+  }
+  // Static functions. We convert them into a member 

<TRUNCATED>

[10/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientFactory.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientFactory.cpp b/rocketmq-cpp/src/MQClientFactory.cpp
new file mode 100644
index 0000000..ab578ee
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientFactory.cpp
@@ -0,0 +1,1142 @@
+/*
+ * 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 "MQClientFactory.h"
+#include "ConsumerRunningInfo.h"
+#include "Logging.h"
+#include "MQClientManager.h"
+#include "PullRequest.h"
+#include "Rebalance.h"
+#include "TopicPublishInfo.h"
+
+#define MAX_BUFF_SIZE 8192
+#define SAFE_BUFF_SIZE 7936  // 8192 - 256 = 7936
+#define PROCESS_NAME_BUF_SIZE 256
+
+namespace rocketmq {
+//<!***************************************************************************
+MQClientFactory::MQClientFactory(const string& clientID, int pullThreadNum,
+                                 uint64_t tcpConnectTimeout,
+                                 uint64_t tcpTransportTryLockTimeout,
+                                 string unitName)
+    : m_bFetchNSService(true) {
+  m_clientId = clientID;
+  // default Topic register;
+  boost::shared_ptr<TopicPublishInfo> pDefaultTopicInfo(new TopicPublishInfo());
+  m_topicPublishInfoTable[DEFAULT_TOPIC] = pDefaultTopicInfo;
+  m_pClientRemotingProcessor.reset(new ClientRemotingProcessor(this));
+  m_pClientAPIImpl.reset(new MQClientAPIImpl(
+      m_clientId, m_pClientRemotingProcessor.get(), pullThreadNum, tcpConnectTimeout,
+      tcpTransportTryLockTimeout, unitName));
+  m_serviceState = CREATE_JUST;
+  LOG_DEBUG("MQClientFactory construct");
+}
+
+MQClientFactory::~MQClientFactory() {
+  LOG_INFO("MQClientFactory:%s destruct", m_clientId.c_str());
+
+  for (TRDMAP::iterator itp = m_topicRouteTable.begin();
+       itp != m_topicRouteTable.end(); ++itp) {
+    delete itp->second;
+  }
+
+  m_producerTable.clear();
+  m_consumerTable.clear();
+  m_topicRouteTable.clear();
+  m_brokerAddrTable.clear();
+  m_topicPublishInfoTable.clear();
+
+  m_pClientAPIImpl = NULL;
+}
+
+void MQClientFactory::start() {
+  switch (m_serviceState) {
+    case CREATE_JUST:
+      LOG_INFO("MQClientFactory:%s start", m_clientId.c_str());
+      m_serviceState = START_FAILED;
+      //<!start time task;
+      m_async_service_thread.reset(new boost::thread(boost::bind(
+          &MQClientFactory::startScheduledTask, this, m_bFetchNSService)));
+      m_serviceState = RUNNING;
+      break;
+    case RUNNING:
+    case SHUTDOWN_ALREADY:
+    case START_FAILED:
+      LOG_INFO("The Factory object:%s start failed with fault state:%d",
+               m_clientId.c_str(), m_serviceState);
+      break;
+    default:
+      break;
+  }
+}
+
+void MQClientFactory::updateTopicRouteInfo(boost::system::error_code& ec,
+                                           boost::asio::deadline_timer* t) {
+  if ((getConsumerTableSize() == 0) && (getProducerTableSize() == 0)) {
+    return;
+  }
+
+  set<string> topicList;
+  //<!Consumer;
+  getTopicListFromConsumerSubscription(topicList);
+
+  //<!Producer;
+  getTopicListFromTopicPublishInfo(topicList);
+
+  //<! update;
+  {
+    SessionCredentials session_credentials;
+    getSessionCredentialsFromOneOfProducerOrConsumer(session_credentials);
+    set<string>::iterator it = topicList.begin();
+    for (; it != topicList.end(); ++it) {
+      updateTopicRouteInfoFromNameServer(*it, session_credentials);
+    }
+  }
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::seconds(30), e);
+  t->async_wait(
+      boost::bind(&MQClientFactory::updateTopicRouteInfo, this, ec, t));
+}
+
+TopicRouteData* MQClientFactory::getTopicRouteData(const string& topic) {
+  boost::lock_guard<boost::mutex> lock(m_topicRouteTableMutex);
+  if (m_topicRouteTable.find(topic) != m_topicRouteTable.end()) {
+    return m_topicRouteTable[topic];
+  }
+  return NULL;
+}
+
+void MQClientFactory::addTopicRouteData(const string& topic,
+                                        TopicRouteData* pTopicRouteData) {
+  boost::lock_guard<boost::mutex> lock(m_topicRouteTableMutex);
+  if (m_topicRouteTable.find(topic) != m_topicRouteTable.end()) {
+    delete m_topicRouteTable[topic];
+    m_topicRouteTable.erase(topic);
+  }
+  m_topicRouteTable[topic] = pTopicRouteData;
+}
+
+boost::shared_ptr<TopicPublishInfo> MQClientFactory::tryToFindTopicPublishInfo(
+    const string& topic, const SessionCredentials& session_credentials) {
+  boost::lock_guard<boost::mutex> lock(
+      m_topicPublishInfoLock);  // add topicPublishInfoLock to avoid con-current
+                                // excuting updateTopicRouteInfoFromNameServer
+                                // when producer send msg  before topicRouteInfo
+                                // was got;
+  if (!isTopicInfoValidInTable(topic)) {
+    updateTopicRouteInfoFromNameServer(topic, session_credentials);
+  }
+  //<!if not exsit ,update dafult topic;
+  if (!isTopicInfoValidInTable(topic)) {
+    LOG_INFO("updateTopicRouteInfoFromNameServer with default");
+    updateTopicRouteInfoFromNameServer(topic, session_credentials, true);
+  }
+
+  if (!isTopicInfoValidInTable(topic)) {
+    LOG_WARN("tryToFindTopicPublishInfo null:%s", topic.c_str());
+    boost::shared_ptr<TopicPublishInfo> pTopicPublishInfo;
+    return pTopicPublishInfo;
+  }
+
+  return getTopicPublishInfoFromTable(topic);
+}
+
+bool MQClientFactory::updateTopicRouteInfoFromNameServer(
+    const string& topic, const SessionCredentials& session_credentials,
+    bool isDefault /* = false */) {
+  boost::lock_guard<boost::mutex> lock(m_factoryLock);
+  unique_ptr<TopicRouteData> pTopicRouteData;
+  LOG_INFO("updateTopicRouteInfoFromNameServer start:%s", topic.c_str());
+
+  if (isDefault) {
+    pTopicRouteData.reset(m_pClientAPIImpl->getTopicRouteInfoFromNameServer(
+        DEFAULT_TOPIC, 1000 * 5, session_credentials));
+    if (pTopicRouteData != NULL) {
+      vector<QueueData>& queueDatas = pTopicRouteData->getQueueDatas();
+      vector<QueueData>::iterator it = queueDatas.begin();
+      for (; it != queueDatas.end(); ++it) {
+        // ¶Áд·ÖÇø¸öÊýÊÇÒ»Ö£¬¹ÊÖ»×öÒ»´ÎÅжÏ;
+        int queueNums = std::min(4, it->readQueueNums);
+        it->readQueueNums = queueNums;
+        it->writeQueueNums = queueNums;
+      }
+    }
+  } else {
+    pTopicRouteData.reset(m_pClientAPIImpl->getTopicRouteInfoFromNameServer(
+        topic, 1000 * 5, session_credentials));
+  }
+
+  if (pTopicRouteData != NULL) {
+    LOG_INFO("updateTopicRouteInfoFromNameServer has data");
+    TopicRouteData* pTemp = getTopicRouteData(topic);
+    bool changed = true;
+    if (pTemp != NULL) {
+      changed = !(*pTemp == *pTopicRouteData);
+    }
+
+    if (getConsumerTableSize() > 0) {
+      vector<MQMessageQueue> mqs;
+      topicRouteData2TopicSubscribeInfo(topic, pTopicRouteData.get(), mqs);
+      updateConsumerSubscribeTopicInfo(topic, mqs);
+    }
+
+    if (changed) {
+      //<!update Broker addr
+      LOG_INFO("updateTopicRouteInfoFromNameServer changed:%s", topic.c_str());
+      vector<BrokerData> brokerList = pTopicRouteData->getBrokerDatas();
+      vector<BrokerData>::iterator it = brokerList.begin();
+      for (; it != brokerList.end(); ++it) {
+        LOG_INFO(
+            "updateTopicRouteInfoFromNameServer changed with broker name:%s",
+            (*it).brokerName.c_str());
+        addBrokerToAddrMap((*it).brokerName, (*it).brokerAddrs);
+      }
+
+      //<! update publish info;
+      {
+        boost::shared_ptr<TopicPublishInfo> publishInfo(
+            topicRouteData2TopicPublishInfo(topic, pTopicRouteData.get()));
+        addTopicInfoToTable(topic, publishInfo);  // erase first, then add
+      }
+
+      //<! update subscribe info
+      addTopicRouteData(topic, pTopicRouteData.release());
+    }
+    LOG_DEBUG("updateTopicRouteInfoFromNameServer end:%s", topic.c_str());
+    return true;
+  }
+  LOG_DEBUG("updateTopicRouteInfoFromNameServer end null:%s", topic.c_str());
+  return false;
+}
+
+boost::shared_ptr<TopicPublishInfo>
+MQClientFactory::topicRouteData2TopicPublishInfo(const string& topic,
+                                                 TopicRouteData* pRoute) {
+  boost::shared_ptr<TopicPublishInfo> info(new TopicPublishInfo());
+  string OrderTopicConf = pRoute->getOrderTopicConf();
+  //<! order msg
+  if (!OrderTopicConf.empty()) {
+    // "broker-a:8";"broker-b:8"
+    vector<string> brokers;
+    UtilAll::Split(brokers, OrderTopicConf, ';');
+    for (size_t i = 0; i < brokers.size(); i++) {
+      vector<string> item;
+      UtilAll::Split(item, brokers[i], ':');
+      int nums = atoi(item[1].c_str());
+      for (int i = 0; i < nums; i++) {
+        MQMessageQueue mq(topic, item[0], i);
+        info->updateMessageQueueList(mq);
+      }
+    }
+  }
+  //<!no order msg
+  else {
+    vector<QueueData>& queueDatas = pRoute->getQueueDatas();
+    vector<QueueData>::iterator it = queueDatas.begin();
+    for (; it != queueDatas.end(); ++it) {
+      QueueData& qd = (*it);
+      if (PermName::isWriteable(qd.perm)) {
+        string addr = findBrokerAddressInPublish(qd.brokerName);
+        if (addr.empty()) {
+          continue;
+        }
+        for (int i = 0; i < qd.writeQueueNums; i++) {
+          MQMessageQueue mq(topic, qd.brokerName, i);
+          info->updateMessageQueueList(mq);
+        }
+      }
+    }
+  }
+  return info;
+}
+
+void MQClientFactory::topicRouteData2TopicSubscribeInfo(
+    const string& topic, TopicRouteData* pRoute, vector<MQMessageQueue>& mqs) {
+  mqs.clear();
+  vector<QueueData>& queueDatas = pRoute->getQueueDatas();
+  vector<QueueData>::iterator it = queueDatas.begin();
+  for (; it != queueDatas.end(); ++it) {
+    QueueData& qd = (*it);
+    if (PermName::isReadable(qd.perm)) {
+      for (int i = 0; i < qd.readQueueNums; i++) {
+        MQMessageQueue mq(topic, qd.brokerName, i);
+        mqs.push_back(mq);
+      }
+    }
+  }
+}
+
+void MQClientFactory::shutdown() {
+  if (getConsumerTableSize() != 0) return;
+
+  if (getProducerTableSize() != 0) return;
+
+  switch (m_serviceState) {
+    case RUNNING: {
+      //<! stop;
+      if (m_consumer_async_service_thread) {
+        m_consumer_async_ioService.stop();
+        m_consumer_async_service_thread->interrupt();
+        m_consumer_async_service_thread->join();
+      }
+      m_async_ioService.stop();
+      m_async_service_thread->interrupt();
+      m_async_service_thread->join();
+      m_pClientAPIImpl->stopAllTcpTransportThread();  // Note: stop all
+                                                      // TcpTransport Threads
+                                                      // and release all
+                                                      // responseFuture
+                                                      // conditions
+      m_serviceState = SHUTDOWN_ALREADY;
+      LOG_INFO("MQClientFactory:%s shutdown", m_clientId.c_str());
+      break;
+    }
+    case SHUTDOWN_ALREADY:
+    case CREATE_JUST:
+      break;
+    default:
+      break;
+  }
+
+  //<!ɾ³ý×Ô¼º;
+  MQClientManager::getInstance()->removeClientFactory(m_clientId);
+}
+
+bool MQClientFactory::registerProducer(MQProducer* pProducer) {
+  string groupName = pProducer->getGroupName();
+  string namesrvaddr = pProducer->getNamesrvAddr();
+  if (groupName.empty()) {
+    return false;
+  }
+
+  if (!addProducerToTable(groupName, pProducer)) {
+    return false;
+  }
+
+  LOG_DEBUG("registerProducer success:%s", groupName.c_str());
+  //<!set nameserver;
+  if (namesrvaddr.empty()) {
+    string nameSrvDomain(pProducer->getNamesrvDomain());
+    if (!nameSrvDomain.empty()) m_nameSrvDomain = nameSrvDomain;
+    pProducer->setNamesrvAddr(
+        m_pClientAPIImpl->fetchNameServerAddr(m_nameSrvDomain));
+  } else {
+    m_bFetchNSService = false;
+    m_pClientAPIImpl->updateNameServerAddr(namesrvaddr);
+    LOG_INFO("user specfied name server address: %s", namesrvaddr.c_str());
+  }
+  return true;
+}
+
+void MQClientFactory::unregisterProducer(MQProducer* pProducer) {
+  string groupName = pProducer->getGroupName();
+  unregisterClient(groupName, "", pProducer->getSessionCredentials());
+
+  eraseProducerFromTable(groupName);
+}
+
+bool MQClientFactory::registerConsumer(MQConsumer* pConsumer) {
+  string groupName = pConsumer->getGroupName();
+  string namesrvaddr = pConsumer->getNamesrvAddr();
+  if (groupName.empty()) {
+    return false;
+  }
+
+  if (!addConsumerToTable(groupName, pConsumer)) {
+    return false;
+  }
+  LOG_DEBUG("registerConsumer success:%s", groupName.c_str());
+  //<!set nameserver;
+  if (namesrvaddr.empty()) {
+    string nameSrvDomain(pConsumer->getNamesrvDomain());
+    if (!nameSrvDomain.empty()) m_nameSrvDomain = nameSrvDomain;
+    pConsumer->setNamesrvAddr(
+        m_pClientAPIImpl->fetchNameServerAddr(m_nameSrvDomain));
+  } else {
+    m_bFetchNSService = false;
+    m_pClientAPIImpl->updateNameServerAddr(namesrvaddr);
+    LOG_INFO("user specfied name server address: %s", namesrvaddr.c_str());
+  }
+
+  return true;
+}
+
+void MQClientFactory::unregisterConsumer(MQConsumer* pConsumer) {
+  string groupName = pConsumer->getGroupName();
+  unregisterClient("", groupName, pConsumer->getSessionCredentials());
+
+  eraseConsumerFromTable(groupName);
+}
+
+MQProducer* MQClientFactory::selectProducer(const string& producerName) {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  if (m_producerTable.find(producerName) != m_producerTable.end()) {
+    return m_producerTable[producerName];
+  }
+  return NULL;
+}
+
+bool MQClientFactory::getSessionCredentialFromProducerTable(
+    SessionCredentials& sessionCredentials) {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  for (MQPMAP::iterator it = m_producerTable.begin();
+       it != m_producerTable.end(); ++it) {
+    if (it->second) sessionCredentials = it->second->getSessionCredentials();
+  }
+
+  if (sessionCredentials.isValid()) return true;
+
+  return false;
+}
+
+bool MQClientFactory::addProducerToTable(const string& producerName,
+                                         MQProducer* pMQProducer) {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  if (m_producerTable.find(producerName) != m_producerTable.end()) return false;
+  m_producerTable[producerName] = pMQProducer;
+  return true;
+}
+
+void MQClientFactory::eraseProducerFromTable(const string& producerName) {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  if (m_producerTable.find(producerName) != m_producerTable.end())
+    m_producerTable.erase(producerName);
+}
+
+int MQClientFactory::getProducerTableSize() {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  return m_producerTable.size();
+}
+
+void MQClientFactory::insertProducerInfoToHeartBeatData(
+    HeartbeatData* pHeartbeatData) {
+  boost::lock_guard<boost::mutex> lock(m_producerTableMutex);
+  for (MQPMAP::iterator it = m_producerTable.begin();
+       it != m_producerTable.end(); ++it) {
+    ProducerData producerData;
+    producerData.groupName = it->first;
+    pHeartbeatData->insertDataToProducerDataSet(producerData);
+  }
+}
+
+MQConsumer* MQClientFactory::selectConsumer(const string& group) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  if (m_consumerTable.find(group) != m_consumerTable.end()) {
+    return m_consumerTable[group];
+  }
+  return NULL;
+}
+
+bool MQClientFactory::getSessionCredentialFromConsumerTable(
+    SessionCredentials& sessionCredentials) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  for (MQCMAP::iterator it = m_consumerTable.begin();
+       it != m_consumerTable.end(); ++it) {
+    if (it->second) sessionCredentials = it->second->getSessionCredentials();
+  }
+
+  if (sessionCredentials.isValid()) return true;
+
+  return false;
+}
+
+bool MQClientFactory::getSessionCredentialFromConsumer(
+    const string& consumerGroup, SessionCredentials& sessionCredentials) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  if (m_consumerTable.find(consumerGroup) != m_consumerTable.end()) {
+    sessionCredentials =
+        m_consumerTable[consumerGroup]->getSessionCredentials();
+  }
+
+  if (sessionCredentials.isValid()) return true;
+
+  return false;
+}
+
+bool MQClientFactory::addConsumerToTable(const string& consumerName,
+                                         MQConsumer* pMQConsumer) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  if (m_consumerTable.find(consumerName) != m_consumerTable.end()) return false;
+  m_consumerTable[consumerName] = pMQConsumer;
+  return true;
+}
+
+void MQClientFactory::eraseConsumerFromTable(const string& consumerName) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  if (m_consumerTable.find(consumerName) != m_consumerTable.end())
+    m_consumerTable.erase(consumerName);  // do not need freee pConsumer, as it
+                                          // was allocated by user
+  else
+    LOG_WARN("could not find consumer:%s from table", consumerName.c_str());
+}
+
+int MQClientFactory::getConsumerTableSize() {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  return m_consumerTable.size();
+}
+
+void MQClientFactory::getTopicListFromConsumerSubscription(
+    set<string>& topicList) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  for (MQCMAP::iterator it = m_consumerTable.begin();
+       it != m_consumerTable.end(); ++it) {
+    vector<SubscriptionData> result;
+    it->second->getSubscriptions(result);
+
+    vector<SubscriptionData>::iterator iter = result.begin();
+    for (; iter != result.end(); ++iter) {
+      topicList.insert((*iter).getTopic());
+    }
+  }
+}
+
+void MQClientFactory::updateConsumerSubscribeTopicInfo(
+    const string& topic, vector<MQMessageQueue> mqs) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  for (MQCMAP::iterator it = m_consumerTable.begin();
+       it != m_consumerTable.end(); ++it) {
+    it->second->updateTopicSubscribeInfo(topic, mqs);
+  }
+}
+
+void MQClientFactory::insertConsumerInfoToHeartBeatData(
+    HeartbeatData* pHeartbeatData) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  for (MQCMAP::iterator it = m_consumerTable.begin();
+       it != m_consumerTable.end(); ++it) {
+    MQConsumer* pConsumer = it->second;
+    ConsumerData consumerData;
+    consumerData.groupName = pConsumer->getGroupName();
+    consumerData.consumeType = pConsumer->getConsumeType();
+    consumerData.messageModel = pConsumer->getMessageModel();
+    consumerData.consumeFromWhere = pConsumer->getConsumeFromWhere();
+
+    //<!fill data;
+    vector<SubscriptionData> result;
+    pConsumer->getSubscriptions(result);
+    consumerData.subscriptionDataSet.swap(result);
+
+    pHeartbeatData->insertDataToConsumerDataSet(consumerData);
+  }
+}
+
+void MQClientFactory::addTopicInfoToTable(
+    const string& topic,
+    boost::shared_ptr<TopicPublishInfo> pTopicPublishInfo) {
+  boost::lock_guard<boost::mutex> lock(m_topicPublishInfoTableMutex);
+  if (m_topicPublishInfoTable.find(topic) != m_topicPublishInfoTable.end()) {
+    m_topicPublishInfoTable.erase(topic);
+  }
+  m_topicPublishInfoTable[topic] = pTopicPublishInfo;
+}
+
+void MQClientFactory::eraseTopicInfoFromTable(const string& topic) {
+  boost::lock_guard<boost::mutex> lock(m_topicPublishInfoTableMutex);
+  if (m_topicPublishInfoTable.find(topic) != m_topicPublishInfoTable.end()) {
+    m_topicPublishInfoTable.erase(topic);
+  }
+}
+
+bool MQClientFactory::isTopicInfoValidInTable(const string& topic) {
+  boost::lock_guard<boost::mutex> lock(m_topicPublishInfoTableMutex);
+  if (m_topicPublishInfoTable.find(topic) != m_topicPublishInfoTable.end()) {
+    if (m_topicPublishInfoTable[topic]->ok()) return true;
+  }
+  return false;
+}
+
+boost::shared_ptr<TopicPublishInfo>
+MQClientFactory::getTopicPublishInfoFromTable(const string& topic) {
+  boost::lock_guard<boost::mutex> lock(m_topicPublishInfoTableMutex);
+  if (m_topicPublishInfoTable.find(topic) != m_topicPublishInfoTable.end()) {
+    return m_topicPublishInfoTable[topic];
+  }
+  boost::shared_ptr<TopicPublishInfo> pTopicPublishInfo;
+  return pTopicPublishInfo;
+}
+
+void MQClientFactory::getTopicListFromTopicPublishInfo(set<string>& topicList) {
+  boost::lock_guard<boost::mutex> lock(m_topicPublishInfoTableMutex);
+  for (TPMap::iterator itp = m_topicPublishInfoTable.begin();
+       itp != m_topicPublishInfoTable.end(); ++itp) {
+    topicList.insert(itp->first);
+  }
+}
+
+void MQClientFactory::clearBrokerAddrMap() {
+  boost::lock_guard<boost::mutex> lock(m_brokerAddrlock);
+  m_brokerAddrTable.clear();
+}
+
+void MQClientFactory::addBrokerToAddrMap(const string& brokerName,
+                                         map<int, string>& brokerAddrs) {
+  boost::lock_guard<boost::mutex> lock(m_brokerAddrlock);
+  if (m_brokerAddrTable.find(brokerName) != m_brokerAddrTable.end()) {
+    m_brokerAddrTable.erase(brokerName);
+  }
+  m_brokerAddrTable[brokerName] = brokerAddrs;
+}
+
+MQClientFactory::BrokerAddrMAP MQClientFactory::getBrokerAddrMap() {
+  boost::lock_guard<boost::mutex> lock(m_brokerAddrlock);
+  return m_brokerAddrTable;
+}
+
+string MQClientFactory::findBrokerAddressInPublish(const string& brokerName) {
+  /*reslove the concurrent access m_brokerAddrTable by
+  findBrokerAddressInPublish(called by sendKernlImpl) And
+  sendHeartbeatToAllBroker, which leads hign RT of sendMsg
+  1. change m_brokerAddrTable from hashMap to map;
+  2. do not add m_factoryLock here, but copy m_brokerAddrTable,
+      this is used to avoid con-current access m_factoryLock by
+  findBrokerAddressInPublish(called by sendKernlImpl) And
+  updateTopicRouteInfoFromNameServer
+
+   Note: after copying m_brokerAddrTable, updateTopicRouteInfoFromNameServer
+  modify m_brokerAddrTable imediatly,
+           after 1st send fail, producer will get topicPushlibshInfo again
+  before next try, so 2nd try will get correct broker to send ms;
+   */
+  BrokerAddrMAP brokerTable(getBrokerAddrMap());
+  string brokerAddr;
+  bool found = false;
+
+  if (brokerTable.find(brokerName) != brokerTable.end()) {
+    map<int, string> brokerMap(brokerTable[brokerName]);
+    map<int, string>::iterator it1 = brokerMap.find(MASTER_ID);
+    if (it1 != brokerMap.end()) {
+      brokerAddr = it1->second;
+      found = true;
+    }
+  }
+
+  brokerTable.clear();
+  if (found) return brokerAddr;
+
+  return "";
+}
+
+FindBrokerResult* MQClientFactory::findBrokerAddressInSubscribe(
+    const string& brokerName, int brokerId, bool onlyThisBroker) {
+  string brokerAddr;
+  bool slave = false;
+  bool found = false;
+  BrokerAddrMAP brokerTable(getBrokerAddrMap());
+
+  if (brokerTable.find(brokerName) != brokerTable.end()) {
+    map<int, string> brokerMap(brokerTable[brokerName]);
+    map<int, string>::iterator it1 = brokerMap.find(brokerId);
+    if (it1 != brokerMap.end()) {
+      brokerAddr = it1->second;
+      slave = (brokerId != MASTER_ID);
+      found = true;
+    } else  // from master
+    {
+      it1 = brokerMap.find(MASTER_ID);
+      if (it1 != brokerMap.end()) {
+        brokerAddr = it1->second;
+        slave = false;
+        found = true;
+      }
+    }
+  }
+
+  brokerTable.clear();
+  if (found) return new FindBrokerResult(brokerAddr, slave);
+
+  return NULL;
+}
+
+FindBrokerResult* MQClientFactory::findBrokerAddressInAdmin(
+    const string& brokerName) {
+  BrokerAddrMAP brokerTable(getBrokerAddrMap());
+  bool found = false;
+  bool slave = false;
+  string brokerAddr;
+
+  if (brokerTable.find(brokerName) != brokerTable.end()) {
+    map<int, string> brokerMap(brokerTable[brokerName]);
+    map<int, string>::iterator it1 = brokerMap.begin();
+    if (it1 != brokerMap.end()) {
+      slave = (it1->first != MASTER_ID);
+      found = true;
+      brokerAddr = it1->second;
+    }
+  }
+
+  brokerTable.clear();
+  if (found) return new FindBrokerResult(brokerAddr, slave);
+
+  return NULL;
+}
+
+MQClientAPIImpl* MQClientFactory::getMQClientAPIImpl() const {
+  return m_pClientAPIImpl.get();
+}
+
+void MQClientFactory::sendHeartbeatToAllBroker() {
+  BrokerAddrMAP brokerTable(getBrokerAddrMap());
+  if (brokerTable.size() == 0) {
+    LOG_WARN("sendheartbeat brokeradd is empty");
+    return;
+  }
+
+  unique_ptr<HeartbeatData> heartbeatData(prepareHeartbeatData());
+  bool producerEmpty = heartbeatData->isProducerDataSetEmpty();
+  bool consumerEmpty = heartbeatData->isConsumerDataSetEmpty();
+  if (producerEmpty && consumerEmpty) {
+    LOG_WARN("sendheartbeat heartbeatData empty");
+    brokerTable.clear();
+    return;
+  }
+
+  SessionCredentials session_credentials;
+  getSessionCredentialsFromOneOfProducerOrConsumer(session_credentials);
+  for (BrokerAddrMAP::iterator it = brokerTable.begin();
+       it != brokerTable.end(); ++it) {
+    map<int, string> brokerMap(it->second);
+    map<int, string>::iterator it1 = brokerMap.begin();
+    for (; it1 != brokerMap.end(); ++it1) {
+      string& addr = it1->second;
+      if (consumerEmpty && it1->first != MASTER_ID) continue;
+
+      try {
+        m_pClientAPIImpl->sendHearbeat(addr, heartbeatData.get(),
+                                       session_credentials);
+      } catch (MQException& e) {
+        LOG_ERROR(e.what());
+      }
+    }
+  }
+  brokerTable.clear();
+}
+
+void MQClientFactory::persistAllConsumerOffset(boost::system::error_code& ec,
+                                               boost::asio::deadline_timer* t) {
+  {
+    boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+    if (m_consumerTable.size() > 0) {
+      for (MQCMAP::iterator it = m_consumerTable.begin();
+           it != m_consumerTable.end(); ++it) {
+        LOG_DEBUG("Client factory start persistAllConsumerOffset");
+        it->second->persistConsumerOffset();
+      }
+    }
+  }
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::seconds(5), e);
+  t->async_wait(
+      boost::bind(&MQClientFactory::persistAllConsumerOffset, this, ec, t));
+}
+
+HeartbeatData* MQClientFactory::prepareHeartbeatData() {
+  HeartbeatData* pHeartbeatData = new HeartbeatData();
+  // clientID
+  pHeartbeatData->setClientID(m_clientId);
+
+  // Consumer
+  insertConsumerInfoToHeartBeatData(pHeartbeatData);
+
+  // Producer
+  insertProducerInfoToHeartBeatData(pHeartbeatData);
+
+  return pHeartbeatData;
+}
+
+void MQClientFactory::timerCB_sendHeartbeatToAllBroker(
+    boost::system::error_code& ec, boost::asio::deadline_timer* t) {
+  sendHeartbeatToAllBroker();
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::seconds(30), e);
+  t->async_wait(boost::bind(&MQClientFactory::timerCB_sendHeartbeatToAllBroker,
+                            this, ec, t));
+}
+
+void MQClientFactory::fetchNameServerAddr(boost::system::error_code& ec,
+                                          boost::asio::deadline_timer* t) {
+  m_pClientAPIImpl->fetchNameServerAddr(m_nameSrvDomain);
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::seconds(60 * 2), e);
+  t->async_wait(
+      boost::bind(&MQClientFactory::fetchNameServerAddr, this, ec, t));
+}
+
+void MQClientFactory::startScheduledTask(bool startFetchNSService) {
+  boost::asio::io_service::work work(m_async_ioService);  // avoid async io
+                                                          // service stops after
+                                                          // first timer timeout
+                                                          // callback
+
+  boost::system::error_code ec1;
+  boost::asio::deadline_timer t1(m_async_ioService,
+                                 boost::posix_time::seconds(3));
+  t1.async_wait(
+      boost::bind(&MQClientFactory::updateTopicRouteInfo, this, ec1, &t1));
+
+  boost::system::error_code ec2;
+  boost::asio::deadline_timer t2(m_async_ioService,
+                                 boost::posix_time::milliseconds(10));
+  t2.async_wait(boost::bind(&MQClientFactory::timerCB_sendHeartbeatToAllBroker,
+                            this, ec2, &t2));
+
+  if (startFetchNSService) {
+    boost::system::error_code ec5;
+    boost::asio::deadline_timer* t5 = new boost::asio::deadline_timer(
+        m_async_ioService, boost::posix_time::seconds(60 * 2));
+    t5->async_wait(
+        boost::bind(&MQClientFactory::fetchNameServerAddr, this, ec5, t5));
+  }
+
+  LOG_INFO("start scheduled task:%s", m_clientId.c_str());
+  boost::system::error_code ec;
+  m_async_ioService.run(ec);
+}
+
+void MQClientFactory::rebalanceImmediately() {
+  // m_consumer_async_service_thread will be only started once for all consumer
+  if (m_consumer_async_service_thread == NULL) {
+    doRebalance();
+    m_consumer_async_service_thread.reset(new boost::thread(
+        boost::bind(&MQClientFactory::consumer_timerOperation, this)));
+  }
+}
+
+void MQClientFactory::consumer_timerOperation() {
+  LOG_INFO("clientFactory:%s start consumer_timerOperation",
+           m_clientId.c_str());
+  boost::asio::io_service::work work(
+      m_consumer_async_ioService);  // avoid async io
+                                    // service stops after
+                                    // first timer timeout
+                                    // callback
+
+  boost::system::error_code ec1;
+  boost::asio::deadline_timer t(m_consumer_async_ioService,
+                                boost::posix_time::seconds(10));
+  t.async_wait(
+      boost::bind(&MQClientFactory::timerCB_doRebalance, this, ec1, &t));
+
+  boost::system::error_code ec2;
+  boost::asio::deadline_timer t2(m_consumer_async_ioService,
+                                 boost::posix_time::seconds(5));
+  t2.async_wait(
+      boost::bind(&MQClientFactory::persistAllConsumerOffset, this, ec2, &t2));
+
+  boost::system::error_code ec;
+  m_consumer_async_ioService.run(ec);
+  LOG_INFO("clientFactory:%s stop consumer_timerOperation", m_clientId.c_str());
+}
+
+void MQClientFactory::timerCB_doRebalance(boost::system::error_code& ec,
+                                          boost::asio::deadline_timer* t) {
+  doRebalance();
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::seconds(10), e);
+  t->async_wait(
+      boost::bind(&MQClientFactory::timerCB_doRebalance, this, ec, t));
+}
+
+void MQClientFactory::doRebalance() {
+  LOG_INFO("Client factory:%s start dorebalance", m_clientId.c_str());
+  if (getConsumerTableSize() > 0) {
+    boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+    for (MQCMAP::iterator it = m_consumerTable.begin();
+         it != m_consumerTable.end(); ++it) {
+      it->second->doRebalance();
+    }
+  }
+}
+
+void MQClientFactory::doRebalanceByConsumerGroup(const string& consumerGroup) {
+  boost::lock_guard<boost::mutex> lock(m_consumerTableMutex);
+  if (m_consumerTable.find(consumerGroup) != m_consumerTable.end()) {
+    LOG_INFO("Client factory:%s start dorebalance for consumer:%s",
+             m_clientId.c_str(), consumerGroup.c_str());
+    MQConsumer* pMQConsumer = m_consumerTable[consumerGroup];
+    pMQConsumer->doRebalance();
+  }
+}
+
+void MQClientFactory::unregisterClient(
+    const string& producerGroup, const string& consumerGroup,
+    const SessionCredentials& sessionCredentials) {
+  BrokerAddrMAP brokerTable(getBrokerAddrMap());
+  for (BrokerAddrMAP::iterator it = brokerTable.begin();
+       it != brokerTable.end(); ++it) {
+    map<int, string> brokerMap(it->second);
+    map<int, string>::iterator it1 = brokerMap.begin();
+    for (; it1 != brokerMap.end(); ++it1) {
+      string& addr = it1->second;
+      m_pClientAPIImpl->unregisterClient(addr, m_clientId, producerGroup,
+                                         consumerGroup, sessionCredentials);
+    }
+  }
+}
+
+//<!************************************************************************
+void MQClientFactory::fetchSubscribeMessageQueues(
+    const string& topic, vector<MQMessageQueue>& mqs,
+    const SessionCredentials& sessionCredentials) {
+  TopicRouteData* pTopicRouteData = getTopicRouteData(topic);
+  if (pTopicRouteData == NULL) {
+    updateTopicRouteInfoFromNameServer(topic, sessionCredentials);
+    pTopicRouteData = getTopicRouteData(topic);
+  }
+  if (pTopicRouteData != NULL) {
+    topicRouteData2TopicSubscribeInfo(topic, pTopicRouteData, mqs);
+    if (mqs.empty()) {
+      THROW_MQEXCEPTION(MQClientException, "Can not find Message Queue", -1);
+    }
+    return;
+  }
+  THROW_MQEXCEPTION(MQClientException, "Can not find Message Queue", -1);
+}
+
+//<!***************************************************************************
+void MQClientFactory::createTopic(
+    const string& key, const string& newTopic, int queueNum,
+    const SessionCredentials& sessionCredentials) {}
+
+int64 MQClientFactory::minOffset(const MQMessageQueue& mq,
+                                 const SessionCredentials& sessionCredentials) {
+  string brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  if (brokerAddr.empty()) {
+    updateTopicRouteInfoFromNameServer(mq.getTopic(), sessionCredentials);
+    brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  }
+
+  if (!brokerAddr.empty()) {
+    try {
+      return m_pClientAPIImpl->getMinOffset(brokerAddr, mq.getTopic(),
+                                            mq.getQueueId(), 1000 * 3,
+                                            sessionCredentials);
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+    }
+  }
+  THROW_MQEXCEPTION(MQClientException, "The broker is not exist", -1);
+}
+
+int64 MQClientFactory::maxOffset(const MQMessageQueue& mq,
+                                 const SessionCredentials& sessionCredentials) {
+  string brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  if (brokerAddr.empty()) {
+    updateTopicRouteInfoFromNameServer(mq.getTopic(), sessionCredentials);
+    brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  }
+
+  if (!brokerAddr.empty()) {
+    try {
+      return m_pClientAPIImpl->getMaxOffset(brokerAddr, mq.getTopic(),
+                                            mq.getQueueId(), 1000 * 3,
+                                            sessionCredentials);
+    } catch (MQException& e) {
+      THROW_MQEXCEPTION(MQClientException, "Invoke Broker exception", -1);
+    }
+  }
+  THROW_MQEXCEPTION(MQClientException, "The broker is not exist", -1);
+}
+
+int64 MQClientFactory::searchOffset(
+    const MQMessageQueue& mq, int64 timestamp,
+    const SessionCredentials& sessionCredentials) {
+  string brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  if (brokerAddr.empty()) {
+    updateTopicRouteInfoFromNameServer(mq.getTopic(), sessionCredentials);
+    brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  }
+
+  if (!brokerAddr.empty()) {
+    try {
+      return m_pClientAPIImpl->searchOffset(brokerAddr, mq.getTopic(),
+                                            mq.getQueueId(), timestamp,
+                                            1000 * 3, sessionCredentials);
+    } catch (MQException& e) {
+      THROW_MQEXCEPTION(MQClientException, "Invoke Broker exception", -1);
+    }
+  }
+  THROW_MQEXCEPTION(MQClientException, "The broker is not exist", -1);
+}
+
+MQMessageExt* MQClientFactory::viewMessage(
+    const string& msgId, const SessionCredentials& sessionCredentials) {
+  try {
+    return NULL;
+  } catch (MQException& e) {
+    THROW_MQEXCEPTION(MQClientException, "message id illegal", -1);
+  }
+}
+
+int64 MQClientFactory::earliestMsgStoreTime(
+    const MQMessageQueue& mq, const SessionCredentials& sessionCredentials) {
+  string brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  if (brokerAddr.empty()) {
+    updateTopicRouteInfoFromNameServer(mq.getTopic(), sessionCredentials);
+    brokerAddr = findBrokerAddressInPublish(mq.getBrokerName());
+  }
+
+  if (!brokerAddr.empty()) {
+    try {
+      return m_pClientAPIImpl->getEarliestMsgStoretime(
+          brokerAddr, mq.getTopic(), mq.getQueueId(), 1000 * 3,
+          sessionCredentials);
+    } catch (MQException& e) {
+      THROW_MQEXCEPTION(MQClientException, "Invoke Broker exception", -1);
+    }
+  }
+  THROW_MQEXCEPTION(MQClientException, "The broker is not exist", -1);
+}
+
+QueryResult MQClientFactory::queryMessage(
+    const string& topic, const string& key, int maxNum, int64 begin, int64 end,
+    const SessionCredentials& sessionCredentials) {
+  THROW_MQEXCEPTION(MQClientException, "queryMessage", -1);
+}
+
+void MQClientFactory::findConsumerIds(
+    const string& topic, const string& group, vector<string>& cids,
+    const SessionCredentials& sessionCredentials) {
+  string brokerAddr;
+  TopicRouteData* pTopicRouteData = getTopicRouteData(topic);
+  if (pTopicRouteData == NULL) {
+    updateTopicRouteInfoFromNameServer(topic, sessionCredentials);
+    pTopicRouteData = getTopicRouteData(topic);
+  }
+  if (pTopicRouteData != NULL) {
+    brokerAddr = pTopicRouteData->selectBrokerAddr();
+  }
+
+  if (!brokerAddr.empty()) {
+    try {
+      LOG_INFO("getConsumerIdList from broker:%s", brokerAddr.c_str());
+      return m_pClientAPIImpl->getConsumerIdListByGroup(
+          brokerAddr, group, cids, 5000, sessionCredentials);
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+    }
+  }
+}
+
+void MQClientFactory::resetOffset(
+    const string& group, const string& topic,
+    const map<MQMessageQueue, int64>& offsetTable) {
+  MQConsumer* pConsumer = selectConsumer(group);
+  if (pConsumer) {
+    map<MQMessageQueue, int64>::const_iterator it = offsetTable.begin();
+
+    for (; it != offsetTable.end(); ++it) {
+      MQMessageQueue mq = it->first;
+      PullRequest* pullreq = pConsumer->getRebalance()->getPullRequest(mq);
+      if (pullreq) {
+        pullreq->setDroped(true);
+        pullreq->clearAllMsgs();
+        pullreq->updateQueueMaxOffset(it->second);
+      } else {
+        LOG_ERROR("no corresponding pullRequest found for topic:%s",
+                  topic.c_str());
+      }
+    }
+
+    for (it = offsetTable.begin(); it != offsetTable.end(); ++it) {
+      MQMessageQueue mq = it->first;
+      if (topic == mq.getTopic()) {
+        LOG_INFO("offset sets to:%lld", it->second);
+        pConsumer->updateConsumeOffset(mq, it->second);
+      }
+    }
+    pConsumer->persistConsumerOffsetByResetOffset();
+
+    poll(0, 0, 10);
+
+    for (it = offsetTable.begin(); it != offsetTable.end(); ++it) {
+      MQMessageQueue mq = it->first;
+      if (topic == mq.getTopic()) {
+        LOG_DEBUG("resetOffset sets to:%lld", it->second);
+        pConsumer->updateConsumeOffset(mq, it->second);
+      }
+    }
+    pConsumer->persistConsumerOffsetByResetOffset();
+
+    for (it = offsetTable.begin(); it != offsetTable.end(); ++it) {
+      MQMessageQueue mq = it->first;
+      if (topic == mq.getTopic()) {
+        pConsumer->removeConsumeOffset(mq);
+      }
+    }
+
+    pConsumer->doRebalance();
+  } else {
+    LOG_ERROR("no corresponding consumer found for group:%s", group.c_str());
+  }
+}
+
+ConsumerRunningInfo* MQClientFactory::consumerRunningInfo(
+    const string& consumerGroup) {
+  MQConsumer* pConsumer = selectConsumer(consumerGroup);
+  if (pConsumer) {
+    ConsumerRunningInfo* runningInfo = pConsumer->getConsumerRunningInfo();
+    if (runningInfo) {
+      runningInfo->setProperty(ConsumerRunningInfo::PROP_NAMESERVER_ADDR,
+                               pConsumer->getNamesrvAddr());
+      if (pConsumer->getConsumeType() == CONSUME_PASSIVELY) {
+        runningInfo->setProperty(ConsumerRunningInfo::PROP_CONSUME_TYPE,
+                                 "CONSUME_PASSIVELY");
+      } else {
+        runningInfo->setProperty(ConsumerRunningInfo::PROP_CONSUME_TYPE,
+                                 "CONSUME_ACTIVELY");
+      }
+      runningInfo->setProperty(ConsumerRunningInfo::PROP_CLIENT_VERSION,
+                               "V3_1_8");  // MQVersion::s_CurrentVersion ));
+
+      return runningInfo;
+    }
+  }
+
+  LOG_ERROR("no corresponding consumer found for group:%s",
+            consumerGroup.c_str());
+  return NULL;
+}
+
+void MQClientFactory::getSessionCredentialsFromOneOfProducerOrConsumer(
+    SessionCredentials& session_credentials) {
+  // Note: on the same MQClientFactory, all producers and consumers used the
+  // same
+  // sessionCredentials,
+  // So only need get sessionCredentials from the first one producer or consumer
+  // now.
+  // this function was only used by updateTopicRouteInfo() and
+  // sendHeartbeatToAllBrokers() now.
+  // if this strategy was changed in future, need get sessionCredentials for
+  // each
+  // producer and consumer.
+  getSessionCredentialFromProducerTable(session_credentials);
+  if (!session_credentials.isValid())
+    getSessionCredentialFromConsumerTable(session_credentials);
+
+  if (!session_credentials.isValid()) {
+    LOG_ERROR(
+        "updateTopicRouteInfo: didn't get the session_credentials from any "
+        "producers and consumers, please re-intialize it");
+  }
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientFactory.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientFactory.h b/rocketmq-cpp/src/MQClientFactory.h
new file mode 100644
index 0000000..34ac2a9
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientFactory.h
@@ -0,0 +1,220 @@
+/*
+ * 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 __MQCLIENTFACTORY_H__
+#define __MQCLIENTFACTORY_H__
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/thread/thread.hpp>
+#include "FindBrokerResult.h"
+#include "MQClientAPIImpl.h"
+#include "MQClientException.h"
+#include "MQConsumer.h"
+#include "MQDecoder.h"
+#include "MQMessageQueue.h"
+#include "MQProducer.h"
+#include "PermName.h"
+#include "QueryResult.h"
+#include "ServiceState.h"
+#include "SocketUtil.h"
+#include "TopicConfig.h"
+#include "TopicRouteData.h"
+
+namespace rocketmq {
+//<!************************************************************************
+class TopicPublishInfo;
+class MQClientFactory {
+ public:
+  MQClientFactory(const string& clientID, int pullThreadNum,
+                  uint64_t tcpConnectTimeout,
+                  uint64_t tcpTransportTryLockTimeout, string unitName);
+  virtual ~MQClientFactory();
+
+  void start();
+  void shutdown();
+  bool registerProducer(MQProducer* pProducer);
+  void unregisterProducer(MQProducer* pProducer);
+  bool registerConsumer(MQConsumer* pConsumer);
+  void unregisterConsumer(MQConsumer* pConsumer);
+
+  void createTopic(const string& key, const string& newTopic, int queueNum,
+                   const SessionCredentials& session_credentials);
+  int64 minOffset(const MQMessageQueue& mq,
+                  const SessionCredentials& session_credentials);
+  int64 maxOffset(const MQMessageQueue& mq,
+                  const SessionCredentials& session_credentials);
+  int64 searchOffset(const MQMessageQueue& mq, int64 timestamp,
+                     const SessionCredentials& session_credentials);
+  int64 earliestMsgStoreTime(const MQMessageQueue& mq,
+                             const SessionCredentials& session_credentials);
+  MQMessageExt* viewMessage(const string& msgId,
+                            const SessionCredentials& session_credentials);
+  QueryResult queryMessage(const string& topic, const string& key, int maxNum,
+                           int64 begin, int64 end,
+                           const SessionCredentials& session_credentials);
+
+  MQClientAPIImpl* getMQClientAPIImpl() const;
+  MQProducer* selectProducer(const string& group);
+  MQConsumer* selectConsumer(const string& group);
+
+  boost::shared_ptr<TopicPublishInfo> topicRouteData2TopicPublishInfo(
+      const string& topic, TopicRouteData* pRoute);
+
+  void topicRouteData2TopicSubscribeInfo(const string& topic,
+                                         TopicRouteData* pRoute,
+                                         vector<MQMessageQueue>& mqs);
+
+  FindBrokerResult* findBrokerAddressInSubscribe(const string& brokerName,
+                                                 int brokerId,
+                                                 bool onlyThisBroker);
+
+  FindBrokerResult* findBrokerAddressInAdmin(const string& brokerName);
+
+  string findBrokerAddressInPublish(const string& brokerName);
+
+  boost::shared_ptr<TopicPublishInfo> tryToFindTopicPublishInfo(
+      const string& topic, const SessionCredentials& session_credentials);
+
+  void fetchSubscribeMessageQueues(
+      const string& topic, vector<MQMessageQueue>& mqs,
+      const SessionCredentials& session_credentials);
+
+  bool updateTopicRouteInfoFromNameServer(
+      const string& topic, const SessionCredentials& session_credentials,
+      bool isDefault = false);
+  void rebalanceImmediately();
+  void doRebalanceByConsumerGroup(const string& consumerGroup);
+  void sendHeartbeatToAllBroker();
+
+  void findConsumerIds(const string& topic, const string& group,
+                       vector<string>& cids,
+                       const SessionCredentials& session_credentials);
+  void resetOffset(const string& group, const string& topic,
+                   const map<MQMessageQueue, int64>& offsetTable);
+  ConsumerRunningInfo* consumerRunningInfo(const string& consumerGroup);
+  bool getSessionCredentialFromConsumer(const string& consumerGroup,
+                                        SessionCredentials& sessionCredentials);
+  void addBrokerToAddrMap(const string& brokerName,
+                          map<int, string>& brokerAddrs);
+  map<string, map<int, string>> getBrokerAddrMap();
+  void clearBrokerAddrMap();
+
+ private:
+  void unregisterClient(const string& producerGroup,
+                        const string& consumerGroup,
+                        const SessionCredentials& session_credentials);
+  TopicRouteData* getTopicRouteData(const string& topic);
+  void addTopicRouteData(const string& topic, TopicRouteData* pTopicRouteData);
+  HeartbeatData* prepareHeartbeatData();
+
+  void startScheduledTask(bool startFetchNSService = true);
+  //<!timer async callback
+  void fetchNameServerAddr(boost::system::error_code& ec,
+                           boost::asio::deadline_timer* t);
+  void updateTopicRouteInfo(boost::system::error_code& ec,
+                            boost::asio::deadline_timer* t);
+  void timerCB_sendHeartbeatToAllBroker(boost::system::error_code& ec,
+                                        boost::asio::deadline_timer* t);
+
+  // consumer related operation
+  void consumer_timerOperation();
+  void persistAllConsumerOffset(boost::system::error_code& ec,
+                                boost::asio::deadline_timer* t);
+  void doRebalance();
+  void timerCB_doRebalance(boost::system::error_code& ec,
+                           boost::asio::deadline_timer* t);
+  bool getSessionCredentialFromConsumerTable(
+      SessionCredentials& sessionCredentials);
+  bool addConsumerToTable(const string& consumerName, MQConsumer* pMQConsumer);
+  void eraseConsumerFromTable(const string& consumerName);
+  int getConsumerTableSize();
+  void getTopicListFromConsumerSubscription(set<string>& topicList);
+  void updateConsumerSubscribeTopicInfo(const string& topic,
+                                        vector<MQMessageQueue> mqs);
+  void insertConsumerInfoToHeartBeatData(HeartbeatData* pHeartbeatData);
+
+  // producer related operation
+  bool getSessionCredentialFromProducerTable(
+      SessionCredentials& sessionCredentials);
+  bool addProducerToTable(const string& producerName, MQProducer* pMQProducer);
+  void eraseProducerFromTable(const string& producerName);
+  int getProducerTableSize();
+  void insertProducerInfoToHeartBeatData(HeartbeatData* pHeartbeatData);
+
+  // topicPublishInfo related operation
+  void addTopicInfoToTable(
+      const string& topic,
+      boost::shared_ptr<TopicPublishInfo> pTopicPublishInfo);
+  void eraseTopicInfoFromTable(const string& topic);
+  bool isTopicInfoValidInTable(const string& topic);
+  boost::shared_ptr<TopicPublishInfo> getTopicPublishInfoFromTable(
+      const string& topic);
+  void getTopicListFromTopicPublishInfo(set<string>& topicList);
+
+  void getSessionCredentialsFromOneOfProducerOrConsumer(
+      SessionCredentials& session_credentials);
+
+ private:
+  string m_clientId;
+  string m_nameSrvDomain;  // per clientId
+  ServiceState m_serviceState;
+  bool m_bFetchNSService;
+
+  //<! group --> MQProducer;
+  typedef map<string, MQProducer*> MQPMAP;
+  boost::mutex m_producerTableMutex;
+  MQPMAP m_producerTable;
+
+  //<! group --> MQConsumer;
+  typedef map<string, MQConsumer*> MQCMAP;
+  boost::mutex m_consumerTableMutex;
+  MQCMAP m_consumerTable;
+
+  //<! Topic---> TopicRouteData
+  typedef map<string, TopicRouteData*> TRDMAP;
+  boost::mutex m_topicRouteTableMutex;
+  TRDMAP m_topicRouteTable;
+
+  //<!-----brokerName
+  //<!     ------brokerid;
+  //<!     ------add;
+  boost::mutex m_brokerAddrlock;
+  typedef map<string, map<int, string>> BrokerAddrMAP;
+  BrokerAddrMAP m_brokerAddrTable;
+
+  //<!topic ---->TopicPublishInfo> ;
+  typedef map<string, boost::shared_ptr<TopicPublishInfo>> TPMap;
+  boost::mutex m_topicPublishInfoTableMutex;
+  TPMap m_topicPublishInfoTable;
+  boost::mutex m_factoryLock;
+  boost::mutex m_topicPublishInfoLock;
+
+  //<!clientapi;
+  unique_ptr<MQClientAPIImpl> m_pClientAPIImpl;
+  unique_ptr<ClientRemotingProcessor> m_pClientRemotingProcessor;
+
+  boost::asio::io_service m_async_ioService;
+  unique_ptr<boost::thread> m_async_service_thread;
+
+  boost::asio::io_service m_consumer_async_ioService;
+  unique_ptr<boost::thread> m_consumer_async_service_thread;
+};
+
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientManager.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientManager.cpp b/rocketmq-cpp/src/MQClientManager.cpp
new file mode 100755
index 0000000..79ef77e
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientManager.cpp
@@ -0,0 +1,54 @@
+/*
+ * 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 "MQClientManager.h"
+#include "Logging.h"
+
+namespace rocketmq {
+//<!************************************************************************
+MQClientManager::MQClientManager() {}
+
+MQClientManager::~MQClientManager() { m_factoryTable.clear(); }
+
+MQClientManager* MQClientManager::getInstance() {
+  static MQClientManager instance;
+  return &instance;
+}
+
+MQClientFactory* MQClientManager::getMQClientFactory(
+    const string& clientId, int pullThreadNum, uint64_t tcpConnectTimeout,
+    uint64_t tcpTransportTryLockTimeout, string unitName) {
+  FTMAP::iterator it = m_factoryTable.find(clientId);
+  if (it != m_factoryTable.end()) {
+    return it->second;
+  } else {
+    MQClientFactory* factory =
+        new MQClientFactory(clientId, pullThreadNum, tcpConnectTimeout,
+                            tcpTransportTryLockTimeout, unitName);
+    m_factoryTable[clientId] = factory;
+    return factory;
+  }
+}
+
+void MQClientManager::removeClientFactory(const string& clientId) {
+  FTMAP::iterator it = m_factoryTable.find(clientId);
+  if (it != m_factoryTable.end()) {
+    deleteAndZero(it->second);
+    m_factoryTable.erase(it);
+  }
+}
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientManager.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientManager.h b/rocketmq-cpp/src/MQClientManager.h
new file mode 100755
index 0000000..7703c15
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientManager.h
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __MQCLIENTMANAGER_H__
+#define __MQCLIENTMANAGER_H__
+
+#include <map>
+#include <string>
+#include "Logging.h"
+#include "MQClientFactory.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class MQClientManager {
+ public:
+  virtual ~MQClientManager();
+  MQClientFactory* getMQClientFactory(const string& clientId, int pullThreadNum,
+                                      uint64_t tcpConnectTimeout,
+                                      uint64_t tcpTransportTryLockTimeout,
+                                      string unitName);
+  void removeClientFactory(const string& clientId);
+
+  static MQClientManager* getInstance();
+
+ private:
+  MQClientManager();
+
+ private:
+  typedef map<string, MQClientFactory*> FTMAP;
+  FTMAP m_factoryTable;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/Arg_helper.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/Arg_helper.cpp b/rocketmq-cpp/src/common/Arg_helper.cpp
new file mode 100755
index 0000000..61ac549
--- /dev/null
+++ b/rocketmq-cpp/src/common/Arg_helper.cpp
@@ -0,0 +1,67 @@
+/*
+ * 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 "Arg_helper.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+Arg_helper::Arg_helper(int argc, char* argv[]) {
+  for (int i = 0; i < argc; i++) {
+    m_args.push_back(argv[i]);
+  }
+}
+
+Arg_helper::Arg_helper(string arg_str_) {
+  vector<string> v;
+  UtilAll::Split(v, arg_str_, " ");
+  m_args.insert(m_args.end(), v.begin(), v.end());
+}
+
+string Arg_helper::get_option(int idx_) const {
+  if ((size_t)idx_ >= m_args.size()) {
+    return "";
+  }
+  return m_args[idx_];
+}
+
+bool Arg_helper::is_enable_option(string opt_) const {
+  for (size_t i = 0; i < m_args.size(); ++i) {
+    if (opt_ == m_args[i]) {
+      return true;
+    }
+  }
+  return false;
+}
+
+string Arg_helper::get_option_value(string opt_) const {
+  string ret = "";
+  for (size_t i = 0; i < m_args.size(); ++i) {
+    if (opt_ == m_args[i]) {
+      size_t value_idx = ++i;
+      if (value_idx >= m_args.size()) {
+        return ret;
+      }
+      ret = m_args[value_idx];
+      return ret;
+    }
+  }
+  return ret;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/AsyncArg.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/AsyncArg.h b/rocketmq-cpp/src/common/AsyncArg.h
new file mode 100755
index 0000000..fc358cb
--- /dev/null
+++ b/rocketmq-cpp/src/common/AsyncArg.h
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _AsyncArg_H_
+#define _AsyncArg_H_
+
+#include "MQMessageQueue.h"
+#include "PullAPIWrapper.h"
+#include "SubscriptionData.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+
+struct AsyncArg {
+  MQMessageQueue mq;
+  SubscriptionData subData;
+  PullAPIWrapper* pPullWrapper;
+};
+
+//<!***************************************************************************
+}
+#endif  //<! _AsyncArg_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/AsyncCallbackWrap.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/AsyncCallbackWrap.cpp b/rocketmq-cpp/src/common/AsyncCallbackWrap.cpp
new file mode 100755
index 0000000..a61e0ee
--- /dev/null
+++ b/rocketmq-cpp/src/common/AsyncCallbackWrap.cpp
@@ -0,0 +1,168 @@
+/*
+ * 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 "AsyncCallbackWrap.h"
+#include "Logging.h"
+#include "MQClientAPIImpl.h"
+#include "MQDecoder.h"
+#include "MQMessageQueue.h"
+#include "MQProtos.h"
+#include "PullAPIWrapper.h"
+#include "PullResultExt.h"
+#include "ResponseFuture.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+AsyncCallbackWrap::AsyncCallbackWrap(AsyncCallback* pAsyncCallback,
+                                     MQClientAPIImpl* pclientAPI)
+    : m_pAsyncCallBack(pAsyncCallback), m_pClientAPI(pclientAPI) {}
+
+AsyncCallbackWrap::~AsyncCallbackWrap() {
+  m_pAsyncCallBack = NULL;
+  m_pClientAPI = NULL;
+}
+
+//<!************************************************************************
+SendCallbackWrap::SendCallbackWrap(const string& brokerName,
+                                   const MQMessage& msg,
+                                   AsyncCallback* pAsyncCallback,
+                                   MQClientAPIImpl* pclientAPI)
+    : AsyncCallbackWrap(pAsyncCallback, pclientAPI),
+      m_msg(msg),
+      m_brokerName(brokerName) {}
+
+void SendCallbackWrap::onException() {
+  if (m_pAsyncCallBack == NULL) return;
+
+  SendCallback* pCallback = static_cast<SendCallback*>(m_pAsyncCallBack);
+  if (pCallback) {
+    unique_ptr<MQException> exception(new MQException(
+        "send msg failed due to wait response timeout or network error", -1,
+        __FILE__, __LINE__));
+    pCallback->onException(*exception);
+    if (pCallback->getSendCallbackType() == autoDeleteSendCallback) {
+      deleteAndZero(pCallback);
+    }
+  }
+}
+
+void SendCallbackWrap::operationComplete(ResponseFuture* pResponseFuture,
+                                         bool bProducePullRequest) {
+  unique_ptr<RemotingCommand> pResponse(pResponseFuture->getCommand());
+
+  if (m_pAsyncCallBack == NULL) {
+    return;
+  }
+  SendCallback* pCallback = static_cast<SendCallback*>(m_pAsyncCallBack);
+
+  if (!pResponse) {
+    string err = "unknow reseaon";
+    if (!pResponseFuture->isSendRequestOK()) {
+      err = "send request failed";
+
+    } else if (pResponseFuture->isTimeOut()) {
+      // pResponseFuture->setAsyncResponseFlag();
+      err = "wait response timeout";
+    }
+    if (pCallback) {
+      MQException exception(err, -1, __FILE__, __LINE__);
+      pCallback->onException(exception);
+    }
+    LOG_ERROR("send failed of:%d", pResponseFuture->getOpaque());
+  } else {
+    try {
+      SendResult ret = m_pClientAPI->processSendResponse(m_brokerName, m_msg,
+                                                         pResponse.get());
+      if (pCallback) pCallback->onSuccess(ret);
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+      if (pCallback) {
+        MQException exception("process send response error", -1, __FILE__,
+                              __LINE__);
+        pCallback->onException(exception);
+      }
+    }
+  }
+  if (pCallback && pCallback->getSendCallbackType() == autoDeleteSendCallback) {
+    deleteAndZero(pCallback);
+  }
+}
+
+//<!************************************************************************
+PullCallbackWarp::PullCallbackWarp(AsyncCallback* pAsyncCallback,
+                                   MQClientAPIImpl* pclientAPI, void* pArg)
+    : AsyncCallbackWrap(pAsyncCallback, pclientAPI) {
+  m_pArg = *static_cast<AsyncArg*>(pArg);
+}
+
+PullCallbackWarp::~PullCallbackWarp() {}
+
+void PullCallbackWarp::onException() {
+  if (m_pAsyncCallBack == NULL) return;
+
+  PullCallback* pCallback = static_cast<PullCallback*>(m_pAsyncCallBack);
+  if (pCallback) {
+    MQException exception("wait response timeout", -1, __FILE__, __LINE__);
+    pCallback->onException(exception);
+  } else {
+    LOG_ERROR("PullCallback is NULL, AsyncPull could not continue");
+  }
+}
+
+void PullCallbackWarp::operationComplete(ResponseFuture* pResponseFuture,
+                                         bool bProducePullRequest) {
+  unique_ptr<RemotingCommand> pResponse(pResponseFuture->getCommand());
+  if (m_pAsyncCallBack == NULL) {
+    LOG_ERROR("m_pAsyncCallBack is NULL, AsyncPull could not continue");
+    return;
+  }
+  PullCallback* pCallback = static_cast<PullCallback*>(m_pAsyncCallBack);
+  if (!pResponse) {
+    string err = "unknow reseaon";
+    if (!pResponseFuture->isSendRequestOK()) {
+      err = "send request failed";
+
+    } else if (pResponseFuture->isTimeOut()) {
+      // pResponseFuture->setAsyncResponseFlag();
+      err = "wait response timeout";
+    }
+    MQException exception(err, -1, __FILE__, __LINE__);
+    LOG_ERROR("Async pull exception of opaque:%d",
+              pResponseFuture->getOpaque());
+    if (pCallback && bProducePullRequest) pCallback->onException(exception);
+  } else {
+    try {
+      if (m_pArg.pPullWrapper) {
+        unique_ptr<PullResult> pullResult(
+            m_pClientAPI->processPullResponse(pResponse.get()));
+        PullResult result = m_pArg.pPullWrapper->processPullResult(
+            m_pArg.mq, pullResult.get(), &m_pArg.subData);
+        if (pCallback)
+          pCallback->onSuccess(m_pArg.mq, result, bProducePullRequest);
+      } else {
+        LOG_ERROR("pPullWrapper had been destroyed with consumer");
+      }
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+      MQException exception("pullResult error", -1, __FILE__, __LINE__);
+      if (pCallback && bProducePullRequest) pCallback->onException(exception);
+    }
+  }
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/AsyncCallbackWrap.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/AsyncCallbackWrap.h b/rocketmq-cpp/src/common/AsyncCallbackWrap.h
new file mode 100755
index 0000000..50968c9
--- /dev/null
+++ b/rocketmq-cpp/src/common/AsyncCallbackWrap.h
@@ -0,0 +1,84 @@
+/*
+ * 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 __ASYNCCALLBACKWRAP_H__
+#define __ASYNCCALLBACKWRAP_H__
+
+#include "AsyncArg.h"
+#include "AsyncCallback.h"
+#include "MQMessage.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+class ResponseFuture;
+class MQClientAPIImpl;
+//<!***************************************************************************
+enum asyncCallBackType {
+  asyncCallbackWrap = 0,
+  sendCallbackWrap = 1,
+  pullCallbackWarp = 2
+};
+
+struct AsyncCallbackWrap {
+ public:
+  AsyncCallbackWrap(AsyncCallback* pAsyncCallback, MQClientAPIImpl* pclientAPI);
+  virtual ~AsyncCallbackWrap();
+  virtual void operationComplete(ResponseFuture* pResponseFuture,
+                                 bool bProducePullRequest) = 0;
+  virtual void onException() = 0;
+  virtual asyncCallBackType getCallbackType() = 0;
+
+ protected:
+  AsyncCallback* m_pAsyncCallBack;
+  MQClientAPIImpl* m_pClientAPI;
+};
+
+//<!************************************************************************
+class SendCallbackWrap : public AsyncCallbackWrap {
+ public:
+  SendCallbackWrap(const string& brokerName, const MQMessage& msg,
+                   AsyncCallback* pAsyncCallback, MQClientAPIImpl* pclientAPI);
+
+  virtual ~SendCallbackWrap(){};
+  virtual void operationComplete(ResponseFuture* pResponseFuture,
+                                 bool bProducePullRequest);
+  virtual void onException();
+  virtual asyncCallBackType getCallbackType() { return sendCallbackWrap; }
+
+ private:
+  MQMessage m_msg;
+  string m_brokerName;
+};
+
+//<!***************************************************************************
+class PullCallbackWarp : public AsyncCallbackWrap {
+ public:
+  PullCallbackWarp(AsyncCallback* pAsyncCallback, MQClientAPIImpl* pclientAPI,
+                   void* pArg);
+  virtual ~PullCallbackWarp();
+  virtual void operationComplete(ResponseFuture* pResponseFuture,
+                                 bool bProducePullRequest);
+  virtual void onException();
+  virtual asyncCallBackType getCallbackType() { return pullCallbackWarp; }
+
+ private:
+  AsyncArg m_pArg;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif  //<! _AsyncCallbackWrap_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/ByteOrder.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/ByteOrder.h b/rocketmq-cpp/src/common/ByteOrder.h
new file mode 100644
index 0000000..351c5ae
--- /dev/null
+++ b/rocketmq-cpp/src/common/ByteOrder.h
@@ -0,0 +1,179 @@
+
+#ifndef BYTEORDER_H_INCLUDED
+#define BYTEORDER_H_INCLUDED
+
+#include <stddef.h>
+#include <stdint.h>
+#include <algorithm>
+#include <boost/detail/endian.hpp>
+#include "RocketMQClient.h"
+#include "UtilAll.h"
+//==============================================================================
+/** Contains static methods for converting the byte order between different
+    endiannesses.
+*/
+namespace rocketmq {
+
+class ROCKETMQCLIENT_API ByteOrder {
+ public:
+  //==============================================================================
+  /** Swaps the upper and lower bytes of a 16-bit integer. */
+  static uint16 swap(uint16 value);
+
+  /** Reverses the order of the 4 bytes in a 32-bit integer. */
+  static uint32 swap(uint32 value);
+
+  /** Reverses the order of the 8 bytes in a 64-bit integer. */
+  static uint64 swap(uint64 value);
+
+  //==============================================================================
+  /** Swaps the byte order of a 16-bit int if the CPU is big-endian */
+  static uint16 swapIfBigEndian(uint16 value);
+
+  /** Swaps the byte order of a 32-bit int if the CPU is big-endian */
+  static uint32 swapIfBigEndian(uint32 value);
+
+  /** Swaps the byte order of a 64-bit int if the CPU is big-endian */
+  static uint64 swapIfBigEndian(uint64 value);
+
+  /** Swaps the byte order of a 16-bit int if the CPU is little-endian */
+  static uint16 swapIfLittleEndian(uint16 value);
+
+  /** Swaps the byte order of a 32-bit int if the CPU is little-endian */
+  static uint32 swapIfLittleEndian(uint32 value);
+
+  /** Swaps the byte order of a 64-bit int if the CPU is little-endian */
+  static uint64 swapIfLittleEndian(uint64 value);
+
+  //==============================================================================
+  /** Turns 4 bytes into a little-endian integer. */
+  static uint32 littleEndianInt(const void* bytes);
+
+  /** Turns 8 bytes into a little-endian integer. */
+  static uint64 littleEndianInt64(const void* bytes);
+
+  /** Turns 2 bytes into a little-endian integer. */
+  static uint16 littleEndianShort(const void* bytes);
+
+  /** Turns 4 bytes into a big-endian integer. */
+  static uint32 bigEndianInt(const void* bytes);
+
+  /** Turns 8 bytes into a big-endian integer. */
+  static uint64 bigEndianInt64(const void* bytes);
+
+  /** Turns 2 bytes into a big-endian integer. */
+  static uint16 bigEndianShort(const void* bytes);
+
+  //==============================================================================
+  /** Converts 3 little-endian bytes into a signed 24-bit value (which is
+   * sign-extended to 32 bits). */
+  static int littleEndian24Bit(const void* bytes);
+
+  /** Converts 3 big-endian bytes into a signed 24-bit value (which is
+   * sign-extended to 32 bits). */
+  static int bigEndian24Bit(const void* bytes);
+
+  /** Copies a 24-bit number to 3 little-endian bytes. */
+  static void littleEndian24BitToChars(int value, void* destBytes);
+
+  /** Copies a 24-bit number to 3 big-endian bytes. */
+  static void bigEndian24BitToChars(int value, void* destBytes);
+
+  //==============================================================================
+  /** Returns true if the current CPU is big-endian. */
+  static bool isBigEndian();
+};
+
+//==============================================================================
+
+inline uint16 ByteOrder::swap(uint16 n) {
+  return static_cast<uint16>((n << 8) | (n >> 8));
+}
+
+inline uint32 ByteOrder::swap(uint32 n) {
+  return (n << 24) | (n >> 24) | ((n & 0xff00) << 8) | ((n & 0xff0000) >> 8);
+}
+
+inline uint64 ByteOrder::swap(uint64 value) {
+  return (((uint64)swap((uint32)value)) << 32) | swap((uint32)(value >> 32));
+}
+
+#if __BYTE_ORDER__ == \
+    __ORDER_LITTLE_ENDIAN__  //__BYTE_ORDER__ is defined by GCC
+inline uint16 ByteOrder::swapIfBigEndian(const uint16 v) { return v; }
+inline uint32 ByteOrder::swapIfBigEndian(const uint32 v) { return v; }
+inline uint64 ByteOrder::swapIfBigEndian(const uint64 v) { return v; }
+inline uint16 ByteOrder::swapIfLittleEndian(const uint16 v) { return swap(v); }
+inline uint32 ByteOrder::swapIfLittleEndian(const uint32 v) { return swap(v); }
+inline uint64 ByteOrder::swapIfLittleEndian(const uint64 v) { return swap(v); }
+inline uint32 ByteOrder::littleEndianInt(const void* const bytes) {
+  return *static_cast<const uint32*>(bytes);
+}
+inline uint64 ByteOrder::littleEndianInt64(const void* const bytes) {
+  return *static_cast<const uint64*>(bytes);
+}
+inline uint16 ByteOrder::littleEndianShort(const void* const bytes) {
+  return *static_cast<const uint16*>(bytes);
+}
+inline uint32 ByteOrder::bigEndianInt(const void* const bytes) {
+  return swap(*static_cast<const uint32*>(bytes));
+}
+inline uint64 ByteOrder::bigEndianInt64(const void* const bytes) {
+  return swap(*static_cast<const uint64*>(bytes));
+}
+inline uint16 ByteOrder::bigEndianShort(const void* const bytes) {
+  return swap(*static_cast<const uint16*>(bytes));
+}
+inline bool ByteOrder::isBigEndian() { return false; }
+#else
+inline uint16 ByteOrder::swapIfBigEndian(const uint16 v) { return swap(v); }
+inline uint32 ByteOrder::swapIfBigEndian(const uint32 v) { return swap(v); }
+inline uint64 ByteOrder::swapIfBigEndian(const uint64 v) { return swap(v); }
+inline uint16 ByteOrder::swapIfLittleEndian(const uint16 v) { return v; }
+inline uint32 ByteOrder::swapIfLittleEndian(const uint32 v) { return v; }
+inline uint64 ByteOrder::swapIfLittleEndian(const uint64 v) { return v; }
+inline uint32 ByteOrder::littleEndianInt(const void* const bytes) {
+  return swap(*static_cast<const uint32*>(bytes));
+}
+inline uint64 ByteOrder::littleEndianInt64(const void* const bytes) {
+  return swap(*static_cast<const uint64*>(bytes));
+}
+inline uint16 ByteOrder::littleEndianShort(const void* const bytes) {
+  return swap(*static_cast<const uint16*>(bytes));
+}
+inline uint32 ByteOrder::bigEndianInt(const void* const bytes) {
+  return *static_cast<const uint32*>(bytes);
+}
+inline uint64 ByteOrder::bigEndianInt64(const void* const bytes) {
+  return *static_cast<const uint64*>(bytes);
+}
+inline uint16 ByteOrder::bigEndianShort(const void* const bytes) {
+  return *static_cast<const uint16*>(bytes);
+}
+inline bool ByteOrder::isBigEndian() { return true; }
+#endif
+
+inline int ByteOrder::littleEndian24Bit(const void* const bytes) {
+  return (((int)static_cast<const int8*>(bytes)[2]) << 16) |
+         (((int)static_cast<const uint8*>(bytes)[1]) << 8) |
+         ((int)static_cast<const uint8*>(bytes)[0]);
+}
+inline int ByteOrder::bigEndian24Bit(const void* const bytes) {
+  return (((int)static_cast<const int8*>(bytes)[0]) << 16) |
+         (((int)static_cast<const uint8*>(bytes)[1]) << 8) |
+         ((int)static_cast<const uint8*>(bytes)[2]);
+}
+inline void ByteOrder::littleEndian24BitToChars(const int value,
+                                                void* const destBytes) {
+  static_cast<uint8*>(destBytes)[0] = (uint8)value;
+  static_cast<uint8*>(destBytes)[1] = (uint8)(value >> 8);
+  static_cast<uint8*>(destBytes)[2] = (uint8)(value >> 16);
+}
+inline void ByteOrder::bigEndian24BitToChars(const int value,
+                                             void* const destBytes) {
+  static_cast<uint8*>(destBytes)[0] = (uint8)(value >> 16);
+  static_cast<uint8*>(destBytes)[1] = (uint8)(value >> 8);
+  static_cast<uint8*>(destBytes)[2] = (uint8)value;
+}
+}
+#endif  // BYTEORDER_H_INCLUDED

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/ClientRPCHook.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/ClientRPCHook.cpp b/rocketmq-cpp/src/common/ClientRPCHook.cpp
new file mode 100755
index 0000000..3fa3d78
--- /dev/null
+++ b/rocketmq-cpp/src/common/ClientRPCHook.cpp
@@ -0,0 +1,80 @@
+/*
+ * 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 "ClientRPCHook.h"
+#include "CommandHeader.h"
+#include "Logging.h"
+#include "spas_client.h"
+#include "string"
+
+namespace rocketmq {
+
+const string SessionCredentials::AccessKey = "AccessKey";
+const string SessionCredentials::SecretKey = "SecretKey";
+const string SessionCredentials::Signature = "Signature";
+const string SessionCredentials::SignatureMethod = "SignatureMethod";
+const string SessionCredentials::ONSChannelKey = "OnsChannel";
+
+void ClientRPCHook::doBeforeRequest(const string& remoteAddr,
+                                    RemotingCommand& request) {
+  CommandHeader* header = request.getCommandHeader();
+
+  map<string, string> requestMap;
+  string totalMsg;
+
+  requestMap.insert(pair<string, string>(SessionCredentials::AccessKey,
+                                         sessionCredentials.getAccessKey()));
+  requestMap.insert(pair<string, string>(SessionCredentials::ONSChannelKey,
+                                         sessionCredentials.getAuthChannel()));
+
+  LOG_DEBUG("before insert declared filed,MAP SIZE is:%zu", requestMap.size());
+  if (header != NULL) {
+    header->SetDeclaredFieldOfCommandHeader(requestMap);
+  }
+  LOG_DEBUG("after insert declared filed, MAP SIZE is:%zu", requestMap.size());
+
+  map<string, string>::iterator it = requestMap.begin();
+  for (; it != requestMap.end(); ++it) {
+    totalMsg.append(it->second);
+  }
+  if (request.getMsgBody().length() > 0) {
+    LOG_DEBUG("msgBody is:%s, msgBody length is:%zu",
+              request.getMsgBody().c_str(), request.getMsgBody().length());
+
+    totalMsg.append(request.getMsgBody());
+  }
+  LOG_DEBUG("total msg info are:%s, size is:%zu", totalMsg.c_str(),
+            totalMsg.size());
+  char* pSignature =
+      metaqSignature::spas_sign(totalMsg.c_str(), totalMsg.size(),
+                                sessionCredentials.getSecretKey().c_str());
+  // char *pSignature = spas_sign(totalMsg.c_str(),
+  // sessionCredentials.getSecretKey().c_str());
+
+  if (pSignature != NULL) {
+    string signature(static_cast<const char*>(pSignature));
+    request.addExtField(SessionCredentials::Signature, signature);
+    request.addExtField(SessionCredentials::AccessKey,
+                        sessionCredentials.getAccessKey());
+    request.addExtField(SessionCredentials::ONSChannelKey,
+                        sessionCredentials.getAuthChannel());
+    metaqSignature::spas_mem_free(pSignature);
+  } else {
+    LOG_ERROR("signature for request failed");
+  }
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/ClientRPCHook.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/ClientRPCHook.h b/rocketmq-cpp/src/common/ClientRPCHook.h
new file mode 100755
index 0000000..8abab31
--- /dev/null
+++ b/rocketmq-cpp/src/common/ClientRPCHook.h
@@ -0,0 +1,51 @@
+/*
+ * 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 __CLIENTRPCHOOK_H__
+#define __CLIENTRPCHOOK_H__
+
+#include "RemotingCommand.h"
+#include "RocketMQClient.h"
+#include "SessionCredentials.h"
+namespace rocketmq {
+class RPCHook {
+ public:
+  RPCHook() {}
+  virtual ~RPCHook() {}
+  virtual void doBeforeRequest(const string& remoteAddr,
+                               RemotingCommand& request) = 0;
+  virtual void doAfterResponse(RemotingCommand& request,
+                               RemotingCommand& response) = 0;
+};
+
+class ClientRPCHook : public RPCHook {
+ private:
+  SessionCredentials sessionCredentials;
+
+ public:
+  ClientRPCHook(const SessionCredentials& session_credentials)
+      : sessionCredentials(session_credentials) {}
+  virtual ~ClientRPCHook() {}
+
+  virtual void doBeforeRequest(const string& remoteAddr,
+                               RemotingCommand& request);
+
+  virtual void doAfterResponse(RemotingCommand& request,
+                               RemotingCommand& response) {}
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/CommunicationMode.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/CommunicationMode.h b/rocketmq-cpp/src/common/CommunicationMode.h
new file mode 100755
index 0000000..9d9b283
--- /dev/null
+++ b/rocketmq-cpp/src/common/CommunicationMode.h
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+#ifndef __COMMUNICATIONMODE_H__
+#define __COMMUNICATIONMODE_H__
+
+namespace rocketmq {
+//<!***************************************************************************
+enum CommunicationMode { ComMode_SYNC, ComMode_ASYNC, ComMode_ONEWAY };
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif


[14/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
upload rocketmq-cpp code

remove unused boost file

update README.md

update readme.md

update readme.md

update readme

add gitkeep to bin

update readme.md

update apache license info

remove static dependency

update readme.md

add jsoncpp and libevent install discription

udpate makefile

update jsoncpp version

unify dependency

update readme

support cmake

instead alog by boost::log

For cmake, remove alog

remove alog comment

[add rocketmq-cpp code #27] add cmake to readme

use rocketmq namespace, update json version to 0.10.6

update cmakelist

update json version to 0.10.6

remove makefile

add default search path for jsoncpp and libevent

Update README.md

fix could not find jsoncpp issue

support build on new g++ version

Remove built .o files


Project: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/commit/70ce5c77
Tree: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/tree/70ce5c77
Diff: http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/diff/70ce5c77

Branch: refs/heads/master
Commit: 70ce5c770ecbe2e76f39d8c76ad12db518b540b7
Parents: 343ab19
Author: qiwei.wqw <qi...@alibaba-inc.com>
Authored: Tue Aug 8 10:55:19 2017 +0800
Committer: shutian.lzh <sh...@alibaba-inc.com>
Committed: Mon Sep 4 13:03:51 2017 +0800

----------------------------------------------------------------------
 rocketmq-cpp/.gitignore                         |   40 +
 rocketmq-cpp/CMakeLists.txt                     |  161 ++
 rocketmq-cpp/README.md                          |   98 +
 rocketmq-cpp/bin/.gitkeep                       |    1 +
 rocketmq-cpp/cmake/FindJsoncpp.cmake            |   90 +
 rocketmq-cpp/cmake/FindLibevent.cmake           |  111 +
 rocketmq-cpp/deploy.sh                          |   26 +
 rocketmq-cpp/example/AsyncProducer.cpp          |  125 +
 rocketmq-cpp/example/AsyncPushConsumer.cpp      |   94 +
 rocketmq-cpp/example/CMakeLists.txt             |   49 +
 rocketmq-cpp/example/OrderProducer.cpp          |   99 +
 rocketmq-cpp/example/OrderlyPushConsumer.cpp    |   95 +
 rocketmq-cpp/example/PullConsumer.cpp           |  108 +
 rocketmq-cpp/example/PushConsumer.cpp           |  110 +
 rocketmq-cpp/example/README.md                  |    4 +
 rocketmq-cpp/example/SendDelayMsg.cpp           |   51 +
 rocketmq-cpp/example/SyncProducer.cpp           |  101 +
 rocketmq-cpp/example/common.h                   |  183 ++
 rocketmq-cpp/format.sh                          |   62 +
 rocketmq-cpp/include/Arg_helper.h               |   42 +
 rocketmq-cpp/include/AsyncCallback.h            |   66 +
 rocketmq-cpp/include/ConsumeType.h              |   61 +
 rocketmq-cpp/include/DefaultMQProducer.h        |  106 +
 rocketmq-cpp/include/DefaultMQPullConsumer.h    |  155 ++
 rocketmq-cpp/include/DefaultMQPushConsumer.h    |  160 ++
 rocketmq-cpp/include/MQClient.h                 |  203 ++
 rocketmq-cpp/include/MQClientException.h        |   95 +
 rocketmq-cpp/include/MQConsumer.h               |   70 +
 rocketmq-cpp/include/MQMessage.h                |  105 +
 rocketmq-cpp/include/MQMessageExt.h             |  117 +
 rocketmq-cpp/include/MQMessageListener.h        |   90 +
 rocketmq-cpp/include/MQMessageQueue.h           |   64 +
 rocketmq-cpp/include/MQProducer.h               |   62 +
 rocketmq-cpp/include/MQSelector.h               |   33 +
 rocketmq-cpp/include/MQueueListener.h           |   34 +
 rocketmq-cpp/include/PullResult.h               |   69 +
 rocketmq-cpp/include/QueryResult.h              |   43 +
 rocketmq-cpp/include/RocketMQClient.h           |   47 +
 rocketmq-cpp/include/SendMessageHook.h          |   47 +
 rocketmq-cpp/include/SendResult.h               |   58 +
 rocketmq-cpp/include/SessionCredentials.h       |   62 +
 rocketmq-cpp/libs/CMakeLists.txt                |   16 +
 rocketmq-cpp/libs/signature/CMakeLists.txt      |   25 +
 rocketmq-cpp/libs/signature/Makefile            |   63 +
 rocketmq-cpp/libs/signature/include/base64.h    |   54 +
 rocketmq-cpp/libs/signature/include/hmac.h      |   55 +
 .../libs/signature/include/param_list.h         |   39 +
 rocketmq-cpp/libs/signature/include/sha1.h      |   93 +
 rocketmq-cpp/libs/signature/include/sha256.h    |   91 +
 rocketmq-cpp/libs/signature/include/sha512.h    |   95 +
 .../libs/signature/include/spas_client.h        |   85 +
 rocketmq-cpp/libs/signature/include/u64.h       |  159 ++
 rocketmq-cpp/libs/signature/lib/.gitkeep        |    1 +
 .../libs/signature/msvc13/Metaqsignature.sln    |   22 +
 .../signature/msvc13/Metaqsignature.vcxproj     |  168 ++
 .../msvc13/Metaqsignature.vcxproj.filters       |   66 +
 rocketmq-cpp/libs/signature/src/base64.c        |  437 +++
 rocketmq-cpp/libs/signature/src/hmac.c          |  165 ++
 rocketmq-cpp/libs/signature/src/param_list.c    |  130 +
 rocketmq-cpp/libs/signature/src/sha1.c          |  516 ++++
 rocketmq-cpp/libs/signature/src/sha256.c        |  566 ++++
 rocketmq-cpp/libs/signature/src/sha512.c        |  616 ++++
 rocketmq-cpp/libs/signature/src/spas_client.c   |  508 ++++
 rocketmq-cpp/project/CMakeLists.txt             |   63 +
 rocketmq-cpp/project/tool.mak                   |   21 +
 rocketmq-cpp/src/MQClientAPIImpl.cpp            |  922 ++++++
 rocketmq-cpp/src/MQClientAPIImpl.h              |  189 ++
 rocketmq-cpp/src/MQClientFactory.cpp            | 1142 ++++++++
 rocketmq-cpp/src/MQClientFactory.h              |  220 ++
 rocketmq-cpp/src/MQClientManager.cpp            |   54 +
 rocketmq-cpp/src/MQClientManager.h              |   49 +
 rocketmq-cpp/src/common/Arg_helper.cpp          |   67 +
 rocketmq-cpp/src/common/AsyncArg.h              |   36 +
 rocketmq-cpp/src/common/AsyncCallbackWrap.cpp   |  168 ++
 rocketmq-cpp/src/common/AsyncCallbackWrap.h     |   84 +
 rocketmq-cpp/src/common/ByteOrder.h             |  179 ++
 rocketmq-cpp/src/common/ClientRPCHook.cpp       |   80 +
 rocketmq-cpp/src/common/ClientRPCHook.h         |   51 +
 rocketmq-cpp/src/common/CommunicationMode.h     |   27 +
 rocketmq-cpp/src/common/FastDelegate.h          | 2626 ++++++++++++++++++
 rocketmq-cpp/src/common/FilterAPI.h             |   63 +
 rocketmq-cpp/src/common/InputStream.cpp         |  149 +
 rocketmq-cpp/src/common/InputStream.h           |  241 ++
 rocketmq-cpp/src/common/MQClient.cpp            |  201 ++
 rocketmq-cpp/src/common/MQVersion.cpp           |   31 +
 rocketmq-cpp/src/common/MQVersion.h             |  229 ++
 rocketmq-cpp/src/common/MemoryInputStream.cpp   |   55 +
 rocketmq-cpp/src/common/MemoryInputStream.h     |   80 +
 rocketmq-cpp/src/common/MemoryOutputStream.cpp  |  148 +
 rocketmq-cpp/src/common/MemoryOutputStream.h    |  115 +
 rocketmq-cpp/src/common/MessageSysFlag.cpp      |   37 +
 rocketmq-cpp/src/common/MessageSysFlag.h        |   38 +
 rocketmq-cpp/src/common/NamesrvConfig.h         |   56 +
 rocketmq-cpp/src/common/OutputStream.cpp        |  129 +
 rocketmq-cpp/src/common/OutputStream.h          |  184 ++
 rocketmq-cpp/src/common/PermName.cpp            |   55 +
 rocketmq-cpp/src/common/PermName.h              |   39 +
 rocketmq-cpp/src/common/PullSysFlag.cpp         |   70 +
 rocketmq-cpp/src/common/PullSysFlag.h           |   41 +
 rocketmq-cpp/src/common/ServiceState.h          |   25 +
 .../src/common/SubscriptionGroupConfig.h        |   49 +
 rocketmq-cpp/src/common/TopAddressing.cpp       |  108 +
 rocketmq-cpp/src/common/TopAddressing.h         |   48 +
 rocketmq-cpp/src/common/TopicConfig.cpp         |  104 +
 rocketmq-cpp/src/common/TopicConfig.h           |   61 +
 rocketmq-cpp/src/common/TopicFilterType.h       |   34 +
 rocketmq-cpp/src/common/UtilAll.cpp             |  301 ++
 rocketmq-cpp/src/common/UtilAll.h               |  131 +
 rocketmq-cpp/src/common/Validators.cpp          |  115 +
 rocketmq-cpp/src/common/Validators.h            |   43 +
 rocketmq-cpp/src/common/VirtualEnvUtil.cpp      |   56 +
 rocketmq-cpp/src/common/VirtualEnvUtil.h        |   36 +
 rocketmq-cpp/src/common/dataBlock.cpp           |  173 ++
 rocketmq-cpp/src/common/dataBlock.h             |  185 ++
 rocketmq-cpp/src/common/sync_http_client.cpp    |  156 ++
 rocketmq-cpp/src/common/sync_http_client.h      |   29 +
 rocketmq-cpp/src/common/url.cpp                 |   63 +
 rocketmq-cpp/src/common/url.h                   |   38 +
 rocketmq-cpp/src/consumer/AllocateMQStrategy.h  |   98 +
 .../ConsumeMessageConcurrentlyService.cpp       |  147 +
 .../consumer/ConsumeMessageOrderlyService.cpp   |  214 ++
 rocketmq-cpp/src/consumer/ConsumeMsgService.h   |  112 +
 .../src/consumer/DefaultMQPullConsumer.cpp      |  371 +++
 .../src/consumer/DefaultMQPushConsumer.cpp      |  897 ++++++
 rocketmq-cpp/src/consumer/FindBrokerResult.h    |   33 +
 rocketmq-cpp/src/consumer/OffsetStore.cpp       |  344 +++
 rocketmq-cpp/src/consumer/OffsetStore.h         |  108 +
 rocketmq-cpp/src/consumer/PullAPIWrapper.cpp    |  139 +
 rocketmq-cpp/src/consumer/PullAPIWrapper.h      |   67 +
 rocketmq-cpp/src/consumer/PullRequest.cpp       |  244 ++
 rocketmq-cpp/src/consumer/PullRequest.h         |   95 +
 rocketmq-cpp/src/consumer/PullResult.cpp        |   54 +
 rocketmq-cpp/src/consumer/PullResultExt.h       |   45 +
 rocketmq-cpp/src/consumer/Rebalance.cpp         |  677 +++++
 rocketmq-cpp/src/consumer/Rebalance.h           |  123 +
 rocketmq-cpp/src/consumer/SubscriptionData.cpp  |  120 +
 rocketmq-cpp/src/consumer/SubscriptionData.h    |   62 +
 rocketmq-cpp/src/dllmain.cpp                    |   34 +
 rocketmq-cpp/src/log/Logging.cpp                |   96 +
 rocketmq-cpp/src/log/Logging.h                  |   75 +
 rocketmq-cpp/src/message/MQDecoder.cpp          |  254 ++
 rocketmq-cpp/src/message/MQDecoder.h            |   57 +
 rocketmq-cpp/src/message/MQMessage.cpp          |  197 ++
 rocketmq-cpp/src/message/MQMessageExt.cpp       |  148 +
 rocketmq-cpp/src/message/MQMessageId.h          |   45 +
 rocketmq-cpp/src/message/MQMessageQueue.cpp     |   98 +
 rocketmq-cpp/src/producer/DefaultMQProducer.cpp |  494 ++++
 rocketmq-cpp/src/producer/SendResult.cpp        |   60 +
 rocketmq-cpp/src/producer/TopicPublishInfo.h    |  277 ++
 rocketmq-cpp/src/protocol/CommandHeader.cpp     |  592 ++++
 rocketmq-cpp/src/protocol/CommandHeader.h       |  431 +++
 .../src/protocol/ConsumerRunningInfo.cpp        |  109 +
 rocketmq-cpp/src/protocol/ConsumerRunningInfo.h |   50 +
 rocketmq-cpp/src/protocol/HeartbeatData.h       |  143 +
 rocketmq-cpp/src/protocol/KVTable.h             |   41 +
 rocketmq-cpp/src/protocol/LockBatchBody.cpp     |  124 +
 rocketmq-cpp/src/protocol/LockBatchBody.h       |   80 +
 rocketmq-cpp/src/protocol/MQProtos.h            |  199 ++
 rocketmq-cpp/src/protocol/MessageQueue.cpp      |  108 +
 rocketmq-cpp/src/protocol/MessageQueue.h        |   56 +
 rocketmq-cpp/src/protocol/ProcessQueueInfo.h    |   86 +
 rocketmq-cpp/src/protocol/RemotingCommand.cpp   |  253 ++
 rocketmq-cpp/src/protocol/RemotingCommand.h     |   90 +
 .../src/protocol/RemotingSerializable.h         |   32 +
 rocketmq-cpp/src/protocol/TopicList.h           |   35 +
 rocketmq-cpp/src/protocol/TopicRouteData.h      |  177 ++
 .../src/thread/disruptor/batch_descriptor.h     |   70 +
 .../src/thread/disruptor/claim_strategy.h       |  231 ++
 .../src/thread/disruptor/event_processor.h      |  130 +
 .../src/thread/disruptor/event_publisher.h      |   50 +
 .../src/thread/disruptor/exception_handler.h    |   59 +
 rocketmq-cpp/src/thread/disruptor/exceptions.h  |   38 +
 rocketmq-cpp/src/thread/disruptor/interface.h   |  278 ++
 rocketmq-cpp/src/thread/disruptor/ring_buffer.h |   90 +
 rocketmq-cpp/src/thread/disruptor/sequence.h    |  139 +
 .../src/thread/disruptor/sequence_barrier.h     |   92 +
 rocketmq-cpp/src/thread/disruptor/sequencer.h   |  190 ++
 rocketmq-cpp/src/thread/disruptor/utils.h       |   35 +
 .../src/thread/disruptor/wait_strategy.h        |  377 +++
 rocketmq-cpp/src/thread/disruptorLFQ.h          |  113 +
 rocketmq-cpp/src/thread/task_queue.cpp          |  103 +
 rocketmq-cpp/src/thread/task_queue.h            |  679 +++++
 .../src/transport/ClientRemotingProcessor.cpp   |  146 +
 .../src/transport/ClientRemotingProcessor.h     |   39 +
 rocketmq-cpp/src/transport/ResponseFuture.cpp   |  176 ++
 rocketmq-cpp/src/transport/ResponseFuture.h     |   83 +
 rocketmq-cpp/src/transport/SocketUtil.cpp       |   86 +
 rocketmq-cpp/src/transport/SocketUtil.h         |   62 +
 .../src/transport/TcpRemotingClient.cpp         |  728 +++++
 rocketmq-cpp/src/transport/TcpRemotingClient.h  |  133 +
 rocketmq-cpp/src/transport/TcpTransport.cpp     |  330 +++
 rocketmq-cpp/src/transport/TcpTransport.h       |   91 +
 192 files changed, 30000 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/.gitignore
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/.gitignore b/rocketmq-cpp/.gitignore
new file mode 100644
index 0000000..23d1d0a
--- /dev/null
+++ b/rocketmq-cpp/.gitignore
@@ -0,0 +1,40 @@
+# Compiled source #
+###################
+*.com
+*.class
+*.dll
+*.exe
+*.o
+*.lo
+*.so
+*.a
+/build
+# Packages #
+############
+# it's better to unpack these files and commit the raw source
+# git has its own built in compression methods
+*.7z
+*.dmg
+*.gz
+*.iso
+*.jar
+*.rar
+*.tar
+*.zip
+rpm/.dep_create/*
+# Logs and databases #
+######################
+*.log
+*.sql
+*.sqlite
+
+# OS generated files #
+######################
+.DS_Store
+.DS_Store?
+._*
+.Spotlight-V100
+.Trashes
+Icon?
+ehthumbs.db
+Thumbs.db

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/CMakeLists.txt b/rocketmq-cpp/CMakeLists.txt
new file mode 100755
index 0000000..2d36856
--- /dev/null
+++ b/rocketmq-cpp/CMakeLists.txt
@@ -0,0 +1,161 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+ 
+cmake_minimum_required(VERSION 2.8)
+
+
+# CMake complains if we don't have this.
+if (COMMAND cmake_policy)
+    cmake_policy(SET CMP0003 NEW)
+endif()
+
+# We're escaping quotes in the Windows version number, because
+# for some reason CMake won't do it at config version 2.4.7
+# It seems that this restores the newer behaviour where define
+# args are not auto-escaped.
+if (COMMAND cmake_policy)
+    cmake_policy(SET CMP0005 NEW)
+endif()
+
+# First, declare project (important for prerequisite checks).
+project(all)
+set(CMAKE_MODULE_PATH ${PROJECT_SOURCE_DIR}/cmake)
+set(CMAKE_ALLOW_LOOSE_LOOP_CONSTRUCTS ON)
+set(CMAKE_VERBOSE_MAKEFILE 1)
+
+#Find dependency 
+#set(BOOST_INCLUDEDIR xxx)
+find_package(Boost 1.56.0)
+if(Boost_FOUND)
+   include_directories(${Boost_INCLUDE_DIRS})
+endif()
+       
+#set(LIBEVENT_INCLUDE_DIR xxx)
+find_package(Libevent 2.0.22)
+if(LIBEVENT_FOUND)
+   include_directories(${LIBEVENT_INCLUDE_DIRS})
+endif()
+            
+#set(JSONCPP_INCLUDE_DIR xxx)
+find_package(Jsoncpp 0.10.6)
+if(JSONCPP_FOUND)
+   include_directories(${JSONCPP_INCLUDE_DIR})
+endif()
+
+# put binaries in a different dir to make them easier to find.
+set(EXECUTABLE_OUTPUT_PATH ${PROJECT_SOURCE_DIR}/bin)
+set(LIBRARY_OUTPUT_PATH ${PROJECT_SOURCE_DIR}/bin)
+
+# for unix, put debug files in a separate bin "debug" dir.
+# release bin files should stay in the root of the bin dir.
+# if (CMAKE_GENERATOR STREQUAL "Unix Makefiles")
+#     if (CMAKE_BUILD_TYPE STREQUAL Debug)
+#         set(EXECUTABLE_OUTPUT_PATH ${PROJECT_SOURCE_DIR}/bin/debug)
+#         set(LIBRARY_OUTPUT_PATH ${PROJECT_SOURCE_DIR}/bin/debug)
+#     endif()
+# endif()
+
+if(NOT CMAKE_BUILD_TYPE)
+  set(CMAKE_BUILD_TYPE "Release")
+endif()
+
+set(C_FLAGS
+  -g
+  -Wall
+  -Wno-deprecated
+  -fPIC
+  -fno-strict-aliasing 
+)
+set(CXX_FLAGS
+ -g
+ -Wall
+ -Wno-deprecated
+ -fPIC
+ -fno-strict-aliasing
+ -std=c++0x
+ # -finline-limit=1000
+ # -Wextra
+ # -pedantic
+ # -pedantic-errors
+ # -D_FILE_OFFSET_BITS=64
+ # -DVALGRIND
+ # -DCHECK_PTHREAD_RETURN_VALUE
+ # -Werror
+ # -Wconversion
+ # -Wno-unused-parameter
+ # -Wunused-but-set-variable
+ # -Wold-style-cast
+ # -Woverloaded-virtual
+ # -Wpointer-arith
+ # -Wshadow
+ # -Wwrite-strings
+ # -Wdeprecated-declarations
+ # -march=native
+ # -MMD
+ # -std=c++0x
+ # -rdynamic
+ )
+
+if(CMAKE_BUILD_BITS EQUAL 32)
+  list(APPEND CXX_FLAGS "-m32")
+else() #not-condition
+  list(APPEND CXX_FLAGS "-m64")
+endif()
+
+string(REPLACE ";" " " CMAKE_CXX_FLAGS "${CXX_FLAGS}")
+string(REPLACE ";" " " CMAKE_C_FLAGS "${C_FLAGS}")
+set(CMAKE_CXX_FLAGS_DEBUG   "-O0 -DDEBUG")
+set(CMAKE_CXX_FLAGS_RELEASE "-O3 -DNDEBUG")
+
+# Declare deplibs, so we can use list in linker later. There's probably
+# a more elegant way of doing this; with SCons, when you check for the
+# lib, it is automatically passed to the linker.
+set(deplibs)
+
+# For some reason, the check_function_exists macro doesn't detect
+# the inet_aton on some pure Unix platforms (e.g. sunos5). So we
+# need to do a more detailed check and also include some extra deplibs.
+list(APPEND deplibs dl)
+list(APPEND deplibs pthread)
+list(APPEND deplibs rt)
+list(APPEND deplibs z)
+
+# add include dir for bsd (posix uses /usr/include/)
+set(CMAKE_INCLUDE_PATH "${CMAKE_INCLUDE_PATH}:/usr/local/include")
+
+# For config.h, set some static values; it may be a good idea to make
+# these values dynamic for non-standard UNIX compilers.
+set(ACCEPT_TYPE_ARG3 socklen_t)
+set(HAVE_CXX_BOOL 1)
+set(HAVE_CXX_CASTS 1)
+set(HAVE_CXX_EXCEPTIONS 1)
+set(HAVE_CXX_MUTABLE 1)
+set(HAVE_CXX_STDLIB 1)
+set(HAVE_PTHREAD_SIGNAL 1)
+set(SELECT_TYPE_ARG1 int)
+set(SELECT_TYPE_ARG234 "(fd_set *)")
+set(SELECT_TYPE_ARG5 "(struct timeval *)")
+set(STDC_HEADERS 1)
+set(TIME_WITH_SYS_TIME 1)
+set(HAVE_SOCKLEN_T 1)
+
+# For config.h, save the results based on a template (config.h.in).
+# configure_file(res/config.h.in ${root_dir}/config.h)
+
+# add_definitions(-DSYSAPI_UNIX=1 -DHAVE_CONFIG_H)
+
+add_subdirectory(libs)
+add_subdirectory(project)
+add_subdirectory(example)

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/README.md
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/README.md b/rocketmq-cpp/README.md
index e69de29..45abce6 100644
--- a/rocketmq-cpp/README.md
+++ b/rocketmq-cpp/README.md
@@ -0,0 +1,98 @@
+================build and install========================
+- linux platform:
+
+  -1. install dependency
+	- 1>. install libevent 2.0.22 dependency
+	  - <1>. dowload libevent 2.0.22
+	    - https://github.com/libevent/libevent/releases/download/release-2.0.22-stable/libevent-2.0.22-stable.tar.gz
+	  - <2>. build and install libevent
+	    - ./configure
+	    - make
+	    - make install 
+	- 2>. install JsonCPP 0.7.0 dependency
+  	  - <1> download jsoncpp 0.7.0
+	    - https://github.com/open-source-parsers/jsoncpp/archive/0.10.6.zip
+	  - <2> build and install jsoncpp
+	    - cmake .
+	    - make
+	    - make install
+	- 3>. install boost 1.56.0 dependency
+	  - <1>. dowload boost 1.56.0
+	    - http://www.boost.org/users/history/version_1_56_0.html
+	  - <2>. build and install boost 1.56.0
+	    - <1>. cd path/to/boost_1_56_0
+	    - <2>. config boost:./bootstrap.sh
+	    - <3>. build boost:     
+	      - build static boost lib: ./b2 link=static runtime-link=static
+	      - build dynamic boost lib: ./b2 link=shared runtime-link=shared
+	  - <3>. install boost: ./b2 install
+    
+  -2. make&install
+  	- default install path:
+		- header files: /usr/local/include
+		- lib: /usr/local/lib
+    - 1>.make&install by cmake	
+        - <1>. cmake will auto find_package, if failes, change BOOST_INCLUDEDIR/LIBEVENT_INCLUDE_DIR/JSONCPP_INCLUDE_DIR in CMakeList.txt, according to its real install path
+        - <2>. make
+        - <3>. make install
+	
+- Windows platform:
+  - will be supported later
+
+
+
+- check verion:
+  - strings librocketmq.so |grep VERSION
+
+- log path:$HOME/logs/metaq-client4cpp
+
+- Before Run:
+  - export LD_LIBRARY_PATH=/xxx/rocketmq-client4cpp/bin/:$LD_LIBRARY_PATH;LD_LIBRARY_PATH=/A/lib:$LD_LIBRARY_PATH
+
+=================meaning of each parameter===================
+- -n	: nameserver addr, if not set -n and -i ,no nameSrv will be got
+- -i	: nameserver domain name,  if not set -n and -i ,no nameSrv will be got
+- Notice: oper should only set one option from -n and -i, 
+- -g	: groupName, contains producer groupName and consumer groupName
+- -t	: msg topic
+- -m	: message count(default value:1)
+- -c 	: msg content(default value: only test)
+- -b	: consume model(default value: CLUSTER)
+- -a	: set sync push(default value: async)
+- -r	: setup retry times(default value:5 times)
+- -u	: select active broker to send msg(default value: false)
+- -d	: use AutoDeleteSendcallback by cpp client(defalut value: false)
+- -T	: thread count of send msg or consume msg(defalut value: system cpu core number)
+- -v 	: print more details information
+
+- Example:
+  - sync producer: ./SyncProducer -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr
+  - async producer: ./AsyncProducer  -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr 
+  - send delay msg: ./SendDelayMsg  -g producerGroup -t topic -c msgContent -n nameServerAddr
+  - sync pushConsumer: ./PushConsumer  -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr -s sync
+  - async pushConsumer: ./AsyncPushConsumer  -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr
+  - orderly sync pushConsumer:  ./OrderlyPushConsumer -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr -s sync
+  - orderly async pushConsumer: ./OrderlyPushConsumer -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr
+  - sync pullConsumer:./PullConsumer  -g producerGroup -t topic -c msgContent -m msgCount -n nameServerAddr 
+
+==================================Notice=============================================
+- producer must invoke following interface:
+  - DefaultMQProducer g_producer("please_rename_unique_group_name");
+  - g_producer.start();
+  - g_producer.send(...);
+  - g_producer.shutdown();
+
+- pullconsumer must invoke following interface:
+  - DefaultMQPullConsumer     g_consumer("please_rename_unique_group_name");
+  - g_consumer.start();
+  - g_consumer.fetchSubscribeMessageQueues(..., ...);
+  - g_consumer.pull(...)
+  - g_consumer.shutdown();
+
+- pushconsumer must invoke following interface:
+  - DefaultMQPushConsumer g_consumer("please_rename_unique_group_name_1");
+  - g_consumer.subscribe("test_topic", "*");
+  - g_consumer.registerMessageListener(listener);
+  - g_consumer.start();
+  - g_consumer.shutdown();
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/bin/.gitkeep
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/bin/.gitkeep b/rocketmq-cpp/bin/.gitkeep
new file mode 100644
index 0000000..533bd5f
--- /dev/null
+++ b/rocketmq-cpp/bin/.gitkeep
@@ -0,0 +1 @@
+#keep

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/cmake/FindJsoncpp.cmake
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/cmake/FindJsoncpp.cmake b/rocketmq-cpp/cmake/FindJsoncpp.cmake
new file mode 100755
index 0000000..456d97f
--- /dev/null
+++ b/rocketmq-cpp/cmake/FindJsoncpp.cmake
@@ -0,0 +1,90 @@
+# 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.
+
+# Find jsoncpp
+#
+# Find the jsoncpp includes and library
+#
+# if you nee to add a custom library search path, do it via via CMAKE_PREFIX_PATH
+#
+# This module defines
+#  JSONCPP_INCLUDE_DIR, where to find header, etc.
+#  JSONCPP_LIBRARY, the libraries needed to use jsoncpp.
+#  JSONCPP_FOUND, If false, do not try to use jsoncpp.
+#  JSONCPP_INCLUDE_PREFIX, include prefix for jsoncpp.
+#  jsoncpp_lib_static imported library.
+
+# only look in default directories
+find_path(
+	JSONCPP_INCLUDE_DIR
+	NAMES json/json.h jsoncpp/json/json.h
+    PATHS /usr/include/jsoncp /usr/local/include/jsoncpp 
+    DOC "jsoncpp include dir"
+)
+
+find_library(
+    JSONCPP_LIBRARY
+    NAMES jsoncpp
+    PATHS /usr/lib /usr/local/lib
+    DOC "jsoncpp library"
+)
+
+add_library(jsoncpp_lib_static UNKNOWN IMPORTED)
+set_target_properties(
+	jsoncpp_lib_static
+	PROPERTIES
+	IMPORTED_LOCATION "${JSONCPP_LIBRARY}"
+	INTERFACE_INCLUDE_DIRECTORIES "${JSONCPP_INCLUDE_DIR}"
+)
+
+# debug library on windows
+# same naming convention as in qt (appending debug library with d)
+# boost is using the same "hack" as us with "optimized" and "debug"
+if ("${CMAKE_CXX_COMPILER_ID}" STREQUAL "MSVC")
+	find_library(
+		JSONCPP_LIBRARY_DEBUG
+		NAMES jsoncppd
+		DOC "jsoncpp debug library"
+	)
+
+	set_target_properties(
+		jsoncpp_lib_static
+		PROPERTIES
+		IMPORTED_LOCATION_DEBUG "${JSONCPP_LIBRARY_DEBUG}"
+	)
+	set(JSONCPP_LIBRARY optimized ${JSONCPP_LIBRARY} debug ${JSONCPP_LIBRARY_DEBUG})
+
+endif()
+
+# find JSONCPP_INCLUDE_PREFIX
+find_path(
+    JSONCPP_INCLUDE_PREFIX
+    NAMES json.h
+    PATH_SUFFIXES jsoncpp/json json
+)
+
+if (${JSONCPP_INCLUDE_PREFIX} MATCHES "jsoncpp")
+    set(JSONCPP_INCLUDE_PREFIX "jsoncpp/json")
+else()
+    set(JSONCPP_INCLUDE_PREFIX "json")
+endif()
+
+
+
+# handle the QUIETLY and REQUIRED arguments and set JSONCPP_FOUND to TRUE
+# if all listed variables are TRUE, hide their existence from configuration view
+include(FindPackageHandleStandardArgs)
+find_package_handle_standard_args(jsoncpp DEFAULT_MSG JSONCPP_INCLUDE_DIR JSONCPP_LIBRARY)
+mark_as_advanced (JSONCPP_INCLUDE_DIR JSONCPP_LIBRARY)

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/cmake/FindLibevent.cmake
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/cmake/FindLibevent.cmake b/rocketmq-cpp/cmake/FindLibevent.cmake
new file mode 100755
index 0000000..dc09fb3
--- /dev/null
+++ b/rocketmq-cpp/cmake/FindLibevent.cmake
@@ -0,0 +1,111 @@
+# 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.
+
+# - Try to find libevent
+#.rst
+# FindLibevent
+# ------------
+#
+# Find Libevent include directories and libraries. Invoke as::
+#
+#   find_package(Libevent
+#     [version] [EXACT]   # Minimum or exact version
+#     [REQUIRED]          # Fail if Libevent is not found
+#     [COMPONENT <C>...]) # Libraries to look for
+#
+# Valid components are one or more of:: libevent core extra pthreads openssl.
+# Note that 'libevent' contains both core and extra. You must specify one of
+# them for the other components.
+#
+# This module will define the following variables::
+#
+#  LIBEVENT_FOUND        - True if headers and requested libraries were found
+#  LIBEVENT_INCLUDE_DIRS - Libevent include directories
+#  LIBEVENT_LIBRARIES    - Libevent libraries to be linked
+#  LIBEVENT_<C>_FOUND    - Component <C> was found (<C> is uppercase)
+#  LIBEVENT_<C>_LIBRARY  - Library to be linked for Libevent component <C>.
+
+find_package(PkgConfig QUIET)
+pkg_check_modules(PC_LIBEVENT QUIET libevent)
+
+# Look for the Libevent 2.0 or 1.4 headers
+find_path(LIBEVENT_INCLUDE_DIR
+  NAMES
+    event2/event-config.h
+    event-config.h
+  PATHS /usr/include /usr/local/include
+  HINTS
+    ${PC_LIBEVENT_INCLUDE_DIRS}
+)
+
+if(LIBEVENT_INCLUDE_DIR)
+  set(_version_regex "^#define[ \t]+_EVENT_VERSION[ \t]+\"([^\"]+)\".*")
+  if(EXISTS "${LIBEVENT_INCLUDE_DIR}/event2/event-config.h")
+    # Libevent 2.0
+    file(STRINGS "${LIBEVENT_INCLUDE_DIR}/event2/event-config.h"
+      LIBEVENT_VERSION REGEX "${_version_regex}")
+  else()
+    # Libevent 1.4
+    file(STRINGS "${LIBEVENT_INCLUDE_DIR}/event-config.h"
+      LIBEVENT_VERSION REGEX "${_version_regex}")
+  endif()
+  string(REGEX REPLACE "${_version_regex}" "\\1"
+    LIBEVENT_VERSION "${LIBEVENT_VERSION}")
+  unset(_version_regex)
+endif()
+
+set(_LIBEVENT_REQUIRED_VARS)
+foreach(COMPONENT ${Libevent_FIND_COMPONENTS})
+  set(_LIBEVENT_LIBNAME libevent)
+  # Note: compare two variables to avoid a CMP0054 policy warning
+  if(COMPONENT STREQUAL _LIBEVENT_LIBNAME)
+    set(_LIBEVENT_LIBNAME event)
+  else()
+    set(_LIBEVENT_LIBNAME "event_${COMPONENT}")
+  endif()
+  string(TOUPPER "${COMPONENT}" COMPONENT_UPPER)
+  find_library(LIBEVENT_${COMPONENT_UPPER}_LIBRARY
+    NAMES ${_LIBEVENT_LIBNAME}
+    PATHS /usr/lib /usr/local/lib
+    HINTS ${PC_LIBEVENT_LIBRARY_DIRS}
+  )
+  if(LIBEVENT_${COMPONENT_UPPER}_LIBRARY)
+    set(Libevent_${COMPONENT}_FOUND 1)
+  endif()
+  list(APPEND _LIBEVENT_REQUIRED_VARS LIBEVENT_${COMPONENT_UPPER}_LIBRARY)
+endforeach()
+unset(_LIBEVENT_LIBNAME)
+
+include(FindPackageHandleStandardArgs)
+# handle the QUIETLY and REQUIRED arguments and set LIBEVENT_FOUND to TRUE
+# if all listed variables are TRUE and the requested version matches.
+find_package_handle_standard_args(Libevent REQUIRED_VARS
+                                  ${_LIBEVENT_REQUIRED_VARS}
+                                  LIBEVENT_INCLUDE_DIR
+                                  VERSION_VAR LIBEVENT_VERSION
+                                  HANDLE_COMPONENTS)
+
+if(LIBEVENT_FOUND)
+  set(LIBEVENT_INCLUDE_DIRS  ${LIBEVENT_INCLUDE_DIR})
+  set(LIBEVENT_LIBRARIES)
+  foreach(COMPONENT ${Libevent_FIND_COMPONENTS})
+    string(TOUPPER "${COMPONENT}" COMPONENT_UPPER)
+    list(APPEND LIBEVENT_LIBRARIES ${LIBEVENT_${COMPONENT_UPPER}_LIBRARY})
+    set(LIBEVENT_${COMPONENT_UPPER}_FOUND ${Libevent_${COMPONENT}_FOUND})
+  endforeach()
+endif()
+
+mark_as_advanced(LIBEVENT_INCLUDE_DIR ${_LIBEVENT_REQUIRED_VARS})
+unset(_LIBEVENT_REQUIRED_VARS)

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/deploy.sh
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/deploy.sh b/rocketmq-cpp/deploy.sh
new file mode 100755
index 0000000..c29cb0c
--- /dev/null
+++ b/rocketmq-cpp/deploy.sh
@@ -0,0 +1,26 @@
+#!/bin/sh
+
+VERSION="rocketmq-client4cpp"
+CWD_DIR=$(cd "$(dirname "$0")"; pwd)
+DEPLOY_BUILD_HOME=${CWD_DIR}/${VERSION}
+
+# ##====================================================================
+make
+# ##====================================================================
+# # deploy
+rm -rf   ${DEPLOY_BUILD_HOME}
+mkdir -p ${DEPLOY_BUILD_HOME}/lib
+mkdir -p ${DEPLOY_BUILD_HOME}/logs
+rm -rf ${CWD_DIR}/bin/*.log
+cp -rf ${CWD_DIR}/bin/*.a   ${DEPLOY_BUILD_HOME}/lib/
+cp -rf ${CWD_DIR}/bin/*.so  ${DEPLOY_BUILD_HOME}/lib/
+cp -rf ${CWD_DIR}/include ${DEPLOY_BUILD_HOME}/
+cp -rf ${CWD_DIR}/example ${DEPLOY_BUILD_HOME}/
+cp -rf ${CWD_DIR}/doc 	  ${DEPLOY_BUILD_HOME}/
+cp -rf ${CWD_DIR}/readme  ${DEPLOY_BUILD_HOME}/
+
+
+cd ${CWD_DIR} && tar -cvzf ./${VERSION}.tar.gz ./${VERSION}  >/dev/null 2>&1
+rm -rf ${DEPLOY_BUILD_HOME}
+# # ##====================================================================
+make clean

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/AsyncProducer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/AsyncProducer.cpp b/rocketmq-cpp/example/AsyncProducer.cpp
new file mode 100755
index 0000000..1ee9e41
--- /dev/null
+++ b/rocketmq-cpp/example/AsyncProducer.cpp
@@ -0,0 +1,125 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <chrono>
+#include <condition_variable>
+#include <iomanip>
+#include <iostream>
+#include <mutex>
+#include <thread>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+boost::atomic<bool> g_quit;
+std::mutex g_mtx;
+std::condition_variable g_finished;
+SendCallback* g_callback = NULL;
+TpsReportService g_tps;
+
+class MySendCallback : public SendCallback {
+  virtual void onSuccess(SendResult& sendResult) {
+    g_msgCount--;
+    g_tps.Increment();
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+  }
+  virtual void onException(MQException& e) { cout << "send Exception\n"; }
+};
+
+class MyAutoDeleteSendCallback : public AutoDeleteSendCallBack {
+ public:
+  virtual ~MyAutoDeleteSendCallback() {}
+  virtual void onSuccess(SendResult& sendResult) {
+    g_msgCount--;
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+  }
+  virtual void onException(MQException& e) {
+    std::cout << "send Exception" << e << "\n";
+  }
+};
+
+void AsyncProducerWorker(RocketmqSendAndConsumerArgs* info,
+                         DefaultMQProducer* producer) {
+  while (!g_quit.load()) {
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+    MQMessage msg(info->topic,  // topic
+                  "*",          // tag
+                  info->body);  // body
+
+    if (info->IsAutoDeleteSendCallback) {
+      g_callback = new MyAutoDeleteSendCallback();  // auto delete
+    }
+
+    try {
+      producer->send(msg, g_callback);
+    } catch (MQException& e) {
+      std::cout << e << endl;  // if catch excepiton , need re-send this msg by
+                               // service
+    }
+  }
+}
+
+int main(int argc, char* argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+
+  DefaultMQProducer producer("please_rename_unique_group_name");
+  if (!info.IsAutoDeleteSendCallback) {
+    g_callback = new MySendCallback();
+  }
+
+  PrintRocketmqSendAndConsumerArgs(info);
+
+  if (!info.namesrv.empty()) producer.setNamesrvAddr(info.namesrv);
+
+  producer.setGroupName(info.groupname);
+  producer.setInstanceName(info.groupname);
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.start();
+  g_tps.start();
+  std::vector<std::shared_ptr<std::thread>> work_pool;
+  auto start = std::chrono::system_clock::now();
+  int msgcount = g_msgCount.load();
+  for (int j = 0; j < info.thread_count; j++) {
+    std::shared_ptr<std::thread> th =
+        std::make_shared<std::thread>(AsyncProducerWorker, &info, &producer);
+    work_pool.push_back(th);
+  }
+
+  {
+    std::unique_lock<std::mutex> lck(g_mtx);
+    g_finished.wait(lck);
+    g_quit.store(true);
+  }
+
+  auto end = std::chrono::system_clock::now();
+  auto duration =
+      std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
+
+  std::cout
+      << "per msg time: " << duration.count() / (double)msgcount << "ms \n"
+      << "========================finished==============================\n";
+
+  producer.shutdown();
+  for (size_t th = 0; th != work_pool.size(); ++th) {
+    work_pool[th]->join();
+  }
+  if (!info.IsAutoDeleteSendCallback) {
+    delete g_callback;
+  }
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/AsyncPushConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/AsyncPushConsumer.cpp b/rocketmq-cpp/example/AsyncPushConsumer.cpp
new file mode 100755
index 0000000..36a17f7
--- /dev/null
+++ b/rocketmq-cpp/example/AsyncPushConsumer.cpp
@@ -0,0 +1,94 @@
+#include <stdlib.h>
+#include <string.h>
+
+#include <condition_variable>
+#include <iomanip>
+#include <iostream>
+#include <map>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include "common.h"
+
+std::mutex g_mtx;
+std::condition_variable g_finished;
+
+using namespace rocketmq;
+
+class MyMsgListener : public MessageListenerConcurrently {
+ public:
+  MyMsgListener() {}
+  virtual ~MyMsgListener() {}
+
+  virtual ConsumeStatus consumeMessage(const std::vector<MQMessageExt> &msgs) {
+    g_msgCount.store(g_msgCount.load() - msgs.size());
+    for (size_t i = 0; i < msgs.size(); ++i) {
+      //      std::cout << i << ": " << msgs[i].toString() << std::endl;
+    }
+
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+    return CONSUME_SUCCESS;
+  }
+};
+
+int main(int argc, char *argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQPushConsumer consumer("please_rename_unique_group_name");
+  DefaultMQProducer producer("please_rename_unique_group_name");
+
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setGroupName("msg-persist-group_producer_sandbox");
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.start();
+
+  consumer.setNamesrvAddr(info.namesrv);
+  consumer.setGroupName(info.groupname);
+  consumer.setNamesrvDomain(info.namesrv_domain);
+  consumer.setConsumeFromWhere(CONSUME_FROM_LAST_OFFSET);
+
+  consumer.setInstanceName(info.groupname);
+
+  consumer.subscribe(info.topic, "*");
+  consumer.setConsumeThreadCount(15);
+  consumer.setTcpTransportTryLockTimeout(1000);
+  consumer.setTcpTransportConnectTimeout(400);
+
+  MyMsgListener msglistener;
+  consumer.registerMessageListener(&msglistener);
+
+  try {
+    consumer.start();
+  } catch (MQClientException &e) {
+    cout << e << endl;
+  }
+
+  int msgcount = g_msgCount.load();
+  for (int i = 0; i < msgcount; ++i) {
+    MQMessage msg(info.topic,  // topic
+                  "*",         // tag
+                  info.body);  // body
+
+    try {
+      producer.send(msg);
+    } catch (MQException &e) {
+      std::cout << e << endl;  // if catch excepiton , need re-send this msg by
+                               // service
+    }
+  }
+
+  {
+    std::unique_lock<std::mutex> lck(g_mtx);
+    g_finished.wait(lck);
+  }
+  producer.shutdown();
+  consumer.shutdown();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/CMakeLists.txt b/rocketmq-cpp/example/CMakeLists.txt
new file mode 100755
index 0000000..8141f21
--- /dev/null
+++ b/rocketmq-cpp/example/CMakeLists.txt
@@ -0,0 +1,49 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+project(example)
+
+set(EXECUTABLE_OUTPUT_PATH ${CMAKE_SOURCE_DIR}/bin)
+
+include_directories(${CMAKE_SOURCE_DIR}/include)
+include_directories(${Boost_INCLUDE_DIRS})
+
+link_directories(${Boost_LIBRARY_DIRS})
+link_directories(${LIBEVENT_LIBRARY})
+link_directories(${JSONCPP_LIBRARY})
+
+set(SYS_LIBS ${SYS_LIBS} libboost_thread.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_system.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_iostreams.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_locale.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_date_time.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_chrono.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_serialization.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_log.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_log_setup.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_regex.a)
+set(SYS_LIBS ${SYS_LIBS} libboost_filesystem.a)
+set(SYS_LIBS ${SYS_LIBS} libevent.a)
+set(SYS_LIBS ${SYS_LIBS} libevent_core.a)
+set(SYS_LIBS ${SYS_LIBS} libevent_pthreads.a)
+set(SYS_LIBS ${SYS_LIBS} libevent_openssl.a)
+set(SYS_LIBS ${SYS_LIBS} libevent_extra.a)
+set(SYS_LIBS ${SYS_LIBS} libjsoncpp.a)
+file(GLOB files "*.cpp")
+foreach(file ${files})
+    get_filename_component(basename ${file} NAME_WE)
+    add_executable(${basename} ${file})
+    target_link_libraries (${basename}  rocketmq_static ${deplibs}  ${SYS_LIBS})
+endforeach()

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/OrderProducer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/OrderProducer.cpp b/rocketmq-cpp/example/OrderProducer.cpp
new file mode 100755
index 0000000..808d0ea
--- /dev/null
+++ b/rocketmq-cpp/example/OrderProducer.cpp
@@ -0,0 +1,99 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <condition_variable>
+#include <iomanip>
+#include <iostream>
+#include <mutex>
+#include <thread>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+std::condition_variable g_finished;
+std::mutex g_mtx;
+boost::atomic<bool> g_quit(false);
+
+class SelectMessageQueueByHash : public MessageQueueSelector {
+ public:
+  MQMessageQueue select(const std::vector<MQMessageQueue> &mqs,
+                        const MQMessage &msg, void *arg) {
+    int orderId = *static_cast<int *>(arg);
+    int index = orderId % mqs.size();
+    return mqs[index];
+  }
+};
+
+SelectMessageQueueByHash g_mySelector;
+
+void ProducerWorker(RocketmqSendAndConsumerArgs *info,
+                    DefaultMQProducer *producer) {
+  while (!g_quit.load()) {
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+    MQMessage msg(info->topic,  // topic
+                  "*",          // tag
+                  info->body);  // body
+
+    int orderId = 1;
+    SendResult sendResult =
+        producer->send(msg, &g_mySelector, static_cast<void *>(&orderId),
+                       info->retrytimes, info->SelectUnactiveBroker);
+    --g_msgCount;
+  }
+}
+
+int main(int argc, char *argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+
+  DefaultMQProducer producer("please_rename_unique_group_name");
+  PrintRocketmqSendAndConsumerArgs(info);
+
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.setGroupName(info.groupname);
+  producer.setInstanceName(info.groupname);
+
+  producer.start();
+
+  int msgcount = g_msgCount.load();
+  std::vector<std::shared_ptr<std::thread>> work_pool;
+
+  int threadCount = info.thread_count;
+  for (int j = 0; j < threadCount; j++) {
+    std::shared_ptr<std::thread> th =
+        std::make_shared<std::thread>(ProducerWorker, &info, &producer);
+    work_pool.push_back(th);
+  }
+
+  auto start = std::chrono::system_clock::now();
+  {
+    std::unique_lock<std::mutex> lck(g_mtx);
+    g_finished.wait(lck);
+    g_quit.store(true);
+  }
+
+  auto end = std::chrono::system_clock::now();
+  auto duration =
+      std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
+
+  std::cout
+      << "per msg time: " << duration.count() / (double)msgcount << "ms \n"
+      << "========================finished==============================\n";
+
+  for (size_t th = 0; th != work_pool.size(); ++th) {
+    work_pool[th]->join();
+  }
+
+  producer.shutdown();
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/OrderlyPushConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/OrderlyPushConsumer.cpp b/rocketmq-cpp/example/OrderlyPushConsumer.cpp
new file mode 100755
index 0000000..cc1aded
--- /dev/null
+++ b/rocketmq-cpp/example/OrderlyPushConsumer.cpp
@@ -0,0 +1,95 @@
+#include <stdlib.h>
+#include <string.h>
+
+#include <condition_variable>
+#include <iomanip>
+#include <iostream>
+#include <map>
+#include <mutex>
+#include <thread>
+#include <vector>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+std::condition_variable g_finished;
+std::mutex g_mtx;
+boost::atomic<int> g_consumedCount(0);
+boost::atomic<bool> g_quit(false);
+TpsReportService g_tps;
+
+class MyMsgListener : public MessageListenerOrderly {
+ public:
+  MyMsgListener() {}
+  virtual ~MyMsgListener() {}
+
+  virtual ConsumeStatus consumeMessage(const vector<MQMessageExt> &msgs) {
+    if (g_consumedCount.load() >= g_msgCount) {
+      std::unique_lock<std::mutex> lK(g_mtx);
+      g_quit.store(true);
+      g_finished.notify_one();
+    }
+    for (size_t i = 0; i < msgs.size(); i++) {
+      ++g_consumedCount;
+      g_tps.Increment();
+    }
+    return CONSUME_SUCCESS;
+  }
+};
+
+int main(int argc, char *argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQPushConsumer consumer("please_rename_unique_group_name");
+  DefaultMQProducer producer("please_rename_unique_group_name");
+
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setGroupName("msg-persist-group_producer_sandbox");
+  producer.start();
+
+  consumer.setNamesrvAddr(info.namesrv);
+  consumer.setNamesrvDomain(info.namesrv_domain);
+  consumer.setGroupName(info.groupname);
+  consumer.setConsumeFromWhere(CONSUME_FROM_LAST_OFFSET);
+  consumer.subscribe(info.topic, "*");
+  consumer.setConsumeThreadCount(info.thread_count);
+  consumer.setConsumeMessageBatchMaxSize(31);
+  if (info.syncpush) consumer.setAsyncPull(false);
+
+  MyMsgListener msglistener;
+  consumer.registerMessageListener(&msglistener);
+  g_tps.start();
+
+  try {
+    consumer.start();
+  } catch (MQClientException &e) {
+    std::cout << e << std::endl;
+  }
+
+  int msgcount = g_msgCount.load();
+  for (int i = 0; i < msgcount; ++i) {
+    MQMessage msg(info.topic,  // topic
+                  "*",         // tag
+                  info.body);  // body
+
+    try {
+      producer.send(msg);
+    } catch (MQException &e) {
+      std::cout << e << endl;  // if catch excepiton , need re-send this msg by
+                               // service
+    }
+  }
+
+  while (!g_quit.load()) {
+    std::unique_lock<std::mutex> lk(g_mtx);
+    g_finished.wait(lk);
+  }
+
+  producer.shutdown();
+  consumer.shutdown();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/PullConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/PullConsumer.cpp b/rocketmq-cpp/example/PullConsumer.cpp
new file mode 100755
index 0000000..1709b26
--- /dev/null
+++ b/rocketmq-cpp/example/PullConsumer.cpp
@@ -0,0 +1,108 @@
+#include <stdlib.h>
+#include <string.h>
+
+#include <chrono>
+#include <iomanip>
+#include <iostream>
+#include <map>
+#include <vector>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+std::map<MQMessageQueue, uint64_t> g_offseTable;
+
+void putMessageQueueOffset(MQMessageQueue mq, uint64_t offset) {
+  g_offseTable[mq] = offset;
+}
+
+uint64_t getMessageQueueOffset(MQMessageQueue mq) {
+  map<MQMessageQueue, uint64_t>::iterator it = g_offseTable.find(mq);
+  if (it != g_offseTable.end()) {
+    return it->second;
+  }
+  return 0;
+}
+
+int main(int argc, char *argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQPullConsumer consumer("please_rename_unique_group_name");
+  consumer.setNamesrvAddr(info.namesrv);
+  consumer.setNamesrvDomain(info.namesrv_domain);
+  consumer.setGroupName(info.groupname);
+  consumer.setInstanceName(info.groupname);
+  consumer.registerMessageQueueListener(info.topic, NULL);
+  consumer.start();
+  std::vector<MQMessageQueue> mqs;
+
+  try {
+    consumer.fetchSubscribeMessageQueues(info.topic, mqs);
+    auto iter = mqs.begin();
+    for (; iter != mqs.end(); ++iter) {
+      std::cout << "mq:" << (*iter).toString() << endl;
+    }
+  } catch (MQException &e) {
+    std::cout << e << endl;
+  }
+
+  auto start = std::chrono::system_clock::now();
+  auto iter = mqs.begin();
+  for (; iter != mqs.end(); ++iter) {
+    MQMessageQueue mq = (*iter);
+    // if cluster model
+    // putMessageQueueOffset(mq, g_consumer.fetchConsumeOffset(mq,true));
+    // if broadcast model
+    // putMessageQueueOffset(mq, your last consume offset);
+
+    bool noNewMsg = false;
+    do {
+      try {
+        PullResult result =
+            consumer.pull(mq, "*", getMessageQueueOffset(mq), 32);
+        g_msgCount += result.msgFoundList.size();
+        std::cout << result.msgFoundList.size() << std::endl;
+        // if pull request timeout or received NULL response, pullStatus will be
+        // setted to BROKER_TIMEOUT,
+        // And nextBeginOffset/minOffset/MaxOffset will be setted to 0
+        if (result.pullStatus != BROKER_TIMEOUT) {
+          putMessageQueueOffset(mq, result.nextBeginOffset);
+          PrintPullResult(&result);
+        } else {
+          cout << "broker timeout occur" << endl;
+        }
+        switch (result.pullStatus) {
+          case FOUND:
+          case NO_MATCHED_MSG:
+          case OFFSET_ILLEGAL:
+          case BROKER_TIMEOUT:
+            break;
+          case NO_NEW_MSG:
+            noNewMsg = true;
+            break;
+          default:
+            break;
+        }
+      } catch (MQClientException &e) {
+        std::cout << e << std::endl;
+      }
+    } while (!noNewMsg);
+  }
+
+  auto end = std::chrono::system_clock::now();
+  auto duration =
+      std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
+
+  std::cout << "msg count: " << g_msgCount.load() << "\n";
+  std::cout
+      << "per msg time: " << duration.count() / (double)g_msgCount.load()
+      << "ms \n"
+      << "========================finished==============================\n";
+
+  consumer.shutdown();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/PushConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/PushConsumer.cpp b/rocketmq-cpp/example/PushConsumer.cpp
new file mode 100755
index 0000000..7c85b64
--- /dev/null
+++ b/rocketmq-cpp/example/PushConsumer.cpp
@@ -0,0 +1,110 @@
+#include <stdlib.h>
+#include <string.h>
+
+#include <chrono>
+#include <condition_variable>
+#include <iomanip>
+#include <iostream>
+#include <map>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include "common.h"
+
+std::mutex g_mtx;
+std::condition_variable g_finished;
+TpsReportService g_tps;
+
+using namespace rocketmq;
+
+class MyMsgListener : public MessageListenerConcurrently {
+ public:
+  MyMsgListener() {}
+  virtual ~MyMsgListener() {}
+
+  virtual ConsumeStatus consumeMessage(const std::vector<MQMessageExt> &msgs) {
+    g_msgCount.store(g_msgCount.load() - msgs.size());
+    for (size_t i = 0; i < msgs.size(); ++i) {
+      g_tps.Increment();
+      // cout << "msg body: "<<  msgs[i].getBody() << endl;
+    }
+
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+    return CONSUME_SUCCESS;
+  }
+};
+
+int main(int argc, char *argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQPushConsumer consumer("please_rename_unique_group_name");
+  DefaultMQProducer producer("please_rename_unique_group_name");
+  producer.setSessionCredentials("mEbjOEonoo5TREFS",
+                                 "xZRP6rejrDjxLxGFHbDfppfJt1S0VJ", "ALIYUN");
+  producer.setTcpTransportTryLockTimeout(1000);
+  producer.setTcpTransportConnectTimeout(400);
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setGroupName("msg-persist-group_producer_sandbox");
+  producer.start();
+
+  consumer.setNamesrvAddr(info.namesrv);
+  consumer.setGroupName(info.groupname);
+  consumer.setSessionCredentials("mEbjOEonoo5TREFS",
+                                 "xZRP6rejrDjxLxGFHbDfppfJt1S0VJ", "ALIYUN");
+  consumer.setConsumeThreadCount(info.thread_count);
+  consumer.setNamesrvDomain(info.namesrv_domain);
+  consumer.setConsumeFromWhere(CONSUME_FROM_LAST_OFFSET);
+
+  if (info.syncpush) consumer.setAsyncPull(false);  // set sync pull
+  if (info.broadcasting) {
+    consumer.setMessageModel(BROADCASTING);
+  }
+
+  consumer.setInstanceName(info.groupname);
+
+  consumer.subscribe(info.topic, "*");
+  consumer.setConsumeThreadCount(15);
+  consumer.setTcpTransportTryLockTimeout(1000);
+  consumer.setTcpTransportConnectTimeout(400);
+
+  MyMsgListener msglistener;
+  consumer.registerMessageListener(&msglistener);
+
+  try {
+    consumer.start();
+  } catch (MQClientException &e) {
+    cout << e << endl;
+  }
+  g_tps.start();
+
+  int msgcount = g_msgCount.load();
+  for (int i = 0; i < msgcount; ++i) {
+    MQMessage msg(info.topic,  // topic
+                  "*",         // tag
+                  info.body);  // body
+
+    //    std::this_thread::sleep_for(std::chrono::seconds(100000));
+    try {
+      producer.send(msg);
+    } catch (MQException &e) {
+      std::cout << e << endl;  // if catch excepiton , need re-send this msg by
+                               // service
+    }
+  }
+
+  {
+    std::unique_lock<std::mutex> lck(g_mtx);
+    g_finished.wait(lck);
+  }
+  producer.shutdown();
+  consumer.shutdown();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/README.md
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/README.md b/rocketmq-cpp/example/README.md
new file mode 100644
index 0000000..91965d0
--- /dev/null
+++ b/rocketmq-cpp/example/README.md
@@ -0,0 +1,4 @@
+1. AsyncProducer
+2. OrderlyProducer
+3. SyncProducer
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/SendDelayMsg.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/SendDelayMsg.cpp b/rocketmq-cpp/example/SendDelayMsg.cpp
new file mode 100755
index 0000000..a078c80
--- /dev/null
+++ b/rocketmq-cpp/example/SendDelayMsg.cpp
@@ -0,0 +1,51 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <condition_variable>
+#include <iomanip>
+#include <iomanip>
+#include <iostream>
+#include <iostream>
+#include <mutex>
+#include <thread>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+int main(int argc, char* argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQProducer producer("please_rename_unique_group_name");
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.setGroupName(info.groupname);
+  producer.setInstanceName(info.groupname);
+
+  producer.setSendMsgTimeout(500);
+  producer.setTcpTransportTryLockTimeout(1000);
+  producer.setTcpTransportConnectTimeout(400);
+
+  producer.start();
+
+  MQMessage msg(info.topic,  // topic
+                "*",         // tag
+                info.body);  // body
+
+  // messageDelayLevel=1s 5s 10s 30s 1m 2m 3m 4m 5m 6m 7m 8m 9m 10m 20m 30m 1h
+  // 2h
+  msg.setDelayTimeLevel(5);  // 1m
+  try {
+    SendResult sendResult = producer.send(msg, info.SelectUnactiveBroker);
+  } catch (const MQException& e) {
+    std::cout << "send failed: " << std::endl;
+  }
+
+  producer.shutdown();
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/SyncProducer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/SyncProducer.cpp b/rocketmq-cpp/example/SyncProducer.cpp
new file mode 100755
index 0000000..f4f6e41
--- /dev/null
+++ b/rocketmq-cpp/example/SyncProducer.cpp
@@ -0,0 +1,101 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+#include <condition_variable>
+#include <iomanip>
+#include <iomanip>
+#include <iostream>
+#include <iostream>
+#include <mutex>
+#include <thread>
+
+#include "common.h"
+
+using namespace rocketmq;
+
+boost::atomic<bool> g_quit;
+std::mutex g_mtx;
+std::condition_variable g_finished;
+TpsReportService g_tps;
+
+void SyncProducerWorker(RocketmqSendAndConsumerArgs* info,
+                        DefaultMQProducer* producer) {
+  while (!g_quit.load()) {
+    if (g_msgCount.load() <= 0) {
+      std::unique_lock<std::mutex> lck(g_mtx);
+      g_finished.notify_one();
+    }
+    MQMessage msg(info->topic,  // topic
+                  "*",          // tag
+                  info->body);  // body
+    try {
+      auto start = std::chrono::system_clock::now();
+      SendResult sendResult = producer->send(msg, info->SelectUnactiveBroker);
+      g_tps.Increment();
+      --g_msgCount;
+      auto end = std::chrono::system_clock::now();
+      auto duration =
+          std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
+      if (duration.count() >= 500) {
+        std::cout << "send RT more than: " << duration.count()
+                  << " ms with msgid: " << sendResult.getMsgId() << endl;
+      }
+    } catch (const MQException& e) {
+      std::cout << "send failed: " << std::endl;
+    }
+  }
+}
+
+int main(int argc, char* argv[]) {
+  RocketmqSendAndConsumerArgs info;
+  if (!ParseArgs(argc, argv, &info)) {
+    exit(-1);
+  }
+  PrintRocketmqSendAndConsumerArgs(info);
+  DefaultMQProducer producer("please_rename_unique_group_name");
+  producer.setNamesrvAddr(info.namesrv);
+  producer.setNamesrvDomain(info.namesrv_domain);
+  producer.setGroupName(info.groupname);
+  producer.setInstanceName(info.groupname);
+  producer.setSessionCredentials("mq acesskey", "mq secretkey", "ALIYUN");
+  producer.setSendMsgTimeout(500);
+  producer.setTcpTransportTryLockTimeout(1000);
+  producer.setTcpTransportConnectTimeout(400);
+
+  producer.start();
+  std::vector<std::shared_ptr<std::thread>> work_pool;
+  auto start = std::chrono::system_clock::now();
+  int msgcount = g_msgCount.load();
+  g_tps.start();
+
+  int threadCount = info.thread_count;
+  for (int j = 0; j < threadCount; j++) {
+    std::shared_ptr<std::thread> th =
+        std::make_shared<std::thread>(SyncProducerWorker, &info, &producer);
+    work_pool.push_back(th);
+  }
+
+  {
+    std::unique_lock<std::mutex> lck(g_mtx);
+    g_finished.wait(lck);
+    g_quit.store(true);
+  }
+
+  auto end = std::chrono::system_clock::now();
+  auto duration =
+      std::chrono::duration_cast<std::chrono::milliseconds>(end - start);
+
+  std::cout
+      << "per msg time: " << duration.count() / (double)msgcount << "ms \n"
+      << "========================finished==============================\n";
+
+  for (size_t th = 0; th != work_pool.size(); ++th) {
+    work_pool[th]->join();
+  }
+
+  producer.shutdown();
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/example/common.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/example/common.h b/rocketmq-cpp/example/common.h
new file mode 100755
index 0000000..d96af6d
--- /dev/null
+++ b/rocketmq-cpp/example/common.h
@@ -0,0 +1,183 @@
+#ifndef ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_
+#define ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_
+
+#include <boost/atomic.hpp>
+#include <boost/chrono.hpp>
+#include <boost/shared_ptr.hpp>
+#include <boost/thread.hpp>
+#include <iostream>
+#include <string>
+#include <thread>
+#include <vector>
+
+#include "DefaultMQProducer.h"
+#include "DefaultMQPullConsumer.h"
+#include "DefaultMQPushConsumer.h"
+using namespace std;
+
+boost::atomic<int> g_msgCount(1);
+
+class RocketmqSendAndConsumerArgs {
+ public:
+  RocketmqSendAndConsumerArgs()
+      : body("msgbody for test"),
+        thread_count(boost::thread::hardware_concurrency()),
+        broadcasting(false),
+        syncpush(false),
+        SelectUnactiveBroker(false),
+        IsAutoDeleteSendCallback(false),
+        retrytimes(5),
+        PrintMoreInfo(false) {}
+
+ public:
+  std::string namesrv;
+  std::string namesrv_domain;
+  std::string groupname;
+  std::string topic;
+  std::string body;
+  int thread_count;
+  bool broadcasting;
+  bool syncpush;
+  bool SelectUnactiveBroker;  // default select active broker
+  bool IsAutoDeleteSendCallback;
+  int retrytimes;  // default retry 5 times;
+  bool PrintMoreInfo;
+};
+
+class TpsReportService {
+ public:
+  TpsReportService() : tps_interval_(1), quit_flag_(false), tps_count_(0) {}
+  void start() {
+    tps_thread_.reset(
+        new boost::thread(boost::bind(&TpsReportService::TpsReport, this)));
+  }
+
+  ~TpsReportService() {
+    quit_flag_.store(true);
+    if (tps_thread_->joinable()) tps_thread_->join();
+  }
+
+  void Increment() { ++tps_count_; }
+
+  void TpsReport() {
+    while (!quit_flag_.load()) {
+      boost::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<boost::thread> tps_thread_;
+  boost::atomic<bool> quit_flag_;
+  boost::atomic<long> 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) {
+    std::cout << result->toString() << endl;
+    std::vector<rocketmq::MQMessageExt>::iterator it =
+        result->msgFoundList.begin();
+    for (; it != result->msgFoundList.end(); ++it) {
+      cout << "=======================================================" << endl
+           << (*it).toString() << endl;
+    }
+  }
+}
+
+static void PrintRocketmqSendAndConsumerArgs(
+    const RocketmqSendAndConsumerArgs& info) {
+  std::cout << "nameserver: " << info.namesrv << endl
+            << "topic: " << info.topic << endl
+            << "groupname: " << info.groupname << endl
+            << "produce content: " << info.body << endl
+            << "msg  count: " << g_msgCount.load() << endl;
+}
+
+static void help() {
+  std::cout
+      << "need option,like follow: \n"
+      << "-n nameserver addr, if not set -n and -i ,no nameSrv will be got \n"
+         "-i nameserver domain name,  if not set -n and -i ,no nameSrv will be "
+         "got \n"
+         "-g groupname \n"
+         "-t  msg topic \n"
+         "-m messagecout(default value: 1) \n"
+         "-c content(default value: only test ) \n"
+         "-b (BROADCASTING model, default value: CLUSTER) \n"
+         "-s sync push(default is async push)\n"
+         "-r setup retry times(default value: 5 times)\n"
+         "-u select active broker to send msg(default value: false)\n"
+         "-d use AutoDeleteSendcallback by cpp client(defalut value: false) \n"
+         "-T thread count of send msg or consume msg(defalut value: system cpu "
+         "core number) \n"
+         "-v print more details information \n";
+}
+
+static bool ParseArgs(int argc, char* argv[],
+                      RocketmqSendAndConsumerArgs* info) {
+  int ch;
+  while ((ch = getopt(argc, argv, "n:i:g:t:m:c:b:s:h:r:T:bu")) != -1) {
+    switch (ch) {
+      case 'n':
+        info->namesrv.insert(0, optarg);
+        break;
+      case 'i':
+        info->namesrv_domain.insert(0, optarg);
+        break;
+      case 'g':
+        info->groupname.insert(0, optarg);
+        break;
+      case 't':
+        info->topic.insert(0, optarg);
+        break;
+      case 'm':
+        g_msgCount.store(atoi(optarg));
+        break;
+      case 'c':
+        info->body.insert(0, optarg);
+        break;
+      case 'b':
+        info->broadcasting = true;
+        break;
+      case 's':
+        info->syncpush = true;
+        break;
+      case 'r':
+        info->retrytimes = atoi(optarg);
+        break;
+      case 'u':
+        info->SelectUnactiveBroker = true;
+        break;
+      case 'T':
+        info->thread_count = atoi(optarg);
+        break;
+      case 'v':
+        info->PrintMoreInfo = true;
+        break;
+      case 'h':
+        help();
+        return false;
+      default:
+        help();
+        return false;
+    }
+  }
+  if (info->groupname.empty() || info->topic.empty() ||
+      (info->namesrv_domain.empty() && info->namesrv.empty())) {
+    std::cout << "please use -g to setup groupname and -t setup topic \n";
+    help();
+    return false;
+  }
+  return true;
+}
+#endif  // ROCKETMQ_CLIENT4CPP_EXAMPLE_COMMON_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/format.sh
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/format.sh b/rocketmq-cpp/format.sh
new file mode 100755
index 0000000..20dbe71
--- /dev/null
+++ b/rocketmq-cpp/format.sh
@@ -0,0 +1,62 @@
+#!/bin/bash
+
+
+TMPFILE=".clang_format_file.tmp"
+FORMAT="{BasedOnStyle: Google,IndentWidth: 2,ColumnLimit: 80}"
+
+function Usage
+{
+    echo "Usage: $0 want-format-file|want-format-dir ..."
+    #echo "Currently only format a file or dir at a time"
+}
+
+#Setp1 check clang-format support
+if ! which clang-format &>/dev/null; then
+    echo -e "\033[32m !!!!!!please install clang-format  \033[0m"
+    exit 1
+fi
+
+
+#Setp2 check weather incoming format file
+if [ ! $# -ge 1 ];then
+    Usage
+    exit 1
+fi
+
+for dest in "$@"
+do
+  if [ ! -e $dest ]; then
+    echo -e "\033[32m $dest not exists,please check this file weather exists \033[0m"
+  fi
+done
+
+
+#Setp3 get filelist
+for dest in $*
+do
+  if [ -f $dest ];then
+      files="$files $dest"
+  elif [ -d $dest ];then
+      files="$files `ls $dest/*.cpp $dest/*.h $dest/*.cc 2>/dev/null`"
+  else
+      echo -e "\033[32m $dest sorry current $0 only support regular file or dir \033[0m"
+  fi
+done
+
+#Setp4 use clang-format format dest file
+for file in $files
+do
+    echo $file
+    clang-format -style="$FORMAT" $file > $TMPFILE
+
+    if [ -e $TMPFILE ];then
+        filesize=`wc -c $TMPFILE |cut -d " " -f1`
+        if [ $filesize -eq 0 ];then
+            echo -e "\033[32m formt file error,May be because of the size of the source file is 0, or format program error \033[0m"
+            exit 1
+        fi
+    fi
+
+    #Setp4 replace source file
+    mv -f $TMPFILE $file
+done

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/Arg_helper.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/Arg_helper.h b/rocketmq-cpp/include/Arg_helper.h
new file mode 100755
index 0000000..f243fc1
--- /dev/null
+++ b/rocketmq-cpp/include/Arg_helper.h
@@ -0,0 +1,42 @@
+/*
+ * 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 _ARG_HELPER_H_
+#define _ARG_HELPER_H_
+
+#include <string>
+#include <vector>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API Arg_helper {
+ public:
+  Arg_helper(int argc, char* argv[]);
+  Arg_helper(std::string arg_str_);
+  std::string get_option(int idx_) const;
+  bool is_enable_option(std::string opt_) const;
+  std::string get_option_value(std::string opt_) const;
+
+ private:
+  std::vector<std::string> m_args;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif  //<!_ARG_HELPER_H_;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/AsyncCallback.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/AsyncCallback.h b/rocketmq-cpp/include/AsyncCallback.h
new file mode 100755
index 0000000..698575c
--- /dev/null
+++ b/rocketmq-cpp/include/AsyncCallback.h
@@ -0,0 +1,66 @@
+/*
+ * 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 __SENDCALLBACK_H__
+#define __SENDCALLBACK_H__
+
+#include "MQClientException.h"
+#include "PullResult.h"
+#include "RocketMQClient.h"
+#include "SendResult.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+struct AsyncCallback {};
+//<!***************************************************************************
+typedef enum sendCallbackType {
+  noAutoDeleteSendCallback = 0,
+  autoDeleteSendCallback = 1
+} sendCallbackType;
+
+class ROCKETMQCLIENT_API SendCallback : public AsyncCallback {
+ public:
+  virtual ~SendCallback() {}
+  virtual void onSuccess(SendResult& sendResult) = 0;
+  virtual void onException(MQException& e) = 0;
+  virtual sendCallbackType getSendCallbackType() {
+    return noAutoDeleteSendCallback;
+  }
+};
+
+//async SendCallback will be deleted automatically by metaq cpp after invoke callback interface
+class ROCKETMQCLIENT_API AutoDeleteSendCallBack : public SendCallback {
+ public:
+  virtual ~AutoDeleteSendCallBack() {}
+  virtual void onSuccess(SendResult& sendResult) = 0;
+  virtual void onException(MQException& e) = 0;
+  virtual sendCallbackType getSendCallbackType() {
+    return autoDeleteSendCallback;
+  }
+};
+
+//<!************************************************************************
+class ROCKETMQCLIENT_API PullCallback : public AsyncCallback {
+ public:
+  virtual ~PullCallback() {}
+  virtual void onSuccess(MQMessageQueue& mq, PullResult& result,
+                         bool bProducePullRequest) = 0;
+  virtual void onException(MQException& e) = 0;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/ConsumeType.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/ConsumeType.h b/rocketmq-cpp/include/ConsumeType.h
new file mode 100755
index 0000000..f804655
--- /dev/null
+++ b/rocketmq-cpp/include/ConsumeType.h
@@ -0,0 +1,61 @@
+/*
+ * 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 __CONSUMETYPE_H__
+#define __CONSUMETYPE_H__
+
+namespace rocketmq {
+//<!***************************************************************************
+enum ConsumeType {
+  CONSUME_ACTIVELY,
+  CONSUME_PASSIVELY,
+};
+
+//<!***************************************************************************
+enum ConsumeFromWhere {
+  /**
+  *new consumer will consume from end offset of queue, 
+  * and then consume from last consumed offset of queue follow-up 
+  */
+  CONSUME_FROM_LAST_OFFSET,
+
+  // @Deprecated
+  CONSUME_FROM_LAST_OFFSET_AND_FROM_MIN_WHEN_BOOT_FIRST,
+  // @Deprecated
+  CONSUME_FROM_MIN_OFFSET,
+  // @Deprecated
+  CONSUME_FROM_MAX_OFFSET,
+  /**
+  *new consumer will consume from first offset of queue, 
+  * and then consume from last consumed offset of queue follow-up 
+  */
+  CONSUME_FROM_FIRST_OFFSET,
+  /**
+  *new consumer will consume from the queue offset specified by timestamp, 
+  * and then consume from last consumed offset of queue follow-up 
+  */
+  CONSUME_FROM_TIMESTAMP,
+};
+
+//<!***************************************************************************
+enum MessageModel {
+  BROADCASTING,
+  CLUSTERING,
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/DefaultMQProducer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/DefaultMQProducer.h b/rocketmq-cpp/include/DefaultMQProducer.h
new file mode 100755
index 0000000..fafd1a4
--- /dev/null
+++ b/rocketmq-cpp/include/DefaultMQProducer.h
@@ -0,0 +1,106 @@
+/*
+ * 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 __DEFAULTMQPRODUCER_H__
+#define __DEFAULTMQPRODUCER_H__
+
+#include "MQMessageQueue.h"
+#include "MQProducer.h"
+#include "RocketMQClient.h"
+#include "SendResult.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API DefaultMQProducer : public MQProducer {
+ public:
+  DefaultMQProducer(const std::string& groupname);
+  virtual ~DefaultMQProducer();
+
+  //<!begin mqadmin;
+  virtual void start();
+  virtual void shutdown();
+  //<!end mqadmin;
+
+  //<! begin MQProducer;
+  virtual SendResult send(MQMessage& msg, bool bSelectActiveBroker = false);
+  virtual SendResult send(MQMessage& msg, const MQMessageQueue& mq);
+  virtual SendResult send(MQMessage& msg, MessageQueueSelector* selector,
+                          void* arg);
+  virtual SendResult send(MQMessage& msg, MessageQueueSelector* selector,
+                          void* arg, int autoRetryTimes,
+                          bool bActiveBroker = false);
+  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);
+  //<! end MQProducer;
+
+  //set and get timeout of per msg
+  int getSendMsgTimeout() const;
+  void setSendMsgTimeout(int sendMsgTimeout);
+
+  /*
+  *  if msgBody size is large than m_compressMsgBodyOverHowmuch
+      metaq 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);
+
+  //set msg max retry times, default retry times is 5
+  int getRetryTimes() const;
+  void setRetryTimes(int times);
+
+ protected:
+  SendResult sendAutoRetrySelectImpl(MQMessage& msg,
+                                     MessageQueueSelector* pSelector,
+                                     void* pArg, int communicationMode,
+                                     SendCallback* pSendCallback,
+                                     int retryTimes,
+                                     bool bActiveBroker = false);
+  SendResult sendSelectImpl(MQMessage& msg, MessageQueueSelector* pSelector,
+                            void* pArg, int communicationMode,
+                            SendCallback* sendCallback);
+  SendResult sendDefaultImpl(MQMessage& msg, int communicationMode,
+                             SendCallback* pSendCallback,
+                             bool bActiveBroker = false);
+  SendResult sendKernelImpl(MQMessage& msg, const MQMessageQueue& mq,
+                            int communicationMode, SendCallback* pSendCallback);
+  bool tryToCompressMessage(MQMessage& msg);
+
+ private:
+  int m_sendMsgTimeout;
+  int m_compressMsgBodyOverHowmuch;
+  int m_maxMessageSize;  //<! default:128K;
+  bool m_retryAnotherBrokerWhenNotStoreOK;
+  int m_compressLevel;
+  int m_retryTimes;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/DefaultMQPullConsumer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/DefaultMQPullConsumer.h b/rocketmq-cpp/include/DefaultMQPullConsumer.h
new file mode 100755
index 0000000..30599a2
--- /dev/null
+++ b/rocketmq-cpp/include/DefaultMQPullConsumer.h
@@ -0,0 +1,155 @@
+/*
+ * 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 __DEFAULTMQPULLCONSUMER_H__
+#define __DEFAULTMQPULLCONSUMER_H__
+
+#include <set>
+#include <string>
+#include "MQConsumer.h"
+#include "MQMessageQueue.h"
+#include "MQueueListener.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+class Rebalance;
+class SubscriptionData;
+class OffsetStore;
+class PullAPIWrapper;
+class ConsumerRunningInfo;
+//<!***************************************************************************
+class ROCKETMQCLIENT_API DefaultMQPullConsumer : public MQConsumer {
+ public:
+  DefaultMQPullConsumer(const std::string& groupname);
+  virtual ~DefaultMQPullConsumer();
+
+  //<!begin mqadmin;
+  virtual void start();
+  virtual void shutdown();
+  //<!end mqadmin;
+
+  //<!begin MQConsumer
+  virtual void sendMessageBack(MQMessageExt& msg, int delayLevel);
+  virtual void fetchSubscribeMessageQueues(const std::string& topic,
+                                           std::vector<MQMessageQueue>& mqs);
+  virtual void doRebalance();
+  virtual void persistConsumerOffset();
+  virtual void persistConsumerOffsetByResetOffset();
+  virtual void updateTopicSubscribeInfo(const std::string& topic,
+                                        std::vector<MQMessageQueue>& info);
+  virtual ConsumeType getConsumeType();
+  virtual ConsumeFromWhere getConsumeFromWhere();
+  virtual void getSubscriptions(std::vector<SubscriptionData>&);
+  virtual void updateConsumeOffset(const MQMessageQueue& mq, int64 offset);
+  virtual void removeConsumeOffset(const MQMessageQueue& mq);
+  virtual void producePullMsgTask(PullRequest*);
+  virtual Rebalance* getRebalance() const;
+  //<!end MQConsumer;
+
+  void registerMessageQueueListener(const std::string& topic,
+                                    MQueueListener* pListener);
+  /**
+  * pull msg from specified queue, if no msg in queue, return directly
+  *
+  * @param mq
+  *            specify the pulled queue
+  * @param subExpression
+  *            set filter expression for pulled msg, broker will filter msg actively
+  *            Now only OR operation is supported, eg: "tag1 || tag2 || tag3"
+  *            if subExpression is setted to "null" or "*"��all msg will be subscribed
+  * @param offset
+  *            specify the started pull offset
+  * @param maxNums
+  *            specify max msg num by per pull
+  * @return  
+  *            accroding to PullResult
+  */
+  virtual PullResult pull(const MQMessageQueue& mq, const std::string& subExpression,
+                          int64 offset, int maxNums);
+  virtual void pull(const MQMessageQueue& mq, const std::string& subExpression,
+                    int64 offset, int maxNums, PullCallback* pPullCallback);
+
+  /**
+  * pull msg from specified queue, if no msg, broker will suspend the pull request 20s
+  *
+  * @param mq
+  *            specify the pulled queue
+  * @param subExpression
+  *            set filter expression for pulled msg, broker will filter msg actively
+  *            Now only OR operation is supported, eg: "tag1 || tag2 || tag3"
+  *            if subExpression is setted to "null" or "*"��all msg will be subscribed
+  * @param offset
+  *            specify the started pull offset
+  * @param maxNums
+  *            specify max msg num by per pull
+  * @return  
+  *            accroding to PullResult
+  */
+  PullResult pullBlockIfNotFound(const MQMessageQueue& mq,
+                                 const std::string& subExpression, int64 offset,
+                                 int maxNums);
+  void pullBlockIfNotFound(const MQMessageQueue& mq,
+                           const std::string& subExpression, int64 offset,
+                           int maxNums, PullCallback* pPullCallback);
+
+  virtual ConsumerRunningInfo* getConsumerRunningInfo() { return NULL; }
+  /**
+  * ��ȡ���ѽ��ȣ�����-1��ʾ����
+  *
+  * @param mq
+  * @param fromStore
+  * @return
+  */
+  int64 fetchConsumeOffset(const MQMessageQueue& mq, bool fromStore);
+  /**
+  * ����topic��ȡMessageQueue���Ծ��ⷽʽ�����ڶ����Ա֮�����
+  *
+  * @param topic
+  *            ��ϢTopic
+  * @return ���ض��м���
+  */
+  void fetchMessageQueuesInBalance(const std::string& topic,
+                                   std::vector<MQMessageQueue> 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();
+
+  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<std::string> m_registerTopics;
+
+  MQueueListener* m_pMessageQueueListener;
+  OffsetStore* m_pOffsetStore;
+  Rebalance* m_pRebalance;
+  PullAPIWrapper* m_pPullAPIWrapper;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif



[11/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/sha512.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/sha512.c b/rocketmq-cpp/libs/signature/src/sha512.c
new file mode 100755
index 0000000..c64aec6
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/sha512.c
@@ -0,0 +1,616 @@
+/* sha512.c - Functions to compute SHA512 and SHA384 message digest of files or
+   memory blocks according to the NIST specification FIPS-180-2.
+
+   Copyright (C) 2005, 2006, 2008 Free Software Foundation, Inc.
+
+   This program is free software: you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation, either version 3 of the License, or
+   (at your option) any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+/* Written by David Madore, considerably copypasting from
+   Scott G. Miller's sha1.c
+*/
+
+/* #include <config.h> */
+
+#include "sha512.h"
+
+#include <stddef.h>
+#include <string.h>
+
+#if USE_UNLOCKED_IO
+# include "unlocked-io.h"
+#endif
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+#ifdef WORDS_BIGENDIAN
+# define SWAP(n) (n)
+#else
+# define SWAP(n) \
+    u64or (u64or (u64or (u64shl (n, 56),				\
+			 u64shl (u64and (n, u64lo (0x0000ff00)), 40)),	\
+		  u64or (u64shl (u64and (n, u64lo (0x00ff0000)), 24),	\
+			 u64shl (u64and (n, u64lo (0xff000000)),  8))),	\
+	   u64or (u64or (u64and (u64shr (n,  8), u64lo (0xff000000)),	\
+			 u64and (u64shr (n, 24), u64lo (0x00ff0000))),	\
+		  u64or (u64and (u64shr (n, 40), u64lo (0x0000ff00)),	\
+			 u64shr (n, 56))))
+#endif
+
+#define BLOCKSIZE 4096
+#if BLOCKSIZE % 128 != 0
+# error "invalid BLOCKSIZE"
+#endif
+
+/* This array contains the bytes used to pad the buffer to the next
+   128-byte boundary.  */
+static const unsigned char fillbuf[128] = { 0x80, 0 /* , 0, 0, ...  */ };
+
+
+/*
+  Takes a pointer to a 512 bit block of data (eight 64 bit ints) and
+  intializes it to the start constants of the SHA512 algorithm.  This
+  must be called before using hash in the call to sha512_hash
+*/
+void
+sha512_init_ctx (struct sha512_ctx *ctx)
+{
+  ctx->state[0] = u64hilo (0x6a09e667, 0xf3bcc908);
+  ctx->state[1] = u64hilo (0xbb67ae85, 0x84caa73b);
+  ctx->state[2] = u64hilo (0x3c6ef372, 0xfe94f82b);
+  ctx->state[3] = u64hilo (0xa54ff53a, 0x5f1d36f1);
+  ctx->state[4] = u64hilo (0x510e527f, 0xade682d1);
+  ctx->state[5] = u64hilo (0x9b05688c, 0x2b3e6c1f);
+  ctx->state[6] = u64hilo (0x1f83d9ab, 0xfb41bd6b);
+  ctx->state[7] = u64hilo (0x5be0cd19, 0x137e2179);
+
+  ctx->total[0] = ctx->total[1] = u64lo (0);
+  ctx->buflen = 0;
+}
+
+void
+sha384_init_ctx (struct sha512_ctx *ctx)
+{
+  ctx->state[0] = u64hilo (0xcbbb9d5d, 0xc1059ed8);
+  ctx->state[1] = u64hilo (0x629a292a, 0x367cd507);
+  ctx->state[2] = u64hilo (0x9159015a, 0x3070dd17);
+  ctx->state[3] = u64hilo (0x152fecd8, 0xf70e5939);
+  ctx->state[4] = u64hilo (0x67332667, 0xffc00b31);
+  ctx->state[5] = u64hilo (0x8eb44a87, 0x68581511);
+  ctx->state[6] = u64hilo (0xdb0c2e0d, 0x64f98fa7);
+  ctx->state[7] = u64hilo (0x47b5481d, 0xbefa4fa4);
+
+  ctx->total[0] = ctx->total[1] = u64lo (0);
+  ctx->buflen = 0;
+}
+
+/* Copy the value from V into the memory location pointed to by *CP,
+   If your architecture allows unaligned access, this is equivalent to
+   * (__typeof__ (v) *) cp = v  */
+#ifdef WIN32
+static _inline void
+#else
+static __inline__ void
+#endif
+set_uint64 (char *cp, u64 v)
+{
+  memcpy (cp, &v, sizeof v);
+}
+
+/* Put result from CTX in first 64 bytes following RESBUF.
+   The result must be in little endian byte order.  */
+void *
+sha512_read_ctx (const struct sha512_ctx *ctx, void *resbuf)
+{
+  int i;
+  char *r = (char*)resbuf;
+
+  for (i = 0; i < 8; i++)
+    set_uint64 (r + i * sizeof ctx->state[0], SWAP (ctx->state[i]));
+
+  return resbuf;
+}
+
+void *
+sha384_read_ctx (const struct sha512_ctx *ctx, void *resbuf)
+{
+  int i;
+  char *r = (char*)resbuf;
+
+  for (i = 0; i < 6; i++)
+    set_uint64 (r + i * sizeof ctx->state[0], SWAP (ctx->state[i]));
+
+  return resbuf;
+}
+
+/* Process the remaining bytes in the internal buffer and the usual
+   prolog according to the standard and write the result to RESBUF.  */
+static void
+sha512_conclude_ctx (struct sha512_ctx *ctx)
+{
+  /* Take yet unprocessed bytes into account.  */
+  size_t bytes = ctx->buflen;
+  size_t size = (bytes < 112) ? 128 / 8 : 128 * 2 / 8;
+
+  /* Now count remaining bytes.  */
+  ctx->total[0] = u64plus (ctx->total[0], u64lo (bytes));
+  if (u64lt (ctx->total[0], u64lo (bytes)))
+    ctx->total[1] = u64plus (ctx->total[1], u64lo (1));
+
+  /* Put the 128-bit file length in *bits* at the end of the buffer.
+     Use set_uint64 rather than a simple assignment, to avoid risk of
+     unaligned access.  */
+  set_uint64 ((char *) &ctx->buffer[size - 2],
+	      SWAP (u64or (u64shl (ctx->total[1], 3),
+			   u64shr (ctx->total[0], 61))));
+  set_uint64 ((char *) &ctx->buffer[size - 1],
+	      SWAP (u64shl (ctx->total[0], 3)));
+
+  memcpy (&((char *) ctx->buffer)[bytes], fillbuf, (size - 2) * 8 - bytes);
+
+  /* Process last bytes.  */
+  sha512_process_block (ctx->buffer, size * 8, ctx);
+}
+
+void *
+sha512_finish_ctx (struct sha512_ctx *ctx, void *resbuf)
+{
+  sha512_conclude_ctx (ctx);
+  return sha512_read_ctx (ctx, resbuf);
+}
+
+void *
+sha384_finish_ctx (struct sha512_ctx *ctx, void *resbuf)
+{
+  sha512_conclude_ctx (ctx);
+  return sha384_read_ctx (ctx, resbuf);
+}
+
+/* Compute SHA512 message digest for bytes read from STREAM.  The
+   resulting message digest number will be written into the 64 bytes
+   beginning at RESBLOCK.  */
+int
+sha512_stream (FILE *stream, void *resblock)
+{
+  struct sha512_ctx ctx;
+  char buffer[BLOCKSIZE + 72];
+  size_t sum;
+
+  /* Initialize the computation context.  */
+  sha512_init_ctx (&ctx);
+
+  /* Iterate over full file contents.  */
+  while (1)
+    {
+      /* We read the file in blocks of BLOCKSIZE bytes.  One call of the
+	 computation function processes the whole buffer so that with the
+	 next round of the loop another block can be read.  */
+      size_t n;
+      sum = 0;
+
+      /* Read block.  Take care for partial reads.  */
+      while (1)
+	{
+	  n = fread (buffer + sum, 1, BLOCKSIZE - sum, stream);
+
+	  sum += n;
+
+	  if (sum == BLOCKSIZE)
+	    break;
+
+	  if (n == 0)
+	    {
+	      /* Check for the error flag IFF N == 0, so that we don't
+		 exit the loop after a partial read due to e.g., EAGAIN
+		 or EWOULDBLOCK.  */
+	      if (ferror (stream))
+		return 1;
+	      goto process_partial_block;
+	    }
+
+	  /* We've read at least one byte, so ignore errors.  But always
+	     check for EOF, since feof may be true even though N > 0.
+	     Otherwise, we could end up calling fread after EOF.  */
+	  if (feof (stream))
+	    goto process_partial_block;
+	}
+
+      /* Process buffer with BLOCKSIZE bytes.  Note that
+			BLOCKSIZE % 128 == 0
+       */
+      sha512_process_block (buffer, BLOCKSIZE, &ctx);
+    }
+
+ process_partial_block:;
+
+  /* Process any remaining bytes.  */
+  if (sum > 0)
+    sha512_process_bytes (buffer, sum, &ctx);
+
+  /* Construct result in desired memory.  */
+  sha512_finish_ctx (&ctx, resblock);
+  return 0;
+}
+
+/* FIXME: Avoid code duplication */
+int
+sha384_stream (FILE *stream, void *resblock)
+{
+  struct sha512_ctx ctx;
+  char buffer[BLOCKSIZE + 72];
+  size_t sum;
+
+  /* Initialize the computation context.  */
+  sha384_init_ctx (&ctx);
+
+  /* Iterate over full file contents.  */
+  while (1)
+    {
+      /* We read the file in blocks of BLOCKSIZE bytes.  One call of the
+	 computation function processes the whole buffer so that with the
+	 next round of the loop another block can be read.  */
+      size_t n;
+      sum = 0;
+
+      /* Read block.  Take care for partial reads.  */
+      while (1)
+	{
+	  n = fread (buffer + sum, 1, BLOCKSIZE - sum, stream);
+
+	  sum += n;
+
+	  if (sum == BLOCKSIZE)
+	    break;
+
+	  if (n == 0)
+	    {
+	      /* Check for the error flag IFF N == 0, so that we don't
+		 exit the loop after a partial read due to e.g., EAGAIN
+		 or EWOULDBLOCK.  */
+	      if (ferror (stream))
+		return 1;
+	      goto process_partial_block;
+	    }
+
+	  /* We've read at least one byte, so ignore errors.  But always
+	     check for EOF, since feof may be true even though N > 0.
+	     Otherwise, we could end up calling fread after EOF.  */
+	  if (feof (stream))
+	    goto process_partial_block;
+	}
+
+      /* Process buffer with BLOCKSIZE bytes.  Note that
+			BLOCKSIZE % 128 == 0
+       */
+      sha512_process_block (buffer, BLOCKSIZE, &ctx);
+    }
+
+ process_partial_block:;
+
+  /* Process any remaining bytes.  */
+  if (sum > 0)
+    sha512_process_bytes (buffer, sum, &ctx);
+
+  /* Construct result in desired memory.  */
+  sha384_finish_ctx (&ctx, resblock);
+  return 0;
+}
+
+/* Compute SHA512 message digest for LEN bytes beginning at BUFFER.  The
+   result is always in little endian byte order, so that a byte-wise
+   output yields to the wanted ASCII representation of the message
+   digest.  */
+void *
+sha512_buffer (const char *buffer, size_t len, void *resblock)
+{
+  struct sha512_ctx ctx;
+
+  /* Initialize the computation context.  */
+  sha512_init_ctx (&ctx);
+
+  /* Process whole buffer but last len % 128 bytes.  */
+  sha512_process_bytes (buffer, len, &ctx);
+
+  /* Put result in desired memory area.  */
+  return sha512_finish_ctx (&ctx, resblock);
+}
+
+void *
+sha384_buffer (const char *buffer, size_t len, void *resblock)
+{
+  struct sha512_ctx ctx;
+
+  /* Initialize the computation context.  */
+  sha384_init_ctx (&ctx);
+
+  /* Process whole buffer but last len % 128 bytes.  */
+  sha512_process_bytes (buffer, len, &ctx);
+
+  /* Put result in desired memory area.  */
+  return sha384_finish_ctx (&ctx, resblock);
+}
+
+void
+sha512_process_bytes (const void *buffer, size_t len, struct sha512_ctx *ctx)
+{
+  /* When we already have some bits in our internal buffer concatenate
+     both inputs first.  */
+  if (ctx->buflen != 0)
+    {
+      size_t left_over = ctx->buflen;
+      size_t add = 256 - left_over > len ? len : 256 - left_over;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, add);
+      ctx->buflen += add;
+
+      if (ctx->buflen > 128)
+	{
+	  sha512_process_block (ctx->buffer, ctx->buflen & ~127, ctx);
+
+	  ctx->buflen &= 127;
+	  /* The regions in the following copy operation cannot overlap.  */
+	  memcpy (ctx->buffer,
+		  &((char *) ctx->buffer)[(left_over + add) & ~127],
+		  ctx->buflen);
+	}
+
+      buffer = (const char *) buffer + add;
+      len -= add;
+    }
+
+  /* Process available complete blocks.  */
+  if (len >= 128)
+    {
+#if !_STRING_ARCH_unaligned
+# define alignof(type) offsetof (struct { char c; type x; }, x)
+# define UNALIGNED_P(p) (((size_t) p) % alignof (u64) != 0)
+      if (UNALIGNED_P (buffer))
+	while (len > 128)
+	  {
+	    sha512_process_block (memcpy (ctx->buffer, buffer, 128), 128, ctx);
+	    buffer = (const char *) buffer + 128;
+	    len -= 128;
+	  }
+      else
+#endif
+	{
+	  sha512_process_block (buffer, len & ~127, ctx);
+	  buffer = (const char *) buffer + (len & ~127);
+	  len &= 127;
+	}
+    }
+
+  /* Move remaining bytes in internal buffer.  */
+  if (len > 0)
+    {
+      size_t left_over = ctx->buflen;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, len);
+      left_over += len;
+      if (left_over >= 128)
+	{
+	  sha512_process_block (ctx->buffer, 128, ctx);
+	  left_over -= 128;
+	  memcpy (ctx->buffer, &ctx->buffer[16], left_over);
+	}
+      ctx->buflen = left_over;
+    }
+}
+
+/* --- Code below is the primary difference between sha1.c and sha512.c --- */
+
+/* SHA512 round constants */
+#define K(I) sha512_round_constants[I]
+static u64 const sha512_round_constants[80] = {
+  u64init (0x428a2f98, 0xd728ae22), u64init (0x71374491, 0x23ef65cd),
+  u64init (0xb5c0fbcf, 0xec4d3b2f), u64init (0xe9b5dba5, 0x8189dbbc),
+  u64init (0x3956c25b, 0xf348b538), u64init (0x59f111f1, 0xb605d019),
+  u64init (0x923f82a4, 0xaf194f9b), u64init (0xab1c5ed5, 0xda6d8118),
+  u64init (0xd807aa98, 0xa3030242), u64init (0x12835b01, 0x45706fbe),
+  u64init (0x243185be, 0x4ee4b28c), u64init (0x550c7dc3, 0xd5ffb4e2),
+  u64init (0x72be5d74, 0xf27b896f), u64init (0x80deb1fe, 0x3b1696b1),
+  u64init (0x9bdc06a7, 0x25c71235), u64init (0xc19bf174, 0xcf692694),
+  u64init (0xe49b69c1, 0x9ef14ad2), u64init (0xefbe4786, 0x384f25e3),
+  u64init (0x0fc19dc6, 0x8b8cd5b5), u64init (0x240ca1cc, 0x77ac9c65),
+  u64init (0x2de92c6f, 0x592b0275), u64init (0x4a7484aa, 0x6ea6e483),
+  u64init (0x5cb0a9dc, 0xbd41fbd4), u64init (0x76f988da, 0x831153b5),
+  u64init (0x983e5152, 0xee66dfab), u64init (0xa831c66d, 0x2db43210),
+  u64init (0xb00327c8, 0x98fb213f), u64init (0xbf597fc7, 0xbeef0ee4),
+  u64init (0xc6e00bf3, 0x3da88fc2), u64init (0xd5a79147, 0x930aa725),
+  u64init (0x06ca6351, 0xe003826f), u64init (0x14292967, 0x0a0e6e70),
+  u64init (0x27b70a85, 0x46d22ffc), u64init (0x2e1b2138, 0x5c26c926),
+  u64init (0x4d2c6dfc, 0x5ac42aed), u64init (0x53380d13, 0x9d95b3df),
+  u64init (0x650a7354, 0x8baf63de), u64init (0x766a0abb, 0x3c77b2a8),
+  u64init (0x81c2c92e, 0x47edaee6), u64init (0x92722c85, 0x1482353b),
+  u64init (0xa2bfe8a1, 0x4cf10364), u64init (0xa81a664b, 0xbc423001),
+  u64init (0xc24b8b70, 0xd0f89791), u64init (0xc76c51a3, 0x0654be30),
+  u64init (0xd192e819, 0xd6ef5218), u64init (0xd6990624, 0x5565a910),
+  u64init (0xf40e3585, 0x5771202a), u64init (0x106aa070, 0x32bbd1b8),
+  u64init (0x19a4c116, 0xb8d2d0c8), u64init (0x1e376c08, 0x5141ab53),
+  u64init (0x2748774c, 0xdf8eeb99), u64init (0x34b0bcb5, 0xe19b48a8),
+  u64init (0x391c0cb3, 0xc5c95a63), u64init (0x4ed8aa4a, 0xe3418acb),
+  u64init (0x5b9cca4f, 0x7763e373), u64init (0x682e6ff3, 0xd6b2b8a3),
+  u64init (0x748f82ee, 0x5defb2fc), u64init (0x78a5636f, 0x43172f60),
+  u64init (0x84c87814, 0xa1f0ab72), u64init (0x8cc70208, 0x1a6439ec),
+  u64init (0x90befffa, 0x23631e28), u64init (0xa4506ceb, 0xde82bde9),
+  u64init (0xbef9a3f7, 0xb2c67915), u64init (0xc67178f2, 0xe372532b),
+  u64init (0xca273ece, 0xea26619c), u64init (0xd186b8c7, 0x21c0c207),
+  u64init (0xeada7dd6, 0xcde0eb1e), u64init (0xf57d4f7f, 0xee6ed178),
+  u64init (0x06f067aa, 0x72176fba), u64init (0x0a637dc5, 0xa2c898a6),
+  u64init (0x113f9804, 0xbef90dae), u64init (0x1b710b35, 0x131c471b),
+  u64init (0x28db77f5, 0x23047d84), u64init (0x32caab7b, 0x40c72493),
+  u64init (0x3c9ebe0a, 0x15c9bebc), u64init (0x431d67c4, 0x9c100d4c),
+  u64init (0x4cc5d4be, 0xcb3e42b6), u64init (0x597f299c, 0xfc657e2a),
+  u64init (0x5fcb6fab, 0x3ad6faec), u64init (0x6c44198c, 0x4a475817),
+};
+
+/* Round functions.  */
+#define F2(A, B, C) u64or (u64and (A, B), u64and (C, u64or (A, B)))
+#define F1(E, F, G) u64xor (G, u64and (E, u64xor (F, G)))
+
+/* Process LEN bytes of BUFFER, accumulating context into CTX.
+   It is assumed that LEN % 128 == 0.
+   Most of this code comes from GnuPG's cipher/sha1.c.  */
+
+void
+sha512_process_block (const void *buffer, size_t len, struct sha512_ctx *ctx)
+{
+  u64 const *words = (u64 const *)buffer;
+  u64 const *endp = words + len / sizeof (u64);
+  u64 x[16];
+  u64 a = ctx->state[0];
+  u64 b = ctx->state[1];
+  u64 c = ctx->state[2];
+  u64 d = ctx->state[3];
+  u64 e = ctx->state[4];
+  u64 f = ctx->state[5];
+  u64 g = ctx->state[6];
+  u64 h = ctx->state[7];
+
+  /* First increment the byte count.  FIPS PUB 180-2 specifies the possible
+     length of the file up to 2^128 bits.  Here we only compute the
+     number of bytes.  Do a double word increment.  */
+  ctx->total[0] = u64plus (ctx->total[0], u64lo (len));
+  if (u64lt (ctx->total[0], u64lo (len)))
+    ctx->total[1] = u64plus (ctx->total[1], u64lo (1));
+
+#define S0(x) u64xor (u64rol(x, 63), u64xor (u64rol (x, 56), u64shr (x, 7)))
+#define S1(x) u64xor (u64rol (x, 45), u64xor (u64rol (x, 3), u64shr (x, 6)))
+#define SS0(x) u64xor (u64rol (x, 36), u64xor (u64rol (x, 30), u64rol (x, 25)))
+#define SS1(x) u64xor (u64rol(x, 50), u64xor (u64rol (x, 46), u64rol (x, 23)))
+
+#define M(I) (x[(I) & 15]						  \
+	      = u64plus (x[(I) & 15],					  \
+			 u64plus (S1 (x[((I) - 2) & 15]),		  \
+				  u64plus (x[((I) - 7) & 15],		  \
+					   S0 (x[((I) - 15) & 15])))))
+
+#define R(A, B, C, D, E, F, G, H, K, M)					  \
+  do									  \
+    {									  \
+      u64 t0 = u64plus (SS0 (A), F2 (A, B, C));				  \
+      u64 t1 =								  \
+	u64plus (H, u64plus (SS1 (E),					  \
+			     u64plus (F1 (E, F, G), u64plus (K, M))));	  \
+      D = u64plus (D, t1);						  \
+      H = u64plus (t0, t1);						  \
+    }									  \
+  while (0)
+
+  while (words < endp)
+    {
+      int t;
+      /* FIXME: see sha1.c for a better implementation.  */
+      for (t = 0; t < 16; t++)
+	{
+	  x[t] = SWAP (*words);
+	  words++;
+	}
+
+      R( a, b, c, d, e, f, g, h, K( 0), x[ 0] );
+      R( h, a, b, c, d, e, f, g, K( 1), x[ 1] );
+      R( g, h, a, b, c, d, e, f, K( 2), x[ 2] );
+      R( f, g, h, a, b, c, d, e, K( 3), x[ 3] );
+      R( e, f, g, h, a, b, c, d, K( 4), x[ 4] );
+      R( d, e, f, g, h, a, b, c, K( 5), x[ 5] );
+      R( c, d, e, f, g, h, a, b, K( 6), x[ 6] );
+      R( b, c, d, e, f, g, h, a, K( 7), x[ 7] );
+      R( a, b, c, d, e, f, g, h, K( 8), x[ 8] );
+      R( h, a, b, c, d, e, f, g, K( 9), x[ 9] );
+      R( g, h, a, b, c, d, e, f, K(10), x[10] );
+      R( f, g, h, a, b, c, d, e, K(11), x[11] );
+      R( e, f, g, h, a, b, c, d, K(12), x[12] );
+      R( d, e, f, g, h, a, b, c, K(13), x[13] );
+      R( c, d, e, f, g, h, a, b, K(14), x[14] );
+      R( b, c, d, e, f, g, h, a, K(15), x[15] );
+      R( a, b, c, d, e, f, g, h, K(16), M(16) );
+      R( h, a, b, c, d, e, f, g, K(17), M(17) );
+      R( g, h, a, b, c, d, e, f, K(18), M(18) );
+      R( f, g, h, a, b, c, d, e, K(19), M(19) );
+      R( e, f, g, h, a, b, c, d, K(20), M(20) );
+      R( d, e, f, g, h, a, b, c, K(21), M(21) );
+      R( c, d, e, f, g, h, a, b, K(22), M(22) );
+      R( b, c, d, e, f, g, h, a, K(23), M(23) );
+      R( a, b, c, d, e, f, g, h, K(24), M(24) );
+      R( h, a, b, c, d, e, f, g, K(25), M(25) );
+      R( g, h, a, b, c, d, e, f, K(26), M(26) );
+      R( f, g, h, a, b, c, d, e, K(27), M(27) );
+      R( e, f, g, h, a, b, c, d, K(28), M(28) );
+      R( d, e, f, g, h, a, b, c, K(29), M(29) );
+      R( c, d, e, f, g, h, a, b, K(30), M(30) );
+      R( b, c, d, e, f, g, h, a, K(31), M(31) );
+      R( a, b, c, d, e, f, g, h, K(32), M(32) );
+      R( h, a, b, c, d, e, f, g, K(33), M(33) );
+      R( g, h, a, b, c, d, e, f, K(34), M(34) );
+      R( f, g, h, a, b, c, d, e, K(35), M(35) );
+      R( e, f, g, h, a, b, c, d, K(36), M(36) );
+      R( d, e, f, g, h, a, b, c, K(37), M(37) );
+      R( c, d, e, f, g, h, a, b, K(38), M(38) );
+      R( b, c, d, e, f, g, h, a, K(39), M(39) );
+      R( a, b, c, d, e, f, g, h, K(40), M(40) );
+      R( h, a, b, c, d, e, f, g, K(41), M(41) );
+      R( g, h, a, b, c, d, e, f, K(42), M(42) );
+      R( f, g, h, a, b, c, d, e, K(43), M(43) );
+      R( e, f, g, h, a, b, c, d, K(44), M(44) );
+      R( d, e, f, g, h, a, b, c, K(45), M(45) );
+      R( c, d, e, f, g, h, a, b, K(46), M(46) );
+      R( b, c, d, e, f, g, h, a, K(47), M(47) );
+      R( a, b, c, d, e, f, g, h, K(48), M(48) );
+      R( h, a, b, c, d, e, f, g, K(49), M(49) );
+      R( g, h, a, b, c, d, e, f, K(50), M(50) );
+      R( f, g, h, a, b, c, d, e, K(51), M(51) );
+      R( e, f, g, h, a, b, c, d, K(52), M(52) );
+      R( d, e, f, g, h, a, b, c, K(53), M(53) );
+      R( c, d, e, f, g, h, a, b, K(54), M(54) );
+      R( b, c, d, e, f, g, h, a, K(55), M(55) );
+      R( a, b, c, d, e, f, g, h, K(56), M(56) );
+      R( h, a, b, c, d, e, f, g, K(57), M(57) );
+      R( g, h, a, b, c, d, e, f, K(58), M(58) );
+      R( f, g, h, a, b, c, d, e, K(59), M(59) );
+      R( e, f, g, h, a, b, c, d, K(60), M(60) );
+      R( d, e, f, g, h, a, b, c, K(61), M(61) );
+      R( c, d, e, f, g, h, a, b, K(62), M(62) );
+      R( b, c, d, e, f, g, h, a, K(63), M(63) );
+      R( a, b, c, d, e, f, g, h, K(64), M(64) );
+      R( h, a, b, c, d, e, f, g, K(65), M(65) );
+      R( g, h, a, b, c, d, e, f, K(66), M(66) );
+      R( f, g, h, a, b, c, d, e, K(67), M(67) );
+      R( e, f, g, h, a, b, c, d, K(68), M(68) );
+      R( d, e, f, g, h, a, b, c, K(69), M(69) );
+      R( c, d, e, f, g, h, a, b, K(70), M(70) );
+      R( b, c, d, e, f, g, h, a, K(71), M(71) );
+      R( a, b, c, d, e, f, g, h, K(72), M(72) );
+      R( h, a, b, c, d, e, f, g, K(73), M(73) );
+      R( g, h, a, b, c, d, e, f, K(74), M(74) );
+      R( f, g, h, a, b, c, d, e, K(75), M(75) );
+      R( e, f, g, h, a, b, c, d, K(76), M(76) );
+      R( d, e, f, g, h, a, b, c, K(77), M(77) );
+      R( c, d, e, f, g, h, a, b, K(78), M(78) );
+      R( b, c, d, e, f, g, h, a, K(79), M(79) );
+
+      a = ctx->state[0] = u64plus (ctx->state[0], a);
+      b = ctx->state[1] = u64plus (ctx->state[1], b);
+      c = ctx->state[2] = u64plus (ctx->state[2], c);
+      d = ctx->state[3] = u64plus (ctx->state[3], d);
+      e = ctx->state[4] = u64plus (ctx->state[4], e);
+      f = ctx->state[5] = u64plus (ctx->state[5], f);
+      g = ctx->state[6] = u64plus (ctx->state[6], g);
+      h = ctx->state[7] = u64plus (ctx->state[7], h);
+   }
+}
+
+#ifdef __cplusplus
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/spas_client.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/spas_client.c b/rocketmq-cpp/libs/signature/src/spas_client.c
new file mode 100755
index 0000000..08b5a2d
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/spas_client.c
@@ -0,0 +1,508 @@
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <signal.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+
+#include "spas_client.h"
+#include "sha1.h"
+#include "sha256.h"
+#include "hmac.h"
+#include "base64.h"
+
+#ifdef WIN32
+#include <io.h>  
+#include <process.h>  
+#else
+#include <unistd.h>
+#endif
+#ifdef SPAS_MT
+#include <pthread.h>
+#endif
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+#define SPAS_VERSION "SPAS_V1_0"
+
+static SPAS_CREDENTIAL g_credential;
+static char g_path[SPAS_MAX_PATH];
+static int g_loaded = 0;
+static unsigned int refresh = 10;
+static time_t modified = 0;
+
+#ifdef SPAS_MT
+
+static pthread_mutex_t cred_mutex = PTHREAD_MUTEX_INITIALIZER;
+static pthread_once_t cred_once = PTHREAD_ONCE_INIT;
+static pthread_key_t cred_key;
+
+#endif
+
+extern void * _mem_alloc(unsigned int size);
+extern void * _mem_realloc(void *ptr, unsigned int old_size, unsigned int new_size);
+extern void _mem_free(void *ptr);
+extern void _trim(char *str);
+
+
+void * _mem_alloc(unsigned int size) {
+	void *p = malloc(size);
+	if (p != NULL) {
+		memset(p, 0, size);
+	}
+	return p;
+}
+
+void * _mem_realloc(void *ptr, unsigned int old_size, unsigned int new_size) {
+	void *p = realloc(ptr, new_size);
+	if (p != NULL && new_size > old_size) {
+		memset((unsigned int *)p + old_size, 0, new_size - old_size);
+	}
+	return p;
+}
+
+
+void _mem_free(void *ptr) {
+	free(ptr);
+}
+
+void _trim(char *str) {
+	int len = strlen(str);
+	int i;
+	int done = 0;
+	for (i = len - 1; i >= 0; i--) {
+		switch (str[i]) {
+		case ' ':
+		case '\t':
+		case '\r':
+		case '\n':
+			str[i] = '\0';
+			break;
+		default:
+			done = 1;
+			break;
+		}
+		if (done) {
+			break;
+		}
+	}
+}
+
+static int _load_credential(SPAS_CREDENTIAL *pcred, char *path) {
+	FILE *fp = NULL;
+	char buf[SPAS_MAX_KEY_LEN * 2];
+	if (pcred == NULL || path == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	fp = fopen(path, "r");
+	if (fp == NULL) {
+		return ERROR_FILE_OPEN;
+	}
+	memset(pcred, 0, sizeof(SPAS_CREDENTIAL));
+	while (fgets(buf, sizeof(buf), fp)) {
+		_trim(buf);
+		int len = strlen(SPAS_ACCESS_KEY_TAG);
+		if (strncmp(buf, SPAS_ACCESS_KEY_TAG, len) == 0 && buf[len] == '=') {
+			strncpy(pcred->access_key, buf + len + 1, SPAS_MAX_KEY_LEN - 1);
+		}
+		else {
+			len = strlen(SPAS_SECRET_KEY_TAG);
+			if (strncmp(buf, SPAS_SECRET_KEY_TAG, len) == 0 && buf[len] == '=') {
+				strncpy(pcred->secret_key, buf + len + 1, SPAS_MAX_KEY_LEN - 1);
+			}
+		}
+	}
+	fclose(fp);
+	if (strlen(pcred->access_key) == 0 || strlen(pcred->secret_key) == 0) {
+		return ERROR_MISSING_KEY;
+	}
+	return NO_ERROR;
+}
+
+#ifndef WIN32
+static void _reload_credential(int sig) {
+	int ret;
+	SPAS_CREDENTIAL credential;
+	struct stat status;
+	struct sigaction act;
+	
+	if (sig != SIGALRM) {
+		return;
+	}
+	
+	memset(&act, 0, sizeof(act));
+	act.sa_handler = _reload_credential;
+	sigaction(SIGALRM, &act, NULL);
+	alarm(refresh);
+	if (g_path[0] != '\0') {
+		ret = stat(g_path, &status);
+		if (ret != 0) {
+			return;
+		}
+		if (status.st_mtime == modified) {
+			return;
+		}
+		ret = _load_credential(&credential, g_path);
+		if (ret != NO_ERROR) {
+			return;
+		}
+#ifdef SPAS_MT
+		pthread_mutex_lock(&cred_mutex);
+#endif
+		memcpy(&g_credential, &credential, sizeof(SPAS_CREDENTIAL));
+#ifdef SPAS_MT
+		pthread_mutex_unlock(&cred_mutex);
+#endif
+		modified = status.st_mtime;
+	}
+}
+
+static int _update_credential_by_alarm() {
+	struct sigaction act;
+	
+	memset(&act, 0, sizeof(act));
+	act.sa_handler = _reload_credential;
+	sigaction(SIGALRM, &act, NULL);
+	alarm(refresh);
+	return NO_ERROR;
+}
+#endif 
+
+#ifdef SPAS_MT
+
+static void * _update_credential_entry(void *arg) {
+	int ret;
+	SPAS_CREDENTIAL credential;
+	struct stat status;
+	struct timeval tv;
+	while (1) {
+		tv.tv_sec = refresh;
+		tv.tv_usec = 0;
+		select(0, NULL, NULL, NULL, &tv);
+		if (g_path[0] != '\0') {
+			ret = stat(g_path, &status);
+			if (ret != 0) {
+				continue;
+			}
+			if (status.st_mtime == modified) {
+				continue;
+			}
+			ret = _load_credential(&credential, g_path);
+			if (ret != NO_ERROR) {
+				continue;
+			}
+			pthread_mutex_lock(&cred_mutex);
+			memcpy(&g_credential, &credential, sizeof(SPAS_CREDENTIAL));
+			pthread_mutex_unlock(&cred_mutex);
+			modified = status.st_mtime;
+		}
+	}
+	return NULL;
+}
+
+static int _update_credential_by_thread() {
+	pthread_t tid;
+	int ret;
+
+	ret = pthread_create(&tid, NULL, _update_credential_entry, NULL);
+	if (ret != 0) {
+		return ERROR_UPDATE_CREDENTIAL;
+	}
+	pthread_detach(tid);
+	return NO_ERROR;
+}
+
+
+
+int spas_load_credential(char *path, CREDENTIAL_UPDATE_MODE mode) {
+	int ret = NO_ERROR;
+	SPAS_CREDENTIAL credential;
+	
+	if (g_loaded) {
+		return NO_ERROR;
+	}
+	if (path == NULL) {
+		path = getenv(SPAS_CREDENTIAL_ENV);
+		if (path == NULL) {
+			return ERROR_NO_CREDENTIAL;
+		}
+	}
+	strncpy(g_path, path, SPAS_MAX_PATH - 1);
+	ret = _load_credential(&credential, path);
+	if (ret != NO_ERROR) {
+		return ret;
+	}
+#ifdef SPAS_MT
+	pthread_mutex_lock(&cred_mutex);
+#endif
+	if (!g_loaded) {
+		memcpy(&g_credential, &credential, sizeof(SPAS_CREDENTIAL));
+		g_loaded = 1;
+	}
+#ifdef SPAS_MT
+	pthread_mutex_unlock(&cred_mutex);
+#endif
+	switch (mode) {
+	case UPDATE_BY_ALARM:
+		ret = _update_credential_by_alarm();
+		break;
+#ifdef SPAS_MT
+	case UPDATE_BY_THREAD:
+		ret = _update_credential_by_thread();
+		break;
+#endif
+	case NO_UPDATE:
+	default:
+		ret = NO_ERROR;
+		break;
+	}
+	return ret;
+}
+
+#endif
+
+SPAS_CREDENTIAL * spas_get_credential(void) {
+	SPAS_CREDENTIAL *credential = (SPAS_CREDENTIAL *)_mem_alloc(sizeof(SPAS_CREDENTIAL));
+	if (credential != NULL) {
+#ifdef SPAS_MT
+		pthread_mutex_lock(&cred_mutex);
+#endif
+		memcpy(credential, &g_credential, sizeof(SPAS_CREDENTIAL));
+#ifdef SPAS_MT
+		pthread_mutex_unlock(&cred_mutex);
+#endif
+	}
+	return credential;
+}
+
+
+int spas_set_access_key(char *key) {
+	int len = 0;
+	if (key == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	len = strlen(key);
+	if (len == 0 || len >= SPAS_MAX_KEY_LEN) {
+		return ERROR_KEY_LENGTH;
+	}
+#ifdef SPAS_MT
+	pthread_mutex_lock(&cred_mutex);
+#endif
+	memcpy(g_credential.access_key, key, len + 1);
+#ifdef SPAS_MT
+	pthread_mutex_unlock(&cred_mutex);
+#endif
+	return NO_ERROR;
+}
+
+int spas_set_secret_key(char *key) {
+	int len = 0;
+	if (key == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	len = strlen(key);
+	if (len == 0 || len >= SPAS_MAX_KEY_LEN) {
+		return ERROR_KEY_LENGTH;
+	}
+#ifdef SPAS_MT
+	pthread_mutex_lock(&cred_mutex);
+#endif
+	memcpy(g_credential.secret_key, key, len + 1);
+#ifdef SPAS_MT
+	pthread_mutex_unlock(&cred_mutex);
+#endif
+	return NO_ERROR;
+}
+
+char * spas_get_access_key() {
+	return g_credential.access_key;
+}
+
+char * spas_get_secret_key() {
+	return g_credential.secret_key;
+}
+
+#ifdef SPAS_MT
+
+static void _free_thread_credential(void* credential)
+{
+	if (credential != NULL) {
+		_mem_free(credential);
+	}
+}
+
+static void _init_credential_key(void) {
+	pthread_key_create(&cred_key, _free_thread_credential);
+}
+
+static SPAS_CREDENTIAL * _get_thread_credential(void) {
+	int ret = 0;
+	SPAS_CREDENTIAL *credential = NULL;
+	ret = pthread_once(&cred_once, _init_credential_key);
+	if (ret != 0) {
+		return NULL;
+	}
+	credential = pthread_getspecific(cred_key);
+	if (credential == NULL) {
+		credential = _mem_alloc(sizeof(SPAS_CREDENTIAL));
+		if (credential == NULL) {
+			return NULL;
+		}
+		ret = pthread_setspecific(cred_key, credential);
+		if (ret != 0) {
+			_mem_free(credential);
+			return NULL;
+		}
+	}
+	return credential;
+}
+
+int spas_load_thread_credential(char *path) {
+	int ret = NO_ERROR;
+	SPAS_CREDENTIAL * credential = NULL;
+	credential = _get_thread_credential();
+	if (credential == NULL) {
+		return ERROR_MEM_ALLOC;
+	}
+	ret = _load_credential(credential, path);
+	if (ret != NO_ERROR) {
+		memset(credential, 0, sizeof(SPAS_CREDENTIAL));
+		return ret;
+	}
+	return NO_ERROR;
+}
+
+int spas_set_thread_access_key(char *key) {
+	int len = 0;
+	SPAS_CREDENTIAL * credential = NULL;
+	if (key == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	len = strlen(key);
+	if (len == 0 || len >= SPAS_MAX_KEY_LEN) {
+		return ERROR_KEY_LENGTH;
+	}
+	credential = _get_thread_credential();
+	if (credential == NULL) {
+		return ERROR_MEM_ALLOC;
+	}
+	memcpy(credential->access_key, key, len + 1);
+	return NO_ERROR;
+}
+
+int spas_set_thread_secret_key(char *key) {
+	int len = 0;
+	SPAS_CREDENTIAL * credential = NULL;
+	if (key == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	len = strlen(key);
+	if (len == 0 || len >= SPAS_MAX_KEY_LEN) {
+		return ERROR_KEY_LENGTH;
+	}
+	credential = _get_thread_credential();
+	if (credential == NULL) {
+		return ERROR_MEM_ALLOC;
+	}
+	memcpy(credential->secret_key, key, len + 1);
+	return NO_ERROR;
+
+}
+
+char * spas_get_thread_access_key(void) {
+	SPAS_CREDENTIAL * credential = _get_thread_credential();
+	if (credential == NULL) {
+		return NULL;
+	}
+	return credential->access_key;
+}
+
+char * spas_get_thread_secret_key(void) {
+	SPAS_CREDENTIAL * credential = _get_thread_credential();
+	if (credential == NULL) {
+		return NULL;
+	}
+	return credential->secret_key;
+}
+
+#endif
+
+
+char * spas_get_signature(const SPAS_PARAM_LIST *list, const char *key) {
+	return spas_get_signature2(list, key, SIGN_HMACSHA1);
+}
+
+char * spas_get_signature2(const SPAS_PARAM_LIST *list, const char *key, SPAS_SIGN_ALGORITHM algorithm) {
+	char *sign = NULL;
+	char *data = NULL;
+	if (list == NULL || key == NULL) {
+		return NULL;
+	}
+	data = param_list_to_str(list);
+	if (data == NULL) {
+		return NULL;
+	}
+	sign = spas_sign2(data, strlen(data),key, algorithm);
+	_mem_free(data);
+	return sign;
+}
+
+char * spas_sign(const char *data, size_t size, const char *key) {
+	return spas_sign2(data, size, key, SIGN_HMACSHA1);
+}
+
+char * spas_sign2(const char *data, size_t size, const char *key, SPAS_SIGN_ALGORITHM algorithm) {
+	int ret;
+	int dsize = 0;
+	char *sha_buf = NULL;
+	char *base64_ret = NULL;
+	if (data == NULL || key == NULL) {
+		return NULL;
+	}
+	if (algorithm == SIGN_HMACSHA1) {
+		dsize = SHA1_DIGEST_SIZE;
+		sha_buf = (char *)_mem_alloc(dsize + 1);
+		if (sha_buf == NULL) {
+			return NULL;
+		}
+		ret = hmac_sha1(key, strlen(key), data, size, sha_buf);
+		if (ret < 0) {
+			_mem_free(sha_buf);
+			return NULL;
+		}
+	}
+	else if (algorithm == SIGN_HMACSHA256) {
+		dsize = SHA256_DIGEST_SIZE;
+		sha_buf = (char *)_mem_alloc(dsize + 1);
+		if (sha_buf == NULL) {
+			return NULL;
+		}
+		ret = hmac_sha256(key, strlen(key), data, strlen(data), sha_buf);
+		if (ret < 0) {
+			_mem_free(sha_buf);
+			return NULL;
+		}
+	}
+	else {
+		return NULL;
+	}
+	ret = base64_encode_alloc(sha_buf, dsize, &base64_ret);
+	_mem_free(sha_buf);
+	return base64_ret;
+	
+}
+
+void    spas_mem_free(char *pSignature)
+{
+	_mem_free(pSignature);
+}
+
+char * spas_get_version(void) {
+	return SPAS_VERSION;
+}
+
+#ifdef __cplusplus
+}
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/project/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/project/CMakeLists.txt b/rocketmq-cpp/project/CMakeLists.txt
new file mode 100755
index 0000000..02723af
--- /dev/null
+++ b/rocketmq-cpp/project/CMakeLists.txt
@@ -0,0 +1,63 @@
+# 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.
+
+# source files
+project(rocketmq4cpp)
+
+file(GLOB_RECURSE SRC_FILES   ${CMAKE_SOURCE_DIR}/src/*)
+list(REMOVE_ITEM  SRC_FILES   ${CMAKE_SOURCE_DIR}/src/dllmain.cpp)
+
+# subdirs
+SET(SUB_DIRS)
+file(GLOB children ${CMAKE_SOURCE_DIR}/src/*)
+FOREACH(child ${children})
+	IF(IS_DIRECTORY ${child})
+	    LIST(APPEND SUB_DIRS ${child})
+	ENDIF()
+ENDFOREACH()
+LIST(APPEND SUB_DIRS ${CMAKE_SOURCE_DIR}/src)
+
+include_directories(${CMAKE_SOURCE_DIR}/include)
+include_directories(${SUB_DIRS})
+
+# libs_directories
+file(GLOB LIB_DIRS ${CMAKE_SOURCE_DIR}/libs/*)
+foreach(dir ${LIB_DIRS})
+    if(IS_DIRECTORY ${dir})
+        set(CMAKE_PREFIX_PATH ${CMAKE_PREFIX_PATH};${dir})
+        include_directories(${dir}/include)
+    endif()
+endforeach()
+
+# static
+add_library(rocketmq_static STATIC ${SRC_FILES})
+set_target_properties(rocketmq_static PROPERTIES OUTPUT_NAME "rocketmq")
+add_dependencies(rocketmq_static Signature)
+target_link_libraries(rocketmq_static ${deplibs})
+target_link_libraries(rocketmq_static Signature)
+
+# shared
+set(CMAKE_SHARED_LINKER_FLAGS "-DBOOST_ALL_DYN_LINK -shared")
+add_library(rocketmq_shared SHARED ${SRC_FILES})
+set_target_properties(rocketmq_shared PROPERTIES OUTPUT_NAME "rocketmq")
+add_dependencies(rocketmq_shared Signature)
+target_link_libraries(rocketmq_shared ${deplibs})
+target_link_libraries(rocketmq_shared  Signature)
+
+# install
+install (TARGETS   rocketmq_static             DESTINATION bin)
+install (DIRECTORY ${CMAKE_SOURCE_DIR}/include/ DESTINATION include)
+install (DIRECTORY ${CMAKE_SOURCE_DIR}/doc/     DESTINATION doc)
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/project/tool.mak
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/project/tool.mak b/rocketmq-cpp/project/tool.mak
new file mode 100644
index 0000000..e8f57fc
--- /dev/null
+++ b/rocketmq-cpp/project/tool.mak
@@ -0,0 +1,21 @@
+define BUILD_LIBRARY
+$(if $(wildcard $@),@$(RM) $@)
+$(if $(wildcard ar.mac),@$(RM) ar.mac)
+$(if $(filter %.a, $^),
+@echo CREATE $@ > ar.mac
+@echo SAVE >> ar.mac
+@echo END >> ar.mac
+@$(AR) -M < ar.mac
+)
+$(if $(filter %.o,$^),@$(AR) -q $@ $(filter %.o, $^))
+$(if $(filter %.a, $^),
+@echo OPEN $@ > ar.mac
+$(foreach LIB, $(filter %.a, $^),
+@echo ADDLIB $(LIB) >> ar.mac
+)
+@echo SAVE >> ar.mac
+@echo END >> ar.mac
+@$(AR) -M < ar.mac
+@$(RM) ar.mac
+)
+endef
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientAPIImpl.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientAPIImpl.cpp b/rocketmq-cpp/src/MQClientAPIImpl.cpp
new file mode 100755
index 0000000..2d9c39c
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientAPIImpl.cpp
@@ -0,0 +1,922 @@
+/*
+ * 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 "MQClientAPIImpl.h"
+#include <assert.h>
+#include <fstream>
+#include "CommunicationMode.h"
+#include "Logging.h"
+#include "MQDecoder.h"
+#include "PullResultExt.h"
+
+namespace rocketmq {
+//<!************************************************************************
+MQClientAPIImpl::MQClientAPIImpl(
+    const string& mqClientId, ClientRemotingProcessor* clientRemotingProcessor, int pullThreadNum,
+    uint64_t tcpConnectTimeout, uint64_t tcpTransportTryLockTimeout,
+    string unitName)
+    : m_firstFetchNameSrv(true), m_mqClientId(mqClientId) {
+  m_pRemotingClient.reset(new TcpRemotingClient(
+      pullThreadNum, tcpConnectTimeout, tcpTransportTryLockTimeout));
+  m_pRemotingClient->registerProcessor(CHECK_TRANSACTION_STATE,
+                                       clientRemotingProcessor);
+  m_pRemotingClient->registerProcessor(RESET_CONSUMER_CLIENT_OFFSET,
+                                       clientRemotingProcessor);
+  m_pRemotingClient->registerProcessor(GET_CONSUMER_STATUS_FROM_CLIENT,
+                                       clientRemotingProcessor);
+  m_pRemotingClient->registerProcessor(GET_CONSUMER_RUNNING_INFO,
+                                       clientRemotingProcessor);
+  m_pRemotingClient->registerProcessor(NOTIFY_CONSUMER_IDS_CHANGED,
+                                       clientRemotingProcessor);
+  m_pRemotingClient->registerProcessor(CONSUME_MESSAGE_DIRECTLY,
+                                       clientRemotingProcessor);
+
+  m_topAddressing.reset(new TopAddressing(unitName));
+}
+
+MQClientAPIImpl::~MQClientAPIImpl() {
+  m_pRemotingClient = NULL;
+  m_topAddressing = NULL;
+}
+
+void MQClientAPIImpl::stopAllTcpTransportThread() {
+  m_pRemotingClient->stopAllTcpTransportThread();
+}
+
+bool MQClientAPIImpl::writeDataToFile(string filename, string data,
+                                      bool isSync) {
+  if (data.size() == 0) return false;
+
+  int fd = open(filename.c_str(), O_RDWR | O_CREAT, 0755);
+  if (fd < 0) {
+    LOG_ERROR("open file failed, file:%s, msg:%s", filename.c_str(),
+              strerror(errno));
+    return false;
+  }
+
+  int byte_write = 0;
+  int byte_left = data.size();
+  const char* pData = data.c_str();
+  while (byte_left > 0) {
+    byte_write = write(fd, pData, byte_left);
+    if (byte_write == -1) {
+      LOG_ERROR("write data fail, data len:%zu, file:%s, msg:%s", data.size(),
+                filename.c_str(), strerror(errno));
+      close(fd);
+      return false;
+    }
+    byte_left -= byte_write;
+    pData += byte_write;
+  }
+  pData = NULL;
+
+  if (isSync) {
+    LOG_INFO("fsync with filename:%s", filename.c_str());
+    fsync(fd);
+  }
+  close(fd);
+
+  return true;
+}
+
+string MQClientAPIImpl::fetchNameServerAddr(const string& NSDomain) {
+  try {
+    string homeDir(UtilAll::getHomeDirectory());
+    string storePath = homeDir + "/logs/metaq-client4cpp/snapshot";
+
+    if (access(storePath.c_str(), F_OK) != 0) {
+      if (mkdir(storePath.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) != 0) {
+        LOG_ERROR("create data dir:%s error", storePath.c_str());
+      }
+    }
+    string file(storePath);
+    string fileBak(storePath);
+    vector<string> ret_;
+    int retSize = UtilAll::Split(ret_, m_mqClientId, "@");
+    if(retSize==2){
+      file.append("/nameserver_addr-").append(ret_[retSize-1]);
+    }else{
+      LOG_ERROR("split mqClientId:%s fail", m_mqClientId.c_str());
+      file.append("/nameserver_addr-DEFAULT");
+    }
+    fileBak.append("/nameserver_addr.bak");
+    const string addrs = m_topAddressing->fetchNSAddr(NSDomain);
+    if (addrs.empty()) {
+      if (m_nameSrvAddr.empty()) {
+        LOG_INFO("Load the name server snapshot local file:%s", file.c_str());
+        if (access(file.c_str(), F_OK) == 0) {
+          ifstream snapshot_file(file, ios::binary);
+          istreambuf_iterator<char> beg(snapshot_file), end;
+          string filecontent(beg, end);
+          updateNameServerAddr(filecontent);
+          m_nameSrvAddr = filecontent;
+        } else {
+          LOG_WARN("The name server snapshot local file not exists");
+        }
+      }
+    } else {
+      if (m_firstFetchNameSrv == true) {
+        // it is the first time, so need to create the name server snapshot
+        // local file
+        m_firstFetchNameSrv = false;
+      }
+      if (addrs.compare(m_nameSrvAddr) != 0) {
+        LOG_INFO("name server address changed, old: %s, new: %s",
+                 m_nameSrvAddr.c_str(), addrs.c_str());
+        updateNameServerAddr(addrs);
+        m_nameSrvAddr = addrs;
+      } else {
+        if (!m_firstFetchNameSrv) return m_nameSrvAddr;
+      }
+      // update the snapshot local file if nameSrv changes or
+      // m_firstFetchNameSrv==true
+      if (writeDataToFile(fileBak, addrs, true)) {
+        if (rename(fileBak.c_str(), file.c_str()) == -1)
+          LOG_ERROR("could not rename bak file:%s", strerror(errno));
+      }
+    }
+
+    if (access(file.c_str(), F_OK) != 0) {
+      // the name server snapshot local file maybe deleted by force, create it
+      if (writeDataToFile(fileBak, m_nameSrvAddr, true)) {
+        if (rename(fileBak.c_str(), file.c_str()) == -1)
+          LOG_ERROR("could not rename bak file:%s", strerror(errno));
+      }
+    }
+  } catch (...) {
+  }
+  return m_nameSrvAddr;
+}
+
+void MQClientAPIImpl::updateNameServerAddr(const string& addrs) {
+  if (m_pRemotingClient != NULL)
+    m_pRemotingClient->updateNameServerAddressList(addrs);
+}
+
+void MQClientAPIImpl::callSignatureBeforeRequest(
+    const string& addr, RemotingCommand& request,
+    const SessionCredentials& session_credentials) {
+  ClientRPCHook rpcHook(session_credentials);
+  rpcHook.doBeforeRequest(addr, request);
+}
+
+// Note: all request rules: throw exception if got broker error response,
+// exclude getTopicRouteInfoFromNameServer and unregisterClient
+void MQClientAPIImpl::createTopic(
+    const string& addr, const string& defaultTopic, TopicConfig topicConfig,
+    const SessionCredentials& sessionCredentials) {
+  string topicWithProjectGroup = topicConfig.getTopicName();
+  CreateTopicRequestHeader* requestHeader = new CreateTopicRequestHeader();
+  requestHeader->topic = (topicWithProjectGroup);
+  requestHeader->defaultTopic = (defaultTopic);
+  requestHeader->readQueueNums = (topicConfig.getReadQueueNums());
+  requestHeader->writeQueueNums = (topicConfig.getWriteQueueNums());
+  requestHeader->perm = (topicConfig.getPerm());
+  requestHeader->topicFilterType = (topicConfig.getTopicFilterType());
+
+  RemotingCommand request(UPDATE_AND_CREATE_TOPIC, requestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE:
+        return;
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+SendResult MQClientAPIImpl::sendMessage(
+    const string& addr, const string& brokerName, const MQMessage& msg,
+    SendMessageRequestHeader* pRequestHeader, int timeoutMillis,
+    int communicationMode, SendCallback* pSendCallback,
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(SEND_MESSAGE, pRequestHeader);
+  string body = msg.getBody();
+  request.SetBody(body.c_str(), body.length());
+  request.setMsgBody(body);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  switch (communicationMode) {
+    case ComMode_ONEWAY:
+      m_pRemotingClient->invokeOneway(addr, request);
+      break;
+    case ComMode_ASYNC:
+      sendMessageAsync(addr, brokerName, msg, request, pSendCallback,
+                       timeoutMillis);
+      break;
+    case ComMode_SYNC:
+      return sendMessageSync(addr, brokerName, msg, request, timeoutMillis);
+    default:
+      break;
+  }
+  return SendResult();
+}
+
+void MQClientAPIImpl::sendHearbeat(
+    const string& addr, HeartbeatData* pHeartbeatData,
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(HEART_BEAT, NULL);
+
+  string body;
+  pHeartbeatData->Encode(body);
+  request.SetBody(body.data(), body.length());
+  request.setMsgBody(body);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  if (m_pRemotingClient->invokeHeartBeat(addr, request)) {
+    LOG_INFO("sendheartbeat to broker:%s success", addr.c_str());
+  }
+}
+
+void MQClientAPIImpl::unregisterClient(
+    const string& addr, const string& clientID, const string& producerGroup,
+    const string& consumerGroup, const SessionCredentials& sessionCredentials) {
+  LOG_INFO("unregisterClient to broker:%s", addr.c_str());
+  RemotingCommand request(UNREGISTER_CLIENT,
+                          new UnregisterClientRequestHeader(
+                              clientID, producerGroup, consumerGroup));
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE:
+        LOG_INFO("unregisterClient to:%s success", addr.c_str());
+        return;
+      default:
+        break;
+    }
+    LOG_WARN("unregisterClient fail:%s,%d", response->getRemark().c_str(),
+             response->getCode());
+  }
+}
+
+// return NULL if got no response or error response
+TopicRouteData* MQClientAPIImpl::getTopicRouteInfoFromNameServer(
+    const string& topic, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(GET_ROUTEINTO_BY_TOPIC,
+                          new GetRouteInfoRequestHeader(topic));
+  callSignatureBeforeRequest("", request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync("", request, timeoutMillis));
+
+  if (pResponse != NULL) {
+    if (((*(pResponse->GetBody())).getSize() == 0) ||
+        ((*(pResponse->GetBody())).getData() != NULL)) {
+      switch (pResponse->getCode()) {
+        case SUCCESS_VALUE: {
+          const MemoryBlock* pbody = pResponse->GetBody();
+          if (pbody->getSize()) {
+            TopicRouteData* topicRoute = TopicRouteData::Decode(pbody);
+            return topicRoute;
+          }
+        }
+        case TOPIC_NOT_EXIST: {
+          return NULL;
+        }
+        default:
+          break;
+      }
+      LOG_WARN("%s,%d", pResponse->getRemark().c_str(), pResponse->getCode());
+      return NULL;
+    }
+  }
+  return NULL;
+}
+
+TopicList* MQClientAPIImpl::getTopicListFromNameServer(
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(GET_ALL_TOPIC_LIST_FROM_NAMESERVER, NULL);
+  callSignatureBeforeRequest("", request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync("", request));
+  if (pResponse != NULL) {
+    if (((*(pResponse->GetBody())).getSize() == 0) ||
+        ((*(pResponse->GetBody())).getData() != NULL)) {
+      switch (pResponse->getCode()) {
+        case SUCCESS_VALUE: {
+          const MemoryBlock* pbody = pResponse->GetBody();
+          if (pbody->getSize()) {
+            TopicList* topicList = TopicList::Decode(pbody);
+            return topicList;
+          }
+        }
+        default:
+          break;
+      }
+
+      THROW_MQEXCEPTION(MQClientException, pResponse->getRemark(),
+                        pResponse->getCode());
+    }
+  }
+  return NULL;
+}
+
+int MQClientAPIImpl::wipeWritePermOfBroker(const string& namesrvAddr,
+                                           const string& brokerName,
+                                           int timeoutMillis) {
+  return 0;
+}
+
+void MQClientAPIImpl::deleteTopicInBroker(const string& addr,
+                                          const string& topic,
+                                          int timeoutMillis) {}
+
+void MQClientAPIImpl::deleteTopicInNameServer(const string& addr,
+                                              const string& topic,
+                                              int timeoutMillis) {}
+
+void MQClientAPIImpl::deleteSubscriptionGroup(const string& addr,
+                                              const string& groupName,
+                                              int timeoutMillis) {}
+
+string MQClientAPIImpl::getKVConfigByValue(const string& projectNamespace,
+                                           const string& projectGroup,
+                                           int timeoutMillis) {
+  return "";
+}
+
+KVTable MQClientAPIImpl::getKVListByNamespace(const string& projectNamespace,
+                                              int timeoutMillis) {
+  return KVTable();
+}
+
+void MQClientAPIImpl::deleteKVConfigByValue(const string& projectNamespace,
+                                            const string& projectGroup,
+                                            int timeoutMillis) {}
+
+SendResult MQClientAPIImpl::sendMessageSync(const string& addr,
+                                            const string& brokerName,
+                                            const MQMessage& msg,
+                                            RemotingCommand& request,
+                                            int timeoutMillis) {
+  //<!block util response;
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+  if (pResponse != NULL) {
+    try {
+      LOG_DEBUG("sendMessageSync success:%s to addr:%s,brokername:%s",
+                msg.toString().c_str(), addr.c_str(), brokerName.c_str());
+      SendResult result = processSendResponse(brokerName, msg, pResponse.get());
+      return result;
+    } catch (...) {
+      LOG_ERROR("send error");
+    }
+  }
+  THROW_MQEXCEPTION(MQClientException, "response is null", -1);
+}
+
+void MQClientAPIImpl::sendMessageAsync(const string& addr,
+                                       const string& brokerName,
+                                       const MQMessage& msg,
+                                       RemotingCommand& request,
+                                       SendCallback* pSendCallback,
+                                       int64 timeoutMilliseconds) {
+  //<!delete in future;
+  AsyncCallbackWrap* cbw =
+      new SendCallbackWrap(brokerName, msg, pSendCallback, this);
+  if (m_pRemotingClient->invokeAsync(addr, request, cbw, timeoutMilliseconds) ==
+      false) {
+    LOG_ERROR("sendMessageAsync failed to addr:%s", addr.c_str());
+    if (cbw) {
+      cbw->onException();
+      deleteAndZero(cbw);
+    } else {
+      THROW_MQEXCEPTION(MQClientException, "sendMessageAsync failed", -1);
+    }
+  }
+}
+
+PullResult* MQClientAPIImpl::pullMessage(
+    const string& addr, PullMessageRequestHeader* pRequestHeader,
+    int timeoutMillis, int communicationMode, PullCallback* pullCallback,
+    void* pArg, const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(PULL_MESSAGE, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  switch (communicationMode) {
+    case ComMode_ONEWAY:
+      break;
+    case ComMode_ASYNC:
+      pullMessageAsync(addr, request, timeoutMillis, pullCallback, pArg);
+      break;
+    case ComMode_SYNC:
+      return pullMessageSync(addr, request, timeoutMillis);
+    default:
+      break;
+  }
+
+  return NULL;
+}
+
+void MQClientAPIImpl::pullMessageAsync(const string& addr,
+                                       RemotingCommand& request,
+                                       int timeoutMillis,
+                                       PullCallback* pullCallback, void* pArg) {
+  //<!delete in future;
+  AsyncCallbackWrap* cbw = new PullCallbackWarp(pullCallback, this, pArg);
+  if (m_pRemotingClient->invokeAsync(addr, request, cbw, timeoutMillis) ==
+      false) {
+    LOG_ERROR("pullMessageAsync failed of addr:%s", addr.c_str());
+    deleteAndZero(cbw);
+    THROW_MQEXCEPTION(MQClientException, "pullMessageAsync failed", -1);
+  }
+}
+
+PullResult* MQClientAPIImpl::pullMessageSync(const string& addr,
+                                             RemotingCommand& request,
+                                             int timeoutMillis) {
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+  if (pResponse != NULL) {
+    if (((*(pResponse->GetBody())).getSize() == 0) ||
+        ((*(pResponse->GetBody())).getData() != NULL)) {
+      try {
+        PullResult* pullResult =
+            processPullResponse(pResponse.get());  // pullMessage will handle
+                                                   // exception from
+                                                   // processPullResponse
+        return pullResult;
+      } catch (MQException& e) {
+        LOG_ERROR(e.what());
+        return NULL;
+      }
+    }
+  }
+  return NULL;
+}
+
+SendResult MQClientAPIImpl::processSendResponse(const string& brokerName,
+                                                const MQMessage& msg,
+                                                RemotingCommand* pResponse) {
+  SendStatus sendStatus = SEND_OK;
+  int res = 0;
+  switch (pResponse->getCode()) {
+    case FLUSH_DISK_TIMEOUT:
+      sendStatus = SEND_FLUSH_DISK_TIMEOUT;
+      break;
+    case FLUSH_SLAVE_TIMEOUT:
+      sendStatus = SEND_FLUSH_SLAVE_TIMEOUT;
+      break;
+    case SLAVE_NOT_AVAILABLE:
+      sendStatus = SEND_SLAVE_NOT_AVAILABLE;
+      break;
+    case SUCCESS_VALUE:
+      sendStatus = SEND_OK;
+      break;
+    default:
+      res = -1;
+      break;
+  }
+  if (res == 0) {
+    SendMessageResponseHeader* responseHeader =
+        (SendMessageResponseHeader*)pResponse->getCommandHeader();
+    MQMessageQueue messageQueue(msg.getTopic(), brokerName,
+                                responseHeader->queueId);
+    return SendResult(sendStatus, responseHeader->msgId, messageQueue,
+                      responseHeader->queueOffset);
+  }
+  LOG_ERROR("processSendResponse error remark:%s, error code:%d",
+            (pResponse->getRemark()).c_str(), pResponse->getCode());
+  THROW_MQEXCEPTION(MQClientException, pResponse->getRemark(),
+                    pResponse->getCode());
+}
+
+PullResult* MQClientAPIImpl::processPullResponse(RemotingCommand* pResponse) {
+  PullStatus pullStatus = NO_NEW_MSG;
+  switch (pResponse->getCode()) {
+    case SUCCESS_VALUE:
+      pullStatus = FOUND;
+      break;
+    case PULL_NOT_FOUND:
+      pullStatus = NO_NEW_MSG;
+      break;
+    case PULL_RETRY_IMMEDIATELY:
+      pullStatus = NO_MATCHED_MSG;
+      break;
+    case PULL_OFFSET_MOVED:
+      pullStatus = OFFSET_ILLEGAL;
+      break;
+    default:
+      THROW_MQEXCEPTION(MQBrokerException, pResponse->getRemark(),
+                        pResponse->getCode());
+      break;
+  }
+
+  PullMessageResponseHeader* responseHeader =
+      static_cast<PullMessageResponseHeader*>(pResponse->getCommandHeader());
+
+  if (!responseHeader) {
+    LOG_ERROR("processPullResponse:responseHeader is NULL");
+    THROW_MQEXCEPTION(MQClientException,
+                      "processPullResponse:responseHeader is NULL", -1);
+  }
+  //<!get body,delete outsite;
+  MemoryBlock bodyFromResponse =
+      *(pResponse->GetBody());  // response data judgement had been done outside
+                                // of processPullResponse
+  if (bodyFromResponse.getSize() == 0) {
+    if (pullStatus != FOUND) {
+      return new PullResultExt(pullStatus, responseHeader->nextBeginOffset,
+                               responseHeader->minOffset,
+                               responseHeader->maxOffset,
+                               (int)responseHeader->suggestWhichBrokerId);
+    } else {
+      THROW_MQEXCEPTION(MQClientException,
+                        "memoryBody size is 0, but pullStatus equals found",
+                        -1);
+    }
+  } else {
+    return new PullResultExt(
+        pullStatus, responseHeader->nextBeginOffset, responseHeader->minOffset,
+        responseHeader->maxOffset, (int)responseHeader->suggestWhichBrokerId,
+        bodyFromResponse);
+  }
+}
+
+//<!***************************************************************************
+int64 MQClientAPIImpl::getMinOffset(
+    const string& addr, const string& topic, int queueId, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  GetMinOffsetRequestHeader* pRequestHeader = new GetMinOffsetRequestHeader();
+  pRequestHeader->topic = topic;
+  pRequestHeader->queueId = queueId;
+
+  RemotingCommand request(GET_MIN_OFFSET, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        GetMinOffsetResponseHeader* responseHeader =
+            (GetMinOffsetResponseHeader*)response->getCommandHeader();
+
+        int64 offset = responseHeader->offset;
+        return offset;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+int64 MQClientAPIImpl::getMaxOffset(
+    const string& addr, const string& topic, int queueId, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  GetMaxOffsetRequestHeader* pRequestHeader = new GetMaxOffsetRequestHeader();
+  pRequestHeader->topic = topic;
+  pRequestHeader->queueId = queueId;
+
+  RemotingCommand request(GET_MAX_OFFSET, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        GetMaxOffsetResponseHeader* responseHeader =
+            (GetMaxOffsetResponseHeader*)response->getCommandHeader();
+
+        int64 offset = responseHeader->offset;
+        return offset;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+int64 MQClientAPIImpl::searchOffset(
+    const string& addr, const string& topic, int queueId, uint64_t timestamp,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  SearchOffsetRequestHeader* pRequestHeader = new SearchOffsetRequestHeader();
+  pRequestHeader->topic = topic;
+  pRequestHeader->queueId = queueId;
+  pRequestHeader->timestamp = timestamp;
+
+  RemotingCommand request(SEARCH_OFFSET_BY_TIMESTAMP, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        SearchOffsetResponseHeader* responseHeader =
+            (SearchOffsetResponseHeader*)response->getCommandHeader();
+
+        int64 offset = responseHeader->offset;
+        return offset;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+MQMessageExt* MQClientAPIImpl::viewMessage(
+    const string& addr, int64 phyoffset, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  ViewMessageRequestHeader* pRequestHeader = new ViewMessageRequestHeader();
+  pRequestHeader->offset = phyoffset;
+
+  RemotingCommand request(VIEW_MESSAGE_BY_ID, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+int64 MQClientAPIImpl::getEarliestMsgStoretime(
+    const string& addr, const string& topic, int queueId, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  GetEarliestMsgStoretimeRequestHeader* pRequestHeader =
+      new GetEarliestMsgStoretimeRequestHeader();
+  pRequestHeader->topic = topic;
+  pRequestHeader->queueId = queueId;
+
+  RemotingCommand request(GET_EARLIEST_MSG_STORETIME, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        GetEarliestMsgStoretimeResponseHeader* responseHeader =
+            (GetEarliestMsgStoretimeResponseHeader*)
+                response->getCommandHeader();
+
+        int64 timestamp = responseHeader->timestamp;
+        return timestamp;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+void MQClientAPIImpl::getConsumerIdListByGroup(
+    const string& addr, const string& consumerGroup, vector<string>& cids,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  GetConsumerListByGroupRequestHeader* pRequestHeader =
+      new GetConsumerListByGroupRequestHeader();
+  pRequestHeader->consumerGroup = consumerGroup;
+
+  RemotingCommand request(GET_CONSUMER_LIST_BY_GROUP, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (pResponse != NULL) {
+    if ((pResponse->GetBody()->getSize() == 0) ||
+        (pResponse->GetBody()->getData() != NULL)) {
+      switch (pResponse->getCode()) {
+        case SUCCESS_VALUE: {
+          const MemoryBlock* pbody = pResponse->GetBody();
+          if (pbody->getSize()) {
+            GetConsumerListByGroupResponseBody::Decode(pbody, cids);
+            return;
+          }
+        }
+        default:
+          break;
+      }
+      THROW_MQEXCEPTION(MQBrokerException, pResponse->getRemark(),
+                        pResponse->getCode());
+    }
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+int64 MQClientAPIImpl::queryConsumerOffset(
+    const string& addr, QueryConsumerOffsetRequestHeader* pRequestHeader,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(QUERY_CONSUMER_OFFSET, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        QueryConsumerOffsetResponseHeader* responseHeader =
+            (QueryConsumerOffsetResponseHeader*)response->getCommandHeader();
+        int64 consumerOffset = responseHeader->offset;
+        return consumerOffset;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+  return -1;
+}
+
+void MQClientAPIImpl::updateConsumerOffset(
+    const string& addr, UpdateConsumerOffsetRequestHeader* pRequestHeader,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(UPDATE_CONSUMER_OFFSET, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        return;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+void MQClientAPIImpl::updateConsumerOffsetOneway(
+    const string& addr, UpdateConsumerOffsetRequestHeader* pRequestHeader,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(UPDATE_CONSUMER_OFFSET, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  m_pRemotingClient->invokeOneway(addr, request);
+}
+
+void MQClientAPIImpl::consumerSendMessageBack(
+    MQMessageExt& msg, const string& consumerGroup, int delayLevel,
+    int timeoutMillis, const SessionCredentials& sessionCredentials) {
+  ConsumerSendMsgBackRequestHeader* pRequestHeader =
+      new ConsumerSendMsgBackRequestHeader();
+  pRequestHeader->group = consumerGroup;
+  pRequestHeader->offset = msg.getCommitLogOffset();
+  pRequestHeader->delayLevel = delayLevel;
+
+  string addr = socketAddress2IPPort(msg.getStoreHost());
+  RemotingCommand request(CONSUMER_SEND_MSG_BACK, pRequestHeader);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> response(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (response) {
+    switch (response->getCode()) {
+      case SUCCESS_VALUE: {
+        return;
+      }
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, response->getRemark(),
+                      response->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+void MQClientAPIImpl::lockBatchMQ(
+    const string& addr, LockBatchRequestBody* requestBody,
+    vector<MQMessageQueue>& mqs, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(LOCK_BATCH_MQ, NULL);
+  string body;
+  requestBody->Encode(body);
+  request.SetBody(body.data(), body.length());
+  request.setMsgBody(body);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (pResponse != NULL) {
+    if (((*(pResponse->GetBody())).getSize() == 0) ||
+        ((*(pResponse->GetBody())).getData() != NULL)) {
+      switch (pResponse->getCode()) {
+        case SUCCESS_VALUE: {
+          const MemoryBlock* pbody = pResponse->GetBody();
+          if (pbody->getSize()) {
+            LockBatchResponseBody::Decode(pbody, mqs);
+          }
+          return;
+        } break;
+        default:
+          break;
+      }
+      THROW_MQEXCEPTION(MQBrokerException, pResponse->getRemark(),
+                        pResponse->getCode());
+    }
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+void MQClientAPIImpl::unlockBatchMQ(
+    const string& addr, UnlockBatchRequestBody* requestBody, int timeoutMillis,
+    const SessionCredentials& sessionCredentials) {
+  RemotingCommand request(UNLOCK_BATCH_MQ, NULL);
+  string body;
+  requestBody->Encode(body);
+  request.SetBody(body.data(), body.length());
+  request.setMsgBody(body);
+  callSignatureBeforeRequest(addr, request, sessionCredentials);
+  request.Encode();
+
+  unique_ptr<RemotingCommand> pResponse(
+      m_pRemotingClient->invokeSync(addr, request, timeoutMillis));
+
+  if (pResponse != NULL) {
+    switch (pResponse->getCode()) {
+      case SUCCESS_VALUE: {
+        return;
+      } break;
+      default:
+        break;
+    }
+    THROW_MQEXCEPTION(MQBrokerException, pResponse->getRemark(),
+                      pResponse->getCode());
+  }
+  THROW_MQEXCEPTION(MQBrokerException, "response is null", -1);
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/MQClientAPIImpl.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/MQClientAPIImpl.h b/rocketmq-cpp/src/MQClientAPIImpl.h
new file mode 100644
index 0000000..31e61a0
--- /dev/null
+++ b/rocketmq-cpp/src/MQClientAPIImpl.h
@@ -0,0 +1,189 @@
+/*
+ * 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 __MQCLIENTAPIIMPL_H__
+#define __MQCLIENTAPIIMPL_H__
+#include "AsyncCallback.h"
+#include "ClientRPCHook.h"
+#include "ClientRemotingProcessor.h"
+#include "CommandHeader.h"
+#include "HeartbeatData.h"
+#include "KVTable.h"
+#include "LockBatchBody.h"
+#include "MQClientException.h"
+#include "MQMessageExt.h"
+#include "MQProtos.h"
+#include "SendResult.h"
+#include "SocketUtil.h"
+#include "TcpRemotingClient.h"
+#include "TopAddressing.h"
+#include "TopicConfig.h"
+#include "TopicList.h"
+#include "TopicRouteData.h"
+#include "UtilAll.h"
+#include "VirtualEnvUtil.h"
+
+namespace rocketmq {
+//<!wrap all API to net ;
+//<!************************************************************************
+class MQClientAPIImpl {
+ public:
+  MQClientAPIImpl(const string& mqClientId, ClientRemotingProcessor* clientRemotingProcessor,
+                  int pullThreadNum, uint64_t tcpConnectTimeout,
+                  uint64_t tcpTransportTryLockTimeout, string unitName);
+  virtual ~MQClientAPIImpl();
+  void stopAllTcpTransportThread();
+  bool writeDataToFile(string filename, string data, bool isSync);
+  string fetchNameServerAddr(const string& NSDomain);
+  void updateNameServerAddr(const string& addrs);
+
+  void callSignatureBeforeRequest(
+      const string& addr, RemotingCommand& request,
+      const SessionCredentials& session_credentials);
+  void createTopic(const string& addr, const string& defaultTopic,
+                   TopicConfig topicConfig,
+                   const SessionCredentials& sessionCredentials);
+
+  SendResult sendMessage(const string& addr, const string& brokerName,
+                         const MQMessage& msg,
+                         SendMessageRequestHeader* pRequestHeader,
+                         int timeoutMillis, int communicationMode,
+                         SendCallback* pSendCallback,
+                         const SessionCredentials& sessionCredentials);
+
+  PullResult* pullMessage(const string& addr,
+                          PullMessageRequestHeader* pRequestHeader,
+                          int timeoutMillis, int communicationMode,
+                          PullCallback* pullCallback, void* pArg,
+                          const SessionCredentials& sessionCredentials);
+
+  void sendHearbeat(const string& addr, HeartbeatData* pHeartbeatData,
+                    const SessionCredentials& sessionCredentials);
+
+  void unregisterClient(const string& addr, const string& clientID,
+                        const string& producerGroup,
+                        const string& consumerGroup,
+                        const SessionCredentials& sessionCredentials);
+
+  TopicRouteData* getTopicRouteInfoFromNameServer(
+      const string& topic, int timeoutMillis,
+      const SessionCredentials& sessionCredentials);
+
+  TopicList* getTopicListFromNameServer(
+      const SessionCredentials& sessionCredentials);
+
+  int wipeWritePermOfBroker(const string& namesrvAddr, const string& brokerName,
+                            int timeoutMillis);
+
+  void deleteTopicInBroker(const string& addr, const string& topic,
+                           int timeoutMillis);
+
+  void deleteTopicInNameServer(const string& addr, const string& topic,
+                               int timeoutMillis);
+
+  void deleteSubscriptionGroup(const string& addr, const string& groupName,
+                               int timeoutMillis);
+
+  string getKVConfigByValue(const string& projectNamespace,
+                            const string& projectGroup, int timeoutMillis);
+
+  KVTable getKVListByNamespace(const string& projectNamespace,
+                               int timeoutMillis);
+
+  void deleteKVConfigByValue(const string& projectNamespace,
+                             const string& projectGroup, int timeoutMillis);
+
+  SendResult processSendResponse(const string& brokerName, const MQMessage& msg,
+                                 RemotingCommand* pResponse);
+
+  PullResult* processPullResponse(RemotingCommand* pResponse);
+
+  int64 getMinOffset(const string& addr, const string& topic, int queueId,
+                     int timeoutMillis,
+                     const SessionCredentials& sessionCredentials);
+
+  int64 getMaxOffset(const string& addr, const string& topic, int queueId,
+                     int timeoutMillis,
+                     const SessionCredentials& sessionCredentials);
+
+  int64 searchOffset(const string& addr, const string& topic, int queueId,
+                     uint64_t timestamp, int timeoutMillis,
+                     const SessionCredentials& sessionCredentials);
+
+  MQMessageExt* viewMessage(const string& addr, int64 phyoffset,
+                            int timeoutMillis,
+                            const SessionCredentials& sessionCredentials);
+
+  int64 getEarliestMsgStoretime(const string& addr, const string& topic,
+                                int queueId, int timeoutMillis,
+                                const SessionCredentials& sessionCredentials);
+
+  void getConsumerIdListByGroup(const string& addr, const string& consumerGroup,
+                                vector<string>& cids, int timeoutMillis,
+                                const SessionCredentials& sessionCredentials);
+
+  int64 queryConsumerOffset(const string& addr,
+                            QueryConsumerOffsetRequestHeader* pRequestHeader,
+                            int timeoutMillis,
+                            const SessionCredentials& sessionCredentials);
+
+  void updateConsumerOffset(const string& addr,
+                            UpdateConsumerOffsetRequestHeader* pRequestHeader,
+                            int timeoutMillis,
+                            const SessionCredentials& sessionCredentials);
+
+  void updateConsumerOffsetOneway(
+      const string& addr, UpdateConsumerOffsetRequestHeader* pRequestHeader,
+      int timeoutMillis, const SessionCredentials& sessionCredentials);
+
+  void consumerSendMessageBack(MQMessageExt& msg, const string& consumerGroup,
+                               int delayLevel, int timeoutMillis,
+                               const SessionCredentials& sessionCredentials);
+
+  void lockBatchMQ(const string& addr, LockBatchRequestBody* requestBody,
+                   vector<MQMessageQueue>& mqs, int timeoutMillis,
+                   const SessionCredentials& sessionCredentials);
+
+  void unlockBatchMQ(const string& addr, UnlockBatchRequestBody* requestBody,
+                     int timeoutMillis,
+                     const SessionCredentials& sessionCredentials);
+
+ private:
+  SendResult sendMessageSync(const string& addr, const string& brokerName,
+                             const MQMessage& msg, RemotingCommand& request,
+                             int timeoutMillis);
+
+  void sendMessageAsync(const string& addr, const string& brokerName,
+                        const MQMessage& msg, RemotingCommand& request,
+                        SendCallback* pSendCallback, int64 timeoutMilliseconds);
+
+  PullResult* pullMessageSync(const string& addr, RemotingCommand& request,
+                              int timeoutMillis);
+
+  void pullMessageAsync(const string& addr, RemotingCommand& request,
+                        int timeoutMillis, PullCallback* pullCallback,
+                        void* pArg);
+
+ private:
+  unique_ptr<TcpRemotingClient> m_pRemotingClient;
+  unique_ptr<TopAddressing> m_topAddressing;
+  string m_nameSrvAddr;
+  bool m_firstFetchNameSrv;
+  string m_mqClientId;
+};
+}  //<!end namespace;
+//<!***************************************************************************
+#endif


[04/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/producer/DefaultMQProducer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/producer/DefaultMQProducer.cpp b/rocketmq-cpp/src/producer/DefaultMQProducer.cpp
new file mode 100755
index 0000000..9c53930
--- /dev/null
+++ b/rocketmq-cpp/src/producer/DefaultMQProducer.cpp
@@ -0,0 +1,494 @@
+/*
+ * 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 <assert.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 "MessageSysFlag.h"
+#include "TopicPublishInfo.h"
+#include "Validators.h"
+
+namespace rocketmq {
+
+//<!************************************************************************
+DefaultMQProducer::DefaultMQProducer(const string& groupname)
+    : m_sendMsgTimeout(3000),
+      m_compressMsgBodyOverHowmuch(4 * 1024),
+      m_maxMessageSize(1024 * 128),
+      m_retryAnotherBrokerWhenNotStoreOK(false),
+      m_compressLevel(5),
+      m_retryTimes(5) {
+  //<!set default group name;
+  string gname = groupname.empty() ? DEFAULT_PRODUCER_GROUP : groupname;
+  setGroupName(gname);
+}
+
+DefaultMQProducer::~DefaultMQProducer() {}
+
+void DefaultMQProducer::start() {
+  /* Ignore the SIGPIPE */
+  struct sigaction sa;
+  sa.sa_handler = SIG_IGN;
+  sa.sa_flags = 0;
+  sigaction(SIGPIPE, &sa, 0);
+
+  switch (m_serviceState) {
+    case CREATE_JUST: {
+      m_serviceState = START_FAILED;
+      MQClient::start();
+      LOG_INFO("DefaultMQProducer:%s start", m_GroupName.c_str());
+
+      bool registerOK = getFactory()->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 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;
+  }
+}
+
+SendResult DefaultMQProducer::send(MQMessage& msg, bool bSelectActiveBroker) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  try {
+    return sendDefaultImpl(msg, ComMode_SYNC, NULL, bSelectActiveBroker);
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    throw e;
+  }
+  return SendResult();
+}
+
+void DefaultMQProducer::send(MQMessage& msg, SendCallback* pSendCallback,
+                             bool bSelectActiveBroker) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  try {
+    sendDefaultImpl(msg, ComMode_ASYNC, pSendCallback, bSelectActiveBroker);
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    throw e;
+  }
+}
+
+SendResult DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  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 DefaultMQProducer::send(MQMessage& msg, const MQMessageQueue& mq,
+                             SendCallback* pSendCallback) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  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::sendOneway(MQMessage& msg, bool bSelectActiveBroker) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  try {
+    sendDefaultImpl(msg, ComMode_ONEWAY, NULL, bSelectActiveBroker);
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    throw e;
+  }
+}
+
+void DefaultMQProducer::sendOneway(MQMessage& msg, const MQMessageQueue& mq) {
+  Validators::checkMessage(msg, getMaxMessageSize());
+  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 {
+    return sendSelectImpl(msg, pSelector, arg, ComMode_SYNC, NULL);
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    throw e;
+  }
+  return SendResult();
+}
+
+SendResult DefaultMQProducer::send(MQMessage& msg,
+                                   MessageQueueSelector* pSelector, void* arg,
+                                   int autoRetryTimes, bool bActiveBroker) {
+  try {
+    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 {
+    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 {
+    sendSelectImpl(msg, pSelector, arg, ComMode_ONEWAY, NULL);
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    throw e;
+  }
+}
+
+int DefaultMQProducer::getSendMsgTimeout() const { return m_sendMsgTimeout; }
+
+void DefaultMQProducer::setSendMsgTimeout(int sendMsgTimeout) {
+  m_sendMsgTimeout = sendMsgTimeout;
+}
+
+int DefaultMQProducer::getCompressMsgBodyOverHowmuch() const {
+  return m_compressMsgBodyOverHowmuch;
+}
+
+void DefaultMQProducer::setCompressMsgBodyOverHowmuch(
+    int compressMsgBodyOverHowmuch) {
+  m_compressMsgBodyOverHowmuch = compressMsgBodyOverHowmuch;
+}
+
+int DefaultMQProducer::getMaxMessageSize() const { return m_maxMessageSize; }
+
+void DefaultMQProducer::setMaxMessageSize(int maxMessageSize) {
+  m_maxMessageSize = maxMessageSize;
+}
+
+int DefaultMQProducer::getCompressLevel() const { return m_compressLevel; }
+
+void DefaultMQProducer::setCompressLevel(int compressLevel) {
+  assert(compressLevel >= 0 && compressLevel <= 9 || compressLevel == -1);
+
+  m_compressLevel = compressLevel;
+}
+
+//<!************************************************************************
+SendResult DefaultMQProducer::sendDefaultImpl(MQMessage& msg,
+                                              int communicationMode,
+                                              SendCallback* pSendCallback,
+                                              bool bActiveMQ) {
+  MQMessageQueue lastmq;
+  int mq_index = 0;
+  for (int times = 1; times <= m_retryTimes; times++) {
+    boost::weak_ptr<TopicPublishInfo> weak_topicPublishInfo(
+        getFactory()->tryToFindTopicPublishInfo(msg.getTopic(),
+                                                getSessionCredentials()));
+    boost::shared_ptr<TopicPublishInfo> 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 brokerName:%s", mq.getBrokerName().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,brokerName:%s", times,
+                  mq.getBrokerName().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);
+}
+
+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 {
+      LOG_DEBUG("produce before:%s to %s", msg.toString().c_str(),
+                mq.toString().c_str());
+      int sysFlag = 0;
+      if (tryToCompressMessage(msg)) {
+        sysFlag |= MessageSysFlag::CompressedFlag;
+      }
+
+      string tranMsg =
+          msg.getProperty(MQMessage::PROPERTY_TRANSACTION_PREPARED);
+      if (!tranMsg.empty() && tranMsg == "true") {
+        sysFlag |= MessageSysFlag::TransactionPreparedType;
+      }
+
+      SendMessageRequestHeader* requestHeader = new SendMessageRequestHeader();
+      requestHeader->producerGroup = getGroupName();
+      requestHeader->topic = (msg.getTopic());
+      requestHeader->defaultTopic = DEFAULT_TOPIC;
+      requestHeader->defaultTopicQueueNums = 4;
+      requestHeader->queueId = (mq.getQueueId());
+      requestHeader->sysFlag = (sysFlag);
+      requestHeader->bornTimestamp = UtilAll::currentTimeMillis();
+      requestHeader->flag = (msg.getFlag());
+      requestHeader->properties =
+          (MQDecoder::messageProperties2String(msg.getProperties()));
+
+      return getFactory()->getMQClientAPIImpl()->sendMessage(
+          brokerAddr, mq.getBrokerName(), msg, requestHeader,
+          getSendMsgTimeout(), 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<TopicPublishInfo> weak_topicPublishInfo(
+      getFactory()->tryToFindTopicPublishInfo(msg.getTopic(),
+                                              getSessionCredentials()));
+  boost::shared_ptr<TopicPublishInfo> 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<TopicPublishInfo> weak_topicPublishInfo(
+        getFactory()->tryToFindTopicPublishInfo(msg.getTopic(),
+                                                getSessionCredentials()));
+    boost::shared_ptr<TopicPublishInfo> 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);
+        vector<MQMessageQueue> 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) {
+  string body = msg.getBody();
+  if ((int)body.length() >= getCompressMsgBodyOverHowmuch()) {
+    string outBody;
+    if (UtilAll::deflate(body, outBody, getCompressLevel())) {
+      msg.setBody(outBody);
+      return true;
+    }
+  }
+
+  return false;
+}
+int DefaultMQProducer::getRetryTimes() const { return m_retryTimes; }
+void DefaultMQProducer::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;
+}
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/producer/SendResult.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/producer/SendResult.cpp b/rocketmq-cpp/src/producer/SendResult.cpp
new file mode 100755
index 0000000..7fd844e
--- /dev/null
+++ b/rocketmq-cpp/src/producer/SendResult.cpp
@@ -0,0 +1,60 @@
+/*
+ * 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 "SendResult.h"
+#include "UtilAll.h"
+#include "VirtualEnvUtil.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+SendResult::SendResult() : m_sendStatus(SEND_OK), m_queueOffset(0) {}
+
+SendResult::SendResult(const SendStatus& sendStatus, const string& msgId,
+                       const MQMessageQueue& messageQueue, int64 queueOffset)
+    : m_sendStatus(sendStatus),
+      m_msgId(msgId),
+      m_messageQueue(messageQueue),
+      m_queueOffset(queueOffset) {}
+
+SendResult::SendResult(const SendResult& other) {
+  m_sendStatus = other.m_sendStatus;
+  m_msgId = other.m_msgId;
+  m_messageQueue = other.m_messageQueue;
+  m_queueOffset = other.m_queueOffset;
+}
+
+SendResult& SendResult::operator=(const SendResult& other) {
+  if (this != &other) {
+    m_sendStatus = other.m_sendStatus;
+    m_msgId = other.m_msgId;
+    m_messageQueue = other.m_messageQueue;
+    m_queueOffset = other.m_queueOffset;
+  }
+  return *this;
+}
+
+SendResult::~SendResult() {}
+
+const string& SendResult::getMsgId() const { return m_msgId; }
+
+SendStatus SendResult::getSendStatus() const { return m_sendStatus; }
+
+MQMessageQueue SendResult::getMessageQueue() const { return m_messageQueue; }
+
+int64 SendResult::getQueueOffset() const { return m_queueOffset; }
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/producer/TopicPublishInfo.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/producer/TopicPublishInfo.h b/rocketmq-cpp/src/producer/TopicPublishInfo.h
new file mode 100755
index 0000000..726b231
--- /dev/null
+++ b/rocketmq-cpp/src/producer/TopicPublishInfo.h
@@ -0,0 +1,277 @@
+/*
+ * 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 __TOPICPUBLISHINFO_H__
+#define __TOPICPUBLISHINFO_H__
+
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/atomic.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/scoped_ptr.hpp>
+#include <boost/thread/thread.hpp>
+#include "Logging.h"
+#include "MQMessageQueue.h"
+
+namespace rocketmq {
+//<!************************************************************************/
+class TopicPublishInfo {
+ public:
+  TopicPublishInfo() : m_sendWhichQueue(0) {
+    m_async_service_thread.reset(new boost::thread(
+        boost::bind(&TopicPublishInfo::boost_asio_work, this)));
+  }
+
+  void boost_asio_work() {
+    boost::asio::io_service::work work(m_async_ioService);  // avoid async io
+                                                            // service stops
+                                                            // after first timer
+                                                            // timeout callback
+    boost::system::error_code e;
+    boost::asio::deadline_timer t(m_async_ioService,
+                                  boost::posix_time::seconds(60));
+    t.async_wait(boost::bind(
+        &TopicPublishInfo::op_resumeNonServiceMessageQueueList, this, e, &t));
+    boost::system::error_code ec;
+    m_async_ioService.run(ec);
+  }
+
+  virtual ~TopicPublishInfo() {
+    m_async_ioService.stop();
+    m_async_service_thread->interrupt();
+    m_async_service_thread->join();
+      
+    m_nonSerivceQueues.clear();
+    m_onSerivceQueues.clear();
+    m_brokerTimerMap.clear();
+    m_queues.clear();
+  }
+
+  bool ok() {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+    return !m_queues.empty();
+  }
+
+  void updateMessageQueueList(const MQMessageQueue& mq) {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+    m_queues.push_back(mq);
+    string key = mq.getBrokerName() + UtilAll::to_string(mq.getQueueId());
+    m_onSerivceQueues[key] = mq;
+    if (m_nonSerivceQueues.find(key) != m_nonSerivceQueues.end()) {
+      m_nonSerivceQueues.erase(key);  // if topicPublishInfo changed, erase this
+                                      // mq from m_nonSerivceQueues to avoid 2
+                                      // copies both in m_onSerivceQueues and
+                                      // m_nonSerivceQueues
+    }
+  }
+
+  void op_resumeNonServiceMessageQueueList(boost::system::error_code& ec,
+                                           boost::asio::deadline_timer* t) {
+    resumeNonServiceMessageQueueList();
+    boost::system::error_code e;
+    t->expires_at(t->expires_at() + boost::posix_time::seconds(60), e);
+    t->async_wait(boost::bind(
+        &TopicPublishInfo::op_resumeNonServiceMessageQueueList, this, e, t));
+  }
+
+  void resumeNonServiceMessageQueueList() {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+    for (map<MQMessageQueue, int64>::iterator it = m_brokerTimerMap.begin();
+         it != m_brokerTimerMap.end(); ++it) {
+      if (UtilAll::currentTimeMillis() - it->second >= 1000 * 60 * 5) {
+        string key = it->first.getBrokerName() +
+                     UtilAll::to_string(it->first.getQueueId());
+        if (m_nonSerivceQueues.find(key) != m_nonSerivceQueues.end()) {
+          m_nonSerivceQueues.erase(key);
+        }
+        m_onSerivceQueues[key] = it->first;
+      }
+    }
+  }
+
+  void updateNonServiceMessageQueue(const MQMessageQueue& mq,
+                                    int timeoutMilliseconds) {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+
+    string key = mq.getBrokerName() + UtilAll::to_string(mq.getQueueId());
+    if (m_nonSerivceQueues.find(key) != m_nonSerivceQueues.end()) {
+      return;
+    }
+    LOG_INFO("updateNonServiceMessageQueue of mq:%s", mq.toString().c_str());
+    m_brokerTimerMap[mq] = UtilAll::currentTimeMillis();
+    m_nonSerivceQueues[key] = mq;
+    if (m_onSerivceQueues.find(key) != m_onSerivceQueues.end()) {
+      m_onSerivceQueues.erase(key);
+    }
+  }
+
+  vector<MQMessageQueue>& getMessageQueueList() {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+    return m_queues;
+  }
+
+  int getWhichQueue() {
+    return m_sendWhichQueue.load(boost::memory_order_acquire);
+  }
+
+  MQMessageQueue selectOneMessageQueue(const MQMessageQueue& lastmq,
+                                       int& mq_index) {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+
+    if (m_queues.size() > 0) {
+      LOG_DEBUG("selectOneMessageQueue Enter, queue size:%zu", m_queues.size());
+      unsigned int pos = 0;
+      if (mq_index >= 0) {
+        pos = mq_index % m_queues.size();
+      } else {
+        LOG_ERROR("mq_index is negative");
+        return MQMessageQueue();
+      }
+      if (!lastmq.getBrokerName().empty()) {
+        for (size_t i = 0; i < m_queues.size(); i++) {
+          if (m_sendWhichQueue.load(boost::memory_order_acquire) ==
+              numeric_limits<int>::max()) {
+            m_sendWhichQueue.store(0, boost::memory_order_release);
+          }
+
+          if (pos >= m_queues.size()) pos = pos % m_queues.size();
+
+          ++m_sendWhichQueue;
+          MQMessageQueue mq = m_queues.at(pos);
+          LOG_DEBUG("lastmq broker not empty, m_sendWhichQueue:%d, pos:%d",
+                    m_sendWhichQueue.load(boost::memory_order_acquire), pos);
+          if (mq.getBrokerName().compare(lastmq.getBrokerName()) != 0) {
+            mq_index = pos;
+            return mq;
+          }
+          ++pos;
+        }
+        LOG_ERROR("could not find property mq");
+        return MQMessageQueue();
+      } else {
+        if (m_sendWhichQueue.load(boost::memory_order_acquire) ==
+            numeric_limits<int>::max()) {
+          m_sendWhichQueue.store(0, boost::memory_order_release);
+        }
+
+        ++m_sendWhichQueue;
+        LOG_DEBUG("lastmq broker empty, m_sendWhichQueue:%d, pos:%d",
+                  m_sendWhichQueue.load(boost::memory_order_acquire), pos);
+        mq_index = pos;
+        return m_queues.at(pos);
+      }
+    } else {
+      LOG_ERROR("m_queues empty");
+      return MQMessageQueue();
+    }
+  }
+
+  MQMessageQueue selectOneActiveMessageQueue(const MQMessageQueue& lastmq,
+                                             int& mq_index) {
+    boost::lock_guard<boost::mutex> lock(m_queuelock);
+
+    if (m_queues.size() > 0) {
+      unsigned int pos = 0;
+      if (mq_index >= 0) {
+        pos = mq_index % m_queues.size();
+      } else {
+        LOG_ERROR("mq_index is negative");
+        return MQMessageQueue();
+      }
+      if (!lastmq.getBrokerName().empty()) {
+        for (size_t i = 0; i < m_queues.size(); i++) {
+          if (m_sendWhichQueue.load(boost::memory_order_acquire) ==
+              numeric_limits<int>::max()) {
+            m_sendWhichQueue.store(0, boost::memory_order_release);
+          }
+
+          if (pos >= m_queues.size()) pos = pos % m_queues.size();
+
+          ++m_sendWhichQueue;
+          MQMessageQueue mq = m_queues.at(pos);
+          string key = mq.getBrokerName() + UtilAll::to_string(mq.getQueueId());
+          if ((mq.getBrokerName().compare(lastmq.getBrokerName()) != 0) &&
+              (m_onSerivceQueues.find(key) != m_onSerivceQueues.end())) {
+            mq_index = pos;
+            return mq;
+          }
+          ++pos;
+        }
+
+        for (MQMAP::iterator it = m_nonSerivceQueues.begin();
+             it != m_nonSerivceQueues.end();
+             ++it) {  // if no MQMessageQueue(except lastmq) in
+                      // m_onSerivceQueues, search m_nonSerivceQueues
+          if (it->second.getBrokerName().compare(lastmq.getBrokerName()) != 0)
+            return it->second;
+        }
+        LOG_ERROR("can not find property mq");
+        return MQMessageQueue();
+      } else {
+        for (size_t i = 0; i < m_queues.size(); i++) {
+          if (m_sendWhichQueue.load(boost::memory_order_acquire) ==
+              numeric_limits<int>::max()) {
+            m_sendWhichQueue.store(0, boost::memory_order_release);
+          }
+          if (pos >= m_queues.size()) pos = pos % m_queues.size();
+
+          ++m_sendWhichQueue;
+          LOG_DEBUG("lastmq broker empty, m_sendWhichQueue:%d, pos:%d",
+                    m_sendWhichQueue.load(boost::memory_order_acquire), pos);
+          mq_index = pos;
+          MQMessageQueue mq = m_queues.at(pos);
+          string key = mq.getBrokerName() + UtilAll::to_string(mq.getQueueId());
+          if (m_onSerivceQueues.find(key) != m_onSerivceQueues.end()) {
+            return mq;
+          } else {
+            ++pos;
+          }
+        }
+
+        for (MQMAP::iterator it = m_nonSerivceQueues.begin();
+             it != m_nonSerivceQueues.end();
+             ++it) {  // if no MQMessageQueue(except lastmq) in
+                      // m_onSerivceQueues, search m_nonSerivceQueues
+          if (it->second.getBrokerName().compare(lastmq.getBrokerName()) != 0)
+            return it->second;
+        }
+        LOG_ERROR("can not find property mq");
+        return MQMessageQueue();
+      }
+    } else {
+      LOG_ERROR("m_queues empty");
+      return MQMessageQueue();
+    }
+  }
+
+ private:
+  boost::mutex m_queuelock;
+  typedef vector<MQMessageQueue> QueuesVec;
+  QueuesVec m_queues;
+  typedef map<string, MQMessageQueue> MQMAP;
+  MQMAP m_onSerivceQueues;
+  MQMAP m_nonSerivceQueues;
+  boost::atomic<int> m_sendWhichQueue;
+  map<MQMessageQueue, int64> m_brokerTimerMap;
+  boost::asio::io_service m_async_ioService;
+  boost::scoped_ptr<boost::thread> m_async_service_thread;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/CommandHeader.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/CommandHeader.cpp b/rocketmq-cpp/src/protocol/CommandHeader.cpp
new file mode 100644
index 0000000..366ac2e
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/CommandHeader.cpp
@@ -0,0 +1,592 @@
+/*
+ * 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 "CommandHeader.h"
+#include <cstdlib>
+#include <sstream>
+#include "Logging.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!************************************************************************
+void GetRouteInfoRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+}
+
+void GetRouteInfoRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+}
+//<!***************************************************************************
+void UnregisterClientRequestHeader::Encode(Json::Value& outData) {
+  outData["clientID"] = clientID;
+  outData["producerGroup"] = producerGroup;
+  outData["consumerGroup"] = consumerGroup;
+}
+
+void UnregisterClientRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("clientID", clientID));
+  requestMap.insert(pair<string, string>("producerGroup", producerGroup));
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+}
+//<!************************************************************************
+void CreateTopicRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+  outData["defaultTopic"] = defaultTopic;
+  outData["readQueueNums"] = readQueueNums;
+  outData["writeQueueNums"] = writeQueueNums;
+  outData["perm"] = perm;
+  outData["topicFilterType"] = topicFilterType;
+}
+void CreateTopicRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(pair<string, string>("defaultTopic", defaultTopic));
+  requestMap.insert(
+      pair<string, string>("readQueueNums", UtilAll::to_string(readQueueNums)));
+  requestMap.insert(pair<string, string>("writeQueueNums",
+                                         UtilAll::to_string(writeQueueNums)));
+  requestMap.insert(pair<string, string>("perm", UtilAll::to_string(perm)));
+  requestMap.insert(pair<string, string>("topicFilterType", topicFilterType));
+}
+
+//<!************************************************************************
+void SendMessageRequestHeader::Encode(Json::Value& outData) {
+  outData["producerGroup"] = producerGroup;
+  outData["topic"] = topic;
+  outData["defaultTopic"] = defaultTopic;
+  outData["defaultTopicQueueNums"] = defaultTopicQueueNums;
+  outData["queueId"] = queueId;
+  outData["sysFlag"] = sysFlag;
+  outData["bornTimestamp"] = UtilAll::to_string(bornTimestamp);
+  outData["flag"] = flag;
+  outData["properties"] = properties;
+#ifdef ONS
+  outData["reconsumeTimes"] = UtilAll::to_string(reconsumeTimes);
+  outData["unitMode"] = UtilAll::to_string(unitMode);
+#endif
+}
+
+int SendMessageRequestHeader::getReconsumeTimes() { return reconsumeTimes; }
+
+void SendMessageRequestHeader::setReconsumeTimes(int input_reconsumeTimes) {
+  reconsumeTimes = input_reconsumeTimes;
+}
+
+void SendMessageRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  LOG_DEBUG(
+      "SendMessageRequestHeader producerGroup is:%s,topic is:%s, defaulttopic "
+      "is:%s, properties is:%s,UtilAll::to_string( defaultTopicQueueNums) "
+      "is:%s,UtilAll::to_string( queueId):%s, UtilAll::to_string( sysFlag) "
+      "is:%s, UtilAll::to_string( bornTimestamp) is:%s,UtilAll::to_string( "
+      "flag) is:%s",
+      producerGroup.c_str(), topic.c_str(), defaultTopic.c_str(),
+      properties.c_str(), UtilAll::to_string(defaultTopicQueueNums).c_str(),
+      UtilAll::to_string(queueId).c_str(), UtilAll::to_string(sysFlag).c_str(),
+      UtilAll::to_string(bornTimestamp).c_str(),
+      UtilAll::to_string(flag).c_str());
+
+  requestMap.insert(pair<string, string>("producerGroup", producerGroup));
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(pair<string, string>("defaultTopic", defaultTopic));
+  requestMap.insert(pair<string, string>(
+      "defaultTopicQueueNums", UtilAll::to_string(defaultTopicQueueNums)));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+  requestMap.insert(
+      pair<string, string>("sysFlag", UtilAll::to_string(sysFlag)));
+  requestMap.insert(
+      pair<string, string>("bornTimestamp", UtilAll::to_string(bornTimestamp)));
+  requestMap.insert(pair<string, string>("flag", UtilAll::to_string(flag)));
+  requestMap.insert(pair<string, string>("properties", properties));
+#ifdef ONS
+  requestMap.insert(pair<string, string>("reconsumeTimes",
+                                         UtilAll::to_string(reconsumeTimes)));
+  requestMap.insert(
+      pair<string, string>("unitMode", UtilAll::to_string(unitMode)));
+#endif
+}
+
+//<!************************************************************************
+CommandHeader* SendMessageResponseHeader::Decode(Json::Value& ext) {
+  SendMessageResponseHeader* h = new SendMessageResponseHeader();
+
+  Json::Value& tempValue = ext["msgId"];
+  if (tempValue.isString()) {
+    h->msgId = tempValue.asString();
+  }
+
+  tempValue = ext["queueId"];
+  if (tempValue.isString()) {
+    h->queueId = atoi(tempValue.asCString());
+  }
+
+  tempValue = ext["queueOffset"];
+  if (tempValue.isString()) {
+    h->queueOffset = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void SendMessageResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("msgId", msgId));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+  requestMap.insert(
+      pair<string, string>("queueOffset", UtilAll::to_string(queueOffset)));
+}
+//<!************************************************************************
+void PullMessageRequestHeader::Encode(Json::Value& outData) {
+  outData["consumerGroup"] = consumerGroup;
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+  outData["queueOffset"] = UtilAll::to_string(queueOffset);
+  ;
+  outData["maxMsgNums"] = maxMsgNums;
+  outData["sysFlag"] = sysFlag;
+  outData["commitOffset"] = UtilAll::to_string(commitOffset);
+  ;
+  outData["subVersion"] = UtilAll::to_string(subVersion);
+  ;
+  outData["suspendTimeoutMillis"] = UtilAll::to_string(suspendTimeoutMillis);
+  ;
+  outData["subscription"] = subscription;
+}
+
+void PullMessageRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+  requestMap.insert(
+      pair<string, string>("queueOffset", UtilAll::to_string(queueOffset)));
+  requestMap.insert(
+      pair<string, string>("maxMsgNums", UtilAll::to_string(maxMsgNums)));
+  requestMap.insert(
+      pair<string, string>("sysFlag", UtilAll::to_string(sysFlag)));
+  requestMap.insert(
+      pair<string, string>("commitOffset", UtilAll::to_string(commitOffset)));
+  requestMap.insert(
+      pair<string, string>("subVersion", UtilAll::to_string(subVersion)));
+  requestMap.insert(pair<string, string>(
+      "suspendTimeoutMillis", UtilAll::to_string(suspendTimeoutMillis)));
+  requestMap.insert(pair<string, string>("subscription", subscription));
+}
+//<!************************************************************************
+CommandHeader* PullMessageResponseHeader::Decode(Json::Value& ext) {
+  PullMessageResponseHeader* h = new PullMessageResponseHeader();
+
+  Json::Value& tempValue = ext["suggestWhichBrokerId"];
+  if (tempValue.isString()) {
+    h->suggestWhichBrokerId = UtilAll::str2ll(tempValue.asCString());
+  }
+
+  tempValue = ext["nextBeginOffset"];
+  if (tempValue.isString()) {
+    h->nextBeginOffset = UtilAll::str2ll(tempValue.asCString());
+  }
+
+  tempValue = ext["minOffset"];
+  if (tempValue.isString()) {
+    h->minOffset = UtilAll::str2ll(tempValue.asCString());
+  }
+
+  tempValue = ext["maxOffset"];
+  if (tempValue.isString()) {
+    h->maxOffset = UtilAll::str2ll(tempValue.asCString());
+  }
+
+  return h;
+}
+
+void PullMessageResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>(
+      "suggestWhichBrokerId", UtilAll::to_string(suggestWhichBrokerId)));
+  requestMap.insert(pair<string, string>("nextBeginOffset",
+                                         UtilAll::to_string(nextBeginOffset)));
+  requestMap.insert(
+      pair<string, string>("minOffset", UtilAll::to_string(minOffset)));
+  requestMap.insert(
+      pair<string, string>("maxOffset", UtilAll::to_string(maxOffset)));
+}
+//<!************************************************************************
+void GetConsumerListByGroupResponseHeader::Encode(Json::Value& outData) {
+  // outData = "{}";
+}
+
+void GetConsumerListByGroupResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {}
+//<!***************************************************************************
+void GetMinOffsetRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+}
+
+void GetMinOffsetRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+}
+//<!***************************************************************************
+CommandHeader* GetMinOffsetResponseHeader::Decode(Json::Value& ext) {
+  GetMinOffsetResponseHeader* h = new GetMinOffsetResponseHeader();
+
+  Json::Value& tempValue = ext["offset"];
+  if (tempValue.isString()) {
+    h->offset = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void GetMinOffsetResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void GetMaxOffsetRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+}
+
+void GetMaxOffsetRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+}
+//<!***************************************************************************
+CommandHeader* GetMaxOffsetResponseHeader::Decode(Json::Value& ext) {
+  GetMaxOffsetResponseHeader* h = new GetMaxOffsetResponseHeader();
+
+  Json::Value& tempValue = ext["offset"];
+  if (tempValue.isString()) {
+    h->offset = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void GetMaxOffsetResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void SearchOffsetRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+  outData["timestamp"] = UtilAll::to_string(timestamp);
+}
+
+void SearchOffsetRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+  requestMap.insert(
+      pair<string, string>("timestamp", UtilAll::to_string(timestamp)));
+}
+//<!***************************************************************************
+CommandHeader* SearchOffsetResponseHeader::Decode(Json::Value& ext) {
+  SearchOffsetResponseHeader* h = new SearchOffsetResponseHeader();
+
+  Json::Value& tempValue = ext["offset"];
+  if (tempValue.isString()) {
+    h->offset = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void SearchOffsetResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void ViewMessageRequestHeader::Encode(Json::Value& outData) {
+  outData["offset"] = UtilAll::to_string(offset);
+}
+
+void ViewMessageRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void GetEarliestMsgStoretimeRequestHeader::Encode(Json::Value& outData) {
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+}
+
+void GetEarliestMsgStoretimeRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+}
+//<!***************************************************************************
+CommandHeader* GetEarliestMsgStoretimeResponseHeader::Decode(
+    Json::Value& ext) {
+  GetEarliestMsgStoretimeResponseHeader* h =
+      new GetEarliestMsgStoretimeResponseHeader();
+
+  Json::Value& tempValue = ext["timestamp"];
+  if (tempValue.isString()) {
+    h->timestamp = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void GetEarliestMsgStoretimeResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(
+      pair<string, string>("timestamp", UtilAll::to_string(timestamp)));
+}
+//<!***************************************************************************
+void GetConsumerListByGroupRequestHeader::Encode(Json::Value& outData) {
+  outData["consumerGroup"] = consumerGroup;
+}
+
+void GetConsumerListByGroupRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+}
+//<!***************************************************************************
+void QueryConsumerOffsetRequestHeader::Encode(Json::Value& outData) {
+  outData["consumerGroup"] = consumerGroup;
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+}
+
+void QueryConsumerOffsetRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+}
+//<!***************************************************************************
+CommandHeader* QueryConsumerOffsetResponseHeader::Decode(
+    Json::Value& ext) {
+  QueryConsumerOffsetResponseHeader* h =
+      new QueryConsumerOffsetResponseHeader();
+  Json::Value& tempValue = ext["offset"];
+  if (tempValue.isString()) {
+    h->offset = UtilAll::str2ll(tempValue.asCString());
+  }
+  return h;
+}
+
+void QueryConsumerOffsetResponseHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void UpdateConsumerOffsetRequestHeader::Encode(Json::Value& outData) {
+  outData["consumerGroup"] = consumerGroup;
+  outData["topic"] = topic;
+  outData["queueId"] = queueId;
+  outData["commitOffset"] = UtilAll::to_string(commitOffset);
+}
+
+void UpdateConsumerOffsetRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+  requestMap.insert(pair<string, string>("topic", topic));
+  requestMap.insert(
+      pair<string, string>("queueId", UtilAll::to_string(queueId)));
+  requestMap.insert(
+      pair<string, string>("commitOffset", UtilAll::to_string(commitOffset)));
+}
+//<!***************************************************************************
+void ConsumerSendMsgBackRequestHeader::Encode(Json::Value& outData) {
+  outData["group"] = group;
+  outData["delayLevel"] = delayLevel;
+  outData["offset"] = UtilAll::to_string(offset);
+#ifdef ONS
+  outData["originMsgId"] = originMsgId;
+  outData["originTopic"] = originTopic;
+#endif
+}
+
+void ConsumerSendMsgBackRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("group", group));
+  requestMap.insert(
+      pair<string, string>("delayLevel", UtilAll::to_string(delayLevel)));
+  requestMap.insert(pair<string, string>("offset", UtilAll::to_string(offset)));
+}
+//<!***************************************************************************
+void GetConsumerListByGroupResponseBody::Decode(const MemoryBlock* mem,
+                                                vector<string>& cids) {
+  cids.clear();
+  //<! decode;
+  const char* const pData = static_cast<const char*>(mem->getData());
+
+  Json::Reader reader;
+  Json::Value root;
+  if (!reader.parse(pData, root)) {
+    LOG_ERROR("GetConsumerListByGroupResponse error");
+    return;
+  }
+
+  Json::Value ids = root["consumerIdList"];
+  for (unsigned int i = 0; i < ids.size(); i++) {
+    if (ids[i].isString()) {
+      cids.push_back(ids[i].asString());
+    }
+  }
+}
+
+void GetConsumerListByGroupResponseBody::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {}
+
+void ResetOffsetRequestHeader::setTopic(const string& tmp) { topic = tmp; }
+
+void ResetOffsetRequestHeader::setGroup(const string& tmp) { group = tmp; }
+
+void ResetOffsetRequestHeader::setTimeStamp(const int64& tmp) {
+  timestamp = tmp;
+}
+
+void ResetOffsetRequestHeader::setForceFlag(const bool& tmp) { isForce = tmp; }
+
+const string ResetOffsetRequestHeader::getTopic() const { return topic; }
+
+const string ResetOffsetRequestHeader::getGroup() const { return group; }
+
+const int64 ResetOffsetRequestHeader::getTimeStamp() const { return timestamp; }
+
+const bool ResetOffsetRequestHeader::getForceFlag() const { return isForce; }
+
+CommandHeader* ResetOffsetRequestHeader::Decode(Json::Value& ext) {
+  ResetOffsetRequestHeader* h = new ResetOffsetRequestHeader();
+
+  Json::Value& tempValue = ext["topic"];
+  if (tempValue.isString()) {
+    h->topic = tempValue.asString();
+  }
+
+  tempValue = ext["group"];
+  if (tempValue.isString()) {
+    h->group = tempValue.asString();
+  }
+
+  tempValue = ext["timestamp"];
+  if (tempValue.isString()) {
+    h->timestamp = UtilAll::str2ll(tempValue.asCString());
+  }
+
+  tempValue = ext["isForce"];
+  if (tempValue.isString()) {
+    h->isForce = UtilAll::to_bool(tempValue.asCString());
+  }
+  LOG_INFO("topic:%s, group:%s, timestamp:%lld, isForce:%d,isForce:%s",
+           h->topic.c_str(), h->group.c_str(), h->timestamp, h->isForce,
+           tempValue.asCString());
+  return h;
+}
+
+CommandHeader* GetConsumerRunningInfoRequestHeader::Decode(
+    Json::Value& ext) {
+  GetConsumerRunningInfoRequestHeader* h =
+      new GetConsumerRunningInfoRequestHeader();
+
+  Json::Value& tempValue = ext["consumerGroup"];
+  if (tempValue.isString()) {
+    h->consumerGroup = tempValue.asString();
+  }
+
+  tempValue = ext["clientId"];
+  if (tempValue.isString()) {
+    h->clientId = tempValue.asString();
+  }
+
+  tempValue = ext["jstackEnable"];
+  if (tempValue.isString()) {
+    h->jstackEnable = UtilAll::to_bool(tempValue.asCString());
+  }
+  LOG_INFO("consumerGroup:%s, clientId:%s,  jstackEnable:%d",
+           h->consumerGroup.c_str(), h->clientId.c_str(), h->jstackEnable);
+  return h;
+}
+
+void GetConsumerRunningInfoRequestHeader::Encode(Json::Value& outData) {
+  outData["consumerGroup"] = consumerGroup;
+  outData["clientId"] = clientId;
+  outData["jstackEnable"] = jstackEnable;
+}
+
+void GetConsumerRunningInfoRequestHeader::SetDeclaredFieldOfCommandHeader(
+    map<string, string>& requestMap) {
+  requestMap.insert(pair<string, string>("consumerGroup", consumerGroup));
+  requestMap.insert(pair<string, string>("clientId", clientId));
+  requestMap.insert(
+      pair<string, string>("jstackEnable", UtilAll::to_string(jstackEnable)));
+}
+
+const string GetConsumerRunningInfoRequestHeader::getConsumerGroup() const {
+  return consumerGroup;
+}
+
+void GetConsumerRunningInfoRequestHeader::setConsumerGroup(
+    const string& Group) {
+  consumerGroup = Group;
+}
+
+const string GetConsumerRunningInfoRequestHeader::getClientId() const {
+  return clientId;
+}
+
+void GetConsumerRunningInfoRequestHeader::setClientId(
+    const string& input_clientId) {
+  clientId = input_clientId;
+}
+
+const bool GetConsumerRunningInfoRequestHeader::isJstackEnable() const {
+  return jstackEnable;
+}
+
+void GetConsumerRunningInfoRequestHeader::setJstackEnable(
+    const bool& input_jstackEnable) {
+  jstackEnable = input_jstackEnable;
+}
+
+CommandHeader* NotifyConsumerIdsChangedRequestHeader::Decode(
+    Json::Value& ext) {
+  NotifyConsumerIdsChangedRequestHeader* h =
+      new NotifyConsumerIdsChangedRequestHeader();
+
+  Json::Value& tempValue = ext["consumerGroup"];
+  if (tempValue.isString()) {
+    h->consumerGroup = tempValue.asString();
+  }
+
+  return h;
+}
+
+void NotifyConsumerIdsChangedRequestHeader::setGroup(const string& tmp) {
+  consumerGroup = tmp;
+}
+const string NotifyConsumerIdsChangedRequestHeader::getGroup() const {
+  return consumerGroup;
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/CommandHeader.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/CommandHeader.h b/rocketmq-cpp/src/protocol/CommandHeader.h
new file mode 100644
index 0000000..5a55c55
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/CommandHeader.h
@@ -0,0 +1,431 @@
+/*
+ * 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 __COMMANDCUSTOMHEADER_H__
+#define __COMMANDCUSTOMHEADER_H__
+
+#include <string>
+#include "MQClientException.h"
+#include "MessageSysFlag.h"
+#include "UtilAll.h"
+#include "dataBlock.h"
+#include "json/json.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+
+class CommandHeader {
+ public:
+  virtual ~CommandHeader() {}
+  virtual void Encode(Json::Value& outData) {}
+  virtual void SetDeclaredFieldOfCommandHeader(
+      map<string, string>& requestMap) {}
+};
+
+//<!************************************************************************
+class GetRouteInfoRequestHeader : public CommandHeader {
+ public:
+  GetRouteInfoRequestHeader(const string& top) : topic(top) {}
+  virtual ~GetRouteInfoRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ private:
+  string topic;
+};
+
+//<!************************************************************************
+class UnregisterClientRequestHeader : public CommandHeader {
+ public:
+  UnregisterClientRequestHeader(string cID, string proGroup, string conGroup)
+      : clientID(cID), producerGroup(proGroup), consumerGroup(conGroup) {}
+  virtual ~UnregisterClientRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ private:
+  string clientID;
+  string producerGroup;
+  string consumerGroup;
+};
+
+//<!************************************************************************
+class CreateTopicRequestHeader : public CommandHeader {
+ public:
+  CreateTopicRequestHeader() : readQueueNums(0), writeQueueNums(0), perm(0) {}
+  virtual ~CreateTopicRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string topic;
+  string defaultTopic;
+  int readQueueNums;
+  int writeQueueNums;
+  int perm;
+  string topicFilterType;
+};
+
+//<!************************************************************************
+class SendMessageRequestHeader : public CommandHeader {
+ public:
+  SendMessageRequestHeader()
+      : defaultTopicQueueNums(0),
+        queueId(0),
+        sysFlag(0),
+        bornTimestamp(0),
+        flag(0),
+        reconsumeTimes(0),
+        unitMode(false) {}
+  virtual ~SendMessageRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+  int getReconsumeTimes();
+  void setReconsumeTimes(int input_reconsumeTimes);
+
+ public:
+  string producerGroup;
+  string topic;
+  string defaultTopic;
+  int defaultTopicQueueNums;
+  int queueId;
+  int sysFlag;
+  int64 bornTimestamp;
+  int flag;
+  string properties;
+  int reconsumeTimes;
+  bool unitMode;
+};
+
+//<!************************************************************************
+class SendMessageResponseHeader : public CommandHeader {
+ public:
+  SendMessageResponseHeader() : queueId(0), queueOffset(0) { msgId.clear(); }
+  virtual ~SendMessageResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string msgId;
+  int queueId;
+  int64 queueOffset;
+};
+
+//<!************************************************************************
+class PullMessageRequestHeader : public CommandHeader {
+ public:
+  PullMessageRequestHeader()
+      : queueId(0),
+        maxMsgNums(0),
+        sysFlag(0),
+        queueOffset(0),
+        commitOffset(0),
+        suspendTimeoutMillis(0),
+        subVersion(0) {}
+  virtual ~PullMessageRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string consumerGroup;
+  string topic;
+  int queueId;
+  int maxMsgNums;
+  int sysFlag;
+  string subscription;
+  int64 queueOffset;
+  int64 commitOffset;
+  int64 suspendTimeoutMillis;
+  int64 subVersion;
+};
+
+//<!************************************************************************
+class PullMessageResponseHeader : public CommandHeader {
+ public:
+  PullMessageResponseHeader()
+      : suggestWhichBrokerId(0),
+        nextBeginOffset(0),
+        minOffset(0),
+        maxOffset(0) {}
+  virtual ~PullMessageResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 suggestWhichBrokerId;
+  int64 nextBeginOffset;
+  int64 minOffset;
+  int64 maxOffset;
+};
+
+//<!************************************************************************
+class GetConsumerListByGroupResponseHeader : public CommandHeader {
+ public:
+  GetConsumerListByGroupResponseHeader() {}
+  virtual ~GetConsumerListByGroupResponseHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+};
+
+//<!***************************************************************************
+class GetMinOffsetRequestHeader : public CommandHeader {
+ public:
+  GetMinOffsetRequestHeader() : queueId(0){};
+  virtual ~GetMinOffsetRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string topic;
+  int queueId;
+};
+
+//<!***************************************************************************
+class GetMinOffsetResponseHeader : public CommandHeader {
+ public:
+  GetMinOffsetResponseHeader() : offset(0){};
+  virtual ~GetMinOffsetResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 offset;
+};
+
+//<!***************************************************************************
+class GetMaxOffsetRequestHeader : public CommandHeader {
+ public:
+  GetMaxOffsetRequestHeader() : queueId(0){};
+  virtual ~GetMaxOffsetRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string topic;
+  int queueId;
+};
+
+//<!***************************************************************************
+class GetMaxOffsetResponseHeader : public CommandHeader {
+ public:
+  GetMaxOffsetResponseHeader() : offset(0){};
+  virtual ~GetMaxOffsetResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 offset;
+};
+
+//<!***************************************************************************
+class SearchOffsetRequestHeader : public CommandHeader {
+ public:
+  SearchOffsetRequestHeader() : queueId(0), timestamp(0){};
+  virtual ~SearchOffsetRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string topic;
+  int queueId;
+  int64 timestamp;
+};
+
+//<!***************************************************************************
+class SearchOffsetResponseHeader : public CommandHeader {
+ public:
+  SearchOffsetResponseHeader() : offset(0){};
+  virtual ~SearchOffsetResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 offset;
+};
+
+//<!***************************************************************************
+class ViewMessageRequestHeader : public CommandHeader {
+ public:
+  ViewMessageRequestHeader() : offset(0){};
+  virtual ~ViewMessageRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 offset;
+};
+
+//<!***************************************************************************
+class GetEarliestMsgStoretimeRequestHeader : public CommandHeader {
+ public:
+  GetEarliestMsgStoretimeRequestHeader() : queueId(0){};
+  virtual ~GetEarliestMsgStoretimeRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string topic;
+  int queueId;
+};
+
+//<!***************************************************************************
+class GetEarliestMsgStoretimeResponseHeader : public CommandHeader {
+ public:
+  GetEarliestMsgStoretimeResponseHeader() : timestamp(0){};
+  virtual ~GetEarliestMsgStoretimeResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 timestamp;
+};
+
+//<!***************************************************************************
+class GetConsumerListByGroupRequestHeader : public CommandHeader {
+ public:
+  GetConsumerListByGroupRequestHeader(){};
+  virtual ~GetConsumerListByGroupRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string consumerGroup;
+};
+
+//<!************************************************************************
+class QueryConsumerOffsetRequestHeader : public CommandHeader {
+ public:
+  QueryConsumerOffsetRequestHeader() : queueId(0){};
+  virtual ~QueryConsumerOffsetRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string consumerGroup;
+  string topic;
+  int queueId;
+};
+
+//<!************************************************************************
+class QueryConsumerOffsetResponseHeader : public CommandHeader {
+ public:
+  QueryConsumerOffsetResponseHeader() : offset(0){};
+  virtual ~QueryConsumerOffsetResponseHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  int64 offset;
+};
+
+//<!************************************************************************
+class UpdateConsumerOffsetRequestHeader : public CommandHeader {
+ public:
+  UpdateConsumerOffsetRequestHeader() : queueId(0), commitOffset(0){};
+  virtual ~UpdateConsumerOffsetRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string consumerGroup;
+  string topic;
+  int queueId;
+  int64 commitOffset;
+};
+
+//<!***************************************************************************
+class ConsumerSendMsgBackRequestHeader : public CommandHeader {
+ public:
+  ConsumerSendMsgBackRequestHeader() : delayLevel(0), offset(0){};
+  virtual ~ConsumerSendMsgBackRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  string group;
+  int delayLevel;
+  int64 offset;
+};
+
+//<!***************************************************************************
+class GetConsumerListByGroupResponseBody {
+ public:
+  GetConsumerListByGroupResponseBody(){};
+  virtual ~GetConsumerListByGroupResponseBody() {}
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+
+ public:
+  static void Decode(const MemoryBlock* mem, vector<string>& cids);
+};
+
+class ResetOffsetRequestHeader : public CommandHeader {
+ public:
+  ResetOffsetRequestHeader() {}
+  ~ResetOffsetRequestHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  void setTopic(const string& tmp);
+  void setGroup(const string& tmp);
+  void setTimeStamp(const int64& tmp);
+  void setForceFlag(const bool& tmp);
+  const string getTopic() const;
+  const string getGroup() const;
+  const int64 getTimeStamp() const;
+  const bool getForceFlag() const;
+
+ private:
+  string topic;
+  string group;
+  int64 timestamp;
+  bool isForce;
+};
+
+class GetConsumerRunningInfoRequestHeader : public CommandHeader {
+ public:
+  GetConsumerRunningInfoRequestHeader() {}
+  virtual ~GetConsumerRunningInfoRequestHeader() {}
+  virtual void Encode(Json::Value& outData);
+  virtual void SetDeclaredFieldOfCommandHeader(map<string, string>& requestMap);
+  static CommandHeader* Decode(Json::Value& ext);
+  const string getConsumerGroup() const;
+  void setConsumerGroup(const string& consumerGroup);
+  const string getClientId() const;
+  void setClientId(const string& clientId);
+  const bool isJstackEnable() const;
+  void setJstackEnable(const bool& jstackEnable);
+
+ private:
+  string consumerGroup;
+  string clientId;
+  bool jstackEnable;
+};
+
+class NotifyConsumerIdsChangedRequestHeader : public CommandHeader {
+ public:
+  NotifyConsumerIdsChangedRequestHeader() {}
+  virtual ~NotifyConsumerIdsChangedRequestHeader() {}
+  static CommandHeader* Decode(Json::Value& ext);
+  void setGroup(const string& tmp);
+  const string getGroup() const;
+
+ private:
+  string consumerGroup;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/ConsumerRunningInfo.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/ConsumerRunningInfo.cpp b/rocketmq-cpp/src/protocol/ConsumerRunningInfo.cpp
new file mode 100644
index 0000000..10ac0aa
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/ConsumerRunningInfo.cpp
@@ -0,0 +1,109 @@
+#include "ConsumerRunningInfo.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+const string ConsumerRunningInfo::PROP_NAMESERVER_ADDR = "PROP_NAMESERVER_ADDR";
+const string ConsumerRunningInfo::PROP_THREADPOOL_CORE_SIZE =
+    "PROP_THREADPOOL_CORE_SIZE";
+const string ConsumerRunningInfo::PROP_CONSUME_ORDERLY = "PROP_CONSUMEORDERLY";
+const string ConsumerRunningInfo::PROP_CONSUME_TYPE = "PROP_CONSUME_TYPE";
+const string ConsumerRunningInfo::PROP_CLIENT_VERSION = "PROP_CLIENT_VERSION";
+const string ConsumerRunningInfo::PROP_CONSUMER_START_TIMESTAMP =
+    "PROP_CONSUMER_START_TIMESTAMP";
+
+const map<string, string> ConsumerRunningInfo::getProperties() const {
+  return properties;
+}
+
+void ConsumerRunningInfo::setProperties(
+    const map<string, string>& input_properties) {
+  properties = input_properties;
+}
+
+void ConsumerRunningInfo::setProperty(const string& key, const string& value) {
+  properties[key] = value;
+}
+
+const map<MessageQueue, ProcessQueueInfo> ConsumerRunningInfo::getMqTable()
+    const {
+  return mqTable;
+}
+
+void ConsumerRunningInfo::setMqTable(MessageQueue queue,
+                                     ProcessQueueInfo queueInfo) {
+  mqTable[queue] = queueInfo;
+}
+
+/*const map<string, ConsumeStatus> ConsumerRunningInfo::getStatusTable() const
+{
+return statusTable;
+}
+
+
+void ConsumerRunningInfo::setStatusTable(const map<string, ConsumeStatus>&
+input_statusTable)
+{
+statusTable = input_statusTable;
+}	*/
+
+const vector<SubscriptionData> ConsumerRunningInfo::getSubscriptionSet() const {
+  return subscriptionSet;
+}
+
+void ConsumerRunningInfo::setSubscriptionSet(
+    const vector<SubscriptionData>& input_subscriptionSet) {
+  subscriptionSet = input_subscriptionSet;
+}
+
+const string ConsumerRunningInfo::getJstack() const { return jstack; }
+
+void ConsumerRunningInfo::setJstack(const string& input_jstack) {
+  jstack = input_jstack;
+}
+
+string ConsumerRunningInfo::encode() {
+  Json::Value outData;
+
+  outData[PROP_NAMESERVER_ADDR] = properties[PROP_NAMESERVER_ADDR];
+  outData[PROP_CONSUME_TYPE] = properties[PROP_CONSUME_TYPE];
+  outData[PROP_CLIENT_VERSION] = properties[PROP_CLIENT_VERSION];
+  outData[PROP_CONSUMER_START_TIMESTAMP] =
+      properties[PROP_CONSUMER_START_TIMESTAMP];
+  outData[PROP_CONSUME_ORDERLY] = properties[PROP_CONSUME_ORDERLY];
+  outData[PROP_THREADPOOL_CORE_SIZE] = properties[PROP_THREADPOOL_CORE_SIZE];
+
+  Json::Value root;
+  root["jstack"] = jstack;
+  root["properties"] = outData;
+
+  {
+    vector<SubscriptionData>::const_iterator it = subscriptionSet.begin();
+    for (; it != subscriptionSet.end(); it++) {
+      root["subscriptionSet"].append(it->toJson());
+    }
+  }
+
+  Json::FastWriter fastwrite;
+  string finals = fastwrite.write(root);
+
+  Json::Value mq;
+  string key = "\"mqTable\":";
+  key.append("{");
+  for (map<MessageQueue, ProcessQueueInfo>::iterator it = mqTable.begin();
+       it != mqTable.end(); ++it) {
+    key.append((it->first).toJson().toStyledString());
+    key.erase(key.end() - 1);
+    key.append(":");
+    key.append((it->second).toJson().toStyledString());
+    key.append(",");
+  }
+  key.erase(key.end() - 1);
+  key.append("}");
+
+  // insert mqTable to final string
+  key.append(",");
+  finals.insert(1, key);
+
+  return finals;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/ConsumerRunningInfo.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/ConsumerRunningInfo.h b/rocketmq-cpp/src/protocol/ConsumerRunningInfo.h
new file mode 100644
index 0000000..6467ad5
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/ConsumerRunningInfo.h
@@ -0,0 +1,50 @@
+#ifndef __CONSUMERRUNNINGINFO_H__
+#define __CONSUMERRUNNINGINFO_H__
+
+#include "MessageQueue.h"
+#include "ProcessQueueInfo.h"
+#include "SubscriptionData.h"
+
+namespace rocketmq {
+
+class ConsumerRunningInfo {
+ public:
+  ConsumerRunningInfo() {}
+  virtual ~ConsumerRunningInfo() {
+    properties.clear();
+    mqTable.clear();
+    subscriptionSet.clear();
+  }
+
+ public:
+  static const string PROP_NAMESERVER_ADDR;
+  static const string PROP_THREADPOOL_CORE_SIZE;
+  static const string PROP_CONSUME_ORDERLY;
+  static const string PROP_CONSUME_TYPE;
+  static const string PROP_CLIENT_VERSION;
+  static const string PROP_CONSUMER_START_TIMESTAMP;
+
+ public:
+  const map<string, string> getProperties() const;
+  void setProperties(const map<string, string>& input_properties);
+  void setProperty(const string& key, const string& value);
+  const map<MessageQueue, ProcessQueueInfo> getMqTable() const;
+  void setMqTable(MessageQueue queue, ProcessQueueInfo queueInfo);
+  // const map<string, ConsumeStatus> getStatusTable() const;
+  // void setStatusTable(const map<string, ConsumeStatus>& input_statusTable) ;
+  const vector<SubscriptionData> getSubscriptionSet() const;
+  void setSubscriptionSet(
+      const vector<SubscriptionData>& input_subscriptionSet);
+  const string getJstack() const;
+  void setJstack(const string& input_jstack);
+  string encode();
+
+ private:
+  map<string, string> properties;
+  vector<SubscriptionData> subscriptionSet;
+  map<MessageQueue, ProcessQueueInfo> mqTable;
+  // map<string, ConsumeStatus> statusTable;
+  string jstack;
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/HeartbeatData.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/HeartbeatData.h b/rocketmq-cpp/src/protocol/HeartbeatData.h
new file mode 100755
index 0000000..9b74280
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/HeartbeatData.h
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __HEARTBEATDATA_H__
+#define __HEARTBEATDATA_H__
+#include <boost/thread/thread.hpp>
+#include <cstdlib>
+#include <string>
+#include <vector>
+#include "ConsumeType.h"
+#include "SubscriptionData.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ProducerData {
+ public:
+  ProducerData(){};
+  bool operator<(const ProducerData& pd) const {
+    return groupName < pd.groupName;
+  }
+  Json::Value toJson() const {
+    Json::Value outJson;
+    outJson["groupName"] = groupName;
+    return outJson;
+  }
+
+ public:
+  string groupName;
+};
+
+//<!***************************************************************************
+class ConsumerData {
+ public:
+  ConsumerData(){};
+  virtual ~ConsumerData() { subscriptionDataSet.clear(); }
+  bool operator<(const ConsumerData& cd) const {
+    return groupName < cd.groupName;
+  }
+
+  Json::Value toJson() const {
+    Json::Value outJson;
+    outJson["groupName"] = groupName;
+    outJson["consumeFromWhere"] = consumeFromWhere;
+    outJson["consumeType"] = consumeType;
+    outJson["messageModel"] = messageModel;
+
+    vector<SubscriptionData>::const_iterator it = subscriptionDataSet.begin();
+    for (; it != subscriptionDataSet.end(); it++) {
+      outJson["subscriptionDataSet"].append((*it).toJson());
+    }
+
+    return outJson;
+  }
+
+ public:
+  string groupName;
+  ConsumeType consumeType;
+  MessageModel messageModel;
+  ConsumeFromWhere consumeFromWhere;
+  vector<SubscriptionData> subscriptionDataSet;
+};
+
+//<!***************************************************************************
+class HeartbeatData {
+ public:
+  virtual ~HeartbeatData() {
+    m_producerDataSet.clear();
+    m_consumerDataSet.clear();
+  }
+  void Encode(string& outData) {
+    Json::Value root;
+
+    //<!id;
+    root["clientID"] = m_clientID;
+
+    //<!consumer;
+    {
+      boost::lock_guard<boost::mutex> lock(m_consumerDataMutex);
+      vector<ConsumerData>::iterator itc = m_consumerDataSet.begin();
+      for (; itc != m_consumerDataSet.end(); itc++) {
+        root["consumerDataSet"].append((*itc).toJson());
+      }
+    }
+
+    //<!producer;
+    {
+      boost::lock_guard<boost::mutex> lock(m_producerDataMutex);
+      vector<ProducerData>::iterator itp = m_producerDataSet.begin();
+      for (; itp != m_producerDataSet.end(); itp++) {
+        root["producerDataSet"].append((*itp).toJson());
+      }
+    }
+    //<!output;
+    Json::FastWriter fastwrite;
+    outData = fastwrite.write(root);
+  }
+
+  void setClientID(const string& clientID) { m_clientID = clientID; }
+
+  bool isProducerDataSetEmpty() {
+    boost::lock_guard<boost::mutex> lock(m_producerDataMutex);
+    return m_producerDataSet.empty();
+  }
+
+  void insertDataToProducerDataSet(ProducerData& producerData) {
+    boost::lock_guard<boost::mutex> lock(m_producerDataMutex);
+    m_producerDataSet.push_back(producerData);
+  }
+
+  bool isConsumerDataSetEmpty() {
+    boost::lock_guard<boost::mutex> lock(m_consumerDataMutex);
+    return m_consumerDataSet.empty();
+  }
+
+  void insertDataToConsumerDataSet(ConsumerData& consumerData) {
+    boost::lock_guard<boost::mutex> lock(m_consumerDataMutex);
+    m_consumerDataSet.push_back(consumerData);
+  }
+
+ private:
+  string m_clientID;
+  vector<ProducerData> m_producerDataSet;
+  vector<ConsumerData> m_consumerDataSet;
+  boost::mutex m_producerDataMutex;
+  boost::mutex m_consumerDataMutex;
+};
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/KVTable.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/KVTable.h b/rocketmq-cpp/src/protocol/KVTable.h
new file mode 100755
index 0000000..69191b7
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/KVTable.h
@@ -0,0 +1,41 @@
+/*
+ * 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 __KVTABLE_H__
+#define __KVTABLE_H__
+#include <map>
+#include <string>
+#include "RemotingSerializable.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class KVTable : public RemotingSerializable {
+ public:
+  virtual ~KVTable() { m_table.clear(); }
+
+  void Encode(string& outData) {}
+
+  const map<string, string>& getTable() { return m_table; }
+
+  void setTable(const map<string, string>& table) { m_table = table; }
+
+ private:
+  map<string, string> m_table;
+};
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/LockBatchBody.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/LockBatchBody.cpp b/rocketmq-cpp/src/protocol/LockBatchBody.cpp
new file mode 100755
index 0000000..c56c17f
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/LockBatchBody.cpp
@@ -0,0 +1,124 @@
+/*
+ * 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 "LockBatchBody.h"
+#include "Logging.h"
+namespace rocketmq {  //<!end namespace;
+
+string LockBatchRequestBody::getConsumerGroup() { return consumerGroup; }
+void LockBatchRequestBody::setConsumerGroup(string in_consumerGroup) {
+  consumerGroup = in_consumerGroup;
+}
+string LockBatchRequestBody::getClientId() { return clientId; }
+void LockBatchRequestBody::setClientId(string in_clientId) {
+  clientId = in_clientId;
+}
+vector<MQMessageQueue> LockBatchRequestBody::getMqSet() { return mqSet; }
+void LockBatchRequestBody::setMqSet(vector<MQMessageQueue> in_mqSet) {
+  mqSet.swap(in_mqSet);
+}
+void LockBatchRequestBody::Encode(string& outData) {
+  Json::Value root;
+  root["consumerGroup"] = consumerGroup;
+  root["clientId"] = clientId;
+
+  vector<MQMessageQueue>::const_iterator it = mqSet.begin();
+  for (; it != mqSet.end(); it++) {
+    root["mqSet"].append(toJson(*it));
+  }
+
+  Json::FastWriter fastwrite;
+  outData = fastwrite.write(root);
+}
+
+Json::Value LockBatchRequestBody::toJson(const MQMessageQueue& mq) const {
+  Json::Value outJson;
+  outJson["topic"] = mq.getTopic();
+  outJson["brokerName"] = mq.getBrokerName();
+  outJson["queueId"] = mq.getQueueId();
+  return outJson;
+}
+
+vector<MQMessageQueue> LockBatchResponseBody::getLockOKMQSet() {
+  return lockOKMQSet;
+}
+void LockBatchResponseBody::setLockOKMQSet(
+    vector<MQMessageQueue> in_lockOKMQSet) {
+  lockOKMQSet.swap(in_lockOKMQSet);
+}
+
+void LockBatchResponseBody::Decode(const MemoryBlock* mem,
+                                   vector<MQMessageQueue>& messageQueues) {
+  messageQueues.clear();
+  //<! decode;
+  const char* const pData = static_cast<const char*>(mem->getData());
+
+  Json::Reader reader;
+  Json::Value root;
+  if (!reader.parse(pData, root)) {
+    LOG_WARN("decode LockBatchResponseBody error");
+    return;
+  }
+
+  Json::Value mqs = root["lockOKMQSet"];
+  LOG_DEBUG("LockBatchResponseBody mqs size:%d", mqs.size());
+  for (unsigned int i = 0; i < mqs.size(); i++) {
+    MQMessageQueue mq;
+    Json::Value qd = mqs[i];
+    mq.setTopic(qd["topic"].asString());
+    mq.setBrokerName(qd["brokerName"].asString());
+    mq.setQueueId(qd["queueId"].asInt());
+    LOG_INFO("LockBatchResponseBody MQ:%s", mq.toString().c_str());
+    messageQueues.push_back(mq);
+  }
+}
+
+string UnlockBatchRequestBody::getConsumerGroup() { return consumerGroup; }
+void UnlockBatchRequestBody::setConsumerGroup(string in_consumerGroup) {
+  consumerGroup = in_consumerGroup;
+}
+string UnlockBatchRequestBody::getClientId() { return clientId; }
+void UnlockBatchRequestBody::setClientId(string in_clientId) {
+  clientId = in_clientId;
+}
+vector<MQMessageQueue> UnlockBatchRequestBody::getMqSet() { return mqSet; }
+void UnlockBatchRequestBody::setMqSet(vector<MQMessageQueue> in_mqSet) {
+  mqSet.swap(in_mqSet);
+}
+void UnlockBatchRequestBody::Encode(string& outData) {
+  Json::Value root;
+  root["consumerGroup"] = consumerGroup;
+  root["clientId"] = clientId;
+
+  vector<MQMessageQueue>::const_iterator it = mqSet.begin();
+  for (; it != mqSet.end(); it++) {
+    root["mqSet"].append(toJson(*it));
+  }
+
+  Json::FastWriter fastwrite;
+  outData = fastwrite.write(root);
+}
+
+Json::Value UnlockBatchRequestBody::toJson(
+    const MQMessageQueue& mq) const {
+  Json::Value outJson;
+  outJson["topic"] = mq.getTopic();
+  outJson["brokerName"] = mq.getBrokerName();
+  outJson["queueId"] = mq.getQueueId();
+  return outJson;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/LockBatchBody.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/LockBatchBody.h b/rocketmq-cpp/src/protocol/LockBatchBody.h
new file mode 100755
index 0000000..c1d7155
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/LockBatchBody.h
@@ -0,0 +1,80 @@
+/*
+ * 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 __LOCKBATCHBODY_H__
+#define __LOCKBATCHBODY_H__
+#include <set>
+#include <string>
+#include "MQMessageQueue.h"
+#include "RemotingSerializable.h"
+#include "dataBlock.h"
+#include "json/json.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+
+class LockBatchRequestBody {
+ public:
+  virtual ~LockBatchRequestBody() { mqSet.clear(); }
+  string getConsumerGroup();
+  void setConsumerGroup(string consumerGroup);
+  string getClientId();
+  void setClientId(string clientId);
+  vector<MQMessageQueue> getMqSet();
+  void setMqSet(vector<MQMessageQueue> mqSet);
+  void Encode(string& outData);
+  Json::Value toJson(const MQMessageQueue& mq) const;
+
+ private:
+  string consumerGroup;
+  string clientId;
+  vector<MQMessageQueue> mqSet;
+};
+
+class LockBatchResponseBody {
+ public:
+  virtual ~LockBatchResponseBody() { lockOKMQSet.clear(); }
+  vector<MQMessageQueue> getLockOKMQSet();
+  void setLockOKMQSet(vector<MQMessageQueue> lockOKMQSet);
+  static void Decode(const MemoryBlock* mem,
+                     vector<MQMessageQueue>& messageQueues);
+
+ private:
+  vector<MQMessageQueue> lockOKMQSet;
+};
+
+class UnlockBatchRequestBody {
+ public:
+  virtual ~UnlockBatchRequestBody() { mqSet.clear(); }
+  string getConsumerGroup();
+  void setConsumerGroup(string consumerGroup);
+  string getClientId();
+  void setClientId(string clientId);
+  vector<MQMessageQueue> getMqSet();
+  void setMqSet(vector<MQMessageQueue> mqSet);
+  void Encode(string& outData);
+  Json::Value toJson(const MQMessageQueue& mq) const;
+
+ private:
+  string consumerGroup;
+  string clientId;
+  vector<MQMessageQueue> mqSet;
+};
+
+}  //<!end namespace;
+#endif



[13/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/DefaultMQPushConsumer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/DefaultMQPushConsumer.h b/rocketmq-cpp/include/DefaultMQPushConsumer.h
new file mode 100755
index 0000000..9a39484
--- /dev/null
+++ b/rocketmq-cpp/include/DefaultMQPushConsumer.h
@@ -0,0 +1,160 @@
+/*
+ * 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 __DEFAULTMQPUSHCONSUMER_H__
+#define __DEFAULTMQPUSHCONSUMER_H__
+
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/scoped_ptr.hpp>
+#include <boost/thread/thread.hpp>
+#include <string>
+#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;
+//<!***************************************************************************
+class ROCKETMQCLIENT_API DefaultMQPushConsumer : public MQConsumer {
+ public:
+  DefaultMQPushConsumer(const std::string& groupname);
+  void boost_asio_work();
+  virtual ~DefaultMQPushConsumer();
+
+  //<!begin mqadmin;
+  virtual void start();
+  virtual void shutdown();
+  //<!end mqadmin;
+
+  //<!begin MQConsumer
+  virtual void sendMessageBack(MQMessageExt& msg, int delayLevel);
+  virtual void fetchSubscribeMessageQueues(const std::string& topic,
+                                           std::vector<MQMessageQueue>& mqs);
+  virtual void doRebalance();
+  virtual void persistConsumerOffset();
+  virtual void persistConsumerOffsetByResetOffset();
+  virtual void updateTopicSubscribeInfo(const std::string& topic,
+                                        std::vector<MQMessageQueue>& info);
+  virtual ConsumeType getConsumeType();
+  virtual ConsumeFromWhere getConsumeFromWhere();
+  void setConsumeFromWhere(ConsumeFromWhere consumeFromWhere);
+  virtual void getSubscriptions(std::vector<SubscriptionData>&);
+  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();
+  //<!end MQConsumer;
+
+  void registerMessageListener(MQMessageListener* pMessageListener);
+  MessageListenerType getMessageListenerType();
+  void subscribe(const std::string& topic, const std::string& subExpression);
+
+  OffsetStore* getOffsetStore() const;
+  virtual Rebalance* getRebalance() const;
+  ConsumeMsgService* getConsumerMsgService() const;
+
+  virtual void producePullMsgTask(PullRequest*);
+  void triggerNextPullRequest(boost::asio::deadline_timer* t,
+                              PullRequest* request);
+  void runPullMsgQueue(TaskQueue* pTaskQueue);
+  void pullMessage(PullRequest* pullrequest);       // sync pullMsg
+  void pullMessageAsync(PullRequest* pullrequest);  // async pullMsg
+  void setAsyncPull(bool asyncFlag);
+  AsyncPullCallback* getAsyncPullCallBack(PullRequest* request,
+                                          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;
+
+  /*
+    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();
+
+ private:
+  uint64_t m_startTime;
+  ConsumeFromWhere m_consumeFromWhere;
+  std::map<std::string, std::string> m_subTopics;
+  int m_consumeThreadCount;
+  OffsetStore* m_pOffsetStore;
+  Rebalance* m_pRebalance;
+  PullAPIWrapper* m_pPullAPIWrapper;
+  ConsumeMsgService* m_consumerServeice;
+  MQMessageListener* m_pMessageListener;
+  int m_consumeMessageBatchMaxSize;
+  int m_maxMsgCacheSize;
+  boost::asio::io_service m_async_ioService;
+  boost::scoped_ptr<boost::thread> m_async_service_thread;
+
+  typedef std::map<MQMessageQueue, AsyncPullCallback*> PullMAP;
+  PullMAP m_PullCallback;
+  bool m_asyncPull;
+  int m_asyncPullTimeout;
+  int m_pullMsgThreadPoolNum;
+
+ private:
+  TaskQueue* m_pullmsgQueue;
+  std::unique_ptr<boost::thread> m_pullmsgThread;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQClient.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQClient.h b/rocketmq-cpp/include/MQClient.h
new file mode 100755
index 0000000..b3a1ba8
--- /dev/null
+++ b/rocketmq-cpp/include/MQClient.h
@@ -0,0 +1,203 @@
+/*
+ * 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 __MQADMIN_H__
+#define __MQADMIN_H__
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include "MQMessageExt.h"
+#include "MQMessageQueue.h"
+#include "QueryResult.h"
+#include "RocketMQClient.h"
+#include "SessionCredentials.h"
+
+namespace rocketmq {
+class MQClientFactory;
+//<!***************************************************************************
+
+enum elogLevel {
+  eLOG_LEVEL_DISABLE = 0,
+  eLOG_LEVEL_FATAL = 1,
+  eLOG_LEVEL_ERROR = 2,
+  eLOG_LEVEL_WARN = 3,
+  eLOG_LEVEL_INFO = 4,
+  eLOG_LEVEL_DEBUG = 5,
+  eLOG_LEVEL_TRACE = 6,
+  eLOG_LEVEL_LEVEL_NUM = 7
+};
+
+class ROCKETMQCLIENT_API MQClient {
+ public:
+  MQClient();
+  virtual ~MQClient();
+
+ public:
+  // clientid=processId-ipAddr@instanceName;
+  std::string getMQClientId() const;
+  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);
+  //<!groupName;
+  const std::string& getGroupName() const;
+  void setGroupName(const std::string& groupname);
+  
+  /**
+  * no realization
+  */
+  void createTopic(const std::string& key, const std::string& newTopic, int queueNum);
+  /**
+  * search earliest msg store time for specified queue
+  *
+  * @param mq
+  *            message queue
+  * @return earliest store time, ms
+  */
+  int64 earliestMsgStoreTime(const MQMessageQueue& mq);
+  /**
+  * search maxOffset of queue
+  *
+  * @param mq
+  *            message queue
+  * @return minOffset of queue
+  */
+  int64 minOffset(const MQMessageQueue& mq);
+  /**
+  * search maxOffset of queue
+  * Note: maxOffset-1 is max offset that could get msg
+  * @param mq
+  *            message queue
+  * @return maxOffset of queue
+  */
+  int64 maxOffset(const MQMessageQueue& mq);
+  /**
+  * get queue offset by timestamp
+  *
+  * @param mq
+  *            mq queue
+  * @param timestamp
+  *            timestamp with ms unit
+  * @return queue offset according to timestamp
+  */
+  int64 searchOffset(const MQMessageQueue& mq, uint64_t timestamp);
+  /**
+  * get whole msg info from broker by msgId
+  *
+  * @param msgId
+  * @return MQMessageExt
+  */
+  MQMessageExt* viewMessage(const std::string& msgId);
+  /**
+  * query message by topic and key
+  *
+  * @param topic
+  *            topic name
+  * @param key
+  *            topic key
+  * @param maxNum
+  *            query num
+  * @param begin
+  *            begin timestamp
+  * @param end
+  *            end timestamp
+  * @return
+  *            according to QueryResult
+  */
+  QueryResult queryMessage(const std::string& topic, const std::string& key, int maxNum,
+                           int64 begin, int64 end);
+
+  std::vector<MQMessageQueue> 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 setMetaqLogLevel(elogLevel inputLevel);
+  void setMetaqLogFileSizeAndNum(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;
+  //<!  the name is globle only
+  std::string m_GroupName;
+  //<!factory;
+  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;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQClientException.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQClientException.h b/rocketmq-cpp/include/MQClientException.h
new file mode 100755
index 0000000..9b64197
--- /dev/null
+++ b/rocketmq-cpp/include/MQClientException.h
@@ -0,0 +1,95 @@
+/*
+ * 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 __MQCLIENTEXCEPTION_H__
+#define __MQCLIENTEXCEPTION_H__
+
+#include <exception>
+#include <ostream>
+#include <sstream>
+#include <string>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQException : public std::exception {
+ public:
+  MQException(const std::string& msg, int error, const char* file,
+              int line) throw()
+      : m_error(error), m_line(line), m_file(file) {
+    try {
+      std::stringstream ss;
+      ss << "msg: " << msg << ",error:" << error << ",in file <" << file
+         << "> line:" << line;
+      m_msg = ss.str();
+    } catch (...) {
+    }
+  }
+
+  MQException(const std::string& msg, int error, const char* file,
+              const char* type, int line) throw()
+      : m_error(error), m_line(line), m_file(file), m_type(type) {
+    try {
+      std::stringstream ss;
+      ss << "msg: " << msg << ",error:" << error << ",in file <" << file
+         << "> line:" << line;
+      m_msg = ss.str();
+    } catch (...) {
+    }
+  }
+
+  virtual ~MQException() throw() {}
+
+  const char* what() const throw() { return m_msg.c_str(); }
+
+  int GetError() const throw() { return m_error; }
+
+  virtual const char* GetType() const throw() { return m_type.c_str(); }
+
+ protected:
+  int m_error;
+  int m_line;
+  std::string m_msg;
+  std::string m_file;
+  std::string m_type;
+};
+
+inline std::ostream& operator<<(std::ostream& os, const MQException& e) {
+  os << "Type: " << e.GetType() << " , " << e.what();
+  return os;
+}
+
+#define DEFINE_MQCLIENTEXCEPTION(name)                                     \
+  class ROCKETMQCLIENT_API name : public MQException {                     \
+   public:                                                                 \
+    name(const std::string& msg, int error, const char* file,              \
+         int line) throw()                                                 \
+        : MQException(msg, error, file, #name, line) {}                    \
+    virtual const char* GetType() const throw() { return m_type.c_str(); } \
+  };
+
+DEFINE_MQCLIENTEXCEPTION(MQClientException)
+DEFINE_MQCLIENTEXCEPTION(MQBrokerException)
+DEFINE_MQCLIENTEXCEPTION(InterruptedException)
+DEFINE_MQCLIENTEXCEPTION(RemotingException)
+DEFINE_MQCLIENTEXCEPTION(UnknownHostException)
+
+#define THROW_MQEXCEPTION(e, msg, err) throw e(msg, err, __FILE__, __LINE__)
+#define NEW_MQEXCEPTION(e, msg, err) e(msg, err, __FILE__, __LINE__)
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQConsumer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQConsumer.h b/rocketmq-cpp/include/MQConsumer.h
new file mode 100755
index 0000000..89763a8
--- /dev/null
+++ b/rocketmq-cpp/include/MQConsumer.h
@@ -0,0 +1,70 @@
+/*
+ * 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 __MQCONSUMER_H__
+#define __MQCONSUMER_H__
+
+#include <string>
+#include "AsyncCallback.h"
+#include "ConsumeType.h"
+#include "MQClient.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+class SubscriptionData;
+class PullRequest;
+class Rebalance;
+class ConsumerRunningInfo;
+//<!************************************************************************
+class ROCKETMQCLIENT_API MQConsumer : public MQClient {
+ public:
+  virtual ~MQConsumer() {}
+  virtual void sendMessageBack(MQMessageExt& msg, int delayLevel) = 0;
+  virtual void fetchSubscribeMessageQueues(const std::string& topic,
+                                           std::vector<MQMessageQueue>& mqs) = 0;
+  virtual void doRebalance() = 0;
+  virtual void persistConsumerOffset() = 0;
+  virtual void persistConsumerOffsetByResetOffset() = 0;
+  virtual void updateTopicSubscribeInfo(const std::string& topic,
+                                        std::vector<MQMessageQueue>& 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<SubscriptionData>&) = 0;
+  virtual void producePullMsgTask(PullRequest*) = 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;
+  }
+
+ protected:
+  MessageModel m_messageModel;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQMessage.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQMessage.h b/rocketmq-cpp/include/MQMessage.h
new file mode 100755
index 0000000..fc18ac9
--- /dev/null
+++ b/rocketmq-cpp/include/MQMessage.h
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __MESSAGE_H__
+#define __MESSAGE_H__
+
+#include <map>
+#include <sstream>
+#include <string>
+#include <vector>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQMessage {
+ public:
+  MQMessage();
+  MQMessage(const std::string& topic, const std::string& body);
+  MQMessage(const std::string& topic, const std::string& tags, const std::string& body);
+  MQMessage(const std::string& topic, const std::string& tags, const std::string& keys,
+            const std::string& body);
+  MQMessage(const std::string& topic, const std::string& tags, const std::string& keys,
+            const int flag, const std::string& body, bool waitStoreMsgOK);
+
+  virtual ~MQMessage();
+  MQMessage(const MQMessage& other);
+  MQMessage& operator=(const MQMessage& other);
+
+  void setProperty(const std::string& name, const std::string& value);
+  std::string getProperty(const std::string& name) const;
+
+  std::string getTopic() const;
+  void setTopic(const std::string& topic);
+  void setTopic(const char* body, int len);
+
+  std::string getTags() const;
+  void setTags(const std::string& tags);
+
+  std::string getKeys() const;
+  void setKeys(const std::string& keys);
+  void setKeys(const std::vector<std::string>& keys);
+
+  int getDelayTimeLevel() const;
+  void setDelayTimeLevel(int level);
+
+  bool isWaitStoreMsgOK();
+  void setWaitStoreMsgOK(bool waitStoreMsgOK);
+
+  int getFlag() const;
+  void setFlag(int flag);
+
+  std::string getBody() const;
+  void setBody(const char* body, int len);
+  void setBody(const std::string& body);
+
+  std::map<std::string, std::string> getProperties() const;
+  void setProperties(std::map<std::string, std::string>& properties);
+
+  const std::string toString() const {
+    std::stringstream ss;
+    ss << "Message [topic=" << m_topic << ", flag=" << m_flag
+       << ", tag=" << getTags() << "]";
+    return ss.str();
+  }
+
+ protected:
+  void Init(const std::string& topic, const std::string& tags, const std::string& keys,
+            const int flag, const std::string& body, bool waitStoreMsgOK);
+
+ public:
+  static const std::string PROPERTY_KEYS;
+  static const std::string PROPERTY_TAGS;
+  static const std::string PROPERTY_WAIT_STORE_MSG_OK;
+  static const std::string PROPERTY_DELAY_TIME_LEVEL;
+  static const std::string PROPERTY_RETRY_TOPIC;
+  static const std::string PROPERTY_REAL_TOPIC;
+  static const std::string PROPERTY_REAL_QUEUE_ID;
+  static const std::string PROPERTY_TRANSACTION_PREPARED;
+  static const std::string PROPERTY_PRODUCER_GROUP;
+  static const std::string PROPERTY_MIN_OFFSET;
+  static const std::string PROPERTY_MAX_OFFSET;
+  static const std::string KEY_SEPARATOR;
+
+ private:
+  std::string m_topic;
+  int m_flag;
+  std::string m_body;
+  std::map<std::string, std::string> m_properties;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQMessageExt.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQMessageExt.h b/rocketmq-cpp/include/MQMessageExt.h
new file mode 100755
index 0000000..cf911e3
--- /dev/null
+++ b/rocketmq-cpp/include/MQMessageExt.h
@@ -0,0 +1,117 @@
+/*
+ * 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 __MESSAGEEXT_H__
+#define __MESSAGEEXT_H__
+
+#ifdef WIN32
+#include <Windows.h>
+#include <Winsock2.h>
+#else
+#include <sys/socket.h>
+#endif
+
+#include "MQMessage.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!message extend class, which was generated on broker;
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQMessageExt : public MQMessage {
+ public:
+  MQMessageExt();
+  MQMessageExt(int queueId, int64 bornTimestamp, sockaddr bornHost,
+               int64 storeTimestamp, sockaddr storeHost, std::string msgId);
+
+  virtual ~MQMessageExt();
+
+  static int parseTopicFilterType(int sysFlag);
+
+  int getQueueId() const;
+  void setQueueId(int queueId);
+
+  int64 getBornTimestamp() const;
+  void setBornTimestamp(int64 bornTimestamp);
+
+  sockaddr getBornHost() const;
+  std::string getBornHostString() const;
+  std::string getBornHostNameString() const;
+  void setBornHost(const sockaddr& bornHost);
+
+  int64 getStoreTimestamp() const;
+  void setStoreTimestamp(int64 storeTimestamp);
+
+  sockaddr getStoreHost() const;
+  std::string getStoreHostString() const;
+  void setStoreHost(const sockaddr& storeHost);
+
+  const std::string& getMsgId() const;
+  void setMsgId(const std::string& msgId);
+
+  int getSysFlag() const;
+  void setSysFlag(int sysFlag);
+
+  int getBodyCRC() const;
+  void setBodyCRC(int bodyCRC);
+
+  int64 getQueueOffset() const;
+  void setQueueOffset(int64 queueOffset);
+
+  int64 getCommitLogOffset() const;
+  void setCommitLogOffset(int64 physicOffset);
+
+  int getStoreSize() const;
+  void setStoreSize(int storeSize);
+
+  int getReconsumeTimes() const;
+  void setReconsumeTimes(int reconsumeTimes);
+
+  int64 getPreparedTransactionOffset() const;
+  void setPreparedTransactionOffset(int64 preparedTransactionOffset);
+
+  std::string toString() const {
+    std::stringstream ss;
+    ss << "MessageExt [queueId=" << m_queueId << ", storeSize=" << m_storeSize
+       << ", queueOffset=" << m_queueOffset << ", sysFlag=" << m_sysFlag
+       << ", bornTimestamp=" << m_bornTimestamp
+       << ", bornHost=" << getBornHostString()
+       << ", storeTimestamp=" << m_storeTimestamp
+       << ", storeHost=" << getStoreHostString() << ", msgId=" << m_msgId
+       << ", commitLogOffset=" << m_commitLogOffset << ", bodyCRC=" << m_bodyCRC
+       << ", reconsumeTimes=" << m_reconsumeTimes
+       << ", preparedTransactionOffset=" << m_preparedTransactionOffset << ",  "
+       << MQMessage::toString() << "]";
+    return ss.str();
+  }
+
+ private:
+  int64 m_queueOffset;
+  int64 m_commitLogOffset;
+  int64 m_bornTimestamp;
+  int64 m_storeTimestamp;
+  int64 m_preparedTransactionOffset;
+  int m_queueId;
+  int m_storeSize;
+  int m_sysFlag;
+  int m_bodyCRC;
+  int m_reconsumeTimes;
+  sockaddr m_bornHost;
+  sockaddr m_storeHost;
+  std::string m_msgId;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQMessageListener.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQMessageListener.h b/rocketmq-cpp/include/MQMessageListener.h
new file mode 100755
index 0000000..63bd74d
--- /dev/null
+++ b/rocketmq-cpp/include/MQMessageListener.h
@@ -0,0 +1,90 @@
+/*
+ * 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 __MESSAGELISTENER_H__
+#define __MESSAGELISTENER_H__
+
+#include <limits.h>
+#include "MQMessageExt.h"
+#include "MQMessageQueue.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+enum ConsumeStatus {
+  //consume success, msg will be cleard from memory
+  CONSUME_SUCCESS,
+  //consume fail, but will be re-consume by call messageLisenter again
+  RECONSUME_LATER
+};
+
+/*enum ConsumeOrderlyStatus
+{*/
+/**
+ * Success consumption
+ */
+// SUCCESS,
+/**
+ * Rollback consumption(only for binlog consumption)
+ */
+// ROLLBACK,
+/**
+ * Commit offset(only for binlog consumption)
+ */
+// COMMIT,
+/**
+ * Suspend current queue a moment
+ */
+// SUSPEND_CURRENT_QUEUE_A_MOMENT
+/*};*/
+
+enum MessageListenerType {
+  messageListenerDefaultly = 0,
+  messageListenerOrderly = 1,
+  messageListenerConcurrently = 2
+};
+
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQMessageListener {
+ public:
+  virtual ~MQMessageListener() {}
+  virtual ConsumeStatus consumeMessage(const std::vector<MQMessageExt>& msgs) = 0;
+  virtual MessageListenerType getMessageListenerType() {
+    return messageListenerDefaultly;
+  }
+};
+
+class ROCKETMQCLIENT_API MessageListenerOrderly : public MQMessageListener {
+ public:
+  virtual ~MessageListenerOrderly() {}
+  virtual ConsumeStatus consumeMessage(const std::vector<MQMessageExt>& msgs) = 0;
+  virtual MessageListenerType getMessageListenerType() {
+    return messageListenerOrderly;
+  }
+};
+
+class ROCKETMQCLIENT_API MessageListenerConcurrently
+    : public MQMessageListener {
+ public:
+  virtual ~MessageListenerConcurrently() {}
+  virtual ConsumeStatus consumeMessage(const std::vector<MQMessageExt>& msgs) = 0;
+  virtual MessageListenerType getMessageListenerType() {
+    return messageListenerConcurrently;
+  }
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQMessageQueue.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQMessageQueue.h b/rocketmq-cpp/include/MQMessageQueue.h
new file mode 100755
index 0000000..bb1c4ae
--- /dev/null
+++ b/rocketmq-cpp/include/MQMessageQueue.h
@@ -0,0 +1,64 @@
+/*
+ * 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 __MQMESSAGEQUEUE_H__
+#define __MQMESSAGEQUEUE_H__
+
+#include <iomanip>
+#include <sstream>
+#include <string>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!************************************************************************/
+//<!* MQ(T,B,ID);
+//<!************************************************************************/
+class ROCKETMQCLIENT_API MQMessageQueue {
+ public:
+  MQMessageQueue();
+  MQMessageQueue(const std::string& topic, const std::string& brokerName, int queueId);
+  MQMessageQueue(const MQMessageQueue& other);
+  MQMessageQueue& operator=(const MQMessageQueue& other);
+
+  std::string getTopic() const;
+  void setTopic(const std::string& topic);
+
+  std::string getBrokerName() const;
+  void setBrokerName(const std::string& brokerName);
+
+  int getQueueId() const;
+  void setQueueId(int queueId);
+
+  bool operator==(const MQMessageQueue& mq) const;
+  bool operator<(const MQMessageQueue& mq) const;
+  int compareTo(const MQMessageQueue& mq) const;
+
+  const std::string toString() const {
+    std::stringstream ss;
+    ss << "MessageQueue [topic=" << m_topic << ", brokerName=" << m_brokerName
+       << ", queueId=" << m_queueId << "]";
+
+    return ss.str();
+  }
+
+ private:
+  std::string m_topic;
+  std::string m_brokerName;
+  int m_queueId;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQProducer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQProducer.h b/rocketmq-cpp/include/MQProducer.h
new file mode 100755
index 0000000..e5df9ee
--- /dev/null
+++ b/rocketmq-cpp/include/MQProducer.h
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __MQPRODUCER_H__
+#define __MQPRODUCER_H__
+
+#include "AsyncCallback.h"
+#include "MQClient.h"
+#include "MQMessageQueue.h"
+#include "MQSelector.h"
+#include "RocketMQClient.h"
+#include "SendResult.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQProducer : public MQClient {
+ public:
+  MQProducer() {}
+  virtual ~MQProducer() {}
+  // if setted bActiveBroker, will search brokers with best service state
+  // firstly, then search brokers that had been sent failed by last time;
+  virtual SendResult send(MQMessage& msg, bool bSelectActiveBroker = false) = 0;
+  virtual SendResult send(MQMessage& msg, const MQMessageQueue& mq) = 0;
+  // strict order msg, if send failed on seleted MessageQueue, throw exception
+  // to up layer
+  virtual SendResult send(MQMessage& msg, MessageQueueSelector* selector,
+                          void* arg) = 0;
+  // non-strict order msg, if send failed on seleted MessageQueue, will auto
+  // retry others Broker queues with autoRetryTimes;
+  // if setted bActiveBroker, if send failed on seleted MessageQueue, , and then
+  // search brokers with best service state, lastly will search brokers that had
+  // been sent failed by last time;
+  virtual SendResult send(MQMessage& msg, MessageQueueSelector* selector,
+                          void* arg, int autoRetryTimes,
+                          bool bActiveBroker = false) = 0;
+  virtual void send(MQMessage& msg, SendCallback* sendCallback,
+                    bool bSelectActiveBroker = false) = 0;
+  virtual void send(MQMessage& msg, const MQMessageQueue& mq,
+                    SendCallback* sendCallback) = 0;
+  virtual void send(MQMessage& msg, MessageQueueSelector* selector, void* arg,
+                    SendCallback* sendCallback) = 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;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQSelector.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQSelector.h b/rocketmq-cpp/include/MQSelector.h
new file mode 100755
index 0000000..77309b8
--- /dev/null
+++ b/rocketmq-cpp/include/MQSelector.h
@@ -0,0 +1,33 @@
+/*
+ * 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 _MQSELECTOR_H_
+#define _MQSELECTOR_H_
+#include "MQMessage.h"
+#include "MQMessageQueue.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MessageQueueSelector {
+ public:
+  virtual ~MessageQueueSelector() {}
+  virtual MQMessageQueue select(const std::vector<MQMessageQueue>& mqs,
+                                const MQMessage& msg, void* arg) = 0;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif  //<! _MQSELECTOR_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/MQueueListener.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/MQueueListener.h b/rocketmq-cpp/include/MQueueListener.h
new file mode 100755
index 0000000..5938ebf
--- /dev/null
+++ b/rocketmq-cpp/include/MQueueListener.h
@@ -0,0 +1,34 @@
+/*
+ * 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 __MESSAGEQUEUELISTENER_H__
+#define __MESSAGEQUEUELISTENER_H__
+
+#include <vector>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API MQueueListener {
+ public:
+  virtual ~MQueueListener() {}
+  virtual void messageQueueChanged(const std::string& topic,
+                                   std::vector<MQMessageQueue>& mqAll,
+                                   std::vector<MQMessageQueue>& mqDivided) = 0;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/PullResult.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/PullResult.h b/rocketmq-cpp/include/PullResult.h
new file mode 100755
index 0000000..69a6aef
--- /dev/null
+++ b/rocketmq-cpp/include/PullResult.h
@@ -0,0 +1,69 @@
+/*
+ * 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 __PULLRESULT_H__
+#define __PULLRESULT_H__
+
+#include <sstream>
+#include "MQMessageExt.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+enum PullStatus {
+  FOUND,
+  NO_NEW_MSG,
+  NO_MATCHED_MSG,
+  OFFSET_ILLEGAL,
+  BROKER_TIMEOUT  // indicate pull request timeout or received NULL response
+};
+
+static const char* EnumStrings[] = {"FOUND", "NO_NEW_MSG", "NO_MATCHED_MSG",
+                                    "OFFSET_ILLEGAL", "BROKER_TIMEOUT"};
+
+//<!***************************************************************************
+class ROCKETMQCLIENT_API PullResult {
+ public:
+  PullResult();
+  PullResult(PullStatus status);
+  PullResult(PullStatus pullStatus, int64 nextBeginOffset,
+             int64 minOffset, int64 maxOffset);
+
+  PullResult(PullStatus pullStatus, int64 nextBeginOffset,
+             int64 minOffset, int64 maxOffset,
+             const std::vector<MQMessageExt>& src);
+
+  virtual ~PullResult();
+
+  std::string toString() {
+    std::stringstream ss;
+    ss << "PullResult [ pullStatus=" << EnumStrings[pullStatus]
+       << ", nextBeginOffset=" << nextBeginOffset << ", minOffset=" << minOffset
+       << ", maxOffset=" << maxOffset
+       << ", msgFoundList=" << msgFoundList.size() << " ]";
+    return ss.str();
+  }
+
+ public:
+  PullStatus pullStatus;
+  int64 nextBeginOffset;
+  int64 minOffset;
+  int64 maxOffset;
+  std::vector<MQMessageExt> msgFoundList;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/QueryResult.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/QueryResult.h b/rocketmq-cpp/include/QueryResult.h
new file mode 100755
index 0000000..c9861a0
--- /dev/null
+++ b/rocketmq-cpp/include/QueryResult.h
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __QUERYRESULT_H__
+#define __QUERYRESULT_H__
+
+#include "MQMessageExt.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!************************************************************************
+class ROCKETMQCLIENT_API QueryResult {
+ public:
+  QueryResult(uint64 indexLastUpdateTimestamp,
+              const std::vector<MQMessageExt*>& messageList) {
+    m_indexLastUpdateTimestamp = indexLastUpdateTimestamp;
+    m_messageList = messageList;
+  }
+
+  uint64 getIndexLastUpdateTimestamp() { return m_indexLastUpdateTimestamp; }
+
+  std::vector<MQMessageExt*>& getMessageList() { return m_messageList; }
+
+ private:
+  uint64 m_indexLastUpdateTimestamp;
+  std::vector<MQMessageExt*> m_messageList;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/RocketMQClient.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/RocketMQClient.h b/rocketmq-cpp/include/RocketMQClient.h
new file mode 100755
index 0000000..93be2d6
--- /dev/null
+++ b/rocketmq-cpp/include/RocketMQClient.h
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __ROCKETMQCLIENT_H__
+#define __ROCKETMQCLIENT_H__
+
+#ifdef WIN32
+#ifdef ROCKETMQCLIENT_EXPORTS
+#define ROCKETMQCLIENT_API __declspec(dllexport)
+#else
+#define ROCKETMQCLIENT_API __declspec(dllimport)
+#endif
+#else
+#define ROCKETMQCLIENT_API
+#endif
+
+/** A platform-independent 8-bit signed integer type. */
+typedef signed char int8;
+/** A platform-independent 8-bit unsigned integer type. */
+typedef unsigned char uint8;
+/** A platform-independent 16-bit signed integer type. */
+typedef signed short int16;
+/** A platform-independent 16-bit unsigned integer type. */
+typedef unsigned short uint16;
+/** A platform-independent 32-bit signed integer type. */
+typedef signed int int32;
+/** A platform-independent 32-bit unsigned integer type. */
+typedef unsigned int uint32;
+/** A platform-independent 64-bit integer type. */
+typedef long long int64;
+/** A platform-independent 64-bit unsigned integer type. */
+typedef unsigned long long uint64;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/SendMessageHook.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/SendMessageHook.h b/rocketmq-cpp/include/SendMessageHook.h
new file mode 100755
index 0000000..f5a4835
--- /dev/null
+++ b/rocketmq-cpp/include/SendMessageHook.h
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __SENDMESSAGEHOOK_H__
+#define __SENDMESSAGEHOOK_H__
+
+#include "MQClientException.h"
+#include "MQMessage.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class ROCKETMQCLIENT_API SendMessageContext {
+ public:
+  string producerGroup;
+  MQMessage msg;
+  MQMessageQueue mq;
+  string brokerAddr;
+  int communicationMode;
+  SendResult sendResult;
+  MQException* pException;
+  void* pArg;
+};
+
+class ROCKETMQCLIENT_API SendMessageHook {
+ public:
+  virtual ~SendMessageHook() {}
+  virtual string hookName() = 0;
+  virtual void sendMessageBefore(const SendMessageContext& context) = 0;
+  virtual void sendMessageAfter(const SendMessageContext& context) = 0;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/SendResult.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/SendResult.h b/rocketmq-cpp/include/SendResult.h
new file mode 100755
index 0000000..0f51854
--- /dev/null
+++ b/rocketmq-cpp/include/SendResult.h
@@ -0,0 +1,58 @@
+/*
+ * 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 __SENDRESULT_H__
+#define __SENDRESULT_H__
+
+#include "MQMessageQueue.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+//<!all to Master;
+enum SendStatus {
+  SEND_OK,
+  SEND_FLUSH_DISK_TIMEOUT,
+  SEND_FLUSH_SLAVE_TIMEOUT,
+  SEND_SLAVE_NOT_AVAILABLE
+};
+
+//<!***************************************************************************
+class ROCKETMQCLIENT_API SendResult {
+ public:
+  SendResult();
+  SendResult(const SendStatus& sendStatus, const std::string& msgId,
+             const MQMessageQueue& messageQueue, int64 queueOffset);
+
+  virtual ~SendResult();
+  SendResult(const SendResult& other);
+  SendResult& operator=(const SendResult& other);
+
+  const std::string& getMsgId() const;
+  SendStatus getSendStatus() const;
+  MQMessageQueue getMessageQueue() const;
+  int64 getQueueOffset() const;
+
+ private:
+  SendStatus m_sendStatus;
+  std::string m_msgId;
+  MQMessageQueue m_messageQueue;
+  int64 m_queueOffset;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/include/SessionCredentials.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/include/SessionCredentials.h b/rocketmq-cpp/include/SessionCredentials.h
new file mode 100755
index 0000000..d4a4de9
--- /dev/null
+++ b/rocketmq-cpp/include/SessionCredentials.h
@@ -0,0 +1,62 @@
+
+#ifndef __SESSIONCREDENTIALS_H__
+#define __SESSIONCREDENTIALS_H__
+
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+
+class SessionCredentials {
+ public:
+  static const std::string AccessKey;
+  static const std::string SecretKey;
+  static const std::string Signature;
+  static const std::string SignatureMethod;
+  static const std::string ONSChannelKey;
+
+  SessionCredentials(std::string input_accessKey, std::string input_secretKey,
+                     const std::string& input_authChannel)
+      : accessKey(input_accessKey),
+        secretKey(input_secretKey),
+        authChannel(input_authChannel) {}
+  SessionCredentials() : authChannel("ALIYUN") {}
+  ~SessionCredentials() {}
+
+  std::string getAccessKey() const { return accessKey; }
+
+  void setAccessKey(std::string input_accessKey) { accessKey = input_accessKey; }
+
+  std::string getSecretKey() const { return secretKey; }
+
+  void setSecretKey(std::string input_secretKey) { secretKey = input_secretKey; }
+
+  std::string getSignature() const { return signature; }
+
+  void setSignature(std::string input_signature) { signature = input_signature; }
+
+  std::string getSignatureMethod() const { return signatureMethod; }
+
+  void setSignatureMethod(std::string input_signatureMethod) {
+    signatureMethod = input_signatureMethod;
+  }
+
+  std::string getAuthChannel() const { return authChannel; }
+
+  void setAuthChannel(std::string input_channel) { authChannel = input_channel; }
+
+  bool isValid() const {
+    if (accessKey.empty() || secretKey.empty() || authChannel.empty())
+      return false;
+
+    return true;
+  }
+
+ private:
+  std::string accessKey;
+  std::string secretKey;
+  std::string signature;
+  std::string signatureMethod;
+  std::string authChannel;
+};
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/CMakeLists.txt b/rocketmq-cpp/libs/CMakeLists.txt
new file mode 100755
index 0000000..015951d
--- /dev/null
+++ b/rocketmq-cpp/libs/CMakeLists.txt
@@ -0,0 +1,16 @@
+# 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.
+
+add_subdirectory(signature)

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/CMakeLists.txt b/rocketmq-cpp/libs/signature/CMakeLists.txt
new file mode 100755
index 0000000..84ca0c5
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/CMakeLists.txt
@@ -0,0 +1,25 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+project(signature)
+
+include_directories(${CMAKE_CURRENT_SOURCE_DIR}/include)
+set(LIBRARY_OUTPUT_PATH ${CMAKE_CURRENT_SOURCE_DIR}/lib)
+
+aux_source_directory(src/  DIR_LIB_SRCS)
+
+add_library(Signature STATIC ${DIR_LIB_SRCS})
+target_link_libraries(Signature ${deplibs})
+set_target_properties(Signature PROPERTIES OUTPUT_NAME "Signature")

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/Makefile
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/Makefile b/rocketmq-cpp/libs/signature/Makefile
new file mode 100644
index 0000000..f3368cb
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/Makefile
@@ -0,0 +1,63 @@
+##====================================================================
+# make release=0 debug版。
+# make release=1 release版。
+CXXFLAGS = -g -fPIC -Wno-deprecated -O3
+
+ifeq ($(shell uname -m),x86_64)
+	CXXFLAGS += -m64
+	BITS:= 64
+else
+	CXXFLAGS += -m32
+	BITS := 32
+endif
+
+ifeq ($(release),0)
+	CXXFLAGS += -DDEBUG
+	OUTCONFIG := debug
+else
+	CXXFLAGS += -DNDEBUG
+	OUTCONFIG := release
+endif
+
+##====================================================================
+TOPDIR := .
+TARGET := $(TOPDIR)/lib/libSignature.a
+
+CPP_SRCDIR  := $(TOPDIR)/src
+CPP_SRC := $(foreach dir,$(CPP_SRCDIR), $(wildcard $(dir)/*.c))
+CPP_OBJS := $(patsubst %.c, %.o, $(CPP_SRC))
+VPATH := $(CPP_SRCDIR)
+
+LDLIBS :=
+LIBPATH :=
+CPPFLAGS := -I$(TOPDIR)/include $(addprefix -I,$(CPP_SRCDIR))
+
+CXX := g++
+AR := ar
+ARFLAGS := rcs
+##====================================================================
+all: build
+
+build:$(TARGET)
+
+
+$(TARGET):$(CPP_OBJS)
+	$(AR) $(ARFLAGS) $@ $^ $(LIBPATH) $(LDLIBS)
+
+%.o: %.c
+	$(CXX) -c $(CXXFLAGS) $(CPPFLAGS) -o $@ $<
+
+
+rebuild:clean build
+
+test:
+	@echo $(CPP_SRC)
+	@echo $(CPP_OBJS)
+
+clean:
+	$(RM) -rf $(CPP_OBJS)
+	$(RM) -rf $(TARGET)
+	$(RM) -rf tmp
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/base64.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/base64.h b/rocketmq-cpp/libs/signature/include/base64.h
new file mode 100755
index 0000000..7b2b3b7
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/base64.h
@@ -0,0 +1,54 @@
+/* base64.h -- Encode binary data using printable characters.
+   Copyright (C) 2004, 2005, 2006 Free Software Foundation, Inc.
+   Written by Simon Josefsson.
+
+   This program is free software; you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation; either version 2, or (at your option)
+   any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program; if not, write to the Free Software Foundation,
+   Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.  */
+
+#ifndef BASE64_H
+# define BASE64_H
+
+/* Get size_t. */
+# include <stddef.h>
+
+/* Get bool. */
+# include <stdbool.h>
+
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+		/* This uses that the expression (n+(k-1))/k means the smallest
+			 integer >= n/k, i.e., the ceiling of n/k.  */
+# define BASE64_LENGTH(inlen) ((((inlen) + 2) / 3) * 4)
+
+extern bool isbase64(char ch);
+
+extern void base64_encode(const char *in, size_t inlen,
+	char *out, size_t outlen);
+
+extern size_t base64_encode_alloc(const char *in, size_t inlen, char **out);
+
+extern bool base64_decode(const char *in, size_t inlen,
+	char *out, size_t *outlen);
+
+extern bool base64_decode_alloc(const char *in, size_t inlen,
+	char **out, size_t *outlen);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif /* BASE64_H */

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/hmac.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/hmac.h b/rocketmq-cpp/libs/signature/include/hmac.h
new file mode 100755
index 0000000..9ddd621
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/hmac.h
@@ -0,0 +1,55 @@
+
+#ifndef _HMAC_HMAC_H
+#define _HMAC_HMAC_H
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#include <sys/types.h>
+
+#ifndef SHA1_DIGEST_LEN
+#define SHA1_DIGEST_LEN		20
+#endif
+
+#ifndef SHA256_DIGEST_LEN
+#define SHA256_DIGEST_LEN	32
+#endif
+
+#ifndef SHA512_DIGEST_LEN
+#define SHA512_DIGEST_LEN	64
+#endif
+
+/*
+ * hmac_sha1:
+ * hmac_sha256:
+ * hmac_sha512:
+ *	Calculate Hashed Message Authentication Code with sha1/256/512 algorithm
+ *	Caution: ret_buf should provide enough space for HMAC result.
+ *
+ *	@key [in]: the secure-key string
+ *	@key_len [in]: the length of secure-key
+ *	@data [in]: data string could be calculated.
+ *	@data_len [in]: the length of data. length is needed because strlen could not take effect.
+ *	@ret_buf [out]: HMAC result stored in ret_buf.
+ */
+
+#ifdef __cplusplus
+namespace metaqSignature{
+
+#endif
+
+extern int hmac_sha1(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf);
+extern int hmac_sha256(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf);
+extern int hmac_sha512(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf);
+
+#ifdef __cplusplus
+}
+#endif
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/param_list.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/param_list.h b/rocketmq-cpp/libs/signature/include/param_list.h
new file mode 100755
index 0000000..0fd4e3d
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/param_list.h
@@ -0,0 +1,39 @@
+#ifndef PARAM_LIST_H
+#define PARAM_LIST_H
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+
+typedef struct _spas_param_node {
+	char *name;
+	char *value;
+	struct _spas_param_node *pnext;
+} SPAS_PARAM_NODE;
+
+typedef struct _spas_param_list {
+	SPAS_PARAM_NODE *phead;
+	unsigned int length; /* count of nodes */
+	unsigned int size; /* total size of string presentation */
+} SPAS_PARAM_LIST;
+
+extern SPAS_PARAM_LIST * create_param_list(void);
+extern int add_param_to_list(SPAS_PARAM_LIST *list, const char *name, const char *value);
+extern void free_param_list(SPAS_PARAM_LIST *list);
+extern char * param_list_to_str(const SPAS_PARAM_LIST *list);
+
+#ifdef __cplusplus
+}
+#endif
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/sha1.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/sha1.h b/rocketmq-cpp/libs/signature/include/sha1.h
new file mode 100755
index 0000000..cfb718a
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/sha1.h
@@ -0,0 +1,93 @@
+/* Declarations of functions and data types used for SHA1 sum
+   library functions.
+   Copyright (C) 2000, 2001, 2003, 2005, 2006, 2008
+   Free Software Foundation, Inc.
+
+   This program is free software; you can redistribute it and/or modify it
+   under the terms of the GNU General Public License as published by the
+   Free Software Foundation; either version 2, or (at your option) any
+   later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program; if not, write to the Free Software Foundation,
+   Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.  */
+
+#ifndef SHA1_H
+# define SHA1_H 1
+
+# include <stdio.h>
+# include <stdint.h>
+
+#ifdef __cplusplus
+namespace metaqSignature {
+#endif
+
+#define SHA1_DIGEST_SIZE 20
+
+/* Structure to save state of computation between the single steps.  */
+struct sha1_ctx
+{
+  uint32_t A;
+  uint32_t B;
+  uint32_t C;
+  uint32_t D;
+  uint32_t E;
+
+  uint32_t total[2];
+  uint32_t buflen;
+  uint32_t buffer[32];
+};
+
+
+/* Initialize structure containing state of computation. */
+extern void sha1_init_ctx (struct sha1_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is necessary that LEN is a multiple of 64!!! */
+extern void sha1_process_block (const void *buffer, size_t len,
+				struct sha1_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is NOT required that LEN is a multiple of 64.  */
+extern void sha1_process_bytes (const void *buffer, size_t len,
+				struct sha1_ctx *ctx);
+
+/* Process the remaining bytes in the buffer and put result from CTX
+   in first 20 bytes following RESBUF.  The result is always in little
+   endian byte order, so that a byte-wise output yields to the wanted
+   ASCII representation of the message digest.  */
+extern void *sha1_finish_ctx (struct sha1_ctx *ctx, void *resbuf);
+
+
+/* Put result from CTX in first 20 bytes following RESBUF.  The result is
+   always in little endian byte order, so that a byte-wise output yields
+   to the wanted ASCII representation of the message digest.  */
+extern void *sha1_read_ctx (const struct sha1_ctx *ctx, void *resbuf);
+
+
+/* Compute SHA1 message digest for bytes read from STREAM.  The
+   resulting message digest number will be written into the 20 bytes
+   beginning at RESBLOCK.  */
+extern int sha1_stream (FILE *stream, void *resblock);
+
+
+/* Compute SHA1 message digest for LEN bytes beginning at BUFFER.  The
+   result is always in little endian byte order, so that a byte-wise
+   output yields to the wanted ASCII representation of the message
+   digest.  */
+extern void *sha1_buffer (const char *buffer, size_t len, void *resblock);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/sha256.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/sha256.h b/rocketmq-cpp/libs/signature/include/sha256.h
new file mode 100755
index 0000000..1756e84
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/sha256.h
@@ -0,0 +1,91 @@
+/* Declarations of functions and data types used for SHA256 and SHA224 sum
+   library functions.
+   Copyright (C) 2005, 2006, 2008 Free Software Foundation, Inc.
+
+   This program is free software: you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation, either version 3 of the License, or
+   (at your option) any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+#ifndef SHA256_H
+# define SHA256_H 1
+
+# include <stdio.h>
+# include <stdint.h>
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+/* Structure to save state of computation between the single steps.  */
+struct sha256_ctx
+{
+  uint32_t state[8];
+
+  uint32_t total[2];
+  size_t buflen;
+  uint32_t buffer[32];
+};
+
+enum { SHA224_DIGEST_SIZE = 28 };
+enum { SHA256_DIGEST_SIZE = 32 };
+
+/* Initialize structure containing state of computation. */
+extern void sha256_init_ctx (struct sha256_ctx *ctx);
+extern void sha224_init_ctx (struct sha256_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is necessary that LEN is a multiple of 64!!! */
+extern void sha256_process_block (const void *buffer, size_t len,
+				  struct sha256_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is NOT required that LEN is a multiple of 64.  */
+extern void sha256_process_bytes (const void *buffer, size_t len,
+				  struct sha256_ctx *ctx);
+
+/* Process the remaining bytes in the buffer and put result from CTX
+   in first 32 (28) bytes following RESBUF.  The result is always in little
+   endian byte order, so that a byte-wise output yields to the wanted
+   ASCII representation of the message digest.  */
+extern void *sha256_finish_ctx (struct sha256_ctx *ctx, void *resbuf);
+extern void *sha224_finish_ctx (struct sha256_ctx *ctx, void *resbuf);
+
+
+/* Put result from CTX in first 32 (28) bytes following RESBUF.  The result is
+   always in little endian byte order, so that a byte-wise output yields
+   to the wanted ASCII representation of the message digest.  */
+extern void *sha256_read_ctx (const struct sha256_ctx *ctx, void *resbuf);
+extern void *sha224_read_ctx (const struct sha256_ctx *ctx, void *resbuf);
+
+
+/* Compute SHA256 (SHA224) message digest for bytes read from STREAM.  The
+   resulting message digest number will be written into the 32 (28) bytes
+   beginning at RESBLOCK.  */
+extern int sha256_stream (FILE *stream, void *resblock);
+extern int sha224_stream (FILE *stream, void *resblock);
+
+/* Compute SHA256 (SHA224) message digest for LEN bytes beginning at BUFFER.  The
+   result is always in little endian byte order, so that a byte-wise
+   output yields to the wanted ASCII representation of the message
+   digest.  */
+extern void *sha256_buffer (const char *buffer, size_t len, void *resblock);
+extern void *sha224_buffer (const char *buffer, size_t len, void *resblock);
+
+#ifdef __cplusplus
+}
+#endif 
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/sha512.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/sha512.h b/rocketmq-cpp/libs/signature/include/sha512.h
new file mode 100755
index 0000000..0b3b5e7
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/sha512.h
@@ -0,0 +1,95 @@
+/* Declarations of functions and data types used for SHA512 and SHA384 sum
+   library functions.
+   Copyright (C) 2005, 2006, 2008 Free Software Foundation, Inc.
+
+   This program is free software: you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation, either version 3 of the License, or
+   (at your option) any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+#ifndef SHA512_H
+# define SHA512_H 1
+
+# include <stdio.h>
+
+# include "u64.h"
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+/* Structure to save state of computation between the single steps.  */
+struct sha512_ctx
+{
+  u64 state[8];
+
+  u64 total[2];
+  size_t buflen;
+  u64 buffer[32];
+};
+
+enum { SHA384_DIGEST_SIZE = 48 };
+enum { SHA512_DIGEST_SIZE = 64 };
+
+/* Initialize structure containing state of computation. */
+extern void sha512_init_ctx (struct sha512_ctx *ctx);
+extern void sha384_init_ctx (struct sha512_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is necessary that LEN is a multiple of 128!!! */
+extern void sha512_process_block (const void *buffer, size_t len,
+				  struct sha512_ctx *ctx);
+
+/* Starting with the result of former calls of this function (or the
+   initialization function update the context for the next LEN bytes
+   starting at BUFFER.
+   It is NOT required that LEN is a multiple of 128.  */
+extern void sha512_process_bytes (const void *buffer, size_t len,
+				  struct sha512_ctx *ctx);
+
+/* Process the remaining bytes in the buffer and put result from CTX
+   in first 64 (48) bytes following RESBUF.  The result is always in little
+   endian byte order, so that a byte-wise output yields to the wanted
+   ASCII representation of the message digest.  */
+extern void *sha512_finish_ctx (struct sha512_ctx *ctx, void *resbuf);
+extern void *sha384_finish_ctx (struct sha512_ctx *ctx, void *resbuf);
+
+
+/* Put result from CTX in first 64 (48) bytes following RESBUF.  The result is
+   always in little endian byte order, so that a byte-wise output yields
+   to the wanted ASCII representation of the message digest.
+
+   IMPORTANT: On some systems it is required that RESBUF is correctly
+   aligned for a 32 bits value.  */
+extern void *sha512_read_ctx (const struct sha512_ctx *ctx, void *resbuf);
+extern void *sha384_read_ctx (const struct sha512_ctx *ctx, void *resbuf);
+
+
+/* Compute SHA512 (SHA384) message digest for bytes read from STREAM.  The
+   resulting message digest number will be written into the 64 (48) bytes
+   beginning at RESBLOCK.  */
+extern int sha512_stream (FILE *stream, void *resblock);
+extern int sha384_stream (FILE *stream, void *resblock);
+
+/* Compute SHA512 (SHA384) message digest for LEN bytes beginning at BUFFER.  The
+   result is always in little endian byte order, so that a byte-wise
+   output yields to the wanted ASCII representation of the message
+   digest.  */
+extern void *sha512_buffer (const char *buffer, size_t len, void *resblock);
+extern void *sha384_buffer (const char *buffer, size_t len, void *resblock);
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/spas_client.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/spas_client.h b/rocketmq-cpp/libs/signature/include/spas_client.h
new file mode 100755
index 0000000..b8fa02c
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/spas_client.h
@@ -0,0 +1,85 @@
+#ifndef SPAS_CLIENT_H
+#define SPAS_CLIENT_H
+
+#include "param_list.h"
+
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+#ifdef __cplusplus
+	namespace metaqSignature {
+#endif
+
+#define SPAS_MAX_KEY_LEN 128 /* max access_key/secret_key length */
+#define SPAS_MAX_PATH 256 /* max credential file path length */
+#define SPAS_ACCESS_KEY_TAG "accessKey" /* access_key tag in credential file */
+#define SPAS_SECRET_KEY_TAG "secretKey" /* secret_key tag in credential file */
+#define SPAS_CREDENTIAL_ENV "SPAS_CREDENTIAL" /* credential file environment variable */
+
+
+		typedef enum {
+			SIGN_HMACSHA1 = 0, /* HmacSHA1 */
+			SIGN_HMACSHA256 = 1, /* HmacSHA256 */
+		} SPAS_SIGN_ALGORITHM;
+
+		typedef enum {
+			NO_UPDATE = 0, /* do not update credential */
+			UPDATE_BY_ALARM = 1, /* update credential by SIGALRM */
+#ifdef SPAS_MT
+			UPDATE_BY_THREAD = 2, /* update credential by standalone thread */
+#endif
+		} CREDENTIAL_UPDATE_MODE;
+
+		typedef enum {
+			NO_ERROR = 0, /* success */
+			ERROR_INVALID_PARAM = -1, /* invalid parameter */
+			ERROR_NO_CREDENTIAL = -2, /* credential file not specified */
+			ERROR_FILE_OPEN = -3, /* file open failed */
+			ERROR_MEM_ALLOC = -4, /* memory allocation failed */
+			ERROR_MISSING_KEY = -5, /* missing access_key/secret_key */
+			ERROR_KEY_LENGTH = -6, /* key length exceed limit */
+			ERROR_UPDATE_CREDENTIAL = -7, /* update credential file failed */
+
+		} SPAS_ERROR_CODE;
+
+		typedef struct _spas_credential {
+			char access_key[SPAS_MAX_KEY_LEN];
+			char secret_key[SPAS_MAX_KEY_LEN];
+		} SPAS_CREDENTIAL;
+
+
+		extern int spas_load_credential(char *path, CREDENTIAL_UPDATE_MODE mode);
+		extern int spas_set_access_key(char *key);
+		extern int spas_set_secret_key(char *key);
+		extern char * spas_get_access_key(void);
+		extern char * spas_get_secret_key(void);
+		extern SPAS_CREDENTIAL * spas_get_credential(void);
+
+#ifdef SPAS_MT
+
+		extern int spas_load_thread_credential(char *path);
+		extern int spas_set_thread_access_key(char *key);
+		extern int spas_set_thread_secret_key(char *key);
+		extern char * spas_get_thread_access_key(void);
+		extern char * spas_get_thread_secret_key(void);
+
+#endif
+
+		extern char * spas_get_signature(const SPAS_PARAM_LIST *list, const char *key);
+		extern char * spas_get_signature2(const SPAS_PARAM_LIST *list, const char *key, SPAS_SIGN_ALGORITHM algorithm);
+		extern char * spas_sign(const char *data, size_t size, const char *key);
+		extern char * spas_sign2(const char *data, size_t size, const char *key, SPAS_SIGN_ALGORITHM algorithm);
+		extern void    spas_mem_free(char *pSignature);
+		extern char * spas_get_version(void);
+
+#ifdef __cplusplus
+	}
+#endif
+
+#ifdef __cplusplus
+}
+#endif
+
+#endif
+

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/include/u64.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/include/u64.h b/rocketmq-cpp/libs/signature/include/u64.h
new file mode 100644
index 0000000..34fd32b
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/include/u64.h
@@ -0,0 +1,159 @@
+/* uint64_t-like operations that work even on hosts lacking uint64_t
+
+   Copyright (C) 2006 Free Software Foundation, Inc.
+
+   This program is free software: you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation, either version 3 of the License, or
+   (at your option) any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+/* Written by Paul Eggert.  */
+
+#include <stddef.h>
+#include <stdint.h>
+
+/* Return X rotated left by N bits, where 0 < N < 64.  */
+#define u64rol(x, n) u64or (u64shl (x, n), u64shr (x, 64 - n))
+
+#ifdef UINT64_MAX
+
+/* Native implementations are trivial.  See below for comments on what
+   these operations do.  */
+typedef uint64_t u64;
+# define u64hilo(hi, lo) ((u64) (((u64) (hi) << 32) + (lo)))
+# define u64init(hi, lo) u64hilo (hi, lo)
+# define u64lo(x) ((u64) (x))
+# define u64lt(x, y) ((x) < (y))
+# define u64and(x, y) ((x) & (y))
+# define u64or(x, y) ((x) | (y))
+# define u64xor(x, y) ((x) ^ (y))
+# define u64plus(x, y) ((x) + (y))
+# define u64shl(x, n) ((x) << (n))
+# define u64shr(x, n) ((x) >> (n))
+
+#else
+
+/* u64 is a 64-bit unsigned integer value.
+   u64init (HI, LO), is like u64hilo (HI, LO), but for use in
+   initializer contexts.  */
+# ifdef WORDS_BIGENDIAN
+typedef struct { uint32_t hi, lo; } u64;
+#  define u64init(hi, lo) { hi, lo }
+# else
+typedef struct { uint32_t lo, hi; } u64;
+#  define u64init(hi, lo) { lo, hi }
+# endif
+
+/* Given the high and low-order 32-bit quantities HI and LO, return a u64
+   value representing (HI << 32) + LO.  */
+static inline u64
+u64hilo (uint32_t hi, uint32_t lo)
+{
+  u64 r;
+  r.hi = hi;
+  r.lo = lo;
+  return r;
+}
+
+/* Return a u64 value representing LO.  */
+static inline u64
+u64lo (uint32_t lo)
+{
+  u64 r;
+  r.hi = 0;
+  r.lo = lo;
+  return r;
+}
+
+/* Return X < Y.  */
+static inline int
+u64lt (u64 x, u64 y)
+{
+  return x.hi < y.hi || (x.hi == y.hi && x.lo < y.lo);
+}
+
+/* Return X & Y.  */
+static inline u64
+u64and (u64 x, u64 y)
+{
+  u64 r;
+  r.hi = x.hi & y.hi;
+  r.lo = x.lo & y.lo;
+  return r;
+}
+
+/* Return X | Y.  */
+static inline u64
+u64or (u64 x, u64 y)
+{
+  u64 r;
+  r.hi = x.hi | y.hi;
+  r.lo = x.lo | y.lo;
+  return r;
+}
+
+/* Return X ^ Y.  */
+static inline u64
+u64xor (u64 x, u64 y)
+{
+  u64 r;
+  r.hi = x.hi ^ y.hi;
+  r.lo = x.lo ^ y.lo;
+  return r;
+}
+
+/* Return X + Y.  */
+static inline u64
+u64plus (u64 x, u64 y)
+{
+  u64 r;
+  r.lo = x.lo + y.lo;
+  r.hi = x.hi + y.hi + (r.lo < x.lo);
+  return r;
+}
+
+/* Return X << N.  */
+static inline u64
+u64shl (u64 x, int n)
+{
+  u64 r;
+  if (n < 32)
+    {
+      r.hi = (x.hi << n) | (x.lo >> (32 - n));
+      r.lo = x.lo << n;
+    }
+  else
+    {
+      r.hi = x.lo << (n - 32);
+      r.lo = 0;
+    }
+  return r;
+}
+
+/* Return X >> N.  */
+static inline u64
+u64shr (u64 x, int n)
+{
+  u64 r;
+  if (n < 32)
+    {
+      r.hi = x.hi >> n;
+      r.lo = (x.hi << (32 - n)) | (x.lo >> n);
+    }
+  else
+    {
+      r.hi = 0;
+      r.lo = x.hi >> (n - 32);
+    }
+  return r;
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/lib/.gitkeep
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/lib/.gitkeep b/rocketmq-cpp/libs/signature/lib/.gitkeep
new file mode 100644
index 0000000..533bd5f
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/lib/.gitkeep
@@ -0,0 +1 @@
+#keep

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.sln
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.sln b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.sln
new file mode 100755
index 0000000..ea15e8e
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.sln
@@ -0,0 +1,22 @@
+
+Microsoft Visual Studio Solution File, Format Version 12.00
+# Visual Studio 2013
+VisualStudioVersion = 12.0.21005.1
+MinimumVisualStudioVersion = 10.0.40219.1
+Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "Metaqsignature", "Metaqsignature.vcxproj", "{F93E745C-232C-46A0-8D4B-91D3A53EE699}"
+EndProject
+Global
+	GlobalSection(SolutionConfigurationPlatforms) = preSolution
+		Debug|Win32 = Debug|Win32
+		Release|Win32 = Release|Win32
+	EndGlobalSection
+	GlobalSection(ProjectConfigurationPlatforms) = postSolution
+		{F93E745C-232C-46A0-8D4B-91D3A53EE699}.Debug|Win32.ActiveCfg = Debug|Win32
+		{F93E745C-232C-46A0-8D4B-91D3A53EE699}.Debug|Win32.Build.0 = Debug|Win32
+		{F93E745C-232C-46A0-8D4B-91D3A53EE699}.Release|Win32.ActiveCfg = Release|Win32
+		{F93E745C-232C-46A0-8D4B-91D3A53EE699}.Release|Win32.Build.0 = Release|Win32
+	EndGlobalSection
+	GlobalSection(SolutionProperties) = preSolution
+		HideSolutionNode = FALSE
+	EndGlobalSection
+EndGlobal


[02/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/sequencer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/sequencer.h b/rocketmq-cpp/src/thread/disruptor/sequencer.h
new file mode 100755
index 0000000..98d617f
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/sequencer.h
@@ -0,0 +1,190 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_SEQUENCER_H_ // NOLINT
+#define DISRUPTOR_SEQUENCER_H_ // NOLINT
+
+#include <vector>
+
+#include "batch_descriptor.h"
+#include "claim_strategy.h"
+#include "interface.h"
+#include "sequence_barrier.h"
+#include "wait_strategy.h"
+
+namespace rocketmq {
+
+// Coordinator for claiming sequences for access to a data structures while
+// tracking dependent {@link Sequence}s
+class Sequencer: public boost::noncopyable {
+ public:
+    // Construct a Sequencer with the selected strategies.
+    //
+    // @param buffer_size over which sequences are valid.
+    // @param claim_strategy_option for those claiming sequences.
+    // @param wait_strategy_option for those waiting on sequences.
+    Sequencer(int buffer_size,
+              ClaimStrategyOption claim_strategy_option,
+              WaitStrategyOption wait_strategy_option) :
+            buffer_size_(buffer_size),
+            claim_strategy_(CreateClaimStrategy(claim_strategy_option,
+                                                buffer_size_)),
+            wait_strategy_(CreateWaitStrategy(wait_strategy_option)) { }
+
+    ~Sequencer() {
+        delete claim_strategy_;
+        delete wait_strategy_;
+    }
+
+    // Set the sequences that will gate publishers to prevent the buffer
+    // wrapping.
+    //
+    // @param sequences to be gated on.
+    void set_gating_sequences(
+            const std::vector<Sequence*>& sequences) {
+        gating_sequences_ = sequences;
+    }
+
+    // Create a {@link SequenceBarrier} that gates on the cursor and a list of
+    // {@link Sequence}s.
+    //
+    // @param sequences_to_track this barrier will track.
+    // @return the barrier gated as required.
+    ProcessingSequenceBarrier* NewBarrier(
+            const std::vector<Sequence*>& sequences_to_track) {
+        return new ProcessingSequenceBarrier(wait_strategy_, &cursor_,
+                                             sequences_to_track);
+    }
+
+    // Create a new {@link BatchDescriptor} that is the minimum of the
+    // requested size and the buffer_size.
+    //
+    // @param size for the new batch.
+    // @return the new {@link BatchDescriptor}.
+    BatchDescriptor* NewBatchDescriptor(const int& size) {
+        return new BatchDescriptor(size<buffer_size_?size:buffer_size_);
+    }
+
+    // The capacity of the data structure to hold entries.
+    //
+    // @return capacity of the data structure.
+    int buffer_size() { return buffer_size_; }
+
+
+    // Get the value of the cursor indicating the published sequence.
+    //
+    // @return value of the cursor for events that have been published.
+    int64_t GetCursor() { return cursor_.sequence(); }
+
+    // Has the buffer capacity left to allocate another sequence. This is a
+    // concurrent method so the response should only be taken as an indication
+    // of available capacity.
+    //
+    // @return true if the buffer has the capacity to allocated another event.
+    bool HasAvalaibleCapacity() {
+        return claim_strategy_->HasAvalaibleCapacity(gating_sequences_);
+    }
+
+    // Claim the next event in sequence for publishing to the {@link RingBuffer}.
+    //
+    // @return the claimed sequence.
+    int64_t Next() {
+        return claim_strategy_->IncrementAndGet(gating_sequences_);
+    }
+
+    // Claim the next batch of sequence numbers for publishing.
+    //
+    // @param batch_descriptor to be updated for the batch range.
+    // @return the updated batch_descriptor.
+    BatchDescriptor* Next(BatchDescriptor* batch_descriptor) {
+        int64_t sequence = claim_strategy_->IncrementAndGet(batch_descriptor->size(), gating_sequences_);
+        batch_descriptor->set_end(sequence);
+        return batch_descriptor;
+    }
+
+    // Claim a specific sequence when only one publisher is involved.
+    //
+    // @param sequence to be claimed.
+    // @return sequence just claime.
+    int64_t Claim(const int64_t& sequence) {
+        claim_strategy_->SetSequence(sequence, gating_sequences_);
+        return sequence;
+    }
+
+    // Publish an event and make it visible to {@link EventProcessor}s.
+    //
+    // @param sequence to be published.
+    void Publish(const int64_t& sequence) {
+        Publish(sequence, 1);
+    }
+
+    // Publish the batch of events in sequence.
+    //
+    // @param sequence to be published.
+    void Publish(const BatchDescriptor& batch_descriptor) {
+        Publish(batch_descriptor.end(), batch_descriptor.size());
+    }
+
+    // Force the publication of a cursor sequence.
+    //
+    // Only use this method when forcing a sequence and you are sure only one
+    // publisher exists. This will cause the cursor to advance to this
+    // sequence.
+    //
+    // @param sequence to which is to be forced for publication.
+    void ForcePublish(const int64_t& sequence) {
+        cursor_.set_sequence(sequence);
+        wait_strategy_->SignalAllWhenBlocking();
+    }
+
+    // TODO(fsaintjacques): This was added to overcome
+    // NoOpEventProcessor::GetSequence(), this is not a clean solution.
+    Sequence* GetSequencePtr() {
+        return &cursor_;
+    }
+
+ private:
+    // Helpers
+    void Publish(const int64_t& sequence, const int64_t& batch_size) {
+        //LOG_DEBUG("publish sequence:%d", sequence);
+        claim_strategy_->SerialisePublishing(sequence, cursor_, batch_size);
+        cursor_.set_sequence(sequence);
+        wait_strategy_->SignalAllWhenBlocking();
+    }
+
+    // Members
+    const int buffer_size_;
+
+    PaddedSequence cursor_;
+    std::vector<Sequence*> gating_sequences_;
+
+    ClaimStrategyInterface* claim_strategy_;
+    WaitStrategyInterface* wait_strategy_;
+
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_RING_BUFFER_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/utils.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/utils.h b/rocketmq-cpp/src/thread/disruptor/utils.h
new file mode 100755
index 0000000..0730093
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/utils.h
@@ -0,0 +1,35 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_UTILS_H_ // NOLINT
+#define DISRUPTOR_UTILS_H_ // NOLINT
+
+// From Google C++ Standard, modified to use C++11 deleted functions.
+// A macro to disallow the copy constructor and operator= functions.
+#define DISALLOW_COPY_AND_ASSIGN(TypeName) \
+  TypeName(const TypeName&)  delete      \
+  void operator=(const TypeName&) delete;
+
+#endif // DISRUPTOR_UTILS_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/wait_strategy.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/wait_strategy.h b/rocketmq-cpp/src/thread/disruptor/wait_strategy.h
new file mode 100755
index 0000000..fb2e58a
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/wait_strategy.h
@@ -0,0 +1,377 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_WAITSTRATEGY_H_  // NOLINT
+#define DISRUPTOR_WAITSTRATEGY_H_  // NOLINT
+
+#include <sys/time.h>
+
+#include <boost/chrono.hpp>
+#include <boost/thread.hpp>
+#include <vector>
+
+#include "exceptions.h"
+#include "interface.h"
+#include "sequence.h"
+
+namespace rocketmq {
+
+// Strategy options which are available to those waiting on a
+// {@link RingBuffer}
+enum WaitStrategyOption {
+    // This strategy uses a condition variable inside a lock to block the
+    // event procesor which saves CPU resource at the expense of lock
+    // contention.
+    kBlockingStrategy,
+    // This strategy uses a progressive back off strategy by first spinning,
+    // then yielding, then sleeping for 1ms period. This is a good strategy
+    // for burst traffic then quiet periods when latency is not critical.
+    kSleepingStrategy,
+    // This strategy calls Thread.yield() in a loop as a waiting strategy
+    // which reduces contention at the expense of CPU resource.
+    kYieldingStrategy,
+    // This strategy call spins in a loop as a waiting strategy which is
+    // lowest and most consistent latency but ties up a CPU.
+    kBusySpinStrategy
+};
+
+// Blocking strategy that uses a lock and condition variable for
+// {@link Consumer}s waiting on a barrier.
+// This strategy should be used when performance and low-latency are not as
+// important as CPU resource.
+class BlockingStrategy :  public WaitStrategyInterface {
+ public:
+    BlockingStrategy() {}
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence) {
+        int64_t available_sequence = 0;
+        // We need to wait.
+        if ((available_sequence = cursor.sequence()) < sequence) {
+            // acquire lock
+            boost::unique_lock<boost::recursive_mutex> ulock(mutex_);
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                barrier.CheckAlert();
+                consumer_notify_condition_.wait(ulock);
+            }
+        } // unlock happens here, on ulock destruction.
+
+        if (0 != dependents.size()) {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                barrier.CheckAlert();
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence,
+                            const int64_t& timeout_micros) {
+        int64_t available_sequence = 0;
+        // We have to wait
+        if ((available_sequence = cursor.sequence()) < sequence) {
+            boost::unique_lock<boost::recursive_mutex> ulock(mutex_);
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                barrier.CheckAlert();
+                if (boost::cv_status::timeout == consumer_notify_condition_.wait_for(ulock,
+                    boost::chrono::microseconds(timeout_micros)))
+                    break;
+
+            }
+        } // unlock happens here, on ulock destruction
+
+        if (0 != dependents.size()) {
+            while ((available_sequence = GetMinimumSequence(dependents)) \
+                    < sequence) {
+                barrier.CheckAlert();
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual void SignalAllWhenBlocking() {
+        boost::unique_lock<boost::recursive_mutex> ulock(mutex_);
+        consumer_notify_condition_.notify_all();
+    }
+
+ private:
+    boost::recursive_mutex mutex_;
+    boost::condition_variable_any consumer_notify_condition_;
+
+};
+
+// Sleeping strategy
+class SleepingStrategy :  public WaitStrategyInterface {
+ public:
+    SleepingStrategy() {}
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence) {
+        int64_t available_sequence = 0;
+        int counter = kRetries;
+
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence,
+                            const int64_t& timeout_micros) {
+        // timing
+        struct timeval start_time, end_time;
+        gettimeofday(&start_time, NULL);
+        int64_t start_micro = start_time.tv_sec*1000000 + start_time.tv_usec;
+
+        int64_t available_sequence = 0;
+        int counter = kRetries;
+
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual void SignalAllWhenBlocking() {}
+
+    static const int kRetries = 200;
+
+ private:
+    int ApplyWaitMethod(const SequenceBarrierInterface& barrier, int counter) {
+        barrier.CheckAlert();
+        if (counter > 100) {
+            counter--;
+        } else if (counter > 0) {
+            counter--;
+            boost::this_thread::yield();
+        } else {
+            boost::this_thread::sleep_for(boost::chrono::milliseconds(1));
+        }
+
+        return counter;
+    }
+
+};
+
+// Yielding strategy that uses a sleep(0) for {@link EventProcessor}s waiting
+// on a barrier. This strategy is a good compromise between performance and
+// CPU resource.
+class YieldingStrategy :  public WaitStrategyInterface {
+ public:
+    YieldingStrategy() {}
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence) {
+        int64_t available_sequence = 0;
+        int counter = kSpinTries;
+
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence,
+                            const int64_t& timeout_micros) {
+        struct timeval start_time, end_time;
+        gettimeofday(&start_time, NULL);
+        int64_t start_micro = start_time.tv_sec*1000000 + start_time.tv_usec;
+
+        int64_t available_sequence = 0;
+        int counter = kSpinTries;
+
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                counter = ApplyWaitMethod(barrier, counter);
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual void SignalAllWhenBlocking() {}
+
+    static const int kSpinTries = 100;
+
+ private:
+    int ApplyWaitMethod(const SequenceBarrierInterface& barrier, int counter) {
+        barrier.CheckAlert();
+        if (counter == 0) {
+            boost::this_thread::yield();
+        } else {
+            counter--;
+        }
+
+        return counter;
+    }
+
+};
+
+
+// Busy Spin strategy that uses a busy spin loop for {@link EventProcessor}s
+// waiting on a barrier.
+// This strategy will use CPU resource to avoid syscalls which can introduce
+// latency jitter.  It is best used when threads can be bound to specific
+// CPU cores.
+class BusySpinStrategy :  public WaitStrategyInterface {
+ public:
+    BusySpinStrategy() {}
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence) {
+        int64_t available_sequence = 0;
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                barrier.CheckAlert();
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                barrier.CheckAlert();
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence,
+                            const int64_t& timeout_micros) {
+        struct timeval start_time, end_time;
+        gettimeofday(&start_time, NULL);
+        int64_t start_micro = start_time.tv_sec*1000000 + start_time.tv_usec;
+        int64_t available_sequence = 0;
+
+        if (0 == dependents.size()) {
+            while ((available_sequence = cursor.sequence()) < sequence) {
+                barrier.CheckAlert();
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        } else {
+            while ((available_sequence = GetMinimumSequence(dependents)) < \
+                    sequence) {
+                barrier.CheckAlert();
+                gettimeofday(&end_time, NULL);
+                int64_t end_micro = end_time.tv_sec*1000000 + end_time.tv_usec;
+                if (timeout_micros < (end_micro - start_micro))
+                    break;
+            }
+        }
+
+        return available_sequence;
+    }
+
+    virtual void SignalAllWhenBlocking() {}
+
+};
+
+WaitStrategyInterface* CreateWaitStrategy(WaitStrategyOption wait_option) {
+    switch (wait_option) {
+        case kBlockingStrategy:
+            return new BlockingStrategy();
+        case kSleepingStrategy:
+            return new SleepingStrategy();
+        case kYieldingStrategy:
+            return new YieldingStrategy();
+        case kBusySpinStrategy:
+            return new BusySpinStrategy();
+        default:
+            return NULL;
+    }
+}
+
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_WAITSTRATEGY_H_  NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptorLFQ.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptorLFQ.h b/rocketmq-cpp/src/thread/disruptorLFQ.h
new file mode 100644
index 0000000..a05b0cd
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptorLFQ.h
@@ -0,0 +1,113 @@
+#ifndef _DISRUPTORLFQ_
+#define _DISRUPTORLFQ_
+
+#include <disruptor/event_processor.h>
+#include <disruptor/event_publisher.h>
+#include <disruptor/exception_handler.h>
+#include <disruptor/interface.h>
+
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/scoped_ptr.hpp>
+#include <boost/thread/thread.hpp>
+
+namespace rocketmq {
+class Task;
+class taskEventFactory : public EventFactoryInterface<Task> {
+ public:
+  virtual Task* NewInstance(const int& size) const;
+};
+
+class taskBatchHandler : public EventHandlerInterface<Task> {
+ public:
+  taskBatchHandler(int pullMsgThreadPoolNum);
+  virtual ~taskBatchHandler() {}
+
+  virtual void OnEvent(const int64_t& sequence, const bool& end_of_batch,
+                       Task* event);
+  virtual void OnStart() {}
+  virtual void OnShutdown() {}
+  void runTaskEvent(Task event, int64_t sequence);
+  void stopIOService();
+
+ private:
+  boost::asio::io_service m_ioService;
+  boost::thread_group m_threadpool;
+  boost::asio::io_service::work m_ioServiceWork;
+};
+
+class taskEventTranslator : public EventTranslatorInterface<Task> {
+ public:
+  taskEventTranslator(Task* event);
+  virtual ~taskEventTranslator() {}
+  virtual Task* TranslateTo(const int64_t& sequence, Task* event);
+
+ private:
+  Task* m_taskEvent;
+};
+
+class taskExceptionHandler : public ExceptionHandlerInterface<Task> {
+ public:
+  virtual void Handle(const std::exception& exception, const int64_t& sequence,
+                      Task* event) {}
+};
+
+class disruptorLFQ {
+ public:
+  disruptorLFQ(int threadCount) {
+    m_task_factory.reset(new taskEventFactory());
+    m_ring_buffer.reset(new RingBuffer<Task>(
+        m_task_factory.get(),
+        1024,  // default size is 1024, must be n power of 2
+        kSingleThreadedStrategy,
+        // metaq::kBusySpinStrategy);//load normal, high cpu occupy, and
+        // smallest consume latency
+        // metaq::kYieldingStrategy); //load normal, high cpu occupy, and
+        // smaller consume latency
+        // metaq::kSleepingStrategy);//load normal, lowest cpu occupy, but
+        // largest consume latency
+        kBlockingStrategy));  // load normal, lowest CPU occupy, but
+                                     // largest consume latency
+
+    m_sequence_to_track.reset(new std::vector<Sequence*>(0));
+    m_sequenceBarrier.reset(
+        m_ring_buffer->NewBarrier(*(m_sequence_to_track.get())));
+
+    m_task_handler.reset(new taskBatchHandler(threadCount));
+    m_task_exception_handler.reset(new taskExceptionHandler());
+    m_processor.reset(new BatchEventProcessor<Task>(
+        m_ring_buffer.get(),
+        (SequenceBarrierInterface*)m_sequenceBarrier.get(),
+        m_task_handler.get(), m_task_exception_handler.get()));
+
+    /*
+    Publisher will try to publish BUFFER_SIZE + 1 events.
+    The last event should wait for at least one consume before publishing, thus
+    preventing an overwrite.
+    After the single consume, the publisher should resume and publish the last
+    event.
+    */
+    m_gating_sequences.push_back(m_processor.get()->GetSequence());
+    m_ring_buffer->set_gating_sequences(
+        m_gating_sequences);  // prevent overlap, publishEvent will be blocked
+                              // on ring_buffer_->Next();
+
+    m_publisher.reset(new EventPublisher<Task>(m_ring_buffer.get()));
+  }
+  virtual ~disruptorLFQ() {}
+
+ public:
+  boost::scoped_ptr<taskEventFactory> m_task_factory;
+  boost::scoped_ptr<taskBatchHandler> m_task_handler;
+  boost::scoped_ptr<taskExceptionHandler> m_task_exception_handler;
+  boost::scoped_ptr<std::vector<Sequence*>> m_sequence_to_track;
+  boost::scoped_ptr<RingBuffer<Task>> m_ring_buffer;
+  boost::scoped_ptr<ProcessingSequenceBarrier> m_sequenceBarrier;
+  boost::scoped_ptr<BatchEventProcessor<Task>> m_processor;
+  boost::scoped_ptr<EventPublisher<Task>> m_publisher;
+  std::vector<Sequence*> m_gating_sequences;
+};
+}
+//<!***************************************************************************
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/task_queue.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/task_queue.cpp b/rocketmq-cpp/src/thread/task_queue.cpp
new file mode 100755
index 0000000..510425c
--- /dev/null
+++ b/rocketmq-cpp/src/thread/task_queue.cpp
@@ -0,0 +1,103 @@
+/*
+ * 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 "task_queue.h"
+#include <sys/prctl.h>
+#include "UtilAll.h"
+#include "disruptorLFQ.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+Task* taskEventFactory::NewInstance(const int& size) const {
+  return new Task[size];
+}
+
+taskBatchHandler::taskBatchHandler(int pullMsgThreadPoolNum)
+    : m_ioServiceWork(m_ioService) {
+  string taskName = UtilAll::getProcessName();
+  prctl(PR_SET_NAME, "PullMsgTP", 0, 0, 0);
+  for (int i = 0; i != pullMsgThreadPoolNum; ++i) {
+    m_threadpool.create_thread(
+        boost::bind(&boost::asio::io_service::run, &m_ioService));
+  }
+  prctl(PR_SET_NAME, taskName.c_str(), 0, 0, 0);
+}
+
+void taskBatchHandler::OnEvent(const int64_t& sequence,
+                               const bool& end_of_batch, Task* event) {
+   //cp Task event out, avoid publish event override current Task event                               
+  Task currentTask(*event);
+  m_ioService.post(boost::bind(&taskBatchHandler::runTaskEvent, this,
+                               currentTask, sequence));
+}
+
+void taskBatchHandler::runTaskEvent(Task event, int64_t sequence) {
+  // LOG_INFO("processor event sequence:%lld",  sequence);
+  event.run();
+}
+
+void taskBatchHandler::stopIOService() {
+  m_ioService.stop();
+  m_threadpool.join_all();
+}
+
+taskEventTranslator::taskEventTranslator(Task* event) : m_taskEvent(event) {}
+
+Task* taskEventTranslator::TranslateTo(const int64_t& sequence, Task* event) {
+  // LOG_INFO("publish sequence:%lld, event:%x", sequence, event);
+  *event = *m_taskEvent;
+  return event;
+};
+
+//******************************************************************************************8
+TaskQueue::TaskQueue(int threadCount) {
+  m_flag.store(true, boost::memory_order_release);
+  m_disruptorLFQ = new disruptorLFQ(threadCount);
+}
+
+TaskQueue::~TaskQueue() {
+  delete m_disruptorLFQ;
+  m_disruptorLFQ = NULL;
+}
+
+void TaskQueue::close() {
+  m_flag.store(false, boost::memory_order_release);
+  m_disruptorLFQ->m_task_handler->stopIOService();
+  m_disruptorLFQ->m_processor->Halt();
+}
+
+bool TaskQueue::bTaskQueueStatusOK() {
+  return m_flag.load(boost::memory_order_acquire) == true;
+}
+
+void TaskQueue::produce(const Task& task) {
+  boost::mutex::scoped_lock lock(m_publishLock);
+  taskEventTranslator pTranslator(const_cast<Task*>(&task));
+  m_disruptorLFQ->m_publisher->PublishEvent(&pTranslator);
+}
+
+int TaskQueue::run() {
+  while (true) {
+    m_disruptorLFQ->m_processor->Run();
+    if (m_flag.load(boost::memory_order_acquire) == false) {
+      break;
+    }
+  }
+  return 0;
+}
+
+//<!***************************************************************************
+}  //<! end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/task_queue.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/task_queue.h b/rocketmq-cpp/src/thread/task_queue.h
new file mode 100755
index 0000000..e60607c
--- /dev/null
+++ b/rocketmq-cpp/src/thread/task_queue.h
@@ -0,0 +1,679 @@
+/*
+ * 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.
+ *//********************************************************************
+author:  qiwei.wqw@alibaba-inc.com
+*********************************************************************/
+#ifndef _TASK_QUEUE_I_
+#define _TASK_QUEUE_I_
+
+#include <boost/atomic.hpp>
+#include <boost/thread/mutex.hpp>
+#include <list>
+#include <vector>
+using namespace std;
+
+namespace rocketmq {
+
+//<!***************************************************************************
+typedef void (*taskfunc)(void*);
+
+//<! 数据加操作的集合
+class ITask_impl {
+ public:
+  virtual ~ITask_impl() {}
+  virtual void run() = 0;
+  virtual ITask_impl* fork() = 0;
+};
+
+//<!***************************************************************************
+class Task_impl : public ITask_impl {
+ public:
+  Task_impl(taskfunc func, void* arg_) : m_func(func), m_arg(arg_) {}
+  virtual ~Task_impl() {
+    m_func = 0;
+    m_arg = 0;
+  }
+  virtual void run() {
+    if (m_func != 0) m_func(m_arg);
+  }
+  virtual ITask_impl* fork() { return new Task_impl(m_func, m_arg); }
+
+ protected:
+  taskfunc m_func;
+  void* m_arg;
+};
+
+//<!***************************************************************************
+//<! 构造ITask_impl的子类对象时,为其赋予不同的数据和操作即可。
+//<! 这里使用了组合的方式实现了接口和实现的分离;
+//<!***************************************************************************
+struct Task {
+  static void dumy(void*) {}
+
+  Task(taskfunc f_, void* d_) { m_pTaskImpl = new Task_impl(f_, d_); }
+  Task(ITask_impl* task_imp_) : m_pTaskImpl(task_imp_) {}
+  Task(const Task& src_) : m_pTaskImpl(src_.m_pTaskImpl->fork()) {}
+  Task() { m_pTaskImpl = new Task_impl(&Task::dumy, 0); }
+  virtual ~Task() { delete m_pTaskImpl; }
+  Task& operator=(const Task& src_) {
+    delete m_pTaskImpl;
+    m_pTaskImpl = src_.m_pTaskImpl->fork();
+    return *this;
+  }
+  void run() {
+    if (m_pTaskImpl) m_pTaskImpl->run();
+  }
+
+ private:
+  ITask_impl* m_pTaskImpl;
+};
+
+//<!***************************************************************************
+class ITaskQueue {
+ public:
+  typedef list<Task> TaskList;
+
+ public:
+  virtual ~ITaskQueue() {}
+  virtual void close() = 0;
+  virtual void produce(const Task& task) = 0;
+  // virtual void multi_produce(const TaskList& tasks) = 0;
+  // virtual int  consume(Task& task)                  = 0;
+  // virtual int  consume_all(TaskList& tasks)         = 0;
+  virtual int run() = 0;
+  // virtual int  batch_run()                          = 0;
+  virtual bool bTaskQueueStatusOK() = 0;
+};
+
+//<!***************************************************************************
+//<! 由于不同的操作和数据可能需要构造不同ITask_impl子类,
+//<!
+//我们需要提供一些泛型函数,能够将用户的所有操作和数据都能轻易的转换成Task对象。
+//<! TaskBinder 提供一系列的gen函数,能够转换用户的普通函数和数据为Task对象;
+//<!***************************************************************************
+struct TaskBinder {
+  static Task gen(void (*func)(void*), void* p_) { return Task(func, p_); }
+
+  template <typename RET>
+  static Task gen(RET (*func)(void)) {
+    struct lambda {
+      static void taskfunc(void* p_) { (*(RET(*)(void))p_)(); };
+    };
+    return Task(lambda::taskfunc, (void*)func);
+  }
+
+  template <typename FUNCT, typename ARG1>
+  static Task gen(FUNCT func, ARG1 arg1) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      lambda(FUNCT func, const ARG1& arg1) : dest_func(func), arg1(arg1) {}
+      virtual void run() { (*dest_func)(arg1); }
+      virtual ITask_impl* fork() { return new lambda(dest_func, arg1); }
+    };
+    return Task(new lambda(func, arg1));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2)
+          : dest_func(func), arg1(arg1), arg2(arg2) {}
+      virtual void run() { (*dest_func)(arg1, arg2); }
+      virtual ITask_impl* fork() { return new lambda(dest_func, arg1, arg2); }
+    };
+    return Task(new lambda(func, arg1, arg2));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3)
+          : dest_func(func), arg1(arg1), arg2(arg2), arg3(arg3) {}
+      virtual void run() { (*dest_func)(arg1, arg2, arg3); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3);
+      }
+    };
+    return Task(new lambda(func, arg1, arg2, arg3));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4)
+          : dest_func(func), arg1(arg1), arg2(arg2), arg3(arg3), arg4(arg4) {}
+      virtual void run() { (*dest_func)(arg1, arg2, arg3, arg4); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4);
+      }
+    };
+    return Task(new lambda(func, arg1, arg2, arg3, arg4));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4,
+                  ARG5 arg5) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5)
+          : dest_func(func),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5) {}
+      virtual void run() { (*dest_func)(arg1, arg2, arg3, arg4, arg5); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4, arg5);
+      }
+    };
+    return Task(new lambda(func, arg1, arg2, arg3, arg4, arg5));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5, typename ARG6>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4,
+                  ARG5 arg5, ARG6 arg6) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6)
+          : dest_func(func),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6) {}
+      virtual void run() { (*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4, arg5, arg6);
+      }
+    };
+    return Task(new lambda(func, arg1, arg2, arg3, arg4, arg5, arg6));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5, typename ARG6, typename ARG7>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4,
+                  ARG5 arg5, ARG6 arg6, ARG7 arg7) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7)
+          : dest_func(func),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7) {}
+      virtual void run() {
+        (*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4, arg5, arg6, arg7);
+      }
+    };
+    return Task(new lambda(func, arg1, arg2, arg3, arg4, arg5, arg6, arg7));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5, typename ARG6, typename ARG7,
+            typename ARG8>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4,
+                  ARG5 arg5, ARG6 arg6, ARG7 arg7, ARG8 arg8) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      ARG8 arg8;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7, const ARG8& arg8)
+          : dest_func(func),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7),
+            arg8(arg8) {}
+      virtual void run() {
+        (*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4, arg5, arg6, arg7,
+                          arg8);
+      }
+    };
+    return Task(
+        new lambda(func, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8));
+  }
+
+  template <typename FUNCT, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5, typename ARG6, typename ARG7,
+            typename ARG8, typename ARG9>
+  static Task gen(FUNCT func, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4,
+                  ARG5 arg5, ARG6 arg6, ARG7 arg7, ARG8 arg8, ARG9 arg9) {
+    struct lambda : public ITask_impl {
+      FUNCT dest_func;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      ARG8 arg8;
+      ARG9 arg9;
+      lambda(FUNCT func, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7, const ARG8& arg8, const ARG9& arg9)
+          : dest_func(func),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7),
+            arg8(arg8),
+            arg9(arg9) {}
+      virtual void run() {
+        (*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, arg1, arg2, arg3, arg4, arg5, arg6, arg7,
+                          arg8, arg9);
+      }
+    };
+    return Task(
+        new lambda(func, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9));
+  }
+
+  //<!***************************************************************************
+  //<! class fuctions;;
+  //<!***************************************************************************
+  template <typename T, typename RET>
+  static Task gen(RET (T::*func)(void), T* obj) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(void);
+      T* obj;
+      lambda(RET (T::*func)(void), T* obj) : dest_func(func), obj(obj) {}
+      virtual void run() { (obj->*dest_func)(); }
+      virtual ITask_impl* fork() { return new lambda(dest_func, obj); }
+    };
+    return Task(new lambda(func, obj));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename ARG1>
+  static Task gen(RET (T::*func)(FARG1), T* obj, ARG1 arg1) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1);
+      T* obj;
+      ARG1 arg1;
+      lambda(RET (T::*pfunc)(FARG1), T* obj, const ARG1& arg1)
+          : dest_func(pfunc), obj(obj), arg1(arg1) {}
+      virtual void run() { (obj->*dest_func)(arg1); }
+      virtual ITask_impl* fork() { return new lambda(dest_func, obj, arg1); }
+    };
+    return Task(new lambda(func, obj, arg1));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename ARG1, typename ARG2>
+  static Task gen(RET (T::*func)(FARG1, FARG2), T* obj, ARG1 arg1, ARG2 arg2) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      lambda(RET (T::*func)(FARG1, FARG2), T* obj, const ARG1& arg1,
+             const ARG2& arg2)
+          : dest_func(func), obj(obj), arg1(arg1), arg2(arg2) {}
+      virtual void run() { (obj->*dest_func)(arg1, arg2); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename ARG1, typename ARG2, typename ARG3>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3), T* obj, ARG1 arg1,
+                  ARG2 arg2, ARG3 arg3) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2, FARG3);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3), T* obj, const ARG1& arg1,
+             const ARG2& arg2, const ARG3& arg3)
+          : dest_func(func), obj(obj), arg1(arg1), arg2(arg2), arg3(arg3) {}
+      virtual void run() { (obj->*dest_func)(arg1, arg2, arg3); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2, arg3));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename ARG1, typename ARG2,
+            typename ARG3, typename ARG4>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4), T* obj, ARG1 arg1,
+                  ARG2 arg2, ARG3 arg3, ARG4 arg4) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2, FARG3, FARG4);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4), T* obj,
+             const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4) {}
+      virtual void run() { (obj->*dest_func)(arg1, arg2, arg3, arg4); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2, arg3, arg4));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename FARG5, typename ARG1,
+            typename ARG2, typename ARG3, typename ARG4, typename ARG5>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5), T* obj,
+                  ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4, ARG5 arg5) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2, FARG3, FARG4, FARG5);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5), T* obj,
+             const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5) {}
+      virtual void run() { (obj->*dest_func)(arg1, arg2, arg3, arg4, arg5); }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4, arg5);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2, arg3, arg4, arg5));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename FARG5, typename FARG6,
+            typename ARG1, typename ARG2, typename ARG3, typename ARG4,
+            typename ARG5, typename ARG6>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6),
+                  T* obj, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4, ARG5 arg5,
+                  ARG6 arg6) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6), T* obj,
+             const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6) {}
+      virtual void run() {
+        (obj->*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4, arg5, arg6);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2, arg3, arg4, arg5, arg6));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename FARG5, typename FARG6,
+            typename FARG7, typename ARG1, typename ARG2, typename ARG3,
+            typename ARG4, typename ARG5, typename ARG6, typename ARG7>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6,
+                                 FARG7),
+                  T* obj, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4, ARG5 arg5,
+                  ARG6 arg6, ARG7 arg7) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7),
+             T* obj, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7) {}
+      virtual void run() {
+        (obj->*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4, arg5, arg6,
+                          arg7);
+      }
+    };
+    return Task(
+        new lambda(func, obj, arg1, arg2, arg3, arg4, arg5, arg6, arg7));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename FARG5, typename FARG6,
+            typename FARG7, typename FARG8, typename ARG1, typename ARG2,
+            typename ARG3, typename ARG4, typename ARG5, typename ARG6,
+            typename ARG7, typename ARG8>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6,
+                                 FARG7, FARG8),
+                  T* obj, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4, ARG5 arg5,
+                  ARG6 arg6, ARG7 arg7, ARG8 arg8) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)
+      (FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7, FARG8);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      ARG8 arg8;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7,
+                            FARG8),
+             T* obj, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7, const ARG8& arg8)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7),
+            arg8(arg8) {}
+      virtual void run() {
+        (obj->*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4, arg5, arg6,
+                          arg7, arg8);
+      }
+    };
+    return Task(
+        new lambda(func, obj, arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8));
+  }
+
+  template <typename T, typename RET, typename FARG1, typename FARG2,
+            typename FARG3, typename FARG4, typename FARG5, typename FARG6,
+            typename FARG7, typename FARG8, typename FARG9, typename ARG1,
+            typename ARG2, typename ARG3, typename ARG4, typename ARG5,
+            typename ARG6, typename ARG7, typename ARG8, typename ARG9>
+  static Task gen(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6,
+                                 FARG7, FARG8, FARG9),
+                  T* obj, ARG1 arg1, ARG2 arg2, ARG3 arg3, ARG4 arg4, ARG5 arg5,
+                  ARG6 arg6, ARG7 arg7, ARG8 arg8, ARG9 arg9) {
+    struct lambda : public ITask_impl {
+      RET (T::*dest_func)
+      (FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7, FARG8, FARG9);
+      T* obj;
+      ARG1 arg1;
+      ARG2 arg2;
+      ARG3 arg3;
+      ARG4 arg4;
+      ARG5 arg5;
+      ARG6 arg6;
+      ARG7 arg7;
+      ARG8 arg8;
+      ARG9 arg9;
+      lambda(RET (T::*func)(FARG1, FARG2, FARG3, FARG4, FARG5, FARG6, FARG7,
+                            FARG8, FARG9),
+             T* obj, const ARG1& arg1, const ARG2& arg2, const ARG3& arg3,
+             const ARG4& arg4, const ARG5& arg5, const ARG6& arg6,
+             const ARG7& arg7, const ARG8& arg8, const ARG9& arg9)
+          : dest_func(func),
+            obj(obj),
+            arg1(arg1),
+            arg2(arg2),
+            arg3(arg3),
+            arg4(arg4),
+            arg5(arg5),
+            arg6(arg6),
+            arg7(arg7),
+            arg8(arg8),
+            arg9(arg9) {}
+      virtual void run() {
+        (obj->*dest_func)(arg1, arg2, arg3, arg4, arg5, arg6, arg7, arg8, arg9);
+      }
+      virtual ITask_impl* fork() {
+        return new lambda(dest_func, obj, arg1, arg2, arg3, arg4, arg5, arg6,
+                          arg7, arg8, arg9);
+      }
+    };
+    return Task(new lambda(func, obj, arg1, arg2, arg3, arg4, arg5, arg6, arg7,
+                           arg8, arg9));
+  }
+};
+
+//<!***************************************************************************
+class disruptorLFQ;
+class TaskQueue : public ITaskQueue {
+ public:
+  TaskQueue(int threadCount);
+  virtual ~TaskQueue();
+  virtual void close();
+  virtual void produce(const Task& task);
+  virtual int run();
+  virtual bool bTaskQueueStatusOK();
+
+ private:
+  boost::atomic<bool> m_flag;
+  disruptorLFQ* m_disruptorLFQ;
+  boost::mutex m_publishLock;
+};
+
+//<!***************************************************************************
+}  //<! end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/ClientRemotingProcessor.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/ClientRemotingProcessor.cpp b/rocketmq-cpp/src/transport/ClientRemotingProcessor.cpp
new file mode 100644
index 0000000..a1462e6
--- /dev/null
+++ b/rocketmq-cpp/src/transport/ClientRemotingProcessor.cpp
@@ -0,0 +1,146 @@
+#include "ClientRemotingProcessor.h"
+#include "ClientRPCHook.h"
+#include "ConsumerRunningInfo.h"
+#include "MQClientFactory.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+ClientRemotingProcessor::ClientRemotingProcessor(
+    MQClientFactory* mqClientFactory)
+    : m_mqClientFactory(mqClientFactory) {}
+
+ClientRemotingProcessor::~ClientRemotingProcessor() {}
+
+RemotingCommand* ClientRemotingProcessor::processRequest(
+    const string& addr, RemotingCommand* request) {
+  LOG_DEBUG("request Command received:processRequest");
+  switch (request->getCode()) {
+    case CHECK_TRANSACTION_STATE:
+      //  return checkTransactionState( request);
+      break;
+    case NOTIFY_CONSUMER_IDS_CHANGED:
+      return notifyConsumerIdsChanged(request);
+      break;
+    case RESET_CONSUMER_CLIENT_OFFSET:  // oneWayRPC
+      return resetOffset(request);
+    case GET_CONSUMER_STATUS_FROM_CLIENT:
+      // return getConsumeStatus( request);
+      break;
+    case GET_CONSUMER_RUNNING_INFO:
+      return getConsumerRunningInfo(addr, request);
+      break;
+    case CONSUME_MESSAGE_DIRECTLY:
+      // return consumeMessageDirectly( request);
+      break;
+    default:
+      break;
+  }
+  return NULL;
+}
+
+RemotingCommand* ClientRemotingProcessor::resetOffset(
+    RemotingCommand* request) {
+  request->SetExtHeader(request->getCode());
+  const MemoryBlock* pbody = request->GetBody();
+  if (pbody->getSize()) {
+    ResetOffsetBody* offsetBody = ResetOffsetBody::Decode(pbody);
+    ResetOffsetRequestHeader* offsetHeader =
+        (ResetOffsetRequestHeader*)request->getCommandHeader();
+    if (offsetBody) {
+      m_mqClientFactory->resetOffset(offsetHeader->getGroup(),
+                                     offsetHeader->getTopic(),
+                                     offsetBody->getOffsetTable());
+    } else {
+      LOG_ERROR(
+          "resetOffset failed as received data could not be unserialized");
+    }
+  }
+  return NULL;  // as resetOffset is oneWayRPC, do not need return any response
+}
+
+std::map<MQMessageQueue, int64> ResetOffsetBody::getOffsetTable() {
+  return m_offsetTable;
+}
+
+void ResetOffsetBody::setOffsetTable(MQMessageQueue mq, int64 offset) {
+  m_offsetTable[mq] = offset;
+}
+
+ResetOffsetBody* ResetOffsetBody::Decode(const MemoryBlock* mem) {
+  const char* const pData = static_cast<const char*>(mem->getData());
+  Json::Reader reader;
+  Json::Value root;
+  const char* begin = pData;
+  const char* end = pData + mem->getSize();
+
+  if (!reader.parse(begin, end, root, true)) {
+    LOG_ERROR("ResetOffsetBody::Decode fail");
+    return NULL;
+  }
+
+  ResetOffsetBody* rfb = new ResetOffsetBody();
+  Json::Value qds = root["offsetTable"];
+  for (unsigned int i = 0; i < qds.size(); i++) {
+    MQMessageQueue mq;
+    Json::Value qd = qds[i];
+    mq.setBrokerName(qd["brokerName"].asString());
+    mq.setQueueId(qd["queueId"].asInt());
+    mq.setTopic(qd["topic"].asString());
+    int64 offset = qd["offset"].asInt();
+    LOG_INFO("ResetOffsetBody brokerName:%s, queueID:%d, topic:%s, offset:%lld",
+             mq.getBrokerName().c_str(), mq.getQueueId(), mq.getTopic().c_str(),
+             offset);
+    rfb->setOffsetTable(mq, offset);
+  }
+  return rfb;
+}
+
+RemotingCommand* ClientRemotingProcessor::getConsumerRunningInfo(
+    const string& addr, RemotingCommand* request) {
+  request->SetExtHeader(request->getCode());
+  GetConsumerRunningInfoRequestHeader* requestHeader =
+      (GetConsumerRunningInfoRequestHeader*)request->getCommandHeader();
+  LOG_INFO("getConsumerRunningInfo:%s",
+           requestHeader->getConsumerGroup().c_str());
+
+  RemotingCommand* pResponse = new RemotingCommand(
+      request->getCode(), "CPP", request->getVersion(), request->getOpaque(),
+      request->getFlag(), request->getRemark(), NULL);
+
+  unique_ptr<ConsumerRunningInfo> runningInfo(
+      m_mqClientFactory->consumerRunningInfo(
+          requestHeader->getConsumerGroup()));
+  if (runningInfo) {
+    if (requestHeader->isJstackEnable()) {
+      /*string jstack = UtilAll::jstack();
+       consumerRunningInfo->setJstack(jstack);*/
+    }
+    pResponse->setCode(SUCCESS_VALUE);
+    string body = runningInfo->encode();
+    pResponse->SetBody(body.c_str(), body.length());
+    pResponse->setMsgBody(body);
+  } else {
+    pResponse->setCode(SYSTEM_ERROR);
+    pResponse->setRemark("The Consumer Group not exist in this consumer");
+  }
+
+  SessionCredentials sessionCredentials;
+  m_mqClientFactory->getSessionCredentialFromConsumer(
+      requestHeader->getConsumerGroup(), sessionCredentials);
+  ClientRPCHook rpcHook(sessionCredentials);
+  rpcHook.doBeforeRequest(addr, *pResponse);
+  pResponse->Encode();
+  return pResponse;
+}
+
+RemotingCommand* ClientRemotingProcessor::notifyConsumerIdsChanged(
+    RemotingCommand* request) {
+  request->SetExtHeader(request->getCode());
+  NotifyConsumerIdsChangedRequestHeader* requestHeader =
+      (NotifyConsumerIdsChangedRequestHeader*)request->getCommandHeader();
+  LOG_INFO("notifyConsumerIdsChanged:%s", requestHeader->getGroup().c_str());
+  m_mqClientFactory->doRebalanceByConsumerGroup(requestHeader->getGroup());
+  return NULL;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/ClientRemotingProcessor.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/ClientRemotingProcessor.h b/rocketmq-cpp/src/transport/ClientRemotingProcessor.h
new file mode 100644
index 0000000..1bb757e
--- /dev/null
+++ b/rocketmq-cpp/src/transport/ClientRemotingProcessor.h
@@ -0,0 +1,39 @@
+#ifndef __CLIENTREMOTINGPROCESSOR_H__
+#define __CLIENTREMOTINGPROCESSOR_H__
+
+#include "MQMessageQueue.h"
+#include "MQProtos.h"
+#include "RemotingCommand.h"
+
+namespace rocketmq {
+
+class MQClientFactory;
+class ClientRemotingProcessor {
+ public:
+  ClientRemotingProcessor(MQClientFactory* mqClientFactory);
+  virtual ~ClientRemotingProcessor();
+
+  RemotingCommand* processRequest(const string& addr, RemotingCommand* request);
+  RemotingCommand* resetOffset(RemotingCommand* request);
+  RemotingCommand* getConsumerRunningInfo(const string& addr,
+                                          RemotingCommand* request);
+  RemotingCommand* notifyConsumerIdsChanged(RemotingCommand* request);
+
+ private:
+  MQClientFactory* m_mqClientFactory;
+};
+
+class ResetOffsetBody {
+ public:
+  ResetOffsetBody() {}
+  virtual ~ResetOffsetBody() { m_offsetTable.clear(); }
+  void setOffsetTable(MQMessageQueue mq, int64 offset);
+  std::map<MQMessageQueue, int64> getOffsetTable();
+  static ResetOffsetBody* Decode(const MemoryBlock* mem);
+
+ private:
+  std::map<MQMessageQueue, int64> m_offsetTable;
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/ResponseFuture.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/ResponseFuture.cpp b/rocketmq-cpp/src/transport/ResponseFuture.cpp
new file mode 100755
index 0000000..05cef84
--- /dev/null
+++ b/rocketmq-cpp/src/transport/ResponseFuture.cpp
@@ -0,0 +1,176 @@
+/*
+ * 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 "ResponseFuture.h"
+#include "Logging.h"
+#include "TcpRemotingClient.h"
+
+namespace rocketmq {
+//<!************************************************************************
+ResponseFuture::ResponseFuture(int requestCode, int opaque,
+                               TcpRemotingClient* powner, int64 timeout,
+                               bool bAsync /* = false */,
+                               AsyncCallbackWrap* pcall /* = NULL */) {
+  m_bAsync.store(bAsync);
+  m_requestCode = requestCode;
+  m_opaque = opaque;
+  m_timeout = timeout;
+  m_pCallbackWrap = pcall;
+  m_pResponseCommand = NULL;
+  m_sendRequestOK = false;
+  m_beginTimestamp = UtilAll::currentTimeMillis();
+  m_asyncCallbackStatus = asyncCallBackStatus_init;
+  if (getASyncFlag()) {
+    m_asyncResponse.store(false);
+    m_syncResponse.store(true);
+  } else {
+    m_asyncResponse.store(true);
+    m_syncResponse.store(false);
+  }
+}
+
+ResponseFuture::~ResponseFuture() {
+  deleteAndZero(m_pCallbackWrap);
+  /*
+    do not set m_pResponseCommand to NULL when destruct, as m_pResponseCommand
+    is used by MQClientAPIImpl concurrently, and will be released by producer or
+    consumer;
+    m_pResponseCommand = NULL;
+  */
+}
+
+void ResponseFuture::releaseThreadCondition() { m_defaultEvent.notify_all(); }
+
+RemotingCommand* ResponseFuture::waitResponse(int timeoutMillis) {
+  boost::unique_lock<boost::mutex> lk(m_defaultEventLock);
+  if (!m_defaultEvent.timed_wait(
+          lk, boost::posix_time::milliseconds(timeoutMillis))) {
+    LOG_WARN("waitResponse of code:%d with opaque:%d timeout", m_requestCode,
+             m_opaque);
+    m_syncResponse.store(true);
+  }
+  return m_pResponseCommand;
+}
+
+void ResponseFuture::setResponse(RemotingCommand* pResponseCommand) {
+  // LOG_DEBUG("setResponse of opaque:%d",m_opaque);
+  m_pResponseCommand = pResponseCommand;
+
+  if (!getASyncFlag()) {
+    if (m_syncResponse.load() == false) {
+      m_defaultEvent.notify_all();
+      m_syncResponse.store(true);
+    }
+  }
+}
+
+const bool ResponseFuture::getSyncResponseFlag() {
+  if (m_syncResponse.load() == true) {
+    return true;
+  }
+  return false;
+}
+
+const bool ResponseFuture::getAsyncResponseFlag() {
+  if (m_asyncResponse.load() == true) {
+    // LOG_DEBUG("ASYNC flag is TRUE,opaque is:%d",getOpaque() );
+    return true;
+  }
+
+  return false;
+}
+
+void ResponseFuture::setAsyncResponseFlag() { m_asyncResponse.store(true); }
+
+const bool ResponseFuture::getASyncFlag() {
+  if (m_bAsync.load() == true) {
+    // LOG_DEBUG("ASYNC flag is TRUE,opaque is:%d",getOpaque() );
+    return true;
+  }
+  return false;
+}
+
+bool ResponseFuture::isSendRequestOK() { return m_sendRequestOK; }
+
+void ResponseFuture::setSendRequestOK(bool sendRequestOK) {
+  m_sendRequestOK = sendRequestOK;
+}
+
+int ResponseFuture::getOpaque() const { return m_opaque; }
+
+int ResponseFuture::getRequestCode() const { return m_requestCode; }
+
+void ResponseFuture::setAsyncCallBackStatus(
+    asyncCallBackStatus asyncCallbackStatus) {
+  boost::lock_guard<boost::mutex> lock(m_asyncCallbackLock);
+  if (m_asyncCallbackStatus == asyncCallBackStatus_init) {
+    m_asyncCallbackStatus = asyncCallbackStatus;
+  }
+}
+
+void ResponseFuture::executeInvokeCallback() {
+  if (m_pCallbackWrap == NULL) {
+    deleteAndZero(m_pResponseCommand);
+    return;
+  } else {
+    if (m_asyncCallbackStatus == asyncCallBackStatus_response) {
+      m_pCallbackWrap->operationComplete(this, true);
+    } else {
+      if (m_pResponseCommand)
+        deleteAndZero(m_pResponseCommand);  // the responseCommand from
+                                            // RemotingCommand::Decode(mem) will
+                                            // only deleted by operationComplete
+                                            // automatically
+      LOG_WARN(
+          "timeout and response incoming concurrently of opaque:%d, and "
+          "executeInvokeCallbackException was called earlier",
+          m_opaque);
+    }
+  }
+}
+
+void ResponseFuture::executeInvokeCallbackException() {
+  if (m_pCallbackWrap == NULL) {
+    LOG_ERROR("m_pCallbackWrap is NULL, critical error");
+    return;
+  } else {
+    if (m_asyncCallbackStatus == asyncCallBackStatus_timeout) {
+      m_pCallbackWrap->onException();
+    } else {
+      LOG_WARN(
+          "timeout and response incoming concurrently of opaque:%d, and "
+          "executeInvokeCallback was called earlier",
+          m_opaque);
+    }
+  }
+}
+
+bool ResponseFuture::isTimeOut() const {
+  int64 diff = UtilAll::currentTimeMillis() - m_beginTimestamp;
+  //<!only async;
+  return m_bAsync.load() == 1 && diff > m_timeout;
+}
+
+RemotingCommand* ResponseFuture::getCommand() const {
+  return m_pResponseCommand;
+}
+
+AsyncCallbackWrap* ResponseFuture::getAsyncCallbackWrap() {
+  return m_pCallbackWrap;
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/ResponseFuture.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/ResponseFuture.h b/rocketmq-cpp/src/transport/ResponseFuture.h
new file mode 100755
index 0000000..92fa772
--- /dev/null
+++ b/rocketmq-cpp/src/transport/ResponseFuture.h
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __RESPONSEFUTURE_H__
+#define __RESPONSEFUTURE_H__
+#include <boost/atomic.hpp>
+#include <boost/thread/condition_variable.hpp>
+#include "AsyncCallbackWrap.h"
+#include "RemotingCommand.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+typedef enum asyncCallBackStatus {
+  asyncCallBackStatus_init = 0,
+  asyncCallBackStatus_response = 1,
+  asyncCallBackStatus_timeout = 2
+} asyncCallBackStatus;
+
+class TcpRemotingClient;
+//<!***************************************************************************
+class ResponseFuture {
+ public:
+  ResponseFuture(int requestCode, int opaque, TcpRemotingClient* powner,
+                 int64 timeoutMilliseconds, bool bAsync = false,
+                 AsyncCallbackWrap* pcall = NULL);
+  virtual ~ResponseFuture();
+  void releaseThreadCondition();
+  RemotingCommand* waitResponse(int timeoutMillis);
+  RemotingCommand* getCommand() const;
+
+  void setResponse(RemotingCommand* pResponseCommand);
+  bool isSendRequestOK();
+  void setSendRequestOK(bool sendRequestOK);
+  int getRequestCode() const;
+  int getOpaque() const;
+
+  //<!callback;
+  void executeInvokeCallback();
+  void executeInvokeCallbackException();
+  bool isTimeOut() const;
+  // bool    isTimeOutMoreThan30s() const;
+  const bool getASyncFlag();
+  void setAsyncResponseFlag();
+  const bool getAsyncResponseFlag();
+  const bool getSyncResponseFlag();
+  AsyncCallbackWrap* getAsyncCallbackWrap();
+  void setAsyncCallBackStatus(asyncCallBackStatus asyncCallbackStatus);
+
+ private:
+  int m_requestCode;
+  int m_opaque;
+  bool m_sendRequestOK;
+  boost::mutex m_defaultEventLock;
+  boost::condition_variable_any m_defaultEvent;
+  int64 m_beginTimestamp;
+  int64 m_timeout;  // ms
+  boost::atomic<bool> m_bAsync;
+  RemotingCommand* m_pResponseCommand;  //<!delete outside;
+  AsyncCallbackWrap* m_pCallbackWrap;
+  boost::mutex m_asyncCallbackLock;
+  asyncCallBackStatus m_asyncCallbackStatus;
+  boost::atomic<bool> m_asyncResponse;
+  boost::atomic<bool> m_syncResponse;
+  // TcpRemotingClient*    m_tcpRemoteClient;
+};
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/SocketUtil.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/SocketUtil.cpp b/rocketmq-cpp/src/transport/SocketUtil.cpp
new file mode 100755
index 0000000..d428f24
--- /dev/null
+++ b/rocketmq-cpp/src/transport/SocketUtil.cpp
@@ -0,0 +1,86 @@
+/*
+ * 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 "SocketUtil.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+sockaddr IPPort2socketAddress(int host, int port) {
+  struct sockaddr_in sa;
+  sa.sin_family = AF_INET;
+  sa.sin_port = htons((uint16)port);
+  sa.sin_addr.s_addr = htonl(host);
+
+  sockaddr bornAddr;
+  memcpy(&bornAddr, &sa, sizeof(sockaddr));
+  return bornAddr;
+}
+
+string socketAddress2IPPort(sockaddr addr) {
+  sockaddr_in sa;
+  memcpy(&sa, &addr, sizeof(sockaddr));
+
+  char tmp[32];
+  sprintf(tmp, "%s:%d", inet_ntoa(sa.sin_addr), ntohs(sa.sin_port));
+
+  string ipport = tmp;
+  return ipport;
+}
+
+void socketAddress2IPPort(sockaddr addr, int& host, int& port) {
+  struct sockaddr_in sa;
+  memcpy(&sa, &addr, sizeof(sockaddr));
+
+  host = ntohl(sa.sin_addr.s_addr);
+  port = ntohs(sa.sin_port);
+}
+
+string socketAddress2String(sockaddr addr) {
+  sockaddr_in in;
+  memcpy(&in, &addr, sizeof(sockaddr));
+
+  return inet_ntoa(in.sin_addr);
+}
+
+string getHostName(sockaddr addr) {
+  sockaddr_in in;
+  memcpy(&in, &addr, sizeof(sockaddr));
+
+  struct hostent* remoteHost = gethostbyaddr((char*)&(in.sin_addr), 4, AF_INET);
+  char** alias = remoteHost->h_aliases;
+  if (*alias != 0) {
+    return *alias;
+  } else {
+    return inet_ntoa(in.sin_addr);
+  }
+}
+
+uint64 swapll(uint64 v) {
+#ifdef ENDIANMODE_BIG
+  return v;
+#else
+  uint64 ret = ((v << 56) | ((v & 0xff00) << 40) | ((v & 0xff0000) << 24) |
+                ((v & 0xff000000) << 8) | ((v >> 8) & 0xff000000) |
+                ((v >> 24) & 0xff0000) | ((v >> 40) & 0xff00) | (v >> 56));
+
+  return ret;
+#endif
+}
+
+uint64 h2nll(uint64 v) { return swapll(v); }
+
+uint64 n2hll(uint64 v) { return swapll(v); }
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/transport/SocketUtil.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/transport/SocketUtil.h b/rocketmq-cpp/src/transport/SocketUtil.h
new file mode 100755
index 0000000..7cbba0c
--- /dev/null
+++ b/rocketmq-cpp/src/transport/SocketUtil.h
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __SOCKETUTIL_H__
+#define __SOCKETUTIL_H__
+
+#ifdef WIN32
+#include <WS2tcpip.h>
+#include <Windows.h>
+#include <Winsock2.h>
+#pragma comment(lib, "ws2_32.lib")
+#else
+#include <arpa/inet.h>
+#include <errno.h>
+#include <fcntl.h>
+#include <net/if.h>
+#include <netdb.h>
+#include <netinet/in.h>
+#include <netinet/tcp.h>
+#include <signal.h>
+#include <sys/ioctl.h>
+#include <sys/select.h>
+#include <sys/socket.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <unistd.h>
+#endif
+#include <sys/socket.h>
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!************************************************************************
+/**
+* IP:PORT
+*/
+sockaddr IPPort2socketAddress(int host, int port);
+string socketAddress2IPPort(sockaddr addr);
+void socketAddress2IPPort(sockaddr addr, int& host, int& port);
+
+string socketAddress2String(sockaddr addr);
+string getHostName(sockaddr addr);
+
+uint64 h2nll(uint64 v);
+uint64 n2hll(uint64 v);
+
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif


[03/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/MQProtos.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/MQProtos.h b/rocketmq-cpp/src/protocol/MQProtos.h
new file mode 100755
index 0000000..50c1841
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/MQProtos.h
@@ -0,0 +1,199 @@
+/*
+ * 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 __MQPROTOS_H__
+#define __MQPROTOS_H__
+
+namespace rocketmq {
+//<!***************************************************************************
+enum MQRequestCode {
+  // send msg to Broker
+  SEND_MESSAGE = 10,
+  // subscribe msg from Broker
+  PULL_MESSAGE = 11,
+  // query msg from Broker
+  QUERY_MESSAGE = 12,
+  // query Broker Offset
+  QUERY_BROKER_OFFSET = 13,
+  // query Consumer Offset from broker
+  QUERY_CONSUMER_OFFSET = 14,
+  // update Consumer Offset to broker
+  UPDATE_CONSUMER_OFFSET = 15,
+  // create or update Topic to broker
+  UPDATE_AND_CREATE_TOPIC = 17,
+  // get all topic config info from broker
+  GET_ALL_TOPIC_CONFIG = 21,
+  //git all topic list from broker
+  GET_TOPIC_CONFIG_LIST = 22,
+  //get topic name list from broker
+  GET_TOPIC_NAME_LIST = 23,
+  UPDATE_BROKER_CONFIG = 25,
+  GET_BROKER_CONFIG = 26,
+  TRIGGER_DELETE_FILES = 27,
+  GET_BROKER_RUNTIME_INFO = 28,
+  SEARCH_OFFSET_BY_TIMESTAMP = 29,
+  GET_MAX_OFFSET = 30,
+  GET_MIN_OFFSET = 31,
+  GET_EARLIEST_MSG_STORETIME = 32,
+  VIEW_MESSAGE_BY_ID = 33,
+  //send heartbeat to broker, and register itself
+  HEART_BEAT = 34,
+  //unregister client to broker
+  UNREGISTER_CLIENT = 35,
+  //send back consume fail msg to broker
+  CONSUMER_SEND_MSG_BACK = 36,
+  //Commit Or Rollback transaction
+  END_TRANSACTION = 37,
+  // get consumer list by group from broker
+  GET_CONSUMER_LIST_BY_GROUP = 38,
+
+  CHECK_TRANSACTION_STATE = 39,
+  //broker send notify to consumer when consumer lists changes
+  NOTIFY_CONSUMER_IDS_CHANGED = 40,
+  //lock mq before orderly consume
+  LOCK_BATCH_MQ = 41,
+  //unlock mq after orderly consume
+  UNLOCK_BATCH_MQ = 42,
+  GET_ALL_CONSUMER_OFFSET = 43,
+  GET_ALL_DELAY_OFFSET = 45,
+  PUT_KV_CONFIG = 100,
+  GET_KV_CONFIG = 101,
+  DELETE_KV_CONFIG = 102,
+  REGISTER_BROKER = 103,
+  UNREGISTER_BROKER = 104,
+  GET_ROUTEINTO_BY_TOPIC = 105,
+  GET_BROKER_CLUSTER_INFO = 106,
+  UPDATE_AND_CREATE_SUBSCRIPTIONGROUP = 200,
+  GET_ALL_SUBSCRIPTIONGROUP_CONFIG = 201,
+  GET_TOPIC_STATS_INFO = 202,
+  GET_CONSUMER_CONNECTION_LIST = 203,
+  GET_PRODUCER_CONNECTION_LIST = 204,
+  WIPE_WRITE_PERM_OF_BROKER = 205,
+
+  GET_ALL_TOPIC_LIST_FROM_NAMESERVER = 206,
+  DELETE_SUBSCRIPTIONGROUP = 207,
+  GET_CONSUME_STATS = 208,
+  SUSPEND_CONSUMER = 209,
+  RESUME_CONSUMER = 210,
+  RESET_CONSUMER_OFFSET_IN_CONSUMER = 211,
+  RESET_CONSUMER_OFFSET_IN_BROKER = 212,
+  ADJUST_CONSUMER_THREAD_POOL = 213,
+  WHO_CONSUME_THE_MESSAGE = 214,
+
+  DELETE_TOPIC_IN_BROKER = 215,
+  DELETE_TOPIC_IN_NAMESRV = 216,
+
+  GET_KV_CONFIG_BY_VALUE = 217,
+
+  DELETE_KV_CONFIG_BY_VALUE = 218,
+
+  GET_KVLIST_BY_NAMESPACE = 219,
+
+
+  RESET_CONSUMER_CLIENT_OFFSET = 220,
+
+  GET_CONSUMER_STATUS_FROM_CLIENT = 221,
+
+  INVOKE_BROKER_TO_RESET_OFFSET = 222,
+
+  INVOKE_BROKER_TO_GET_CONSUMER_STATUS = 223,
+
+  QUERY_TOPIC_CONSUME_BY_WHO = 300,
+
+  GET_TOPICS_BY_CLUSTER = 224,
+
+  REGISTER_FILTER_SERVER = 301,
+
+  REGISTER_MESSAGE_FILTER_CLASS = 302,
+
+  QUERY_CONSUME_TIME_SPAN = 303,
+
+  GET_SYSTEM_TOPIC_LIST_FROM_NS = 304,
+  GET_SYSTEM_TOPIC_LIST_FROM_BROKER = 305,
+
+  CLEAN_EXPIRED_CONSUMEQUEUE = 306,
+
+  GET_CONSUMER_RUNNING_INFO = 307,
+
+  QUERY_CORRECTION_OFFSET = 308,
+
+  CONSUME_MESSAGE_DIRECTLY = 309,
+
+  SEND_MESSAGE_V2 = 310,
+
+  GET_UNIT_TOPIC_LIST = 311,
+  GET_HAS_UNIT_SUB_TOPIC_LIST = 312,
+  GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST = 313,
+
+  CLONE_GROUP_OFFSET = 314,
+
+  VIEW_BROKER_STATS_DATA = 315
+};
+
+//<!***************************************************************************
+enum MQResponseCode {
+  //rcv success response from broker
+  SUCCESS_VALUE = 0,
+  //rcv exception from broker
+  SYSTEM_ERROR = 1,
+  //rcv symtem busy from broker
+  SYSTEM_BUSY = 2,
+  //broker don't support the request code
+  REQUEST_CODE_NOT_SUPPORTED = 3,
+  //broker flush disk timeout error
+  FLUSH_DISK_TIMEOUT = 10,
+  //broker sync double write, slave broker not available
+  SLAVE_NOT_AVAILABLE = 11,
+  //broker sync double write, slave broker flush msg timeout
+  FLUSH_SLAVE_TIMEOUT = 12,
+  //broker rcv illegal mesage
+  MESSAGE_ILLEGAL = 13,
+  //service not available due to broker or namesrv in shutdown status
+  SERVICE_NOT_AVAILABLE = 14,
+  //this version is not supported on broker or namesrv
+  VERSION_NOT_SUPPORTED = 15,
+  //broker or Namesrv has no permission to do this operation
+  NO_PERMISSION = 16,
+  //topic is not exist on broker
+  TOPIC_NOT_EXIST = 17,
+  //broker already created this topic
+  TOPIC_EXIST_ALREADY = 18,
+  //pulled msg was not found
+  PULL_NOT_FOUND = 19,
+  //retry later
+  PULL_RETRY_IMMEDIATELY = 20,
+  //pull msg with wrong offset
+  PULL_OFFSET_MOVED = 21,
+  //could not find the query msg
+  QUERY_NOT_FOUND = 22,
+
+  SUBSCRIPTION_PARSE_FAILED = 23,
+  SUBSCRIPTION_NOT_EXIST = 24,
+  SUBSCRIPTION_NOT_LATEST = 25,
+  SUBSCRIPTION_GROUP_NOT_EXIST = 26,
+
+  TRANSACTION_SHOULD_COMMIT = 200,
+  TRANSACTION_SHOULD_ROLLBACK = 201,
+  TRANSACTION_STATE_UNKNOW = 202,
+  TRANSACTION_STATE_GROUP_WRONG = 203,
+
+  CONSUMER_NOT_ONLINE = 206,
+  CONSUME_MSG_TIMEOUT = 207
+};
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/MessageQueue.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/MessageQueue.cpp b/rocketmq-cpp/src/protocol/MessageQueue.cpp
new file mode 100755
index 0000000..f1b3f8f
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/MessageQueue.cpp
@@ -0,0 +1,108 @@
+/*
+ * 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 "MessageQueue.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!************************************************************************
+MessageQueue::MessageQueue() {
+  m_queueId = -1;  // invalide mq
+  m_topic.clear();
+  m_brokerName.clear();
+}
+
+MessageQueue::MessageQueue(const string& topic, const string& brokerName,
+                           int queueId)
+    : m_topic(topic), m_brokerName(brokerName), m_queueId(queueId) {}
+
+MessageQueue::MessageQueue(const MessageQueue& other)
+    : m_topic(other.m_topic),
+      m_brokerName(other.m_brokerName),
+      m_queueId(other.m_queueId) {}
+
+MessageQueue& MessageQueue::operator=(const MessageQueue& other) {
+  if (this != &other) {
+    m_brokerName = other.m_brokerName;
+    m_topic = other.m_topic;
+    m_queueId = other.m_queueId;
+  }
+  return *this;
+}
+
+string MessageQueue::getTopic() const { return m_topic; }
+
+void MessageQueue::setTopic(const string& topic) { m_topic = topic; }
+
+string MessageQueue::getBrokerName() const { return m_brokerName; }
+
+void MessageQueue::setBrokerName(const string& brokerName) {
+  m_brokerName = brokerName;
+}
+
+int MessageQueue::getQueueId() const { return m_queueId; }
+
+void MessageQueue::setQueueId(int queueId) { m_queueId = queueId; }
+
+bool MessageQueue::operator==(const MessageQueue& mq) const {
+  if (this == &mq) {
+    return true;
+  }
+
+  if (m_brokerName != mq.m_brokerName) {
+    return false;
+  }
+
+  if (m_queueId != mq.m_queueId) {
+    return false;
+  }
+
+  if (m_topic != mq.m_topic) {
+    return false;
+  }
+
+  return true;
+}
+
+int MessageQueue::compareTo(const MessageQueue& mq) const {
+  int result = m_topic.compare(mq.m_topic);
+  if (result != 0) {
+    return result;
+  }
+
+  result = m_brokerName.compare(mq.m_brokerName);
+  if (result != 0) {
+    return result;
+  }
+
+  return m_queueId - mq.m_queueId;
+}
+
+bool MessageQueue::operator<(const MessageQueue& mq) const {
+  return compareTo(mq) < 0;
+}
+
+Json::Value MessageQueue::toJson() const {
+  Json::Value outJson;
+  outJson["topic"] = m_topic;
+  outJson["brokerName"] = m_brokerName;
+  outJson["queueId"] = m_queueId;
+  return outJson;
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/MessageQueue.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/MessageQueue.h b/rocketmq-cpp/src/protocol/MessageQueue.h
new file mode 100755
index 0000000..0d47bf8
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/MessageQueue.h
@@ -0,0 +1,56 @@
+/*
+ * 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 __MESSAGEQUEUE_H__
+#define __MESSAGEQUEUE_H__
+
+#include <string>
+#include "json/json.h"
+
+namespace rocketmq {
+//<!************************************************************************/
+//<!* MQ(T,B,ID);
+//<!************************************************************************/
+class MessageQueue {
+ public:
+  MessageQueue();
+  MessageQueue(const std::string& topic, const std::string& brokerName,
+               int queueId);
+  MessageQueue(const MessageQueue& other);
+  MessageQueue& operator=(const MessageQueue& other);
+
+  std::string getTopic() const;
+  void setTopic(const std::string& topic);
+
+  std::string getBrokerName() const;
+  void setBrokerName(const std::string& brokerName);
+
+  int getQueueId() const;
+  void setQueueId(int queueId);
+
+  bool operator==(const MessageQueue& mq) const;
+  bool operator<(const MessageQueue& mq) const;
+  int compareTo(const MessageQueue& mq) const;
+  Json::Value toJson() const;
+
+ private:
+  std::string m_topic;
+  std::string m_brokerName;
+  int m_queueId;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/ProcessQueueInfo.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/ProcessQueueInfo.h b/rocketmq-cpp/src/protocol/ProcessQueueInfo.h
new file mode 100644
index 0000000..d7493a5
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/ProcessQueueInfo.h
@@ -0,0 +1,86 @@
+#ifndef __PROCESSQUEUEINFO_H__
+#define __PROCESSQUEUEINFO_H__
+
+#include "UtilAll.h"
+#include "json/json.h"
+
+namespace rocketmq {
+class ProcessQueueInfo {
+ public:
+  ProcessQueueInfo() {
+    commitOffset = 0;
+    cachedMsgMinOffset = 0;
+    cachedMsgMaxOffset = 0;
+    cachedMsgCount = 0;
+    transactionMsgMinOffset = 0;
+    transactionMsgMaxOffset = 0;
+    transactionMsgCount = 0;
+    locked = false;
+    tryUnlockTimes = 0;
+    lastLockTimestamp = 123;
+    droped = false;
+    lastPullTimestamp = 0;
+    lastConsumeTimestamp = 0;
+  }
+  virtual ~ProcessQueueInfo() {}
+
+ public:
+  const uint64 getCommitOffset() const { return commitOffset; }
+
+  void setCommitOffset(uint64 input_commitOffset) {
+    commitOffset = input_commitOffset;
+  }
+
+  void setLocked(bool in_locked) { locked = in_locked; }
+
+  const bool isLocked() const { return locked; }
+
+  void setDroped(bool in_dropped) { droped = in_dropped; }
+
+  const bool isDroped() const { return droped; }
+
+  Json::Value toJson() const {
+    Json::Value outJson;
+    outJson["commitOffset"] = (UtilAll::to_string(commitOffset)).c_str();
+    outJson["cachedMsgMinOffset"] =
+        (UtilAll::to_string(cachedMsgMinOffset)).c_str();
+    outJson["cachedMsgMaxOffset"] =
+        (UtilAll::to_string(cachedMsgMaxOffset)).c_str();
+    outJson["cachedMsgCount"] = (int)(cachedMsgCount);
+    outJson["transactionMsgMinOffset"] =
+        (UtilAll::to_string(transactionMsgMinOffset)).c_str();
+    outJson["transactionMsgMaxOffset"] =
+        (UtilAll::to_string(transactionMsgMaxOffset)).c_str();
+    outJson["transactionMsgCount"] = (int)(transactionMsgCount);
+    outJson["locked"] = (locked);
+    outJson["tryUnlockTimes"] = (int)(tryUnlockTimes);
+    outJson["lastLockTimestamp"] =
+        (UtilAll::to_string(lastLockTimestamp)).c_str();
+    outJson["droped"] = (droped);
+    outJson["lastPullTimestamp"] =
+        (UtilAll::to_string(lastPullTimestamp)).c_str();
+    outJson["lastConsumeTimestamp"] =
+        (UtilAll::to_string(lastConsumeTimestamp)).c_str();
+
+    return outJson;
+  }
+
+ public:
+  uint64 commitOffset;
+  uint64 cachedMsgMinOffset;
+  uint64 cachedMsgMaxOffset;
+  int cachedMsgCount;
+  uint64 transactionMsgMinOffset;
+  uint64 transactionMsgMaxOffset;
+  int transactionMsgCount;
+  bool locked;
+  int tryUnlockTimes;
+  uint64 lastLockTimestamp;
+
+  bool droped;
+  uint64 lastPullTimestamp;
+  uint64 lastConsumeTimestamp;
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/RemotingCommand.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/RemotingCommand.cpp b/rocketmq-cpp/src/protocol/RemotingCommand.cpp
new file mode 100644
index 0000000..ff6e53e
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/RemotingCommand.cpp
@@ -0,0 +1,253 @@
+/*
+ * 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 "RemotingCommand.h"
+#include "ByteOrder.h"
+#include "Logging.h"
+#include "MQProtos.h"
+#include "MQVersion.h"
+#include "SessionCredentials.h"
+
+namespace rocketmq {
+boost::atomic<int> RemotingCommand::s_seqNumber;
+boost::mutex RemotingCommand::m_clock;
+//<!************************************************************************
+RemotingCommand::RemotingCommand(int code,
+                                 CommandHeader* pExtHeader /* = NULL */)
+    : m_code(code),
+      m_language("CPP"),
+      m_version(MQVersion::s_CurrentVersion),
+      m_flag(0),
+      m_remark(""),
+      m_pExtHeader(pExtHeader) {
+  boost::lock_guard<boost::mutex> lock(m_clock);
+  m_opaque = (s_seqNumber.load(boost::memory_order_acquire)) %
+             (numeric_limits<int>::max());
+  s_seqNumber.store(m_opaque, boost::memory_order_release);
+  ++s_seqNumber;
+}
+
+RemotingCommand::RemotingCommand(int code, string language, int version,
+                                 int opaque, int flag, string remark,
+                                 CommandHeader* pExtHeader)
+    : m_code(code),
+      m_language(language),
+      m_version(version),
+      m_opaque(opaque),
+      m_flag(flag),
+      m_remark(remark),
+      m_pExtHeader(pExtHeader) {}
+
+RemotingCommand::~RemotingCommand() { m_pExtHeader = NULL; }
+
+void RemotingCommand::Encode() {
+  Json::Value root;
+  root["code"] = m_code;
+  root["language"] = "CPP";
+  root["version"] = m_version;
+  root["opaque"] = m_opaque;
+  root["flag"] = m_flag;
+  root["remark"] = m_remark;
+
+  if (m_pExtHeader) {
+    Json::Value extJson;
+    m_pExtHeader->Encode(extJson);
+
+    extJson[SessionCredentials::Signature] =
+        m_extFields[SessionCredentials::Signature];
+    extJson[SessionCredentials::AccessKey] =
+        m_extFields[SessionCredentials::AccessKey];
+    extJson[SessionCredentials::ONSChannelKey] =
+        m_extFields[SessionCredentials::ONSChannelKey];
+
+    root["extFields"] = extJson;
+  } else {  // for heartbeat
+    Json::Value extJson;
+    extJson[SessionCredentials::Signature] =
+        m_extFields[SessionCredentials::Signature];
+    extJson[SessionCredentials::AccessKey] =
+        m_extFields[SessionCredentials::AccessKey];
+    extJson[SessionCredentials::ONSChannelKey] =
+        m_extFields[SessionCredentials::ONSChannelKey];
+    root["extFields"] = extJson;
+  }
+
+  Json::FastWriter fastwrite;
+  string data = fastwrite.write(root);
+
+  uint32 headLen = data.size();
+  uint32 totalLen = 4 + headLen + m_body.getSize();
+
+  uint32 messageHeader[2];
+  messageHeader[0] = ByteOrder::swapIfLittleEndian(totalLen);
+  messageHeader[1] = ByteOrder::swapIfLittleEndian(headLen);
+
+  //<!include self 4 bytes, see : doc/protocol.txt;
+  m_head.setSize(4 + 4 + headLen);
+  m_head.copyFrom(messageHeader, 0, sizeof(messageHeader));
+  m_head.copyFrom(data.c_str(), sizeof(messageHeader), headLen);
+}
+
+const MemoryBlock* RemotingCommand::GetHead() const { return &m_head; }
+
+const MemoryBlock* RemotingCommand::GetBody() const { return &m_body; }
+
+void RemotingCommand::SetBody(const char* pData, int len) {
+  m_body.reset();
+  m_body.setSize(len);
+  m_body.copyFrom(pData, 0, len);
+}
+
+RemotingCommand* RemotingCommand::Decode(const MemoryBlock& mem) {
+  //<!decode 1 bytes,4+head+body
+  uint32 messageHeader[1];
+  mem.copyTo(messageHeader, 0, sizeof(messageHeader));
+  int totalLen = mem.getSize();
+  int headLen = ByteOrder::swapIfLittleEndian(messageHeader[0]);
+  int bodyLen = totalLen - 4 - headLen;
+
+  //<!decode header;
+  const char* const pData = static_cast<const char*>(mem.getData());
+  Json::Reader reader;
+  Json::Value object;
+  const char* begin = pData + 4;
+  const char* end = pData + 4 + headLen;
+
+  if (!reader.parse(begin, end, object)) {
+    THROW_MQEXCEPTION(MQClientException, "conn't parse json", -1);
+  }
+
+  int code = object["code"].asInt();
+
+  string language = object["language"].asString();
+  int version = object["version"].asInt();
+  int opaque = object["opaque"].asInt();
+  int flag = object["flag"].asInt();
+  Json::Value v = object["remark"];
+  string remark = "";
+  if (!v.isNull()) {
+    remark = object["remark"].asString();
+  }
+  LOG_DEBUG("code:%d, remark:%s, version:%d, opaque:%d, flag:%d, remark:%s, headLen:%d, bodyLen:%d ",
+            code, language.c_str(), version, opaque, flag, remark.c_str(), headLen, bodyLen);
+  RemotingCommand* cmd =
+      new RemotingCommand(code, language, version, opaque, flag, remark, NULL);
+  cmd->setParsedJson(object);
+  if (bodyLen > 0) {
+    cmd->SetBody(pData + 4 + headLen, bodyLen);
+  }
+  return cmd;
+}
+
+void RemotingCommand::markResponseType() {
+  int bits = 1 << RPC_TYPE;
+  m_flag |= bits;
+}
+
+bool RemotingCommand::isResponseType() {
+  int bits = 1 << RPC_TYPE;
+  return (m_flag & bits) == bits;
+}
+
+void RemotingCommand::markOnewayRPC() {
+  int bits = 1 << RPC_ONEWAY;
+  m_flag |= bits;
+}
+
+bool RemotingCommand::isOnewayRPC() {
+  int bits = 1 << RPC_ONEWAY;
+  return (m_flag & bits) == bits;
+}
+
+void RemotingCommand::setOpaque(const int opa) { m_opaque = opa; }
+
+void RemotingCommand::SetExtHeader(int code) {
+  try {
+    Json::Value ext = m_parsedJson["extFields"];
+    if (!ext.isNull()) {
+      m_pExtHeader = NULL;
+      switch (code) {
+        case SEND_MESSAGE:
+          m_pExtHeader.reset(SendMessageResponseHeader::Decode(ext));
+          break;
+        case PULL_MESSAGE:
+          m_pExtHeader.reset(PullMessageResponseHeader::Decode(ext));
+          break;
+        case GET_MIN_OFFSET:
+          m_pExtHeader.reset(GetMinOffsetResponseHeader::Decode(ext));
+          break;
+        case GET_MAX_OFFSET:
+          m_pExtHeader.reset(GetMaxOffsetResponseHeader::Decode(ext));
+          break;
+        case SEARCH_OFFSET_BY_TIMESTAMP:
+          m_pExtHeader.reset(SearchOffsetResponseHeader::Decode(ext));
+          break;
+        case GET_EARLIEST_MSG_STORETIME:
+          m_pExtHeader.reset(
+              GetEarliestMsgStoretimeResponseHeader::Decode(ext));
+          break;
+        case QUERY_CONSUMER_OFFSET:
+          m_pExtHeader.reset(QueryConsumerOffsetResponseHeader::Decode(ext));
+          break;
+        case RESET_CONSUMER_CLIENT_OFFSET:
+          m_pExtHeader.reset(ResetOffsetRequestHeader::Decode(ext));
+          break;
+        case GET_CONSUMER_RUNNING_INFO:
+          m_pExtHeader.reset(GetConsumerRunningInfoRequestHeader::Decode(ext));
+          break;
+        case NOTIFY_CONSUMER_IDS_CHANGED:
+          m_pExtHeader.reset(
+              NotifyConsumerIdsChangedRequestHeader::Decode(ext));
+        default:
+          break;
+      }
+    }
+  } catch (MQException& e) {
+    LOG_ERROR("set response head error");
+  }
+}
+
+void RemotingCommand::setCode(int code) { m_code = code; }
+
+int RemotingCommand::getCode() const { return m_code; }
+
+int RemotingCommand::getOpaque() const { return m_opaque; }
+
+string RemotingCommand::getRemark() const { return m_remark; }
+
+void RemotingCommand::setRemark(string mark) { m_remark = mark; }
+
+CommandHeader* RemotingCommand::getCommandHeader() const {
+  return m_pExtHeader.get();
+}
+
+void RemotingCommand::setParsedJson(Json::Value json) {
+  m_parsedJson = json;
+}
+
+const int RemotingCommand::getFlag() const { return m_flag; }
+
+const int RemotingCommand::getVersion() const { return m_version; }
+
+void RemotingCommand::setMsgBody(const string& body) { m_msgBody = body; }
+
+string RemotingCommand::getMsgBody() const { return m_msgBody; }
+
+void RemotingCommand::addExtField(const string& key, const string& value) {
+  m_extFields[key] = value;
+}
+
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/RemotingCommand.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/RemotingCommand.h b/rocketmq-cpp/src/protocol/RemotingCommand.h
new file mode 100755
index 0000000..633a511
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/RemotingCommand.h
@@ -0,0 +1,90 @@
+/*
+ * 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 __REMOTINGCOMMAND_H__
+#define __REMOTINGCOMMAND_H__
+#include <boost/atomic.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include <memory>
+#include <sstream>
+#include "CommandHeader.h"
+#include "dataBlock.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+const int RPC_TYPE = 0;    // 0, REQUEST_COMMAND // 1, RESPONSE_COMMAND;
+const int RPC_ONEWAY = 1;  // 0, RPC // 1, Oneway;
+//<!***************************************************************************
+class RemotingCommand {
+ public:
+  RemotingCommand(int code, CommandHeader* pCustomHeader = NULL);
+  RemotingCommand(int code, string language, int version, int opaque, int flag,
+                  string remark, CommandHeader* pCustomHeader);
+  virtual ~RemotingCommand();
+
+  const MemoryBlock* GetHead() const;
+  const MemoryBlock* GetBody() const;
+
+  void SetBody(const char* pData, int len);
+  void setOpaque(const int opa);
+  void SetExtHeader(int code);
+
+  void setCode(int code);
+  int getCode() const;
+  int getOpaque() const;
+  void setRemark(string mark);
+  string getRemark() const;
+  void markResponseType();
+  bool isResponseType();
+  void markOnewayRPC();
+  bool isOnewayRPC();
+  void setParsedJson(Json::Value json);
+
+  CommandHeader* getCommandHeader() const;
+  const int getFlag() const;
+  const int getVersion() const;
+
+  void addExtField(const string& key, const string& value);
+  string getMsgBody() const;
+  void setMsgBody(const string& body);
+
+ public:
+  void Encode();
+  static RemotingCommand* Decode(const MemoryBlock& mem);
+
+ private:
+  int m_code;
+  string m_language;
+  int m_version;
+  int m_opaque;
+  int m_flag;
+  string m_remark;
+  string m_msgBody;
+  map<string, string> m_extFields;
+
+  static boost::mutex m_clock;
+  MemoryBlock m_head;
+  MemoryBlock m_body;
+  //<!save here
+  Json::Value m_parsedJson;
+  static boost::atomic<int> s_seqNumber;
+  unique_ptr<CommandHeader> m_pExtHeader;
+};
+
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/RemotingSerializable.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/RemotingSerializable.h b/rocketmq-cpp/src/protocol/RemotingSerializable.h
new file mode 100755
index 0000000..812a892
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/RemotingSerializable.h
@@ -0,0 +1,32 @@
+/*
+ * 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 __REMOTINGSERIALIZABLE_H__
+#define __REMOTINGSERIALIZABLE_H__
+#include "json/json.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class RemotingSerializable {
+ public:
+  virtual ~RemotingSerializable(){};
+  virtual void Encode(std::string& outData) = 0;
+};
+
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/TopicList.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/TopicList.h b/rocketmq-cpp/src/protocol/TopicList.h
new file mode 100755
index 0000000..d8d14a7
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/TopicList.h
@@ -0,0 +1,35 @@
+/*
+ * 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 __TOPICLIST_H__
+#define __TOPICLIST_H__
+#include <string>
+#include <vector>
+#include "dataBlock.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class TopicList {
+ public:
+  static TopicList* Decode(const MemoryBlock* mem) { return new TopicList(); }
+
+ private:
+  vector<string> m_topicList;
+};
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/protocol/TopicRouteData.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/protocol/TopicRouteData.h b/rocketmq-cpp/src/protocol/TopicRouteData.h
new file mode 100755
index 0000000..ec8f842
--- /dev/null
+++ b/rocketmq-cpp/src/protocol/TopicRouteData.h
@@ -0,0 +1,177 @@
+/*
+ * 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 __TOPICROUTEDATA_H__
+#define __TOPICROUTEDATA_H__
+#include <algorithm>
+#include "Logging.h"
+#include "UtilAll.h"
+#include "dataBlock.h"
+#include "json/json.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+struct QueueData {
+  string brokerName;
+  int readQueueNums;
+  int writeQueueNums;
+  int perm;
+
+  bool operator<(const QueueData& other) const {
+    return brokerName < other.brokerName;
+  }
+
+  bool operator==(const QueueData& other) const {
+    if (brokerName == other.brokerName &&
+        readQueueNums == other.readQueueNums &&
+        writeQueueNums == other.writeQueueNums && perm == other.perm) {
+      return true;
+    }
+    return false;
+  }
+};
+
+//<!***************************************************************************
+struct BrokerData {
+  string brokerName;
+  map<int, string> brokerAddrs;  //<!0:master,1,2.. slave
+
+  bool operator<(const BrokerData& other) const {
+    return brokerName < other.brokerName;
+  }
+
+  bool operator==(const BrokerData& other) const {
+    if (brokerName == other.brokerName && brokerAddrs == other.brokerAddrs) {
+      return true;
+    }
+    return false;
+  }
+};
+
+//<!************************************************************************/
+class TopicRouteData {
+ public:
+  virtual ~TopicRouteData() {
+    m_brokerDatas.clear();
+    m_queueDatas.clear();
+  }
+
+  static TopicRouteData* Decode(const MemoryBlock* mem) {
+    //<!see doc/TopicRouteData.json;
+    const char* const pData = static_cast<const char*>(mem->getData());
+    string data(pData, mem->getSize());
+    
+    Json::Value root;
+    Json::CharReaderBuilder charReaderBuilder;
+    charReaderBuilder.settings_["allowNumericKeys"] = true;
+    unique_ptr<Json::CharReader> pCharReaderPtr(charReaderBuilder.newCharReader());
+    const char* begin = pData;
+    const char* end = pData + mem->getSize(); 
+    string errs;
+    if (!pCharReaderPtr->parse(begin, end, &root, &errs)) {
+      LOG_ERROR("parse json error:%s, value isArray:%d, isObject:%d", errs.c_str(), root.isArray(), root.isObject());
+      return NULL;
+    }
+
+    TopicRouteData* trd = new TopicRouteData();
+    trd->setOrderTopicConf(root["orderTopicConf"].asString());
+
+    Json::Value qds = root["queueDatas"];
+    for (unsigned int i = 0; i < qds.size(); i++) {
+      QueueData d;
+      Json::Value qd = qds[i];
+      d.brokerName = qd["brokerName"].asString();
+      d.readQueueNums = qd["readQueueNums"].asInt();
+      d.writeQueueNums = qd["writeQueueNums"].asInt();
+      d.perm = qd["perm"].asInt();
+
+      trd->getQueueDatas().push_back(d);
+    }
+
+    sort(trd->getQueueDatas().begin(), trd->getQueueDatas().end());
+
+    Json::Value bds = root["brokerDatas"];
+    for (unsigned int i = 0; i < bds.size(); i++) {
+      BrokerData d;
+      Json::Value bd = bds[i];
+      d.brokerName = bd["brokerName"].asString();
+
+      LOG_DEBUG("brokerName:%s", d.brokerName.c_str());
+
+      Json::Value bas = bd["brokerAddrs"];
+      Json::Value::Members mbs = bas.getMemberNames();
+      for (size_t i = 0; i < mbs.size(); i++) {
+        string key = mbs.at(i);
+        LOG_DEBUG("brokerid:%s,brokerAddr:%s", key.c_str(),
+                  bas[key].asString().c_str());
+        d.brokerAddrs[atoi(key.c_str())] = bas[key].asString();
+      }
+
+      trd->getBrokerDatas().push_back(d);
+    }
+
+    sort(trd->getBrokerDatas().begin(), trd->getBrokerDatas().end());
+
+    return trd;
+  }
+
+  string selectBrokerAddr() {
+    vector<BrokerData>::iterator it = m_brokerDatas.begin();
+    for (; it != m_brokerDatas.end(); ++it) {
+      map<int, string>::iterator it1 = (*it).brokerAddrs.find(MASTER_ID);
+      if (it1 != (*it).brokerAddrs.end()) {
+        return it1->second;
+      }
+    }
+    return "";
+  }
+
+
+  vector<QueueData>& getQueueDatas() { return m_queueDatas; }
+
+  vector<BrokerData>& getBrokerDatas() { return m_brokerDatas; }
+
+  const string& getOrderTopicConf() const { return m_orderTopicConf; }
+
+  void setOrderTopicConf(const string& orderTopicConf) {
+    m_orderTopicConf = orderTopicConf;
+  }
+
+  bool operator==(const TopicRouteData& other) const {
+    if (m_brokerDatas != other.m_brokerDatas) {
+      return false;
+    }
+
+    if (m_orderTopicConf != other.m_orderTopicConf) {
+      return false;
+    }
+
+    if (m_queueDatas != other.m_queueDatas) {
+      return false;
+    }
+    return true;
+  }
+
+ public:
+ private:
+  string m_orderTopicConf;
+  vector<QueueData> m_queueDatas;
+  vector<BrokerData> m_brokerDatas;
+};
+
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/batch_descriptor.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/batch_descriptor.h b/rocketmq-cpp/src/thread/disruptor/batch_descriptor.h
new file mode 100755
index 0000000..ba1a035
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/batch_descriptor.h
@@ -0,0 +1,70 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_BATCH_DESCRIPTOR_H_  // NOLINT
+#define DISRUPTOR_BATCH_DESCRIPTOR_H_  // NOLINT
+
+#include "sequence.h"
+
+namespace rocketmq {
+
+// Used to record the batch of sequences claimed via {@link Sequencer}.
+class BatchDescriptor {
+ public:
+    // Create a holder for tracking a batch of claimed sequences in a
+    // {@link Sequencer}
+    //
+    // @param size of the batch to claim.
+    BatchDescriptor(int size) :
+        size_(size),
+        end_(kInitialCursorValue) {}
+
+    // Get the size of the batch
+    int size() const { return size_; }
+
+    // Get the end sequence of a batch.
+    //
+    // @return the end sequence in the batch.
+    int64_t end() const { return end_; }
+
+    // Set the end sequence of a batch.
+    //
+    // @param end sequence in the batch.
+    void set_end(int64_t end) { end_ = end; }
+
+
+    // Get the starting sequence of the batch.
+    //
+    // @return starting sequence in the batch.
+    int64_t Start() const { return end_ - size_ + 1L; }
+
+ private:
+    int size_;
+    int64_t end_;
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_SEQUENCE_BATCH_H_  NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/claim_strategy.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/claim_strategy.h b/rocketmq-cpp/src/thread/disruptor/claim_strategy.h
new file mode 100755
index 0000000..0f3263a
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/claim_strategy.h
@@ -0,0 +1,231 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_CLAIM_STRATEGY_H_ // NOLINT
+#define DISRUPTOR_CLAIM_STRATEGY_H_ // NOLINT
+
+#include <boost/thread.hpp>
+#include <boost/noncopyable.hpp>
+#include <vector>
+
+#include "interface.h"
+
+namespace rocketmq {
+
+enum ClaimStrategyOption {
+    kSingleThreadedStrategy,
+    kMultiThreadedStrategy
+};
+
+// Optimised strategy can be used when there is a single publisher thread
+// claiming {@link AbstractEvent}s.
+class SingleThreadedStrategy :public noncopyable,  public ClaimStrategyInterface {
+ public:
+    SingleThreadedStrategy(const int& buffer_size) :
+        buffer_size_(buffer_size),
+        sequence_(kInitialCursorValue),
+        min_gating_sequence_(kInitialCursorValue) {}
+
+    virtual int64_t IncrementAndGet(
+            const std::vector<Sequence*>& dependent_sequences) {
+        int64_t next_sequence = sequence_.IncrementAndGet(1L);
+        WaitForFreeSlotAt(next_sequence, dependent_sequences);
+        return next_sequence;
+    }
+
+    virtual int64_t IncrementAndGet(const int& delta,
+            const std::vector<Sequence*>& dependent_sequences) {
+        int64_t next_sequence = sequence_.IncrementAndGet(delta);
+        WaitForFreeSlotAt(next_sequence, dependent_sequences);
+        return next_sequence;
+    }
+
+    virtual bool HasAvalaibleCapacity(
+            const std::vector<Sequence*>& dependent_sequences) {
+        int64_t wrap_point = sequence_.sequence() + 1L - buffer_size_;
+        if (wrap_point > min_gating_sequence_.sequence()) {
+            int64_t min_sequence = GetMinimumSequence(dependent_sequences);
+            min_gating_sequence_.set_sequence(min_sequence);
+            if (wrap_point > min_sequence)
+                return false;
+        }
+        return true;
+    }
+
+    virtual void SetSequence(const int64_t& sequence,
+            const std::vector<Sequence*>& dependent_sequences) {
+        sequence_.set_sequence(sequence);
+        WaitForFreeSlotAt(sequence, dependent_sequences);
+    }
+
+    virtual void SerialisePublishing(const int64_t& sequence,
+                                     const Sequence& cursor,
+                                     const int64_t& batch_size) {}
+
+ private:
+    SingleThreadedStrategy();
+
+    void WaitForFreeSlotAt(const int64_t& sequence,
+            const std::vector<Sequence*>& dependent_sequences) {
+        int64_t wrap_point = sequence - buffer_size_;
+        if (wrap_point > min_gating_sequence_.sequence()) {
+            int64_t min_sequence;
+            while (wrap_point > (min_sequence = GetMinimumSequence(dependent_sequences))) {
+                boost::this_thread::yield();
+            }
+        }
+    }
+
+    const int buffer_size_;
+    PaddedLong sequence_;
+    PaddedLong min_gating_sequence_;
+
+};
+
+// Strategy to be used when there are multiple publisher threads claiming
+// {@link AbstractEvent}s.
+/*
+class MultiThreadedStrategy :  public ClaimStrategyInterface {
+ public:
+    MultiThreadedStrategy(const int& buffer_size) :
+        buffer_size_(buffer_size),
+        sequence_(kInitialCursorValue),
+        min_processor_sequence_(kInitialCursorValue) {}
+
+    virtual int64_t IncrementAndGet(
+            const std::vector<Sequence*>& dependent_sequences) {
+        WaitForCapacity(dependent_sequences, min_gating_sequence_local_);
+        int64_t next_sequence = sequence_.IncrementAndGet();
+        WaitForFreeSlotAt(next_sequence,
+                          dependent_sequences,
+                          min_gating_sequence_local_);
+        return next_sequence;
+    }
+
+    virtual int64_t IncrementAndGet(const int& delta,
+            const std::vector<Sequence*>& dependent_sequences) {
+        int64_t next_sequence = sequence_.IncrementAndGet(delta);
+        WaitForFreeSlotAt(next_sequence,
+                          dependent_sequences,
+                          min_gating_sequence_local_);
+        return next_sequence;
+    }
+    virtual void SetSequence(const int64_t& sequence,
+            const std::vector<Sequence*>& dependent_sequences) {
+        sequence_.set_sequence(sequence);
+        WaitForFreeSlotAt(sequence,
+                          dependent_sequences,
+                          min_gating_sequence_local_);
+    }
+
+    virtual bool HasAvalaibleCapacity(
+            const std::vector<Sequence*>& dependent_sequences) {
+        const int64_t wrap_point = sequence_.sequence() + 1L - buffer_size_;
+        if (wrap_point > min_gating_sequence_local_.sequence()) {
+            int64_t min_sequence = GetMinimumSequence(dependent_sequences);
+            min_gating_sequence_local_.set_sequence(min_sequence);
+            if (wrap_point > min_sequence)
+                return false;
+        }
+        return true;
+    }
+
+    virtual void SerialisePublishing(const Sequence& cursor,
+                                     const int64_t& sequence,
+                                     const int64_t& batch_size) {
+        int64_t expected_sequence = sequence - batch_size;
+        int counter = retries;
+
+        while (expected_sequence != cursor.sequence()) {
+            if (0 == --counter) {
+                counter = retries;
+                std::this_thread::yield();
+            }
+        }
+    }
+
+ private:
+    // Methods
+    void WaitForCapacity(const std::vector<Sequence*>& dependent_sequences,
+                         const MutableLong& min_gating_sequence) {
+        const int64_t wrap_point = sequence_.sequence() + 1L - buffer_size_;
+        if (wrap_point > min_gating_sequence.sequence()) {
+            int counter = retries;
+            int64_t min_sequence;
+            while (wrap_point > (min_sequence = GetMinimumSequence(dependent_sequences))) {
+                counter = ApplyBackPressure(counter);
+            }
+            min_gating_sequence.set_sequence(min_sequence);
+        }
+    }
+
+    void WaitForFreeSlotAt(const int64_t& sequence,
+                           const std::vector<Sequence*>& dependent_sequences,
+                           const MutableLong& min_gating_sequence) {
+        const int64_t wrap_point = sequence - buffer_size_;
+        if (wrap_point > min_gating_sequence.sequence()) {
+            int64_t min_sequence;
+            while (wrap_point > (min_sequence = GetMinimumSequence(dependent_sequences))) {
+                std::this_thread::yield();
+            }
+            min_gating_sequence.set_sequence(min_sequence);
+        }
+    }
+
+    int ApplyBackPressure(int counter) {
+        if (0 != counter) {
+            --counter;
+            std::this_thread::yield();
+        } else {
+            std::this_thread::sleep_for(std::chrono::milliseconds(1));
+        }
+
+        return counter;
+    }
+
+    const int buffer_size_;
+    PaddedSequence sequence_;
+    thread_local PaddedLong min_gating_sequence_local_;
+
+    const int retries = 100;
+
+};
+*/
+
+ClaimStrategyInterface* CreateClaimStrategy(ClaimStrategyOption option,
+                                            const int& buffer_size) {
+    switch (option) {
+        case kSingleThreadedStrategy:
+            return new SingleThreadedStrategy(buffer_size);
+        // case kMultiThreadedStrategy:
+        //     return new MultiThreadedStrategy(buffer_size);
+        default:
+            return NULL;
+    }
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_CLAIM_STRATEGY_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/event_processor.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/event_processor.h b/rocketmq-cpp/src/thread/disruptor/event_processor.h
new file mode 100755
index 0000000..fb62812
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/event_processor.h
@@ -0,0 +1,130 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_EVENT_PROCESSOR_H_ // NOLINT
+#define DISRUPTOR_EVENT_PROCESSOR_H_ // NOLINT
+
+#include <stdexcept>
+#include "ring_buffer.h"
+
+namespace rocketmq {
+
+template <typename T>
+class NoOpEventProcessor : public EventProcessorInterface<T> {
+ public:
+    NoOpEventProcessor(RingBuffer<T>* ring_buffer) :
+        ring_buffer_(ring_buffer) { }
+
+    virtual Sequence* GetSequence() {
+        return ring_buffer_->GetSequencePtr();
+    }
+
+    virtual void Halt() {}
+
+    virtual void Run() {}
+
+ private:
+    RingBuffer<T>* ring_buffer_;
+};
+
+template <typename T>
+class BatchEventProcessor : public boost::noncopyable, public EventProcessorInterface<T> {
+ public:
+    BatchEventProcessor(RingBuffer<T>* ring_buffer,
+                        SequenceBarrierInterface* sequence_barrier,
+                        EventHandlerInterface<T>* event_handler,
+                        ExceptionHandlerInterface<T>* exception_handler) :
+            running_(false),
+            ring_buffer_(ring_buffer),
+            sequence_barrier_(sequence_barrier),
+            event_handler_(event_handler),
+            exception_handler_(exception_handler) {}
+
+
+    virtual Sequence* GetSequence() { return &sequence_; }
+
+    virtual void Halt() {
+        running_.store(false);
+        sequence_barrier_->Alert();
+    }
+
+    virtual void Run() {
+        if (running_.load())
+         {
+            printf("Thread is already running\r\n");
+         }
+        running_.store(true);
+        sequence_barrier_->ClearAlert();
+        event_handler_->OnStart();
+
+        T* event = NULL;
+        int64_t next_sequence = sequence_.sequence() + 1L;
+
+        while (true) {
+            try {
+                int64_t avalaible_sequence = \
+                    sequence_barrier_->WaitFor(next_sequence, 300*1000);//wait 300 milliseconds to avoid taskThread blocking on BlockingStrategy::WaitFor when shutdown
+                //metaq::LOG_INFO("avalaible_sequence:%d, next_sequence:%d", avalaible_sequence,next_sequence);
+                while (next_sequence <= avalaible_sequence) {
+                    event = ring_buffer_->Get(next_sequence);
+                    event_handler_->OnEvent(next_sequence,
+                            next_sequence == avalaible_sequence, event);
+                    next_sequence++;
+                }
+
+                sequence_.set_sequence(next_sequence - 1L);
+            } catch(const AlertException& e) {
+                //metaq::LOG_INFO("catch alertException");
+                if (!running_.load())
+                    break;
+            } catch(const std::exception& e) {
+                //metaq::LOG_ERROR("catch stdException");
+                exception_handler_->Handle(e, next_sequence, event);
+                sequence_.set_sequence(next_sequence);
+                next_sequence++;
+            }
+        }
+        //metaq::LOG_INFO("BatchEventProcessor shutdown");
+        event_handler_->OnShutdown();
+        running_.store(false);
+    }
+
+    void operator()() { Run(); }
+
+ private:
+    boost::atomic<bool> running_;
+    Sequence sequence_;
+
+    RingBuffer<T>* ring_buffer_;
+    SequenceBarrierInterface* sequence_barrier_;
+    EventHandlerInterface<T>* event_handler_;
+    ExceptionHandlerInterface<T>* exception_handler_;
+
+};
+
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_EVENT_PROCESSOR_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/event_publisher.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/event_publisher.h b/rocketmq-cpp/src/thread/disruptor/event_publisher.h
new file mode 100755
index 0000000..ae0efd9
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/event_publisher.h
@@ -0,0 +1,50 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_EVENT_PUBLISHER_H_ // NOLINT
+#define DISRUPTOR_EVENT_PUBLISHER_H_ // NOLINT
+
+#include "ring_buffer.h"
+
+namespace rocketmq {
+
+template<typename T>
+class EventPublisher {
+ public:
+    EventPublisher(RingBuffer<T>* ring_buffer) : ring_buffer_(ring_buffer) {}
+
+    void PublishEvent(EventTranslatorInterface<T>* translator) {
+        int64_t sequence = ring_buffer_->Next();
+        translator->TranslateTo(sequence, ring_buffer_->Get(sequence));
+        ring_buffer_->Publish(sequence);
+    }
+
+ private:
+    RingBuffer<T>* ring_buffer_;
+};
+
+};  // namespace rocketmq
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/exception_handler.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/exception_handler.h b/rocketmq-cpp/src/thread/disruptor/exception_handler.h
new file mode 100755
index 0000000..e7979a0
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/exception_handler.h
@@ -0,0 +1,59 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_EXCEPTION_HANDLER_H_  // NOLINT
+#define DISRUPTOR_EXCEPTION_HANDLER_H_  // NOLINT
+
+#include <exception>
+
+#include "interface.h"
+
+namespace rocketmq {
+
+template<typename T>
+class IgnoreExceptionHandler: public ExceptionHandlerInterface<T> {
+ public:
+    virtual void Handle(const std::exception& exception,
+                         const int64_t& sequence,
+                         T* event) {
+        // do nothing with the exception.
+        ;
+    }
+};
+
+template<typename T>
+class FatalExceptionHandler: public ExceptionHandlerInterface<T> {
+ public:
+    virtual void Handle(const std::exception& exception,
+                         const int64_t& sequence,
+                         T* event) {
+        // rethrow the exception
+        throw exception;
+    }
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_EXCEPTION_HANDLER_H_  NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/exceptions.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/exceptions.h b/rocketmq-cpp/src/thread/disruptor/exceptions.h
new file mode 100755
index 0000000..f968043
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/exceptions.h
@@ -0,0 +1,38 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_EXCEPTIONS_H_  // NOLINT
+#define DISRUPTOR_EXCEPTIONS_H_  // NOLINT
+
+#include <exception>
+
+namespace rocketmq {
+
+class AlertException : public std::exception {
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_EXCEPTIONS_H_  NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/interface.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/interface.h b/rocketmq-cpp/src/thread/disruptor/interface.h
new file mode 100755
index 0000000..0c07774
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/interface.h
@@ -0,0 +1,278 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_INTERFACE_H_ // NOLINT
+#define DISRUPTOR_INTERFACE_H_ // NOLINT
+
+#include <climits>
+#include <vector>
+
+#include "sequence.h"
+#include "batch_descriptor.h"
+
+namespace rocketmq {
+
+// Strategies employed for claiming the sequence of events in the
+// {@link Seqencer} by publishers.
+class ClaimStrategyInterface {
+ public:
+    // Is there available capacity in the buffer for the requested sequence.
+    //
+    // @param dependent_sequences to be checked for range.
+    // @return true if the buffer has capacity for the requested sequence.
+    virtual ~ClaimStrategyInterface() {}
+    virtual bool HasAvalaibleCapacity(
+        const std::vector<Sequence*>& dependent_sequences) = 0;
+
+    // Claim the next sequence in the {@link Sequencer}.
+    //
+    // @param dependent_sequences to be checked for range.
+    // @return the index to be used for the publishing.
+    virtual int64_t IncrementAndGet(
+            const std::vector<Sequence*>& dependent_sequences) = 0;
+
+    // Claim the next sequence in the {@link Sequencer}.
+    //
+    // @param delta to increment by.
+    // @param dependent_sequences to be checked for range.
+    // @return the index to be used for the publishing.
+    virtual int64_t IncrementAndGet(const int& delta,
+            const std::vector<Sequence*>& dependent_sequences) = 0;
+
+    // Set the current sequence value for claiming an event in the
+    // {@link Sequencer}.
+    //
+    // @param sequence to be set as the current value.
+    // @param dependent_sequences to be checked for range.
+    virtual void SetSequence(const int64_t& sequence,
+            const std::vector<Sequence*>& dependent_sequences) = 0;
+
+    // Serialise publishing in sequence.
+    //
+    // @param sequence to be applied.
+    // @param cursor to be serialise against.
+    // @param batch_size of the sequence.
+    virtual void SerialisePublishing(const int64_t& sequence,
+                                     const Sequence& cursor,
+                                     const int64_t& batch_size) = 0;
+};
+
+// Coordination barrier for tracking the cursor for publishers and sequence of
+// dependent {@link EventProcessor}s for processing a data structure.
+class SequenceBarrierInterface {
+ public:
+    // Wait for the given sequence to be available for consumption.
+    //
+    // @param sequence to wait for.
+    // @return the sequence up to which is available.
+    //
+    // @throws AlertException if a status change has occurred for the
+    // Disruptor.
+    virtual ~SequenceBarrierInterface(){}
+    virtual int64_t WaitFor(const int64_t& sequence) = 0;
+
+    // Wait for the given sequence to be available for consumption with a
+    // time out.
+    //
+    // @param sequence to wait for.
+    // @param timeout in microseconds.
+    // @return the sequence up to which is available.
+    //
+    // @throws AlertException if a status change has occurred for the
+    // Disruptor.
+    virtual int64_t WaitFor(const int64_t& sequence,
+                            const int64_t& timeout_micro) = 0;
+
+    // Delegate a call to the {@link Sequencer#getCursor()}
+    //
+    //  @return value of the cursor for entries that have been published.
+    virtual int64_t GetCursor() const = 0;
+
+    // The current alert status for the barrier.
+    //
+    // @return true if in alert otherwise false.
+    virtual bool IsAlerted() const = 0;
+
+    // Alert the {@link EventProcessor}s of a status change and stay in this
+    // status until cleared.
+    virtual void Alert() = 0;
+
+    // Clear the current alert status.
+    virtual void ClearAlert() = 0;
+
+    // Check if barrier is alerted, if so throws an AlertException
+    //
+    // @throws AlertException if barrier is alerted
+    virtual void CheckAlert() const = 0;
+};
+
+// Called by the {@link RingBuffer} to pre-populate all the events to fill the
+// RingBuffer.
+//
+// @param <T> event implementation storing the data for sharing during exchange
+// or parallel coordination of an event.
+template<typename T>
+class EventFactoryInterface {
+ public:
+    virtual ~EventFactoryInterface(){}
+     virtual T* NewInstance(const int& size) const = 0;
+};
+
+// Callback interface to be implemented for processing events as they become
+// available in the {@link RingBuffer}.
+//
+// @param <T> event implementation storing the data for sharing during exchange
+// or parallel coordination of an event.
+template<typename T>
+class EventHandlerInterface {
+ public:
+    // Called when a publisher has published an event to the {@link RingBuffer}
+    //
+    // @param event published to the {@link RingBuffer}
+    // @param sequence of the event being processed
+    // @param end_of_batch flag to indicate if this is the last event in a batch
+    // from the {@link RingBuffer}
+    //
+    // @throws Exception if the EventHandler would like the exception handled
+    // further up the chain.
+    virtual ~EventHandlerInterface(){}
+    virtual void OnEvent(const int64_t& sequence,
+                         const bool& end_of_batch,
+                         T* event)  = 0;
+
+    // Called once on thread start before processing the first event.
+    virtual void OnStart() = 0;
+
+    // Called once on thread stop just before shutdown.
+    virtual void OnShutdown() = 0;
+};
+
+// Implementations translate another data representations into events claimed
+// for the {@link RingBuffer}.
+//
+// @param <T> event implementation storing the data for sharing during exchange
+// or parallel coordination of an event.
+template<typename T>
+class EventTranslatorInterface {
+ public:
+     // Translate a data representation into fields set in given event
+     //
+     // @param event into which the data should be translated.
+     // @param sequence that is assigned to events.
+     // @return the resulting event after it has been translated.
+     virtual ~EventTranslatorInterface(){}
+     virtual T* TranslateTo(const int64_t& sequence, T* event) { return NULL;}
+};
+
+// EventProcessors wait for events to become available for consumption from
+// the {@link RingBuffer}. An event processor should be associated with a
+// thread.
+//
+// @param <T> event implementation storing the data for sharing during exchange
+// or parallel coordination of an event.
+template<typename T>
+class EventProcessorInterface {
+ public:
+     // Get a pointer to the {@link Sequence} being used by this
+     // {@link EventProcessor}.
+     //
+     // @return pointer to the {@link Sequence} for this
+     // {@link EventProcessor}
+     virtual ~EventProcessorInterface(){}
+    virtual Sequence* GetSequence() = 0;
+
+    // Signal that this EventProcessor should stop when it has finished
+    // consuming at the next clean break.
+    // It will call {@link DependencyBarrier#alert()} to notify the thread to
+    // check status.
+    virtual void Halt() = 0;
+};
+
+// Callback handler for uncaught exception in the event processing cycle
+// of the {@link BatchEventProcessor}.
+//
+// @param <T> event type stored in the {@link RingBuffer}.
+template<typename T>
+class ExceptionHandlerInterface {
+ public:
+    // Strategy for handling uncaught exceptions when processing an event.
+    // If the strategy wishes to suspend further processing by the
+    // {@link BatchEventProcessor} then it should throw a std::runtime_error.
+    //
+    // @param exception that propagated from the {@link EventHandler}.
+    // @param sequence of the event which caused the exception.
+    // @param event being processed when the exception occured.
+    virtual ~ExceptionHandlerInterface(){}
+    virtual void Handle(const std::exception& exception,
+                        const int64_t& sequence,
+                        T* event) = 0;
+};
+
+// Strategy employed for making {@link EventProcessor}s wait on a cursor
+// {@link Sequence}.
+class WaitStrategyInterface: public boost::noncopyable {
+ public:
+    //  Wait for the given sequence to be available for consumption.
+    //
+    //  @param dependents further back the chain that must advance first.
+    //  @param cursor on which to wait.
+    //  @param barrier the consumer is waiting on.
+    //  @param sequence to be waited on.
+    //  @return the sequence that is available which may be greater than the
+    //  requested sequence.
+    //
+    //  @throws AlertException if the status of the Disruptor has changed.
+    virtual ~WaitStrategyInterface(){}
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t& sequence) = 0;
+
+    //  Wait for the given sequence to be available for consumption in a
+    //  {@link RingBuffer} with a timeout specified.
+    //
+    //  @param dependents further back the chain that must advance first
+    //  @param cursor on which to wait.
+    //  @param barrier the consumer is waiting on.
+    //  @param sequence to be waited on.
+    //  @param timeout value in micro seconds to abort after.
+    //  @return the sequence that is available which may be greater than the
+    //  requested sequence.
+    //
+    //  @throws AlertException if the status of the Disruptor has changed.
+    //  @throws InterruptedException if the thread is interrupted.
+    virtual int64_t WaitFor(const std::vector<Sequence*>& dependents,
+                            const Sequence& cursor,
+                            const SequenceBarrierInterface& barrier,
+                            const int64_t & sequence,
+                            const int64_t & timeout_micros) = 0;
+
+    // Signal those waiting that the cursor has advanced.
+    virtual void SignalAllWhenBlocking() = 0;
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_INTERFACE_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/ring_buffer.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/ring_buffer.h b/rocketmq-cpp/src/thread/disruptor/ring_buffer.h
new file mode 100755
index 0000000..c7150f1
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/ring_buffer.h
@@ -0,0 +1,90 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_RING_BUFFER_H_ // NOLINT
+#define DISRUPTOR_RING_BUFFER_H_ // NOLINT
+
+#include <boost/array.hpp>
+#include <vector>
+
+#include "interface.h"
+#include "claim_strategy.h"
+#include "wait_strategy.h"
+#include "sequencer.h"
+#include "sequence_barrier.h"
+
+namespace rocketmq {
+
+// Ring based store of reusable entries containing the data representing an
+// event beign exchanged between publisher and {@link EventProcessor}s.
+//
+// @param <T> implementation storing the data for sharing during exchange
+// or parallel coordination of an event.
+template<typename T>
+class RingBuffer : public Sequencer {
+ public:
+    // Construct a RingBuffer with the full option set.
+    //
+    // @param event_factory to instance new entries for filling the RingBuffer.
+    // @param buffer_size of the RingBuffer, must be a power of 2.
+    // @param claim_strategy_option threading strategy for publishers claiming
+    // entries in the ring.
+    // @param wait_strategy_option waiting strategy employed by
+    // processors_to_track waiting in entries becoming available.
+    RingBuffer(EventFactoryInterface<T>* event_factory,
+               int buffer_size,
+               ClaimStrategyOption claim_strategy_option,
+               WaitStrategyOption wait_strategy_option) :
+            Sequencer(buffer_size,
+                      claim_strategy_option,
+                      wait_strategy_option),
+            buffer_size_(buffer_size),
+            mask_(buffer_size - 1),
+            events_(event_factory->NewInstance(buffer_size)) {
+    }
+
+    ~RingBuffer() {
+        delete[] events_;
+    }
+
+    // Get the event for a given sequence in the RingBuffer.
+    //
+    // @param sequence for the event
+    // @return event pointer at the specified sequence position.
+    T* Get(const int64_t& sequence) {
+        return &events_[sequence & mask_];
+    }
+
+ private:
+    // Members
+    int buffer_size_;
+    int mask_;
+    T* events_;
+
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_RING_BUFFER_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/sequence.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/sequence.h b/rocketmq-cpp/src/thread/disruptor/sequence.h
new file mode 100755
index 0000000..f1396f3
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/sequence.h
@@ -0,0 +1,139 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef CACHE_LINE_SIZE_IN_BYTES // NOLINT
+#define CACHE_LINE_SIZE_IN_BYTES 64 // NOLINT
+#endif // NOLINT
+#define ATOMIC_SEQUENCE_PADDING_LENGTH \
+    (CACHE_LINE_SIZE_IN_BYTES - sizeof(boost::atomic<int64_t>))/8
+#define SEQUENCE_PADDING_LENGTH \
+    (CACHE_LINE_SIZE_IN_BYTES - sizeof(int64_t))/8
+
+#ifndef DISRUPTOR_SEQUENCE_H_ // NOLINT
+#define DISRUPTOR_SEQUENCE_H_ // NOLINT
+
+#include <boost/atomic.hpp>
+#include <boost/memory_order.hpp>
+#include <boost/noncopyable.hpp>
+#include <vector>
+#include <limits>
+using namespace boost;
+namespace rocketmq {
+
+const int64_t kInitialCursorValue = -1L;
+
+// Sequence counter.
+class Sequence:public noncopyable {
+ public:
+    // Construct a sequence counter that can be tracked across threads.
+    //
+    // @param initial_value for the counter.
+    Sequence(int64_t initial_value = kInitialCursorValue) :
+            value_(initial_value) {}
+
+    // Get the current value of the {@link Sequence}.
+    //
+    // @return the current value.
+    int64_t sequence() const { return value_.load(boost::memory_order_acquire); }
+
+    // Set the current value of the {@link Sequence}.
+    //
+    // @param the value to which the {@link Sequence} will be set.
+    void set_sequence(int64_t value) { value_.store(value, boost::memory_order_release); }
+
+    // Increment and return the value of the {@link Sequence}.
+    //
+    // @param increment the {@link Sequence}.
+    // @return the new value incremented.
+    int64_t IncrementAndGet(const int64_t& increment) {
+        return value_.fetch_add(increment, boost::memory_order_release) + increment;
+    }
+
+ private:
+    // members
+    boost::atomic<int64_t> value_;
+
+};
+
+// Cache line padded sequence counter.
+//
+// Can be used across threads without worrying about false sharing if a
+// located adjacent to another counter in memory.
+class PaddedSequence : public Sequence {
+ public:
+    PaddedSequence(int64_t initial_value = kInitialCursorValue) :
+            Sequence(initial_value) {}
+
+ private:
+    // padding
+    int64_t padding_[ATOMIC_SEQUENCE_PADDING_LENGTH];
+
+};
+
+// Non-atomic sequence counter.
+//
+// This counter is not thread safe.
+class MutableLong {
+ public:
+     MutableLong(int64_t initial_value = kInitialCursorValue) :
+         sequence_(initial_value) {}
+
+     int64_t sequence() const { return sequence_; }
+
+     void set_sequence(const int64_t& sequence) { sequence_ = sequence; };
+
+     int64_t IncrementAndGet(const int64_t& delta) { sequence_ += delta; return sequence_; }
+
+ private:
+     volatile int64_t sequence_;
+};
+
+// Cache line padded non-atomic sequence counter.
+//
+// This counter is not thread safe.
+class PaddedLong : public MutableLong {
+ public:
+     PaddedLong(int64_t initial_value = kInitialCursorValue) :
+         MutableLong(initial_value) {}
+ private:
+     int64_t padding_[SEQUENCE_PADDING_LENGTH];
+};
+
+int64_t GetMinimumSequence(
+        const std::vector<Sequence*>& sequences) {
+        int64_t minimum = std::numeric_limits<int64_t>::max();
+
+        std::vector<Sequence*>::const_iterator it= sequences.begin();
+        for (;it!=sequences.end();it++) {
+            int64_t sequence = (*it)->sequence();
+            minimum = minimum < sequence ? minimum : sequence;
+        }
+
+        return minimum;
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_SEQUENCE_H_ NOLINT

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/thread/disruptor/sequence_barrier.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/thread/disruptor/sequence_barrier.h b/rocketmq-cpp/src/thread/disruptor/sequence_barrier.h
new file mode 100755
index 0000000..c156388
--- /dev/null
+++ b/rocketmq-cpp/src/thread/disruptor/sequence_barrier.h
@@ -0,0 +1,92 @@
+// Copyright (c) 2011, François Saint-Jacques
+// All rights reserved.
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are met:
+//     * Redistributions of source code must retain the above copyright
+//       notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above copyright
+//       notice, this list of conditions and the following disclaimer in the
+//       documentation and/or other materials provided with the distribution.
+//     * Neither the name of the disruptor-- nor the
+//       names of its contributors may be used to endorse or promote products
+//       derived from this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+// ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+// WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+// DISCLAIMED. IN NO EVENT SHALL FRANÇOIS SAINT-JACQUES BE LIABLE FOR ANY
+// DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+// (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+// LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+// ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+#ifndef DISRUPTOR_SEQUENCE_BARRIER_H_ // NOLINT
+#define DISRUPTOR_SEQUENCE_BARRIER_H_ // NOLINT
+
+#include <memory>
+#include <vector>
+
+#include "exceptions.h"
+#include "interface.h"
+namespace rocketmq {
+
+class ProcessingSequenceBarrier : SequenceBarrierInterface {
+ public:
+    ProcessingSequenceBarrier(WaitStrategyInterface* wait_strategy,
+                              Sequence* sequence,
+                              const std::vector<Sequence*>& sequences) :
+        wait_strategy_(wait_strategy),
+        cursor_(sequence),
+        dependent_sequences_(sequences),
+        alerted_(false) {
+    }
+
+    virtual int64_t WaitFor(const int64_t& sequence) {
+        return wait_strategy_->WaitFor(dependent_sequences_, *cursor_, *this,
+                                       sequence);
+    }
+
+    virtual int64_t WaitFor(const int64_t& sequence,
+                            const int64_t& timeout_micros) {
+        return wait_strategy_->WaitFor(dependent_sequences_, *cursor_, *this,
+                                       sequence, timeout_micros);
+    }
+
+    virtual int64_t GetCursor() const {
+        return cursor_->sequence();
+    }
+
+    virtual bool IsAlerted() const {
+        return alerted_.load(boost::memory_order_acquire);
+    }
+
+    virtual void Alert() {
+        //metaq::LOG_INFO("set alert to true");
+        alerted_.store(true, boost::memory_order_release);
+    }
+
+    virtual void ClearAlert() {
+        alerted_.store(false, boost::memory_order_release);
+    }
+
+    virtual void CheckAlert() const {
+        if (IsAlerted())
+        {
+            //metaq::LOG_INFO("throw alert exception\r\n");
+            throw AlertException();
+        }
+    }
+
+ private:
+    WaitStrategyInterface* wait_strategy_;
+    Sequence* cursor_;
+    std::vector<Sequence*> dependent_sequences_;
+    boost::atomic<bool> alerted_;
+};
+
+};  // namespace rocketmq
+
+#endif // DISRUPTOR_DEPENDENCY_BARRIER_H_ NOLINT



[06/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/DefaultMQPushConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/DefaultMQPushConsumer.cpp b/rocketmq-cpp/src/consumer/DefaultMQPushConsumer.cpp
new file mode 100755
index 0000000..8407350
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/DefaultMQPushConsumer.cpp
@@ -0,0 +1,897 @@
+/*
+ * 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 "DefaultMQPushConsumer.h"
+#include "CommunicationMode.h"
+#include "ConsumeMsgService.h"
+#include "ConsumerRunningInfo.h"
+#include "FilterAPI.h"
+#include "Logging.h"
+#include "MQClientAPIImpl.h"
+#include "MQClientFactory.h"
+#include "MQClientManager.h"
+#include "MQProtos.h"
+#include "OffsetStore.h"
+#include "PullAPIWrapper.h"
+#include "PullSysFlag.h"
+#include "Rebalance.h"
+#include "UtilAll.h"
+#include "Validators.h"
+#include "task_queue.h"
+
+namespace rocketmq {
+
+class AsyncPullCallback : public PullCallback {
+ public:
+  AsyncPullCallback(DefaultMQPushConsumer* pushConsumer, PullRequest* request)
+      : m_callbackOwner(pushConsumer),
+        m_pullRequest(request),
+        m_bShutdown(false) {}
+  virtual ~AsyncPullCallback() {
+    m_callbackOwner = NULL;
+    m_pullRequest = NULL;
+  }
+  virtual void onSuccess(MQMessageQueue& mq, PullResult& result,
+                         bool bProducePullRequest) {
+    if (m_bShutdown == true) {
+      LOG_INFO("pullrequest for:%s in shutdown, return",
+               (m_pullRequest->m_messageQueue).toString().c_str());
+      return;
+    }
+
+    switch (result.pullStatus) {
+      case FOUND: {
+        if (!m_pullRequest->isDroped())  // if request is setted to dropped,
+                                         // don't add msgFoundList to
+                                         // m_msgTreeMap and don't call
+                                         // producePullMsgTask
+        {  // avoid issue: pullMsg is sent out, rebalance is doing concurrently
+           // and this request is dropped, and then received pulled msgs.
+          m_pullRequest->setNextOffset(result.nextBeginOffset);
+          m_pullRequest->putMessage(result.msgFoundList);
+
+          m_callbackOwner->getConsumerMsgService()->submitConsumeRequest(
+              m_pullRequest, result.msgFoundList);
+
+          if (bProducePullRequest)
+            m_callbackOwner->producePullMsgTask(m_pullRequest);
+
+          LOG_DEBUG("FOUND:%s with size:%zu,nextBeginOffset:%lld",
+                    (m_pullRequest->m_messageQueue).toString().c_str(),
+                    result.msgFoundList.size(), result.nextBeginOffset);
+        }
+        break;
+      }
+      case NO_NEW_MSG: {
+        m_pullRequest->setNextOffset(result.nextBeginOffset);
+
+        vector<MQMessageExt> msgs;
+        m_pullRequest->getMessage(msgs);
+        if ((msgs.size() == 0) && (result.nextBeginOffset > 0)) {
+          /*if broker losted/cleared msgs of one msgQueue, but the brokerOffset
+          is kept, then consumer will enter following situation:
+          1>. get pull offset with 0 when do rebalance, and set
+          m_offsetTable[mq] to 0;
+          2>. NO_NEW_MSG or NO_MATCHED_MSG got when pullMessage, and nextBegin
+          offset increase by 800
+          3>. request->getMessage(msgs) always NULL
+          4>. we need update consumerOffset to nextBeginOffset indicated by
+          broker
+          but if really no new msg could be pulled, also go to this CASE
+
+          LOG_INFO("maybe misMatch between broker and client happens, update
+          consumerOffset to nextBeginOffset indicated by broker");*/
+          m_callbackOwner->updateConsumeOffset(m_pullRequest->m_messageQueue,
+                                               result.nextBeginOffset);
+        }
+        if (bProducePullRequest)
+          m_callbackOwner->producePullMsgTask(m_pullRequest);
+
+        /*LOG_INFO("NO_NEW_MSG:%s,nextBeginOffset:%lld",
+                 (m_pullRequest->m_messageQueue).toString().c_str(),
+                 result.nextBeginOffset);*/
+        break;
+      }
+      case NO_MATCHED_MSG: {
+        m_pullRequest->setNextOffset(result.nextBeginOffset);
+
+        vector<MQMessageExt> msgs;
+        m_pullRequest->getMessage(msgs);
+        if ((msgs.size() == 0) && (result.nextBeginOffset > 0)) {
+          /*if broker losted/cleared msgs of one msgQueue, but the brokerOffset
+          is kept, then consumer will enter following situation:
+          1>. get pull offset with 0 when do rebalance, and set
+          m_offsetTable[mq] to 0;
+          2>. NO_NEW_MSG or NO_MATCHED_MSG got when pullMessage, and nextBegin
+          offset increase by 800
+          3>. request->getMessage(msgs) always NULL
+          4>. we need update consumerOffset to nextBeginOffset indicated by
+          broker
+          but if really no new msg could be pulled, also go to this CASE
+
+          LOG_INFO("maybe misMatch between broker and client happens, update
+          consumerOffset to nextBeginOffset indicated by broker");*/
+          m_callbackOwner->updateConsumeOffset(m_pullRequest->m_messageQueue,
+                                               result.nextBeginOffset);
+        }
+        if (bProducePullRequest)
+          m_callbackOwner->producePullMsgTask(m_pullRequest);
+        /*LOG_INFO("NO_MATCHED_MSG:%s,nextBeginOffset:%lld",
+                 (m_pullRequest->m_messageQueue).toString().c_str(),
+                 result.nextBeginOffset);*/
+        break;
+      }
+      case OFFSET_ILLEGAL: {
+        m_pullRequest->setNextOffset(result.nextBeginOffset);
+        if (bProducePullRequest)
+          m_callbackOwner->producePullMsgTask(m_pullRequest);
+
+        /*LOG_INFO("OFFSET_ILLEGAL:%s,nextBeginOffset:%lld",
+                 (m_pullRequest->m_messageQueue).toString().c_str(),
+                 result.nextBeginOffset);*/
+        break;
+      }
+      case BROKER_TIMEOUT: {  // as BROKER_TIMEOUT is defined by client, broker
+                              // will not returns this status, so this case
+                              // could not be entered.
+        LOG_ERROR("impossible BROKER_TIMEOUT Occurs");
+        m_pullRequest->setNextOffset(result.nextBeginOffset);
+        if (bProducePullRequest)
+          m_callbackOwner->producePullMsgTask(m_pullRequest);
+        break;
+      }
+    }
+  }
+
+  virtual void onException(MQException& e) {
+    if (m_bShutdown == true) {
+      LOG_INFO("pullrequest for:%s in shutdown, return",
+               (m_pullRequest->m_messageQueue).toString().c_str());
+      return;
+    }
+    LOG_WARN("pullrequest for:%s occurs exception, reproduce it",
+             (m_pullRequest->m_messageQueue).toString().c_str());
+    m_callbackOwner->producePullMsgTask(m_pullRequest);
+  }
+
+  void setShutdownStatus() { m_bShutdown = true; }
+
+ private:
+  DefaultMQPushConsumer* m_callbackOwner;
+  PullRequest* m_pullRequest;
+  bool m_bShutdown;
+};
+
+//<!***************************************************************************
+static boost::mutex m_asyncCallbackLock;
+DefaultMQPushConsumer::DefaultMQPushConsumer(const string& groupname)
+    : m_consumeFromWhere(CONSUME_FROM_LAST_OFFSET),
+      m_pOffsetStore(NULL),
+      m_pPullAPIWrapper(NULL),
+      m_pMessageListener(NULL),
+      m_consumeMessageBatchMaxSize(1),
+      m_maxMsgCacheSize(1000) {
+  //<!set default group name;
+  string gname = groupname.empty() ? DEFAULT_CONSUMER_GROUP : groupname;
+  setGroupName(gname);
+  m_asyncPull = true;
+  m_asyncPullTimeout = 30 * 1000;
+  setMessageModel(CLUSTERING);
+
+  m_startTime = UtilAll::currentTimeMillis();
+  m_consumeThreadCount = boost::thread::hardware_concurrency();
+  m_pullMsgThreadPoolNum = boost::thread::hardware_concurrency();
+  m_async_service_thread.reset(new boost::thread(
+      boost::bind(&DefaultMQPushConsumer::boost_asio_work, this)));
+}
+
+void DefaultMQPushConsumer::boost_asio_work() {
+  LOG_INFO("DefaultMQPushConsumer::boost asio async service runing");
+  boost::asio::io_service::work work(m_async_ioService);  // avoid async io
+                                                          // service stops after
+                                                          // first timer timeout
+                                                          // callback
+  m_async_ioService.run();
+}
+
+DefaultMQPushConsumer::~DefaultMQPushConsumer() {
+  m_pMessageListener = NULL;
+  deleteAndZero(m_pullmsgQueue);
+  deleteAndZero(m_pRebalance);
+  deleteAndZero(m_pOffsetStore);
+  deleteAndZero(m_pPullAPIWrapper);
+  deleteAndZero(m_consumerServeice);
+  PullMAP::iterator it = m_PullCallback.begin();
+  for (; it != m_PullCallback.end(); ++it) {
+    deleteAndZero(it->second);
+  }
+  m_PullCallback.clear();
+  m_subTopics.clear();
+}
+
+void DefaultMQPushConsumer::sendMessageBack(MQMessageExt& msg, int delayLevel) {
+  try {
+    getFactory()->getMQClientAPIImpl()->consumerSendMessageBack(
+        msg, getGroupName(), delayLevel, 3000, getSessionCredentials());
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+void DefaultMQPushConsumer::fetchSubscribeMessageQueues(
+    const string& topic, vector<MQMessageQueue>& mqs) {
+  mqs.clear();
+  try {
+    getFactory()->fetchSubscribeMessageQueues(topic, mqs,
+                                              getSessionCredentials());
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+void DefaultMQPushConsumer::doRebalance() {
+  if (isServiceStateOk()) {
+    try {
+      m_pRebalance->doRebalance();
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+    }
+  }
+}
+
+void DefaultMQPushConsumer::persistConsumerOffset() {
+  if (isServiceStateOk()) {
+    m_pRebalance->persistConsumerOffset();
+  }
+}
+
+void DefaultMQPushConsumer::persistConsumerOffsetByResetOffset() {
+  if (isServiceStateOk()) {
+    m_pRebalance->persistConsumerOffsetByResetOffset();
+  }
+}
+
+void DefaultMQPushConsumer::start() {
+  /* Ignore the SIGPIPE */
+  struct sigaction sa;
+  sa.sa_handler = SIG_IGN;
+  sa.sa_flags = 0;
+  sigaction(SIGPIPE, &sa, 0);
+
+  switch (m_serviceState) {
+    case CREATE_JUST: {
+      m_serviceState = START_FAILED;
+      MQClient::start();
+      LOG_INFO("DefaultMQPushConsumer:%s start", m_GroupName.c_str());
+
+      //<!data;
+      checkConfig();
+
+      //<!create rebalance;
+      m_pRebalance = new RebalancePush(this, getFactory());
+
+      string groupname = getGroupName();
+      m_pPullAPIWrapper = new PullAPIWrapper(getFactory(), groupname);
+
+      if (m_pMessageListener) {
+        if (m_pMessageListener->getMessageListenerType() ==
+            messageListenerOrderly) {
+          LOG_INFO("start orderly consume service:%s", getGroupName().c_str());
+          m_consumerServeice = new ConsumeMessageOrderlyService(
+              this, m_consumeThreadCount, m_pMessageListener);
+        } else  // for backward compatible, defaultly and concurrently listeners
+                // are allocating ConsumeMessageConcurrentlyService
+        {
+          LOG_INFO("start concurrently consume service:%s",
+                   getGroupName().c_str());
+          m_consumerServeice = new ConsumeMessageConcurrentlyService(
+              this, m_consumeThreadCount, m_pMessageListener);
+        }
+      }
+
+      m_pullmsgQueue = new TaskQueue(m_pullMsgThreadPoolNum);
+      m_pullmsgThread.reset(new boost::thread(boost::bind(
+          &DefaultMQPushConsumer::runPullMsgQueue, this, m_pullmsgQueue)));
+
+      copySubscription();
+
+      //<! registe;
+      bool registerOK = getFactory()->registerConsumer(this);
+      if (!registerOK) {
+        m_serviceState = CREATE_JUST;
+        THROW_MQEXCEPTION(
+            MQClientException,
+            "The cousumer group[" + getGroupName() +
+                "] has been created before, specify another name please.",
+            -1);
+      }
+
+      //<!msg model;
+      switch (getMessageModel()) {
+        case BROADCASTING:
+          m_pOffsetStore = new LocalFileOffsetStore(groupname, getFactory());
+          break;
+        case CLUSTERING:
+          m_pOffsetStore = new RemoteBrokerOffsetStore(groupname, getFactory());
+          break;
+      }
+      m_pOffsetStore->load();
+      m_consumerServeice->start();
+
+      getFactory()->start();
+
+      //<!����ط����ʱ��ܳ���;
+      updateTopicSubscribeInfoWhenSubscriptionChanged();
+      getFactory()->sendHeartbeatToAllBroker();
+
+      m_serviceState = RUNNING;
+      break;
+    }
+    case RUNNING:
+    case START_FAILED:
+    case SHUTDOWN_ALREADY:
+      break;
+    default:
+      break;
+  }
+
+  getFactory()->rebalanceImmediately();
+}
+
+void DefaultMQPushConsumer::shutdown() {
+  switch (m_serviceState) {
+    case RUNNING: {
+      LOG_INFO("DefaultMQPushConsumer shutdown");
+      m_async_ioService.stop();
+      m_async_service_thread->interrupt();
+      m_async_service_thread->join();
+      m_pullmsgQueue->close();
+      m_pullmsgThread->interrupt();
+      m_pullmsgThread->join();
+      m_consumerServeice->shutdown();
+      persistConsumerOffset();
+      shutdownAsyncPullCallBack();  // delete aync pullMsg resources
+      getFactory()->unregisterConsumer(this);
+      getFactory()->shutdown();
+      m_serviceState = SHUTDOWN_ALREADY;
+      break;
+    }
+    case CREATE_JUST:
+    case SHUTDOWN_ALREADY:
+      break;
+    default:
+      break;
+  }
+}
+
+void DefaultMQPushConsumer::registerMessageListener(
+    MQMessageListener* pMessageListener) {
+  if (NULL != pMessageListener) {
+    m_pMessageListener = pMessageListener;
+  }
+}
+
+MessageListenerType DefaultMQPushConsumer::getMessageListenerType() {
+  if (NULL != m_pMessageListener) {
+    return m_pMessageListener->getMessageListenerType();
+  }
+  return messageListenerDefaultly;
+}
+
+ConsumeMsgService* DefaultMQPushConsumer::getConsumerMsgService() const {
+  return m_consumerServeice;
+}
+
+OffsetStore* DefaultMQPushConsumer::getOffsetStore() const {
+  return m_pOffsetStore;
+}
+
+Rebalance* DefaultMQPushConsumer::getRebalance() const { return m_pRebalance; }
+
+void DefaultMQPushConsumer::subscribe(const string& topic,
+                                      const string& subExpression) {
+  m_subTopics[topic] = subExpression;
+}
+
+void DefaultMQPushConsumer::checkConfig() {
+  string groupname = getGroupName();
+  // check consumerGroup
+  Validators::checkGroup(groupname);
+
+  // consumerGroup
+  if (!groupname.compare(DEFAULT_CONSUMER_GROUP)) {
+    THROW_MQEXCEPTION(MQClientException,
+                      "consumerGroup can not equal DEFAULT_CONSUMER", -1);
+  }
+
+  if (getMessageModel() != BROADCASTING && getMessageModel() != CLUSTERING) {
+    THROW_MQEXCEPTION(MQClientException, "messageModel is valid ", -1);
+  }
+
+  if (m_pMessageListener == NULL) {
+    THROW_MQEXCEPTION(MQClientException, "messageListener is null ", -1);
+  }
+}
+
+void DefaultMQPushConsumer::copySubscription() {
+  map<string, string>::iterator it = m_subTopics.begin();
+  for (; it != m_subTopics.end(); ++it) {
+    LOG_INFO("buildSubscriptionData,:%s,%s", it->first.c_str(),
+             it->second.c_str());
+    unique_ptr<SubscriptionData> pSData(
+        FilterAPI::buildSubscriptionData(it->first, it->second));
+
+    m_pRebalance->setSubscriptionData(it->first, pSData.release());
+  }
+
+  switch (getMessageModel()) {
+    case BROADCASTING:
+      break;
+    case CLUSTERING: {
+      string retryTopic = UtilAll::getRetryTopic(getGroupName());
+
+      //<!this sub;
+      unique_ptr<SubscriptionData> pSData(
+          FilterAPI::buildSubscriptionData(retryTopic, SUB_ALL));
+
+      m_pRebalance->setSubscriptionData(retryTopic, pSData.release());
+      break;
+    }
+    default:
+      break;
+  }
+}
+
+void DefaultMQPushConsumer::updateTopicSubscribeInfo(
+    const string& topic, vector<MQMessageQueue>& info) {
+  m_pRebalance->setTopicSubscribeInfo(topic, info);
+}
+
+void DefaultMQPushConsumer::updateTopicSubscribeInfoWhenSubscriptionChanged() {
+  map<string, SubscriptionData*>& subTable =
+      m_pRebalance->getSubscriptionInner();
+  map<string, SubscriptionData*>::iterator it = subTable.begin();
+  for (; it != subTable.end(); ++it) {
+    bool btopic = getFactory()->updateTopicRouteInfoFromNameServer(
+        it->first, getSessionCredentials());
+    if (btopic == false) {
+      LOG_WARN("The topic:[%s] not exist", it->first.c_str());
+    }
+  }
+}
+
+ConsumeType DefaultMQPushConsumer::getConsumeType() {
+  return CONSUME_PASSIVELY;
+}
+
+ConsumeFromWhere DefaultMQPushConsumer::getConsumeFromWhere() {
+  return m_consumeFromWhere;
+}
+
+void DefaultMQPushConsumer::setConsumeFromWhere(
+    ConsumeFromWhere consumeFromWhere) {
+  m_consumeFromWhere = consumeFromWhere;
+}
+
+void DefaultMQPushConsumer::getSubscriptions(vector<SubscriptionData>& result) {
+  map<string, SubscriptionData*>& subTable =
+      m_pRebalance->getSubscriptionInner();
+  map<string, SubscriptionData*>::iterator it = subTable.begin();
+  for (; it != subTable.end(); ++it) {
+    result.push_back(*(it->second));
+  }
+}
+
+void DefaultMQPushConsumer::updateConsumeOffset(const MQMessageQueue& mq,
+                                                int64 offset) {
+  if (offset >= 0) {
+    m_pOffsetStore->updateOffset(mq, offset);
+  } else {
+    LOG_ERROR("updateConsumeOffset of mq:%s error", mq.toString().c_str());
+  }
+}
+void DefaultMQPushConsumer::removeConsumeOffset(const MQMessageQueue& mq) {
+  m_pOffsetStore->removeOffset(mq);
+}
+
+void DefaultMQPushConsumer::triggerNextPullRequest(
+    boost::asio::deadline_timer* t, PullRequest* request) {
+  // LOG_INFO("trigger pullrequest for:%s",
+  // (request->m_messageQueue).toString().c_str());
+  producePullMsgTask(request);
+  deleteAndZero(t);
+}
+
+void DefaultMQPushConsumer::producePullMsgTask(PullRequest* request) {
+  if (m_pullmsgQueue->bTaskQueueStatusOK() && isServiceStateOk()) {
+    if (m_asyncPull) {
+      m_pullmsgQueue->produce(TaskBinder::gen(
+          &DefaultMQPushConsumer::pullMessageAsync, this, request));
+    } else {
+      m_pullmsgQueue->produce(
+          TaskBinder::gen(&DefaultMQPushConsumer::pullMessage, this, request));
+    }
+  }
+}
+
+void DefaultMQPushConsumer::runPullMsgQueue(TaskQueue* pTaskQueue) {
+  pTaskQueue->run();
+}
+
+void DefaultMQPushConsumer::pullMessage(PullRequest* request) {
+  if (request == NULL || request->isDroped()) {
+    LOG_WARN("Pull request is set drop, return");
+    return;
+  }
+
+  MQMessageQueue& messageQueue = request->m_messageQueue;
+  if (m_consumerServeice->getConsumeMsgSerivceListenerType() ==
+      messageListenerOrderly) {
+    if (!request->isLocked() || request->isLockExpired()) {
+      if (!m_pRebalance->lock(messageQueue)) {
+        producePullMsgTask(request);
+        return;
+      }
+    }
+  }
+
+  if (request->getCacheMsgCount() > m_maxMsgCacheSize) {
+    // LOG_INFO("retry pullrequest for:%s after 1s, as cachMsgSize:%d is larger
+    // than:%d",  (request->m_messageQueue).toString().c_str(),
+    // request->getCacheMsgCount(), m_maxMsgCacheSize);
+    boost::asio::deadline_timer* t = new boost::asio::deadline_timer(
+        m_async_ioService, boost::posix_time::milliseconds(1 * 1000));
+    t->async_wait(boost::bind(&DefaultMQPushConsumer::triggerNextPullRequest,
+                              this, t, request));
+    return;
+  }
+
+  bool commitOffsetEnable = false;
+  int64 commitOffsetValue = 0;
+  if (CLUSTERING == getMessageModel()) {
+    commitOffsetValue = m_pOffsetStore->readOffset(
+        messageQueue, READ_FROM_MEMORY, getSessionCredentials());
+    if (commitOffsetValue > 0) {
+      commitOffsetEnable = true;
+    }
+  }
+
+  string subExpression;
+  SubscriptionData* pSdata =
+      m_pRebalance->getSubscriptionData(messageQueue.getTopic());
+  if (pSdata == NULL) {
+    producePullMsgTask(request);
+    return;
+  }
+  subExpression = pSdata->getSubString();
+
+  int sysFlag =
+      PullSysFlag::buildSysFlag(commitOffsetEnable,      // commitOffset
+                                false,                   // suspend
+                                !subExpression.empty(),  // subscription
+                                false);                  // class filter
+
+  try {
+    request->setLastPullTimestamp(UtilAll::currentTimeMillis());
+    unique_ptr<PullResult> result(
+        m_pPullAPIWrapper->pullKernelImpl(messageQueue,              // 1
+                                          subExpression,             // 2
+                                          pSdata->getSubVersion(),   // 3
+                                          request->getNextOffset(),  // 4
+                                          32,                        // 5
+                                          sysFlag,                   // 6
+                                          commitOffsetValue,         // 7
+                                          1000 * 15,                 // 8
+                                          1000 * 30,                 // 9
+                                          ComMode_SYNC,              // 10
+                                          NULL, getSessionCredentials()));
+
+    PullResult pullResult = m_pPullAPIWrapper->processPullResult(
+        messageQueue, result.get(), pSdata);
+
+    switch (pullResult.pullStatus) {
+      case FOUND: {
+        if (!request->isDroped())  // if request is setted to dropped, don't add
+                                   // msgFoundList to m_msgTreeMap and don't
+                                   // call producePullMsgTask
+        {  // avoid issue: pullMsg is sent out, rebalance is doing concurrently
+           // and this request is dropped, and then received pulled msgs.
+          request->setNextOffset(pullResult.nextBeginOffset);
+          request->putMessage(pullResult.msgFoundList);
+
+          m_consumerServeice->submitConsumeRequest(request,
+                                                   pullResult.msgFoundList);
+          producePullMsgTask(request);
+
+          LOG_DEBUG("FOUND:%s with size:%zu,nextBeginOffset:%lld",
+                    messageQueue.toString().c_str(),
+                    pullResult.msgFoundList.size(), pullResult.nextBeginOffset);
+        }
+        break;
+      }
+      case NO_NEW_MSG: {
+        request->setNextOffset(pullResult.nextBeginOffset);
+        vector<MQMessageExt> msgs;
+        request->getMessage(msgs);
+        if ((msgs.size() == 0) && (pullResult.nextBeginOffset > 0)) {
+          /*if broker losted/cleared msgs of one msgQueue, but the brokerOffset
+          is kept, then consumer will enter following situation:
+          1>. get pull offset with 0 when do rebalance, and set
+          m_offsetTable[mq] to 0;
+          2>. NO_NEW_MSG or NO_MATCHED_MSG got when pullMessage, and nextBegin
+          offset increase by 800
+          3>. request->getMessage(msgs) always NULL
+          4>. we need update consumerOffset to nextBeginOffset indicated by
+          broker
+          but if really no new msg could be pulled, also go to this CASE
+       */
+          // LOG_DEBUG("maybe misMatch between broker and client happens, update
+          // consumerOffset to nextBeginOffset indicated by broker");
+          updateConsumeOffset(messageQueue, pullResult.nextBeginOffset);
+        }
+        producePullMsgTask(request);
+        LOG_DEBUG("NO_NEW_MSG:%s,nextBeginOffset:%lld",
+                  messageQueue.toString().c_str(), pullResult.nextBeginOffset);
+        break;
+      }
+      case NO_MATCHED_MSG: {
+        request->setNextOffset(pullResult.nextBeginOffset);
+        vector<MQMessageExt> msgs;
+        request->getMessage(msgs);
+        if ((msgs.size() == 0) && (pullResult.nextBeginOffset > 0)) {
+          // LOG_DEBUG("maybe misMatch between broker and client happens, update
+          // consumerOffset to nextBeginOffset indicated by broker");
+          updateConsumeOffset(messageQueue, pullResult.nextBeginOffset);
+        }
+        producePullMsgTask(request);
+
+        LOG_DEBUG("NO_MATCHED_MSG:%s,nextBeginOffset:%lld",
+                  messageQueue.toString().c_str(), pullResult.nextBeginOffset);
+        break;
+      }
+      case OFFSET_ILLEGAL: {
+        request->setNextOffset(pullResult.nextBeginOffset);
+        producePullMsgTask(request);
+
+        LOG_DEBUG("OFFSET_ILLEGAL:%s,nextBeginOffset:%lld",
+                  messageQueue.toString().c_str(), pullResult.nextBeginOffset);
+        break;
+      }
+      case BROKER_TIMEOUT: {  // as BROKER_TIMEOUT is defined by client, broker
+                              // will not returns this status, so this case
+                              // could not be entered.
+        LOG_ERROR("impossible BROKER_TIMEOUT Occurs");
+        request->setNextOffset(pullResult.nextBeginOffset);
+        producePullMsgTask(request);
+        break;
+      }
+    }
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    producePullMsgTask(request);
+  }
+}
+
+AsyncPullCallback* DefaultMQPushConsumer::getAsyncPullCallBack(
+    PullRequest* request, MQMessageQueue msgQueue) {
+  boost::lock_guard<boost::mutex> lock(m_asyncCallbackLock);
+  if (m_asyncPull && request) {
+    PullMAP::iterator it = m_PullCallback.find(msgQueue);
+    if (it == m_PullCallback.end()) {
+      LOG_INFO("new pull callback for mq:%s", msgQueue.toString().c_str());
+      m_PullCallback[msgQueue] = new AsyncPullCallback(this, request);
+    }
+    return m_PullCallback[msgQueue];
+  }
+
+  return NULL;
+}
+
+void DefaultMQPushConsumer::shutdownAsyncPullCallBack() {
+  boost::lock_guard<boost::mutex> lock(m_asyncCallbackLock);
+  if (m_asyncPull) {
+    PullMAP::iterator it = m_PullCallback.begin();
+    for (; it != m_PullCallback.end(); ++it) {
+      if (it->second) {
+        it->second->setShutdownStatus();
+      } else {
+        LOG_ERROR("could not find asyncPullCallback for:%s",
+                  it->first.toString().c_str());
+      }
+    }
+  }
+}
+
+void DefaultMQPushConsumer::pullMessageAsync(PullRequest* request) {
+  if (request == NULL || request->isDroped()) {
+    LOG_WARN("Pull request is set drop with mq:%s, return",
+             (request->m_messageQueue).toString().c_str());
+    return;
+  }
+
+  MQMessageQueue& messageQueue = request->m_messageQueue;
+  if (m_consumerServeice->getConsumeMsgSerivceListenerType() ==
+      messageListenerOrderly) {
+    if (!request->isLocked() || request->isLockExpired()) {
+      if (!m_pRebalance->lock(messageQueue)) {
+        producePullMsgTask(request);
+        return;
+      }
+    }
+  }
+
+  if (request->getCacheMsgCount() > m_maxMsgCacheSize) {
+    // LOG_INFO("retry pullrequest for:%s after 1s, as cachMsgSize:%d is larger
+    // than:%d",  (request->m_messageQueue).toString().c_str(),
+    // request->getCacheMsgCount(), m_maxMsgCacheSize);
+    boost::asio::deadline_timer* t = new boost::asio::deadline_timer(
+        m_async_ioService, boost::posix_time::milliseconds(1 * 1000));
+    t->async_wait(boost::bind(&DefaultMQPushConsumer::triggerNextPullRequest,
+                              this, t, request));
+    return;
+  }
+
+  bool commitOffsetEnable = false;
+  int64 commitOffsetValue = 0;
+  if (CLUSTERING == getMessageModel()) {
+    commitOffsetValue = m_pOffsetStore->readOffset(
+        messageQueue, READ_FROM_MEMORY, getSessionCredentials());
+    if (commitOffsetValue > 0) {
+      commitOffsetEnable = true;
+    }
+  }
+
+  string subExpression;
+  SubscriptionData* pSdata =
+      (m_pRebalance->getSubscriptionData(messageQueue.getTopic()));
+  if (pSdata == NULL) {
+    producePullMsgTask(request);
+    return;
+  }
+  subExpression = pSdata->getSubString();
+
+  int sysFlag =
+      PullSysFlag::buildSysFlag(commitOffsetEnable,      // commitOffset
+                                true,                    // suspend
+                                !subExpression.empty(),  // subscription
+                                false);                  // class filter
+
+  AsyncArg arg;
+  arg.mq = messageQueue;
+  arg.subData = *pSdata;
+  arg.pPullWrapper = m_pPullAPIWrapper;
+
+  try {
+    request->setLastPullTimestamp(UtilAll::currentTimeMillis());
+    m_pPullAPIWrapper->pullKernelImpl(
+        messageQueue,                                 // 1
+        subExpression,                                // 2
+        pSdata->getSubVersion(),                      // 3
+        request->getNextOffset(),                     // 4
+        32,                                           // 5
+        sysFlag,                                      // 6
+        commitOffsetValue,                            // 7
+        1000 * 15,                                    // 8
+        m_asyncPullTimeout,                           // 9
+        ComMode_ASYNC,                                // 10
+        getAsyncPullCallBack(request, messageQueue),  // 11
+        getSessionCredentials(),                      // 12
+        &arg);                                        // 13
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+    producePullMsgTask(request);
+  }
+}
+
+void DefaultMQPushConsumer::setAsyncPull(bool asyncFlag) {
+  if(asyncFlag) {
+    LOG_INFO("set pushConsumer:%s to async default pull mode", getGroupName().c_str());
+  } else {
+    LOG_INFO("set pushConsumer:%s to sync pull mode", getGroupName().c_str());
+  }
+  m_asyncPull = asyncFlag;
+}
+
+void DefaultMQPushConsumer::setConsumeThreadCount(int threadCount) {
+  if (threadCount > 0) {
+    m_consumeThreadCount = threadCount;
+  } else {
+    LOG_ERROR("setConsumeThreadCount with invalid value");
+  }
+}
+
+int DefaultMQPushConsumer::getConsumeThreadCount() const {
+  return m_consumeThreadCount;
+}
+
+void DefaultMQPushConsumer::setPullMsgThreadPoolCount(int threadCount) {
+  m_pullMsgThreadPoolNum = threadCount;
+}
+
+int DefaultMQPushConsumer::getPullMsgThreadPoolCount() const {
+  return m_pullMsgThreadPoolNum;
+}
+
+int DefaultMQPushConsumer::getConsumeMessageBatchMaxSize() const {
+  return m_consumeMessageBatchMaxSize;
+}
+
+void DefaultMQPushConsumer::setConsumeMessageBatchMaxSize(
+    int consumeMessageBatchMaxSize) {
+  if (consumeMessageBatchMaxSize >= 1)
+    m_consumeMessageBatchMaxSize = consumeMessageBatchMaxSize;
+}
+
+void DefaultMQPushConsumer::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 {
+  return m_maxMsgCacheSize;
+}
+
+ConsumerRunningInfo* DefaultMQPushConsumer::getConsumerRunningInfo() {
+  ConsumerRunningInfo* info = new ConsumerRunningInfo();
+  if (info) {
+    if(m_consumerServeice->getConsumeMsgSerivceListenerType() == messageListenerOrderly)
+      info->setProperty(ConsumerRunningInfo::PROP_CONSUME_ORDERLY, "true");
+    else
+      info->setProperty(ConsumerRunningInfo::PROP_CONSUME_ORDERLY, "flase");
+    info->setProperty(ConsumerRunningInfo::PROP_THREADPOOL_CORE_SIZE, UtilAll::to_string(m_consumeThreadCount));
+    info->setProperty(ConsumerRunningInfo::PROP_CONSUMER_START_TIMESTAMP,
+                      UtilAll::to_string(m_startTime));
+
+    vector<SubscriptionData> result;
+    getSubscriptions(result);
+    info->setSubscriptionSet(result);
+
+    map<MQMessageQueue, PullRequest*> requestTable =
+        m_pRebalance->getPullRequestTable();
+    map<MQMessageQueue, PullRequest*>::iterator it = requestTable.begin();
+
+    for (; it != requestTable.end(); ++it) {
+      if (!it->second->isDroped()) {
+        map<MessageQueue, ProcessQueueInfo> queueTable;
+        MessageQueue queue((it->first).getTopic(), (it->first).getBrokerName(),
+                           (it->first).getQueueId());
+        ProcessQueueInfo processQueue;
+        processQueue.cachedMsgMinOffset = it->second->getCacheMinOffset();
+        processQueue.cachedMsgMaxOffset = it->second->getCacheMaxOffset();
+        processQueue.cachedMsgCount = it->second->getCacheMsgCount();
+        processQueue.setCommitOffset(m_pOffsetStore->readOffset(
+            it->first, MEMORY_FIRST_THEN_STORE, getSessionCredentials()));
+        processQueue.setDroped(it->second->isDroped());
+        processQueue.setLocked(it->second->isLocked());
+        processQueue.lastLockTimestamp = it->second->getLastLockTimestamp();
+        processQueue.lastPullTimestamp = it->second->getLastPullTimestamp();
+        processQueue.lastConsumeTimestamp =
+            it->second->getLastConsumeTimestamp();
+        info->setMqTable(queue, processQueue);
+      }
+    }
+
+    return info;
+  }
+  return NULL;
+}
+
+//<!************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/FindBrokerResult.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/FindBrokerResult.h b/rocketmq-cpp/src/consumer/FindBrokerResult.h
new file mode 100755
index 0000000..a224b14
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/FindBrokerResult.h
@@ -0,0 +1,33 @@
+/*
+ * 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 __FINDBROKERRESULT_H__
+#define __FINDBROKERRESULT_H__
+
+namespace rocketmq {
+//<!************************************************************************
+struct FindBrokerResult {
+  FindBrokerResult(const std::string& sbrokerAddr, bool bslave)
+      : brokerAddr(sbrokerAddr), slave(bslave) {}
+
+ public:
+  std::string brokerAddr;
+  bool slave;
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/OffsetStore.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/OffsetStore.cpp b/rocketmq-cpp/src/consumer/OffsetStore.cpp
new file mode 100755
index 0000000..33cf9ed
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/OffsetStore.cpp
@@ -0,0 +1,344 @@
+/*
+ * 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 "OffsetStore.h"
+#include "Logging.h"
+#include "MQClientFactory.h"
+#include "MessageQueue.h"
+
+#include <fstream>
+#include <sstream>
+
+#include <boost/archive/binary_iarchive.hpp>
+#include <boost/archive/binary_oarchive.hpp>
+#include <boost/archive/text_iarchive.hpp>
+#include <boost/archive/text_oarchive.hpp>
+#include <boost/serialization/map.hpp>
+
+namespace rocketmq {
+
+//<!***************************************************************************
+OffsetStore::OffsetStore(const string& groupName, MQClientFactory* pfactory)
+    : m_groupName(groupName), m_pClientFactory(pfactory) {}
+
+OffsetStore::~OffsetStore() {
+  m_pClientFactory = NULL;
+  m_offsetTable.clear();
+}
+
+//<!***************************************************************************
+LocalFileOffsetStore::LocalFileOffsetStore(const string& groupName,
+                                           MQClientFactory* pfactory)
+    : OffsetStore(groupName, pfactory) {
+  MQConsumer* pConsumer = pfactory->selectConsumer(groupName);
+  if (pConsumer) {
+    LOG_INFO("new LocalFileOffsetStore");
+    string directoryName =
+        UtilAll::getLocalAddress() + "@" + pConsumer->getInstanceName();
+    m_storePath = ".rocketmq_offsets/" + directoryName + "/" + groupName + "/";
+    string homeDir(UtilAll::getHomeDirectory());
+    m_storeFile = homeDir + "/" + m_storePath + "offsets.Json";
+
+    string storePath(homeDir);
+    storePath.append(m_storePath);
+    if (access(storePath.c_str(), F_OK) != 0) {
+      if (mkdir(storePath.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) != 0) {
+        LOG_ERROR("create data dir:%s error", storePath.c_str());
+      }
+    }
+  }
+}
+
+LocalFileOffsetStore::~LocalFileOffsetStore() {}
+
+void LocalFileOffsetStore::load() {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+
+  std::ifstream ifs(m_storeFile.c_str(), std::ios::in);
+  if (ifs.good()) {
+    if (ifs.is_open()) {
+      if (ifs.peek() != std::ifstream::traits_type::eof()) {
+        map<string, int64> m_offsetTable_tmp;
+        boost::archive::text_iarchive ia(ifs);
+        ia >> m_offsetTable_tmp;
+        ifs.close();
+
+        for (map<string, int64>::iterator it = m_offsetTable_tmp.begin();
+             it != m_offsetTable_tmp.end(); ++it) {
+          // LOG_INFO("it->first:%s, it->second:%lld", it->first.c_str(),
+          // it->second);
+          Json::Reader reader;
+          Json::Value object;
+          reader.parse(it->first.c_str(), object);
+          MQMessageQueue mq(object["topic"].asString(),
+                            object["brokerName"].asString(),
+                            object["queueId"].asInt());
+          m_offsetTable[mq] = it->second;
+        }
+        m_offsetTable_tmp.clear();
+        /*for(map<MQMessageQueue, int64>::iterator it2 = m_offsetTable.begin();
+        it2!=m_offsetTable.end();++it2 ){
+            LOG_INFO("it->first:%s, it->second:%lld",
+        it2->first.toString().c_str(), it2->second);
+        }*/
+      } else {
+        LOG_ERROR(
+            "open offset store file failed, please check whether file: %s is "
+            "cleared by operator, if so, delete this offsets.Json file and "
+            "then restart consumer",
+            m_storeFile.c_str());
+        THROW_MQEXCEPTION(MQClientException,
+                          "open offset store file failed, please check whether "
+                          "offsets.Json is cleared by operator, if so, delete "
+                          "this offsets.Json file and then restart consumer",
+                          -1);
+      }
+    } else {
+      LOG_ERROR(
+          "open offset store file failed, please check whether file:%s is "
+          "deleted by operator and then restart consumer",
+          m_storeFile.c_str());
+      THROW_MQEXCEPTION(MQClientException,
+                        "open offset store file failed, please check "
+                        "directory:%s is deleted by operator or offset.Json "
+                        "file is cleared by operator, and then restart "
+                        "consumer",
+                        -1);
+    }
+  } else {
+    LOG_WARN(
+        "offsets.Json file not exist, maybe this is the first time "
+        "consumation");
+  }
+}
+
+void LocalFileOffsetStore::updateOffset(const MQMessageQueue& mq,
+                                        int64 offset) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+  m_offsetTable[mq] = offset;
+}
+
+int64 LocalFileOffsetStore::readOffset(
+    const MQMessageQueue& mq, ReadOffsetType type,
+    const SessionCredentials& session_credentials) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+
+  switch (type) {
+    case MEMORY_FIRST_THEN_STORE:
+    case READ_FROM_MEMORY: {
+      MQ2OFFSET::iterator it = m_offsetTable.find(mq);
+      if (it != m_offsetTable.end()) {
+        return it->second;
+      } else if (READ_FROM_MEMORY == type) {
+        return -1;
+      }
+    }
+    case READ_FROM_STORE: {
+      try {
+        load();
+      } catch (MQException& e) {
+        LOG_ERROR("catch exception when load local file");
+        return -1;
+      }
+      MQ2OFFSET::iterator it = m_offsetTable.find(mq);
+      if (it != m_offsetTable.end()) {
+        return it->second;
+      }
+    }
+    default:
+      break;
+  }
+  LOG_ERROR(
+      "can not readOffset from offsetStore.json, maybe first time consumation");
+  return -1;
+}
+
+void LocalFileOffsetStore::persist(
+    const MQMessageQueue& mq, const SessionCredentials& session_credentials) {}
+
+void LocalFileOffsetStore::persistAll(const std::vector<MQMessageQueue>& mqs) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+
+  map<string, int64> m_offsetTable_tmp;
+  vector<MQMessageQueue>::const_iterator it = mqs.begin();
+  for (; it != mqs.end(); ++it) {
+    MessageQueue mq_tmp((*it).getTopic(), (*it).getBrokerName(),
+                        (*it).getQueueId());
+    string mqKey = mq_tmp.toJson().toStyledString();
+    m_offsetTable_tmp[mqKey] = m_offsetTable[*it];
+  }
+
+  std::ofstream s;
+  s.open(m_storeFile.c_str(), std::ios::out);
+  if (s.is_open()) {
+    boost::archive::text_oarchive oa(s);
+    // Boost is nervous that archiving non-const class instances which might
+    // cause a problem with object tracking if different tracked objects use the
+    // same address.
+    oa << const_cast<const map<string, int64>&>(m_offsetTable_tmp);
+    s.close();
+    m_offsetTable_tmp.clear();
+  } else {
+    LOG_ERROR("open offset store file failed");
+    m_offsetTable_tmp.clear();
+    THROW_MQEXCEPTION(MQClientException,
+                      "persistAll:open offset store file failed", -1);
+  }
+}
+
+void LocalFileOffsetStore::removeOffset(const MQMessageQueue& mq) {}
+
+//<!***************************************************************************
+RemoteBrokerOffsetStore::RemoteBrokerOffsetStore(const string& groupName,
+                                                 MQClientFactory* pfactory)
+    : OffsetStore(groupName, pfactory) {}
+
+RemoteBrokerOffsetStore::~RemoteBrokerOffsetStore() {}
+
+void RemoteBrokerOffsetStore::load() {}
+
+void RemoteBrokerOffsetStore::updateOffset(const MQMessageQueue& mq,
+                                           int64 offset) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+  m_offsetTable[mq] = offset;
+}
+
+int64 RemoteBrokerOffsetStore::readOffset(
+    const MQMessageQueue& mq, ReadOffsetType type,
+    const SessionCredentials& session_credentials) {
+  switch (type) {
+    case MEMORY_FIRST_THEN_STORE:
+    case READ_FROM_MEMORY: {
+      boost::lock_guard<boost::mutex> lock(m_lock);
+
+      MQ2OFFSET::iterator it = m_offsetTable.find(mq);
+      if (it != m_offsetTable.end()) {
+        return it->second;
+      } else if (READ_FROM_MEMORY == type) {
+        return -1;
+      }
+    }
+    case READ_FROM_STORE: {
+      try {
+        int64 brokerOffset =
+            fetchConsumeOffsetFromBroker(mq, session_credentials);
+        //<!update;
+        updateOffset(mq, brokerOffset);
+        return brokerOffset;
+      } catch (MQBrokerException& e) {
+        LOG_ERROR(e.what());
+        return -1;
+      } catch (MQException& e) {
+        LOG_ERROR(e.what());
+        return -2;
+      }
+    }
+    default:
+      break;
+  }
+  return -1;
+}
+
+void RemoteBrokerOffsetStore::persist(
+    const MQMessageQueue& mq, const SessionCredentials& session_credentials) {
+  MQ2OFFSET offsetTable;
+  {
+    boost::lock_guard<boost::mutex> lock(m_lock);
+    offsetTable = m_offsetTable;
+  }
+
+  MQ2OFFSET::iterator it = offsetTable.find(mq);
+  if (it != offsetTable.end()) {
+    try {
+      updateConsumeOffsetToBroker(mq, it->second, session_credentials);
+    } catch (MQException& e) {
+      LOG_ERROR("updateConsumeOffsetToBroker error");
+    }
+  }
+}
+
+void RemoteBrokerOffsetStore::persistAll(
+    const std::vector<MQMessageQueue>& mq) {}
+
+void RemoteBrokerOffsetStore::removeOffset(const MQMessageQueue& mq) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+  if (m_offsetTable.find(mq) != m_offsetTable.end()) m_offsetTable.erase(mq);
+}
+
+void RemoteBrokerOffsetStore::updateConsumeOffsetToBroker(
+    const MQMessageQueue& mq, int64 offset,
+    const SessionCredentials& session_credentials) {
+  unique_ptr<FindBrokerResult> pFindBrokerResult(
+      m_pClientFactory->findBrokerAddressInAdmin(mq.getBrokerName()));
+
+  if (pFindBrokerResult == NULL) {
+    m_pClientFactory->updateTopicRouteInfoFromNameServer(mq.getTopic(),
+                                                         session_credentials);
+    pFindBrokerResult.reset(
+        m_pClientFactory->findBrokerAddressInAdmin(mq.getBrokerName()));
+  }
+
+  if (pFindBrokerResult != NULL) {
+    UpdateConsumerOffsetRequestHeader* pRequestHeader =
+        new UpdateConsumerOffsetRequestHeader();
+    pRequestHeader->topic = mq.getTopic();
+    pRequestHeader->consumerGroup = m_groupName;
+    pRequestHeader->queueId = mq.getQueueId();
+    pRequestHeader->commitOffset = offset;
+
+    try {
+      LOG_INFO(
+          "oneway updateConsumeOffsetToBroker of mq:%s, its offset is:%lld",
+          mq.toString().c_str(), offset);
+      return m_pClientFactory->getMQClientAPIImpl()->updateConsumerOffsetOneway(
+          pFindBrokerResult->brokerAddr, pRequestHeader, 1000 * 5,
+          session_credentials);
+    } catch (MQException& e) {
+      LOG_ERROR(e.what());
+    }
+  }
+  LOG_WARN("The broker not exist");
+}
+
+int64 RemoteBrokerOffsetStore::fetchConsumeOffsetFromBroker(
+    const MQMessageQueue& mq, const SessionCredentials& session_credentials) {
+  unique_ptr<FindBrokerResult> pFindBrokerResult(
+      m_pClientFactory->findBrokerAddressInAdmin(mq.getBrokerName()));
+
+  if (pFindBrokerResult == NULL) {
+    m_pClientFactory->updateTopicRouteInfoFromNameServer(mq.getTopic(),
+                                                         session_credentials);
+    pFindBrokerResult.reset(
+        m_pClientFactory->findBrokerAddressInAdmin(mq.getBrokerName()));
+  }
+
+  if (pFindBrokerResult != NULL) {
+    QueryConsumerOffsetRequestHeader* pRequestHeader =
+        new QueryConsumerOffsetRequestHeader();
+    pRequestHeader->topic = mq.getTopic();
+    pRequestHeader->consumerGroup = m_groupName;
+    pRequestHeader->queueId = mq.getQueueId();
+
+    return m_pClientFactory->getMQClientAPIImpl()->queryConsumerOffset(
+        pFindBrokerResult->brokerAddr, pRequestHeader, 1000 * 5,
+        session_credentials);
+  } else {
+    LOG_ERROR("The broker not exist when fetchConsumeOffsetFromBroker");
+    THROW_MQEXCEPTION(MQClientException, "The broker not exist", -1);
+  }
+}
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/OffsetStore.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/OffsetStore.h b/rocketmq-cpp/src/consumer/OffsetStore.h
new file mode 100755
index 0000000..269198f
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/OffsetStore.h
@@ -0,0 +1,108 @@
+/*
+ * 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 __OFFSETSTORE_H__
+#define __OFFSETSTORE_H__
+
+#include <boost/asio.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include <map>
+#include "MQMessageQueue.h"
+#include "RocketMQClient.h"
+#include "SessionCredentials.h"
+
+namespace rocketmq {
+class MQClientFactory;
+//<!***************************************************************************
+enum ReadOffsetType {
+  //read offset from memory
+  READ_FROM_MEMORY,
+  //read offset from remoting
+  READ_FROM_STORE,
+  //read offset from memory firstly, then from remoting
+  MEMORY_FIRST_THEN_STORE,
+};
+
+//<!***************************************************************************
+class OffsetStore {
+ public:
+  OffsetStore(const std::string& groupName, MQClientFactory*);
+  virtual ~OffsetStore();
+
+  virtual void load() = 0;
+  virtual void updateOffset(const MQMessageQueue& mq, int64 offset) = 0;
+  virtual int64 readOffset(const MQMessageQueue& mq, ReadOffsetType type,
+                           const SessionCredentials& session_credentials) = 0;
+  virtual void persist(const MQMessageQueue& mq,
+                       const SessionCredentials& session_credentials) = 0;
+  virtual void persistAll(const std::vector<MQMessageQueue>& mq) = 0;
+  virtual void removeOffset(const MQMessageQueue& mq) = 0;
+
+ protected:
+  std::string m_groupName;
+  typedef std::map<MQMessageQueue, int64> MQ2OFFSET;
+  MQ2OFFSET m_offsetTable;
+  MQClientFactory* m_pClientFactory;
+  boost::mutex m_lock;
+};
+
+//<!***************************************************************************
+class LocalFileOffsetStore : public OffsetStore {
+ public:
+  LocalFileOffsetStore(const std::string& groupName, MQClientFactory*);
+  virtual ~LocalFileOffsetStore();
+
+  virtual void load();
+  virtual void updateOffset(const MQMessageQueue& mq, int64 offset);
+  virtual int64 readOffset(const MQMessageQueue& mq, ReadOffsetType type,
+                           const SessionCredentials& session_credentials);
+  virtual void persist(const MQMessageQueue& mq,
+                       const SessionCredentials& session_credentials);
+  virtual void persistAll(const std::vector<MQMessageQueue>& mq);
+  virtual void removeOffset(const MQMessageQueue& mq);
+
+ private:
+  std::string m_storePath;
+  std::string m_storeFile;
+};
+
+//<!***************************************************************************
+class RemoteBrokerOffsetStore : public OffsetStore {
+ public:
+  RemoteBrokerOffsetStore(const std::string& groupName, MQClientFactory*);
+  virtual ~RemoteBrokerOffsetStore();
+
+  virtual void load();
+  virtual void updateOffset(const MQMessageQueue& mq, int64 offset);
+  virtual int64 readOffset(const MQMessageQueue& mq, ReadOffsetType type,
+                           const SessionCredentials& session_credentials);
+  virtual void persist(const MQMessageQueue& mq,
+                       const SessionCredentials& session_credentials);
+  virtual void persistAll(const std::vector<MQMessageQueue>& mq);
+  virtual void removeOffset(const MQMessageQueue& mq);
+
+ private:
+  void updateConsumeOffsetToBroker(
+      const MQMessageQueue& mq, int64 offset,
+      const SessionCredentials& session_credentials);
+  int64 fetchConsumeOffsetFromBroker(
+      const MQMessageQueue& mq, const SessionCredentials& session_credentials);
+};
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullAPIWrapper.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullAPIWrapper.cpp b/rocketmq-cpp/src/consumer/PullAPIWrapper.cpp
new file mode 100755
index 0000000..6a4b507
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullAPIWrapper.cpp
@@ -0,0 +1,139 @@
+/*
+ * 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 "PullAPIWrapper.h"
+#include "CommunicationMode.h"
+#include "MQClientFactory.h"
+#include "PullResultExt.h"
+#include "PullSysFlag.h"
+namespace rocketmq {
+//<!************************************************************************
+PullAPIWrapper::PullAPIWrapper(MQClientFactory* mQClientFactory,
+                               const string& consumerGroup) {
+  m_MQClientFactory = mQClientFactory;
+  m_consumerGroup = consumerGroup;
+}
+
+PullAPIWrapper::~PullAPIWrapper() {
+  m_MQClientFactory = NULL;
+  m_pullFromWhichNodeTable.clear();
+}
+
+void PullAPIWrapper::updatePullFromWhichNode(const MQMessageQueue& mq,
+                                             int brokerId) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+  m_pullFromWhichNodeTable[mq] = brokerId;
+}
+
+int PullAPIWrapper::recalculatePullFromWhichNode(const MQMessageQueue& mq) {
+  boost::lock_guard<boost::mutex> lock(m_lock);
+  if (m_pullFromWhichNodeTable.find(mq) != m_pullFromWhichNodeTable.end()) {
+    return m_pullFromWhichNodeTable[mq];
+  }
+  return MASTER_ID;
+}
+
+PullResult PullAPIWrapper::processPullResult(
+    const MQMessageQueue& mq, PullResult* pullResult,
+    SubscriptionData* subscriptionData) {
+  PullResultExt* pResultExt = static_cast<PullResultExt*>(pullResult);
+  if (pResultExt == NULL) {
+    string errMsg("The pullResult NULL of");
+    errMsg.append(mq.toString());
+    THROW_MQEXCEPTION(MQClientException, errMsg, -1);
+  }
+
+  //<!update;
+  updatePullFromWhichNode(mq, pResultExt->suggestWhichBrokerId);
+
+  vector<MQMessageExt> msgFilterList;
+  if (pResultExt->pullStatus == FOUND) {
+    //<!decode all msg list;
+    vector<MQMessageExt> msgAllList;
+    MQDecoder::decodes(&pResultExt->msgMemBlock, msgAllList);
+
+    //<!filter msg list again;
+    if (subscriptionData != NULL && !subscriptionData->getTagsSet().empty()) {
+      msgFilterList.reserve(msgAllList.size());
+      vector<MQMessageExt>::iterator it = msgAllList.begin();
+      for (; it != msgAllList.end(); ++it) {
+        string msgTag = (*it).getTags();
+        if (subscriptionData->containTag(msgTag)) {
+          msgFilterList.push_back(*it);
+        }
+      }
+    } else
+    {
+      msgFilterList.swap(msgAllList);
+    }
+  }
+
+  return PullResult(pResultExt->pullStatus, pResultExt->nextBeginOffset,
+                    pResultExt->minOffset, pResultExt->maxOffset,
+                    msgFilterList);
+}
+
+PullResult* PullAPIWrapper::pullKernelImpl(
+    const MQMessageQueue& mq,        // 1
+    string subExpression,            // 2
+    int64 subVersion,                // 3
+    int64 offset,                    // 4
+    int maxNums,                     // 5
+    int sysFlag,                     // 6
+    int64 commitOffset,              // 7
+    int brokerSuspendMaxTimeMillis,  // 8
+    int timeoutMillis,               // 9
+    int communicationMode,           // 10
+    PullCallback* pullCallback, const SessionCredentials& session_credentials,
+    void* pArg /*= NULL*/) {
+  unique_ptr<FindBrokerResult> pFindBrokerResult(
+      m_MQClientFactory->findBrokerAddressInSubscribe(
+          mq.getBrokerName(), recalculatePullFromWhichNode(mq), false));
+  //<!goto nameserver;
+  if (pFindBrokerResult == NULL) {
+    m_MQClientFactory->updateTopicRouteInfoFromNameServer(mq.getTopic(),
+                                                          session_credentials);
+    pFindBrokerResult.reset(m_MQClientFactory->findBrokerAddressInSubscribe(
+        mq.getBrokerName(), recalculatePullFromWhichNode(mq), false));
+  }
+
+  if (pFindBrokerResult != NULL) {
+    int sysFlagInner = sysFlag;
+
+    if (pFindBrokerResult->slave) {
+      sysFlagInner = PullSysFlag::clearCommitOffsetFlag(sysFlagInner);
+    }
+
+    PullMessageRequestHeader* pRequestHeader = new PullMessageRequestHeader();
+    pRequestHeader->consumerGroup = m_consumerGroup;
+    pRequestHeader->topic = mq.getTopic();
+    pRequestHeader->queueId = mq.getQueueId();
+    pRequestHeader->queueOffset = offset;
+    pRequestHeader->maxMsgNums = maxNums;
+    pRequestHeader->sysFlag = sysFlagInner;
+    pRequestHeader->commitOffset = commitOffset;
+    pRequestHeader->suspendTimeoutMillis = brokerSuspendMaxTimeMillis;
+    pRequestHeader->subscription = subExpression;
+    pRequestHeader->subVersion = subVersion;
+
+    return m_MQClientFactory->getMQClientAPIImpl()->pullMessage(
+        pFindBrokerResult->brokerAddr, pRequestHeader, timeoutMillis,
+        communicationMode, pullCallback, pArg, session_credentials);
+  }
+  THROW_MQEXCEPTION(MQClientException, "The broker not exist", -1);
+}
+
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullAPIWrapper.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullAPIWrapper.h b/rocketmq-cpp/src/consumer/PullAPIWrapper.h
new file mode 100755
index 0000000..e3d0a1e
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullAPIWrapper.h
@@ -0,0 +1,67 @@
+/*
+ * 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 _PULLAPIWRAPPER_H_
+#define _PULLAPIWRAPPER_H_
+
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include "AsyncCallback.h"
+#include "MQMessageQueue.h"
+#include "SessionCredentials.h"
+#include "SubscriptionData.h"
+
+namespace rocketmq {
+class MQClientFactory;
+//<!***************************************************************************
+class PullAPIWrapper {
+ public:
+  PullAPIWrapper(MQClientFactory* mQClientFactory, const string& consumerGroup);
+  ~PullAPIWrapper();
+
+  PullResult processPullResult(const MQMessageQueue& mq, PullResult* pullResult,
+                               SubscriptionData* subscriptionData);
+
+  PullResult* pullKernelImpl(const MQMessageQueue& mq,        // 1
+                             string subExpression,            // 2
+                             int64 subVersion,                // 3
+                             int64 offset,                    // 4
+                             int maxNums,                     // 5
+                             int sysFlag,                     // 6
+                             int64 commitOffset,              // 7
+                             int brokerSuspendMaxTimeMillis,  // 8
+                             int timeoutMillis,               // 9
+                             int communicationMode,           // 10
+                             PullCallback* pullCallback,
+                             const SessionCredentials& session_credentials,
+                             void* pArg = NULL);
+
+ private:
+  void updatePullFromWhichNode(const MQMessageQueue& mq, int brokerId);
+
+  int recalculatePullFromWhichNode(const MQMessageQueue& mq);
+
+ private:
+  MQClientFactory* m_MQClientFactory;
+  string m_consumerGroup;
+  boost::mutex m_lock;
+  map<MQMessageQueue, int /* brokerId */> m_pullFromWhichNodeTable;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif  //<! _PULLAPIWRAPPER_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullRequest.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullRequest.cpp b/rocketmq-cpp/src/consumer/PullRequest.cpp
new file mode 100755
index 0000000..d9b953f
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullRequest.cpp
@@ -0,0 +1,244 @@
+/*
+ * 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 "PullRequest.h"
+#include "Logging.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+const uint64 PullRequest::RebalanceLockInterval = 20 * 1000;
+const uint64 PullRequest::RebalanceLockMaxLiveTime = 30 * 1000;
+
+PullRequest::PullRequest(const string& groupname)
+    : m_groupname(groupname), m_nextOffset(0), m_queueOffsetMax(0), m_bDroped(false), m_bLocked(false) {}
+
+PullRequest::~PullRequest() {
+  m_msgTreeMapTemp.clear();
+  m_msgTreeMap.clear();
+}
+
+PullRequest& PullRequest::operator=(const PullRequest& other) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  if (this != &other) {
+    m_groupname = other.m_groupname;
+    m_nextOffset = other.m_nextOffset;
+    m_bDroped.store(other.m_bDroped.load());
+    m_queueOffsetMax = other.m_queueOffsetMax;
+    m_messageQueue = other.m_messageQueue;
+    m_msgTreeMap = other.m_msgTreeMap;
+    m_msgTreeMapTemp = other.m_msgTreeMapTemp;
+  }
+  return *this;
+}
+
+void PullRequest::putMessage(vector<MQMessageExt>& msgs) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+
+  vector<MQMessageExt>::iterator it = msgs.begin();
+  for (; it != msgs.end(); it++) {
+    m_msgTreeMap[it->getQueueOffset()] = *it;
+    m_queueOffsetMax = std::max(m_queueOffsetMax, it->getQueueOffset());
+  }
+  LOG_DEBUG("PullRequest: putMessage m_queueOffsetMax:%lld ", m_queueOffsetMax);
+}
+
+void PullRequest::getMessage(vector<MQMessageExt>& msgs) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+
+  map<int64, MQMessageExt>::iterator it = m_msgTreeMap.begin();
+  for (; it != m_msgTreeMap.end(); it++) {
+    msgs.push_back(it->second);
+  }
+}
+
+int64 PullRequest::getCacheMinOffset() {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  if (m_msgTreeMap.empty()) {
+    return 0;
+  } else {
+    map<int64, MQMessageExt>::iterator it = m_msgTreeMap.begin();
+    MQMessageExt msg = it->second;
+    return msg.getQueueOffset();
+  }
+}
+
+int64 PullRequest::getCacheMaxOffset() { return m_queueOffsetMax; }
+
+int PullRequest::getCacheMsgCount() {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  return m_msgTreeMap.size();
+}
+
+void PullRequest::getMessageByQueueOffset(vector<MQMessageExt>& msgs,
+                                          int64 minQueueOffset,
+                                          int64 maxQueueOffset) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+
+  int64 it = minQueueOffset;
+  for (; it <= maxQueueOffset; it++) {
+    msgs.push_back(m_msgTreeMap[it]);
+  }
+}
+
+int64 PullRequest::removeMessage(vector<MQMessageExt>& msgs) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+
+  int64 result = -1;
+  LOG_DEBUG("m_queueOffsetMax is:%lld", m_queueOffsetMax);
+  if (!m_msgTreeMap.empty()) {
+    result = m_queueOffsetMax + 1;
+    LOG_DEBUG(" offset result is:%lld, m_queueOffsetMax is:%lld, msgs size:%zu",
+              result, m_queueOffsetMax, msgs.size());
+    vector<MQMessageExt>::iterator it = msgs.begin();
+    for (; it != msgs.end(); it++) {
+      LOG_DEBUG("remove these msg from m_msgTreeMap, its offset:%lld",
+                it->getQueueOffset());
+      m_msgTreeMap.erase(it->getQueueOffset());
+    }
+
+    if (!m_msgTreeMap.empty()) {
+      map<int64, MQMessageExt>::iterator it = m_msgTreeMap.begin();
+      result = it->first;
+      LOG_INFO("cache msg size:%zu of pullRequest:%s, return offset result is:%lld",
+               m_msgTreeMap.size(), m_messageQueue.toString().c_str(), result);
+    }
+  }
+
+  return result;
+}
+
+void PullRequest::clearAllMsgs() {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+
+  if (isDroped()) {
+    LOG_DEBUG("clear m_msgTreeMap as PullRequest had been dropped.");
+    m_msgTreeMap.clear();
+    m_msgTreeMapTemp.clear();
+  }
+}
+
+void PullRequest::updateQueueMaxOffset(int64 queueOffset) {
+  // following 2 cases which may set queueOffset smaller than m_queueOffsetMax:
+  // 1. resetOffset cmd
+  // 2. during  rebalance, if configured with CONSUMER_FROM_FIRST_OFFSET, when
+  // readOffset called by computePullFromWhere was failed,  m_nextOffset will be
+  // setted to 0
+  m_queueOffsetMax = queueOffset;
+}
+
+void PullRequest::setDroped(bool droped) {
+  int temp = (droped == true ? 1 : 0);
+  m_bDroped.store(temp);
+  /*
+  m_queueOffsetMax = 0;
+  m_nextOffset = 0;
+  //the reason why not clear m_queueOffsetMax and m_nextOffset is due to
+  ConsumeMsgService and drop mq are concurrent running.
+      consider following situation:
+      1>. ConsumeMsgService running
+      2>. dorebalance, drop mq, reset m_nextOffset and m_queueOffsetMax
+      3>. ConsumeMsgService calls removeMessages, if no other msgs in
+  m_msgTreeMap, m_queueOffsetMax(0)+1 will return;
+      4>. updateOffset with 1, which is more smaller than correct offset.
+  */
+}
+
+bool PullRequest::isDroped() const { return m_bDroped.load() == 1; }
+
+int64 PullRequest::getNextOffset() {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  return m_nextOffset;
+}
+
+void PullRequest::setLocked(bool Locked) {
+  int temp = (Locked == true ? 1 : 0);
+  m_bLocked.store(temp);
+}
+bool PullRequest::isLocked() const { return m_bLocked.load() == 1; }
+
+bool PullRequest::isLockExpired() const {
+  return (UtilAll::currentTimeMillis() - m_lastLockTimestamp) >
+         RebalanceLockMaxLiveTime;
+}
+
+void PullRequest::setLastLockTimestamp(int64 time) {
+  m_lastLockTimestamp = time;
+}
+
+int64 PullRequest::getLastLockTimestamp() const { return m_lastLockTimestamp; }
+
+void PullRequest::setLastPullTimestamp(uint64 time) {
+  m_lastPullTimestamp = time;
+}
+
+uint64 PullRequest::getLastPullTimestamp() const { return m_lastPullTimestamp; }
+
+void PullRequest::setLastConsumeTimestamp(uint64 time) {
+  m_lastConsumeTimestamp = time;
+}
+
+uint64 PullRequest::getLastConsumeTimestamp() const {
+  return m_lastConsumeTimestamp;
+}
+
+void PullRequest::setTryUnlockTimes(int time) { m_lastLockTimestamp = time; }
+
+int PullRequest::getTryUnlockTimes() const { return m_lastLockTimestamp; }
+
+void PullRequest::setNextOffset(int64 nextoffset) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  m_nextOffset = nextoffset;
+}
+
+string PullRequest::getGroupName() const { return m_groupname; }
+
+boost::timed_mutex& PullRequest::getPullRequestCriticalSection() {
+  return m_consumeLock;
+}
+
+void PullRequest::takeMessages(vector<MQMessageExt>& msgs, int batchSize) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  for (int i = 0; i != batchSize; i++) {
+    map<int64, MQMessageExt>::iterator it = m_msgTreeMap.begin();
+    if (it != m_msgTreeMap.end()) {
+      msgs.push_back(it->second);
+      m_msgTreeMapTemp[it->first] = it->second;
+      m_msgTreeMap.erase(it);
+    }
+  }
+}
+
+void PullRequest::makeMessageToCosumeAgain(vector<MQMessageExt>& msgs) {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  for (unsigned int it = 0; it != msgs.size(); ++it) {
+    m_msgTreeMap[msgs[it].getQueueOffset()] = msgs[it];
+    m_msgTreeMapTemp.erase(msgs[it].getQueueOffset());
+  }
+}
+
+int64 PullRequest::commit() {
+  boost::lock_guard<boost::mutex> lock(m_pullRequestLock);
+  if (!m_msgTreeMapTemp.empty()) {
+    int64 offset = (--m_msgTreeMapTemp.end())->first;
+    m_msgTreeMapTemp.clear();
+    return offset + 1;
+  } else {
+    return -1;
+  }
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullRequest.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullRequest.h b/rocketmq-cpp/src/consumer/PullRequest.h
new file mode 100755
index 0000000..6cd2180
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullRequest.h
@@ -0,0 +1,95 @@
+/*
+ * 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 __PULLREQUEST_H__
+#define __PULLREQUEST_H__
+
+#include <boost/atomic.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/thread.hpp>
+#include "ByteOrder.h"
+#include "MQMessageExt.h"
+#include "MQMessageQueue.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class PullRequest {
+ public:
+  PullRequest(const string& groupname);
+  virtual ~PullRequest();
+
+  void putMessage(vector<MQMessageExt>& msgs);
+  void getMessage(vector<MQMessageExt>& msgs);
+  int64 getCacheMinOffset();
+  int64 getCacheMaxOffset();
+  int getCacheMsgCount();
+  void getMessageByQueueOffset(vector<MQMessageExt>& msgs, int64 minQueueOffset,
+                               int64 maxQueueOffset);
+  int64 removeMessage(vector<MQMessageExt>& msgs);
+  void clearAllMsgs();
+
+  PullRequest& operator=(const PullRequest& other);
+
+  void setDroped(bool droped);
+  bool isDroped() const;
+
+  int64 getNextOffset();
+  void setNextOffset(int64 nextoffset);
+
+  string getGroupName() const;
+
+  void updateQueueMaxOffset(int64 queueOffset);
+
+  void setLocked(bool Locked);
+  bool isLocked() const;
+  bool isLockExpired() const;
+  void setLastLockTimestamp(int64 time);
+  int64 getLastLockTimestamp() const;
+  void setLastPullTimestamp(uint64 time);
+  uint64 getLastPullTimestamp() const;
+  void setLastConsumeTimestamp(uint64 time);
+  uint64 getLastConsumeTimestamp() const;
+  void setTryUnlockTimes(int time);
+  int getTryUnlockTimes() const;
+  void takeMessages(vector<MQMessageExt>& msgs, int batchSize);
+  int64 commit();
+  void makeMessageToCosumeAgain(vector<MQMessageExt>& msgs);
+  boost::timed_mutex& getPullRequestCriticalSection();
+
+ public:
+  MQMessageQueue m_messageQueue;
+  static const uint64 RebalanceLockInterval;     // ms
+  static const uint64 RebalanceLockMaxLiveTime;  // ms
+
+ private:
+  string m_groupname;
+  int64 m_nextOffset;
+  int64 m_queueOffsetMax;
+  boost::atomic<bool> m_bDroped;
+  boost::atomic<bool> m_bLocked;
+  map<int64, MQMessageExt> m_msgTreeMap;
+  map<int64, MQMessageExt> m_msgTreeMapTemp;
+  boost::mutex m_pullRequestLock;
+  uint64 m_lastLockTimestamp;  // ms
+  uint64 m_tryUnlockTimes;
+  uint64 m_lastPullTimestamp;
+  uint64 m_lastConsumeTimestamp;
+  boost::timed_mutex m_consumeLock;
+};
+//<!************************************************************************
+}  //<!end namespace;
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullResult.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullResult.cpp b/rocketmq-cpp/src/consumer/PullResult.cpp
new file mode 100755
index 0000000..8648abe
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullResult.cpp
@@ -0,0 +1,54 @@
+/*
+ * 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 "PullResult.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+//<!************************************************************************
+PullResult::PullResult()
+    : pullStatus(NO_MATCHED_MSG),
+      nextBeginOffset(0),
+      minOffset(0),
+      maxOffset(0) {}
+
+PullResult::PullResult(PullStatus status)
+    : pullStatus(status), nextBeginOffset(0), minOffset(0), maxOffset(0) {}
+
+PullResult::PullResult(PullStatus pullStatus, int64 nextBeginOffset,
+                       int64 minOffset, int64 maxOffset)
+    : pullStatus(pullStatus),
+      nextBeginOffset(nextBeginOffset),
+      minOffset(minOffset),
+      maxOffset(maxOffset) {}
+
+PullResult::PullResult(PullStatus pullStatus, int64 nextBeginOffset,
+                       int64 minOffset, int64 maxOffset,
+                       const vector<MQMessageExt>& src)
+    : pullStatus(pullStatus),
+      nextBeginOffset(nextBeginOffset),
+      minOffset(minOffset),
+      maxOffset(maxOffset) {
+  msgFoundList.reserve(src.size());
+  for (size_t i = 0; i < src.size(); i++) {
+    msgFoundList.push_back(src[i]);
+  }
+}
+
+PullResult::~PullResult() { msgFoundList.clear(); }
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/PullResultExt.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/PullResultExt.h b/rocketmq-cpp/src/consumer/PullResultExt.h
new file mode 100755
index 0000000..ac6b8e9
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/PullResultExt.h
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "PullResult.h"
+#include "UtilAll.h"
+#include "dataBlock.h"
+
+namespace rocketmq {
+/**
+ * ֻ���ڲ�ʹ�ã������⹫��
+ */
+//<!***************************************************************************
+class PullResultExt : public PullResult {
+ public:
+  PullResultExt(PullStatus pullStatus, int64 nextBeginOffset, int64 minOffset,
+                int64 maxOffset, int suggestWhichBrokerId,
+                const MemoryBlock& messageBinary)
+      : PullResult(pullStatus, nextBeginOffset, minOffset, maxOffset),
+        suggestWhichBrokerId(suggestWhichBrokerId),
+        msgMemBlock(messageBinary) {}
+  PullResultExt(PullStatus pullStatus, int64 nextBeginOffset, int64 minOffset,
+                int64 maxOffset, int suggestWhichBrokerId)
+      : PullResult(pullStatus, nextBeginOffset, minOffset, maxOffset),
+        suggestWhichBrokerId(suggestWhichBrokerId) {}
+  virtual ~PullResultExt() {}
+
+ public:
+  int suggestWhichBrokerId;
+  MemoryBlock msgMemBlock;
+};
+
+}  //<!end namespace;


[07/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/UtilAll.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/UtilAll.cpp b/rocketmq-cpp/src/common/UtilAll.cpp
new file mode 100755
index 0000000..8d8ff9c
--- /dev/null
+++ b/rocketmq-cpp/src/common/UtilAll.cpp
@@ -0,0 +1,301 @@
+/*
+ * 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 "UtilAll.h"
+
+namespace rocketmq {
+//<!************************************************************************
+std::string UtilAll::s_localHostName;
+std::string UtilAll::s_localIpAddress;
+
+bool UtilAll::startsWith_retry(const string &topic) {
+  return topic.find(RETRY_GROUP_TOPIC_PREFIX) == 0;
+}
+
+string UtilAll::getRetryTopic(const string &consumerGroup) {
+  return RETRY_GROUP_TOPIC_PREFIX + consumerGroup;
+}
+
+void UtilAll::Trim(string &str) {
+  str.erase(0, str.find_first_not_of(' '));  // prefixing spaces
+  str.erase(str.find_last_not_of(' ') + 1);  // surfixing spaces
+}
+
+bool UtilAll::isBlank(const string &str) {
+  if (str.empty()) {
+    return true;
+  }
+
+  string::size_type left = str.find_first_not_of(WHITESPACE);
+
+  if (left == string::npos) {
+    return true;
+  }
+
+  return false;
+}
+
+uint64 UtilAll::hexstr2ull(const char *str) {
+  return boost::lexical_cast<uint64>(str);
+}
+
+int64 UtilAll::str2ll(const char *str) {
+  return boost::lexical_cast<int64>(str);
+}
+
+string UtilAll::bytes2string(const char *bytes, int len) {
+  if (bytes == NULL || len <= 0) {
+    return string();
+  }
+
+  char hex_str[] = "0123456789ABCDEF";
+  char result[len * 2 + 1];
+  result[len * 2] = 0;
+
+  for (int i = 0; i < len; i++) {
+    result[i * 2 + 0] = hex_str[(bytes[i] >> 4) & 0x0F];
+    result[i * 2 + 1] = hex_str[(bytes[i]) & 0x0F];
+  }
+
+  string buffer(result);
+  return buffer;
+}
+
+bool UtilAll::SplitURL(const string &serverURL, string &addr, short &nPort) {
+  size_t pos = serverURL.find(':');
+  if (pos == string::npos) {
+    return false;
+  }
+
+  addr = serverURL.substr(0, pos);
+  if (0 == addr.compare("localhost")) {
+    addr = "127.0.0.1";
+  }
+
+  pos++;
+  string port = serverURL.substr(pos, serverURL.length() - pos);
+  nPort = atoi(port.c_str());
+  if (nPort == 0) {
+    return false;
+  }
+  return true;
+}
+
+int UtilAll::Split(vector<string> &ret_, const string &strIn, const char sep) {
+  if (strIn.empty()) return 0;
+
+  string tmp;
+  string::size_type pos_begin = strIn.find_first_not_of(sep);
+  string::size_type comma_pos = 0;
+
+  while (pos_begin != string::npos) {
+    comma_pos = strIn.find(sep, pos_begin);
+    if (comma_pos != string::npos) {
+      tmp = strIn.substr(pos_begin, comma_pos - pos_begin);
+      pos_begin = comma_pos + 1;
+    } else {
+      tmp = strIn.substr(pos_begin);
+      pos_begin = comma_pos;
+    }
+
+    if (!tmp.empty()) {
+      ret_.push_back(tmp);
+      tmp.clear();
+    }
+  }
+  return ret_.size();
+}
+int UtilAll::Split(vector<string> &ret_, const string &strIn,
+                   const string &sep) {
+  if (strIn.empty()) return 0;
+
+  string tmp;
+  string::size_type pos_begin = strIn.find_first_not_of(sep);
+  string::size_type comma_pos = 0;
+
+  while (pos_begin != string::npos) {
+    comma_pos = strIn.find(sep, pos_begin);
+    if (comma_pos != string::npos) {
+      tmp = strIn.substr(pos_begin, comma_pos - pos_begin);
+      pos_begin = comma_pos + sep.length();
+    } else {
+      tmp = strIn.substr(pos_begin);
+      pos_begin = comma_pos;
+    }
+
+    if (!tmp.empty()) {
+      ret_.push_back(tmp);
+      tmp.clear();
+    }
+  }
+  return ret_.size();
+}
+
+int32_t UtilAll::StringToInt32(const std::string &str, int32_t &out) {
+  out = 0;
+  if (str.empty()) {
+    return false;
+  }
+
+  char *end = NULL;
+  errno = 0;
+  long l = strtol(str.c_str(), &end, 10);
+  /* Both checks are needed because INT_MAX == LONG_MAX is possible. */
+  if (l > INT_MAX || (errno == ERANGE && l == LONG_MAX)) return false;
+  if (l < INT_MIN || (errno == ERANGE && l == LONG_MIN)) return false;
+  if (*end != '\0') return false;
+  out = l;
+  return true;
+}
+
+int64_t UtilAll::StringToInt64(const std::string &str, int64_t &val) {
+  char *endptr = NULL;
+  errno = 0; /* To distinguish success/failure after call */
+  val = strtoll(str.c_str(), &endptr, 10);
+
+  /* Check for various possible errors */
+  if ((errno == ERANGE && (val == LONG_MAX || val == LONG_MIN)) ||
+      (errno != 0 && val == 0)) {
+    return false;
+  }
+  /*no digit was found Or  Further characters after number*/
+  if (endptr == str.c_str()) {
+    return false;
+  }
+  /*no digit was found Or  Further characters after number*/
+  if (*endptr != '\0') {
+    return false;
+  }
+  /* If we got here, strtol() successfully parsed a number */
+  return true;
+}
+
+string UtilAll::getLocalHostName() {
+  if (s_localHostName.empty()) {
+    // boost::system::error_code error;
+    // s_localHostName = boost::asio::ip::host_name(error);
+
+    char name[1024];
+    boost::system::error_code ec;
+    if (boost::asio::detail::socket_ops::gethostname(name, sizeof(name), ec) !=
+        0) {
+      return std::string();
+    }
+    s_localHostName.append(name, strlen(name));
+  }
+  return s_localHostName;
+}
+
+string UtilAll::getLocalAddress() {
+  if (s_localIpAddress.empty()) {
+    boost::asio::io_service io_service;
+    boost::asio::ip::tcp::resolver resolver(io_service);
+    boost::asio::ip::tcp::resolver::query query(getLocalHostName(), "");
+    boost::system::error_code error;
+    boost::asio::ip::tcp::resolver::iterator iter =
+        resolver.resolve(query, error);
+    if (error) {
+      return "";
+    }
+    boost::asio::ip::tcp::resolver::iterator end;  // End marker.
+    boost::asio::ip::tcp::endpoint ep;
+    while (iter != end) {
+      ep = *iter++;
+    }
+    s_localIpAddress = ep.address().to_string();
+  }
+  return s_localIpAddress;
+}
+
+string UtilAll::getHomeDirectory() {
+  char *homeEnv = getenv("HOME");
+  string homeDir;
+  if (homeEnv == NULL) {
+    homeDir.append(getpwuid(getuid())->pw_dir);
+  } else {
+    homeDir.append(homeEnv);
+  }
+  return homeDir;
+}
+
+int UtilAll::getRandomNum(int baseNum) {
+  struct timeval tv;
+  gettimeofday(&tv, NULL);
+  long time = (tv.tv_sec * long(1000000)) + tv.tv_usec;
+  return (int)(time % baseNum);
+}
+
+string UtilAll::getProcessName() {
+  char buf[PATH_MAX + 1] = {0};
+  int count = PATH_MAX + 1;
+  char procpath[PATH_MAX + 1] = {0};
+  sprintf(procpath, "/proc/%d/exe", getpid());
+
+  if (access(procpath, F_OK) == -1) {
+    return "";
+  }
+
+  int retval = readlink(procpath, buf, count - 1);
+  if ((retval < 0 || retval >= count - 1)) {
+    return "";
+  }
+  if (!strcmp(buf + retval - 10, " (deleted)"))
+    buf[retval - 10] = '\0';  // remove last " (deleted)"
+  else
+    buf[retval] = '\0';
+
+  char *process_name = strrchr(buf, '/');
+  if (process_name) {
+    return std::string(process_name + 1);
+  } else {
+    return "";
+  }
+}
+
+uint64_t UtilAll::currentTimeMillis() {
+  struct timeval tv;
+  gettimeofday(&tv, NULL);
+  return ((uint64_t)tv.tv_sec) * 1000 + tv.tv_usec / 1000;
+}
+
+uint64_t UtilAll::currentTimeSeconds() {
+  struct timeval tv;
+  gettimeofday(&tv, NULL);
+  return (uint64_t)tv.tv_sec;
+}
+
+bool UtilAll::deflate(std::string &input, std::string &out, int level) {
+  boost::iostreams::zlib_params zlibParams(level,
+                                           boost::iostreams::zlib::deflated);
+  boost::iostreams::filtering_ostream compressingStream;
+  compressingStream.push(boost::iostreams::zlib_compressor(zlibParams));
+  compressingStream.push(boost::iostreams::back_inserter(out));
+  compressingStream << input;
+  boost::iostreams::close(compressingStream);
+
+  return true;
+}
+
+bool UtilAll::inflate(std::string &input, std::string &out) {
+  boost::iostreams::filtering_ostream decompressingStream;
+  decompressingStream.push(boost::iostreams::zlib_decompressor());
+  decompressingStream.push(boost::iostreams::back_inserter(out));
+  decompressingStream << input;
+  boost::iostreams::close(decompressingStream);
+
+  return true;
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/UtilAll.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/UtilAll.h b/rocketmq-cpp/src/common/UtilAll.h
new file mode 100755
index 0000000..cf66688
--- /dev/null
+++ b/rocketmq-cpp/src/common/UtilAll.h
@@ -0,0 +1,131 @@
+/*
+ * 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 __UTILALL_H__
+#define __UTILALL_H__
+
+#include <assert.h>
+#include <errno.h>
+#include <pwd.h>
+#include <stdint.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/time.h>
+#include <sys/types.h>
+
+#include <boost/asio.hpp>
+#include <boost/iostreams/copy.hpp>
+#include <boost/iostreams/device/back_inserter.hpp>
+#include <boost/iostreams/filter/gzip.hpp>
+#include <boost/iostreams/filter/zlib.hpp>
+#include <boost/iostreams/filtering_stream.hpp>
+#include <boost/lexical_cast.hpp>
+#include <boost/locale/conversion.hpp>
+#include <boost/locale/encoding.hpp>
+#include <sstream>
+#include "RocketMQClient.h"
+
+using namespace std;
+namespace rocketmq {
+//<!************************************************************************
+const string WHITESPACE = " \t\r\n";
+const int MASTER_ID = 0;
+const string SUB_ALL = "*";
+const string DEFAULT_TOPIC = "TBW102";
+const string BENCHMARK_TOPIC = "BenchmarkTest";
+const string DEFAULT_PRODUCER_GROUP = "DEFAULT_PRODUCER";
+const string DEFAULT_CONSUMER_GROUP = "DEFAULT_CONSUMER";
+const string TOOLS_CONSUMER_GROUP = "TOOLS_CONSUMER";
+const string CLIENT_INNER_PRODUCER_GROUP = "CLIENT_INNER_PRODUCER";
+const string SELF_TEST_TOPIC = "SELF_TEST_TOPIC";
+const string RETRY_GROUP_TOPIC_PREFIX = "%RETRY%";
+const string DLQ_GROUP_TOPIC_PREFIX = "%DLQ%";
+const string ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME";
+const string ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir";
+const string MESSAGE_COMPRESS_LEVEL = "rocketmq.message.compressLevel";
+const int POLL_NAMESERVER_INTEVAL = 1000 * 30;
+const int HEARTBEAT_BROKER_INTERVAL = 1000 * 30;
+const int PERSIST_CONSUMER_OFFSET_INTERVAL = 1000 * 5;
+const string WS_ADDR = "please set nameserver domain by setDomainName, there is no default nameserver domain";
+
+const int LINE_SEPARATOR = 1;  // metaq::UtilAll::charToString((char) 1);
+const int WORD_SEPARATOR = 2;  // metaq::UtilAll::charToString((char) 2);
+
+const int HTTP_TIMEOUT = 3000;  // 3S
+const int HTTP_CONFLICT = 409;
+const int HTTP_OK = 200;
+const int HTTP_NOTFOUND = 404;
+const int CONNETERROR = -1;
+const string null = "";
+
+template <typename Type>
+inline void deleteAndZero(Type &pointer) {
+  delete pointer;
+  pointer = NULL;
+}
+#define EMPTY_STR_PTR(ptr) (ptr == NULL || ptr[0] == '\0')
+
+//<!************************************************************************
+class UtilAll {
+ public:
+  static bool startsWith_retry(const string &topic);
+  static string getRetryTopic(const string &consumerGroup);
+
+  static void Trim(string &str);
+  static bool isBlank(const string &str);
+  static uint64 hexstr2ull(const char *str);
+  static int64 str2ll(const char *str);
+  static string bytes2string(const char *bytes, int len);
+
+  template <typename T>
+  static string to_string(const T &n) {
+    std::ostringstream stm;
+    stm << n;
+    return stm.str();
+  }
+
+  static bool to_bool(std::string const &s) { return atoi(s.c_str()); }
+
+  static bool SplitURL(const string &serverURL, string &addr, short &nPort);
+  static int Split(vector<string> &ret_, const string &strIn, const char sep);
+  static int Split(vector<string> &ret_, const string &strIn,
+                   const string &sep);
+
+  static int32_t StringToInt32(const std::string &str, int32_t &out);
+  static int64_t StringToInt64(const std::string &str, int64_t &val);
+
+  static string getLocalHostName();
+  static string getLocalAddress();
+  static string getHomeDirectory();
+
+  static int getRandomNum(int baseNum);
+
+  static string getProcessName();
+
+  static uint64_t currentTimeMillis();
+  static uint64_t currentTimeSeconds();
+
+  static bool deflate(std::string &input, std::string &out, int level);
+  static bool inflate(std::string &input, std::string &out);
+
+ private:
+  static std::string s_localHostName;
+  static std::string s_localIpAddress;
+};
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/Validators.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/Validators.cpp b/rocketmq-cpp/src/common/Validators.cpp
new file mode 100755
index 0000000..7484c98
--- /dev/null
+++ b/rocketmq-cpp/src/common/Validators.cpp
@@ -0,0 +1,115 @@
+/*
+ * 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 "Validators.h"
+#include <stdio.h>
+#include <stdlib.h>
+namespace rocketmq {
+
+const string Validators::validPatternStr = "^[a-zA-Z0-9_-]+$";
+const int Validators::CHARACTER_MAX_LENGTH = 255;
+//<!***************************************************************************
+bool Validators::regularExpressionMatcher(const string& origin,
+                                          const string& patternStr) {
+  if (UtilAll::isBlank(origin)) {
+    return false;
+  }
+
+  if (UtilAll::isBlank(patternStr)) {
+    return true;
+  }
+
+  // Pattern pattern = Pattern.compile(patternStr);
+  // Matcher matcher = pattern.matcher(origin);
+
+  // return matcher.matches();
+  return true;
+}
+
+string Validators::getGroupWithRegularExpression(const string& origin,
+                                                 const string& patternStr) {
+  /*Pattern pattern = Pattern.compile(patternStr);
+  Matcher matcher = pattern.matcher(origin);
+  while (matcher.find()) {
+  return matcher.group(0);
+  }*/
+  return "";
+}
+
+void Validators::checkTopic(const string& topic) {
+  if (UtilAll::isBlank(topic)) {
+    THROW_MQEXCEPTION(MQClientException, "the specified topic is blank", -1);
+  }
+
+  if ((int)topic.length() > CHARACTER_MAX_LENGTH) {
+    THROW_MQEXCEPTION(
+        MQClientException,
+        "the specified topic is longer than topic max length 255.", -1);
+  }
+
+  if (topic == DEFAULT_TOPIC) {
+    THROW_MQEXCEPTION(
+        MQClientException,
+        "the topic[" + topic + "] is conflict with default topic.", -1);
+  }
+
+  if (!regularExpressionMatcher(topic, validPatternStr)) {
+    string str;
+    str = "the specified topic[" + topic +
+          "] contains illegal characters, allowing only" + validPatternStr;
+
+    THROW_MQEXCEPTION(MQClientException, str.c_str(), -1);
+  }
+}
+
+void Validators::checkGroup(const string& group) {
+  if (UtilAll::isBlank(group)) {
+    THROW_MQEXCEPTION(MQClientException, "the specified group is blank", -1);
+  }
+
+  if (!regularExpressionMatcher(group, validPatternStr)) {
+    string str;
+    str = "the specified group[" + group +
+          "] contains illegal characters, allowing only" + validPatternStr;
+
+    THROW_MQEXCEPTION(MQClientException, str.c_str(), -1);
+  }
+  if ((int)group.length() > CHARACTER_MAX_LENGTH) {
+    THROW_MQEXCEPTION(
+        MQClientException,
+        "the specified group is longer than group max length 255.", -1);
+  }
+}
+
+void Validators::checkMessage(const MQMessage& msg, int maxMessageSize) {
+  checkTopic(msg.getTopic());
+
+  string body = msg.getBody();
+  //<!body;
+  if (body.empty()) {
+    THROW_MQEXCEPTION(MQClientException, "the message body is empty", -1);
+  }
+
+  if ((int)body.length() > maxMessageSize) {
+    char info[256];
+    sprintf(info, "the message body size over max value, MAX: %d",
+            maxMessageSize);
+    THROW_MQEXCEPTION(MQClientException, info, -1);
+  }
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/Validators.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/Validators.h b/rocketmq-cpp/src/common/Validators.h
new file mode 100755
index 0000000..4190588
--- /dev/null
+++ b/rocketmq-cpp/src/common/Validators.h
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __VALIDATORST_H__
+#define __VALIDATORST_H__
+
+#include <string>
+#include "MQClientException.h"
+#include "MQMessage.h"
+#include "UtilAll.h"
+namespace rocketmq {
+//<!***************************************************************************
+class Validators {
+ public:
+  static bool regularExpressionMatcher(const string& origin,
+                                       const string& patternStr);
+  static string getGroupWithRegularExpression(const string& origin,
+                                              const string& patternStr);
+  static void checkTopic(const string& topic);
+  static void checkGroup(const string& group);
+  static void checkMessage(const MQMessage& msg, int maxMessageSize);
+
+ public:
+  static const string validPatternStr;
+  static const int CHARACTER_MAX_LENGTH;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/VirtualEnvUtil.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/VirtualEnvUtil.cpp b/rocketmq-cpp/src/common/VirtualEnvUtil.cpp
new file mode 100755
index 0000000..6daf731
--- /dev/null
+++ b/rocketmq-cpp/src/common/VirtualEnvUtil.cpp
@@ -0,0 +1,56 @@
+/*
+ * 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 "VirtualEnvUtil.h"
+#include <stdio.h>
+#include <stdlib.h>
+#include "UtilAll.h"
+
+namespace rocketmq {
+const char* VirtualEnvUtil::VIRTUAL_APPGROUP_PREFIX = "%%PROJECT_%s%%";
+
+//<!***************************************************************************
+string VirtualEnvUtil::buildWithProjectGroup(const string& origin,
+                                             const string& projectGroup) {
+  if (!UtilAll::isBlank(projectGroup)) {
+    char prefix[1024];
+    sprintf(prefix, VIRTUAL_APPGROUP_PREFIX, projectGroup.c_str());
+
+    if (origin.find_last_of(prefix) == string::npos) {
+      return origin + prefix;
+    } else {
+      return origin;
+    }
+  } else {
+    return origin;
+  }
+}
+
+string VirtualEnvUtil::clearProjectGroup(const string& origin,
+                                         const string& projectGroup) {
+  char prefix[1024];
+  sprintf(prefix, VIRTUAL_APPGROUP_PREFIX, projectGroup.c_str());
+  string::size_type pos = origin.find_last_of(prefix);
+
+  if (!UtilAll::isBlank(prefix) && pos != string::npos) {
+    return origin.substr(0, pos);
+  } else {
+    return origin;
+  }
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/VirtualEnvUtil.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/VirtualEnvUtil.h b/rocketmq-cpp/src/common/VirtualEnvUtil.h
new file mode 100755
index 0000000..88e44f8
--- /dev/null
+++ b/rocketmq-cpp/src/common/VirtualEnvUtil.h
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef __VIRTUALENVUTIL_H__
+#define __VIRTUALENVUTIL_H__
+
+#include <string>
+namespace rocketmq {
+//<!***************************************************************************
+class VirtualEnvUtil {
+ public:
+  static std::string buildWithProjectGroup(const std::string& origin,
+                                      const std::string& projectGroup);
+  static std::string clearProjectGroup(const std::string& origin,
+                                  const std::string& projectGroup);
+
+ public:
+  static const char* VIRTUAL_APPGROUP_PREFIX;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/dataBlock.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/dataBlock.cpp b/rocketmq-cpp/src/common/dataBlock.cpp
new file mode 100644
index 0000000..1916271
--- /dev/null
+++ b/rocketmq-cpp/src/common/dataBlock.cpp
@@ -0,0 +1,173 @@
+#include "dataBlock.h"
+#include <algorithm>
+
+namespace rocketmq {
+MemoryBlock::MemoryBlock() : size(0), data(NULL) {}
+
+MemoryBlock::MemoryBlock(const int initialSize, const bool initialiseToZero)
+    : size(0), data(NULL) {
+  if (initialSize > 0) {
+    size = initialSize;
+    data = static_cast<char*>(initialiseToZero
+                                  ? std::calloc(initialSize, sizeof(char))
+                                  : std::malloc(initialSize * sizeof(char)));
+  }
+}
+
+MemoryBlock::MemoryBlock(const void* const dataToInitialiseFrom,
+                         const size_t sizeInBytes)
+    : size(sizeInBytes), data(NULL) {
+  if (size > 0) {
+    data = static_cast<char*>(std::malloc(size * sizeof(char)));
+
+    if (dataToInitialiseFrom != NULL) memcpy(data, dataToInitialiseFrom, size);
+  }
+}
+
+MemoryBlock::MemoryBlock(const MemoryBlock& other)
+    : size(other.size), data(NULL) {
+  if (size > 0) {
+    data = static_cast<char*>(std::malloc(size * sizeof(char)));
+    memcpy(data, other.data, size);
+  }
+}
+
+MemoryBlock::~MemoryBlock() { std::free(data); }
+
+MemoryBlock& MemoryBlock::operator=(const MemoryBlock& other) {
+  if (this != &other) {
+    setSize(other.size, false);
+    memcpy(data, other.data, size);
+  }
+
+  return *this;
+}
+
+//==============================================================================
+bool MemoryBlock::operator==(const MemoryBlock& other) const {
+  return matches(other.data, other.size);
+}
+
+bool MemoryBlock::operator!=(const MemoryBlock& other) const {
+  return !operator==(other);
+}
+
+bool MemoryBlock::matches(const void* dataToCompare, int dataSize) const {
+  return size == dataSize && memcmp(data, dataToCompare, size) == 0;
+}
+
+//==============================================================================
+// this will resize the block to this size
+void MemoryBlock::setSize(const int newSize, const bool initialiseToZero) {
+  if (size != newSize) {
+    if (newSize <= 0) {
+      reset();
+    } else {
+      if (data != NULL) {
+        data = static_cast<char*>(
+            data == NULL ? std::malloc(newSize * sizeof(char))
+                         : std::realloc(data, newSize * sizeof(char)));
+
+        if (initialiseToZero && (newSize > size))
+          memset(data + size, 0, newSize - size);
+      } else {
+        std::free(data);
+        data = static_cast<char*>(initialiseToZero
+                                      ? std::calloc(newSize, sizeof(char))
+                                      : std::malloc(newSize * sizeof(char)));
+      }
+
+      size = newSize;
+    }
+  }
+}
+
+void MemoryBlock::reset() {
+  std::free(data);
+  data = NULL;
+  size = 0;
+}
+
+void MemoryBlock::ensureSize(const int minimumSize,
+                             const bool initialiseToZero) {
+  if (size < minimumSize) setSize(minimumSize, initialiseToZero);
+}
+
+//==============================================================================
+void MemoryBlock::fillWith(const int value) { memset(data, (int)value, size); }
+
+void MemoryBlock::append(const void* const srcData, const int numBytes) {
+  if (numBytes > 0) {
+    const int oldSize = size;
+    setSize(size + numBytes);
+    memcpy(data + oldSize, srcData, numBytes);
+  }
+}
+
+void MemoryBlock::replaceWith(const void* const srcData, const int numBytes) {
+  if (numBytes > 0) {
+    setSize(numBytes);
+    memcpy(data, srcData, numBytes);
+  }
+}
+
+void MemoryBlock::insert(const void* const srcData, const int numBytes,
+                         int insertPosition) {
+  if (numBytes > 0) {
+    insertPosition = std::min(insertPosition, size);
+    const int trailingDataSize = size - insertPosition;
+    setSize(size + numBytes, false);
+
+    if (trailingDataSize > 0)
+      memmove(data + insertPosition + numBytes, data + insertPosition,
+              trailingDataSize);
+
+    memcpy(data + insertPosition, srcData, numBytes);
+  }
+}
+
+void MemoryBlock::removeSection(const int startByte,
+                                const int numBytesToRemove) {
+  if (startByte + numBytesToRemove >= size) {
+    setSize(startByte);
+  } else if (numBytesToRemove > 0) {
+    memmove(data + startByte, data + startByte + numBytesToRemove,
+            size - (startByte + numBytesToRemove));
+
+    setSize(size - numBytesToRemove);
+  }
+}
+
+void MemoryBlock::copyFrom(const void* const src, int offset, int num) {
+  const char* d = static_cast<const char*>(src);
+
+  if (offset < 0) {
+    d -= offset;
+    num += (size_t)-offset;
+    offset = 0;
+  }
+
+  if ((size_t)offset + num > (unsigned int)size) num = size - (size_t)offset;
+
+  if (num > 0) memcpy(data + offset, d, num);
+}
+
+void MemoryBlock::copyTo(void* const dst, int offset, int num) const {
+  char* d = static_cast<char*>(dst);
+
+  if (offset < 0) {
+    memset(d, 0, (size_t)-offset);
+    d -= offset;
+    num -= (size_t)-offset;
+    offset = 0;
+  }
+
+  if ((size_t)offset + num > (unsigned int)size) {
+    const int newNum = (size_t)size - (size_t)offset;
+    memset(d + newNum, 0, num - newNum);
+    num = newNum;
+  }
+
+  if (num > 0) memcpy(d, data + offset, num);
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/dataBlock.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/dataBlock.h b/rocketmq-cpp/src/common/dataBlock.h
new file mode 100755
index 0000000..b86404a
--- /dev/null
+++ b/rocketmq-cpp/src/common/dataBlock.h
@@ -0,0 +1,185 @@
+#ifndef __DATABLOCK_H__
+#define __DATABLOCK_H__
+
+#include <cstdlib>
+#include <cstring>
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+
+class ROCKETMQCLIENT_API MemoryBlock {
+ public:
+  //==============================================================================
+  /** Create an uninitialised block with 0 size. */
+  MemoryBlock();
+
+  /** Creates a memory block with a given initial size.
+
+      @param initialSize          the size of block to create
+      @param initialiseToZero     whether to clear the memory or just leave it
+     uninitialised
+  */
+  MemoryBlock(const int initialSize, bool initialiseToZero = false);
+
+  /** Creates a memory block using a copy of a block of data.
+
+      @param dataToInitialiseFrom     some data to copy into this block
+      @param sizeInBytes              how much space to use
+  */
+  MemoryBlock(const void* dataToInitialiseFrom, size_t sizeInBytes);
+
+  /** Creates a copy of another memory block. */
+  MemoryBlock(const MemoryBlock&);
+
+  /** Destructor. */
+  ~MemoryBlock();
+
+  /** Copies another memory block onto this one.
+      This block will be resized and copied to exactly match the other one.
+  */
+  MemoryBlock& operator=(const MemoryBlock&);
+
+  //==============================================================================
+  /** Compares two memory blocks.
+      @returns true only if the two blocks are the same size and have identical
+     contents.
+  */
+  bool operator==(const MemoryBlock& other) const;
+
+  /** Compares two memory blocks.
+      @returns true if the two blocks are different sizes or have different
+     contents.
+  */
+  bool operator!=(const MemoryBlock& other) const;
+
+  //==============================================================================
+  /** Returns a void pointer to the data.
+
+      Note that the pointer returned will probably become invalid when the
+      block is resized.
+  */
+  void* getData() const { return data; }
+
+  /** Returns a byte from the memory block.
+      This returns a reference, so you can also use it to set a byte.
+  */
+  template <typename Type>
+  char& operator[](const Type offset) const {
+    return data[offset];
+  }
+
+  /** Returns true if the data in this MemoryBlock matches the raw bytes
+   * passed-in. */
+  bool matches(const void* data, int dataSize) const;
+
+  //==============================================================================
+  /** Returns the block's current allocated size, in bytes. */
+  int getSize() const { return size; }
+
+  /** Resizes the memory block.
+
+      Any data that is present in both the old and new sizes will be retained.
+      When enlarging the block, the new space that is allocated at the end can
+     either be
+      cleared, or left uninitialised.
+
+      @param newSize                      the new desired size for the block
+      @param initialiseNewSpaceToZero     if the block gets enlarged, this
+     determines
+                                          whether to clear the new section or
+     just leave it
+                                          uninitialised
+      @see ensureSize
+  */
+  void setSize(const int newSize, bool initialiseNewSpaceToZero = false);
+
+  /** Increases the block's size only if it's smaller than a given size.
+
+      @param minimumSize                  if the block is already bigger than
+     this size, no action
+                                          will be taken; otherwise it will be
+     increased to this size
+      @param initialiseNewSpaceToZero     if the block gets enlarged, this
+     determines
+                                          whether to clear the new section or
+     just leave it
+                                          uninitialised
+      @see setSize
+  */
+  void ensureSize(const int minimumSize, bool initialiseNewSpaceToZero = false);
+
+  /** Frees all the blocks data, setting its size to 0. */
+  void reset();
+
+  //==============================================================================
+  /** Fills the entire memory block with a repeated byte value.
+      This is handy for clearing a block of memory to zero.
+  */
+  void fillWith(int valueToUse);
+
+  /** Adds another block of data to the end of this one.
+      The data pointer must not be null. This block's size will be increased
+     accordingly.
+  */
+  void append(const void* data, int numBytes);
+
+  /** Resizes this block to the given size and fills its contents from the
+     supplied buffer.
+      The data pointer must not be null.
+  */
+  void replaceWith(const void* data, int numBytes);
+
+  /** Inserts some data into the block.
+      The dataToInsert pointer must not be null. This block's size will be
+     increased accordingly.
+      If the insert position lies outside the valid range of the block, it will
+     be clipped to
+      within the range before being used.
+  */
+  void insert(const void* dataToInsert, int numBytesToInsert,
+              int insertPosition);
+
+  /** Chops out a section  of the block.
+
+      This will remove a section of the memory block and close the gap around
+     it,
+      shifting any subsequent data downwards and reducing the size of the block.
+
+      If the range specified goes beyond the size of the block, it will be
+     clipped.
+  */
+  void removeSection(int startByte, int numBytesToRemove);
+
+  //==============================================================================
+  /** Copies data into this MemoryBlock from a memory address.
+
+      @param srcData              the memory location of the data to copy into
+     this block
+      @param destinationOffset    the offset in this block at which the data
+     being copied should begin
+      @param numBytes             how much to copy in (if this goes beyond the
+     size of the memory block,
+                                  it will be clipped so not to do anything
+     nasty)
+  */
+  void copyFrom(const void* srcData, int destinationOffset, int numBytes);
+
+  /** Copies data from this MemoryBlock to a memory address.
+
+      @param destData         the memory location to write to
+      @param sourceOffset     the offset within this block from which the copied
+     data will be read
+      @param numBytes         how much to copy (if this extends beyond the
+     limits of the memory block,
+                              zeros will be used for that portion of the data)
+  */
+  void copyTo(void* destData, int sourceOffset, int numBytes) const;
+
+ private:
+  //==============================================================================
+  int size;
+  char* data;
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/sync_http_client.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/sync_http_client.cpp b/rocketmq-cpp/src/common/sync_http_client.cpp
new file mode 100755
index 0000000..f07ac0d
--- /dev/null
+++ b/rocketmq-cpp/src/common/sync_http_client.cpp
@@ -0,0 +1,156 @@
+/*
+ * 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 <boost/asio.hpp>
+#include <boost/bind.hpp>
+#include <boost/lambda/bind.hpp>
+#include <boost/lambda/lambda.hpp>
+#include <iostream>
+#include <istream>
+#include <ostream>
+#include <string>
+
+#include "Logging.h"
+#include "url.h"
+
+using boost::lambda::var;
+using boost::asio::ip::tcp;
+using boost::asio::deadline_timer;
+
+namespace {
+void check_deadline(deadline_timer* deadline, tcp::socket* socket,
+                    const boost::system::error_code& ec) {
+  // Check whether the deadline has passed. We compare the deadline against
+  // the current time since a new asynchronous operation may have moved the
+  // deadline before this actor had a chance to run.
+  if (deadline->expires_at() <= deadline_timer::traits_type::now()) {
+    // The deadline has passed. The socket is closed so that any outstanding
+    // asynchronous operations are cancelled. This allows the blocked
+    // connect(), read_line() or write_line() functions to return.
+    boost::system::error_code ignored_ec;
+    socket->close(ignored_ec);
+
+    // There is no longer an active deadline. The expiry is set to positive
+    // infinity so that the actor takes no action until a new deadline is set.
+    deadline->expires_at(boost::posix_time::pos_infin);
+  }
+
+  // Put the actor back to sleep.
+  deadline->async_wait(boost::bind(&check_deadline, deadline, socket,
+                                   boost::asio::placeholders::error));
+}
+}  // namespace
+
+namespace rocketmq {
+bool SyncfetchNsAddr(const Url& url_s, std::string& body) {
+  bool ret = true;
+  try {
+    boost::asio::io_service io_service;
+    // Get a list of endpoints corresponding to the server name.
+    tcp::resolver resolver(io_service);
+    tcp::resolver::query query(url_s.host_, url_s.port_);
+    tcp::resolver::iterator endpoint_iterator = resolver.resolve(query);
+    boost::system::error_code ec = boost::asio::error::would_block;
+    deadline_timer deadline(io_service);
+    // TODO hardcode
+    boost::posix_time::seconds timeout(3);
+    deadline.expires_from_now(timeout);
+    // Try each endpoint until we successfully establish a connection.
+    tcp::socket socket(io_service);
+    boost::system::error_code deadline_ec;
+    check_deadline(&deadline, &socket, deadline_ec);
+
+    boost::asio::async_connect(socket, endpoint_iterator,
+                               boost::lambda::var(ec) = boost::lambda::_1);
+
+    do {
+      io_service.run_one();
+    } while (ec == boost::asio::error::would_block);
+
+    if (ec || !socket.is_open()) {
+      LOG_ERROR("socket connect failure, connect timeout or connect failure");
+      return false;
+    }
+
+    // Form the request. We specify the "Connection: close" header so that the
+    // server will close the socket after transmitting the response. This will
+    // allow us to treat all data up until the EOF as the content.
+    boost::asio::streambuf request;
+    std::ostream request_stream(&request);
+    request_stream << "GET " << url_s.path_ << " HTTP/1.0\r\n";
+    request_stream << "Host: " << url_s.host_ << "\r\n";
+    request_stream << "Accept: */*\r\n";
+    request_stream << "Connection: close\r\n\r\n";
+
+    // Send the request.
+    boost::asio::write(socket, request);
+
+    // Read the response status line. The response streambuf will automatically
+    // grow to accommodate the entire line. The growth may be limited by passing
+    // a maximum size to the streambuf constructor.
+    boost::asio::streambuf response;
+    boost::asio::read_until(socket, response, "\r\n");
+
+    // Check that response is OK.
+    std::istream response_stream(&response);
+    std::string http_version;
+    response_stream >> http_version;
+    unsigned int status_code;
+    response_stream >> status_code;
+    std::string status_message;
+    std::getline(response_stream, status_message);
+    if (!response_stream || http_version.substr(0, 5) != "HTTP/") {
+      LOG_INFO("Invalid response %s\n", status_message.c_str());
+      return false;
+    }
+
+    if (status_code != 200) {
+      LOG_INFO("Response returned with status code %d ", status_code);
+      return false;
+    }
+
+    // Read the response headers, which are terminated by a blank line.
+    boost::asio::read_until(socket, response, "\r\n\r\n");
+
+    // Process the response headers.
+    std::string header;
+    while (std::getline(response_stream, header) && header != "\r")
+      ;
+
+    // Write whatever content we already have to output.
+    if (response.size() > 0) {
+      boost::asio::streambuf::const_buffers_type cbt = response.data();
+      body.clear();
+      body.insert(body.begin(), boost::asio::buffers_begin(cbt),
+                  boost::asio::buffers_end(cbt));
+    }
+
+    // Read until EOF, writing data to output as we go.
+    boost::system::error_code error;
+    while (boost::asio::read(socket, response,
+                             boost::asio::transfer_at_least(1), error))
+      std::cout << &response;
+    if (error != boost::asio::error::eof)
+      throw boost::system::system_error(error);
+
+  } catch (std::exception& e) {
+    LOG_ERROR("Exception:  %s", e.what());
+    ret = false;
+  }
+
+  return ret;
+}
+}  // end of namespace ons

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/sync_http_client.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/sync_http_client.h b/rocketmq-cpp/src/common/sync_http_client.h
new file mode 100755
index 0000000..b25cc77
--- /dev/null
+++ b/rocketmq-cpp/src/common/sync_http_client.h
@@ -0,0 +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.
+ */
+#ifndef ROCKETMQ_CLIENT4CPP__SYNC_HTTP_CLIENT_H_
+#define ROCKETMQ_CLIENT4CPP_SYNC_HTTP_CLIENT_H_
+
+#include <string>
+
+namespace rocketmq {
+class Url;
+
+extern bool SyncfetchNsAddr(const Url& url_s, std::string& body);
+
+}  // namespace ons
+
+#endif  // ROCKETMQ_CLIENT4CPP__SYNC_HTTP_CLIENT_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/url.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/url.cpp b/rocketmq-cpp/src/common/url.cpp
new file mode 100755
index 0000000..348f27a
--- /dev/null
+++ b/rocketmq-cpp/src/common/url.cpp
@@ -0,0 +1,63 @@
+/*
+ * 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 "url.h"
+#include <algorithm>
+#include <cctype>
+#include <functional>
+#include <iterator>
+#include <string>
+
+namespace rocketmq {
+
+Url::Url(const std::string& url_s) { parse(url_s); }
+
+void Url::parse(const std::string& url_s) {
+  const std::string prot_end("://");
+  auto prot_i =
+      std::search(url_s.begin(), url_s.end(), prot_end.begin(), prot_end.end());
+  protocol_.reserve(std::distance(url_s.begin(), prot_i));
+  std::transform(url_s.begin(), prot_i, std::back_inserter(protocol_),
+                 std::ptr_fun<int, int>(tolower));  // protocol is icase
+
+  if (prot_i == url_s.end()) return;
+
+  std::advance(prot_i, prot_end.length());
+
+  auto path_i = find(prot_i, url_s.end(), ':');
+  std::string::const_iterator path_end_i;
+  if (path_i == url_s.end()) {
+    // not include port, use default port
+    port_ = "80";
+    path_i = std::find(prot_i, url_s.end(), '/');
+    path_end_i = path_i;
+  } else {
+    auto port_i = find(path_i + 1, url_s.end(), '/');
+    port_.insert(port_.begin(), path_i + 1, port_i);
+    path_end_i = path_i + port_.length() + 1;
+  }
+
+  host_.reserve(distance(prot_i, path_i));
+  std::transform(prot_i, path_i, std::back_inserter(host_),
+                 std::ptr_fun<int, int>(tolower));  // host is icase}
+
+  auto query_i = find(path_end_i, url_s.end(), '?');
+  path_.assign(path_end_i, query_i);
+  if (query_i != url_s.end()) ++query_i;
+  query_.assign(query_i, url_s.end());
+}
+
+}  // namespace ons

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/common/url.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/common/url.h b/rocketmq-cpp/src/common/url.h
new file mode 100755
index 0000000..0fbcb85
--- /dev/null
+++ b/rocketmq-cpp/src/common/url.h
@@ -0,0 +1,38 @@
+/*
+ * 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_URL_HH_
+#define ROCKETMQ_CLIENT4CPP_URL_HH_
+
+#include <string>
+
+namespace rocketmq {
+class Url {
+ public:
+  Url(const std::string& url_s);  // omitted copy, ==, accessors, ...
+
+ private:
+  void parse(const std::string& url_s);
+
+ public:
+  std::string protocol_;
+  std::string host_;
+  std::string port_;
+  std::string path_;
+  std::string query_;
+};
+}
+#endif  // ROCKETMQ_CLIENT4CPP_URL_HH_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/AllocateMQStrategy.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/AllocateMQStrategy.h b/rocketmq-cpp/src/consumer/AllocateMQStrategy.h
new file mode 100755
index 0000000..f77cff7
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/AllocateMQStrategy.h
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __ALLOCATEMESSAGEQUEUESTRATEGY_H__
+#define __ALLOCATEMESSAGEQUEUESTRATEGY_H__
+
+#include "Logging.h"
+#include "MQClientException.h"
+#include "MQMessageQueue.h"
+#include "RocketMQClient.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+class AllocateMQStrategy {
+ public:
+  virtual ~AllocateMQStrategy() {}
+  virtual void allocate(const std::string& currentCID, std::vector<MQMessageQueue>& mqAll,
+                        std::vector<std::string>& cidAll,
+                        std::vector<MQMessageQueue>& outReuslt) = 0;
+};
+
+//<!************************************************************************
+class AllocateMQAveragely : public AllocateMQStrategy {
+ public:
+  virtual ~AllocateMQAveragely() {}
+  virtual void allocate(const std::string& currentCID, std::vector<MQMessageQueue>& mqAll,
+                        std::vector<std::string>& cidAll,
+                        std::vector<MQMessageQueue>& outReuslt) {
+    outReuslt.clear();
+    if (currentCID.empty()) {
+      THROW_MQEXCEPTION(MQClientException, "currentCID is empty", -1);
+    }
+
+    if (mqAll.empty()) {
+      THROW_MQEXCEPTION(MQClientException, "mqAll is empty", -1);
+    }
+
+    if (cidAll.empty()) {
+      THROW_MQEXCEPTION(MQClientException, "cidAll is empty", -1);
+    }
+
+    int index = -1;
+    int cidAllSize = cidAll.size();
+    for (int i = 0; i < cidAllSize; i++) {
+      if (cidAll[i] == currentCID) {
+        index = i;
+        break;
+      }
+    }
+
+    if (index == -1) {
+      LOG_ERROR("could not find clientId from Broker");
+      return;
+    }
+
+    int mqAllSize = mqAll.size();
+    int mod = mqAllSize % cidAllSize;
+    int averageSize = mqAllSize <= cidAllSize
+                          ? 1
+                          : (mod > 0 && index < mod ? mqAllSize / cidAllSize + 1
+                                                    : mqAllSize / cidAllSize);
+    int startIndex = (mod > 0 && index < mod) ? index * averageSize
+                                              : index * averageSize + mod;
+    int range = std::min(averageSize, mqAllSize - startIndex);
+    LOG_INFO(
+        "range is:%d, index is:%d, mqAllSize is:%d, averageSize is:%d, "
+        "startIndex is:%d",
+        range, index, mqAllSize, averageSize, startIndex);
+    //<!out;
+    if (range >= 0)  // example: range is:-1, index is:1, mqAllSize is:1,
+                     // averageSize is:1, startIndex is:2
+    {
+      for (int i = 0; i < range; i++) {
+        if ((startIndex + i) >= 0) {
+          outReuslt.push_back(mqAll.at((startIndex + i) % mqAllSize));
+        }
+      }
+    }
+  }
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/ConsumeMessageConcurrentlyService.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/ConsumeMessageConcurrentlyService.cpp b/rocketmq-cpp/src/consumer/ConsumeMessageConcurrentlyService.cpp
new file mode 100755
index 0000000..0a457c7
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/ConsumeMessageConcurrentlyService.cpp
@@ -0,0 +1,147 @@
+/*
+ * 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 <sys/prctl.h>
+#include "ConsumeMsgService.h"
+#include "DefaultMQPushConsumer.h"
+#include "Logging.h"
+#include "UtilAll.h"
+namespace rocketmq {
+
+//<!************************************************************************
+ConsumeMessageConcurrentlyService::ConsumeMessageConcurrentlyService(
+    MQConsumer* consumer, int threadCount, MQMessageListener* msgListener)
+    : m_pConsumer(consumer),
+      m_pMessageListener(msgListener),
+      m_ioServiceWork(m_ioService) {
+  string taskName = UtilAll::getProcessName();
+  prctl(PR_SET_NAME, "ConsumeTP", 0, 0, 0);
+  for (int i = 0; i != threadCount; ++i) {
+    m_threadpool.create_thread(
+        boost::bind(&boost::asio::io_service::run, &m_ioService));
+  }
+  prctl(PR_SET_NAME, taskName.c_str(), 0, 0, 0);
+}
+
+ConsumeMessageConcurrentlyService::~ConsumeMessageConcurrentlyService(void) {
+  m_pConsumer = NULL;
+  m_pMessageListener = NULL;
+}
+
+void ConsumeMessageConcurrentlyService::start() {}
+
+void ConsumeMessageConcurrentlyService::shutdown() { stopThreadPool(); }
+
+void ConsumeMessageConcurrentlyService::stopThreadPool() {
+  m_ioService.stop();
+  m_threadpool.join_all();
+}
+
+MessageListenerType
+ConsumeMessageConcurrentlyService::getConsumeMsgSerivceListenerType() {
+  return m_pMessageListener->getMessageListenerType();
+}
+
+void ConsumeMessageConcurrentlyService::submitConsumeRequest(
+    PullRequest* request, vector<MQMessageExt>& msgs) {
+  m_ioService.post(boost::bind(
+      &ConsumeMessageConcurrentlyService::ConsumeRequest, this, request, msgs));
+}
+
+void ConsumeMessageConcurrentlyService::ConsumeRequest(
+    PullRequest* request, vector<MQMessageExt>& msgs) {
+  if (!request || request->isDroped()) {
+    LOG_WARN("the pull result is NULL or Had been dropped");
+    request->clearAllMsgs();  // add clear operation to avoid bad state when
+                              // dropped pullRequest returns normal
+    return;
+  }
+
+  //<!��ȡ����;
+  if (msgs.empty()) {
+    LOG_WARN("the msg of pull result is NULL,its mq:%s",
+             (request->m_messageQueue).toString().c_str());
+    return;
+  }
+
+  ConsumeStatus status = CONSUME_SUCCESS;
+  if (m_pMessageListener != NULL) {
+    resetRetryTopic(msgs);
+    request->setLastConsumeTimestamp(UtilAll::currentTimeMillis());
+    status = m_pMessageListener->consumeMessage(msgs);
+  }
+
+  /*LOG_DEBUG("Consumed MSG size:%d of mq:%s",
+      msgs.size(), (request->m_messageQueue).toString().c_str());*/
+  int ackIndex = -1;
+  switch (status) {
+    case CONSUME_SUCCESS:
+      ackIndex = msgs.size();
+      break;
+    case RECONSUME_LATER:
+      ackIndex = -1;
+      break;
+    default:
+      break;
+  }
+
+  switch (m_pConsumer->getMessageModel()) {
+    case BROADCASTING:
+      //Note: broadcasting reconsume should do by application, as it has big affect to broker cluster
+      LOG_WARN("BROADCASTING, the message consume failed, drop it:%s", (request->m_messageQueue).toString().c_str());
+      break;
+    case CLUSTERING:
+      //send back msg to broker;
+      for (size_t i = ackIndex + 1; i < msgs.size(); i++) {
+        LOG_WARN(
+            "consume fail, MQ is:%s, its msgId is:%s, index is:%zu, reconsume "
+            "times is:%d",
+            (request->m_messageQueue).toString().c_str(),
+            msgs[i].getMsgId().c_str(), i, msgs[i].getReconsumeTimes());
+        m_pConsumer->sendMessageBack(msgs[i], 0);
+      }
+      break;
+    default:
+      break;
+  }
+
+  //update offset
+  int64 offset = request->removeMessage(msgs);
+  // LOG_DEBUG("update offset:%lld of mq: %s",
+  // offset,(request->m_messageQueue).toString().c_str());
+  if (offset >= 0) {
+    m_pConsumer->updateConsumeOffset(request->m_messageQueue, offset);
+  } else {
+    LOG_WARN("Note: accumulation consume occurs on mq:%s",
+             (request->m_messageQueue).toString().c_str());
+  }
+}
+
+void ConsumeMessageConcurrentlyService::resetRetryTopic(
+    vector<MQMessageExt>& msgs) {
+  string groupTopic = UtilAll::getRetryTopic(m_pConsumer->getGroupName());
+  for (size_t i = 0; i < msgs.size(); i++) {
+    MQMessageExt& msg = msgs[i];
+    string retryTopic = msg.getProperty(MQMessage::PROPERTY_RETRY_TOPIC);
+    if (!retryTopic.empty() && groupTopic.compare(msg.getTopic()) == 0) {
+      msg.setTopic(retryTopic);
+    }
+  }
+}
+
+//<!***************************************************************************
+}  //<!end namespace;

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/ConsumeMessageOrderlyService.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/ConsumeMessageOrderlyService.cpp b/rocketmq-cpp/src/consumer/ConsumeMessageOrderlyService.cpp
new file mode 100755
index 0000000..a336b72
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/ConsumeMessageOrderlyService.cpp
@@ -0,0 +1,214 @@
+/*
+ * 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 <sys/prctl.h>
+#include "ConsumeMsgService.h"
+#include "DefaultMQPushConsumer.h"
+#include "Logging.h"
+#include "Rebalance.h"
+#include "UtilAll.h"
+
+namespace rocketmq {
+
+//<!***************************************************************************
+ConsumeMessageOrderlyService::ConsumeMessageOrderlyService(
+    MQConsumer* consumer, int threadCount, MQMessageListener* msgListener)
+    : m_pConsumer(consumer),
+      m_shutdownInprogress(false),
+      m_pMessageListener(msgListener),
+      m_MaxTimeConsumeContinuously(60 * 1000),
+      m_ioServiceWork(m_ioService),
+      m_async_service_thread(NULL) {
+  string taskName = UtilAll::getProcessName();
+  prctl(PR_SET_NAME, "oderlyConsumeTP", 0, 0, 0);
+  for (int i = 0; i != threadCount; ++i) {
+    m_threadpool.create_thread(
+        boost::bind(&boost::asio::io_service::run, &m_ioService));
+  }
+  prctl(PR_SET_NAME, taskName.c_str(), 0, 0, 0);
+
+}
+
+void ConsumeMessageOrderlyService::boost_asio_work() {
+  LOG_INFO("ConsumeMessageOrderlyService::boost asio async service runing");
+  boost::asio::io_service::work work(m_async_ioService);  // avoid async io
+                                                          // service stops after
+                                                          // first timer timeout
+                                                          // callback
+  boost::system::error_code ec;
+  boost::asio::deadline_timer t(m_async_ioService,
+                                boost::posix_time::milliseconds(PullRequest::RebalanceLockInterval));
+  t.async_wait(
+      boost::bind(&ConsumeMessageOrderlyService::lockMQPeriodically, this, ec, &t));
+  
+  m_async_ioService.run();
+}
+
+ConsumeMessageOrderlyService::~ConsumeMessageOrderlyService(void) {
+  m_pConsumer = NULL;
+  m_pMessageListener = NULL;
+}
+
+void ConsumeMessageOrderlyService::start() {
+    m_async_service_thread.reset(new boost::thread(
+      boost::bind(&ConsumeMessageOrderlyService::boost_asio_work, this)));
+}
+
+void ConsumeMessageOrderlyService::shutdown() {
+  stopThreadPool();
+  unlockAllMQ();
+}
+
+void ConsumeMessageOrderlyService::lockMQPeriodically(
+    boost::system::error_code& ec, boost::asio::deadline_timer* t) {
+  m_pConsumer->getRebalance()->lockAll();
+
+  boost::system::error_code e;
+  t->expires_at(t->expires_at() + boost::posix_time::milliseconds(PullRequest::RebalanceLockInterval), e);
+  t->async_wait(boost::bind(&ConsumeMessageOrderlyService::lockMQPeriodically,
+                             this, ec, t));
+}
+
+void ConsumeMessageOrderlyService::unlockAllMQ() {
+  m_pConsumer->getRebalance()->unlockAll(false);
+}
+
+bool ConsumeMessageOrderlyService::lockOneMQ(const MQMessageQueue& mq) {
+  return m_pConsumer->getRebalance()->lock(mq);
+}
+
+void ConsumeMessageOrderlyService::stopThreadPool() {
+  m_shutdownInprogress = true;
+  m_ioService.stop();
+  m_async_ioService.stop();
+  m_async_service_thread->interrupt();
+  m_async_service_thread->join();
+  m_threadpool.join_all();
+}
+
+MessageListenerType
+ConsumeMessageOrderlyService::getConsumeMsgSerivceListenerType() {
+  return m_pMessageListener->getMessageListenerType();
+}
+
+void ConsumeMessageOrderlyService::submitConsumeRequest(
+    PullRequest* request, vector<MQMessageExt>& msgs) {
+  m_ioService.post(boost::bind(&ConsumeMessageOrderlyService::ConsumeRequest,
+                               this, request));
+}
+
+void ConsumeMessageOrderlyService::static_submitConsumeRequestLater(
+    void* context, PullRequest* request, bool tryLockMQ,
+    boost::asio::deadline_timer* t) {
+  LOG_INFO("submit consumeRequest later for mq:%s",
+           request->m_messageQueue.toString().c_str());
+  vector<MQMessageExt> msgs;
+  ConsumeMessageOrderlyService* orderlyService =
+      (ConsumeMessageOrderlyService*)context;
+  orderlyService->submitConsumeRequest(request, msgs);
+  if (tryLockMQ) {
+    orderlyService->lockOneMQ(request->m_messageQueue);
+  }
+  if (t) deleteAndZero(t);
+}
+
+void ConsumeMessageOrderlyService::ConsumeRequest(PullRequest* request) {
+  bool bGetMutex = false;
+  boost::unique_lock<boost::timed_mutex> lock(
+      request->getPullRequestCriticalSection(), boost::try_to_lock);
+  if (!lock.owns_lock()) {
+    if (!lock.timed_lock(boost::get_system_time() +
+                         boost::posix_time::seconds(1))) {
+      LOG_ERROR("ConsumeRequest of:%s get timed_mutex timeout",
+                request->m_messageQueue.toString().c_str());
+      return;
+    } else {
+      bGetMutex = true;
+    }
+  } else {
+    bGetMutex = true;
+  }
+  if (!bGetMutex) {
+    // LOG_INFO("pullrequest of mq:%s consume inprogress",
+    // request->m_messageQueue.toString().c_str());
+    return;
+  }
+  if (!request || request->isDroped()) {
+    LOG_WARN("the pull result is NULL or Had been dropped");
+    request->clearAllMsgs();  // add clear operation to avoid bad state when
+                              // dropped pullRequest returns normal
+    return;
+  }
+
+  if (m_pMessageListener) {
+    if ((request->isLocked() && !request->isLockExpired()) ||
+        m_pConsumer->getMessageModel() == BROADCASTING) {
+      DefaultMQPushConsumer* pConsumer = (DefaultMQPushConsumer*)m_pConsumer;
+      uint64_t beginTime = UtilAll::currentTimeMillis();
+      bool continueConsume = true;
+      while (continueConsume) {
+        if ((UtilAll::currentTimeMillis() - beginTime) >
+            m_MaxTimeConsumeContinuously) {
+          LOG_INFO(
+              "continuely consume message queue:%s more than 60s, consume it "
+              "later",
+              request->m_messageQueue.toString().c_str());
+          tryLockLaterAndReconsume(request, false);
+          break;
+        }
+        vector<MQMessageExt> msgs;
+        request->takeMessages(msgs, pConsumer->getConsumeMessageBatchMaxSize());
+        if (!msgs.empty()) {
+          request->setLastConsumeTimestamp(UtilAll::currentTimeMillis());
+          ConsumeStatus consumeStatus =
+              m_pMessageListener->consumeMessage(msgs);
+          if (consumeStatus == RECONSUME_LATER) {
+            request->makeMessageToCosumeAgain(msgs);
+            continueConsume = false;
+            tryLockLaterAndReconsume(request, false);
+          } else {
+            m_pConsumer->updateConsumeOffset(request->m_messageQueue,
+                                             request->commit());
+          }
+        } else {
+          continueConsume = false;
+        }
+        msgs.clear();
+        if (m_shutdownInprogress) {
+          LOG_INFO("shutdown inprogress, break the consuming");
+          return;
+        }
+      }
+      LOG_DEBUG("consume once exit of mq:%s",
+                request->m_messageQueue.toString().c_str());
+    } else {
+      LOG_ERROR("message queue:%s was not locked",
+                request->m_messageQueue.toString().c_str());
+      tryLockLaterAndReconsume(request, true);
+    }
+  }
+}
+void ConsumeMessageOrderlyService::tryLockLaterAndReconsume(
+    PullRequest* request, bool tryLockMQ) {
+  int retryTimer = tryLockMQ ? 500 : 100;
+  boost::asio::deadline_timer* t = new boost::asio::deadline_timer(
+      m_async_ioService, boost::posix_time::milliseconds(retryTimer));
+  t->async_wait(boost::bind(
+      &(ConsumeMessageOrderlyService::static_submitConsumeRequestLater), this,
+      request, tryLockMQ, t));
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/ConsumeMsgService.h
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/ConsumeMsgService.h b/rocketmq-cpp/src/consumer/ConsumeMsgService.h
new file mode 100755
index 0000000..5878cb4
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/ConsumeMsgService.h
@@ -0,0 +1,112 @@
+/*
+ * 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 _CONSUMEMESSAGESERVICE_H_
+#define _CONSUMEMESSAGESERVICE_H_
+
+#include <boost/asio.hpp>
+#include <boost/asio/io_service.hpp>
+#include <boost/bind.hpp>
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/scoped_ptr.hpp>
+#include <boost/thread/thread.hpp>
+#include "Logging.h"
+#include "MQMessageListener.h"
+#include "PullRequest.h"
+
+namespace rocketmq {
+class MQConsumer;
+//<!***************************************************************************
+class ConsumeMsgService {
+ public:
+  ConsumeMsgService() {}
+  virtual ~ConsumeMsgService() {}
+  virtual void start() {}
+  virtual void shutdown() {}
+  virtual void stopThreadPool() {}
+  virtual void submitConsumeRequest(PullRequest* request,
+                                    vector<MQMessageExt>& msgs) {}
+  virtual MessageListenerType getConsumeMsgSerivceListenerType() {
+    return messageListenerDefaultly;
+  }
+};
+
+class ConsumeMessageConcurrentlyService : public ConsumeMsgService {
+ public:
+  ConsumeMessageConcurrentlyService(MQConsumer*, int threadCount,
+                                    MQMessageListener* msgListener);
+  virtual ~ConsumeMessageConcurrentlyService();
+  virtual void start();
+  virtual void shutdown();
+  virtual void submitConsumeRequest(PullRequest* request,
+                                    vector<MQMessageExt>& msgs);
+  virtual MessageListenerType getConsumeMsgSerivceListenerType();
+  virtual void stopThreadPool();
+
+  void ConsumeRequest(PullRequest* request, vector<MQMessageExt>& msgs);
+
+ private:
+  void resetRetryTopic(vector<MQMessageExt>& msgs);
+
+ private:
+  MQConsumer* m_pConsumer;
+  MQMessageListener* m_pMessageListener;
+  boost::asio::io_service m_ioService;
+  boost::thread_group m_threadpool;
+  boost::asio::io_service::work m_ioServiceWork;
+};
+
+class ConsumeMessageOrderlyService : public ConsumeMsgService {
+ public:
+  ConsumeMessageOrderlyService(MQConsumer*, int threadCount,
+                               MQMessageListener* msgListener);
+  virtual ~ConsumeMessageOrderlyService();
+  virtual void start();
+  virtual void shutdown();
+  virtual void submitConsumeRequest(PullRequest* request,
+                                    vector<MQMessageExt>& msgs);
+  virtual void stopThreadPool();
+  virtual MessageListenerType getConsumeMsgSerivceListenerType();
+
+  void boost_asio_work();
+  void tryLockLaterAndReconsume(PullRequest* request, bool tryLockMQ);
+  static void static_submitConsumeRequestLater(void* context,
+                                               PullRequest* request,
+                                               bool tryLockMQ,
+                                               boost::asio::deadline_timer* t);
+  void ConsumeRequest(PullRequest* request);
+  void lockMQPeriodically(boost::system::error_code& ec,
+                          boost::asio::deadline_timer* t);
+  void unlockAllMQ();
+  bool lockOneMQ(const MQMessageQueue& mq);
+
+ private:
+  MQConsumer* m_pConsumer;
+  bool m_shutdownInprogress;
+  MQMessageListener* m_pMessageListener;
+  uint64_t m_MaxTimeConsumeContinuously;
+  boost::asio::io_service m_ioService;
+  boost::thread_group m_threadpool;
+  boost::asio::io_service::work m_ioServiceWork;
+  boost::asio::io_service m_async_ioService;
+  boost::scoped_ptr<boost::thread> m_async_service_thread;
+};
+
+//<!***************************************************************************
+}  //<!end namespace;
+
+#endif  //<! _CONSUMEMESSAGESERVICE_H_

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/src/consumer/DefaultMQPullConsumer.cpp
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/src/consumer/DefaultMQPullConsumer.cpp b/rocketmq-cpp/src/consumer/DefaultMQPullConsumer.cpp
new file mode 100755
index 0000000..9ee2ac9
--- /dev/null
+++ b/rocketmq-cpp/src/consumer/DefaultMQPullConsumer.cpp
@@ -0,0 +1,371 @@
+/*
+ * 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 "DefaultMQPullConsumer.h"
+#include "AsyncArg.h"
+#include "CommunicationMode.h"
+#include "FilterAPI.h"
+#include "Logging.h"
+#include "MQClientAPIImpl.h"
+#include "MQClientFactory.h"
+#include "MQClientManager.h"
+#include "MQProtos.h"
+#include "OffsetStore.h"
+#include "PullAPIWrapper.h"
+#include "PullSysFlag.h"
+#include "Rebalance.h"
+#include "Validators.h"
+
+namespace rocketmq {
+//<!***************************************************************************
+DefaultMQPullConsumer::DefaultMQPullConsumer(const string& groupname)
+    : m_pMessageQueueListener(NULL),
+      m_pOffsetStore(NULL),
+      m_pRebalance(NULL),
+      m_pPullAPIWrapper(NULL)
+
+{
+  //<!set default group name;
+  string gname = groupname.empty() ? DEFAULT_CONSUMER_GROUP : groupname;
+  setGroupName(gname);
+
+  setMessageModel(BROADCASTING);
+}
+
+DefaultMQPullConsumer::~DefaultMQPullConsumer() {
+  m_pMessageQueueListener = NULL;
+  deleteAndZero(m_pRebalance);
+  deleteAndZero(m_pOffsetStore);
+  deleteAndZero(m_pPullAPIWrapper);
+}
+
+// MQConsumer
+//<!************************************************************************
+void DefaultMQPullConsumer::start() {
+  /* Ignore the SIGPIPE */
+  struct sigaction sa;
+  sa.sa_handler = SIG_IGN;
+  sa.sa_flags = 0;
+  sigaction(SIGPIPE, &sa, 0);
+
+  switch (m_serviceState) {
+    case CREATE_JUST: {
+      m_serviceState = START_FAILED;
+      MQClient::start();
+      LOG_INFO("DefaultMQPullConsumer:%s start", m_GroupName.c_str());
+
+      //<!create rebalance;
+      m_pRebalance = new RebalancePull(this, getFactory());
+
+      string groupname = getGroupName();
+      m_pPullAPIWrapper = new PullAPIWrapper(getFactory(), groupname);
+
+      //<!data;
+      checkConfig();
+      copySubscription();
+
+      //<! registe;
+      bool registerOK = getFactory()->registerConsumer(this);
+      if (!registerOK) {
+        m_serviceState = CREATE_JUST;
+        THROW_MQEXCEPTION(
+            MQClientException,
+            "The cousumer group[" + getGroupName() +
+                "] has been created before, specify another name please.",
+            -1);
+      }
+
+      //<!msg model;
+      switch (getMessageModel()) {
+        case BROADCASTING:
+          m_pOffsetStore = new LocalFileOffsetStore(groupname, getFactory());
+          break;
+        case CLUSTERING:
+          m_pOffsetStore = new RemoteBrokerOffsetStore(groupname, getFactory());
+          break;
+      }
+      m_pOffsetStore->load();
+
+      getFactory()->start();
+      m_serviceState = RUNNING;
+      break;
+    }
+    case RUNNING:
+    case START_FAILED:
+    case SHUTDOWN_ALREADY:
+      break;
+    default:
+      break;
+  }
+}
+
+void DefaultMQPullConsumer::shutdown() {
+  switch (m_serviceState) {
+    case RUNNING: {
+      LOG_INFO("DefaultMQPullConsumer:%s shutdown", m_GroupName.c_str());
+      persistConsumerOffset();
+      getFactory()->unregisterConsumer(this);
+      getFactory()->shutdown();
+      m_serviceState = SHUTDOWN_ALREADY;
+      break;
+    }
+    case SHUTDOWN_ALREADY:
+    case CREATE_JUST:
+      break;
+    default:
+      break;
+  }
+}
+
+void DefaultMQPullConsumer::sendMessageBack(MQMessageExt& msg, int delayLevel) {
+
+}
+
+void DefaultMQPullConsumer::fetchSubscribeMessageQueues(
+    const string& topic, vector<MQMessageQueue>& mqs) {
+  mqs.clear();
+  try {
+    getFactory()->fetchSubscribeMessageQueues(topic, mqs,
+                                              getSessionCredentials());
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+void DefaultMQPullConsumer::updateTopicSubscribeInfo(
+    const string& topic, vector<MQMessageQueue>& info) {}
+
+void DefaultMQPullConsumer::registerMessageQueueListener(
+    const string& topic, MQueueListener* pListener) {
+  m_registerTopics.insert(topic);
+  if (pListener) {
+    m_pMessageQueueListener = pListener;
+  }
+}
+
+PullResult DefaultMQPullConsumer::pull(const MQMessageQueue& mq,
+                                       const string& subExpression,
+                                       int64 offset, int maxNums) {
+  return pullSyncImpl(mq, subExpression, offset, maxNums, false);
+}
+
+void DefaultMQPullConsumer::pull(const MQMessageQueue& mq,
+                                 const string& subExpression, int64 offset,
+                                 int maxNums, PullCallback* pPullCallback) {
+  pullAsyncImpl(mq, subExpression, offset, maxNums, false, pPullCallback);
+}
+
+PullResult DefaultMQPullConsumer::pullBlockIfNotFound(
+    const MQMessageQueue& mq, const string& subExpression, int64 offset,
+    int maxNums) {
+  return pullSyncImpl(mq, subExpression, offset, maxNums, true);
+}
+
+void DefaultMQPullConsumer::pullBlockIfNotFound(const MQMessageQueue& mq,
+                                                const string& subExpression,
+                                                int64 offset, int maxNums,
+                                                PullCallback* pPullCallback) {
+  pullAsyncImpl(mq, subExpression, offset, maxNums, true, pPullCallback);
+}
+
+PullResult DefaultMQPullConsumer::pullSyncImpl(const MQMessageQueue& mq,
+                                               const string& subExpression,
+                                               int64 offset, int maxNums,
+                                               bool block) {
+  if (offset < 0) THROW_MQEXCEPTION(MQClientException, "offset < 0", -1);
+
+  if (maxNums <= 0) THROW_MQEXCEPTION(MQClientException, "maxNums <= 0", -1);
+
+  //<!auto subscript,all sub;
+  subscriptionAutomatically(mq.getTopic());
+
+  int sysFlag = PullSysFlag::buildSysFlag(false, block, true, false);
+
+  //<!this sub;
+  unique_ptr<SubscriptionData> pSData(
+      FilterAPI::buildSubscriptionData(mq.getTopic(), subExpression));
+
+  int timeoutMillis = block ? 1000 * 30 : 1000 * 10;
+
+  try {
+    unique_ptr<PullResult> pullResult(
+        m_pPullAPIWrapper->pullKernelImpl(mq,                      // 1
+                                          pSData->getSubString(),  // 2
+                                          0L,                      // 3
+                                          offset,                  // 4
+                                          maxNums,                 // 5
+                                          sysFlag,                 // 6
+                                          0,                       // 7
+                                          1000 * 20,               // 8
+                                          timeoutMillis,           // 9
+                                          ComMode_SYNC,            // 10
+                                          NULL,                    //<!callback;
+                                          getSessionCredentials(), NULL));
+    return m_pPullAPIWrapper->processPullResult(mq, pullResult.get(),
+                                                pSData.get());
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+  return PullResult(BROKER_TIMEOUT);
+}
+
+void DefaultMQPullConsumer::pullAsyncImpl(const MQMessageQueue& mq,
+                                          const string& subExpression,
+                                          int64 offset, int maxNums, bool block,
+                                          PullCallback* pPullCallback) {
+  if (offset < 0) THROW_MQEXCEPTION(MQClientException, "offset < 0", -1);
+
+  if (maxNums <= 0) THROW_MQEXCEPTION(MQClientException, "maxNums <= 0", -1);
+
+  if (!pPullCallback)
+    THROW_MQEXCEPTION(MQClientException, "pPullCallback is null", -1);
+
+  //<!auto subscript,all sub;
+  subscriptionAutomatically(mq.getTopic());
+
+  int sysFlag = PullSysFlag::buildSysFlag(false, block, true, false);
+
+  //<!this sub;
+  unique_ptr<SubscriptionData> pSData(
+      FilterAPI::buildSubscriptionData(mq.getTopic(), subExpression));
+
+  int timeoutMillis = block ? 1000 * 30 : 1000 * 10;
+
+  //<!�첽����;
+  AsyncArg arg;
+  arg.mq = mq;
+  arg.subData = *pSData;
+  arg.pPullWrapper = m_pPullAPIWrapper;
+
+  try {
+    unique_ptr<PullResult> pullResult(m_pPullAPIWrapper->pullKernelImpl(
+        mq,                      // 1
+        pSData->getSubString(),  // 2
+        0L,                      // 3
+        offset,                  // 4
+        maxNums,                 // 5
+        sysFlag,                 // 6
+        0,                       // 7
+        1000 * 20,               // 8
+        timeoutMillis,           // 9
+        ComMode_ASYNC,           // 10
+        pPullCallback, getSessionCredentials(), &arg));
+  } catch (MQException& e) {
+    LOG_ERROR(e.what());
+  }
+}
+
+void DefaultMQPullConsumer::subscriptionAutomatically(const string& topic) {
+  SubscriptionData* pSdata = m_pRebalance->getSubscriptionData(topic);
+  if (pSdata == NULL) {
+    unique_ptr<SubscriptionData> subscriptionData(
+        FilterAPI::buildSubscriptionData(topic, SUB_ALL));
+    m_pRebalance->setSubscriptionData(topic, subscriptionData.release());
+  }
+}
+
+void DefaultMQPullConsumer::updateConsumeOffset(const MQMessageQueue& mq,
+                                                int64 offset) {
+  m_pOffsetStore->updateOffset(mq, offset);
+}
+
+void DefaultMQPullConsumer::removeConsumeOffset(const MQMessageQueue& mq) {
+  m_pOffsetStore->removeOffset(mq);
+}
+
+int64 DefaultMQPullConsumer::fetchConsumeOffset(const MQMessageQueue& mq,
+                                                bool fromStore) {
+  return m_pOffsetStore->readOffset(
+      mq, fromStore ? READ_FROM_STORE : MEMORY_FIRST_THEN_STORE,
+      getSessionCredentials());
+}
+
+void DefaultMQPullConsumer::persistConsumerOffset() {
+  /*As do not execute rebalance for pullConsumer now, requestTable is always empty
+  map<MQMessageQueue, PullRequest*> requestTable =
+  m_pRebalance->getPullRequestTable();
+  map<MQMessageQueue, PullRequest*>::iterator it = requestTable.begin();
+  vector<MQMessageQueue> mqs;
+  for (; it != requestTable.end(); ++it)
+  {
+      if (it->second)
+      {
+          mqs.push_back(it->first);
+      }
+  }
+  m_pOffsetStore->persistAll(mqs);*/
+}
+
+void DefaultMQPullConsumer::persistConsumerOffsetByResetOffset() {}
+
+void DefaultMQPullConsumer::persistConsumerOffset4PullConsumer(
+    const MQMessageQueue& mq) {
+  if (isServiceStateOk()) {
+    m_pOffsetStore->persist(mq, getSessionCredentials());
+  }
+}
+
+void DefaultMQPullConsumer::fetchMessageQueuesInBalance(
+    const string& topic, vector<MQMessageQueue> mqs) {}
+
+void DefaultMQPullConsumer::checkConfig() {
+  string groupname = getGroupName();
+  // check consumerGroup
+  Validators::checkGroup(groupname);
+
+  // consumerGroup
+  if (!groupname.compare(DEFAULT_CONSUMER_GROUP)) {
+    THROW_MQEXCEPTION(MQClientException,
+                      "consumerGroup can not equal DEFAULT_CONSUMER", -1);
+  }
+
+  if (getMessageModel() != BROADCASTING && getMessageModel() != CLUSTERING) {
+    THROW_MQEXCEPTION(MQClientException, "messageModel is valid ", -1);
+  }
+}
+
+void DefaultMQPullConsumer::doRebalance() {}
+
+void DefaultMQPullConsumer::copySubscription() {
+  set<string>::iterator it = m_registerTopics.begin();
+  for (; it != m_registerTopics.end(); ++it) {
+    unique_ptr<SubscriptionData> subscriptionData(
+        FilterAPI::buildSubscriptionData((*it), SUB_ALL));
+    m_pRebalance->setSubscriptionData((*it), subscriptionData.release());
+  }
+}
+
+ConsumeType DefaultMQPullConsumer::getConsumeType() { return CONSUME_ACTIVELY; }
+
+ConsumeFromWhere DefaultMQPullConsumer::getConsumeFromWhere() {
+  return CONSUME_FROM_LAST_OFFSET;
+}
+
+void DefaultMQPullConsumer::getSubscriptions(vector<SubscriptionData>& result) {
+  set<string>::iterator it = m_registerTopics.begin();
+  for (; it != m_registerTopics.end(); ++it) {
+    SubscriptionData ms(*it, SUB_ALL);
+    result.push_back(ms);
+  }
+}
+
+void DefaultMQPullConsumer::producePullMsgTask(PullRequest*) {}
+
+Rebalance* DefaultMQPullConsumer::getRebalance() const { return NULL; }
+
+//<!************************************************************************
+}  //<!end namespace;


[12/14] incubator-rocketmq-externals git commit: upload rocketmq-cpp code

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj
new file mode 100755
index 0000000..4cba934
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj
@@ -0,0 +1,168 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project DefaultTargets="Build" ToolsVersion="14.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Debug|Win32">
+      <Configuration>Debug</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Debug|x64">
+      <Configuration>Debug</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|x64">
+      <Configuration>Release</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+  </ItemGroup>
+  <PropertyGroup Label="Globals">
+    <ProjectGuid>{F93E745C-232C-46A0-8D4B-91D3A53EE699}</ProjectGuid>
+    <Keyword>Win32Proj</Keyword>
+    <ProjectName>Metaqsignature</ProjectName>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <PlatformToolset>v140</PlatformToolset>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>true</UseDebugLibraries>
+    <PlatformToolset>v140</PlatformToolset>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v140_xp</PlatformToolset>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
+    <ConfigurationType>StaticLibrary</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v140_xp</PlatformToolset>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <ImportGroup Label="ExtensionSettings">
+  </ImportGroup>
+  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <LinkIncremental>true</LinkIncremental>
+    <OutDir>..\lib</OutDir>
+    <IntDir>..\tmp</IntDir>
+    <TargetName>$(ProjectName)_d</TargetName>
+    <IncludePath>Z:\win\rocketmq-client4cpp\libs\signature\include;$(IncludePath)</IncludePath>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
+    <TargetName>$(ProjectName)_d</TargetName>
+    <IncludePath>Z:\win\rocketmq-client4cpp\libs\signature\include;$(IncludePath)</IncludePath>
+    <LinkIncremental>true</LinkIncremental>
+    <OutDir>..\lib</OutDir>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>true</LinkIncremental>
+    <OutDir>..\lib\</OutDir>
+    <IntDir>..\tmp</IntDir>
+    <TargetName>$(ProjectName)_d</TargetName>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <TargetName>$(ProjectName)_d</TargetName>
+    <LinkIncremental>true</LinkIncremental>
+    <OutDir>..\lib</OutDir>
+    <LibraryPath>C:\Program Files (x86)\Windows Kits\10\Lib\10.0.10150.0\ucrt\;$(LibraryPath)</LibraryPath>
+  </PropertyGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
+    <ClCompile>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <RuntimeLibrary>MultiThreadedDebugDLL</RuntimeLibrary>
+      <WarningLevel>Level3</WarningLevel>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
+      <Optimization>Disabled</Optimization>
+    </ClCompile>
+    <Link>
+      <TargetMachine>MachineX86</TargetMachine>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <SubSystem>Windows</SubSystem>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
+    <ClCompile>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <RuntimeLibrary>MultiThreadedDebugDLL</RuntimeLibrary>
+      <WarningLevel>Level3</WarningLevel>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
+      <Optimization>Disabled</Optimization>
+    </ClCompile>
+    <Link>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <SubSystem>Windows</SubSystem>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <RuntimeLibrary>MultiThreadedDLL</RuntimeLibrary>
+      <WarningLevel>Level3</WarningLevel>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
+      <AdditionalIncludeDirectories>..\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <TargetMachine>MachineX86</TargetMachine>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <SubSystem>Windows</SubSystem>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <ClCompile>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <RuntimeLibrary>MultiThreadedDLL</RuntimeLibrary>
+      <WarningLevel>Level3</WarningLevel>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
+      <AdditionalIncludeDirectories>..\include;C:\Program Files (x86)\Windows Kits\10\Include\10.0.10150.0\ucrt;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <SubSystem>Windows</SubSystem>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+    </Link>
+  </ItemDefinitionGroup>
+  <ItemGroup>
+    <ClCompile Include="..\src\base64.c" />
+    <ClCompile Include="..\src\hmac.c" />
+    <ClCompile Include="..\src\param_list.c" />
+    <ClCompile Include="..\src\sha1.c" />
+    <ClCompile Include="..\src\sha256.c" />
+    <ClCompile Include="..\src\sha512.c" />
+    <ClCompile Include="..\src\spas_client.c" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\include\base64.h" />
+    <ClInclude Include="..\include\hmac.h" />
+    <ClInclude Include="..\include\param_list.h" />
+    <ClInclude Include="..\include\sha1.h" />
+    <ClInclude Include="..\include\sha256.h" />
+    <ClInclude Include="..\include\sha512.h" />
+    <ClInclude Include="..\include\spas_client.h" />
+    <ClInclude Include="..\include\u64.h" />
+  </ItemGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
+  <ImportGroup Label="ExtensionTargets">
+  </ImportGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj.filters
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj.filters b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj.filters
new file mode 100755
index 0000000..92c3147
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/msvc13/Metaqsignature.vcxproj.filters
@@ -0,0 +1,66 @@
+<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup>
+    <Filter Include="Source Files">
+      <UniqueIdentifier>{4FC737F1-C7A5-4376-A066-2A32D752A2FF}</UniqueIdentifier>
+      <Extensions>cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx</Extensions>
+    </Filter>
+    <Filter Include="Header Files">
+      <UniqueIdentifier>{93995380-89BD-4b04-88EB-625FBE52EBFB}</UniqueIdentifier>
+      <Extensions>h;hh;hpp;hxx;hm;inl;inc;xsd</Extensions>
+    </Filter>
+    <Filter Include="Resource Files">
+      <UniqueIdentifier>{67DA6AB6-F800-4c08-8B7A-83BB121AAD01}</UniqueIdentifier>
+      <Extensions>rc;ico;cur;bmp;dlg;rc2;rct;bin;rgs;gif;jpg;jpeg;jpe;resx;tiff;tif;png;wav</Extensions>
+    </Filter>
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\src\sha256.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\param_list.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\base64.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\spas_client.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\sha512.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\sha1.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+    <ClCompile Include="..\src\hmac.c">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\include\sha256.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\sha512.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\u64.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\hmac.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\param_list.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\sha1.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\base64.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\include\spas_client.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/base64.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/base64.c b/rocketmq-cpp/libs/signature/src/base64.c
new file mode 100755
index 0000000..df18c81
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/base64.c
@@ -0,0 +1,437 @@
+/* -*- buffer-read-only: t -*- vi: set ro: */
+/* DO NOT EDIT! GENERATED AUTOMATICALLY! */
+/* base64.c -- Encode binary data using printable characters.
+   Copyright (C) 1999, 2000, 2001, 2004, 2005, 2006 Free Software
+   Foundation, Inc.
+
+   This program is free software; you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation; either version 2, or (at your option)
+   any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program; if not, write to the Free Software Foundation,
+   Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.  */
+
+/* Written by Simon Josefsson.  Partially adapted from GNU MailUtils
+ * (mailbox/filter_trans.c, as of 2004-11-28).  Improved by review
+ * from Paul Eggert, Bruno Haible, and Stepan Kasal.
+ *
+ * See also RFC 3548 <http://www.ietf.org/rfc/rfc3548.txt>.
+ *
+ * Be careful with error checking.  Here is how you would typically
+ * use these functions:
+ *
+ * bool ok = base64_decode_alloc (in, inlen, &out, &outlen);
+ * if (!ok)
+ *   FAIL: input was not valid base64
+ * if (out == NULL)
+ *   FAIL: memory allocation error
+ * OK: data in OUT/OUTLEN
+ *
+ * size_t outlen = base64_encode_alloc (in, inlen, &out);
+ * if (out == NULL && outlen == 0 && inlen != 0)
+ *   FAIL: input too long
+ * if (out == NULL)
+ *   FAIL: memory allocation error
+ * OK: data in OUT/OUTLEN.
+ *
+ */
+
+/* Get prototype. */
+#include "base64.h"
+
+/* Get malloc. */
+#include <stdlib.h>
+
+/* Get UCHAR_MAX. */
+#include <limits.h>
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+/* C89 compliant way to cast 'char' to 'unsigned char'. */
+#ifdef WIN32
+static _inline unsigned char
+#else
+static inline unsigned char
+#endif
+to_uchar (char ch)
+{
+  return ch;
+}
+
+/* Base64 encode IN array of size INLEN into OUT array of size OUTLEN.
+   If OUTLEN is less than BASE64_LENGTH(INLEN), write as many bytes as
+   possible.  If OUTLEN is larger than BASE64_LENGTH(INLEN), also zero
+   terminate the output buffer. */
+void
+base64_encode (const char *in, size_t inlen,
+	       char *out, size_t outlen)
+{
+  static const char b64str[65] =
+    "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/";
+
+  while (inlen && outlen)
+    {
+      *out++ = b64str[(to_uchar (in[0]) >> 2) & 0x3f];
+      if (!--outlen)
+	break;
+      *out++ = b64str[((to_uchar (in[0]) << 4)
+		       + (--inlen ? to_uchar (in[1]) >> 4 : 0))
+		      & 0x3f];
+      if (!--outlen)
+	break;
+      *out++ =
+	(inlen
+	 ? b64str[((to_uchar (in[1]) << 2)
+		   + (--inlen ? to_uchar (in[2]) >> 6 : 0))
+		  & 0x3f]
+	 : '=');
+      if (!--outlen)
+	break;
+      *out++ = inlen ? b64str[to_uchar (in[2]) & 0x3f] : '=';
+      if (!--outlen)
+	break;
+      if (inlen)
+	inlen--;
+      if (inlen)
+	in += 3;
+    }
+
+  if (outlen)
+    *out = '\0';
+}
+
+/* Allocate a buffer and store zero terminated base64 encoded data
+   from array IN of size INLEN, returning BASE64_LENGTH(INLEN), i.e.,
+   the length of the encoded data, excluding the terminating zero.  On
+   return, the OUT variable will hold a pointer to newly allocated
+   memory that must be deallocated by the caller.  If output string
+   length would overflow, 0 is returned and OUT is set to NULL.  If
+   memory allocation failed, OUT is set to NULL, and the return value
+   indicates length of the requested memory block, i.e.,
+   BASE64_LENGTH(inlen) + 1. */
+size_t
+base64_encode_alloc (const char *in, size_t inlen, char **out)
+{
+  size_t outlen = 1 + BASE64_LENGTH (inlen);
+
+  /* Check for overflow in outlen computation.
+   *
+   * If there is no overflow, outlen >= inlen.
+   *
+   * If the operation (inlen + 2) overflows then it yields at most +1, so
+   * outlen is 0.
+   *
+   * If the multiplication overflows, we lose at least half of the
+   * correct value, so the result is < ((inlen + 2) / 3) * 2, which is
+   * less than (inlen + 2) * 0.66667, which is less than inlen as soon as
+   * (inlen > 4).
+   */
+  if (inlen > outlen)
+    {
+      *out = NULL;
+      return 0;
+    }
+
+  *out = (char *)malloc (outlen);
+  if (!*out)
+    return outlen;
+
+  base64_encode (in, inlen, *out, outlen);
+
+  return outlen - 1;
+}
+
+/* With this approach this file works independent of the charset used
+   (think EBCDIC).  However, it does assume that the characters in the
+   Base64 alphabet (A-Za-z0-9+/) are encoded in 0..255.  POSIX
+   1003.1-2001 require that char and unsigned char are 8-bit
+   quantities, though, taking care of that problem.  But this may be a
+   potential problem on non-POSIX C99 platforms.
+
+   IBM C V6 for AIX mishandles "#define B64(x) ...'x'...", so use "_"
+   as the formal parameter rather than "x".  */
+#define B64(_)					\
+  ((_) == 'A' ? 0				\
+   : (_) == 'B' ? 1				\
+   : (_) == 'C' ? 2				\
+   : (_) == 'D' ? 3				\
+   : (_) == 'E' ? 4				\
+   : (_) == 'F' ? 5				\
+   : (_) == 'G' ? 6				\
+   : (_) == 'H' ? 7				\
+   : (_) == 'I' ? 8				\
+   : (_) == 'J' ? 9				\
+   : (_) == 'K' ? 10				\
+   : (_) == 'L' ? 11				\
+   : (_) == 'M' ? 12				\
+   : (_) == 'N' ? 13				\
+   : (_) == 'O' ? 14				\
+   : (_) == 'P' ? 15				\
+   : (_) == 'Q' ? 16				\
+   : (_) == 'R' ? 17				\
+   : (_) == 'S' ? 18				\
+   : (_) == 'T' ? 19				\
+   : (_) == 'U' ? 20				\
+   : (_) == 'V' ? 21				\
+   : (_) == 'W' ? 22				\
+   : (_) == 'X' ? 23				\
+   : (_) == 'Y' ? 24				\
+   : (_) == 'Z' ? 25				\
+   : (_) == 'a' ? 26				\
+   : (_) == 'b' ? 27				\
+   : (_) == 'c' ? 28				\
+   : (_) == 'd' ? 29				\
+   : (_) == 'e' ? 30				\
+   : (_) == 'f' ? 31				\
+   : (_) == 'g' ? 32				\
+   : (_) == 'h' ? 33				\
+   : (_) == 'i' ? 34				\
+   : (_) == 'j' ? 35				\
+   : (_) == 'k' ? 36				\
+   : (_) == 'l' ? 37				\
+   : (_) == 'm' ? 38				\
+   : (_) == 'n' ? 39				\
+   : (_) == 'o' ? 40				\
+   : (_) == 'p' ? 41				\
+   : (_) == 'q' ? 42				\
+   : (_) == 'r' ? 43				\
+   : (_) == 's' ? 44				\
+   : (_) == 't' ? 45				\
+   : (_) == 'u' ? 46				\
+   : (_) == 'v' ? 47				\
+   : (_) == 'w' ? 48				\
+   : (_) == 'x' ? 49				\
+   : (_) == 'y' ? 50				\
+   : (_) == 'z' ? 51				\
+   : (_) == '0' ? 52				\
+   : (_) == '1' ? 53				\
+   : (_) == '2' ? 54				\
+   : (_) == '3' ? 55				\
+   : (_) == '4' ? 56				\
+   : (_) == '5' ? 57				\
+   : (_) == '6' ? 58				\
+   : (_) == '7' ? 59				\
+   : (_) == '8' ? 60				\
+   : (_) == '9' ? 61				\
+   : (_) == '+' ? 62				\
+   : (_) == '/' ? 63				\
+   : -1)
+
+static const signed char b64[0x100] = {
+  B64 (0), B64 (1), B64 (2), B64 (3),
+  B64 (4), B64 (5), B64 (6), B64 (7),
+  B64 (8), B64 (9), B64 (10), B64 (11),
+  B64 (12), B64 (13), B64 (14), B64 (15),
+  B64 (16), B64 (17), B64 (18), B64 (19),
+  B64 (20), B64 (21), B64 (22), B64 (23),
+  B64 (24), B64 (25), B64 (26), B64 (27),
+  B64 (28), B64 (29), B64 (30), B64 (31),
+  B64 (32), B64 (33), B64 (34), B64 (35),
+  B64 (36), B64 (37), B64 (38), B64 (39),
+  B64 (40), B64 (41), B64 (42), B64 (43),
+  B64 (44), B64 (45), B64 (46), B64 (47),
+  B64 (48), B64 (49), B64 (50), B64 (51),
+  B64 (52), B64 (53), B64 (54), B64 (55),
+  B64 (56), B64 (57), B64 (58), B64 (59),
+  B64 (60), B64 (61), B64 (62), B64 (63),
+  B64 (64), B64 (65), B64 (66), B64 (67),
+  B64 (68), B64 (69), B64 (70), B64 (71),
+  B64 (72), B64 (73), B64 (74), B64 (75),
+  B64 (76), B64 (77), B64 (78), B64 (79),
+  B64 (80), B64 (81), B64 (82), B64 (83),
+  B64 (84), B64 (85), B64 (86), B64 (87),
+  B64 (88), B64 (89), B64 (90), B64 (91),
+  B64 (92), B64 (93), B64 (94), B64 (95),
+  B64 (96), B64 (97), B64 (98), B64 (99),
+  B64 (100), B64 (101), B64 (102), B64 (103),
+  B64 (104), B64 (105), B64 (106), B64 (107),
+  B64 (108), B64 (109), B64 (110), B64 (111),
+  B64 (112), B64 (113), B64 (114), B64 (115),
+  B64 (116), B64 (117), B64 (118), B64 (119),
+  B64 (120), B64 (121), B64 (122), B64 (123),
+  B64 (124), B64 (125), B64 (126), B64 (127),
+  B64 (128), B64 (129), B64 (130), B64 (131),
+  B64 (132), B64 (133), B64 (134), B64 (135),
+  B64 (136), B64 (137), B64 (138), B64 (139),
+  B64 (140), B64 (141), B64 (142), B64 (143),
+  B64 (144), B64 (145), B64 (146), B64 (147),
+  B64 (148), B64 (149), B64 (150), B64 (151),
+  B64 (152), B64 (153), B64 (154), B64 (155),
+  B64 (156), B64 (157), B64 (158), B64 (159),
+  B64 (160), B64 (161), B64 (162), B64 (163),
+  B64 (164), B64 (165), B64 (166), B64 (167),
+  B64 (168), B64 (169), B64 (170), B64 (171),
+  B64 (172), B64 (173), B64 (174), B64 (175),
+  B64 (176), B64 (177), B64 (178), B64 (179),
+  B64 (180), B64 (181), B64 (182), B64 (183),
+  B64 (184), B64 (185), B64 (186), B64 (187),
+  B64 (188), B64 (189), B64 (190), B64 (191),
+  B64 (192), B64 (193), B64 (194), B64 (195),
+  B64 (196), B64 (197), B64 (198), B64 (199),
+  B64 (200), B64 (201), B64 (202), B64 (203),
+  B64 (204), B64 (205), B64 (206), B64 (207),
+  B64 (208), B64 (209), B64 (210), B64 (211),
+  B64 (212), B64 (213), B64 (214), B64 (215),
+  B64 (216), B64 (217), B64 (218), B64 (219),
+  B64 (220), B64 (221), B64 (222), B64 (223),
+  B64 (224), B64 (225), B64 (226), B64 (227),
+  B64 (228), B64 (229), B64 (230), B64 (231),
+  B64 (232), B64 (233), B64 (234), B64 (235),
+  B64 (236), B64 (237), B64 (238), B64 (239),
+  B64 (240), B64 (241), B64 (242), B64 (243),
+  B64 (244), B64 (245), B64 (246), B64 (247),
+  B64 (248), B64 (249), B64 (250), B64 (251),
+  B64 (252), B64 (253), B64 (254), B64 (255)
+};
+
+#if UCHAR_MAX == 255
+# define uchar_in_range(c) true
+#else
+# define uchar_in_range(c) ((c) <= 255)
+#endif
+
+/* Return true if CH is a character from the Base64 alphabet, and
+   false otherwise.  Note that '=' is padding and not considered to be
+   part of the alphabet.  */
+bool
+isbase64 (char ch)
+{
+  return uchar_in_range (to_uchar (ch)) && 0 <= b64[to_uchar (ch)];
+}
+
+/* Decode base64 encoded input array IN of length INLEN to output
+   array OUT that can hold *OUTLEN bytes.  Return true if decoding was
+   successful, i.e. if the input was valid base64 data, false
+   otherwise.  If *OUTLEN is too small, as many bytes as possible will
+   be written to OUT.  On return, *OUTLEN holds the length of decoded
+   bytes in OUT.  Note that as soon as any non-alphabet characters are
+   encountered, decoding is stopped and false is returned.  This means
+   that, when applicable, you must remove any line terminators that is
+   part of the data stream before calling this function.  */
+bool
+base64_decode (const char *in, size_t inlen,
+	       char *out, size_t *outlen)
+{
+  size_t outleft = *outlen;
+
+  while (inlen >= 2)
+    {
+      if (!isbase64 (in[0]) || !isbase64 (in[1]))
+	break;
+
+      if (outleft)
+	{
+	  *out++ = ((b64[to_uchar (in[0])] << 2)
+		    | (b64[to_uchar (in[1])] >> 4));
+	  outleft--;
+	}
+
+      if (inlen == 2)
+	break;
+
+      if (in[2] == '=')
+	{
+	  if (inlen != 4)
+	    break;
+
+	  if (in[3] != '=')
+	    break;
+
+	}
+      else
+	{
+	  if (!isbase64 (in[2]))
+	    break;
+
+	  if (outleft)
+	    {
+	      *out++ = (((b64[to_uchar (in[1])] << 4) & 0xf0)
+			| (b64[to_uchar (in[2])] >> 2));
+	      outleft--;
+	    }
+
+	  if (inlen == 3)
+	    break;
+
+	  if (in[3] == '=')
+	    {
+	      if (inlen != 4)
+		break;
+	    }
+	  else
+	    {
+	      if (!isbase64 (in[3]))
+		break;
+
+	      if (outleft)
+		{
+		  *out++ = (((b64[to_uchar (in[2])] << 6) & 0xc0)
+			    | b64[to_uchar (in[3])]);
+		  outleft--;
+		}
+	    }
+	}
+
+      in += 4;
+      inlen -= 4;
+    }
+
+  *outlen -= outleft;
+
+  if (inlen != 0)
+    return false;
+
+  return true;
+}
+
+/* Allocate an output buffer in *OUT, and decode the base64 encoded
+   data stored in IN of size INLEN to the *OUT buffer.  On return, the
+   size of the decoded data is stored in *OUTLEN.  OUTLEN may be NULL,
+   if the caller is not interested in the decoded length.  *OUT may be
+   NULL to indicate an out of memory error, in which case *OUTLEN
+   contains the size of the memory block needed.  The function returns
+   true on successful decoding and memory allocation errors.  (Use the
+   *OUT and *OUTLEN parameters to differentiate between successful
+   decoding and memory error.)  The function returns false if the
+   input was invalid, in which case *OUT is NULL and *OUTLEN is
+   undefined. */
+bool
+base64_decode_alloc (const char *in, size_t inlen, char **out,
+		     size_t *outlen)
+{
+  /* This may allocate a few bytes too much, depending on input,
+     but it's not worth the extra CPU time to compute the exact amount.
+     The exact amount is 3 * inlen / 4, minus 1 if the input ends
+     with "=" and minus another 1 if the input ends with "==".
+     Dividing before multiplying avoids the possibility of overflow.  */
+  size_t needlen = 3 * (inlen / 4) + 2;
+
+  *out = (char *)malloc (needlen);
+  if (!*out)
+    return true;
+
+  if (!base64_decode (in, inlen, *out, &needlen))
+    {
+      free (*out);
+      *out = NULL;
+      return false;
+    }
+
+  if (outlen)
+    *outlen = needlen;
+
+  return true;
+#ifdef __cplusplus
+}
+#endif
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/hmac.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/hmac.c b/rocketmq-cpp/libs/signature/src/hmac.c
new file mode 100755
index 0000000..9cac58d
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/hmac.c
@@ -0,0 +1,165 @@
+
+#include <string.h>
+#include <stdint.h>
+
+#include "hmac.h"
+#include "sha1.h"
+#include "sha256.h"
+#include "sha512.h"
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+#define IPAD 0x36
+#define OPAD 0x5c
+
+int hmac_sha1(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf)
+{
+	uint32_t i;
+	struct sha1_ctx inner;
+	struct sha1_ctx outer;
+	struct sha1_ctx key_hash;
+	char ipad[64] = {0};
+	char opad[64] = {0};
+	char key_buf[SHA1_DIGEST_SIZE] = {0};
+	char inner_buf[SHA1_DIGEST_SIZE] = {0};
+
+	if (key == NULL || data == NULL || ret_buf == NULL) return -1;
+
+ 	if (key_len > 64) {
+		sha1_init_ctx(&key_hash);
+		sha1_process_bytes(key, key_len, &key_hash);
+		sha1_finish_ctx(&key_hash, key_buf);
+
+		key = key_buf;
+		key_len = SHA1_DIGEST_SIZE;
+	}
+
+	sha1_init_ctx (&inner);
+
+	for (i = 0; i < 64; i++) {
+		if (i < key_len) {
+			ipad[i] = ((const char *)key)[i] ^ IPAD;
+			opad[i] = ((const char *)key)[i] ^ OPAD;
+		} else {
+			ipad[i] = IPAD;
+			opad[i] = OPAD;
+		}
+	}
+
+  sha1_process_block (ipad, 64, &inner);
+  sha1_process_bytes (data, data_len, &inner);
+
+  sha1_finish_ctx (&inner, inner_buf);
+
+  sha1_init_ctx (&outer);
+
+  sha1_process_block (opad, 64, &outer);
+  sha1_process_bytes (inner_buf, SHA1_DIGEST_SIZE, &outer);
+
+  sha1_finish_ctx (&outer, ret_buf);
+
+  return 0;
+}
+
+int hmac_sha256(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf)
+{
+	uint32_t i;
+	struct sha256_ctx inner;
+	struct sha256_ctx outer;
+	struct sha256_ctx key_hash;
+	char ipad[64] = {0};
+	char opad[64] = {0};
+	char key_buf[SHA256_DIGEST_SIZE] = {0};
+	char inner_buf[SHA256_DIGEST_SIZE] = {0};
+
+	if (key == NULL || data == NULL || ret_buf == NULL) return -1;
+
+ 	if (key_len > 64) {
+		sha256_init_ctx(&key_hash);
+		sha256_process_bytes(key, key_len, &key_hash);
+		sha256_finish_ctx(&key_hash, key_buf);
+
+		key = key_buf;
+		key_len = SHA256_DIGEST_SIZE;
+	}
+
+	sha256_init_ctx (&inner);
+
+	for (i = 0; i < 64; i++) {
+		if (i < key_len) {
+			ipad[i] = ((const char *)key)[i] ^ IPAD;
+			opad[i] = ((const char *)key)[i] ^ OPAD;
+		} else {
+			ipad[i] = IPAD;
+			opad[i] = OPAD;
+		}
+	}
+
+  sha256_process_block (ipad, 64, &inner);
+  sha256_process_bytes (data, data_len, &inner);
+
+  sha256_finish_ctx (&inner, inner_buf);
+
+  sha256_init_ctx (&outer);
+
+  sha256_process_block (opad, 64, &outer);
+  sha256_process_bytes (inner_buf, SHA256_DIGEST_SIZE, &outer);
+
+  sha256_finish_ctx (&outer, ret_buf);
+
+  return 0;
+}
+
+int hmac_sha512(const void *key, size_t key_len, const void *data, size_t data_len, void *ret_buf)
+{
+	uint32_t i;
+	struct sha512_ctx inner;
+	struct sha512_ctx outer;
+	struct sha512_ctx key_hash;
+	char ipad[128] = {0};
+	char opad[128] = {0};
+	char key_buf[SHA512_DIGEST_SIZE] = {0};
+	char inner_buf[SHA512_DIGEST_SIZE] = {0};
+
+	if (key == NULL || data == NULL || ret_buf == NULL) return -1;
+
+ 	if (key_len > 128) {
+		sha512_init_ctx(&key_hash);
+		sha512_process_bytes(key, key_len, &key_hash);
+		sha512_finish_ctx(&key_hash, key_buf);
+
+		key = key_buf;
+		key_len = SHA512_DIGEST_SIZE;
+	}
+
+	sha512_init_ctx (&inner);
+
+	for (i = 0; i < 128; i++) {
+		if (i < key_len) {
+			ipad[i] = ((const char *)key)[i] ^ IPAD;
+			opad[i] = ((const char *)key)[i] ^ OPAD;
+		} else {
+			ipad[i] = IPAD;
+			opad[i] = OPAD;
+		}
+	}
+
+  sha512_process_block (ipad, 128, &inner);
+  sha512_process_bytes (data, data_len, &inner);
+
+  sha512_finish_ctx (&inner, inner_buf);
+
+  sha512_init_ctx (&outer);
+
+  sha512_process_block (opad, 128, &outer);
+  sha512_process_bytes (inner_buf, SHA512_DIGEST_SIZE, &outer);
+
+  sha512_finish_ctx (&outer, ret_buf);
+
+  return 0;
+}
+#ifdef __cplusplus
+}
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/param_list.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/param_list.c b/rocketmq-cpp/libs/signature/src/param_list.c
new file mode 100755
index 0000000..7476169
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/param_list.c
@@ -0,0 +1,130 @@
+#include <stdio.h>
+#include <string.h>
+#include "param_list.h"
+#include "spas_client.h"
+
+#ifdef __cplusplus
+namespace metaqSignature{
+#endif
+
+extern void * _mem_alloc(unsigned int size);
+extern void _mem_free(void *ptr);
+
+static int _nodecmp(SPAS_PARAM_NODE *n1, SPAS_PARAM_NODE *n2) {
+	int ret = strcmp(n1->name, n2->name);
+	if (ret == 0) {
+		ret = strcmp(n1->value, n2->value);
+	}
+	return ret;
+}
+
+SPAS_PARAM_LIST * create_param_list() {
+	return (SPAS_PARAM_LIST *)_mem_alloc(sizeof(SPAS_PARAM_LIST));
+}
+
+void free_param_list(SPAS_PARAM_LIST *list) {
+	SPAS_PARAM_NODE *pnode = NULL;
+	SPAS_PARAM_NODE *pnext = NULL;
+	if (list == NULL) {
+		return;
+	}
+	pnode = list->phead;
+	while (pnode != NULL) {
+		pnext = pnode->pnext;
+		_mem_free(pnode->name);
+		_mem_free(pnode->value);
+		_mem_free(pnode);
+		pnode = pnext;
+	}
+	_mem_free(list);
+}
+
+
+int add_param_to_list(SPAS_PARAM_LIST *list, const char *name, const char *value) {
+	SPAS_PARAM_NODE *pnode = NULL;
+	SPAS_PARAM_NODE *plast = NULL;
+	int nlen = 0;
+	int vlen = 0;
+	if (list == NULL || name == NULL || value == NULL) {
+		return ERROR_INVALID_PARAM;
+	}
+	nlen = strlen(name);
+	vlen = strlen(value);
+	pnode = (SPAS_PARAM_NODE *)_mem_alloc(sizeof(SPAS_PARAM_NODE));
+	if (pnode == NULL) {
+		return ERROR_MEM_ALLOC;
+	}
+	pnode->name = (char *)_mem_alloc(nlen + 1);
+	if (pnode->name == NULL) {
+		_mem_free(pnode);
+		return ERROR_MEM_ALLOC;
+	}
+	pnode->value = (char *)_mem_alloc(vlen + 1);
+	if (pnode->value == NULL) {
+		_mem_free(pnode->name);
+		_mem_free(pnode);
+		return ERROR_MEM_ALLOC;
+	}
+	memcpy(pnode->name, name, nlen);
+	memcpy(pnode->value, value, vlen);
+	if (list->phead == NULL) {
+		list->phead = pnode;
+	}
+	else if (_nodecmp(pnode, list->phead) <= 0) {
+		pnode->pnext = list->phead;
+		list->phead = pnode;
+	}
+	else {
+		plast = list->phead;
+		while (plast->pnext!= NULL) {
+			if (_nodecmp(pnode, plast->pnext) <= 0) {
+				pnode->pnext = plast->pnext;
+				plast->pnext= pnode;
+				break;
+			}
+			else {
+				plast = plast->pnext;
+			}
+		}
+		if (plast->pnext == NULL) {
+			plast->pnext = pnode;
+		}
+	}
+	list->length++;
+	list->size += nlen + vlen + 1; /* 1 overhead for '=' */
+	return NO_ERROR;
+}
+
+
+char * param_list_to_str(const SPAS_PARAM_LIST *list) {
+	int size = 0;
+	int pos = 0;
+	char *buf = NULL;
+	SPAS_PARAM_NODE *pnode = NULL;
+	if (list == NULL) {
+		return NULL;
+	}
+	if (list->length == 0) {
+		return (char *)_mem_alloc(1);
+	}
+	size = list->size + list->length - 1; /* overhead for '&' */
+	buf = (char *)_mem_alloc(size);
+	if (buf == NULL) {
+		return NULL;
+	}
+	pnode = list->phead;
+	if (pnode != NULL) {
+		sprintf(buf, "%s=%s", pnode->name, pnode->value);
+		pos += strlen(pnode->name) + strlen(pnode->value) + 1;
+		pnode = pnode->pnext;
+	}
+	while (pnode != NULL) {
+		sprintf(buf + pos, "&%s=%s", pnode->name, pnode->value);
+		pos += strlen(pnode->name) + strlen(pnode->value) + 2;
+		pnode = pnode->pnext;
+	}
+	return buf;
+}
+#ifdef __cplusplus
+}
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/sha1.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/sha1.c b/rocketmq-cpp/libs/signature/src/sha1.c
new file mode 100755
index 0000000..5316204
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/sha1.c
@@ -0,0 +1,516 @@
+/* sha1.c - Functions to compute SHA1 message digest of files or
+   memory blocks according to the NIST specification FIPS-180-1.
+
+   Copyright (C) 2000, 2001, 2003, 2004, 2005, 2006, 2008 Free Software
+   Foundation, Inc.
+
+   This program is free software; you can redistribute it and/or modify it
+   under the terms of the GNU General Public License as published by the
+   Free Software Foundation; either version 2, or (at your option) any
+   later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program; if not, write to the Free Software Foundation,
+   Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.  */
+
+/* Written by Scott G. Miller
+   Credits:
+      Robert Klep <ro...@ilse.nl>  -- Expansion function fix
+*/
+
+
+#include "sha1.h"
+
+#include <stddef.h>
+#include <string.h>
+
+#if USE_UNLOCKED_IO
+# include "unlocked-io.h"
+#endif
+
+#ifdef __cplusplus
+namespace metaqSignature {
+#endif
+
+#ifdef WORDS_BIGENDIAN
+# define SWAP(n) (n)
+#else
+# define SWAP(n) \
+    (((n) << 24) | (((n) & 0xff00) << 8) | (((n) >> 8) & 0xff00) | ((n) >> 24))
+#endif
+
+#define BLOCKSIZE 4096
+#if BLOCKSIZE % 64 != 0
+# error "invalid BLOCKSIZE"
+#endif
+
+/* This array contains the bytes used to pad the buffer to the next
+   64-byte boundary.  (RFC 1321, 3.1: Step 1)  */
+static const unsigned char fillbuf[64] = { 0x80, 0 /* , 0, 0, ...  */ };
+
+/*!
+ * @fn void sha1_init_ctx (struct sha1_ctx *ctx)
+ *
+ * @brief initialize a context with start constants
+ *
+ * @details Take a pointer to a 160 bit block of data (five 32 bit ints) and
+ *          initialize it to the start constants of the SHA1 algorithm.  This
+ *          must be called before using hash in the call to sha1_hash.
+ *
+ * @param[out] ctx  pointer to a context to be initialized
+ */
+void
+sha1_init_ctx (struct sha1_ctx *ctx)
+{
+  ctx->A = 0x67452301;
+  ctx->B = 0xefcdab89;
+  ctx->C = 0x98badcfe;
+  ctx->D = 0x10325476;
+  ctx->E = 0xc3d2e1f0;
+
+  ctx->total[0] = ctx->total[1] = 0;
+  ctx->buflen = 0;
+}
+
+/*!
+ * @fn static __inline__ void set_uint32 (char *cp, uint32_t v)
+ *
+ * @brief Copy the 4 byte value from v into the memory location pointed to
+          by *cp
+ *
+ * @details Copy the 4 byte value from v into the memory location pointed to by
+ *          *cp, If your architecture allows unaligned access this is equivalent
+ *          to * (uint32_t *) cp = v
+ *
+ * @param[out]  cp  memory location to copy v into
+ * @param[in]   v   4 byte value to be copied
+ */
+#ifdef WIN32
+static _inline void
+#else
+static __inline__ void
+#endif
+set_uint32 (char *cp, uint32_t v)
+{
+  memcpy (cp, &v, sizeof v);
+}
+
+/*!
+ * @fn void *sha1_read_ctx (const struct sha1_ctx *ctx, void *resbuf)
+ *
+ * @brief Put result from CTX in first 20 bytes following RESBUF
+ *
+ * @details Put result from CTX in first 20 bytes following RESBUF.  The result
+ *          must be in little endian byte order.
+ *
+ * @param[in]   ctx     context whose results will be copied
+ * @param[out]  resbuf  result of copies saved in little endian byte order
+ * @return resbuf
+ */
+void *
+sha1_read_ctx (const struct sha1_ctx *ctx, void *resbuf)
+{
+  char *r = (char*)resbuf;
+  set_uint32 (r + 0 * sizeof ctx->A, SWAP (ctx->A));
+  set_uint32 (r + 1 * sizeof ctx->B, SWAP (ctx->B));
+  set_uint32 (r + 2 * sizeof ctx->C, SWAP (ctx->C));
+  set_uint32 (r + 3 * sizeof ctx->D, SWAP (ctx->D));
+  set_uint32 (r + 4 * sizeof ctx->E, SWAP (ctx->E));
+
+  return resbuf;
+}
+
+/*!
+ * @fn void *sha1_finish_ctx (struct sha1_ctx *ctx, void *resbuf)
+ *
+ * @brief Process the remaining bytes in the internal buffer and write
+          the result to RESBUF.
+ *
+ * @details Process the remaining bytes in the internal buffer and the usual
+ *          prolog according to the standard and write the result to RESBUF.
+ *
+ * @param[in]  ctx     context to be used
+ * @param[out] resbuf  resultant SHA1 hash
+ * @return resultant SHA1 hash
+ */
+void *
+sha1_finish_ctx (struct sha1_ctx *ctx, void *resbuf)
+{
+  /* Take yet unprocessed bytes into account.  */
+  uint32_t bytes = ctx->buflen;
+  size_t size = (bytes < 56) ? 64 / 4 : 64 * 2 / 4;
+
+  /* Now count remaining bytes.  */
+  ctx->total[0] += bytes;
+  if (ctx->total[0] < bytes)
+    ++ctx->total[1];
+
+  /* Put the 64-bit file length in *bits* at the end of the buffer.  */
+  ctx->buffer[size - 2] = SWAP ((ctx->total[1] << 3) | (ctx->total[0] >> 29));
+  ctx->buffer[size - 1] = SWAP (ctx->total[0] << 3);
+
+  memcpy (&((char *) ctx->buffer)[bytes], fillbuf, (size - 2) * 4 - bytes);
+
+  /* Process last bytes.  */
+  sha1_process_block (ctx->buffer, size * 4, ctx);
+
+  return sha1_read_ctx (ctx, resbuf);
+}
+
+
+/*
+ * @fn void *sha1_stream (FILE *stream, void *resblock)
+ *
+ * @brief Compute SHA1 message digest for A Stream.
+ *
+ * @details Compute SHA1 message digest for Stream.  The
+ *          result is always in little endian byte order, so that a byte-wise
+ *          output yields to the wanted ASCII representation of the message
+ *          digest.
+ *
+ * @param[in]  stream    message stream to be hashed
+ * @param[out] resblock  resultant hash in little endian byte order
+ * @return resultant hash in little endian byte order
+ */
+int
+sha1_stream (FILE *stream, void *resblock)
+{
+  struct sha1_ctx ctx;
+  char buffer[BLOCKSIZE + 72];
+  size_t sum;
+
+  /* Initialize the computation context.  */
+  sha1_init_ctx (&ctx);
+
+  /* Iterate over full file contents.  */
+  while (1)
+    {
+      /* We read the file in blocks of BLOCKSIZE bytes.  One call of the
+	 computation function processes the whole buffer so that with the
+	 next round of the loop another block can be read.  */
+      size_t n;
+      sum = 0;
+
+      /* Read block.  Take care for partial reads.  */
+      while (1)
+	{
+	  n = fread (buffer + sum, 1, BLOCKSIZE - sum, stream);
+
+	  sum += n;
+
+	  if (sum == BLOCKSIZE)
+	    break;
+
+	  if (n == 0)
+	    {
+	      /* Check for the error flag IFF N == 0, so that we don't
+		 exit the loop after a partial read due to e.g., EAGAIN
+		 or EWOULDBLOCK.  */
+	      if (ferror (stream))
+		return 1;
+	      goto process_partial_block;
+	    }
+
+	  /* We've read at least one byte, so ignore errors.  But always
+	     check for EOF, since feof may be true even though N > 0.
+	     Otherwise, we could end up calling fread after EOF.  */
+	  if (feof (stream))
+	    goto process_partial_block;
+	}
+
+      /* Process buffer with BLOCKSIZE bytes.  Note that
+			BLOCKSIZE % 64 == 0
+       */
+      sha1_process_block (buffer, BLOCKSIZE, &ctx);
+    }
+
+ process_partial_block:;
+
+  /* Process any remaining bytes.  */
+  if (sum > 0)
+    sha1_process_bytes (buffer, sum, &ctx);
+
+  /* Construct result in desired memory.  */
+  sha1_finish_ctx (&ctx, resblock);
+  return 0;
+}
+
+
+/*
+ * @fn void *sha1_buffer (const char *buffer, size_t len, void *resblock)
+ *
+ * @brief Compute SHA1 message digest for LEN bytes beginning at BUFFER.
+ *
+ * @details Compute SHA1 message digest for LEN bytes beginning at BUFFER.  The
+ *          result is always in little endian byte order, so that a byte-wise
+ *          output yields to the wanted ASCII representation of the message
+ *          digest.
+ *
+ * @param[in]  buffer    message to be hashed
+ * @param[in]  len       length of buffer
+ * @param[out] resblock  resultant hash in little endian byte order
+ * @return resultant hash in little endian byte order
+ */
+void *
+sha1_buffer (const char *buffer, size_t len, void *resblock)
+{
+  struct sha1_ctx ctx;
+
+  /* Initialize the computation context.  */
+  sha1_init_ctx (&ctx);
+
+  /* Process whole buffer but last len % 64 bytes.  */
+  sha1_process_bytes (buffer, len, &ctx);
+
+  /* Put result in desired memory area.  */
+  return sha1_finish_ctx (&ctx, resblock);
+}
+
+/*!
+ * @fn void sha1_process_bytes (const void *buffer, size_t len, struct sha1_ctx *ctx)
+ *
+ * @brief update the context for the next LEN bytes starting at BUFFER.
+ *
+ * @details Starting with the result of former calls of this function (or the
+ *          initialization function) update the context for the next LEN bytes
+ *          starting at BUFFER.
+ *          It is NOT required that LEN is a multiple of 64.
+ *
+ * @param[in]  buffer  buffer used to update context values
+ * @param[in]  len     length of buffer
+ * @param[out] ctx     context to be updated
+ */
+void
+sha1_process_bytes (const void *buffer, size_t len, struct sha1_ctx *ctx)
+{
+  /* When we already have some bits in our internal buffer concatenate
+     both inputs first.  */
+  if (ctx->buflen != 0)
+    {
+      size_t left_over = ctx->buflen;
+      size_t add = 128 - left_over > len ? len : 128 - left_over;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, add);
+      ctx->buflen += add;
+
+      if (ctx->buflen > 64)
+	{
+	  sha1_process_block (ctx->buffer, ctx->buflen & ~63, ctx);
+
+	  ctx->buflen &= 63;
+	  /* The regions in the following copy operation cannot overlap.  */
+	  memcpy (ctx->buffer,
+		  &((char *) ctx->buffer)[(left_over + add) & ~63],
+		  ctx->buflen);
+	}
+
+      buffer = (const char *) buffer + add;
+      len -= add;
+    }
+
+  /* Process available complete blocks.  */
+  if (len >= 64)
+    {
+#if !_STRING_ARCH_unaligned
+# define alignof(type) offsetof (struct { char c; type x; }, x)
+# define UNALIGNED_P(p) (((size_t) p) % alignof (uint32_t) != 0)
+      if (UNALIGNED_P (buffer))
+	while (len > 64)
+	  {
+	    sha1_process_block (memcpy (ctx->buffer, buffer, 64), 64, ctx);
+	    buffer = (const char *) buffer + 64;
+	    len -= 64;
+	  }
+      else
+#endif
+	{
+	  sha1_process_block (buffer, len & ~63, ctx);
+	  buffer = (const char *) buffer + (len & ~63);
+	  len &= 63;
+	}
+    }
+
+  /* Move remaining bytes in internal buffer.  */
+  if (len > 0)
+    {
+      size_t left_over = ctx->buflen;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, len);
+      left_over += len;
+      if (left_over >= 64)
+	{
+	  sha1_process_block (ctx->buffer, 64, ctx);
+	  left_over -= 64;
+	  memcpy (ctx->buffer, &ctx->buffer[16], left_over);
+	}
+      ctx->buflen = left_over;
+    }
+}
+
+/* --- Code below is the primary difference between md5.c and sha1.c --- */
+
+/* SHA1 round constants */
+#define K1 0x5a827999
+#define K2 0x6ed9eba1
+#define K3 0x8f1bbcdc
+#define K4 0xca62c1d6
+
+/* Round functions.  Note that F2 is the same as F4.  */
+#define F1(B,C,D) ( D ^ ( B & ( C ^ D ) ) )
+#define F2(B,C,D) (B ^ C ^ D)
+#define F3(B,C,D) ( ( B & C ) | ( D & ( B | C ) ) )
+#define F4(B,C,D) (B ^ C ^ D)
+
+/*!
+ * @fn void sha1_process_block (const void *buffer, size_t len, struct sha1_ctx *ctx)
+ *
+ * @brief Process LEN bytes of BUFFER, accumulating context into CTX.
+ *
+ * @details Process LEN bytes of BUFFER, accumulating context into CTX.
+ *          It is assumed that LEN % 64 == 0.
+ *          Most of this code comes from GnuPG's cipher/sha1.c.
+ *
+ * @param[in]  buffer buffer to be processed
+ * @param[in]  len    length of buffer
+ * @param[out] ctx    context used to accumulate results
+ */
+
+void
+sha1_process_block (const void *buffer, size_t len, struct sha1_ctx *ctx)
+{
+  const uint32_t *words = (const uint32_t*)buffer;
+  size_t nwords = len / sizeof (uint32_t);
+  const uint32_t *endp = words + nwords;
+  uint32_t x[16];
+  uint32_t a = ctx->A;
+  uint32_t b = ctx->B;
+  uint32_t c = ctx->C;
+  uint32_t d = ctx->D;
+  uint32_t e = ctx->E;
+
+  /* First increment the byte count.  RFC 1321 specifies the possible
+     length of the file up to 2^64 bits.  Here we only compute the
+     number of bytes.  Do a double word increment.  */
+  ctx->total[0] += len;
+  if (ctx->total[0] < len)
+    ++ctx->total[1];
+
+#define rol(x, n) (((x) << (n)) | ((uint32_t) (x) >> (32 - (n))))
+
+#define M(I) ( tm =   x[I&0x0f] ^ x[(I-14)&0x0f] \
+		    ^ x[(I-8)&0x0f] ^ x[(I-3)&0x0f] \
+	       , (x[I&0x0f] = rol(tm, 1)) )
+
+#define R(A,B,C,D,E,F,K,M)  do { E += rol( A, 5 )     \
+				      + F( B, C, D )  \
+				      + K	      \
+				      + M;	      \
+				 B = rol( B, 30 );    \
+			       } while(0)
+
+  while (words < endp)
+    {
+      uint32_t tm;
+      int t;
+      for (t = 0; t < 16; t++)
+	{
+	  x[t] = SWAP (*words);
+	  words++;
+	}
+
+      R( a, b, c, d, e, F1, K1, x[ 0] );
+      R( e, a, b, c, d, F1, K1, x[ 1] );
+      R( d, e, a, b, c, F1, K1, x[ 2] );
+      R( c, d, e, a, b, F1, K1, x[ 3] );
+      R( b, c, d, e, a, F1, K1, x[ 4] );
+      R( a, b, c, d, e, F1, K1, x[ 5] );
+      R( e, a, b, c, d, F1, K1, x[ 6] );
+      R( d, e, a, b, c, F1, K1, x[ 7] );
+      R( c, d, e, a, b, F1, K1, x[ 8] );
+      R( b, c, d, e, a, F1, K1, x[ 9] );
+      R( a, b, c, d, e, F1, K1, x[10] );
+      R( e, a, b, c, d, F1, K1, x[11] );
+      R( d, e, a, b, c, F1, K1, x[12] );
+      R( c, d, e, a, b, F1, K1, x[13] );
+      R( b, c, d, e, a, F1, K1, x[14] );
+      R( a, b, c, d, e, F1, K1, x[15] );
+      R( e, a, b, c, d, F1, K1, M(16) );
+      R( d, e, a, b, c, F1, K1, M(17) );
+      R( c, d, e, a, b, F1, K1, M(18) );
+      R( b, c, d, e, a, F1, K1, M(19) );
+      R( a, b, c, d, e, F2, K2, M(20) );
+      R( e, a, b, c, d, F2, K2, M(21) );
+      R( d, e, a, b, c, F2, K2, M(22) );
+      R( c, d, e, a, b, F2, K2, M(23) );
+      R( b, c, d, e, a, F2, K2, M(24) );
+      R( a, b, c, d, e, F2, K2, M(25) );
+      R( e, a, b, c, d, F2, K2, M(26) );
+      R( d, e, a, b, c, F2, K2, M(27) );
+      R( c, d, e, a, b, F2, K2, M(28) );
+      R( b, c, d, e, a, F2, K2, M(29) );
+      R( a, b, c, d, e, F2, K2, M(30) );
+      R( e, a, b, c, d, F2, K2, M(31) );
+      R( d, e, a, b, c, F2, K2, M(32) );
+      R( c, d, e, a, b, F2, K2, M(33) );
+      R( b, c, d, e, a, F2, K2, M(34) );
+      R( a, b, c, d, e, F2, K2, M(35) );
+      R( e, a, b, c, d, F2, K2, M(36) );
+      R( d, e, a, b, c, F2, K2, M(37) );
+      R( c, d, e, a, b, F2, K2, M(38) );
+      R( b, c, d, e, a, F2, K2, M(39) );
+      R( a, b, c, d, e, F3, K3, M(40) );
+      R( e, a, b, c, d, F3, K3, M(41) );
+      R( d, e, a, b, c, F3, K3, M(42) );
+      R( c, d, e, a, b, F3, K3, M(43) );
+      R( b, c, d, e, a, F3, K3, M(44) );
+      R( a, b, c, d, e, F3, K3, M(45) );
+      R( e, a, b, c, d, F3, K3, M(46) );
+      R( d, e, a, b, c, F3, K3, M(47) );
+      R( c, d, e, a, b, F3, K3, M(48) );
+      R( b, c, d, e, a, F3, K3, M(49) );
+      R( a, b, c, d, e, F3, K3, M(50) );
+      R( e, a, b, c, d, F3, K3, M(51) );
+      R( d, e, a, b, c, F3, K3, M(52) );
+      R( c, d, e, a, b, F3, K3, M(53) );
+      R( b, c, d, e, a, F3, K3, M(54) );
+      R( a, b, c, d, e, F3, K3, M(55) );
+      R( e, a, b, c, d, F3, K3, M(56) );
+      R( d, e, a, b, c, F3, K3, M(57) );
+      R( c, d, e, a, b, F3, K3, M(58) );
+      R( b, c, d, e, a, F3, K3, M(59) );
+      R( a, b, c, d, e, F4, K4, M(60) );
+      R( e, a, b, c, d, F4, K4, M(61) );
+      R( d, e, a, b, c, F4, K4, M(62) );
+      R( c, d, e, a, b, F4, K4, M(63) );
+      R( b, c, d, e, a, F4, K4, M(64) );
+      R( a, b, c, d, e, F4, K4, M(65) );
+      R( e, a, b, c, d, F4, K4, M(66) );
+      R( d, e, a, b, c, F4, K4, M(67) );
+      R( c, d, e, a, b, F4, K4, M(68) );
+      R( b, c, d, e, a, F4, K4, M(69) );
+      R( a, b, c, d, e, F4, K4, M(70) );
+      R( e, a, b, c, d, F4, K4, M(71) );
+      R( d, e, a, b, c, F4, K4, M(72) );
+      R( c, d, e, a, b, F4, K4, M(73) );
+      R( b, c, d, e, a, F4, K4, M(74) );
+      R( a, b, c, d, e, F4, K4, M(75) );
+      R( e, a, b, c, d, F4, K4, M(76) );
+      R( d, e, a, b, c, F4, K4, M(77) );
+      R( c, d, e, a, b, F4, K4, M(78) );
+      R( b, c, d, e, a, F4, K4, M(79) );
+
+      a = ctx->A += a;
+      b = ctx->B += b;
+      c = ctx->C += c;
+      d = ctx->D += d;
+      e = ctx->E += e;
+    }
+}
+#ifdef __cplusplus
+}
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-rocketmq-externals/blob/70ce5c77/rocketmq-cpp/libs/signature/src/sha256.c
----------------------------------------------------------------------
diff --git a/rocketmq-cpp/libs/signature/src/sha256.c b/rocketmq-cpp/libs/signature/src/sha256.c
new file mode 100755
index 0000000..ffa65fa
--- /dev/null
+++ b/rocketmq-cpp/libs/signature/src/sha256.c
@@ -0,0 +1,566 @@
+/* sha256.c - Functions to compute SHA256 and SHA224 message digest of files or
+   memory blocks according to the NIST specification FIPS-180-2.
+
+   Copyright (C) 2005, 2006, 2008 Free Software Foundation, Inc.
+
+   This program is free software: you can redistribute it and/or modify
+   it under the terms of the GNU General Public License as published by
+   the Free Software Foundation, either version 3 of the License, or
+   (at your option) any later version.
+
+   This program is distributed in the hope that it will be useful,
+   but WITHOUT ANY WARRANTY; without even the implied warranty of
+   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+   GNU General Public License for more details.
+
+   You should have received a copy of the GNU General Public License
+   along with this program.  If not, see <http://www.gnu.org/licenses/>.  */
+
+/* Written by David Madore, considerably copypasting from
+   Scott G. Miller's sha1.c
+*/
+
+/* #include <config.h> */
+
+#include "sha256.h"
+
+#include <stddef.h>
+#include <string.h>
+
+#if USE_UNLOCKED_IO
+# include "unlocked-io.h"
+#endif
+
+#ifdef __cplusplus
+namespace metaqSignature {
+#endif
+
+#ifdef WORDS_BIGENDIAN
+# define SWAP(n) (n)
+#else
+# define SWAP(n) \
+    (((n) << 24) | (((n) & 0xff00) << 8) | (((n) >> 8) & 0xff00) | ((n) >> 24))
+#endif
+
+#define BLOCKSIZE 4096
+#if BLOCKSIZE % 64 != 0
+# error "invalid BLOCKSIZE"
+#endif
+
+/* This array contains the bytes used to pad the buffer to the next
+   64-byte boundary.  */
+static const unsigned char fillbuf[64] = { 0x80, 0 /* , 0, 0, ...  */ };
+
+
+/*
+  Takes a pointer to a 256 bit block of data (eight 32 bit ints) and
+  intializes it to the start constants of the SHA256 algorithm.  This
+  must be called before using hash in the call to sha256_hash
+*/
+void
+sha256_init_ctx (struct sha256_ctx *ctx)
+{
+  ctx->state[0] = 0x6a09e667UL;
+  ctx->state[1] = 0xbb67ae85UL;
+  ctx->state[2] = 0x3c6ef372UL;
+  ctx->state[3] = 0xa54ff53aUL;
+  ctx->state[4] = 0x510e527fUL;
+  ctx->state[5] = 0x9b05688cUL;
+  ctx->state[6] = 0x1f83d9abUL;
+  ctx->state[7] = 0x5be0cd19UL;
+
+  ctx->total[0] = ctx->total[1] = 0;
+  ctx->buflen = 0;
+}
+
+void
+sha224_init_ctx (struct sha256_ctx *ctx)
+{
+  ctx->state[0] = 0xc1059ed8UL;
+  ctx->state[1] = 0x367cd507UL;
+  ctx->state[2] = 0x3070dd17UL;
+  ctx->state[3] = 0xf70e5939UL;
+  ctx->state[4] = 0xffc00b31UL;
+  ctx->state[5] = 0x68581511UL;
+  ctx->state[6] = 0x64f98fa7UL;
+  ctx->state[7] = 0xbefa4fa4UL;
+
+  ctx->total[0] = ctx->total[1] = 0;
+  ctx->buflen = 0;
+}
+
+/* Copy the value from v into the memory location pointed to by *cp,
+   If your architecture allows unaligned access this is equivalent to
+   * (uint32_t *) cp = v  */
+#ifdef WIN32
+static _inline void
+#else
+static __inline__ void
+#endif
+set_uint32 (char *cp, uint32_t v)
+{
+  memcpy (cp, &v, sizeof v);
+}
+
+/* Put result from CTX in first 32 bytes following RESBUF.  The result
+   must be in little endian byte order.  */
+void *
+sha256_read_ctx (const struct sha256_ctx *ctx, void *resbuf)
+{
+  int i;
+  char *r = (char*)resbuf;
+
+  for (i = 0; i < 8; i++)
+    set_uint32 (r + i * sizeof ctx->state[0], SWAP (ctx->state[i]));
+
+  return resbuf;
+}
+
+void *
+sha224_read_ctx (const struct sha256_ctx *ctx, void *resbuf)
+{
+  int i;
+  char *r = (char*)resbuf;
+
+  for (i = 0; i < 7; i++)
+    set_uint32 (r + i * sizeof ctx->state[0], SWAP (ctx->state[i]));
+
+  return resbuf;
+}
+
+/* Process the remaining bytes in the internal buffer and the usual
+   prolog according to the standard and write the result to RESBUF.  */
+static void
+sha256_conclude_ctx (struct sha256_ctx *ctx)
+{
+  /* Take yet unprocessed bytes into account.  */
+  size_t bytes = ctx->buflen;
+  size_t size = (bytes < 56) ? 64 / 4 : 64 * 2 / 4;
+
+  /* Now count remaining bytes.  */
+  ctx->total[0] += bytes;
+  if (ctx->total[0] < bytes)
+    ++ctx->total[1];
+
+  /* Put the 64-bit file length in *bits* at the end of the buffer.
+     Use set_uint32 rather than a simple assignment, to avoid risk of
+     unaligned access.  */
+  set_uint32 ((char *) &ctx->buffer[size - 2],
+	      SWAP ((ctx->total[1] << 3) | (ctx->total[0] >> 29)));
+  set_uint32 ((char *) &ctx->buffer[size - 1],
+	      SWAP (ctx->total[0] << 3));
+
+  memcpy (&((char *) ctx->buffer)[bytes], fillbuf, (size - 2) * 4 - bytes);
+
+  /* Process last bytes.  */
+  sha256_process_block (ctx->buffer, size * 4, ctx);
+}
+
+void *
+sha256_finish_ctx (struct sha256_ctx *ctx, void *resbuf)
+{
+  sha256_conclude_ctx (ctx);
+  return sha256_read_ctx (ctx, resbuf);
+}
+
+void *
+sha224_finish_ctx (struct sha256_ctx *ctx, void *resbuf)
+{
+  sha256_conclude_ctx (ctx);
+  return sha224_read_ctx (ctx, resbuf);
+}
+
+/* Compute SHA256 message digest for bytes read from STREAM.  The
+   resulting message digest number will be written into the 32 bytes
+   beginning at RESBLOCK.  */
+int
+sha256_stream (FILE *stream, void *resblock)
+{
+  struct sha256_ctx ctx;
+  char buffer[BLOCKSIZE + 72];
+  size_t sum;
+
+  /* Initialize the computation context.  */
+  sha256_init_ctx (&ctx);
+
+  /* Iterate over full file contents.  */
+  while (1)
+    {
+      /* We read the file in blocks of BLOCKSIZE bytes.  One call of the
+	 computation function processes the whole buffer so that with the
+	 next round of the loop another block can be read.  */
+      size_t n;
+      sum = 0;
+
+      /* Read block.  Take care for partial reads.  */
+      while (1)
+	{
+	  n = fread (buffer + sum, 1, BLOCKSIZE - sum, stream);
+
+	  sum += n;
+
+	  if (sum == BLOCKSIZE)
+	    break;
+
+	  if (n == 0)
+	    {
+	      /* Check for the error flag IFF N == 0, so that we don't
+		 exit the loop after a partial read due to e.g., EAGAIN
+		 or EWOULDBLOCK.  */
+	      if (ferror (stream))
+		return 1;
+	      goto process_partial_block;
+	    }
+
+	  /* We've read at least one byte, so ignore errors.  But always
+	     check for EOF, since feof may be true even though N > 0.
+	     Otherwise, we could end up calling fread after EOF.  */
+	  if (feof (stream))
+	    goto process_partial_block;
+	}
+
+      /* Process buffer with BLOCKSIZE bytes.  Note that
+			BLOCKSIZE % 64 == 0
+       */
+      sha256_process_block (buffer, BLOCKSIZE, &ctx);
+    }
+
+ process_partial_block:;
+
+  /* Process any remaining bytes.  */
+  if (sum > 0)
+    sha256_process_bytes (buffer, sum, &ctx);
+
+  /* Construct result in desired memory.  */
+  sha256_finish_ctx (&ctx, resblock);
+  return 0;
+}
+
+/* FIXME: Avoid code duplication */
+int
+sha224_stream (FILE *stream, void *resblock)
+{
+  struct sha256_ctx ctx;
+  char buffer[BLOCKSIZE + 72];
+  size_t sum;
+
+  /* Initialize the computation context.  */
+  sha224_init_ctx (&ctx);
+
+  /* Iterate over full file contents.  */
+  while (1)
+    {
+      /* We read the file in blocks of BLOCKSIZE bytes.  One call of the
+	 computation function processes the whole buffer so that with the
+	 next round of the loop another block can be read.  */
+      size_t n;
+      sum = 0;
+
+      /* Read block.  Take care for partial reads.  */
+      while (1)
+	{
+	  n = fread (buffer + sum, 1, BLOCKSIZE - sum, stream);
+
+	  sum += n;
+
+	  if (sum == BLOCKSIZE)
+	    break;
+
+	  if (n == 0)
+	    {
+	      /* Check for the error flag IFF N == 0, so that we don't
+		 exit the loop after a partial read due to e.g., EAGAIN
+		 or EWOULDBLOCK.  */
+	      if (ferror (stream))
+		return 1;
+	      goto process_partial_block;
+	    }
+
+	  /* We've read at least one byte, so ignore errors.  But always
+	     check for EOF, since feof may be true even though N > 0.
+	     Otherwise, we could end up calling fread after EOF.  */
+	  if (feof (stream))
+	    goto process_partial_block;
+	}
+
+      /* Process buffer with BLOCKSIZE bytes.  Note that
+			BLOCKSIZE % 64 == 0
+       */
+      sha256_process_block (buffer, BLOCKSIZE, &ctx);
+    }
+
+ process_partial_block:;
+
+  /* Process any remaining bytes.  */
+  if (sum > 0)
+    sha256_process_bytes (buffer, sum, &ctx);
+
+  /* Construct result in desired memory.  */
+  sha224_finish_ctx (&ctx, resblock);
+  return 0;
+}
+
+/* Compute SHA512 message digest for LEN bytes beginning at BUFFER.  The
+   result is always in little endian byte order, so that a byte-wise
+   output yields to the wanted ASCII representation of the message
+   digest.  */
+void *
+sha256_buffer (const char *buffer, size_t len, void *resblock)
+{
+  struct sha256_ctx ctx;
+
+  /* Initialize the computation context.  */
+  sha256_init_ctx (&ctx);
+
+  /* Process whole buffer but last len % 64 bytes.  */
+  sha256_process_bytes (buffer, len, &ctx);
+
+  /* Put result in desired memory area.  */
+  return sha256_finish_ctx (&ctx, resblock);
+}
+
+void *
+sha224_buffer (const char *buffer, size_t len, void *resblock)
+{
+  struct sha256_ctx ctx;
+
+  /* Initialize the computation context.  */
+  sha224_init_ctx (&ctx);
+
+  /* Process whole buffer but last len % 64 bytes.  */
+  sha256_process_bytes (buffer, len, &ctx);
+
+  /* Put result in desired memory area.  */
+  return sha224_finish_ctx (&ctx, resblock);
+}
+
+void
+sha256_process_bytes (const void *buffer, size_t len, struct sha256_ctx *ctx)
+{
+  /* When we already have some bits in our internal buffer concatenate
+     both inputs first.  */
+  if (ctx->buflen != 0)
+    {
+      size_t left_over = ctx->buflen;
+      size_t add = 128 - left_over > len ? len : 128 - left_over;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, add);
+      ctx->buflen += add;
+
+      if (ctx->buflen > 64)
+	{
+	  sha256_process_block (ctx->buffer, ctx->buflen & ~63, ctx);
+
+	  ctx->buflen &= 63;
+	  /* The regions in the following copy operation cannot overlap.  */
+	  memcpy (ctx->buffer,
+		  &((char *) ctx->buffer)[(left_over + add) & ~63],
+		  ctx->buflen);
+	}
+
+      buffer = (const char *) buffer + add;
+      len -= add;
+    }
+
+  /* Process available complete blocks.  */
+  if (len >= 64)
+    {
+#if !_STRING_ARCH_unaligned
+# define alignof(type) offsetof (struct { char c; type x; }, x)
+# define UNALIGNED_P(p) (((size_t) p) % alignof (uint32_t) != 0)
+      if (UNALIGNED_P (buffer))
+	while (len > 64)
+	  {
+	    sha256_process_block (memcpy (ctx->buffer, buffer, 64), 64, ctx);
+	    buffer = (const char *) buffer + 64;
+	    len -= 64;
+	  }
+      else
+#endif
+	{
+	  sha256_process_block (buffer, len & ~63, ctx);
+	  buffer = (const char *) buffer + (len & ~63);
+	  len &= 63;
+	}
+    }
+
+  /* Move remaining bytes in internal buffer.  */
+  if (len > 0)
+    {
+      size_t left_over = ctx->buflen;
+
+      memcpy (&((char *) ctx->buffer)[left_over], buffer, len);
+      left_over += len;
+      if (left_over >= 64)
+	{
+	  sha256_process_block (ctx->buffer, 64, ctx);
+	  left_over -= 64;
+	  memcpy (ctx->buffer, &ctx->buffer[16], left_over);
+	}
+      ctx->buflen = left_over;
+    }
+}
+
+/* --- Code below is the primary difference between sha1.c and sha256.c --- */
+
+/* SHA256 round constants */
+#define K(I) sha256_round_constants[I]
+static const uint32_t sha256_round_constants[64] = {
+  0x428a2f98UL, 0x71374491UL, 0xb5c0fbcfUL, 0xe9b5dba5UL,
+  0x3956c25bUL, 0x59f111f1UL, 0x923f82a4UL, 0xab1c5ed5UL,
+  0xd807aa98UL, 0x12835b01UL, 0x243185beUL, 0x550c7dc3UL,
+  0x72be5d74UL, 0x80deb1feUL, 0x9bdc06a7UL, 0xc19bf174UL,
+  0xe49b69c1UL, 0xefbe4786UL, 0x0fc19dc6UL, 0x240ca1ccUL,
+  0x2de92c6fUL, 0x4a7484aaUL, 0x5cb0a9dcUL, 0x76f988daUL,
+  0x983e5152UL, 0xa831c66dUL, 0xb00327c8UL, 0xbf597fc7UL,
+  0xc6e00bf3UL, 0xd5a79147UL, 0x06ca6351UL, 0x14292967UL,
+  0x27b70a85UL, 0x2e1b2138UL, 0x4d2c6dfcUL, 0x53380d13UL,
+  0x650a7354UL, 0x766a0abbUL, 0x81c2c92eUL, 0x92722c85UL,
+  0xa2bfe8a1UL, 0xa81a664bUL, 0xc24b8b70UL, 0xc76c51a3UL,
+  0xd192e819UL, 0xd6990624UL, 0xf40e3585UL, 0x106aa070UL,
+  0x19a4c116UL, 0x1e376c08UL, 0x2748774cUL, 0x34b0bcb5UL,
+  0x391c0cb3UL, 0x4ed8aa4aUL, 0x5b9cca4fUL, 0x682e6ff3UL,
+  0x748f82eeUL, 0x78a5636fUL, 0x84c87814UL, 0x8cc70208UL,
+  0x90befffaUL, 0xa4506cebUL, 0xbef9a3f7UL, 0xc67178f2UL,
+};
+
+/* Round functions.  */
+#define F2(A,B,C) ( ( A & B ) | ( C & ( A | B ) ) )
+#define F1(E,F,G) ( G ^ ( E & ( F ^ G ) ) )
+
+/* Process LEN bytes of BUFFER, accumulating context into CTX.
+   It is assumed that LEN % 64 == 0.
+   Most of this code comes from GnuPG's cipher/sha1.c.  */
+
+void
+sha256_process_block (const void *buffer, size_t len, struct sha256_ctx *ctx)
+{
+  const uint32_t *words = (const uint32_t *)buffer;
+  size_t nwords = len / sizeof (uint32_t);
+  const uint32_t *endp = words + nwords;
+  uint32_t x[16];
+  uint32_t a = ctx->state[0];
+  uint32_t b = ctx->state[1];
+  uint32_t c = ctx->state[2];
+  uint32_t d = ctx->state[3];
+  uint32_t e = ctx->state[4];
+  uint32_t f = ctx->state[5];
+  uint32_t g = ctx->state[6];
+  uint32_t h = ctx->state[7];
+
+  /* First increment the byte count.  FIPS PUB 180-2 specifies the possible
+     length of the file up to 2^64 bits.  Here we only compute the
+     number of bytes.  Do a double word increment.  */
+  ctx->total[0] += len;
+  if (ctx->total[0] < len)
+    ++ctx->total[1];
+
+#define rol(x, n) (((x) << (n)) | ((x) >> (32 - (n))))
+#define S0(x) (rol(x,25)^rol(x,14)^(x>>3))
+#define S1(x) (rol(x,15)^rol(x,13)^(x>>10))
+#define SS0(x) (rol(x,30)^rol(x,19)^rol(x,10))
+#define SS1(x) (rol(x,26)^rol(x,21)^rol(x,7))
+
+#define M(I) ( tm =   S1(x[(I-2)&0x0f]) + x[(I-7)&0x0f] \
+		    + S0(x[(I-15)&0x0f]) + x[I&0x0f]    \
+	       , x[I&0x0f] = tm )
+
+#define R(A,B,C,D,E,F,G,H,K,M)  do { t0 = SS0(A) + F2(A,B,C); \
+                                     t1 = H + SS1(E)  \
+                                      + F1(E,F,G)     \
+				      + K	      \
+				      + M;	      \
+				     D += t1;  H = t0 + t1; \
+			       } while(0)
+
+  while (words < endp)
+    {
+      uint32_t tm;
+      uint32_t t0, t1;
+      int t;
+      /* FIXME: see sha1.c for a better implementation.  */
+      for (t = 0; t < 16; t++)
+	{
+	  x[t] = SWAP (*words);
+	  words++;
+	}
+
+      R( a, b, c, d, e, f, g, h, K( 0), x[ 0] );
+      R( h, a, b, c, d, e, f, g, K( 1), x[ 1] );
+      R( g, h, a, b, c, d, e, f, K( 2), x[ 2] );
+      R( f, g, h, a, b, c, d, e, K( 3), x[ 3] );
+      R( e, f, g, h, a, b, c, d, K( 4), x[ 4] );
+      R( d, e, f, g, h, a, b, c, K( 5), x[ 5] );
+      R( c, d, e, f, g, h, a, b, K( 6), x[ 6] );
+      R( b, c, d, e, f, g, h, a, K( 7), x[ 7] );
+      R( a, b, c, d, e, f, g, h, K( 8), x[ 8] );
+      R( h, a, b, c, d, e, f, g, K( 9), x[ 9] );
+      R( g, h, a, b, c, d, e, f, K(10), x[10] );
+      R( f, g, h, a, b, c, d, e, K(11), x[11] );
+      R( e, f, g, h, a, b, c, d, K(12), x[12] );
+      R( d, e, f, g, h, a, b, c, K(13), x[13] );
+      R( c, d, e, f, g, h, a, b, K(14), x[14] );
+      R( b, c, d, e, f, g, h, a, K(15), x[15] );
+      R( a, b, c, d, e, f, g, h, K(16), M(16) );
+      R( h, a, b, c, d, e, f, g, K(17), M(17) );
+      R( g, h, a, b, c, d, e, f, K(18), M(18) );
+      R( f, g, h, a, b, c, d, e, K(19), M(19) );
+      R( e, f, g, h, a, b, c, d, K(20), M(20) );
+      R( d, e, f, g, h, a, b, c, K(21), M(21) );
+      R( c, d, e, f, g, h, a, b, K(22), M(22) );
+      R( b, c, d, e, f, g, h, a, K(23), M(23) );
+      R( a, b, c, d, e, f, g, h, K(24), M(24) );
+      R( h, a, b, c, d, e, f, g, K(25), M(25) );
+      R( g, h, a, b, c, d, e, f, K(26), M(26) );
+      R( f, g, h, a, b, c, d, e, K(27), M(27) );
+      R( e, f, g, h, a, b, c, d, K(28), M(28) );
+      R( d, e, f, g, h, a, b, c, K(29), M(29) );
+      R( c, d, e, f, g, h, a, b, K(30), M(30) );
+      R( b, c, d, e, f, g, h, a, K(31), M(31) );
+      R( a, b, c, d, e, f, g, h, K(32), M(32) );
+      R( h, a, b, c, d, e, f, g, K(33), M(33) );
+      R( g, h, a, b, c, d, e, f, K(34), M(34) );
+      R( f, g, h, a, b, c, d, e, K(35), M(35) );
+      R( e, f, g, h, a, b, c, d, K(36), M(36) );
+      R( d, e, f, g, h, a, b, c, K(37), M(37) );
+      R( c, d, e, f, g, h, a, b, K(38), M(38) );
+      R( b, c, d, e, f, g, h, a, K(39), M(39) );
+      R( a, b, c, d, e, f, g, h, K(40), M(40) );
+      R( h, a, b, c, d, e, f, g, K(41), M(41) );
+      R( g, h, a, b, c, d, e, f, K(42), M(42) );
+      R( f, g, h, a, b, c, d, e, K(43), M(43) );
+      R( e, f, g, h, a, b, c, d, K(44), M(44) );
+      R( d, e, f, g, h, a, b, c, K(45), M(45) );
+      R( c, d, e, f, g, h, a, b, K(46), M(46) );
+      R( b, c, d, e, f, g, h, a, K(47), M(47) );
+      R( a, b, c, d, e, f, g, h, K(48), M(48) );
+      R( h, a, b, c, d, e, f, g, K(49), M(49) );
+      R( g, h, a, b, c, d, e, f, K(50), M(50) );
+      R( f, g, h, a, b, c, d, e, K(51), M(51) );
+      R( e, f, g, h, a, b, c, d, K(52), M(52) );
+      R( d, e, f, g, h, a, b, c, K(53), M(53) );
+      R( c, d, e, f, g, h, a, b, K(54), M(54) );
+      R( b, c, d, e, f, g, h, a, K(55), M(55) );
+      R( a, b, c, d, e, f, g, h, K(56), M(56) );
+      R( h, a, b, c, d, e, f, g, K(57), M(57) );
+      R( g, h, a, b, c, d, e, f, K(58), M(58) );
+      R( f, g, h, a, b, c, d, e, K(59), M(59) );
+      R( e, f, g, h, a, b, c, d, K(60), M(60) );
+      R( d, e, f, g, h, a, b, c, K(61), M(61) );
+      R( c, d, e, f, g, h, a, b, K(62), M(62) );
+      R( b, c, d, e, f, g, h, a, K(63), M(63) );
+
+      a = ctx->state[0] += a;
+      b = ctx->state[1] += b;
+      c = ctx->state[2] += c;
+      d = ctx->state[3] += d;
+      e = ctx->state[4] += e;
+      f = ctx->state[5] += f;
+      g = ctx->state[6] += g;
+      h = ctx->state[7] += h;
+    }
+}
+#ifdef __cplusplus
+}
+#endif
+