You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tubemq.apache.org by gx...@apache.org on 2020/09/15 11:46:16 UTC

[incubator-tubemq] branch master updated (2348b77 -> cd5ba19)

This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git.


    from 2348b77  [TUBEMQ-345]Optimize the call logic of getMessage() in Pull mode (#261)
     new cf0ccc0  [TUBEMQ-250] Create C/C++ configure files (#170)
     new 6b112f6  [TUBEMQ-250] Create C/C++ configure files
     new 72b3f55  [TUBEMQ-250] Create C/C++ configure files
     new 94899dd  [TUBEMQ-250] Create C/C++ configure files
     new cef2ff6  [TUBEMQ-250] Create C/C++ configure files
     new 5c216c9  adjust code style
     new f077e02  [TUBEMQ-251] Create C/C++ Codec utils
     new 12f8412  [TUBEMQ-262]Create C++ flow control handler
     new 3cf5c56  [TUBEMQ-262]Create C++ flow control handler
     new d1e8b3c  [TUBEMQ-262]Create C++ flow control handler
     new d52d146  [TUBEMQ-262]Create C++ flow control handler
     new d171c37  [TUBEMQ-262]Create C++ flow control handler
     new 86b88f3  [TUBEMQ-267]Create C/C++ Message class (#182)
     new 28d5ec3  [TUBEMQ-266] Add Tencent/rapidjson as submodule
     new eee2db0  [TUBEMQ-263] Create C/C++ ini file read utils (#184)
     new ebf8d48  [TUBEMQ-263] Create C/C++ ini file read utils [addendum] (#185)
     new 318c8cc  [TUBEMQ-252] Create C/C++ Metadata classes (#186)
     new 7880069  [TUBEMQ-252] Create C/C++ Metadata classes (#187)
     new e9ce604  [TUBEMQ-268] C++ SDK log module (#190)
     new 881e4be  [TUBEMQ-251] Create C/C++ Codec utils (#191)
     new e6df8da  [TUBEMQ-262] Create C++ flow control handler (#192)
     new 87ab59a  [TUBEMQ-272]Unified C/C++ files's code style (#193)
     new f4356b3  [TUBEMQ-272]Unified C/C++ files's code style (#194)
     new d7cebd5  [TUBEMQ-269] Create C/C++ RmtDataCache class (#195)
     new 38844d0  [TUBEMQ-272]Unified C/C++ files's code style (#196)
     new 82e02a8  [TUBEMQ-274]Support CMake compilation (#197)
     new 6a42851  [TUBEMQ-269] Create C/C++ RmtDataCache class (#198)
     new 3be5ddc  [TUBEMQ-269]Create C/C++ RmtDataCache class (#199)
     new 37055d1  [TUBEMQ-269]Create C/C++ RmtDataCache class (#202)
     new 325073c  [TUBEMQ-269]Create C/C++ RmtDataCache class (#204)
     new 30e6fcf  [TUBEMQ-275]Thread Pool & Timer (#205)
     new f88d38a  [TUBEMQ-280]Create C/C++ subscribe info class (#206)
     new 48649c7  [TUBEMQ-282]Create C/C++ return result class (#207)
     new cd4b647  [TUBEMQ-283]Adjust C/C++ some file names: add "tubemq_" prefix (#208)
     new 2016990  [TUBEMQ-285]Replace C/C++ pthread's mutex to std::mutex (#210)
     new a9a6e07  [TUBEMQ-286]Create C/C++ SDK's manager class (#211)
     new 1fb720f  [TUBEMQ-286]Create C/C++ SDK's manager class (#212)
     new 2922467  [TUBEMQ-286]Create C/C++ SDK's manager class (#213)
     new d43d8e9  [TUBEMQ-281]atomic_def.h use C++11 stdlib class (#216)
     new 2d6b66b  [TUBEMQ-292]C++ SDK singleton & executor_pool optimization (#217)
     new ad04f00  [TUBEMQ-290]C++ SDK TCP Connect (#252)
     new 9929b10  TUBEMQ-287 add io buffer (#253)
     new 09d2c9b  [TUBEMQ-288]C++ SDK Codec interface (#254)
     new de6fb61  [TUBEMQ-289]C++ SDK Codec TubeMQ proto support (#255)
     new 8d20dc3  [TUBEMQ-293]C++ SDK Create Future class (#257)
     new 99c9630  [TUBEMQ-347]C++ SDK Create client API (#258)
     new 8ff63dd  [TUBEMQ-349]C++ SDK Create Thread Pool (#259)
     new de4960d  [TUBEMQ-348]C++SDK Client handler detail (#260)
     new 1cc56e5  [TUBEMQ-350]C++ SDK client code adj (#262)
     new cd5ba19  [TUBEMQ-351]C++ SDK example&tests (#263)

The 50 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .gitignore                                         |    1 +
 .gitmodules                                        |    4 +
 pom.xml                                            |    2 +
 .../tubemq-client-cpp/.clang-format                |   25 +-
 .../tubemq-client-cpp/CMakeLists.txt               |   42 +-
 tubemq-client-twins/tubemq-client-cpp/README.md    |   37 +
 .../tubemq-client-cpp/build_linux.sh               |   20 +-
 .../tubemq-client-cpp/example/CMakeLists.txt       |   27 +-
 .../tubemq-client-cpp/example/README.md            |   22 +
 .../example/consumer/CMakeLists.txt                |   16 +-
 .../example/consumer/test_consumer.cc              |  137 ++
 .../example/consumer/test_multithread_pull.cc      |  166 ++
 .../include/tubemq/tubemq_atomic.h                 |   96 ++
 .../include/tubemq/tubemq_client.h                 |   59 +
 .../include/tubemq/tubemq_config.h                 |  158 ++
 .../include/tubemq/tubemq_errcode.h                |  117 ++
 .../include/tubemq/tubemq_message.h                |   75 +
 .../include/tubemq/tubemq_return.h                 |  113 ++
 .../tubemq-client-cpp}/proto/BrokerService.proto   |    6 -
 .../tubemq-client-cpp/proto/CMakeLists.txt         |   20 +-
 .../tubemq-client-cpp/proto/MasterService.proto    |  192 +++
 .../tubemq-client-cpp}/proto/RPC.proto             |    0
 .../tubemq-client-cpp/proto/readme.md              |   43 +-
 .../tubemq-client-cpp/src/CMakeLists.txt           |   23 +-
 tubemq-client-twins/tubemq-client-cpp/src/any.h    |  160 ++
 .../tubemq-client-cpp/src/baseconsumer.cc          | 1613 ++++++++++++++++++++
 .../tubemq-client-cpp/src/baseconsumer.h           |  146 ++
 tubemq-client-twins/tubemq-client-cpp/src/buffer.h |  356 +++++
 .../tubemq-client-cpp/src/client_connection.cc     |  303 ++++
 .../tubemq-client-cpp/src/client_connection.h      |  101 ++
 .../tubemq-client-cpp/src/client_service.cc        |  296 ++++
 .../tubemq-client-cpp/src/client_service.h         |  121 ++
 .../tubemq-client-cpp/src/client_subinfo.cc        |  133 ++
 .../tubemq-client-cpp/src/client_subinfo.h         |   79 +
 .../tubemq-client-cpp/src/codec_protocol.h         |   58 +
 .../tubemq-client-cpp/src/connection.cc            |   25 +
 .../tubemq-client-cpp/src/connection.h             |  120 ++
 .../tubemq-client-cpp/src/connection_pool.h        |  131 ++
 .../tubemq-client-cpp/src/const_config.h           |  120 ++
 .../tubemq-client-cpp/src/const_rpc.h              |   91 ++
 .../tubemq-client-cpp/src/executor_pool.cc         |   88 ++
 .../tubemq-client-cpp/src/executor_pool.h          |   95 ++
 .../tubemq-client-cpp/src/file_ini.cc              |  154 ++
 .../tubemq-client-cpp/src/file_ini.h               |   51 +
 .../tubemq-client-cpp/src/flowctrl_def.cc          |  785 ++++++++++
 .../tubemq-client-cpp/src/flowctrl_def.h           |  144 ++
 tubemq-client-twins/tubemq-client-cpp/src/future.h |  143 ++
 .../tubemq-client-cpp/src/logger.cc                |   80 +
 tubemq-client-twins/tubemq-client-cpp/src/logger.h |  118 ++
 .../tubemq-client-cpp/src/meta_info.cc             |  580 +++++++
 .../tubemq-client-cpp/src/meta_info.h              |  188 +++
 .../tubemq-client-cpp/src/noncopyable.h            |   37 +
 .../tubemq-client-cpp/src/rmt_data_cache.cc        |  669 ++++++++
 .../tubemq-client-cpp/src/rmt_data_cache.h         |  166 ++
 .../tubemq-client-cpp/src/singleton.h              |   62 +
 .../tubemq-client-cpp/src/thread_pool.h            |   69 +
 .../tubemq-client-cpp/src/transport.h              |   60 +
 .../tubemq-client-cpp/src/tubemq_client.cc         |  184 +++
 .../tubemq-client-cpp/src/tubemq_codec.h           |  312 ++++
 .../tubemq-client-cpp/src/tubemq_config.cc         |  612 ++++++++
 .../tubemq-client-cpp/src/tubemq_message.cc        |  224 +++
 .../tubemq-client-cpp/src/tubemq_return.cc         |  180 +++
 .../tubemq-client-cpp/src/tubemq_transport.h       |   58 +
 .../tubemq-client-cpp/src/unique_seq_id.h          |   41 +
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  559 +++++++
 tubemq-client-twins/tubemq-client-cpp/src/utils.h  |   71 +
 .../tubemq-client-cpp/src/version.h                |   33 +
 .../tubemq-client-cpp/tests/CMakeLists.txt         |   28 +-
 .../tubemq-client-cpp/tests/README.md              |   22 +
 .../tests/executor_pool/CMakeLists.txt             |   15 +-
 .../tubemq-client-cpp/tests/executor_pool/main.cc  |   58 +
 .../tubemq-client-cpp/tests/log/CMakeLists.txt     |   15 +-
 .../tubemq-client-cpp/tests/log/main.cc            |   62 +
 .../tests/thread_pool/CMakeLists.txt               |   15 +-
 .../tubemq-client-cpp/tests/thread_pool/main.cc    |   36 +
 .../tubemq-client-cpp/third_party/CMakeLists.txt   |   45 +
 .../tubemq-client-cpp/third_party/README.md        |   23 +
 77 files changed, 11148 insertions(+), 188 deletions(-)
 create mode 100644 .gitmodules
 copy tubemq-docker/tubemq-k8s/templates/_master.tpl => tubemq-client-twins/tubemq-client-cpp/.clang-format (69%)
 copy tubemq-docker/tubemq-k8s/templates/_master.tpl => tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt (54%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/README.md
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/build_linux.sh (69%)
 mode change 100644 => 100755
 copy tubemq-docker/tubemq-k8s/templates/_master.tpl => tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt (60%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/example/README.md
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/example/consumer/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/example/consumer/test_consumer.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/example/consumer/test_multithread_pull.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_atomic.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_client.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
 copy {tubemq-core/src/main => tubemq-client-twins/tubemq-client-cpp}/proto/BrokerService.proto (95%)
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/proto/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/proto/MasterService.proto
 copy {tubemq-core/src/main => tubemq-client-twins/tubemq-client-cpp}/proto/RPC.proto (100%)
 copy tubemq-core/src/main/java/org/apache/tubemq/corebase/Shutdownable.java => tubemq-client-twins/tubemq-client-cpp/proto/readme.md (72%)
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/any.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/buffer.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_service.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/codec_protocol.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/connection.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/connection.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/const_config.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/const_rpc.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/executor_pool.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/file_ini.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/future.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/logger.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/logger.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/meta_info.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/noncopyable.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/singleton.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/transport.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_client.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/tubemq_transport.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/unique_seq_id.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/utils.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/utils.h
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/src/version.h
 copy tubemq-docker/tubemq-k8s/templates/_master.tpl => tubemq-client-twins/tubemq-client-cpp/tests/CMakeLists.txt (57%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/tests/README.md
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/main.cc
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/tests/log/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/tests/log/main.cc
 copy tubemq-docker/tubemq-k8s/templates/_broker.tpl => tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/CMakeLists.txt (69%)
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/main.cc
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
 create mode 100644 tubemq-client-twins/tubemq-client-cpp/third_party/README.md


[incubator-tubemq] 33/50: [TUBEMQ-282]Create C/C++ return result class (#207)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 48649c7f326ce9842bc78520c9e95c8c89c5af9b
Author: gosonzhang <46...@qq.com>
AuthorDate: Thu Jul 16 06:08:42 2020 +0000

    [TUBEMQ-282]Create C/C++ return result class (#207)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/const_config.h                  |  22 ++--
 .../include/tubemq/executor_pool.h                 |   3 +-
 .../include/tubemq/rmt_data_cache.h                |   3 +-
 .../include/tubemq/tubemq_return.h                 |  95 ++++++++++++++
 .../tubemq-client-cpp/include/tubemq/version.h     |   2 +-
 .../tubemq-client-cpp/src/client_subinfo.cc        |   4 +-
 .../tubemq-client-cpp/src/file_ini.cc              |  14 +-
 .../tubemq-client-cpp/src/message.cc               |   4 +-
 .../tubemq-client-cpp/src/meta_info.cc             |   2 +-
 .../tubemq-client-cpp/src/rmt_data_cache.cc        |  19 ++-
 .../tubemq-client-cpp/src/tubemq_return.cc         | 143 +++++++++++++++++++++
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |   2 +-
 12 files changed, 276 insertions(+), 37 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
index bc90194..d843929 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
@@ -76,20 +76,20 @@ static const int32_t kInvalidValue = -2;
 }  // namespace tb_config
 
 namespace delimiter {
-static const string kDelimiterDot = ".";
-static const string kDelimiterEqual = "=";
-static const string kDelimiterAnd = "&";
-static const string kDelimiterComma = ",";
-static const string kDelimiterColon = ":";
-static const string kDelimiterAt = "@";
-static const string kDelimiterPound = "#";
-static const string kDelimiterSemicolon = ";";
+static const char kDelimiterDot[] = ".";
+static const char kDelimiterEqual[] = "=";
+static const char kDelimiterAnd[] = "&";
+static const char kDelimiterComma[] = ",";
+static const char kDelimiterColon[] = ":";
+static const char kDelimiterAt[] = "@";
+static const char kDelimiterPound[] = "#";
+static const char kDelimiterSemicolon[] = ";";
 // Double slash
-static const string kDelimiterDbSlash = "//";
+static const char kDelimiterDbSlash[] = "//";
 // left square bracket
-static const string kDelimiterLftSB = "[";
+static const char kDelimiterLftSB[] = "[";
 // right square bracket
-static const string kDelimiterRgtSB = "]";
+static const char kDelimiterRgtSB[] = "]";
 
 }  // namespace delimiter
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
index a5208a4..404616b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
@@ -85,6 +85,7 @@ class ExecutorPool : noncopyable {
 };
 
 typedef std::shared_ptr<ExecutorPool> ExecutorPoolPtr;
+
 }  // namespace tubemq
 
-#endif  //_TUBEMQ_EXECUTOR_POOL_
+#endif  // _TUBEMQ_EXECUTOR_POOL_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index 2b79f13..aa8f250 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -48,7 +48,7 @@ class RmtDataCacheCsm {
  public:
   RmtDataCacheCsm(const string& client_id, const string& group_name);
   ~RmtDataCacheCsm();
-  void UpdateDefFlowCtrlInfo(int64_t flowctrl_id, 
+  void UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
                                      const string& flowctrl_info);
   void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
                  int64_t flowctrl_id, const string& flowctrl_info);
@@ -109,7 +109,6 @@ class RmtDataCacheCsm {
   FlowCtrlRuleHandler def_flowctrl_handler_;
   AtomicBoolean under_groupctrl_;
   AtomicLong last_checktime_;
-
   // meta info
   pthread_rwlock_t meta_rw_lock_;
   // partiton allocated map
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
new file mode 100644
index 0000000..96194b4
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.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 TUBEMQ_CLIENT_RETURN_H_
+#define TUBEMQ_CLIENT_RETURN_H_
+
+#include <stdlib.h>
+
+#include <list>
+#include <string>
+
+#include "tubemq/message.h"
+#include "tubemq/meta_info.h"
+
+
+
+
+namespace tubemq {
+
+using std::string;
+
+
+class PeerInfo {
+ public:
+  PeerInfo();
+  PeerInfo(const Partition& partition, int64_t offset);
+  PeerInfo& operator=(const PeerInfo& target);
+  void SetMsgSourceInfo(const Partition& partition, int64_t offset);
+  const uint32_t GetPartitionId() const { return partition_id_; }
+  const string& GetBrokerHost() const { return broker_host_; }
+  const string& GetPartitionKey() const { return partition_key_; }
+  const int64_t GetCurrOffset() const { return curr_offset_; }
+
+ private:
+  uint32_t partition_id_;
+  string broker_host_;
+  string partition_key_;
+  int64_t curr_offset_;
+};
+
+
+
+class ConsumerResult {
+ public:
+  ConsumerResult();
+  ConsumerResult(const ConsumerResult& target);
+  ConsumerResult(int32_t err_code, string err_msg);
+  ~ConsumerResult();
+  ConsumerResult& operator=(const ConsumerResult& target);
+  void SetFailureResult(int32_t err_code, string err_msg);
+  void SetFailureResult(int32_t err_code, string err_msg,
+              const string& topic_name, const PeerInfo& peer_info);
+  void SetSuccessResult(int32_t err_code, const string& topic_name,
+                  const PeerInfo& peer_info, const string& confirm_context,
+                  const list<Message>& message_list);
+  bool IsSuccess() { return success_; }
+  const int32_t  GetErrCode() const { return err_code_; }
+  const string& GetErrMessage() const { return err_msg_; }
+  const string& GetTopicName() const { return topic_name_; }
+  const PeerInfo& GetPeerInfo() const { return peer_info_; }
+  const string& GetConfirmContext() const { return confirm_context_; }
+  const list<Message>& GetMessageList() const { return message_list_; }
+  const string& GetPartitionKey() const;
+  const int64_t GetCurrOffset() const;
+
+ private:
+  bool success_;
+  int32_t  err_code_;
+  string err_msg_;
+  string topic_name_;
+  PeerInfo peer_info_;
+  string confirm_context_;
+  list<Message> message_list_;
+};
+
+}
+
+#endif  // TUBEMQ_CLIENT_RETURN_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
index 4aede43..c479ede 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
@@ -26,7 +26,7 @@ namespace tubemq {
 
 using std::string;
 
-static const string kTubeMQClientVersion = "0.1.0-0.5.0";
+static const char kTubeMQClientVersion[] = "0.1.0-0.5.0";
 
 }  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
index 0c8064c..9e959e6 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
@@ -62,7 +62,7 @@ void ClientSubInfo::SetConsumeTarget(bool bound_consume,
     } else {
       topic_filter_map_[it_topic->first] = true;
 
-      //build topic conditions
+      // build topic conditions
       count = 0;
       tmpstr = it_topic->first;
       tmpstr += delimiter::kDelimiterPound;
@@ -77,7 +77,7 @@ void ClientSubInfo::SetConsumeTarget(bool bound_consume,
     }
   }
 
-  //build bound_partition info
+  // build bound_partition info
   if (bound_consume) {
     session_key_ = session_key;
     source_count_ = source_count;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 346dc10..b073cde 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -59,6 +59,9 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   string sector = "";
   string key = "";
   string value = "";
+  string lftsb = delimiter::kDelimiterLftSB;
+  string rgtsb = delimiter::kDelimiterRgtSB;
+  string equal = delimiter::kDelimiterEqual;
   string::size_type lftsb_pos = 0;
   string::size_type rgtsb_pos = 0;
   string::size_type equal_pos = 0;
@@ -71,21 +74,20 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
       continue;
     }
     // check if a sector head
-    lftsb_pos = line_str.find(delimiter::kDelimiterLftSB);
-    rgtsb_pos = line_str.find(delimiter::kDelimiterRgtSB);
+    lftsb_pos = line_str.find(lftsb);
+    rgtsb_pos = line_str.find(rgtsb);
     if (lftsb_pos != string::npos && rgtsb_pos != string::npos) {
-      sector = line_str.substr(lftsb_pos + (delimiter::kDelimiterLftSB).size(),
-                               rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
+      sector = line_str.substr(lftsb_pos + lftsb.size(), rgtsb_pos - rgtsb.size());
       sector = Utils::Trim(sector);
       continue;
     }
     // check if a key=value string
-    equal_pos = line_str.find(delimiter::kDelimiterEqual);
+    equal_pos = line_str.find(equal);
     if (equal_pos == string::npos) {
       continue;
     }
     key = line_str.substr(0, equal_pos);
-    value = line_str.substr(equal_pos + (delimiter::kDelimiterEqual).size(), line_str.size());
+    value = line_str.substr(equal_pos + equal.size(), line_str.size());
     key = Utils::Trim(key);
     value = Utils::Trim(value);
     // get data from file to memory
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index 9d117cd..93e3dd2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -34,9 +34,9 @@ using std::stringstream;
 // message flag's properties settings
 static const int32_t kMsgFlagIncProperties = 0x01;
 // reserved property key Filter Item
-static const string kRsvPropKeyFilterItem = "$msgType$";
+static const char kRsvPropKeyFilterItem[] = "$msgType$";
 // reserved property key message send time
-static const string kRsvPropKeyMsgTime = "$msgTime$";
+static const char kRsvPropKeyMsgTime[] = "$msgTime$";
 
 Message::Message() {
   this->topic_ = "";
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 7694c5b..ae7d72e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -265,7 +265,7 @@ PartitionExt& PartitionExt::operator=(const PartitionExt& target) {
   if (this != &target) {
     // parent class
     Partition::operator=(target);
-    // child class    
+    // child class
     this->is_last_consumed_ = target.is_last_consumed_;
     this->cur_flowctrl_ = target.cur_flowctrl_;
     this->cur_freqctrl_ = target.cur_freqctrl_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index 2164fa9..ec4876b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -19,9 +19,8 @@
 
 #include "tubemq/rmt_data_cache.h"
 
-#include <string>
-
 #include <stdlib.h>
+#include <string>
 
 #include "tubemq/const_config.h"
 #include "tubemq/meta_info.h"
@@ -57,24 +56,24 @@ RmtDataCacheCsm::~RmtDataCacheCsm() {
   pthread_rwlock_destroy(&meta_rw_lock_);
 }
 
+
 void RmtDataCacheCsm::UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
                                                  const string& flowctrl_info) {
   if (flowctrl_id != def_flowctrl_handler_.GetFlowCtrlId()) {
-    def_flowctrl_handler_.UpdateDefFlowCtrlInfo(true, 
+    def_flowctrl_handler_.UpdateDefFlowCtrlInfo(true,
       tb_config::kInvalidValue, flowctrl_id, flowctrl_info);
   }
 }
 void RmtDataCacheCsm::UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
                              int64_t flowctrl_id, const string& flowctrl_info) {
   if (flowctrl_id != group_flowctrl_handler_.GetFlowCtrlId()) {
-    group_flowctrl_handler_.UpdateDefFlowCtrlInfo(false, 
+    group_flowctrl_handler_.UpdateDefFlowCtrlInfo(false,
                 qyrpriority_id, flowctrl_id, flowctrl_info);
   }
   if (qyrpriority_id != group_flowctrl_handler_.GetQryPriorityId()) {
     this->group_flowctrl_handler_.SetQryPriorityId(qyrpriority_id);
-
   }
-  // update current if under group flowctrl 
+  // update current if under group flowctrl
   int64_t cur_time = Utils::GetCurrentTimeMillis();
   if (cur_time - last_checktime_.Get() > 10000) {
     FlowCtrlResult flowctrl_result;
@@ -176,9 +175,9 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
   return result;
 }
 
-void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, 
+void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key,
                      int64_t curr_offset, int32_t err_code, bool esc_limit,
-                     int32_t msg_size,int64_t limit_dlt, int64_t cur_data_dlt,
+                  int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt,
                      bool require_slow) {
   map<string, PartitionExt>::iterator it_part;
   // book partition offset info
@@ -287,7 +286,7 @@ void RmtDataCacheCsm::GetAllBrokerPartitions(
 bool RmtDataCacheCsm::GetPartitionExt(const string& part_key, PartitionExt& partition_ext) {
   bool result = false;
   map<string, PartitionExt>::iterator it_map;
-  
+
   pthread_rwlock_rdlock(&meta_rw_lock_);
   it_map = partitions_.find(part_key);
   if (it_map != partitions_.end()) {
@@ -314,7 +313,7 @@ void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
   set<string>::iterator it_key;
   map<NodeInfo, set<string> >::iterator it_broker;
   map<string, PartitionExt>::iterator it_part;
-  
+
   partition_list.clear();
   pthread_rwlock_rdlock(&meta_rw_lock_);
   it_broker = broker_partition_.find(broker_info);
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
new file mode 100644
index 0000000..5298606
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
@@ -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.
+ */
+
+#include "tubemq/tubemq_return.h"
+#include "tubemq/const_config.h"
+
+
+
+namespace tubemq {
+
+
+PeerInfo::PeerInfo() {
+  broker_host_ = "";
+  partition_key_ = "";
+  partition_id_ = 0;
+  curr_offset_ = tb_config::kInvalidValue;
+}
+
+PeerInfo::PeerInfo(const Partition& partition, int64_t offset) {
+  SetMsgSourceInfo(partition, offset);
+}
+
+PeerInfo& PeerInfo::operator=(const PeerInfo& target) {
+  if (this != &target) {
+    this->partition_id_ = target.partition_id_;
+    this->broker_host_ = target.broker_host_;
+    this->partition_key_ = target.partition_key_;
+    this->curr_offset_ = target.curr_offset_;
+  }
+  return *this;
+}
+
+void PeerInfo::SetMsgSourceInfo(const Partition& partition, int64_t offset) {
+  partition_id_ = partition.GetPartitionId();
+  broker_host_ = partition.GetBrokerHost();
+  partition_key_ = partition.GetPartitionKey();
+  curr_offset_ = offset;
+}
+
+ConsumerResult::ConsumerResult() {
+  success_ = false;
+  err_code_ = tb_config::kInvalidValue;
+  err_msg_ = "";
+  topic_name_ = "";
+  confirm_context_ = "";
+}
+
+ConsumerResult::ConsumerResult(const ConsumerResult& target) {
+  this->success_ = target.success_;
+  this->err_code_ = target.err_code_;
+  this->err_msg_ = target.err_msg_;
+  this->topic_name_ = target.topic_name_;
+  this->peer_info_ = target.peer_info_;
+  this->confirm_context_ = target.confirm_context_;
+  this->message_list_ = target.message_list_;
+}
+
+ConsumerResult::ConsumerResult(int32_t err_code, string err_msg) {
+  success_ = false;
+  err_code_ = err_code;
+  err_msg_ = err_msg;
+  topic_name_ = "";
+  confirm_context_ = "";
+}
+
+ConsumerResult::~ConsumerResult() {
+  this->message_list_.clear();
+  success_ = false;
+  err_code_ = tb_config::kInvalidValue;
+  err_msg_ = "";
+  topic_name_ = "";
+  confirm_context_ = "";
+}
+
+ConsumerResult& ConsumerResult::operator=(const ConsumerResult& target) {
+  if (this != &target) {
+    this->success_ = target.success_;
+    this->err_code_ = target.err_code_;
+    this->err_msg_ = target.err_msg_;
+    this->topic_name_ = target.topic_name_;
+    this->peer_info_ = target.peer_info_;
+    this->confirm_context_ = target.confirm_context_;
+    this->message_list_ = target.message_list_;
+  }
+  return *this;
+}
+
+void ConsumerResult::SetFailureResult(int32_t err_code, string err_msg) {
+  success_ = false;
+  err_code_ = err_code;
+  err_msg_ = err_msg;
+}
+
+void ConsumerResult::SetFailureResult(int32_t err_code, string err_msg,
+                            const string& topic_name, const PeerInfo& peer_info) {
+  success_ = false;
+  err_code_ = err_code;
+  err_msg_ = err_msg;
+  topic_name_ = topic_name;
+  peer_info_ = peer_info;
+}
+
+void ConsumerResult::SetSuccessResult(int32_t err_code,
+                                             const string& topic_name,
+                                             const PeerInfo& peer_info,
+                                             const string& confirm_context,
+                                             const list<Message>& message_list) {
+  this->success_ = true;
+  this->err_code_ = err_code;
+  this->err_msg_ = "Ok";
+  this->topic_name_ = topic_name;
+  this->peer_info_ = peer_info;
+  this->confirm_context_ = confirm_context;
+  this->message_list_ = message_list;
+}
+
+const string& ConsumerResult::GetPartitionKey() const {
+  return this->peer_info_.GetPartitionKey();
+}
+
+const int64_t ConsumerResult::GetCurrOffset() const {
+  return this->peer_info_.GetCurrOffset();
+}
+
+
+}  // namespace tubemq
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index e48c8b6..40d1de5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -34,7 +34,7 @@ namespace tubemq {
 using std::stringstream;
 
 
-static const string kWhitespaceCharSet = " \n\r\t\f\v";
+static const char kWhitespaceCharSet[] = " \n\r\t\f\v";
 
 string Utils::Trim(const string& source) {
   string target = source;


[incubator-tubemq] 03/50: [TUBEMQ-250] Create C/C++ configure files

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 72b3f558b6d600e6362229a461b89a44ae162737
Author: gosonzhang <go...@tencent.com>
AuthorDate: Tue Jun 30 21:13:57 2020 +0800

    [TUBEMQ-250] Create C/C++ configure files
---
 .../tubemq-client-cpp/inc/client_config.h          | 135 +++++++------
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  |  48 ++---
 .../tubemq-client-cpp/inc/version.h                |   6 +-
 .../tubemq-client-cpp/src/client_config.cc         | 224 ++++++++++-----------
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  67 +++---
 5 files changed, 240 insertions(+), 240 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
index 9d3afac..71f2c0e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
@@ -26,80 +26,81 @@
 
 
 
-namespace TubeMQ {
-
-  using namespace std;
-
-  // configuration value setting
-  namespace config {
-  
-    // rpc timeout define  
-    static const int kRpcTimoutDef = 15;
-    static const int kRpcTimoutMax = 300;
-    static const int kRpcTimoutMin = 8;
-    // heartbeat period define
-    static const int kHeartBeatPeriodDef = 10;
-    static const int kHeartBeatFailRetryTimesDef = 5;
-    static const int kHeartBeatSleepPeriodDef = 60;
-    // max masterAddrInfo length
-    static const int kMasterAddrInfoMaxLength = 1024;
-    // max TopicName length
-    static const int kTopicNameMaxLength = 64;
-    // max Consume GroupName length
-    static const int kGroupNameMaxLength = 1024;
-  }  // namespace config
-
-  class BaseConfig {
-   public:
-    BaseConfig();
-    ~BaseConfig();
-    BaseConfig& operator=(const BaseConfig& target);
-    bool SetMasterAddrInfo(string& errInfo, const string& masterAddrInfo);
-    bool SetTlsInfo(string& errInfo, bool tlsEnable, 
-                        const string& trustStorePath, const string& trustStorePassword);
-    bool SetAuthenticInfo(string& errInfo, bool needAuthentic, 
-                                const string& usrName, const string& usrPassWord);
-    const string& GetMasterAddrInfo() const;
-    bool IsTlsEnabled();
-    const string& GetTrustStorePath() const;
-    const string& GetTrustStorePassword() const;
-    bool IsAuthenticEnabled();
-    const string& GetUsrName() const;
-    const string& GetUsrPassWord() const;            
-    // set the rpc timout, unit second, duration [8, 300], default 15 seconds;
-    void SetRpcReadTimeoutSec(int rpcReadTimeoutSec);
-    int GetRpcReadTimeoutSec();
-    // Set the duration of the client's heartbeat cycle, in seconds, the default is 10 seconds
-    void SetHeartbeatPeriodSec(int heartbeatPeriodInSec);
-    int GetHeartbeatPeriodSec();
-    void SetMaxHeartBeatRetryTimes(int maxHeartBeatRetryTimes);
-    int GetMaxHeartBeatRetryTimes();
-    void SetHeartbeatPeriodAftFailSec(int heartbeatPeriodSecAfterFailSec);
-    int GetHeartbeatPeriodAftFailSec();
-    string ToString();
-  
+namespace tubemq {
+
+using namespace std;
+
+
+// configuration value setting
+namespace config {
+// rpc timeout define  
+static const int kRpcTimoutDef = 15;
+static const int kRpcTimoutMax = 300;
+static const int kRpcTimoutMin = 8;
+// heartbeat period define
+static const int kHeartBeatPeriodDef = 10;
+static const int kHeartBeatFailRetryTimesDef = 5;
+static const int kHeartBeatSleepPeriodDef = 60;
+// max masterAddrInfo length
+static const int kMasterAddrInfoMaxLength = 1024;
+// max TopicName length
+static const int kTopicNameMaxLength = 64;
+// max Consume GroupName length
+static const int kGroupNameMaxLength = 1024;
+}  // namespace config
+
+
+class BaseConfig {
+ public:
+  BaseConfig();
+  ~BaseConfig();
+  BaseConfig& operator=(const BaseConfig& target);
+  bool SetMasterAddrInfo(string& err_info, const string& master_addrinfo);
+  bool SetTlsInfo(string& err_info, bool tls_enable, 
+                    const string& trust_store_path, const string& trust_store_password);
+  bool SetAuthenticInfo(string& err_info, bool authentic_enable, 
+                            const string& usr_name, const string& usr_password);
+  const string& GetMasterAddrInfo() const;
+  bool IsTlsEnabled();
+  const string& GetTrustStorePath() const;
+  const string& GetTrustStorePassword() const;
+  bool IsAuthenticEnabled();
+  const string& GetUsrName() const;
+  const string& GetUsrPassWord() const;            
+  // set the rpc timout, unit second, duration [8, 300], default 15 seconds;
+  void SetRpcReadTimeoutSec(int rpc_read_timeout_sec);
+  int GetRpcReadTimeoutSec();
+  // Set the duration of the client's heartbeat cycle, in seconds, the default is 10 seconds
+  void SetHeartbeatPeriodSec(int heartbeat_period_sec);
+  int GetHeartbeatPeriodSec();
+  void SetMaxHeartBeatRetryTimes(int max_heartbeat_retry_times);
+  int GetMaxHeartBeatRetryTimes();
+  void SetHeartbeatPeriodAftFailSec(int heartbeat_period_afterfail_sec);
+  int GetHeartbeatPeriodAftFailSec();
+  string ToString();
+
  private:
-  string masterAddrStr_;
+  string master_addrinfo_;
   // user authenticate
-  bool   authEnable_;
-  string authUsrName_;
-  string authUsrPassWord_;
+  bool   auth_enable_;
+  string auth_usrname_;
+  string auth_usrpassword_;
   // TLS configuration
-  bool   tlsEnabled_;
-  string tlsTrustStorePath_;
-  string tlsTrustStorePassword_;
+  bool   tls_enabled_;
+  string tls_trust_store_path_;
+  string tls_trust_store_password_;
   // other setting
-  int   rpcReadTimeoutSec_;
-  int   heartbeatPeriodSec_;
-  int    maxHeartBeatRetryTimes_;
-  int   heartbeatPeriodAfterFailSec_;
+  int   rpc_read_timeout_sec_;
+  int   heartbeat_period_sec_;
+  int   max_heartbeat_retry_times_;
+  int   heartbeat_period_afterfail_sec_;
 };
 
 
-  class ConsumerConfig {
-   public:
-     ConsumerConfig();
-  };
+class ConsumerConfig {
+ public:
+  ConsumerConfig();
+};
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index d15df37..937eb37 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -23,30 +23,30 @@
 #include <map>
 #include <string>
 
-namespace TubeMQ {
-
-  using namespace std;
-
-  static const string tWhitespaceCharSet = " \n\r\t\f\v";
-
-  namespace delimiter {
-    static const string tDelimiterEqual = "=";
-    static const string tDelimiterAnd   = "&";
-    static const string tDelimiterComma = ",";
-    static const string tDelimiterColon = ":";
-    static const string tDelimiterAt    = "@";
-    static const string tDelimiterPound = "#";
-  }
-
-  class Utils {
-   public:
-    // trim string info
-    static string trim(const string& source);
-    // split string to vector
-    static void split(const string& source, map<string, int>& result, 
-                     const string& delimiterStep1, const string& delimiterStep2);
-
-  };
+namespace tubemq {
+
+using namespace std;
+
+static const string tWhitespaceCharSet = " \n\r\t\f\v";
+
+namespace delimiter {
+  static const string tDelimiterEqual = "=";
+  static const string tDelimiterAnd   = "&";
+  static const string tDelimiterComma = ",";
+  static const string tDelimiterColon = ":";
+  static const string tDelimiterAt    = "@";
+  static const string tDelimiterPound = "#";
+}
+
+class Utils {
+ public:
+  // trim string info
+  static string trim(const string& source);
+  // split string to vector
+  static void split(const string& source, map<string, int>& result, 
+                   const string& delimiter_step1, const string& delimiter_step2);
+
+};
  
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/version.h b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
index 1fc1bda..cdf1444 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/version.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
@@ -22,11 +22,11 @@
 
 #include <string>
 
-namespace TubeMQ {
+namespace tubemq {
 
-  using namespace std;
+using namespace std;
 
-  static const String tTubeMQClientVersion = "0.5.0";
+static const String tTubeMQClientVersion = "0.5.0";
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index bc11183..2f540fa 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -22,21 +22,21 @@
 #include "utils.h"
 
 
-namespace TubeMQ {
+namespace tubemq {
 
 
 BaseConfig::BaseConfig() {
-  this->masterAddrStr_ = "";
-  this->authEnable_ = false;
-  this->authUsrName_ = "";
-  this->authUsrPassWord_ = "";
-  this->tlsEnabled_ = false;
-  this->tlsTrustStorePath_ = "";
-  this->tlsTrustStorePassword_ = "";
-  this->rpcReadTimeoutSec_ = config::kRpcTimoutDef;
-  this->heartbeatPeriodSec_ = config::kHeartBeatPeriodDef;
-  this->maxHeartBeatRetryTimes_ = config::kHeartBeatFailRetryTimesDef;
-  this->heartbeatPeriodAfterFailSec_ = config::kHeartBeatSleepPeriodDef;
+  this->master_addrinfo_ = "";
+  this->auth_enable_ = false;
+  this->auth_usrname_ = "";
+  this->auth_usrpassword_ = "";
+  this->tls_enabled_ = false;
+  this->tls_trust_store_path_ = "";
+  this->tls_trust_store_password_ = "";
+  this->rpc_read_timeout_sec_ = config::kRpcTimoutDef;
+  this->heartbeat_period_sec_ = config::kHeartBeatPeriodDef;
+  this->max_heartbeat_retry_times_ = config::kHeartBeatFailRetryTimesDef;
+  this->heartbeat_period_afterfail_sec_ = config::kHeartBeatSleepPeriodDef;
 }
 
 BaseConfig::~BaseConfig() {
@@ -45,188 +45,188 @@ BaseConfig::~BaseConfig() {
 
 BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
   if(this != &target) {
-    this->masterAddrStr_ = target.masterAddrStr_;
-    this->authEnable_    = target.authEnable_;
-    this->authUsrName_   = target.authUsrName_;
-    this->authUsrPassWord_ = target.authUsrPassWord_;
-    this->tlsEnabled_      = target.tlsEnabled_;
-    this->tlsTrustStorePath_      = target.tlsTrustStorePath_;
-    this->tlsTrustStorePassword_  = target.tlsTrustStorePassword_;
-    this->rpcReadTimeoutSec_      = target.rpcReadTimeoutSec_;
-    this->heartbeatPeriodSec_     = target.heartbeatPeriodSec_;
-    this->maxHeartBeatRetryTimes_ = target.maxHeartBeatRetryTimes_;
-    this->heartbeatPeriodAfterFailSec_ = target.heartbeatPeriodAfterFailSec_;
+    this->master_addrinfo_ = target.master_addrinfo_;
+    this->auth_enable_    = target.auth_enable_;
+    this->auth_usrname_   = target.auth_usrname_;
+    this->auth_usrpassword_ = target.auth_usrpassword_;
+    this->tls_enabled_      = target.tls_enabled_;
+    this->tls_trust_store_path_      = target.tls_trust_store_path_;
+    this->tls_trust_store_password_  = target.tls_trust_store_password_;
+    this->rpc_read_timeout_sec_      = target.rpc_read_timeout_sec_;
+    this->heartbeat_period_sec_     = target.heartbeat_period_sec_;
+    this->max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
+    this->heartbeat_period_afterfail_sec_ = target.heartbeat_period_afterfail_sec_;
   }
   return *this;
 }
-    
-bool BaseConfig::SetMasterAddrInfo(string& errInfo, const string& masterAddrInfo) {
+
+bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrinfo) {
   // check parameter masterAddrInfo
-  string trimMasterAddrInfo = Utils::trim(masterAddrInfo);
-  if(trimMasterAddrInfo.empty()) {
-    errInfo = "Illegal parameter: masterAddrInfo is blank!";
+  string trimed_master_addr_info = Utils::trim(master_addrinfo);
+  if(trimed_master_addr_info.empty()) {
+    err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
-  if(trimMasterAddrInfo.length() > config::kMasterAddrInfoMaxLength) {
+  if(trimed_master_addr_info.length() > config::kMasterAddrInfoMaxLength) {
     stringstream ss;
     ss << "Illegal parameter: over max ";
     ss << config::kMasterAddrInfoMaxLength;
-    ss << " length of masterAddrInfo parameter!";   
-    errInfo = ss.str();
+    ss << " length of master_addrinfo parameter!";   
+    err_info = ss.str();
     return false;
   }
   // parse and verify master address info
-  // masterAddrInfo's format like ip1:port1,ip2:port2,ip3:port3
-  map<string, int> tgtAddressMap;
-  Utils::split(masterAddrInfo, tgtAddressMap, 
+  // master_addrinfo's format like ip1:port1,ip2:port2,ip3:port3
+  map<string, int> tgt_address_map;
+  Utils::split(master_addrinfo, tgt_address_map, 
     delimiter::tDelimiterComma, delimiter::tDelimiterColon);
-  if(tgtAddressMap.empty()) {
-    errInfo = "Illegal parameter: masterAddrInfo is blank!";
+  if(tgt_address_map.empty()) {
+    err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
-  this->masterAddrStr_ = trimMasterAddrInfo;
-  errInfo = "Ok";
+  this->master_addrinfo_ = trimed_master_addr_info;
+  err_info = "Ok";
   return true;
 }
 
-bool BaseConfig::SetTlsInfo(string& errInfo, bool tlsEnable,
-                              const string& trustStorePath, const string& trustStorePassword) {
-  this->tlsEnabled_ = tlsEnable;
-  if(tlsEnable) {
-    string trimTrustStorePath = Utils::trim(trustStorePath);  
-    if(trimTrustStorePath.empty()) {
-      errInfo = "Illegal parameter: trustStorePath is empty!";
+bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
+                const string& trust_store_path, const string& trust_store_password) {
+  this->tls_enabled_ = tls_enable;
+  if(tls_enable) {
+    string trimed_trust_store_path = Utils::trim(trust_store_path);  
+    if(trimed_trust_store_path.empty()) {
+      err_info = "Illegal parameter: trust_store_path is empty!";
       return false;
     }
-    string trimTrustStorePassword = Utils::trim(trustStorePassword);  
-    if(trimTrustStorePassword.empty()) {
-      errInfo = "Illegal parameter: trustStorePassword is empty!";
+    string trimed_trust_store_password = Utils::trim(trust_store_password);  
+    if(trimed_trust_store_password.empty()) {
+      err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
     }
-      this->tlsTrustStorePath_= trimTrustStorePath;
-      this->tlsTrustStorePassword_= trimTrustStorePassword;    
+      this->tls_trust_store_path_= trimed_trust_store_path;
+      this->tls_trust_store_password_= trimed_trust_store_password;    
   } else {
-    this->tlsTrustStorePath_ = "";
-    this->tlsTrustStorePassword_ = "";
+    this->tls_trust_store_path_ = "";
+    this->tls_trust_store_password_ = "";
   }
-  errInfo = "Ok";
+  err_info = "Ok";
   return true;  
 }
 
-bool BaseConfig::SetAuthenticInfo(string& errInfo, bool needAuthentic, 
-                                       const string& usrName, const string& usrPassWord) {
-  this->authEnable_ = needAuthentic;
-  if(needAuthentic) {
-    string trimUsrName = Utils::trim(usrName);
-    if(trimUsrName.empty()) {
-      errInfo = "Illegal parameter: usrName is empty!";
+bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, 
+                                       const string& usr_name, const string& usr_password) {
+  this->auth_enable_ = authentic_enable;
+  if(authentic_enable) {
+    string trimed_usr_name = Utils::trim(usr_name);
+    if(trimed_usr_name.empty()) {
+      err_info = "Illegal parameter: usr_name is empty!";
       return false;
     }
-    string trimUsrPassWord = Utils::trim(usrPassWord);  
-    if(trimUsrPassWord.empty()) {
-      errInfo = "Illegal parameter: usrPassWord is empty!";
+    string trimed_usr_password = Utils::trim(usr_password);  
+    if(trimed_usr_password.empty()) {
+      err_info = "Illegal parameter: usr_password is empty!";
       return false;
     }
-    this->authUsrName_ = trimUsrName;
-    this->authUsrPassWord_ = trimUsrPassWord;
+    this->auth_usrname_ = trimed_usr_name;
+    this->auth_usrpassword_ = trimed_usr_password;
   } else {
-    this->authUsrName_ = "";
-    this->authUsrPassWord_ = "";
+    this->auth_usrname_ = "";
+    this->auth_usrpassword_ = "";
   }
-  errInfo = "Ok";
+  err_info = "Ok";
   return true;
 }
 
 const string& BaseConfig::GetMasterAddrInfo() const {
-    return this->masterAddrStr_;
+    return this->master_addrinfo_;
 }
 
 bool BaseConfig::IsTlsEnabled() {
-  return this->tlsEnabled_;
+  return this->tls_enabled_;
 }
 
 const string& BaseConfig::GetTrustStorePath() const {
-  return this->tlsTrustStorePath_;
+  return this->tls_trust_store_path_;
 }
 
 const string& BaseConfig::GetTrustStorePassword() const {
-  return this->tlsTrustStorePassword_;
+  return this->tls_trust_store_password_;
 }
 
 bool BaseConfig::IsAuthenticEnabled() {
-  return this->authEnable_;
+  return this->auth_enable_;
 }
 
 const string& BaseConfig::GetUsrName() const {
-  return this->authUsrName_;
+  return this->auth_usrname_;
 }
 
 const string& BaseConfig::GetUsrPassWord() const {
-  return this->authUsrPassWord_;
+  return this->auth_usrpassword_;
 }
 
-void BaseConfig::SetRpcReadTimeoutSec(int rpcReadTimeoutSec) {
-  if (rpcReadTimeoutSec >= config::kRpcTimoutMax) {
-    this->rpcReadTimeoutSec_ = config::kRpcTimoutMax;
-  } else if (rpcReadTimeoutSec <= config::kRpcTimoutMin) {
-    this->rpcReadTimeoutSec_ = config::kRpcTimoutMin;
+void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
+  if (rpc_read_timeout_sec >= config::kRpcTimoutMax) {
+    this->rpc_read_timeout_sec_ = config::kRpcTimoutMax;
+  } else if (rpc_read_timeout_sec <= config::kRpcTimoutMin) {
+    this->rpc_read_timeout_sec_ = config::kRpcTimoutMin;
   } else {
-    this->rpcReadTimeoutSec_ = rpcReadTimeoutSec;
+    this->rpc_read_timeout_sec_ = rpc_read_timeout_sec;
   }
 }
 
 int BaseConfig::GetRpcReadTimeoutSec() {
-  return this->rpcReadTimeoutSec_;
+  return this->rpc_read_timeout_sec_;
 }
 
-void BaseConfig::SetHeartbeatPeriodSec(int heartbeatPeriodSec) {
-  this->heartbeatPeriodSec_ = heartbeatPeriodSec;
+void BaseConfig::SetHeartbeatPeriodSec(int heartbeat_period_sec) {
+  this->heartbeat_period_sec_ = heartbeat_period_sec;
 }
 
 int BaseConfig::GetHeartbeatPeriodSec() {
-  return this->heartbeatPeriodSec_;
+  return this->heartbeat_period_sec_;
 }
 
-void BaseConfig::SetMaxHeartBeatRetryTimes(int maxHeartBeatRetryTimes) {
-  this->maxHeartBeatRetryTimes_ = maxHeartBeatRetryTimes;
+void BaseConfig::SetMaxHeartBeatRetryTimes(int max_heartbeat_retry_times) {
+  this->max_heartbeat_retry_times_ = max_heartbeat_retry_times;
 }
 
 int BaseConfig::GetMaxHeartBeatRetryTimes() {
-  return this->maxHeartBeatRetryTimes_;
+  return this->max_heartbeat_retry_times_;
 }
 
-void BaseConfig::SetHeartbeatPeriodAftFailSec(int heartbeatPeriodSecAfterFailSec) {
-  this->heartbeatPeriodAfterFailSec_ = heartbeatPeriodSecAfterFailSec;
+void BaseConfig::SetHeartbeatPeriodAftFailSec(int heartbeat_period_afterfail_sec) {
+  this->heartbeat_period_afterfail_sec_ = heartbeat_period_afterfail_sec;
 }
 
 int BaseConfig::GetHeartbeatPeriodAftFailSec() {
-  return this->heartbeatPeriodAfterFailSec_;
+  return this->heartbeat_period_afterfail_sec_;
 }
 
 string BaseConfig::ToString() {
   stringstream ss;
-  ss << "BaseConfig={masterAddrStr=";
-  ss << this->masterAddrStr_;
+  ss << "BaseConfig={master_addrinfo_=";
+  ss << this->master_addrinfo_;
   ss << ", authEnable=";
-  ss << this->authEnable_;
-  ss << ", authUsrName='";
-  ss << this->authUsrName_;
-  ss << "', authUsrPassWord=";
-  ss << this->authUsrPassWord_;
-  ss << ", tlsEnable=";
-  ss << this->tlsEnabled_;
-  ss << ", tlsTrustStorePath=";
-  ss << this->tlsTrustStorePath_;
-  ss << ", tlsTrustStorePassword=";
-  ss << this->tlsTrustStorePassword_;
-  ss << ", rpcReadTimeoutSec=";
-  ss << this->rpcReadTimeoutSec_;
-  ss << ", heartbeatPeriodSec=";
-  ss << this->heartbeatPeriodSec_;
-  ss << ", maxHeartBeatRetryTimes=";
-  ss << this->maxHeartBeatRetryTimes_;
-  ss << ", heartbeatPeriodAfterFail=";
-  ss << this->heartbeatPeriodAfterFailSec_;
+  ss << this->auth_enable_;
+  ss << ", auth_usrname_='";
+  ss << this->auth_usrname_;
+  ss << "', auth_usrpassword_=";
+  ss << this->auth_usrpassword_;
+  ss << ", tls_enabled_=";
+  ss << this->tls_enabled_;
+  ss << ", tls_trust_store_path_=";
+  ss << this->tls_trust_store_path_;
+  ss << ", tls_trust_store_password_=";
+  ss << this->tls_trust_store_password_;
+  ss << ", rpc_read_timeout_sec_=";
+  ss << this->rpc_read_timeout_sec_;
+  ss << ", heartbeat_period_sec_=";
+  ss << this->heartbeat_period_sec_;
+  ss << ", max_heartbeat_retry_times_=";
+  ss << this->max_heartbeat_retry_times_;
+  ss << ", heartbeat_period_afterfail_sec_=";
+  ss << this->heartbeat_period_afterfail_sec_;
   ss << "}";
   return ss.str();
 }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 8edbe09..c3d1ca7 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -21,66 +21,65 @@
 #include <stdlib.h>
 #include "utils.h"
 
-namespace TubeMQ {
+namespace tubemq {
 
 
 string Utils::trim(const string& source) {
   string target = source;
   if(!target.empty()) {
-    size_t foudPos = target.find_first_not_of(tWhitespaceCharSet);
-    if (foudPos != string::npos) {
-      target = target.substr(foudPos);
+    size_t foud_pos = target.find_first_not_of(tWhitespaceCharSet);
+    if (foud_pos != string::npos) {
+      target = target.substr(foud_pos);
     }
-    foudPos = target.find_last_not_of(tWhitespaceCharSet);
-    if(foudPos != string::npos) {
-      target = target.substr(0, foudPos + 1);
+    foud_pos = target.find_last_not_of(tWhitespaceCharSet);
+    if(foud_pos != string::npos) {
+      target = target.substr(0, foud_pos + 1);
     }
   }
   return target;
 }
 
 void Utils::split(const string& source, map<string, int>& result, 
-      const string& delimiterStep1, const string& delimiterStep2) {
-  int tmpValue;
-  string subStr;
-  string keyStr;
-  string valStr;
+      const string& delimiter_step1, const string& delimiter_step2) {
+  string item_str;
+  string key_str;
+  string val_str;
   string::size_type pos1,pos2,pos3;
   if(!source.empty()) {
     pos1 = 0;
-    pos2 = source.find(delimiterStep1);
+    pos2 = source.find(delimiter_step1);
     while(string::npos != pos2) {
-      subStr = source.substr(pos1, pos2-pos1);
-      subStr = Utils::trim(subStr);
-      pos1 = pos2 + delimiterStep1.length();
-      pos2 = source.find(delimiterStep1, pos1);
-      if(subStr.empty()) {
+      item_str = source.substr(pos1, pos2-pos1);
+      item_str = Utils::trim(item_str);
+      pos1 = pos2 + delimiter_step1.length();
+      pos2 = source.find(delimiter_step1, pos1);
+      if(item_str.empty()) {
         continue;
       }
-      pos3 = subStr.find(delimiterStep2);
+      pos3 = item_str.find(delimiter_step2);
       if(string::npos == pos3) {
         continue;
       }
-      keyStr = subStr.substr(0, pos3);
-      valStr = subStr.substr(pos3+delimiterStep2.length());
-      keyStr = Utils::trim(keyStr);
-      valStr = Utils::trim(valStr);
-      if(keyStr.empty()) {
+      key_str = item_str.substr(0, pos3);
+      val_str = item_str.substr(pos3+delimiter_step2.length());
+      key_str = Utils::trim(key_str);
+      val_str = Utils::trim(val_str);
+      if(key_str.empty()) {
         continue;
       }
-      result[keyStr] = atoi(valStr.c_str());
+      result[key_str] = atoi(val_str.c_str());
     }
     if(pos1 != source.length()) {
-      subStr = source.substr(pos1);
-      subStr = Utils::trim(subStr);
-      pos3 = subStr.find(delimiterStep2);
+      item_str = source.substr(pos1);
+      item_str = Utils::trim(item_str);
+      pos3 = item_str.find(delimiter_step2);
       if(string::npos != pos3) {
-        keyStr = subStr.substr(0, pos3);
-        valStr = subStr.substr(pos3+delimiterStep2.length());
-        keyStr = Utils::trim(keyStr);
-        valStr = Utils::trim(valStr);
-        if(!keyStr.empty()){
-          result[keyStr] = atoi(valStr.c_str());
+        key_str = item_str.substr(0, pos3);
+        val_str = item_str.substr(pos3+delimiter_step2.length());
+        key_str = Utils::trim(key_str);
+        val_str = Utils::trim(val_str);
+        if(!key_str.empty()){
+          result[key_str] = atoi(val_str.c_str());
         }
       }
     }


[incubator-tubemq] 01/50: [TUBEMQ-250] Create C/C++ configure files (#170)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit cf0ccc0898876adc4834de4272995a0272d6fac6
Author: gosonzhang <46...@qq.com>
AuthorDate: Wed Jun 24 06:15:10 2020 +0000

    [TUBEMQ-250] Create C/C++ configure files (#170)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/TubeClientConfig.h       | 59 ++++++++++++++++++++++
 .../tubemq-client-cpp/src/TubeClientConfig.cpp     | 39 ++++++++++++++
 2 files changed, 98 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h b/tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h
new file mode 100644
index 0000000..97f5978
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h
@@ -0,0 +1,59 @@
+/**
+ * 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 _TUBEMQ_CLIENT_CONFIGURE_H_
+#define _TUBEMQ_CLIENT_CONFIGURE_H_
+
+#include <string>
+#include <stdio.h>
+
+
+
+
+namespace TubeMQ {
+
+    using namespace std;
+
+    class TubeConsumerConfig
+       public:
+            TubeConsumerConfig();
+
+    };
+{
+		
+ 
+}
+
+
+#endif
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp b/tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp
new file mode 100644
index 0000000..91107fc
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp
@@ -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.
+ */
+ 
+#include "TubeClientConfig.h"
+
+
+namespace TubeMQ {
+
+
+TubeConsumerConfig::TubeConsumerConfig()
+{
+
+}
+
+TubeConsumerConfig::~TubeConsumerConfig()
+{
+
+}
+
+
+
+}
+


[incubator-tubemq] 35/50: [TUBEMQ-285]Replace C/C++ pthread's mutex to std::mutex (#210)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 20169902998d74d35314f767c6cb9966528c7d43
Author: gosonzhang <46...@qq.com>
AuthorDate: Fri Jul 17 11:46:14 2020 +0000

    [TUBEMQ-285]Replace C/C++ pthread's mutex to std::mutex (#210)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/flowctrl_def.h                  |  16 ++--
 .../include/tubemq/rmt_data_cache.h                |  19 ++--
 .../include/tubemq/tubemq_config.h                 |   5 +
 .../tubemq-client-cpp/src/flowctrl_def.cc          |  71 ++++++++------
 .../tubemq-client-cpp/src/meta_info.cc             |   4 +-
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 102 ++++++---------------
 6 files changed, 101 insertions(+), 116 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
index c0289ca..8b99cbd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
@@ -26,6 +26,7 @@
 #include <algorithm>
 #include <list>
 #include <map>
+#include <mutex>
 #include <string>
 #include <vector>
 
@@ -34,6 +35,7 @@
 namespace tubemq {
 
 using std::map;
+using std::mutex;
 using std::string;
 using std::vector;
 
@@ -94,12 +96,12 @@ class FlowCtrlRuleHandler {
                              const string& flowctrl_info);
   bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const;
   int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const;
+  void GetFilterCtrlItem(FlowCtrlItem& result) const;
+  void GetFlowCtrlInfo(string& flowctrl_info) const;
   int32_t GetMinZeroCnt() const { return this->min_zero_cnt_.Get(); }
   int32_t GetQryPriorityId() const { return this->qrypriority_id_.Get(); }
   void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
   int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
-  const FlowCtrlItem& GetFilterCtrlItem() const { return this->filter_ctrl_item_; }
-  const string& GetFlowCtrlInfo() const { return this->flowctrl_info_; }
 
  private:
   void initialStatisData();
@@ -124,17 +126,17 @@ class FlowCtrlRuleHandler {
                        int32_t& value);
 
  private:
+  mutable mutex config_lock_;
+  string flowctrl_info_;
+  FlowCtrlItem filter_ctrl_item_;
+  map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
+  int64_t last_update_time_;
   AtomicLong flowctrl_id_;
   AtomicInteger qrypriority_id_;
-  string flowctrl_info_;
   AtomicInteger min_zero_cnt_;
   AtomicLong min_datadlt_limt_;
   AtomicInteger datalimit_start_time_;
   AtomicInteger datalimit_end_time_;
-  FlowCtrlItem filter_ctrl_item_;
-  map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
-  pthread_rwlock_t configrw_lock_;
-  int64_t last_update_time_;
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index af12ce4..98f192e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -20,11 +20,12 @@
 #ifndef TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
 #define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
 
-#include <pthread.h>
 #include <stdint.h>
 
+#include <condition_variable>
 #include <list>
 #include <map>
+#include <mutex>
 #include <set>
 #include <string>
 #include <tuple>
@@ -41,9 +42,11 @@
 
 namespace tubemq {
 
+using std::condition_variable;
 using std::map;
 using std::set;
 using std::list;
+using std::mutex;
 using std::string;
 using std::tuple;
 
@@ -110,7 +113,7 @@ class RmtDataCacheCsm {
 
  private:
   // timer executor
-  ExecutorPtr executor_;
+  ExecutorPool executor_;
   // 
   string consumer_id_;
   string group_name_;
@@ -120,7 +123,7 @@ class RmtDataCacheCsm {
   AtomicBoolean under_groupctrl_;
   AtomicLong last_checktime_;
   // meta info
-  pthread_rwlock_t meta_rw_lock_;
+  mutable mutex meta_lock_;
   // partiton allocated map
   map<string, PartitionExt> partitions_;
   // topic partiton map
@@ -129,25 +132,23 @@ class RmtDataCacheCsm {
   map<NodeInfo, set<string> > broker_partition_;
   map<string, SubscribeInfo>  part_subinfo_;
   // for idle partitions occupy
-  pthread_mutex_t  part_mutex_;
-  // for partiton idle map
   list<string> index_partitions_;
   // for partition used map
   map<string, int64_t> partition_useds_;
   // for partiton timer map
   map<string, tuple<int64_t, SteadyTimerPtr> > partition_timeouts_;
   // data book
-  pthread_mutex_t data_book_mutex_;
+  mutable mutex data_book_mutex_;
   // for partition offset cache
   map<string, int64_t> partition_offset_;
   // for partiton register booked
   map<string, bool> part_reg_booked_;
 
   // event
-  pthread_mutex_t  event_read_mutex_;
-  pthread_cond_t   event_read_cond_;
+  mutable mutex event_read_mutex_;
+  condition_variable event_read_cond_;
   list<ConsumerEvent> rebalance_events_;
-  pthread_mutex_t  event_write_mutex_;
+  mutable mutex event_write_mutex_;
   list<ConsumerEvent> rebalance_results_;
 };
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
index df3f09c..3660cbb 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
@@ -99,6 +99,11 @@ class ConsumerConfig : public BaseConfig {
                              const map<string, set<string> >& subscribed_topic_and_filter_map,
                              const string& session_key, uint32_t source_count, bool is_select_big,
                              const map<string, int64_t>& part_offset_map);
+  bool IsBoundConsume() { return is_bound_consume_; }
+  const string& GetSessionKey() const { return session_key_; }
+  const uint32_t GetSourceCount() const { return source_count_; }
+  bool IsSelectBig() { return is_select_big_; }
+  const map<string, int64_t>& GetPartOffsetInfo() const { return part_offset_map_; }
   const string& GetGroupName() const;
   const map<string, set<string> >& GetSubTopicAndFilterMap() const;
   void SetConsumePosition(ConsumePosition consume_from_where);
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index 370e6e6..ef3357f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -32,6 +32,7 @@
 namespace tubemq {
 
 using std::stringstream;
+using std::lock_guard;
 
 FlowCtrlResult::FlowCtrlResult() {
   this->datasize_limit_ = tb_config::kMaxIntValue;
@@ -171,10 +172,11 @@ FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
   this->datalimit_start_time_.Set(2500);
   this->datalimit_end_time_.Set(tb_config::kInvalidValue);
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
-  pthread_rwlock_init(&configrw_lock_, NULL);
 }
 
-FlowCtrlRuleHandler::~FlowCtrlRuleHandler() { pthread_rwlock_destroy(&configrw_lock_); }
+FlowCtrlRuleHandler::~FlowCtrlRuleHandler() {
+  // 
+}
 
 void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id,
                                                 int64_t flowctrl_id, const string& flowctrl_info) {
@@ -186,7 +188,7 @@ void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qryprio
   if (flowctrl_info.length() > 0) {
     parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
   }
-  pthread_rwlock_wrlock(&this->configrw_lock_);
+  lock_guard<mutex> lck(config_lock_);
   this->flowctrl_id_.Set(flowctrl_id);
   this->qrypriority_id_.Set(qrypriority_id);
   clearStatisData();
@@ -199,7 +201,6 @@ void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qryprio
     initialStatisData();
   }
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
-  pthread_rwlock_unlock(&this->configrw_lock_);
   if (is_default) {
     LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id,
              flowctrl_id);
@@ -268,51 +269,69 @@ void FlowCtrlRuleHandler::clearStatisData() {
 bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt,
                                           FlowCtrlResult& flowctrl_result) const {
   struct tm utc_tm;
+  bool result = false;
   vector<FlowCtrlItem>::const_iterator it_vec;
   map<int, vector<FlowCtrlItem> >::const_iterator it_map;
+  // get current data limit
   time_t cur_time = time(NULL);
-
   gmtime_r(&cur_time, &utc_tm);
   int curr_time = (utc_tm.tm_hour + 8) % 24 * 100 + utc_tm.tm_min;
-  if ((last_datadlt < this->min_datadlt_limt_.Get()) ||
-      (curr_time < this->datalimit_start_time_.Get()) ||
-      (curr_time > this->datalimit_end_time_.Get())) {
+  if ((last_datadlt < this->min_datadlt_limt_.Get())
+      || (curr_time < this->datalimit_start_time_.Get())
+      || (curr_time > this->datalimit_end_time_.Get())) {
     return false;
   }
+  // search total flowctrl rule
+  lock_guard<mutex> lck(config_lock_);
   it_map = this->flowctrl_rules_.find(0);
-  if (it_map == this->flowctrl_rules_.end()) {
-    return false;
-  }
-  for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
-    if (it_vec->GetDataLimit(last_datadlt, curr_time, flowctrl_result)) {
-      return true;
+  if (it_map != this->flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin();it_vec != it_map->second.end(); ++it_vec) {
+      if (it_vec->GetDataLimit(last_datadlt, curr_time, flowctrl_result)) {
+        result = true;
+        break;
+      }
     }
   }
-  return false;
+  return result;
 }
 
 int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt,
                                                  int32_t received_limit) const {
-  int32_t rule_val = -2;
+  int32_t limit_data = received_limit;
   vector<FlowCtrlItem>::const_iterator it_vec;
   map<int, vector<FlowCtrlItem> >::const_iterator it_map;
-
+  // check min zero count
   if (msg_zero_cnt < this->min_zero_cnt_.Get()) {
-    return received_limit;
+    return limit_data;
   }
+  // search rule allow value
+  lock_guard<mutex> lck(config_lock_);
   it_map = this->flowctrl_rules_.find(1);
-  if (it_map == this->flowctrl_rules_.end()) {
-    return received_limit;
-  }
-  for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
-    rule_val = it_vec->GetFreLimit(msg_zero_cnt);
-    if (rule_val >= 0) {
-      return rule_val;
+  if (it_map != this->flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+      limit_data = it_vec->GetFreLimit(msg_zero_cnt);
+      if (limit_data >= 0) {
+        break;
+      }
     }
   }
-  return received_limit;
+  return limit_data;
+}
+
+void FlowCtrlRuleHandler::GetFilterCtrlItem(FlowCtrlItem& result) const {
+  result.Clear();
+  lock_guard<mutex> lck(config_lock_);
+  result = this->filter_ctrl_item_;
+}
+
+void FlowCtrlRuleHandler::GetFlowCtrlInfo(string& flowctrl_info) const {
+  flowctrl_info.clear();
+  lock_guard<mutex> lck(config_lock_);
+  flowctrl_info = this->flowctrl_info_;
 }
 
+
+
 bool FlowCtrlRuleHandler::compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2) {
   if (o1.GetStartTime() >= o2.GetStartTime()) {
     return true;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 55e8cdc..9299f71 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -408,9 +408,9 @@ void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_ha
           this->cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 0);
         }
       }
-      this->cur_freqctrl_ = group_flowctrl_handler.GetFilterCtrlItem();
+      group_flowctrl_handler.GetFilterCtrlItem(this->cur_freqctrl_);
       if (this->cur_freqctrl_.GetFreqMsLimit() < 0) {
-        this->cur_freqctrl_ = def_flowctrl_handler.GetFilterCtrlItem();
+        def_flowctrl_handler.GetFilterCtrlItem(this->cur_freqctrl_);
       }
       curr_time = Utils::GetCurrentTimeMillis();
     }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index 8b1e76b..c9c499d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -30,27 +30,18 @@
 
 namespace tubemq {
 
-
+using std::lock_guard;
+using std::unique_lock;
+using namespace std::placeholders;
 
 
 RmtDataCacheCsm::RmtDataCacheCsm() {
   under_groupctrl_.Set(false);
   last_checktime_.Set(0);
-  pthread_rwlock_init(&meta_rw_lock_, NULL);
-  pthread_mutex_init(&part_mutex_, NULL);
-  pthread_mutex_init(&data_book_mutex_, NULL);
-  pthread_mutex_init(&event_read_mutex_, NULL);
-  pthread_cond_init(&event_read_cond_, NULL);
-  pthread_mutex_init(&event_write_mutex_, NULL);
 }
 
 RmtDataCacheCsm::~RmtDataCacheCsm() {
-  pthread_mutex_destroy(&event_write_mutex_);
-  pthread_mutex_destroy(&event_read_mutex_);
-  pthread_mutex_destroy(&data_book_mutex_);
-  pthread_cond_destroy(&event_read_cond_);
-  pthread_mutex_destroy(&part_mutex_);
-  pthread_rwlock_destroy(&meta_rw_lock_);
+  // 
 }
 
 void RmtDataCacheCsm::SetConsumerInfo(const string& client_id,
@@ -103,7 +94,8 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   //
   SubscribeInfo sub_info(consumer_id_, group_name_, partition_ext);
   string partition_key = partition_ext.GetPartitionKey();
-  pthread_rwlock_wrlock(&meta_rw_lock_);
+  // lock operate
+  lock_guard<mutex> lck(meta_lock_);
   it_map = partitions_.find(partition_key);
   if (it_map == partitions_.end()) {
     partitions_[partition_key] = partition_ext;
@@ -130,10 +122,7 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
     part_subinfo_[partition_key] = sub_info;
   }
   // check partition_key status
-  pthread_mutex_lock(&part_mutex_);
   resetIdlePartition(partition_key, true);
-  pthread_mutex_unlock(&part_mutex_);
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 bool RmtDataCacheCsm::SelectPartition(string &err_info,
@@ -142,13 +131,12 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
   int64_t booked_time = 0;
   string partition_key;
   map<string, PartitionExt>::iterator it_map;
-
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  // lock operate
+  lock_guard<mutex> lck(meta_lock_);
   if (partitions_.empty()) {
     err_info = "No partition info in local cache, please retry later!";
     result = false;
   } else {
-    pthread_mutex_lock(&part_mutex_);
     if (index_partitions_.empty()) {
       err_info = "No idle partition to consume, please retry later!";
       result = false;
@@ -167,9 +155,7 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
         err_info = "Ok";
       }
     }
-    pthread_mutex_unlock(&part_mutex_);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
   return result;
 }
 
@@ -180,18 +166,16 @@ void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key,
   map<string, PartitionExt>::iterator it_part;
   // book partition offset info
   if (curr_offset >= 0) {
-    pthread_mutex_lock(&data_book_mutex_);
+    lock_guard<mutex> lck1(data_book_mutex_);
     partition_offset_[partition_key] = curr_offset;
-    pthread_mutex_unlock(&data_book_mutex_);
   }
   // book partition temp info
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck2(meta_lock_);
   it_part = partitions_.find(partition_key);
   if (it_part != partitions_.end()) {
     it_part->second.BookConsumeData(err_code, msg_size,
               esc_limit, limit_dlt, cur_data_dlt, require_slow);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 // success process release partition
@@ -233,7 +217,7 @@ void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info
   // initial return;
   subscribed_partitions.clear();
   unsub_partitions.clear();
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   if (partitions_.empty()) {
     for (it_lst = subscribe_info_lst.begin(); it_lst != subscribe_info_lst.end(); it_lst++) {
       unsub_partitions.push_back(it_lst->GetPartitionExt());
@@ -248,17 +232,15 @@ void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info
       }
     }
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 void RmtDataCacheCsm::GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst) {
   map<string, SubscribeInfo>::iterator it_sub;
   subscribe_info_lst.clear();
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   for (it_sub = part_subinfo_.begin(); it_sub != part_subinfo_.end(); ++it_sub) {
     subscribe_info_lst.push_back(it_sub->second);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 void RmtDataCacheCsm::GetAllBrokerPartitions(
@@ -267,7 +249,7 @@ void RmtDataCacheCsm::GetAllBrokerPartitions(
   map<NodeInfo, list<PartitionExt> >::iterator it_broker;
 
   broker_parts.clear();
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   for (it_part = partitions_.begin(); it_part != partitions_.end(); ++it_part) {
     it_broker = broker_parts.find(it_part->second.GetBrokerInfo());
     if (it_broker == broker_parts.end()) {
@@ -278,20 +260,18 @@ void RmtDataCacheCsm::GetAllBrokerPartitions(
       it_broker->second.push_back(it_part->second);
     }
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 bool RmtDataCacheCsm::GetPartitionExt(const string& part_key, PartitionExt& partition_ext) {
   bool result = false;
   map<string, PartitionExt>::iterator it_map;
 
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   it_map = partitions_.find(part_key);
   if (it_map != partitions_.end()) {
     result = true;
     partition_ext = it_map->second;
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
   return result;
 }
 
@@ -299,11 +279,10 @@ void RmtDataCacheCsm::GetRegBrokers(list<NodeInfo>& brokers) {
   map<NodeInfo, set<string> >::iterator it;
 
   brokers.clear();
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   for (it = broker_partition_.begin(); it != broker_partition_.end(); ++it) {
     brokers.push_back(it->first);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
@@ -313,7 +292,7 @@ void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
   map<string, PartitionExt>::iterator it_part;
 
   partition_list.clear();
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   it_broker = broker_partition_.find(broker_info);
   if (it_broker != broker_partition_.end()) {
     for (it_key = it_broker->second.begin();
@@ -324,7 +303,6 @@ void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
       }
     }
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 
@@ -332,11 +310,10 @@ void RmtDataCacheCsm::GetCurPartitionOffsets(map<string, int64_t> part_offset_ma
   map<string, int64_t>::iterator it;
 
   part_offset_map.clear();
-  pthread_mutex_lock(&data_book_mutex_);
+  lock_guard<mutex> lck(data_book_mutex_);
   for (it = partition_offset_.begin(); it != partition_offset_.end(); ++it) {
     part_offset_map[it->first] = it->second;
   }
-  pthread_mutex_unlock(&data_book_mutex_);
 }
 
 
@@ -373,15 +350,12 @@ void RmtDataCacheCsm::RemovePartition(const list<PartitionExt>& partition_list)
 void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
   set<string>::const_iterator it_lst;
 
-  pthread_rwlock_wrlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   for (it_lst = partition_keys.begin(); it_lst != partition_keys.end(); it_lst++) {
-    pthread_mutex_lock(&part_mutex_);
     resetIdlePartition(*it_lst, false);
-    pthread_mutex_unlock(&part_mutex_);
     // remove meta info set info
     rmvMetaInfo(*it_lst);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
@@ -397,8 +371,7 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
   if (subscribe_infos.empty()) {
     return;
   }
-  pthread_rwlock_wrlock(&meta_rw_lock_);
-  pthread_mutex_lock(&part_mutex_);
+  lock_guard<mutex> lck(meta_lock_);
   for (it = subscribe_infos.begin(); it != subscribe_infos.end(); ++it) {
     part_key = it->GetPartitionExt().GetPartitionKey();
     it_part = partitions_.find(part_key);
@@ -422,8 +395,6 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
     }
     resetIdlePartition(part_key, false);
   }
-  pthread_mutex_unlock(&part_mutex_);
-  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 
@@ -431,64 +402,56 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
 bool RmtDataCacheCsm::BookPartition(const string& partition_key) {
   bool result = false;
   map<string, bool>::iterator it;
-  pthread_mutex_lock(&data_book_mutex_);
+
+  lock_guard<mutex> lck(data_book_mutex_);
   it = part_reg_booked_.find(partition_key);
   if (it == part_reg_booked_.end()) {
     part_reg_booked_[partition_key] = true;
   }
-  pthread_mutex_unlock(&data_book_mutex_);
   return result;
 }
 
 void RmtDataCacheCsm::OfferEvent(const ConsumerEvent& event) {
-  pthread_mutex_lock(&event_read_mutex_);
+  unique_lock<mutex> lck(event_read_mutex_);
   this->rebalance_events_.push_back(event);
-  pthread_cond_broadcast(&event_read_cond_);
-  pthread_mutex_unlock(&event_read_mutex_);
+  event_read_cond_.notify_all();
 }
 
 void RmtDataCacheCsm::TakeEvent(ConsumerEvent& event) {
-  pthread_mutex_lock(&event_read_mutex_);
+  unique_lock<mutex> lck(event_read_mutex_);
   while (this->rebalance_events_.empty()) {
-    pthread_cond_wait(&event_read_cond_, &event_read_mutex_);
+    event_read_cond_.wait(lck);
   }
   event = rebalance_events_.front();
   rebalance_events_.pop_front();
-  pthread_mutex_unlock(&event_read_mutex_);
 }
 
 void RmtDataCacheCsm::ClearEvent() {
-  pthread_mutex_lock(&event_read_mutex_);
+  unique_lock<mutex> lck(event_read_mutex_);
   rebalance_events_.clear();
-  pthread_mutex_unlock(&event_read_mutex_);
 }
 
 void RmtDataCacheCsm::OfferEventResult(const ConsumerEvent& event) {
-  pthread_mutex_lock(&event_write_mutex_);
+  lock_guard<mutex> lck(event_write_mutex_);
   this->rebalance_events_.push_back(event);
-  pthread_mutex_unlock(&event_write_mutex_);
 }
 
 bool RmtDataCacheCsm::PollEventResult(ConsumerEvent& event) {
   bool result = false;
-  pthread_mutex_lock(&event_write_mutex_);
+  lock_guard<mutex> lck(event_write_mutex_);
   if (!rebalance_events_.empty()) {
     event = rebalance_events_.front();
     rebalance_events_.pop_front();
     result = true;
   }
-  pthread_mutex_unlock(&event_write_mutex_);
   return result;
 }
 
 void RmtDataCacheCsm::HandleTimeout(const string partition_key,
                                           const asio::error_code& error) {
   if (!error) {
-    pthread_rwlock_rdlock(&meta_rw_lock_);
-    pthread_mutex_lock(&part_mutex_);
+    lock_guard<mutex> lck(meta_lock_);
     resetIdlePartition(partition_key, true);
-    pthread_mutex_unlock(&part_mutex_);    
-    pthread_rwlock_unlock(&meta_rw_lock_);
   }
 }
 
@@ -579,18 +542,15 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
   if (!result) {
     return false;
   }
-  pthread_rwlock_rdlock(&meta_rw_lock_);
+  lock_guard<mutex> lck(meta_lock_);
   it_part = partitions_.find(partition_key);
   if (it_part == partitions_.end()) {
     // partition is unregister, release partition
-    pthread_mutex_lock(&part_mutex_);
     partition_useds_.erase(partition_key);
     index_partitions_.remove(partition_key);
-    pthread_mutex_unlock(&part_mutex_);
     err_info = "Not found the partition in Consume Partition set!";
     result = false;
   } else {
-    pthread_mutex_lock(&part_mutex_);
     it_used = partition_useds_.find(partition_key);
     if (it_used == partition_useds_.end()) {
       // partition is release but registered
@@ -619,9 +579,7 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
         result = false;
       }
     }
-    pthread_mutex_unlock(&part_mutex_);
   }
-  pthread_rwlock_unlock(&meta_rw_lock_);
   return result;
 }
 


[incubator-tubemq] 06/50: adjust code style

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 5c216c942998fbabfc2de55935e7fe0f058fe139
Author: gosonzhang <go...@tencent.com>
AuthorDate: Wed Jul 1 17:33:09 2020 +0800

    adjust code style
---
 tubemq-client-twins/tubemq-client-cpp/src/client_config.cc | 6 ++----
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc         | 8 ++++----
 2 files changed, 6 insertions(+), 8 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index b596d76..84d6721 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -352,8 +352,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     // check topic_name info
     is_success = Utils::ValidString(err_info, itMap->first, 
                          false, true, true, config::kTopicNameMaxLength);  
-    if (!is_success)
-    {
+    if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
       ss << itMap->first;
@@ -365,8 +364,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     string topic_name = Utils::Trim(itMap->first);
     // check filter info
     set<string> subscribed_filters = itMap->second;
-    for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end(); ++it)
-    {
+    for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end(); ++it) {
       is_success = Utils::ValidFilterItem(err_info, *it, tmp_filteritem);
       if (!is_success) {
         stringstream ss;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index d3f36ff..f9a1b1e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -51,7 +51,7 @@ void Utils::Split(const string& source, vector<string>& result, const string& de
   if (!source.empty()) {
     pos1 = 0;
     pos2 = source.find(delimiter);
-    while(string::npos != pos2) {
+    while (string::npos != pos2) {
       item_str = Utils::Trim(source.substr(pos1, pos2-pos1));
       pos1 = pos2 + delimiter.size();
       pos2 = source.find(delimiter, pos1);
@@ -79,7 +79,7 @@ void Utils::Split(const string& source, map<string, int>& result,
   if (!source.empty()) {
     pos1 = 0;
     pos2 = source.find(delimiter_step1);
-    while(string::npos != pos2) {
+    while (string::npos != pos2) {
       item_str = source.substr(pos1, pos2-pos1);
       item_str = Utils::Trim(item_str);
       pos1 = pos2 + delimiter_step1.length();
@@ -229,8 +229,8 @@ bool Utils::ValidFilterItem(string& err_info,
   int status = regexec(&reg, tgt_filteritem.c_str(), 1, pmatch, 0);
   regfree(&reg);
   if (status == REG_NOMATCH) {
-      err_info = "value only contain characters,numbers,and underscores";
-      return false;
+    err_info = "value only contain characters,numbers,and underscores";
+    return false;
   }
   err_info = "Ok";
   return true;      


[incubator-tubemq] 34/50: [TUBEMQ-283]Adjust C/C++ some file names: add "tubemq_" prefix (#208)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit cd4b64779525cbc2ce6fabf05edf53717a3bf5c9
Author: gosonzhang <46...@qq.com>
AuthorDate: Thu Jul 16 14:28:24 2020 +0000

    [TUBEMQ-283]Adjust C/C++ some file names: add "tubemq_" prefix (#208)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/rmt_data_cache.h                | 20 ++++--
 .../tubemq/{client_config.h => tubemq_config.h}    |  1 +
 .../tubemq/{const_errcode.h => tubemq_errcode.h}   |  0
 .../include/tubemq/{message.h => tubemq_message.h} |  0
 .../include/tubemq/tubemq_return.h                 |  2 +-
 .../tubemq-client-cpp/include/tubemq/utils.h       |  2 +-
 .../tubemq-client-cpp/src/meta_info.cc             |  2 +-
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 84 ++++++++++++++--------
 .../src/{client_config.cc => tubemq_config.cc}     |  4 +-
 .../src/{message.cc => tubemq_message.cc}          |  2 +-
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  2 +-
 11 files changed, 76 insertions(+), 43 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index aa8f250..af12ce4 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -27,10 +27,13 @@
 #include <map>
 #include <set>
 #include <string>
+#include <tuple>
 
 #include "tubemq/atomic_def.h"
 #include "tubemq/flowctrl_def.h"
 #include "tubemq/meta_info.h"
+#include "tubemq/executor_pool.h"
+#include "tubemq/tubemq_errcode.h"
 
 
 
@@ -41,13 +44,17 @@ namespace tubemq {
 using std::map;
 using std::set;
 using std::list;
+using std::string;
+using std::tuple;
+
 
 
 // consumer remote data cache
 class RmtDataCacheCsm {
  public:
-  RmtDataCacheCsm(const string& client_id, const string& group_name);
+  RmtDataCacheCsm();
   ~RmtDataCacheCsm();
+  void SetConsumerInfo(const string& client_id, const string& group_name);
   void UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
                                      const string& flowctrl_info);
   void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
@@ -87,8 +94,11 @@ class RmtDataCacheCsm {
   void ClearEvent();
   void OfferEventResult(const ConsumerEvent& event);
   bool PollEventResult(ConsumerEvent& event);
+  void HandleTimeout(const string partition_key, const asio::error_code& error);
 
  private:
+  void addDelayTimer(const string& part_key, int64_t delay_time);
+  void resetIdlePartition(const string& partition_key, bool need_reuse);
   void rmvMetaInfo(const string& partition_key);
   void buildConfirmContext(const string& partition_key,
                     int64_t booked_time, string& confirm_context);
@@ -99,9 +109,9 @@ class RmtDataCacheCsm {
 
 
  private:
-  // timer begin
-
-  // timer end
+  // timer executor
+  ExecutorPtr executor_;
+  // 
   string consumer_id_;
   string group_name_;
   // flow ctrl
@@ -125,7 +135,7 @@ class RmtDataCacheCsm {
   // for partition used map
   map<string, int64_t> partition_useds_;
   // for partiton timer map
-  map<string, int64_t> partition_timeouts_;
+  map<string, tuple<int64_t, SteadyTimerPtr> > partition_timeouts_;
   // data book
   pthread_mutex_t data_book_mutex_;
   // for partition offset cache
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
similarity index 99%
rename from tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
index fecbeb2..df3f09c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
@@ -33,6 +33,7 @@ using std::map;
 using std::set;
 using std::string;
 
+
 class BaseConfig {
  public:
   BaseConfig();
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
similarity index 100%
rename from tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
similarity index 100%
rename from tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
index 96194b4..2f35b22 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
@@ -25,7 +25,7 @@
 #include <list>
 #include <string>
 
-#include "tubemq/message.h"
+#include "tubemq/tubemq_message.h"
 #include "tubemq/meta_info.h"
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
index b4e0aeb..a25832f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
@@ -39,7 +39,7 @@ class Utils {
   // split string to vector
   static void Split(const string& source, vector<string>& result, const string& delimiter);
   // split string to map<string, int>
-  static void Split(const string& source, map<string, int>& result, const string& delimiter_step1,
+  static void Split(const string& source, map<string, int32_t>& result, const string& delimiter_step1,
                     const string& delimiter_step2);
   static void Join(const vector<string>& vec, const string& delimiter, string& target);
   static bool ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index ae7d72e..55e8cdc 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -25,7 +25,7 @@
 #include <vector>
 
 #include "tubemq/const_config.h"
-#include "tubemq/const_errcode.h"
+#include "tubemq/tubemq_errcode.h"
 #include "tubemq/utils.h"
 
 namespace tubemq {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index ec4876b..8b1e76b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -33,10 +33,7 @@ namespace tubemq {
 
 
 
-RmtDataCacheCsm::RmtDataCacheCsm(const string& client_id,
-                                      const string& group_name) {
-  consumer_id_ = client_id;
-  group_name_ = group_name;
+RmtDataCacheCsm::RmtDataCacheCsm() {
   under_groupctrl_.Set(false);
   last_checktime_.Set(0);
   pthread_rwlock_init(&meta_rw_lock_, NULL);
@@ -56,6 +53,11 @@ RmtDataCacheCsm::~RmtDataCacheCsm() {
   pthread_rwlock_destroy(&meta_rw_lock_);
 }
 
+void RmtDataCacheCsm::SetConsumerInfo(const string& client_id,
+                                            const string& group_name) {
+  consumer_id_ = client_id;
+  group_name_ = group_name;
+}
 
 void RmtDataCacheCsm::UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
                                                  const string& flowctrl_info) {
@@ -129,11 +131,7 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   }
   // check partition_key status
   pthread_mutex_lock(&part_mutex_);
-  if (partition_useds_.find(partition_key) == partition_useds_.end()
-    && partition_timeouts_.find(partition_key) == partition_timeouts_.end()) {
-    index_partitions_.remove(partition_key);
-    index_partitions_.push_back(partition_key);
-  }
+  resetIdlePartition(partition_key, true);
   pthread_mutex_unlock(&part_mutex_);
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
@@ -347,9 +345,7 @@ bool RmtDataCacheCsm::RemovePartition(string &err_info,
                                   const string& confirm_context) {
   int64_t booked_time;
   string  partition_key;
-  map<string, PartitionExt>::iterator it_part;
-  map<string, set<string> >::iterator it_topic;
-  map<NodeInfo, set<string> >::iterator it_broker;
+  set<string> partition_keys;
   // parse confirm context
   bool result = parseConfirmContext(err_info,
                       confirm_context, partition_key, booked_time);
@@ -357,7 +353,6 @@ bool RmtDataCacheCsm::RemovePartition(string &err_info,
     return false;
   }
   // remove partiton
-  set<string> partition_keys;
   partition_keys.insert(partition_key);
   RemovePartition(partition_keys);
   err_info = "Ok";
@@ -381,11 +376,7 @@ void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
   pthread_rwlock_wrlock(&meta_rw_lock_);
   for (it_lst = partition_keys.begin(); it_lst != partition_keys.end(); it_lst++) {
     pthread_mutex_lock(&part_mutex_);
-    partition_useds_.erase(*it_lst);
-    index_partitions_.remove(*it_lst);
-    // todo need modify if timer build finished
-    partition_timeouts_.erase(*it_lst);
-    // end todo
+    resetIdlePartition(*it_lst, false);
     pthread_mutex_unlock(&part_mutex_);
     // remove meta info set info
     rmvMetaInfo(*it_lst);
@@ -427,13 +418,9 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
       } else {
         it_broker->second.push_back(it_part->second);
       }
-      rmvMetaInfo(part_key);  
+      rmvMetaInfo(part_key);
     }
-    partition_useds_.erase(part_key);
-    index_partitions_.remove(part_key);
-    // todo need modify if timer build finished
-    partition_timeouts_.erase(part_key);
-    // end todo
+    resetIdlePartition(part_key, false);
   }
   pthread_mutex_unlock(&part_mutex_);
   pthread_rwlock_unlock(&meta_rw_lock_);
@@ -494,6 +481,43 @@ bool RmtDataCacheCsm::PollEventResult(ConsumerEvent& event) {
   return result;
 }
 
+void RmtDataCacheCsm::HandleTimeout(const string partition_key,
+                                          const asio::error_code& error) {
+  if (!error) {
+    pthread_rwlock_rdlock(&meta_rw_lock_);
+    pthread_mutex_lock(&part_mutex_);
+    resetIdlePartition(partition_key, true);
+    pthread_mutex_unlock(&part_mutex_);    
+    pthread_rwlock_unlock(&meta_rw_lock_);
+  }
+}
+
+void RmtDataCacheCsm::addDelayTimer(const string& partition_key, int64_t delay_time) {
+  // add timer
+  tuple<int64_t, SteadyTimerPtr> timer = 
+      std::make_tuple(Utils::GetCurrentTimeMillis(), executor_.Get()->CreateSteadyTimer());
+  std::get<1>(timer)->expires_after(std::chrono::milliseconds(delay_time));
+  std::get<1>(timer)->async_wait(std::bind(&RmtDataCacheCsm::HandleTimeout, partition_key, _1));
+  partition_timeouts_.insert(std::make_pair(partition_key, timer));          
+}
+
+void RmtDataCacheCsm::resetIdlePartition(const string& partition_key, bool need_reuse) {
+  map<string, PartitionExt>::iterator it_map;
+  map<string, tuple<int64_t, SteadyTimerPtr> >::iterator it_timeout; 
+  partition_useds_.erase(partition_key);
+  it_timeout = partition_timeouts_.find(partition_key);
+  if (it_timeout != partition_timeouts_.end()) {
+    std::get<1>(it_timeout->second)->cancel();
+    partition_timeouts_.erase(partition_key);
+  }
+  index_partitions_.remove(partition_key);
+  if (need_reuse) {
+    if (partitions_.find(partition_key) != partitions_.end()) {
+      index_partitions_.push_back(partition_key);
+    }
+  }
+}
+
 void RmtDataCacheCsm::buildConfirmContext(const string& partition_key,
                                    int64_t booked_time, string& confirm_context) {
   confirm_context.clear();
@@ -544,7 +568,7 @@ void RmtDataCacheCsm::rmvMetaInfo(const string& partition_key) {
 
 bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
                      bool filter_consume, const string& confirm_context, bool is_consumed) {
-  int64_t wait_time;
+  int64_t delay_time;
   int64_t booked_time;
   string  partition_key;
   map<string, PartitionExt>::iterator it_part;
@@ -577,14 +601,13 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
         // wait release
         partition_useds_.erase(partition_key);
         index_partitions_.remove(partition_key);
-        wait_time = 0;
+        delay_time = 0;
         if (need_delay_check) {
-          wait_time = it_part->second.ProcConsumeResult(def_flowctrl_handler_,
+          delay_time = it_part->second.ProcConsumeResult(def_flowctrl_handler_,
                         group_flowctrl_handler_, filter_consume, is_consumed);
         }
-        if (wait_time >= 10) {
-          // todo add timer 
-          // end todo
+        if (delay_time > 10) {
+          addDelayTimer(partition_key, delay_time);
         } else {
           index_partitions_.push_back(partition_key);
         }
@@ -603,5 +626,4 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
 }
 
 
-
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
similarity index 96%
rename from tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
rename to tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
index dfd5b8c..b976add 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "tubemq/client_config.h"
+#include "tubemq/tubemq_config.h"
 
 #include <sstream>
 #include <vector>
@@ -84,7 +84,7 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
   }
   // parse and verify master address info
   // master_addrinfo's format like ip1:port1,ip2:port2,ip3:port3
-  map<string, int> tgt_address_map;
+  map<string, int32_t> tgt_address_map;
   Utils::Split(master_addrinfo, tgt_address_map, delimiter::kDelimiterComma,
                delimiter::kDelimiterColon);
   if (tgt_address_map.empty()) {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
similarity index 96%
rename from tubemq-client-twins/tubemq-client-cpp/src/message.cc
rename to tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
index 93e3dd2..f914741 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "tubemq/message.h"
+#include "tubemq/tubemq_message.h"
 
 #include <string.h>
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 40d1de5..2fbaecd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -76,7 +76,7 @@ void Utils::Split(const string& source, vector<string>& result, const string& de
   }
 }
 
-void Utils::Split(const string& source, map<string, int>& result, const string& delimiter_step1,
+void Utils::Split(const string& source, map<string, int32_t>& result, const string& delimiter_step1,
                   const string& delimiter_step2) {
   string item_str;
   string key_str;


[incubator-tubemq] 39/50: [TUBEMQ-281]atomic_def.h use C++11 stdlib class (#216)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit d43d8e97e755f2b09eb74e62de829096c5edeacb
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Tue Jul 21 17:14:19 2020 +0800

    [TUBEMQ-281]atomic_def.h use C++11 stdlib class (#216)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 tubemq-client-twins/tubemq-client-cpp/README.md    |   5 +
 .../tubemq-client-cpp/build_linux.sh               |   2 +
 .../tubemq-client-cpp/include/tubemq/atomic_def.h  | 334 ++++++---------------
 3 files changed, 103 insertions(+), 238 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/README.md b/tubemq-client-twins/tubemq-client-cpp/README.md
index ecf0f6f..0f566d5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/README.md
+++ b/tubemq-client-twins/tubemq-client-cpp/README.md
@@ -30,4 +30,9 @@
  * [Log4cplus](https://github.com/log4cplus/log4cplus.git)
  * [Rapidjson](https://github.com/Tencent/rapidjson.git)
 
+## Install ASIO
+  * ./autogen.sh
+  * ./configure CFLAGS=-std=c++11 CPPFLAGS=-std=c++11 CXXFLAGS=-std=c++11
+  * make && make install
+
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/build_linux.sh b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
index be79e90..7664d5b 100755
--- a/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
+++ b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
@@ -21,6 +21,8 @@
 #!/bin/bash
 
 cd ../../
+git submodule init
+git submodule update
 git submodule foreach --recursive git submodule init 
 git submodule foreach --recursive git submodule update 
 cd -
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
index efd5332..f991269 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
@@ -1,238 +1,96 @@
-/**
- * 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 TUBEMQ_CLIENT_ATOMIC_DEF_H_
-#define TUBEMQ_CLIENT_ATOMIC_DEF_H_
-
-#include <stdlib.h>
-
-namespace tubemq {
-
-
-class AtomicInteger {
- public:
-  AtomicInteger() { this->counter_ = 0; }
-
-  explicit AtomicInteger(int32_t initial_value) { this->counter_ = initial_value; }
-
-  int32_t Get() const { return this->counter_; }
-
-  void Set(int32_t new_value) { this->counter_ = new_value; }
-
-  int64_t LongValue() const { return (int64_t)this->counter_; }
-
-  int32_t GetAndSet(int32_t new_value) {
-    for (;;) {
-      int32_t current = this->counter_;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
-        return current;
-      }
-    }
-  }
-
-  bool CompareAndSet(int32_t expect, int32_t update) {
-    return __sync_bool_compare_and_swap(&this->counter_, expect, update);
-  }
-
-  int32_t GetAndIncrement() {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current + 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int32_t GetAndDecrement() {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current - 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int32_t GetAndAdd(int32_t delta) {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current + delta;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int32_t IncrementAndGet() {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current + 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
-  int32_t DecrementAndGet() {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current - 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
-  int32_t AddAndGet(int32_t delta) {
-    for (;;) {
-      int32_t current = this->counter_;
-      int32_t next = current + delta;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
- private:
-  volatile int32_t counter_;
-};
-
-class AtomicLong {
- public:
-  AtomicLong() { this->counter_ = 0; }
-
-  explicit AtomicLong(int64_t initial_value) { this->counter_ = initial_value; }
-
-  int64_t Get() const { return this->counter_; }
-
-  void Set(int64_t new_value) { this->counter_ = new_value; }
-
-  int32_t IntValue() const { return (int32_t)this->counter_; }
-
-  int64_t GetAndSet(int64_t new_value) {
-    for (;;) {
-      int64_t current = this->counter_;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
-        return current;
-      }
-    }
-  }
-
-  bool CompareAndSet(int64_t expect, int64_t update) {
-    return __sync_bool_compare_and_swap(&this->counter_, expect, update);
-  }
-
-  int64_t GetAndIncrement() {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current + 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int64_t GetAndDecrement() {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current - 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int64_t GetAndAdd(int64_t delta) {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current + delta;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return current;
-      }
-    }
-  }
-
-  int64_t IncrementAndGet() {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current + 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
-  int64_t DecrementAndGet() {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current - 1;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
-  int64_t AddAndGet(int64_t delta) {
-    for (;;) {
-      int64_t current = this->counter_;
-      int64_t next = current + delta;
-      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
-        return next;
-      }
-    }
-  }
-
- private:
-  volatile int64_t counter_;
-};
-
-class AtomicBoolean {
- public:
-  AtomicBoolean() { this->counter_ = 0; }
-
-  explicit AtomicBoolean(bool initial_value) { this->counter_ = initial_value ? 1 : 0; }
-
-  bool Get() const { return this->counter_ != 0; }
-
-  void Set(bool new_value) { this->counter_ = new_value ? 1 : 0; }
-
-  bool GetAndSet(bool new_value) {
-    int32_t u = new_value ? 1 : 0;
-    for (;;) {
-      int32_t e = this->counter_ ? 1 : 0;
-      if (__sync_bool_compare_and_swap(&this->counter_, e, u)) {
-        return e != 0;
-      }
-    }
-  }
-
-  bool CompareAndSet(bool expect, bool update) {
-    int32_t e = expect ? 1 : 0;
-    int32_t u = update ? 1 : 0;
-    return __sync_bool_compare_and_swap(&this->counter_, e, u);
-  }
-
- private:
-  volatile int32_t counter_;
-};
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_ATOMIC_DEF_H_
+/**
+ * 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 TUBEMQ_CLIENT_ATOMIC_DEF_H_
+#define TUBEMQ_CLIENT_ATOMIC_DEF_H_
+
+#include <stdlib.h>
+
+#include <atomic>
+
+namespace tubemq {
+
+template <class T>
+class Atomic {
+ public:
+  Atomic() : counter_(0) {}
+  explicit Atomic(T initial_value) : counter_(initial_value) {}
+
+  inline T Get() const { return counter_.load(std::memory_order_relaxed); }
+  inline void Set(T new_value) { counter_.store(new_value, std::memory_order_relaxed); }
+
+  inline int64_t LongValue() const {
+    return static_cast<int64_t>(counter_.load(std::memory_order_relaxed));
+  }
+  inline int32_t IntValue() const {
+    return static_cast<int32_t>(counter_.load(std::memory_order_relaxed));
+  }
+
+  inline T GetAndSet(T new_value) { return counter_.exchange(new_value); }
+
+  inline bool CompareAndSet(T expect, T update) {
+    return counter_.compare_exchange_strong(expect, update, std::memory_order_relaxed);
+  }
+
+  // return old value
+  inline T GetAndIncrement() { return counter_.fetch_add(1, std::memory_order_relaxed); }
+
+  inline T GetAndDecrement() { return counter_.fetch_add(-1, std::memory_order_relaxed); }
+
+  inline T GetAndAdd(T delta) { return counter_.fetch_add(delta, std::memory_order_relaxed); }
+
+  // return new value
+  inline T IncrementAndGet() { return AddAndGet(1); }
+
+  inline T DecrementAndGet() { return AddAndGet(-1); }
+
+  inline T AddAndGet(T delta) {
+    return counter_.fetch_add(delta, std::memory_order_relaxed) + delta;
+  }
+
+ private:
+  std::atomic<T> counter_;
+};
+
+using AtomicInteger = Atomic<int32_t>;
+using AtomicLong = Atomic<int64_t>;
+
+class AtomicBoolean {
+ public:
+  AtomicBoolean() : counter_(false) {}
+  explicit AtomicBoolean(bool initial_value) : counter_(initial_value) {}
+
+  inline bool Get() const { return counter_.load(std::memory_order_relaxed); }
+  inline void Set(bool new_value) { counter_.store(new_value, std::memory_order_relaxed); }
+
+  // return old value
+  inline bool GetAndSet(bool new_value) { return counter_.exchange(new_value); }
+
+  // CAS SET
+  inline bool CompareAndSet(bool& expect, bool update) {
+    return counter_.compare_exchange_strong(expect, update, std::memory_order_relaxed);
+  }
+
+ private:
+  std::atomic<bool> counter_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_ATOMIC_DEF_H_


[incubator-tubemq] 07/50: [TUBEMQ-251] Create C/C++ Codec utils

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit f077e0245b42de3bd7c25ea0442d9efcda423ac0
Author: gosonzhang <go...@tencent.com>
AuthorDate: Wed Jul 1 20:12:13 2020 +0800

    [TUBEMQ-251] Create C/C++ Codec utils
---
 .../tubemq-client-cpp/proto/BrokerService.proto    | 136 +++++++++++++++
 .../tubemq-client-cpp/proto/MasterService.proto    | 192 +++++++++++++++++++++
 .../tubemq-client-cpp/proto/RPC.proto              |  62 +++++++
 .../tubemq-client-cpp/proto/readme.md              |  20 +++
 4 files changed, 410 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/proto/BrokerService.proto b/tubemq-client-twins/tubemq-client-cpp/proto/BrokerService.proto
new file mode 100644
index 0000000..d3825bf
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/proto/BrokerService.proto
@@ -0,0 +1,136 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+syntax = "proto2";
+option java_package = "org.apache.tubemq.corebase.protobuf.generated";
+option java_outer_classname = "ClientBroker";
+option java_generate_equals_and_hash = true;
+option java_generic_services = true;
+option optimize_for = SPEED;
+
+
+message TransferedMessage {
+    required int64 messageId = 1;
+    required int32 checkSum = 2;
+    required bytes payLoadData = 3;
+    required int32 flag = 4;
+}
+
+message AuthorizedInfo {
+    required int64 visitAuthorizedToken = 1;
+    optional string authAuthorizedToken = 2;
+}
+
+message SendMessageRequestP2B {
+    required string clientId = 1;
+    required string topicName = 2;
+    required int32 partitionId = 3;
+    required bytes data = 4;
+    required int32 flag = 5;
+    required int32 checkSum = 6;
+    required int32 sentAddr = 7;
+    optional string msgType = 8;
+    optional string msgTime = 9;
+    optional AuthorizedInfo authInfo = 10;
+}
+
+message SendMessageResponseB2P {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    optional bool requireAuth = 4;
+}
+
+message RegisterRequestC2B {
+    required int32 opType = 1;
+    required string clientId = 2;
+    required string groupName = 3;
+    required string topicName = 4;
+    required int32 partitionId = 5;
+    required int32 readStatus = 6;
+    repeated string filterCondStr = 7;
+    optional int64 currOffset = 8;
+    optional string sessionKey = 9;
+    optional int64 sessionTime = 10;
+    optional int64 ssdStoreId = 11;  /* Deprecated  */
+    optional int32 qryPriorityId = 12;
+    optional AuthorizedInfo authInfo = 13;
+}
+
+message RegisterResponseB2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    optional int64 currOffset = 4;
+}
+
+message HeartBeatRequestC2B {
+    required string clientId = 1;
+    required string groupName = 2;
+    required int32 readStatus = 3;
+    /* brokerId:host:port:topic:partitionId:delayTimeStamp */
+    repeated string partitionInfo = 4;
+    optional int64 ssdStoreId = 5;   /* Deprecated  */
+    optional int32 qryPriorityId = 6;
+    optional AuthorizedInfo authInfo = 7;
+}
+
+message HeartBeatResponseB2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    optional bool hasPartFailure = 4;
+    /* failCode:brokerId:host:port:topic:partitionId:delayTimeStamp */
+    repeated string failureInfo = 5;
+    optional bool requireAuth = 6;
+}
+
+message GetMessageRequestC2B {
+    required string clientId = 1;
+    required int32 partitionId = 2;
+    required string groupName = 3;
+    required string topicName = 4;
+    optional bool lastPackConsumed = 5;
+    optional bool manualCommitOffset = 6;
+    optional bool escFlowCtrl = 7;
+}
+
+message GetMessageResponseB2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    optional string errMsg = 3;
+    repeated TransferedMessage messages = 4;
+    optional int64 currOffset = 5;
+    optional int32 minLimitTime = 6;
+    optional bool escFlowCtrl = 7;
+    optional int64 currDataDlt = 8;
+    optional bool requireSlow = 9;
+}
+
+message CommitOffsetRequestC2B {
+    required string clientId = 1;
+    required string topicName = 2;
+    required int32 partitionId = 3;
+    required string groupName = 4;
+    optional bool lastPackConsumed = 5;
+}
+
+message CommitOffsetResponseB2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    optional int64 currOffset=4;
+}
diff --git a/tubemq-client-twins/tubemq-client-cpp/proto/MasterService.proto b/tubemq-client-twins/tubemq-client-cpp/proto/MasterService.proto
new file mode 100644
index 0000000..06d10d2
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/proto/MasterService.proto
@@ -0,0 +1,192 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+syntax = "proto2";
+option java_package = "org.apache.tubemq.corebase.protobuf.generated";
+option java_outer_classname = "ClientMaster";
+option java_generate_equals_and_hash = true;
+option java_generic_services = true;
+option optimize_for = SPEED;
+
+
+message EventProto{
+	optional int64 rebalanceId = 1;
+    optional int32 opType = 2;
+    optional int32 status = 3;
+    /* consumerId@group-brokerId:host:port-topic:partitionId */
+    repeated string subscribeInfo = 4;
+}
+
+message EnableBrokerFunInfo {
+    required bool enableConsumeAuthenticate = 1;
+    required bool enableConsumeAuthorize = 2;
+    required bool enableProduceAuthenticate = 3;
+    required bool enableProduceAuthorize = 4;
+    optional bool enableVisitTokenCheck = 5;
+}
+
+message AuthenticateInfo {
+    required string userName = 1;
+    required int64 timestamp = 2;
+    required int32 nonce = 3;
+    required string othParams = 4;
+    required string signature = 5;
+}
+
+message MasterCertificateInfo {
+    optional AuthenticateInfo authInfo = 1;
+    optional string authorizedToken = 2;
+}
+
+message MasterAuthorizedInfo {
+    required int64 visitAuthorizedToken = 1;
+    optional string authAuthorizedToken = 2;
+}
+
+message MasterBrokerAuthorizedInfo {
+    required string visitAuthorizedToken = 1;
+    optional string authAuthorizedToken = 2;
+}
+
+message RegisterRequestP2M {
+    required string clientId = 1;
+    repeated string topicList = 2;
+    required int64 brokerCheckSum = 3;
+    required string hostName = 4;
+    optional MasterCertificateInfo authInfo = 5;
+    optional string jdkVersion = 6;
+}
+
+message RegisterResponseM2P {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    required int64 brokerCheckSum = 4;
+    repeated string brokerInfos = 5;
+    optional MasterAuthorizedInfo authorizedInfo = 6;
+}
+
+message HeartRequestP2M {
+    required string clientId = 1;
+    required int64 brokerCheckSum = 2;
+    required string hostName = 3;
+    repeated string topicList = 4;
+    optional MasterCertificateInfo authInfo = 5;
+}
+
+message HeartResponseM2P {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    required int64 brokerCheckSum = 4;
+    /* brokerId:host:port-topic:partitionNum */
+    repeated string topicInfos = 5;
+    repeated string brokerInfos = 6;
+    optional bool requireAuth = 7;
+    optional MasterAuthorizedInfo authorizedInfo = 8;
+}
+
+message CloseRequestP2M{
+	required string clientId = 1;
+	optional MasterCertificateInfo authInfo = 2;
+}
+
+message CloseResponseM2P{
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+}
+
+message RegisterRequestC2M {
+    required string clientId = 1;
+    required string groupName = 2;
+    required string hostName = 3;
+    repeated string topicList = 4;
+    /* consumerId@group-brokerId:host:port-topic:partitionId */
+    repeated string subscribeInfo = 5;
+    repeated string  topicCondition = 6;
+    optional bool requireBound = 7;
+    optional int64   sessionTime = 8;
+    optional string  sessionKey = 9;
+    optional int32   totalCount = 10;
+    optional string  requiredPartition = 11;
+    optional bool notAllocated = 12;
+    optional bool selectBig = 13;
+    optional int64 groupFlowCheckId = 14;
+    optional int64 defFlowCheckId = 15;
+    optional int64 ssdStoreId = 16; /* Deprecated  */
+    optional int32 qryPriorityId = 17;
+    optional MasterCertificateInfo authInfo = 18;
+    optional string jdkVersion = 19;
+}
+
+message RegisterResponseM2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    /* brokerId:host:port-topic:partitionNum */
+    repeated string topicInfo = 4;
+    optional bool notAllocated = 5;
+    optional int64 defFlowCheckId = 6;
+    optional string defFlowControlInfo = 7;
+    optional int64 groupFlowCheckId = 8;
+    optional string groupFlowControlInfo = 9;
+    optional int64 ssdStoreId = 10; /* Deprecated  */
+    optional int32 qryPriorityId = 11;
+    optional MasterAuthorizedInfo authorizedInfo = 12;
+}
+
+message HeartRequestC2M {
+    required string clientId = 1;
+    required string groupName = 2;
+    repeated string subscribeInfo = 3;
+    required bool reportSubscribeInfo = 4;
+    optional EventProto event = 5;
+    optional int64 defFlowCheckId = 6;
+    optional int64 groupFlowCheckId = 7;
+    optional int64 ssdStoreId = 8;  /* Deprecated  */
+    optional int32 qryPriorityId = 9;
+    optional MasterCertificateInfo authInfo = 10;
+}
+
+message HeartResponseM2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+    optional EventProto event =4;
+    optional bool notAllocated = 5;
+    optional int64 defFlowCheckId = 6;
+    optional string defFlowControlInfo = 7;
+    optional int64 groupFlowCheckId = 8;
+    optional string groupFlowControlInfo = 9;
+    optional int64 ssdStoreId = 10;   /* Deprecated  */
+    optional int32 qryPriorityId = 11;
+    optional bool requireAuth = 12;
+    optional MasterAuthorizedInfo authorizedInfo = 13;
+}
+
+message CloseRequestC2M {
+    required string clientId = 1;
+    required string groupName = 2;
+	optional MasterCertificateInfo authInfo = 3;
+}
+
+message CloseResponseM2C {
+    required bool success = 1;
+    required int32 errCode = 2;
+    required string errMsg = 3;
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/proto/RPC.proto b/tubemq-client-twins/tubemq-client-cpp/proto/RPC.proto
new file mode 100644
index 0000000..6b4120f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/proto/RPC.proto
@@ -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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+syntax = "proto2";
+option java_package = "org.apache.tubemq.corebase.protobuf.generated";
+option java_outer_classname = "RPCProtos";
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+
+message RpcConnHeader {
+    required int32 flag = 1;
+    optional int64 traceId = 2;
+    optional int64 spanId = 3;
+    optional int64 parentId = 4;
+}
+
+message RequestHeader {
+    optional int32 serviceType = 1;
+    optional int32 protocolVer = 2;
+}
+
+message RequestBody {
+    required int32 method = 1;
+    optional int64 timeout = 2;
+    optional bytes request = 3;
+}
+
+message ResponseHeader {
+    enum Status {
+        SUCCESS = 0;
+        ERROR = 1;
+        FATAL = 2;
+    }
+    required Status status = 1;
+    optional int32 serviceType = 2;
+    optional int32 protocolVer = 3;
+}
+
+message RspExceptionBody {
+    required string exceptionName = 1;
+    optional string stackTrace = 2;
+}
+
+message RspResponseBody {
+    required int32 method = 1;
+    required bytes  data = 2;
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/proto/readme.md b/tubemq-client-twins/tubemq-client-cpp/proto/readme.md
new file mode 100644
index 0000000..7476bae
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/proto/readme.md
@@ -0,0 +1,20 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+
+The proto directory and files are copied from https://github.com/apache/incubator-tubemq/tree/master/tubemq-core/src/main/proto
+
+Corresponding *.cc and *.h files are compiled with protobuf 2.5.0 version, if you need other protobuf version's source code, please directly compile *.proto file.
\ No newline at end of file


[incubator-tubemq] 49/50: [TUBEMQ-350]C++ SDK client code adj (#262)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 1cc56e53bfec60ce7eb5fd9cd4c3d95091463212
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Tue Sep 15 12:26:53 2020 +0800

    [TUBEMQ-350]C++ SDK client code adj (#262)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/src/CMakeLists.txt           |   2 +-
 .../tubemq-client-cpp/src/baseconsumer.cc          |  64 ++--
 .../tubemq-client-cpp/src/client_connection.cc     | 287 ++++++++++++++
 .../tubemq-client-cpp/src/client_connection.h      | 101 +++++
 .../tubemq-client-cpp/src/client_service.h         | 417 ++++++---------------
 .../tubemq-client-cpp/src/client_subinfo.cc        |  55 +--
 .../tubemq-client-cpp/src/client_subinfo.h         |  79 ++++
 .../tubemq-client-cpp/src/connection.h             |  14 +-
 .../tubemq-client-cpp/src/connection_pool.h        |   5 +-
 .../tubemq-client-cpp/src/const_config.h           | 120 ++++++
 .../tubemq-client-cpp/src/const_rpc.h              |  91 +++++
 .../tubemq-client-cpp/src/executor_pool.cc         |  21 +-
 .../tubemq-client-cpp/src/executor_pool.h          |  95 +++++
 .../tubemq-client-cpp/src/file_ini.cc              |  26 +-
 .../tubemq-client-cpp/src/file_ini.h               |  51 +++
 .../tubemq-client-cpp/src/flowctrl_def.cc          | 244 ++++++------
 .../tubemq-client-cpp/src/flowctrl_def.h           | 144 +++++++
 .../tubemq-client-cpp/src/logger.cc                |   9 +-
 tubemq-client-twins/tubemq-client-cpp/src/logger.h | 118 ++++++
 .../tubemq-client-cpp/src/meta_info.cc             | 417 +++++++++++----------
 .../tubemq-client-cpp/src/meta_info.h              | 188 ++++++++++
 .../tubemq-client-cpp/src/noncopyable.h            |  37 ++
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 274 +++++++++-----
 .../tubemq-client-cpp/src/rmt_data_cache.h         | 166 ++++++++
 .../src/{thread_pool.h => singleton.h}             |  60 ++-
 .../tubemq-client-cpp/src/thread_pool.h            |   5 +-
 .../tubemq-client-cpp/src/tubemq_client.cc         | 184 +++++++++
 .../tubemq-client-cpp/src/tubemq_codec.h           |  21 +-
 .../tubemq-client-cpp/src/tubemq_config.cc         | 342 +++++++++--------
 .../tubemq-client-cpp/src/tubemq_message.cc        | 126 ++++---
 .../tubemq-client-cpp/src/tubemq_return.cc         | 123 +++---
 .../tubemq-client-cpp/src/unique_seq_id.h          |  41 ++
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc | 238 +++++++++++-
 tubemq-client-twins/tubemq-client-cpp/src/utils.h  |  71 ++++
 .../tubemq-client-cpp/src/version.h                |  33 ++
 35 files changed, 3115 insertions(+), 1154 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
index 5995d5c..938ca2a 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
@@ -23,7 +23,7 @@ cmake_minimum_required (VERSION 3.1)
 
 AUX_SOURCE_DIRECTORY(. CURRENT_DIR_SRCS)                                        
 ADD_LIBRARY(tubemq STATIC ${CURRENT_DIR_SRCS})   
-TARGET_LINK_LIBRARIES (tubemq)
+TARGET_LINK_LIBRARIES (tubemq libprotobuf.a liblog4cplus.a tubemq_proto)
 
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
index 215892d..714b353 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
@@ -137,7 +137,7 @@ bool BaseConsumer::GetMessage(ConsumerResult& result) {
   string err_info;
   PartitionExt partition_ext;
   string confirm_context;
-  
+
   if (!IsConsumeReady(result)) {
     return false;
   }
@@ -146,7 +146,9 @@ bool BaseConsumer::GetMessage(ConsumerResult& result) {
     result.SetFailureResult(error_code, err_info);
     return false;
   }
-  long curr_offset = tb_config::kInvalidValue;
+
+  int64_t curr_offset = tb_config::kInvalidValue;
+
   bool filter_consume = sub_info_.IsFilterConsume(partition_ext.GetTopic());
   PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
     partition_ext.GetPartitionKey(), curr_offset);
@@ -206,9 +208,11 @@ bool BaseConsumer::IsConsumeReady(ConsumerResult& result) {
     if (err_code::kErrSuccess == ret_code) {
       return true;
     }
-    if ((config_.GetMaxPartCheckPeriodMs() > 0)
-      && (Utils::GetCurrentTimeMillis() - start_time 
-      > config_.GetMaxPartCheckPeriodMs())) {
+
+    if ((config_.GetMaxPartCheckPeriodMs() >= 0)
+      && (Utils::GetCurrentTimeMillis() - start_time
+      >= config_.GetMaxPartCheckPeriodMs())) {
+
       switch (ret_code) {
         case err_code::kErrNoPartAssigned: {
           result.SetFailureResult(ret_code,
@@ -277,7 +281,9 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
   string part_key = Utils::Trim(confirm_context.substr(0, pos1));
   string booked_time_str =
       Utils::Trim(confirm_context.substr(pos1 + token1.size(), confirm_context.size()));
-  long booked_time = atol(booked_time_str.c_str());
+
+  int64_t booked_time = atol(booked_time_str.c_str());
+
   pos1 = part_key.find(token2);
   if (string::npos == pos1) {
     result.SetFailureResult(err_code::kErrBadRequest,
@@ -305,7 +311,9 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
                             "Not found the partition by confirm_context!");
     return false;
   }
-  long curr_offset = tb_config::kInvalidValue;
+
+  int64_t curr_offset = tb_config::kInvalidValue;
+
   PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
     partition_ext.GetPartitionKey(), curr_offset);
   auto request = std::make_shared<RequestContext>();
@@ -344,7 +352,9 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
     if (rsp->success_) {
       CommitOffsetResponseB2C rsp_b2c;
       ret_result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
-                                          (int)(rsp->rsp_body_.data().length()));
+
+                                          (int32_t)(rsp->rsp_body_.data().length()));
+
       if (ret_result) {
         if (rsp_b2c.success()) {
           curr_offset = rsp_b2c.curroffset();
@@ -439,7 +449,8 @@ bool BaseConsumer::register2Master(int32_t& error_code, string& err_info, bool n
       error_code = error.Value();
       err_info = error.Message();
     }
-    if (error_code == err_code::kErrConsumeGroupForbidden 
+
+    if (error_code == err_code::kErrConsumeGroupForbidden
       || error_code == err_code::kErrConsumeContentForbidden) {
       // set regist process status to existed
       master_reg_status_.CompareAndSet(1, 0);
@@ -805,7 +816,8 @@ void BaseConsumer::processHeartBeat2Broker(NodeInfo broker_info) {
           if (rsp->success_) {
             HeartBeatResponseB2C rsp_b2c;
             bool result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
-                                                 (int)(rsp->rsp_body_.data().length()));
+                                                (int32_t)(rsp->rsp_body_.data().length()));
+
             if (result) {
               set<string> partition_keys;
               if (rsp_b2c.success()) {
@@ -1066,7 +1078,8 @@ bool BaseConsumer::processRegisterResponseM2C(int32_t& error_code, string& err_i
   }
   RegisterResponseM2C rsp_m2c;
   bool result = rsp_m2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
-                                       (int)(rsp_protocol->rsp_body_.data().length()));
+                                       (int32_t)(rsp_protocol->rsp_body_.data().length()));
+
   if (!result) {
     error_code = err_code::kErrParseFailure;
     err_info = "Parse RegisterResponseM2C response failure!";
@@ -1223,7 +1236,8 @@ bool BaseConsumer::processRegResponseB2C(int32_t& error_code, string& err_info,
   }
   RegisterResponseB2C rsp_b2c;
   bool result = rsp_b2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
-                                       (int)(rsp_protocol->rsp_body_.data().length()));
+                                       (int32_t)(rsp_protocol->rsp_body_.data().length()));
+
   if (!result) {
     error_code = err_code::kErrParseFailure;
     err_info = "Parse RegisterResponseB2C response failure!";
@@ -1259,10 +1273,8 @@ void BaseConsumer::convertMessages(int32_t& msg_size, list<Message>& message_lis
     int32_t payload_length = tsfMsg.payloaddata().length();
     int32_t calc_checksum = Utils::Crc32(tsfMsg.payloaddata());
     if (in_check_sum != calc_checksum) {
-
       LOG_TRACE("[CONSUMER] convertMessages [%d], Crc32 failure, in=%d, calc=%d, client=%s",
         i, in_check_sum, calc_checksum, client_uuid_.c_str());
-
       continue;
     }
     int read_pos = 0;
@@ -1272,10 +1284,8 @@ void BaseConsumer::convertMessages(int32_t& msg_size, list<Message>& message_lis
     memcpy(&payload_data[0], tsfMsg.payloaddata().c_str(), payload_length);
     if ((flag & tb_config::kMsgFlagIncProperties) == 1) {
       if (payload_length < 4) {
-
         LOG_TRACE("[CONSUMER] convertMessages [%d], payload_length(%d) < 4, client=%s",
           i, payload_length, client_uuid_.c_str());
-
         continue;
       }
       int32_t attr_len = ntohl(*(int*)(&payload_data[0]));
@@ -1322,6 +1332,8 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
                                              bool filter_consume, const PartitionExt& partition_ext,
                                              const string& confirm_context,
                                              const TubeMQCodec::RspProtocolPtr& rsp) {
+
+  // #lizard forgives
   string err_info;
 
   if (!rsp->success_) {
@@ -1334,8 +1346,9 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
     return false;
   }
   GetMessageResponseB2C rsp_b2c;
-  bool ret_result =
-      rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(), (int)(rsp->rsp_body_.data().length()));
+  bool ret_result = rsp_b2c.ParseFromArray(
+    rsp->rsp_body_.data().c_str(), (int32_t)(rsp->rsp_body_.data().length()));
+
   if (!ret_result) {
     rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false);
     result.SetFailureResult(err_code::kErrServerError,
@@ -1343,16 +1356,18 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
                             partition_ext.GetTopic(), peer_info);
 
     LOG_TRACE("[CONSUMER] processGetMessageRspB2C parse failure, client=%s", client_uuid_.c_str());
-
     return false;
   }
 
   switch (rsp_b2c.errcode()) {
     case err_code::kErrSuccess: {
       bool esc_limit = (rsp_b2c.has_escflowctrl() && rsp_b2c.escflowctrl());
-      long data_dltval =
+
+      int64_t data_dltval =
           rsp_b2c.has_currdatadlt() ? rsp_b2c.currdatadlt() : tb_config::kInvalidValue;
-      long curr_offset = rsp_b2c.has_curroffset() ? rsp_b2c.curroffset() : tb_config::kInvalidValue;
+      int64_t curr_offset = rsp_b2c.has_curroffset() ?
+        rsp_b2c.curroffset() : tb_config::kInvalidValue;
+
       bool req_slow = rsp_b2c.has_requireslow() ? rsp_b2c.requireslow() : false;
       int msg_size = 0;
       list<Message> message_list;
@@ -1377,7 +1392,8 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
 
     case err_code::kErrConsumeSpeedLimit: {
       // Process with server side speed limit
-      long def_dlttime = rsp_b2c.has_minlimittime() ? rsp_b2c.minlimittime()
+
+      int64_t def_dlttime = rsp_b2c.has_minlimittime() ? rsp_b2c.minlimittime()
                                                     : config_.GetMsgNotFoundWaitPeriodMs();
       rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false,
                                   tb_config::kInvalidValue, rsp_b2c.errcode(), false, 0,
@@ -1393,7 +1409,7 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
     case err_code::kErrServiceUnavilable:
     default: {
       // Slow down the request based on the limitation configuration when meet these errors
-      long limit_dlt = 300;
+      int64_t limit_dlt = 300;
       switch (rsp_b2c.errcode()) {
         case err_code::kErrForbidden: {
           limit_dlt = 2000;
@@ -1455,8 +1471,6 @@ int32_t BaseConsumer::getConsumeReadStatus(bool is_first_reg) {
       LOG_INFO("[Consumer From Max Offset Always], clientId=%s", client_uuid_.c_str());
     }
   }
-  LOG_INFO("[getConsumeReadStatus], readStatus=%d, is_first_reg=%d, config_.GetConsumePosition()=%d",
-    readStatus, is_first_reg, config_.GetConsumePosition());
   return readStatus;
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
new file mode 100644
index 0000000..c795d35
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
@@ -0,0 +1,287 @@
+/**
+ * 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 "client_connection.h"
+
+using namespace tubemq;
+
+void ClientConnection::AsyncWrite(RequestContextPtr& req) {
+  auto self = shared_from_this();
+  executor_->Post([self, this, req]() {
+    if (request_list_.find(req->request_id_) != request_list_.end()) {
+      LOG_ERROR("Write requestid[%d] is repeat", req->request_id_);
+      return;
+    }
+    auto& transport_req = request_list_[req->request_id_];
+    transport_req.req_ = req;
+    bool queue_empty = write_queue_.empty();
+    write_queue_.push_back(req->request_id_);
+    if (req->timeout_ > 0) {
+      transport_req.deadline_ = executor_->CreateSteadyTimer();
+      transport_req.deadline_->expires_after(std::chrono::milliseconds(req->timeout_));
+      transport_req.deadline_->async_wait(std::bind(&ClientConnection::requestTimeoutHandle,
+                                                    shared_from_this(), std::placeholders::_1,
+                                                    transport_req.req_));
+    }
+    if (IsConnected() && queue_empty) {
+      asyncWrite();
+    }
+  });
+}
+
+void ClientConnection::Close() {
+  auto self = shared_from_this();
+  executor_->Post([self, this]() { close(); });
+}
+
+void ClientConnection::requestTimeoutHandle(const std::error_code& ec, RequestContextPtr req) {
+  if (ec) {
+    return;
+  }
+  auto request_id = req->request_id_;
+  auto err = ErrorCode(err_code::kErrNetWorkTimeout, "Request is timeout");
+  requestCallback(request_id, &err);
+}
+
+void ClientConnection::close(const std::error_code* err) {
+  if (IsStop()) {
+    return;
+  }
+  status_ = kDisconnected;
+  LOG_INFO("%scloseed", ToString().c_str());
+  socket_->close();
+  if (notifier_ != nullptr) {
+    notifier_(err);
+  }
+  releaseAllRequest(err);
+}
+
+void ClientConnection::releaseAllRequest(const std::error_code* err) {
+  std::string msg = "connect close ";
+  if (err != nullptr) {
+    msg += "error_code, value:";
+    msg += std::to_string(err->value());
+    msg += ", msg:";
+    msg += err->message();
+    msg += ", category:";
+    msg += err->category().name();
+  }
+
+  auto terr = ErrorCode(err_code::kErrNetworkError, msg);
+  for (auto& it : request_list_) {
+    it.second.req_->promise_.SetFailed(terr);
+    it.second.deadline_->cancel();
+  }
+  request_list_.clear();
+  write_queue_.clear();
+  recv_buffer_->Reset();
+}
+
+void ClientConnection::connect(const asio::ip::tcp::resolver::results_type& endpoints) {
+  if (IsStop()) {
+    return;
+  }
+  status_ = kConnecting;
+  deadline_->expires_after(std::chrono::milliseconds(kConnnectMaxTimeMs));
+  deadline_->async_wait(std::bind(&ClientConnection::checkDeadline, this, std::placeholders::_1));
+  asio::async_connect(
+      *socket_, endpoints, [this](std::error_code ec, asio::ip::tcp::endpoint endpoint) {
+        deadline_->cancel();
+        if (ec) {
+          status_ = kDisconnected;
+          LOG_ERROR("%s[%s:%d]async connect error:%d, %s, %s", ToString().c_str(), ip_.c_str(),
+                    port_, ec.value(), ec.message().c_str(), ec.category().name());
+          close(&ec);
+          return;
+        }
+        status_ = kConnected;
+        socket_->set_option(asio::ip::tcp::no_delay(true));
+        // socket_->set_option(asio::ip::tcp::socket::reuse_address(true));
+        contextString();
+        LOG_INFO("%sis connected", ToString().c_str());
+
+        asyncWrite();
+        asyncRead();
+      });
+}
+
+void ClientConnection::checkDeadline(const std::error_code& ec) {
+  if (ec) {
+    return;
+  }
+  if (IsStop()) {
+    return;
+  }
+  LOG_ERROR("%s connect timeout", ToString().c_str());
+  close();
+}
+
+void ClientConnection::contextString() {
+  std::stringstream stream;
+  stream << "[" << socket_->local_endpoint() << " -> " << socket_->remote_endpoint() << "] ";
+  context_string_ += stream.str();
+}
+
+void ClientConnection::asyncRead() {
+  if (IsStop()) {
+    return;
+  }
+  if (recv_buffer_->capacity() > rpc_config::kRpcRecvBufferMaxBytes) {
+    LOG_ERROR("%sbuffer capacity over config:%d", ToString().c_str(),
+              rpc_config::kRpcRecvBufferMaxBytes);
+    close();
+    return;
+  }
+  recv_buffer_->EnsureWritableBytes(rpc_config::kRpcEnsureWriteableBytes);
+  auto self = shared_from_this();
+  socket_->async_read_some(
+      asio::buffer(recv_buffer_->WriteBegin(), recv_buffer_->WritableBytes()),
+      [self, this](std::error_code ec, std::size_t len) {
+        if (ec) {
+          LOG_ERROR("[%s]async read error:%d, %s, %s", ToString().c_str(), ec.value(),
+                    ec.message().c_str(), ec.category().name());
+          close(&ec);
+          return;
+        }
+        if (len == 0) {
+          LOG_ERROR("[%s]async read zero", ToString().c_str());
+          close(&ec);
+          return;
+        }
+        recv_time_ = std::time(nullptr);
+        recv_buffer_->WriteBytes(len);
+        checkPackageDone();
+        LOG_TRACE("[%s]async read done, len:%ld, package_length_:%ld, recvbuffer:%s",
+                  ToString().c_str(), len, package_length_, recv_buffer_->String().c_str());
+        asyncRead();
+      });
+}
+
+void ClientConnection::checkPackageDone() {
+  if (check_ == nullptr) {
+    recv_buffer_->Reset();
+    LOG_ERROR("check codec func not set");
+    return;
+  }
+  if (package_length_ > recv_buffer_->length()) {
+    return;
+  }
+  uint32_t request_id = 0;
+  bool has_request_id = false;
+  Any check_out;
+  auto buff = recv_buffer_->Slice();
+  size_t package_length = 0;
+  auto result = check_(buff, check_out, request_id, has_request_id, package_length);
+  if (result < 0) {
+    package_length_ = 0;
+    LOG_ERROR("%s, check codec package result:%d", ToString().c_str(), result);
+    close();
+    return;
+  }
+  if (result == 0) {
+    package_length_ = package_length;
+    return;
+  }
+  ++read_package_number_;
+  package_length_ = 0;
+  recv_buffer_->Skip(result);
+  if (!has_request_id) {
+    auto it = request_list_.begin();
+    if (it == request_list_.end()) {
+      LOG_ERROR("%s, not find request", ToString().c_str());
+      return;
+    }
+    requestCallback(it->first, nullptr, &check_out);
+    return;
+  }
+  requestCallback(request_id, nullptr, &check_out);
+}
+
+void ClientConnection::requestCallback(uint32_t request_id, ErrorCode* err, Any* check_out) {
+  auto it = request_list_.find(request_id);
+  if (it == request_list_.end()) {
+    LOG_INFO("%srequest[%d] not find from request_list_", ToString().c_str(), request_id);
+    return;
+  }
+  auto req = &it->second;
+  req->deadline_->cancel();
+  if (err != nullptr) {
+    LOG_ERROR("%srequest[%d] error:%d, msg:%s", ToString().c_str(), request_id, err->Value(),
+              err->Message().c_str());
+    req->req_->promise_.SetFailed(*err);
+    request_list_.erase(it);
+    return;
+  }
+  if (check_out != nullptr) {
+    ResponseContext rsp;
+    BufferPtr* buff = any_cast<BufferPtr>(check_out);
+    if (buff != nullptr) {
+      req->req_->codec_->Decode(*buff, rsp.rsp_);
+    } else {
+      rsp.rsp_ = *check_out;
+    }
+    req->req_->promise_.SetValue(rsp);
+  } else {
+    req->req_->promise_.SetFailed(ErrorCode(err_code::kErrNetworkError, "response is null"));
+  }
+  request_list_.erase(it);
+}
+
+TransportRequest* ClientConnection::nextTransportRequest() {
+  uint32_t request_id;
+  TransportRequest* transport_req = nullptr;
+  while (!write_queue_.empty()) {
+    request_id = write_queue_.front();
+    write_queue_.pop_front();
+    auto it = request_list_.find(request_id);
+    if (it == request_list_.end()) {
+      continue;
+    }
+    transport_req = &it->second;
+    break;
+  }
+  return transport_req;
+}
+
+void ClientConnection::asyncWrite() {
+  if (IsStop()) {
+    return;
+  }
+  auto transport_req = nextTransportRequest();
+  if (transport_req == nullptr) {
+    return;
+  }
+  auto self = shared_from_this();
+  auto& req = transport_req->req_;
+  asio::async_write(
+      *socket_,
+      asio::buffer(transport_req->req_->buf_->data(), transport_req->req_->buf_->length()),
+      [self, this, req](std::error_code ec, std::size_t length) {
+        if (ec) {
+          close(&ec);
+          LOG_ERROR("[%s]async write error:%d, message:%s, category:%s", ToString().c_str(),
+                    ec.value(), ec.message().c_str(), ec.category().name());
+          return;
+        }
+        ++write_package_number_;
+        LOG_TRACE("[%s]async write done, request_id:%d", ToString().c_str(), req->request_id_);
+        asyncWrite();
+      });
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
new file mode 100644
index 0000000..b6c7a36
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
@@ -0,0 +1,101 @@
+/**
+ * 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 _TUBEMQ_CLIENT_CONNECTION_
+#define _TUBEMQ_CLIENT_CONNECTION_
+
+#include <stdlib.h>
+
+#include <chrono>
+#include <deque>
+#include <exception>
+#include <functional>
+#include <iostream>
+#include <list>
+#include <memory>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "asio.hpp"
+#include "connection.h"
+#include "const_rpc.h"
+#include "executor_pool.h"
+#include "logger.h"
+#include "noncopyable.h"
+#include "transport.h"
+
+namespace tubemq {
+
+struct TransportRequest {
+  RequestContextPtr req_;
+  SteadyTimerPtr deadline_;
+};
+
+class ClientConnection : public Connection, public std::enable_shared_from_this<ClientConnection> {
+ public:
+  // executor: ExecutorPool.Get()
+  // endpoints: executor->CreateTcpResolver()->resolve("ip", port);
+  ClientConnection(ExecutorPtr& executor, const std::string& ip, uint16_t port)
+      : Connection(ip, port),
+        executor_(executor),
+        socket_(std::move(executor->CreateTcpSocket())),
+        recv_buffer_(std::make_shared<Buffer>(rpc_config::kRpcConnectInitBufferSize)),
+        deadline_(std::move(executor->CreateSteadyTimer())) {
+    auto endpoints = executor_->CreateTcpResolver()->resolve(ip_, std::to_string(port_));
+    connect(endpoints);
+  }
+
+  virtual ~ClientConnection() {}
+
+  virtual void AsyncWrite(RequestContextPtr& req);
+
+  virtual void Close();
+
+ private:
+  void requestTimeoutHandle(const std::error_code& ec, RequestContextPtr req);
+
+  void close(const std::error_code* err = nullptr);
+
+  void releaseAllRequest(const std::error_code* err = nullptr);
+  void connect(const asio::ip::tcp::resolver::results_type& endpoints);
+  void checkDeadline(const std::error_code& ec);
+  void contextString();
+  void asyncRead();
+  void checkPackageDone();
+  void requestCallback(uint32_t request_id, ErrorCode* err = nullptr, Any* check_out = nullptr);
+  TransportRequest* nextTransportRequest();
+  void asyncWrite();
+
+ private:
+  using BufferQueue = std::deque<uint32_t>;
+  static const uint32_t kConnnectMaxTimeMs{1000 * 20};  // ms
+  static const uint32_t kReadMaxTimeMs{1000 * 30};      // ms
+  ExecutorPtr executor_;
+  TcpSocketPtr socket_;
+  BufferPtr recv_buffer_;
+  BufferQueue write_queue_;
+  std::unordered_map<uint32_t, TransportRequest> request_list_;  // requestid->request context
+  SteadyTimerPtr deadline_;
+};
+using ClientConnectionPtr = std::shared_ptr<ClientConnection>;
+
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_CLIENT_CONNECTION_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.h b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
index c6e0aaa..11f5b42 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
@@ -1,296 +1,121 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#include "client_service.h"
-
-#include <sstream>
-
-#include "const_config.h"
-#include "logger.h"
-#include "utils.h"
-
-namespace tubemq {
-
-using std::lock_guard;
-using std::stringstream;
-
-BaseClient::BaseClient(bool is_producer) {
-  is_producer_ = is_producer;
-  client_index_ = tb_config::kInvalidValue;
-}
-
-BaseClient::~BaseClient() {
-  // no code
-}
-
-TubeMQService* TubeMQService::_instance = NULL;
-
-static mutex tubemq_mutex_service_;
-
-TubeMQService* TubeMQService::Instance() {
-  if (NULL == _instance) {
-    lock_guard<mutex> lck(tubemq_mutex_service_);
-    if (NULL == _instance) {
-      _instance = new TubeMQService;
-    }
-  }
-  return _instance;
-}
-
-TubeMQService::TubeMQService()
-    : timer_executor_(std::make_shared<ExecutorPool>(2)),
-      network_executor_(std::make_shared<ExecutorPool>(4)) {
-  service_status_.Set(0);
-  client_index_base_.Set(0);
-  last_check_time_ = 0;
-}
-
-TubeMQService::~TubeMQService() {
-  string err_info;
-  Stop(err_info);
-}
-
-bool TubeMQService::Start(string& err_info, string conf_file) {
-  // check configure file
-  bool result = false;
-  Fileini fileini;
-  string sector = "TubeMQ";
-
-  result = Utils::ValidConfigFile(err_info, conf_file);
-  if (!result) {
-    return result;
-  }
-  result = fileini.Loadini(err_info, conf_file);
-  if (!result) {
-    return result;
-  }
-  result = Utils::GetLocalIPV4Address(err_info, local_host_);
-  if (!result) {
-    return result;
-  }
-  if (!service_status_.CompareAndSet(0, 1)) {
-    err_info = "TubeMQ Service has startted or Stopped!";
-    return false;
-  }
-  iniLogger(fileini, sector);
-  iniPoolThreads(fileini, sector);
-  iniXfsThread(fileini, sector);
-  service_status_.Set(2);
-  err_info = "Ok!";
-  LOG_INFO("[TubeMQService] TubeMQ service startted!");
-
-  return true;
-}
-
-bool TubeMQService::Stop(string& err_info) {
-  if (service_status_.CompareAndSet(2, -1)) {
-    LOG_INFO("[TubeMQService] TubeMQ service begin to stop!");
-    if (dns_xfs_thread_.joinable()) {
-      dns_xfs_thread_.join();
-    }
-    shutDownClinets();
-    timer_executor_->Close();
-    network_executor_->Close();
-    connection_pool_ = nullptr;
-    thread_pool_ = nullptr;
-    LOG_INFO("[TubeMQService] TubeMQ service stopped!");
-  }
-  err_info = "OK!";
-  return true;
-}
-
-bool TubeMQService::IsRunning() { return (service_status_.Get() == 2); }
-
-void TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
-  string err_info;
-  int32_t log_num = 10;
-  int32_t log_size = 10;
-  int32_t log_level = 4;
-  string log_path = "../log/tubemq";
-  fileini.GetValue(err_info, sector, "log_num", log_num, 10);
-  fileini.GetValue(err_info, sector, "log_size", log_size, 100);
-  fileini.GetValue(err_info, sector, "log_path", log_path, "../log/tubemq");
-  fileini.GetValue(err_info, sector, "log_level", log_level, 4);
-  log_level = TUBEMQ_MID(log_level, 4, 0);
-  GetLogger().Init(log_path, Logger::Level(log_level), log_size, log_num);
-}
-
-void TubeMQService::iniXfsThread(const Fileini& fileini, const string& sector) {
-  string err_info;
-  int32_t dns_xfs_period_ms = 30 * 1000;
-  fileini.GetValue(err_info, sector, "dns_xfs_period_ms", dns_xfs_period_ms, 30 * 1000);
-  TUBEMQ_MID(dns_xfs_period_ms, tb_config::kMaxIntValue, 10000);
-  dns_xfs_thread_ = std::thread(&TubeMQService::thread_task_dnsxfs, this, dns_xfs_period_ms);
-}
-
-void TubeMQService::iniPoolThreads(const Fileini& fileini, const string& sector) {
-  string err_info;
-  int32_t timer_threads = 2;
-  int32_t network_threads = 4;
-  int32_t signal_threads = 8;
-  fileini.GetValue(err_info, sector, "timer_threads", timer_threads, 2);
-  TUBEMQ_MID(timer_threads, 50, 2);
-  fileini.GetValue(err_info, sector, "network_threads", network_threads, 4);
-  TUBEMQ_MID(network_threads, 50, 4);
-  fileini.GetValue(err_info, sector, "signal_threads", signal_threads, 8);
-  TUBEMQ_MID(signal_threads, 50, 4);
-  timer_executor_->Resize(timer_threads);
-  network_executor_->Resize(network_threads);
-  thread_pool_ = std::make_shared<ThreadPool>(signal_threads);
-  connection_pool_ = std::make_shared<ConnectionPool>(network_executor_);
-}
-
-int32_t TubeMQService::GetClientObjCnt() {
-  lock_guard<mutex> lck(mutex_);
-  return clients_map_.size();
-}
-
-bool TubeMQService::AddClientObj(string& err_info, BaseClient* client_obj) {
-  if (!IsRunning()) {
-    err_info = "Service not startted!";
-    return false;
-  }
-  int32_t client_index = client_index_base_.IncrementAndGet();
-  lock_guard<mutex> lck(mutex_);
-  clients_map_[client_index] = client_obj;
-  client_obj->SetClientIndex(client_index);
-  err_info = "Ok";
-  return true;
-}
-
-BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
-  BaseClient* client_obj = NULL;
-  map<int32_t, BaseClient*>::const_iterator it;
-
-  lock_guard<mutex> lck(mutex_);
-  it = clients_map_.find(client_index);
-  if (it != clients_map_.end()) {
-    client_obj = it->second;
-  }
-  return client_obj;
-}
-
-void TubeMQService::RmvClientObj(BaseClient* client_obj) {
-  map<int32_t, BaseClient*>::iterator it;
-  if (client_obj != NULL) {
-    lock_guard<mutex> lck(mutex_);
-    int32_t client_index = client_obj->GetClientIndex();
-    clients_map_.erase(client_index);
-    client_obj->SetClientIndex(tb_config::kInvalidValue);
-  }
-}
-
-void TubeMQService::shutDownClinets() const {
-  map<int32_t, BaseClient*>::const_iterator it;
-  lock_guard<mutex> lck(mutex_);
-  for (it = clients_map_.begin(); it != clients_map_.end(); it++) {
-    it->second->ShutDown();
-  }
-}
-
-bool TubeMQService::AddMasterAddress(string& err_info, const string& master_info) {
-  map<string, int32_t>::iterator it;
-  map<string, int32_t> tmp_addr_map;
-  Utils::Split(master_info, tmp_addr_map, delimiter::kDelimiterComma, delimiter::kDelimiterColon);
-  if (tmp_addr_map.empty()) {
-    err_info = "Illegal parameter: master_info is blank!";
-    return false;
-  }
-  for (it = tmp_addr_map.begin(); it != tmp_addr_map.end();) {
-    if (!Utils::NeedDnsXfs(it->first)) {
-      tmp_addr_map.erase(it++);
-    }
-  }
-  if (tmp_addr_map.empty()) {
-    err_info = "Ok";
-    return true;
-  }
-  if (addNeedDnsXfsAddr(tmp_addr_map)) {
-    updMasterAddrByDns();
-  }
-  err_info = "Ok";
-  return true;
-}
-
-void TubeMQService::GetXfsMasterAddress(const string& source, string& target) {
-  target = source;
-  lock_guard<mutex> lck(mutex_);
-  if (master_source_.find(source) != master_source_.end()) {
-    target = master_target_[source];
-  }
-}
-
-void TubeMQService::thread_task_dnsxfs(int dns_xfs_period_ms) {
-  LOG_INFO("[TubeMQService] DSN transfer thread startted!");
-  while (true) {
-    if (TubeMQService::Instance()->GetServiceStatus() <= 0) {
-      break;
-    }
-    if ((Utils::GetCurrentTimeMillis() - last_check_time_) >= dns_xfs_period_ms) {
-      TubeMQService::Instance()->updMasterAddrByDns();
-      last_check_time_ = Utils::GetCurrentTimeMillis();
-    }
-    std::this_thread::sleep_for(std::chrono::milliseconds(500));
-  }
-  LOG_INFO("[TubeMQService] DSN transfer thread stopped!");
-}
-
-bool TubeMQService::hasXfsTask(map<string, int32_t>& src_addr_map) {
-  lock_guard<mutex> lck(mutex_);
-  if (!master_source_.empty()) {
-    src_addr_map = master_source_;
-    return true;
-  }
-  return false;
-}
-
-bool TubeMQService::addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map) {
-  bool added = false;
-  map<string, int32_t>::iterator it;
-  if (!src_addr_map.empty()) {
-    lock_guard<mutex> lck(mutex_);
-    for (it = src_addr_map.begin(); it != src_addr_map.end(); it++) {
-      if (master_source_.find(it->first) == master_source_.end()) {
-        added = true;
-        master_source_[it->first] = it->second;
-      }
-    }
-  }
-  return added;
-}
-
-void TubeMQService::updMasterAddrByDns() {
-  map<string, int32_t> tmp_src_addr_map;
-  map<string, string> tmp_tgt_addr_map;
-  map<string, int32_t>::iterator it;
-  if (!hasXfsTask(tmp_src_addr_map)) {
-    return;
-  }
-  Utils::XfsAddrByDns(tmp_src_addr_map, tmp_tgt_addr_map);
-  lock_guard<mutex> lck(mutex_);
-  if (tmp_tgt_addr_map.empty()) {
-    for (it = tmp_src_addr_map.begin(); it != tmp_src_addr_map.end(); it++) {
-      master_target_[it->first] = it->first;
-    }
-  } else {
-    master_target_ = tmp_tgt_addr_map;
-  }
-}
-
-}  // namespace tubemq
+/**
+ * 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 TUBEMQ_CLIENT_BASE_CLIENT_H_
+#define TUBEMQ_CLIENT_BASE_CLIENT_H_
+
+#include <stdint.h>
+
+#include <map>
+#include <mutex>
+#include <string>
+#include <thread>
+
+#include "connection_pool.h"
+#include "file_ini.h"
+#include "noncopyable.h"
+#include "rmt_data_cache.h"
+#include "thread_pool.h"
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
+namespace tubemq {
+
+using std::map;
+using std::mutex;
+using std::string;
+using std::thread;
+
+class BaseClient {
+ public:
+  explicit BaseClient(bool is_producer);
+  virtual ~BaseClient();
+  virtual void ShutDown() {}
+  void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
+  bool IsProducer() { return is_producer_; }
+  const int32_t GetClientIndex() { return client_index_; }
+
+ protected:
+  bool is_producer_;
+  int32_t client_index_;
+};
+
+class TubeMQService : public noncopyable {
+ public:
+  static TubeMQService* Instance();
+  bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
+  bool Stop(string& err_info);
+  bool IsRunning();
+  const int32_t GetServiceStatus() const { return service_status_.Get(); }
+  int32_t GetClientObjCnt();
+  bool AddClientObj(string& err_info, BaseClient* client_obj);
+  BaseClient* GetClientObj(int32_t client_index) const;
+  void RmvClientObj(BaseClient* client_obj);
+  const string& GetLocalHost() const { return local_host_; }
+  ExecutorPoolPtr GetTimerExecutorPool() { return timer_executor_; }
+  SteadyTimerPtr CreateTimer() { return timer_executor_->Get()->CreateSteadyTimer(); }
+  ExecutorPoolPtr GetNetWorkExecutorPool() { return network_executor_; }
+  ConnectionPoolPtr GetConnectionPool() { return connection_pool_; }
+  template <class function>
+  void Post(function f) {
+    if (thread_pool_ != nullptr) {
+      thread_pool_->Post(f);
+    }
+  }
+  bool AddMasterAddress(string& err_info, const string& master_info);
+  void GetXfsMasterAddress(const string& source, string& target);
+
+ protected:
+  void updMasterAddrByDns();
+
+ private:
+  TubeMQService();
+  ~TubeMQService();
+  void iniLogger(const Fileini& fileini, const string& sector);
+  void iniPoolThreads(const Fileini& fileini, const string& sector);
+  void iniXfsThread(const Fileini& fileini, const string& sector);
+  void thread_task_dnsxfs(int dns_xfs_period_ms);
+  void shutDownClinets() const;
+  bool hasXfsTask(map<string, int32_t>& src_addr_map);
+  bool addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map);
+
+ private:
+  static TubeMQService* _instance;
+  string local_host_;
+  AtomicInteger service_status_;
+  AtomicInteger client_index_base_;
+  mutable mutex mutex_;
+  map<int32_t, BaseClient*> clients_map_;
+  ExecutorPoolPtr timer_executor_;
+  ExecutorPoolPtr network_executor_;
+  ConnectionPoolPtr connection_pool_;
+  std::shared_ptr<ThreadPool> thread_pool_;
+  thread dns_xfs_thread_;
+  mutable mutex dns_mutex_;
+  int64_t last_check_time_;
+  map<string, int32_t> master_source_;
+  map<string, string> master_target_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_BASE_CLIENT_H_
\ No newline at end of file
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
index a6c0fc1..8df82fd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
@@ -17,56 +17,15 @@
  * under the License.
  */
 
-#include "tubemq/client_subinfo.h"
-#include "tubemq/const_config.h"
-#include "tubemq/utils.h"
+#include "client_subinfo.h"
 
+#include "const_config.h"
+#include "utils.h"
 
 
-namespace tubemq {
-
-
-MasterAddrInfo::MasterAddrInfo() {
-  curr_master_addr_ = "";
-  master_addr_.clear();
-}
-
-bool MasterAddrInfo::InitMasterAddress(string& err_info, 
-                                       const string& master_info) {
-  master_addr_.clear();
-  Utils::Split(master_info, master_addr_, delimiter::kDelimiterComma,
-               delimiter::kDelimiterColon);
-  if (master_addr_.empty()) {
-    err_info = "Illegal parameter: master_info is blank!";
-    return false;
-  }
-
-  map<string, int32_t>::iterator it = master_addr_.begin();
-  curr_master_addr_ = it->first;
-  err_info = "Ok";
-  return true;
-}
 
-void MasterAddrInfo::GetNextMasterAddr(string& ipaddr, int32_t& port) {
-  map<string, int32_t>::iterator it;
-  it = master_addr_.find(curr_master_addr_);
-  if(it != master_addr_.end()) {
-    it++;
-    if (it == master_addr_.end()) {
-      it = master_addr_.begin();
-    }
-  } else {
-    it = master_addr_.begin();
-  }
-  port   = it->second;
-  ipaddr = it->first;
-  curr_master_addr_ = it->first;
-}
+namespace tubemq {
 
-void MasterAddrInfo::GetCurrentMasterAddr(string& ipaddr, int32_t& port) {
-   ipaddr = curr_master_addr_;
-   port = master_addr_[curr_master_addr_];
-}
 
 
 ClientSubInfo::ClientSubInfo() {
@@ -154,13 +113,15 @@ bool ClientSubInfo::IsFilterConsume(const string& topic) {
 
 void ClientSubInfo::GetAssignedPartOffset(const string& partition_key, int64_t& offset) {
   map<string, int64_t>::iterator it;
-  if (first_registered_.Get() && bound_consume_ && not_allocated_.Get()) {
+  offset = tb_config::kInvalidValue;
+  if (!first_registered_.Get()
+    && bound_consume_
+    && not_allocated_.Get()) {
     it = assigned_part_map_.find(partition_key);
     if (it != assigned_part_map_.end()) {
       offset = it->second;
     }
   }
-  offset = tb_config::kInvalidValue;
 }
 
 const map<string, set<string> >& ClientSubInfo::GetTopicFilterMap() const {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.h b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.h
new file mode 100644
index 0000000..1b1e7a1
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.h
@@ -0,0 +1,79 @@
+/**
+ * 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 TUBEMQ_CLIENT_SUBINFO_H_
+#define TUBEMQ_CLIENT_SUBINFO_H_
+
+#include <stdint.h>
+#include <list>
+#include <map>
+#include <mutex>
+#include <set>
+#include <string>
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_config.h"
+
+namespace tubemq {
+
+using std::list;
+using std::map;
+using std::set;
+using std::string;
+
+
+class ClientSubInfo {
+ public:
+  ClientSubInfo();
+  void SetConsumeTarget(const ConsumerConfig& config);
+  bool CompAndSetNotAllocated(bool expect, bool update);
+  void BookFstRegistered() { first_registered_.Set(true); }
+  bool IsBoundConsume() const { return bound_consume_; }
+  bool IsNotAllocated() const { return not_allocated_.Get(); }
+  const int64_t GetSubscribedTime() const { return subscribed_time_; }
+  const string& GetSessionKey() const { return session_key_; }
+  const uint32_t GetSourceCnt() const { return source_count_; }
+  bool SelectBig() { return select_big_; }
+  bool IsFilterConsume(const string& topic);
+  void GetAssignedPartOffset(const string& partition_key, int64_t& offset);
+  const string& GetBoundPartInfo() const { return bound_partions_; }
+  const list<string>& GetSubTopics() const { return topics_; }
+  const list<string>& GetTopicConds() const { return topic_conds_; }
+  const map<string, set<string> >& GetTopicFilterMap() const;
+
+ private:
+  bool bound_consume_;
+  AtomicBoolean first_registered_;
+  AtomicBoolean not_allocated_;
+  int64_t  subscribed_time_;
+  map<string, set<string> > topic_and_filter_map_;
+  list<string> topics_;
+  list<string> topic_conds_;
+  map<string, bool> topic_filter_map_;
+  // bound info
+  string session_key_;
+  uint32_t source_count_;
+  bool select_big_;
+  map<string, int64_t> assigned_part_map_;
+  string bound_partions_;
+};
+
+}  // namespace tubemq
+
+
+#endif  // TUBEMQ_CLIENT_SUBINFO_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/connection.h b/tubemq-client-twins/tubemq-client-cpp/src/connection.h
index 30db70b..d4423f5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/connection.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/connection.h
@@ -50,7 +50,6 @@ class Connection : noncopyable {
       : connect_id_(unique_id_.Next()),
         status_(kConnecting),
         recv_time_(std::time(nullptr)),
-        package_length_(0),
         create_time_(std::time(nullptr)) {
     formatContextString();
   }
@@ -60,7 +59,6 @@ class Connection : noncopyable {
         connect_id_(unique_id_.Next()),
         status_(kConnecting),
         recv_time_(std::time(nullptr)),
-        package_length_(0),
         create_time_(std::time(nullptr)) {
     formatContextString();
   }
@@ -82,7 +80,13 @@ class Connection : noncopyable {
 
   inline std::time_t GetRecvTime() const { return recv_time_; }
 
-  inline const std::string& ToString() const { return context_string_; }
+  inline const std::string ToString() const {
+    std::stringstream stream;
+    stream << "[recvtime:" << recv_time_ << "]"
+           << "[read_package:" << read_package_number_ << "]"
+           << "[write_package:" << write_package_number_ << "]";
+    return context_string_ + stream.str();
+  }
 
  private:
   void formatContextString() {
@@ -101,7 +105,9 @@ class Connection : noncopyable {
   std::atomic<Status> status_;
   std::string context_string_;  // for log
   std::time_t recv_time_;
-  size_t package_length_;
+  size_t package_length_ = 0;
+  size_t read_package_number_ = 0;
+  size_t write_package_number_ = 0;
 
  private:
   std::time_t create_time_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
index f3ec067..8ca1881 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
@@ -40,6 +40,9 @@
 #include "transport.h"
 
 namespace tubemq {
+
+using std::string;
+
 class ConnectionPool : noncopyable {
  public:
   explicit ConnectionPool(ExecutorPoolPtr& executor_pool)
@@ -70,8 +73,8 @@ class ConnectionPool : noncopyable {
       }
       if (it->second->GetRecvTime() + rpc_config::kRpcInvalidConnectOverTime < std::time(nullptr)) {
         it->second->Close();
-        it = connection_pool_.erase(it);
         LOG_ERROR("connection pool clear overtime connect:%s", it->second->ToString().c_str());
+        it = connection_pool_.erase(it);
         continue;
       }
       ++it;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/const_config.h b/tubemq-client-twins/tubemq-client-cpp/src/const_config.h
new file mode 100644
index 0000000..d22f7e7
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/const_config.h
@@ -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.
+ */
+
+#ifndef TUBEMQ_CLIENT_CONST_CONFIG_H_
+#define TUBEMQ_CLIENT_CONST_CONFIG_H_
+
+#include <stdint.h>
+
+#include <algorithm>
+#include <map>
+#include <string>
+
+namespace tubemq {
+
+using std::string;
+
+#define TUBEMQ_MAX(a, b) std::max((a), (b))
+#define TUBEMQ_MIN(a, b) std::min((a), (b))
+#define TUBEMQ_MID(data, max, min) TUBEMQ_MAX(min, TUBEMQ_MIN((max), (data)))
+
+// configuration value setting
+namespace tb_config {
+// rpc timeout define
+static const int32_t kRpcTimoutDefMs = 15000;
+static const int32_t kRpcTimoutMaxMs = 300000;
+static const int32_t kRpcTimoutMinMs = 8000;
+
+// heartbeat period define
+static const int32_t kHeartBeatPeriodDefMs = 10000;
+static const int32_t kHeartBeatFailRetryTimesDef = 5;
+static const int32_t kHeartBeatSleepPeriodDefMs = 60000;
+// max masterAddrInfo length
+static const int32_t kMasterAddrInfoMaxLength = 1024;
+
+// max TopicName length
+static const int32_t kTopicNameMaxLength = 64;
+// max Consume GroupName length
+static const int32_t kGroupNameMaxLength = 1024;
+// max filter item length
+static const int32_t kFilterItemMaxLength = 256;
+// max allowed filter item count
+static const int32_t kFilterItemMaxCount = 500;
+// max session key length
+static const int32_t kSessionKeyMaxLength = 1024;
+
+// max subscribe info report times
+static const int32_t kSubInfoReportMaxIntervalTimes = 6;
+// default message not found response wait period
+static const int32_t kMsgNotfoundWaitPeriodMsDef = 400;
+// default confirm wait period if rebalance meeting
+static const int32_t kRebConfirmWaitPeriodMsDef = 3000;
+// max confirm wait period anyway
+static const int32_t kConfirmWaitPeriodMsMax = 60000;
+// default rebalance wait if shutdown meeting
+static const int32_t kRebWaitPeriodWhenShutdownMs = 10000;
+// default partition status check period
+static const int32_t kMaxPartCheckPeriodMsDef = 60 * 1000;
+// default partition status check slice
+static const int32_t kPartCheckSliceMsDef = 300;
+
+// max int value
+static const int32_t kMaxIntValue = 0x7fffffff;
+// max long value
+static const int64_t kMaxLongValue = 0x7fffffffffffffffL;
+
+// default broker port
+static const uint32_t kBrokerPortDef = 8123;
+// default broker TLS port
+static const uint32_t kBrokerTlsPortDef = 8124;
+
+// invalid value
+static const int32_t kInvalidValue = -2;
+
+// message flag's properties settings
+static const int32_t kMsgFlagIncProperties = 0x01;
+
+// reserved property key Filter Item
+static const char kRsvPropKeyFilterItem[] = "$msgType$";
+// reserved property key message send time
+static const char kRsvPropKeyMsgTime[] = "$msgTime$";
+
+}  // namespace tb_config
+
+namespace delimiter {
+static const char kDelimiterDot[] = ".";
+static const char kDelimiterEqual[] = "=";
+static const char kDelimiterAnd[] = "&";
+static const char kDelimiterComma[] = ",";
+static const char kDelimiterColon[] = ":";
+static const char kDelimiterAt[] = "@";
+static const char kDelimiterPound[] = "#";
+static const char kDelimiterSemicolon[] = ";";
+// Double slash
+static const char kDelimiterDbSlash[] = "//";
+// left square bracket
+static const char kDelimiterLftSB[] = "[";
+// right square bracket
+static const char kDelimiterRgtSB[] = "]";
+
+}  // namespace delimiter
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_CONST_CONFIG_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/src/const_rpc.h
new file mode 100644
index 0000000..313d536
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/const_rpc.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 TUBEMQ_CLIENT_CONST_RPC_H_
+#define TUBEMQ_CLIENT_CONST_RPC_H_
+
+namespace tubemq {
+
+#include <stdint.h>
+
+namespace rpc_config {
+
+// constant define
+static const uint32_t kRpcPrtBeginToken = 0xFF7FF4FE;
+static const uint32_t kRpcMaxBufferSize = 8192;
+static const uint32_t kRpcMaxFrameListCnt = (uint32_t)((1024 * 1024 * 8) / kRpcMaxBufferSize);
+
+// rpc protocol version
+static const uint32_t kRpcProtocolVersion = 2;
+
+// rps network const
+static const uint32_t kRpcConnectInitBufferSize = 64 * 1024;
+static const uint32_t kRpcEnsureWriteableBytes = 16 * 1024;
+static constexpr uint32_t kRpcRecvBufferMaxBytes =
+    uint32_t(kRpcMaxFrameListCnt * kRpcMaxBufferSize * 2 + 1024);
+static const uint32_t kRpcInvalidConnectOverTime = 60 * 3;  // second
+
+// msg type flag
+static const int32_t kRpcFlagMsgRequest = 0x0;
+static const int32_t kRpcFlagMsgResponse = 0x1;
+
+// service type
+static const int32_t kMasterService = 1;
+static const int32_t kBrokerReadService = 2;
+static const int32_t kBrokerWriteService = 3;
+static const int32_t kBrokerAdminService = 4;
+static const int32_t kMasterAdminService = 5;
+
+// request method
+// master rpc method
+static const int32_t kMasterMethoddProducerRegister = 1;
+static const int32_t kMasterMethoddProducerHeatbeat = 2;
+static const int32_t kMasterMethoddProducerClose = 3;
+static const int32_t kMasterMethoddConsumerRegister = 4;
+static const int32_t kMasterMethoddConsumerHeatbeat = 5;
+static const int32_t kMasterMethoddConsumerClose = 6;
+
+// broker rpc method
+static const int32_t kBrokerMethoddProducerRegister = 11;
+static const int32_t kBrokerMethoddProducerHeatbeat = 12;
+static const int32_t kBrokerMethoddProducerSendMsg = 13;
+static const int32_t kBrokerMethoddProducerClose = 14;
+static const int32_t kBrokerMethoddConsumerRegister = 15;
+static const int32_t kBrokerMethoddConsumerHeatbeat = 16;
+static const int32_t kBrokerMethoddConsumerGetMsg = 17;
+static const int32_t kBrokerMethoddConsumerCommit = 18;
+static const int32_t kBrokerMethoddConsumerClose = 19;
+static const int32_t kMethodInvalid = 99999;
+
+// register operate type
+static const int32_t kRegOpTypeRegister = 31;
+static const int32_t kRegOpTypeUnReg = 32;
+
+// rpc connect node timeout
+static const int32_t kRpcConnectTimeoutMs = 3000;
+
+static const int32_t kConsumeStatusNormal = 0;
+static const int32_t kConsumeStatusFromMax = 1;
+static const int32_t kConsumeStatusFromMaxAlways = 2;
+
+}  // namespace rpc_config
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_CONST_RPC_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
index a2536ae..71a84ac 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
@@ -17,11 +17,13 @@
  * under the License.
  */
 
-#include "tubemq/executor_pool.h"
+#include "executor_pool.h"
 
-#include <asio.hpp>
 #include <functional>
 #include <memory>
+#include <asio.hpp>
+
+
 
 namespace tubemq {
 
@@ -39,19 +41,20 @@ Executor::~Executor() {
 
 void Executor::StartWorker(std::shared_ptr<asio::io_context> io_context) { io_context_->run(); }
 
-SocketPtr Executor::CreateSocket() { return SocketPtr(new asio::ip::tcp::socket(*io_context_)); }
+TcpSocketPtr Executor::CreateTcpSocket() {
+  return std::make_shared<asio::ip::tcp::socket>(*io_context_);
+}
 
-TlsSocketPtr Executor::CreateTlsSocket(SocketPtr &socket, asio::ssl::context &ctx) {
-  return std::shared_ptr<asio::ssl::stream<asio::ip::tcp::socket &> >(
-      new asio::ssl::stream<asio::ip::tcp::socket &>(*socket, ctx));
+TlsSocketPtr Executor::CreateTlsSocket(TcpSocketPtr &socket, asio::ssl::context &ctx) {
+  return std::make_shared<asio::ssl::stream<asio::ip::tcp::socket &>>(*socket, ctx);
 }
 
 TcpResolverPtr Executor::CreateTcpResolver() {
-  return TcpResolverPtr(new asio::ip::tcp::resolver(*io_context_));
+  return std::make_shared<asio::ip::tcp::resolver>(*io_context_);
 }
 
 SteadyTimerPtr Executor::CreateSteadyTimer() {
-  return SteadyTimerPtr(new asio::steady_timer(*io_context_));
+  return std::make_shared<asio::steady_timer>(*io_context_);
 }
 
 void Executor::Close() {
@@ -61,8 +64,6 @@ void Executor::Close() {
   }
 }
 
-void Executor::Post(Executor::func task) { io_context_->post(task); }
-
 ExecutorPool::ExecutorPool(int nthreads) : executors_(nthreads), executorIdx_(0), mutex_() {}
 
 ExecutorPtr ExecutorPool::Get() {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.h
new file mode 100644
index 0000000..e858985
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.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 _TUBEMQ_EXECUTOR_POOL_
+#define _TUBEMQ_EXECUTOR_POOL_
+
+#include <stdlib.h>
+
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <thread>
+#include <vector>
+
+#include <asio.hpp>
+#include <asio/ssl.hpp>
+#include "noncopyable.h"
+
+namespace tubemq {
+
+using TcpSocketPtr = std::shared_ptr<asio::ip::tcp::socket>;
+using TlsSocketPtr = std::shared_ptr<asio::ssl::stream<asio::ip::tcp::socket &> >;
+using TcpResolverPtr = std::shared_ptr<asio::ip::tcp::resolver>;
+using SteadyTimerPtr = std::shared_ptr<asio::steady_timer>;
+
+class Executor : noncopyable {
+ public:
+  Executor();
+  ~Executor();
+
+  TcpSocketPtr CreateTcpSocket();
+  TlsSocketPtr CreateTlsSocket(TcpSocketPtr &socket, asio::ssl::context &ctx);
+  TcpResolverPtr CreateTcpResolver();
+  SteadyTimerPtr CreateSteadyTimer();
+
+  inline void Post(std::function<void(void)> task) { io_context_->post(task); }
+
+  std::shared_ptr<asio::io_context> GetIoContext() { return io_context_; }
+
+  // Close executor and exit thread
+  void Close();
+
+ private:
+  void StartWorker(std::shared_ptr<asio::io_context> io_context);
+  std::shared_ptr<asio::io_context> io_context_;
+  using io_context_work = asio::executor_work_guard<asio::io_context::executor_type>;
+  io_context_work work_;
+  std::thread worker_;
+};
+
+typedef std::shared_ptr<Executor> ExecutorPtr;
+
+class ExecutorPool : noncopyable {
+ public:
+  explicit ExecutorPool(int nthreads = 2);
+
+  ExecutorPtr Get();
+
+  // Resize executor thread
+  void Resize(int nthreads) {
+    Lock lock(mutex_);
+    executors_.resize(nthreads);
+  }
+
+  void Close();
+
+ private:
+  typedef std::vector<ExecutorPtr> ExecutorList;
+  ExecutorList executors_;
+  uint32_t executorIdx_;
+  std::mutex mutex_;
+  typedef std::unique_lock<std::mutex> Lock;
+};
+
+typedef std::shared_ptr<ExecutorPool> ExecutorPoolPtr;
+
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_EXECUTOR_POOL_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 9af90fc..09cc76d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -17,15 +17,15 @@
  * under the License.
  */
 
-#include "tubemq/file_ini.h"
+#include "file_ini.h"
 
 #include <stdlib.h>
 
 #include <fstream>
 #include <sstream>
 
-#include "tubemq/const_config.h"
-#include "tubemq/utils.h"
+#include "const_config.h"
+#include "utils.h"
 
 namespace tubemq {
 
@@ -34,13 +34,13 @@ using std::ifstream;
 
 
 Fileini::Fileini() {
-  this->init_flag_ = false;
-  this->ini_map_.clear();
+  init_flag_ = false;
+  ini_map_.clear();
 }
 
 Fileini::~Fileini() {
-  this->init_flag_ = false;
-  this->ini_map_.clear();
+  init_flag_ = false;
+  ini_map_.clear();
 }
 
 bool Fileini::Loadini(string& err_info, const string& file_name) {
@@ -95,8 +95,8 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
       continue;
     }
     map<string, map<string, string> >::iterator it_sec;
-    it_sec = this->ini_map_.find(sector);
-    if (it_sec == this->ini_map_.end()) {
+    it_sec = ini_map_.find(sector);
+    if (it_sec == ini_map_.end()) {
       map<string, string> tmp_key_val_map;
       tmp_key_val_map[key] = value;
       ini_map_[sector] = tmp_key_val_map;
@@ -108,7 +108,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   conf_file.close();
   conf_file.clear();
   // set parser status
-  this->init_flag_ = true;
+  init_flag_ = true;
   // end
   err_info = "Ok";
   return true;
@@ -116,7 +116,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
 
 bool Fileini::GetValue(string& err_info, const string& sector, const string& key, string& value,
                        const string& def) const {
-  if (!this->init_flag_) {
+  if (!init_flag_) {
     err_info = "Please load configure file first!";
     return false;
   }
@@ -125,8 +125,8 @@ bool Fileini::GetValue(string& err_info, const string& sector, const string& key
   // search key's value in sector
   map<string, map<string, string> >::const_iterator it_sec;
   map<string, string>::const_iterator it_keyval;
-  it_sec = this->ini_map_.find(sector);
-  if (it_sec == this->ini_map_.end()) {
+  it_sec = ini_map_.find(sector);
+  if (it_sec == ini_map_.end()) {
     value = def;
     return true;
   }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.h
new file mode 100644
index 0000000..1bede63
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.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 TUBEMQ_CLIENT_FILE_INI_H_
+#define TUBEMQ_CLIENT_FILE_INI_H_
+
+#include <stdint.h>
+
+#include <map>
+#include <string>
+
+namespace tubemq {
+
+using std::map;
+using std::string;
+
+class Fileini {
+ public:
+  Fileini();
+  ~Fileini();
+  bool Loadini(string& err_info, const string& file_name);
+  bool GetValue(string& err_info, const string& sector, const string& key,
+                    string& value, const string& def) const;
+  bool GetValue(string& err_info, const string& sector, const string& key,
+                   int32_t& value, int32_t def) const;
+
+ private:
+  bool init_flag_;
+  // sector        key    value
+  map<string, map<string, string> > ini_map_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_FILE_INI_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index ef3357f..2b96f52 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "tubemq/flowctrl_def.h"
+#include "flowctrl_def.h"
 
 #include <stdio.h>
 #include <time.h>
@@ -25,9 +25,9 @@
 
 #include <sstream>
 
-#include "tubemq/const_config.h"
-#include "tubemq/logger.h"
-#include "tubemq/utils.h"
+#include "const_config.h"
+#include "logger.h"
+#include "utils.h"
 
 namespace tubemq {
 
@@ -35,172 +35,172 @@ using std::stringstream;
 using std::lock_guard;
 
 FlowCtrlResult::FlowCtrlResult() {
-  this->datasize_limit_ = tb_config::kMaxIntValue;
-  this->freqms_limit_ = 0;
+  datasize_limit_ = tb_config::kMaxIntValue;
+  freqms_limit_ = 0;
 }
 
 FlowCtrlResult::FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit) {
-  this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_ = freqms_limit;
+  datasize_limit_ = datasize_limit;
+  freqms_limit_ = freqms_limit;
 }
 
 FlowCtrlResult& FlowCtrlResult::operator=(const FlowCtrlResult& target) {
   if (this == &target) return *this;
-  this->datasize_limit_ = target.datasize_limit_;
-  this->freqms_limit_ = target.freqms_limit_;
+  datasize_limit_ = target.datasize_limit_;
+  freqms_limit_ = target.freqms_limit_;
   return *this;
 }
 
 void FlowCtrlResult::SetDataDltAndFreqLimit(int64_t datasize_limit, int32_t freqms_limit) {
-  this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_ = freqms_limit;
+  datasize_limit_ = datasize_limit;
+  freqms_limit_ = freqms_limit;
 }
 
 void FlowCtrlResult::SetDataSizeLimit(int64_t datasize_limit) {
-  this->datasize_limit_ = datasize_limit;
+  datasize_limit_ = datasize_limit;
 }
 
-void FlowCtrlResult::SetFreqMsLimit(int32_t freqms_limit) { this->freqms_limit_ = freqms_limit; }
+void FlowCtrlResult::SetFreqMsLimit(int32_t freqms_limit) { freqms_limit_ = freqms_limit; }
 
-int64_t FlowCtrlResult::GetDataSizeLimit() { return this->datasize_limit_; }
+int64_t FlowCtrlResult::GetDataSizeLimit() { return datasize_limit_; }
 
-int32_t FlowCtrlResult::GetFreqMsLimit() { return this->freqms_limit_; }
+int32_t FlowCtrlResult::GetFreqMsLimit() { return freqms_limit_; }
 
 FlowCtrlItem::FlowCtrlItem() {
-  this->type_ = 0;
-  this->start_time_ = 2500;
-  this->end_time_ = tb_config::kInvalidValue;
-  this->datadlt_m_ = tb_config::kInvalidValue;
-  this->datasize_limit_ = tb_config::kInvalidValue;
-  this->freqms_limit_ = tb_config::kInvalidValue;
-  this->zero_cnt_ = tb_config::kInvalidValue;
+  type_ = 0;
+  start_time_ = 2500;
+  end_time_ = tb_config::kInvalidValue;
+  datadlt_m_ = tb_config::kInvalidValue;
+  datasize_limit_ = tb_config::kInvalidValue;
+  freqms_limit_ = tb_config::kInvalidValue;
+  zero_cnt_ = tb_config::kInvalidValue;
 }
 
 FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit) {
-  this->type_ = type;
-  this->start_time_ = 2500;
-  this->end_time_ = tb_config::kInvalidValue;
-  this->datadlt_m_ = tb_config::kInvalidValue;
-  this->datasize_limit_ = tb_config::kInvalidValue;
-  this->freqms_limit_ = freqms_limit;
-  this->zero_cnt_ = zero_cnt;
+  type_ = type;
+  start_time_ = 2500;
+  end_time_ = tb_config::kInvalidValue;
+  datadlt_m_ = tb_config::kInvalidValue;
+  datasize_limit_ = tb_config::kInvalidValue;
+  freqms_limit_ = freqms_limit;
+  zero_cnt_ = zero_cnt;
 }
 
 FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
                            int32_t min_data_filter_freqms) {
-  this->type_ = type;
-  this->start_time_ = 2500;
-  this->end_time_ = tb_config::kInvalidValue;
-  this->datadlt_m_ = tb_config::kInvalidValue;
-  this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_ = freqms_limit;
-  this->zero_cnt_ = min_data_filter_freqms;
+  type_ = type;
+  start_time_ = 2500;
+  end_time_ = tb_config::kInvalidValue;
+  datadlt_m_ = tb_config::kInvalidValue;
+  datasize_limit_ = datasize_limit;
+  freqms_limit_ = freqms_limit;
+  zero_cnt_ = min_data_filter_freqms;
 }
 
 FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, int64_t datadlt_m,
                            int64_t datasize_limit, int32_t freqms_limit) {
-  this->type_ = type;
-  this->start_time_ = start_time;
-  this->end_time_ = end_time;
-  this->datadlt_m_ = datadlt_m;
-  this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_ = freqms_limit;
-  this->zero_cnt_ = tb_config::kInvalidValue;
+  type_ = type;
+  start_time_ = start_time;
+  end_time_ = end_time;
+  datadlt_m_ = datadlt_m;
+  datasize_limit_ = datasize_limit;
+  freqms_limit_ = freqms_limit;
+  zero_cnt_ = tb_config::kInvalidValue;
 }
 
 FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
   if (this == &target) return *this;
-  this->type_ = target.type_;
-  this->start_time_ = target.start_time_;
-  this->end_time_ = target.end_time_;
-  this->datadlt_m_ = target.datadlt_m_;
-  this->datasize_limit_ = target.datasize_limit_;
-  this->freqms_limit_ = target.freqms_limit_;
-  this->zero_cnt_ = target.zero_cnt_;
+  type_ = target.type_;
+  start_time_ = target.start_time_;
+  end_time_ = target.end_time_;
+  datadlt_m_ = target.datadlt_m_;
+  datasize_limit_ = target.datasize_limit_;
+  freqms_limit_ = target.freqms_limit_;
+  zero_cnt_ = target.zero_cnt_;
   return *this;
 }
 
 int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) const {
-  if (this->type_ != 1) {
+  if (type_ != 1) {
     return -1;
   }
-  if (msg_zero_cnt >= this->zero_cnt_) {
-    return this->freqms_limit_;
+  if (msg_zero_cnt >= zero_cnt_) {
+    return freqms_limit_;
   }
   return -1;
 }
 
 void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
                                       int32_t min_data_filter_freqms) {
-  this->type_ = type;
-  this->start_time_ = 2500;
-  this->end_time_ = tb_config::kInvalidValue;
-  this->datadlt_m_ = tb_config::kInvalidValue;
-  this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_ = freqms_limit;
-  this->zero_cnt_ = min_data_filter_freqms;
+  type_ = type;
+  start_time_ = 2500;
+  end_time_ = tb_config::kInvalidValue;
+  datadlt_m_ = tb_config::kInvalidValue;
+  datasize_limit_ = datasize_limit;
+  freqms_limit_ = freqms_limit;
+  zero_cnt_ = min_data_filter_freqms;
 }
 
 void FlowCtrlItem::Clear() {
-  this->type_ = 0;
-  this->start_time_ = 2500;
-  this->end_time_ = tb_config::kInvalidValue;
-  this->datadlt_m_ = tb_config::kInvalidValue;
-  this->datasize_limit_ = tb_config::kInvalidValue;
-  this->freqms_limit_ = tb_config::kInvalidValue;
-  this->zero_cnt_ = tb_config::kInvalidValue;
+  type_ = 0;
+  start_time_ = 2500;
+  end_time_ = tb_config::kInvalidValue;
+  datadlt_m_ = tb_config::kInvalidValue;
+  datasize_limit_ = tb_config::kInvalidValue;
+  freqms_limit_ = tb_config::kInvalidValue;
+  zero_cnt_ = tb_config::kInvalidValue;
 }
 
 bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m, int32_t curr_time,
                                 FlowCtrlResult& flowctrl_result) const {
-  if (this->type_ != 0 || datadlt_m <= this->datadlt_m_) {
+  if (type_ != 0 || datadlt_m <= datadlt_m_) {
     return false;
   }
-  if (curr_time < this->start_time_ || curr_time > this->end_time_) {
+  if (curr_time < start_time_ || curr_time > end_time_) {
     return false;
   }
-  flowctrl_result.SetDataDltAndFreqLimit(this->datasize_limit_, this->freqms_limit_);
+  flowctrl_result.SetDataDltAndFreqLimit(datasize_limit_, freqms_limit_);
   return true;
 }
 
 FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
-  this->flowctrl_id_.GetAndSet(tb_config::kInvalidValue);
-  this->flowctrl_info_ = "";
-  this->min_zero_cnt_.Set(tb_config::kMaxIntValue);
-  this->qrypriority_id_.Set(tb_config::kInvalidValue);
-  this->min_datadlt_limt_.Set(tb_config::kMaxLongValue);
-  this->datalimit_start_time_.Set(2500);
-  this->datalimit_end_time_.Set(tb_config::kInvalidValue);
-  this->last_update_time_ = Utils::GetCurrentTimeMillis();
+  flowctrl_id_.GetAndSet(tb_config::kInvalidValue);
+  flowctrl_info_ = "";
+  min_zero_cnt_.Set(tb_config::kMaxIntValue);
+  qrypriority_id_.Set(tb_config::kInvalidValue);
+  min_datadlt_limt_.Set(tb_config::kMaxLongValue);
+  datalimit_start_time_.Set(2500);
+  datalimit_end_time_.Set(tb_config::kInvalidValue);
+  last_update_time_ = Utils::GetCurrentTimeMillis();
 }
 
 FlowCtrlRuleHandler::~FlowCtrlRuleHandler() {
-  // 
+  //
 }
 
 void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id,
                                                 int64_t flowctrl_id, const string& flowctrl_info) {
   map<int32_t, vector<FlowCtrlItem> > tmp_flowctrl_map;
-  if (flowctrl_id == this->flowctrl_id_.Get()) {
+  if (flowctrl_id == flowctrl_id_.Get()) {
     return;
   }
-  int64_t curr_flowctrl_id = this->flowctrl_id_.Get();
+  int64_t curr_flowctrl_id = flowctrl_id_.Get();
   if (flowctrl_info.length() > 0) {
     parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
   }
   lock_guard<mutex> lck(config_lock_);
-  this->flowctrl_id_.Set(flowctrl_id);
-  this->qrypriority_id_.Set(qrypriority_id);
+  flowctrl_id_.Set(flowctrl_id);
+  qrypriority_id_.Set(qrypriority_id);
   clearStatisData();
   if (tmp_flowctrl_map.empty()) {
-    this->flowctrl_rules_.clear();
-    this->flowctrl_info_ = "";
+    flowctrl_rules_.clear();
+    flowctrl_info_ = "";
   } else {
-    this->flowctrl_rules_ = tmp_flowctrl_map;
-    this->flowctrl_info_ = flowctrl_info;
+    flowctrl_rules_ = tmp_flowctrl_map;
+    flowctrl_info_ = flowctrl_info;
     initialStatisData();
   }
-  this->last_update_time_ = Utils::GetCurrentTimeMillis();
+  last_update_time_ = Utils::GetCurrentTimeMillis();
   if (is_default) {
     LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id,
              flowctrl_id);
@@ -215,55 +215,55 @@ void FlowCtrlRuleHandler::initialStatisData() {
   vector<FlowCtrlItem>::iterator it_vec;
   map<int, vector<FlowCtrlItem> >::iterator it_map;
 
-  it_map = this->flowctrl_rules_.find(0);
-  if (it_map != this->flowctrl_rules_.end()) {
+  it_map = flowctrl_rules_.find(0);
+  if (it_map != flowctrl_rules_.end()) {
     for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       if (it_vec->GetType() != 0) {
         continue;
       }
 
-      if (it_vec->GetDltInM() < this->min_datadlt_limt_.Get()) {
-        this->min_datadlt_limt_.Set(it_vec->GetDltInM());
+      if (it_vec->GetDltInM() < min_datadlt_limt_.Get()) {
+        min_datadlt_limt_.Set(it_vec->GetDltInM());
       }
-      if (it_vec->GetStartTime() < this->datalimit_start_time_.Get()) {
-        this->datalimit_start_time_.Set(it_vec->GetStartTime());
+      if (it_vec->GetStartTime() < datalimit_start_time_.Get()) {
+        datalimit_start_time_.Set(it_vec->GetStartTime());
       }
-      if (it_vec->GetEndTime() > this->datalimit_end_time_.Get()) {
-        this->datalimit_end_time_.Set(it_vec->GetEndTime());
+      if (it_vec->GetEndTime() > datalimit_end_time_.Get()) {
+        datalimit_end_time_.Set(it_vec->GetEndTime());
       }
     }
   }
-  it_map = this->flowctrl_rules_.find(1);
-  if (it_map != this->flowctrl_rules_.end()) {
+  it_map = flowctrl_rules_.find(1);
+  if (it_map != flowctrl_rules_.end()) {
     for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       if (it_vec->GetType() != 1) {
         continue;
       }
-      if (it_vec->GetZeroCnt() < this->min_zero_cnt_.Get()) {
-        this->min_zero_cnt_.Set(it_vec->GetZeroCnt());
+      if (it_vec->GetZeroCnt() < min_zero_cnt_.Get()) {
+        min_zero_cnt_.Set(it_vec->GetZeroCnt());
       }
     }
   }
-  it_map = this->flowctrl_rules_.find(3);
-  if (it_map != this->flowctrl_rules_.end()) {
+  it_map = flowctrl_rules_.find(3);
+  if (it_map != flowctrl_rules_.end()) {
     for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       if (it_vec->GetType() != 3) {
         continue;
       }
       it_vec->GetDataSizeLimit();
-      this->filter_ctrl_item_.ResetFlowCtrlValue(3, (int)(it_vec->GetDataSizeLimit()),
+      filter_ctrl_item_.ResetFlowCtrlValue(3, (int32_t)(it_vec->GetDataSizeLimit()),
                                                  it_vec->GetFreqMsLimit(), it_vec->GetZeroCnt());
     }
   }
 }
 
 void FlowCtrlRuleHandler::clearStatisData() {
-  this->min_zero_cnt_.GetAndSet(tb_config::kMaxIntValue);
-  this->min_datadlt_limt_.GetAndSet(tb_config::kMaxLongValue);
-  this->qrypriority_id_.Set(tb_config::kInvalidValue);
-  this->datalimit_start_time_.Set(2500);
-  this->datalimit_end_time_.Set(tb_config::kInvalidValue);
-  this->filter_ctrl_item_.Clear();
+  min_zero_cnt_.GetAndSet(tb_config::kMaxIntValue);
+  min_datadlt_limt_.GetAndSet(tb_config::kMaxLongValue);
+  qrypriority_id_.Set(tb_config::kInvalidValue);
+  datalimit_start_time_.Set(2500);
+  datalimit_end_time_.Set(tb_config::kInvalidValue);
+  filter_ctrl_item_.Clear();
 }
 
 bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt,
@@ -276,16 +276,16 @@ bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt,
   time_t cur_time = time(NULL);
   gmtime_r(&cur_time, &utc_tm);
   int curr_time = (utc_tm.tm_hour + 8) % 24 * 100 + utc_tm.tm_min;
-  if ((last_datadlt < this->min_datadlt_limt_.Get())
-      || (curr_time < this->datalimit_start_time_.Get())
-      || (curr_time > this->datalimit_end_time_.Get())) {
+  if ((last_datadlt < min_datadlt_limt_.Get())
+      || (curr_time < datalimit_start_time_.Get())
+      || (curr_time > datalimit_end_time_.Get())) {
     return false;
   }
   // search total flowctrl rule
   lock_guard<mutex> lck(config_lock_);
-  it_map = this->flowctrl_rules_.find(0);
-  if (it_map != this->flowctrl_rules_.end()) {
-    for (it_vec = it_map->second.begin();it_vec != it_map->second.end(); ++it_vec) {
+  it_map = flowctrl_rules_.find(0);
+  if (it_map != flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       if (it_vec->GetDataLimit(last_datadlt, curr_time, flowctrl_result)) {
         result = true;
         break;
@@ -301,13 +301,13 @@ int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt,
   vector<FlowCtrlItem>::const_iterator it_vec;
   map<int, vector<FlowCtrlItem> >::const_iterator it_map;
   // check min zero count
-  if (msg_zero_cnt < this->min_zero_cnt_.Get()) {
+  if (msg_zero_cnt < min_zero_cnt_.Get()) {
     return limit_data;
   }
   // search rule allow value
   lock_guard<mutex> lck(config_lock_);
-  it_map = this->flowctrl_rules_.find(1);
-  if (it_map != this->flowctrl_rules_.end()) {
+  it_map = flowctrl_rules_.find(1);
+  if (it_map != flowctrl_rules_.end()) {
     for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       limit_data = it_vec->GetFreLimit(msg_zero_cnt);
       if (limit_data >= 0) {
@@ -321,17 +321,15 @@ int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt,
 void FlowCtrlRuleHandler::GetFilterCtrlItem(FlowCtrlItem& result) const {
   result.Clear();
   lock_guard<mutex> lck(config_lock_);
-  result = this->filter_ctrl_item_;
+  result = filter_ctrl_item_;
 }
 
 void FlowCtrlRuleHandler::GetFlowCtrlInfo(string& flowctrl_info) const {
   flowctrl_info.clear();
   lock_guard<mutex> lck(config_lock_);
-  flowctrl_info = this->flowctrl_info_;
+  flowctrl_info = flowctrl_info_;
 }
 
-
-
 bool FlowCtrlRuleHandler::compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2) {
   if (o1.GetStartTime() >= o2.GetStartTime()) {
     return true;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.h
new file mode 100644
index 0000000..0131c6f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.h
@@ -0,0 +1,144 @@
+/**
+ * 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 TUBEMQ_CLIENT_FLOW_CONTROL_H_
+#define TUBEMQ_CLIENT_FLOW_CONTROL_H_
+
+#include <rapidjson/document.h>
+#include <stdint.h>
+
+#include <algorithm>
+#include <list>
+#include <map>
+#include <mutex>
+#include <string>
+#include <vector>
+
+#include "tubemq/tubemq_atomic.h"
+
+namespace tubemq {
+
+using std::map;
+using std::mutex;
+using std::string;
+using std::vector;
+
+class FlowCtrlResult {
+ public:
+  FlowCtrlResult();
+  FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit);
+  FlowCtrlResult& operator=(const FlowCtrlResult& target);
+  void SetDataDltAndFreqLimit(int64_t datasize_limit, int32_t freqms_limit);
+  void SetDataSizeLimit(int64_t datasize_limit);
+  void SetFreqMsLimit(int32_t freqms_limit);
+  int64_t GetDataSizeLimit();
+  int32_t GetFreqMsLimit();
+
+ private:
+  int64_t datasize_limit_;
+  int32_t freqms_limit_;
+};
+
+class FlowCtrlItem {
+ public:
+  FlowCtrlItem();
+  FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit);
+  FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+               int32_t min_data_filter_freqms);
+  FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, int64_t datadlt_m,
+               int64_t datasize_limit, int32_t freqms_limit);
+  FlowCtrlItem& operator=(const FlowCtrlItem& target);
+  void Clear();
+  void ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+                          int32_t min_data_filter_freqms);
+  int32_t GetFreLimit(int32_t msg_zero_cnt) const;
+  bool GetDataLimit(int64_t datadlt_m,
+    int32_t curr_time, FlowCtrlResult& flowctrl_result) const;
+  const int32_t GetType() const { return type_; }
+  const int32_t GetZeroCnt() const { return zero_cnt_; }
+  const int32_t GetStartTime() const { return start_time_; }
+  const int32_t GetEndTime() const { return end_time_; }
+  const int64_t GetDataSizeLimit() const { return datasize_limit_; }
+  const int32_t GetFreqMsLimit() const { return freqms_limit_; }
+  const int64_t GetDltInM() const { return datadlt_m_; }
+
+ private:
+  int32_t type_;
+  int32_t start_time_;
+  int32_t end_time_;
+  int64_t datadlt_m_;
+  int64_t datasize_limit_;
+  int32_t freqms_limit_;
+  int32_t zero_cnt_;
+};
+
+class FlowCtrlRuleHandler {
+ public:
+  FlowCtrlRuleHandler();
+  ~FlowCtrlRuleHandler();
+  void UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id, int64_t flowctrl_id,
+                             const string& flowctrl_info);
+  bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const;
+  int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const;
+  void GetFilterCtrlItem(FlowCtrlItem& result) const;
+  void GetFlowCtrlInfo(string& flowctrl_info) const;
+  int32_t GetMinZeroCnt() const { return this->min_zero_cnt_.Get(); }
+  int32_t GetQryPriorityId() const { return this->qrypriority_id_.Get(); }
+  void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
+  const int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
+
+ private:
+  void initialStatisData();
+  void clearStatisData();
+  static bool compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2);
+  static bool compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2);
+  bool parseStringMember(string& err_info, const rapidjson::Value& root, const char* key,
+                         string& value, bool compare_value, string required_val);
+  bool parseLongMember(string& err_info, const rapidjson::Value& root, const char* key,
+                       int64_t& value, bool compare_value, int64_t required_val);
+  bool parseIntMember(string& err_info, const rapidjson::Value& root, const char* key,
+                      int32_t& value, bool compare_value, int32_t required_val);
+  bool parseFlowCtrlInfo(const string& flowctrl_info,
+                         map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map);
+  bool parseDataLimit(string& err_info, const rapidjson::Value& root,
+                      vector<FlowCtrlItem>& flowCtrlItems);
+  bool parseFreqLimit(string& err_info, const rapidjson::Value& root,
+                      vector<FlowCtrlItem>& flowctrl_items);
+  bool parseLowFetchLimit(string& err_info, const rapidjson::Value& root,
+                          vector<FlowCtrlItem>& flowctrl_items);
+  bool parseTimeMember(string& err_info, const rapidjson::Value& root, const char* key,
+                       int32_t& value);
+
+ private:
+  mutable mutex config_lock_;
+  string flowctrl_info_;
+  FlowCtrlItem filter_ctrl_item_;
+  map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
+  int64_t last_update_time_;
+  AtomicLong flowctrl_id_;
+  AtomicInteger qrypriority_id_;
+  AtomicInteger min_zero_cnt_;
+  AtomicLong min_datadlt_limt_;
+  AtomicInteger datalimit_start_time_;
+  AtomicInteger datalimit_end_time_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_FLOW_CONTROL_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
index f374d9b..2e51e25 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "tubemq/logger.h"
+#include "logger.h"
 
 #include <log4cplus/fileappender.h>
 #include <log4cplus/layout.h>
@@ -27,14 +27,14 @@
 
 #include <string>
 
-#include "tubemq/singleton.h"
+#include "singleton.h"
 
 namespace tubemq {
 
-Logger& GetLogger() { return Singleton<Logger>::Instance(); }
-
 static const uint32_t kMBSize = 1024 * 1024;
 
+Logger& GetLogger() { return Singleton<Logger>::Instance(); }
+
 bool Logger::Init(const std::string& path, Logger::Level level, uint32_t file_max_size,
                   uint32_t file_num) {
   base_path_ = path;
@@ -67,6 +67,7 @@ void Logger::setup() {
   bool immediate_fush = true;
   std::string pattern = "[%D{%Y-%m-%d %H:%M:%S.%q}][tid:%t]%m%n";
   auto logger_d = log4cplus::Logger::getInstance(instance_);
+  logger_d.removeAllAppenders();
   logger_d.setLogLevel(log4cplus::TRACE_LOG_LEVEL);
   log4cplus::helpers::SharedObjectPtr<log4cplus::Appender> append_d(
       new log4cplus::RollingFileAppender(base_path_ + ".log", file_max_size_ * kMBSize, file_num_,
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.h b/tubemq-client-twins/tubemq-client-cpp/src/logger.h
new file mode 100644
index 0000000..64933a6
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.h
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _TUBEMQ_LOG_FILE_
+#define _TUBEMQ_LOG_FILE_
+
+#include <stdint.h>
+
+#include <string>
+#include <vector>
+
+namespace tubemq {
+class Logger;
+
+Logger& GetLogger();
+
+#define LOG_LEVEL(level, fmt, ...)                                                   \
+  {                                                                                  \
+    if (tubemq::GetLogger().IsEnable(level)) {                                       \
+      tubemq::GetLogger().Write("[%s:%d][%s]" fmt, __func__, __LINE__,               \
+                                tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
+    }                                                                                \
+  }
+
+#define LOG_TRACE(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
+#define LOG_DEBUG(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
+#define LOG_INFO(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
+#define LOG_WARN(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
+#define LOG_ERROR(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
+
+#define LOG_TUBEMQ(logger, level, fmt, ...)                                                    \
+  {                                                                                            \
+    if (logger.IsEnable(level)) {                                                              \
+      logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), \
+                   ##__VA_ARGS__);                                                             \
+    }                                                                                          \
+  }
+
+class Logger {
+ public:
+  enum Level {
+    kTrace = 0,
+    kDebug = 1,
+    kInfo = 2,
+    kWarn = 3,
+    kError = 4,
+  };
+
+  // size: MB
+  Logger()
+      : file_max_size_(100),
+        file_num_(10),
+        level_(kError),
+        base_path_("tubemq"),
+        instance_("TubeMQ") {}
+
+  ~Logger(void) {}
+
+  // path example: ../log/tubemq
+  // size: MB
+  bool Init(const std::string& path, Level level, uint32_t file_max_size = 100,
+            uint32_t file_num = 10);
+
+  bool Write(const char* sFormat, ...) __attribute__((format(printf, 2, 3)));
+  inline bool WriteStream(const std::string& msg) { return writeStream(msg.c_str()); }
+
+  inline void SetInstance(const std::string& instance) { instance_ = instance; }
+  inline bool IsEnable(Level level) {
+    if (level_ <= level) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  static const char* Level2String(Level level) {
+    static const char* level_names[] = {
+        "TRACE", "DEBUG", "INFO", "WARN", "ERROR",
+    };
+    return level_names[level];
+  }
+
+ private:
+  void setup();
+  bool writeStream(const char* msg);
+
+ private:
+  uint32_t file_max_size_;
+  uint16_t file_num_;
+  uint8_t level_;
+
+  std::string base_path_;
+  std::string instance_;
+  std::string err_msg_;
+};
+}  // namespace tubemq
+#endif  // _TUBEMQ_LOG_FILE_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 4f03dfc..38f6f46 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -17,16 +17,16 @@
  * under the License.
  */
 
-#include "tubemq/meta_info.h"
+#include "meta_info.h"
 
 #include <stdlib.h>
 
 #include <sstream>
 #include <vector>
 
-#include "tubemq/const_config.h"
+#include "const_config.h"
 #include "tubemq/tubemq_errcode.h"
-#include "tubemq/utils.h"
+#include "utils.h"
 
 namespace tubemq {
 
@@ -34,9 +34,9 @@ using std::stringstream;
 using std::vector;
 
 NodeInfo::NodeInfo() {
-  this->node_id_ = 0;
-  this->node_host_ = " ";
-  this->node_port_ = tb_config::kBrokerPortDef;
+  node_id_ = 0;
+  node_host_ = " ";
+  node_port_ = tb_config::kBrokerPortDef;
   buildStrInfo();
 }
 
@@ -45,34 +45,34 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
   vector<string> result;
   Utils::Split(node_info, result, delimiter::kDelimiterColon);
   if (is_broker) {
-    this->node_id_ = atoi(result[0].c_str());
-    this->node_host_ = result[1];
-    this->node_port_ = tb_config::kBrokerPortDef;
+    node_id_ = atoi(result[0].c_str());
+    node_host_ = result[1];
+    node_port_ = tb_config::kBrokerPortDef;
     if (result.size() >= 3) {
-      this->node_port_ = atoi(result[2].c_str());
+      node_port_ = atoi(result[2].c_str());
     }
   } else {
-    this->node_id_ = 0;
-    this->node_host_ = result[0];
-    this->node_port_ = tb_config::kBrokerPortDef;
+    node_id_ = 0;
+    node_host_ = result[0];
+    node_port_ = tb_config::kBrokerPortDef;
     if (result.size() >= 2) {
-      this->node_port_ = atoi(result[1].c_str());
+      node_port_ = atoi(result[1].c_str());
     }
   }
   buildStrInfo();
 }
 
 NodeInfo::NodeInfo(const string& node_host, uint32_t node_port) {
-  this->node_id_ = tb_config::kInvalidValue;
-  this->node_host_ = node_host;
-  this->node_port_ = node_port;
+  node_id_ = tb_config::kInvalidValue;
+  node_host_ = node_host;
+  node_port_ = node_port;
   buildStrInfo();
 }
 
 NodeInfo::NodeInfo(int node_id, const string& node_host, uint32_t node_port) {
-  this->node_id_ = node_id;
-  this->node_host_ = node_host;
-  this->node_port_ = node_port;
+  node_id_ = node_id;
+  node_host_ = node_host;
+  node_port_ = node_port;
   buildStrInfo();
 }
 
@@ -82,11 +82,12 @@ NodeInfo::~NodeInfo() {
 
 NodeInfo& NodeInfo::operator=(const NodeInfo& target) {
   if (this != &target) {
-    this->node_id_ = target.node_id_;
-    this->node_host_ = target.node_host_;
-    this->node_port_ = target.node_port_;
-    this->addr_info_ = target.addr_info_;
-    this->node_info_ = target.node_info_;
+    node_id_ = target.node_id_;
+    node_host_ = target.node_host_;
+    node_port_ = target.node_port_;
+    addr_info_ = target.addr_info_;
+    node_info_ = target.node_info_;
+    buildStrInfo();
   }
   return *this;
 }
@@ -95,50 +96,50 @@ bool NodeInfo::operator==(const NodeInfo& target) {
   if (this == &target) {
     return true;
   }
-  if (this->node_info_ == target.node_info_) {
+  if (node_info_ == target.node_info_) {
     return true;
   }
   return false;
 }
 
 bool NodeInfo::operator<(const NodeInfo& target) const {
-  return this->node_info_ < target.node_info_;
+  return node_info_ < target.node_info_;
 }
-const uint32_t NodeInfo::GetNodeId() const { return this->node_id_; }
+const uint32_t NodeInfo::GetNodeId() const { return node_id_; }
 
-const string& NodeInfo::GetHost() const { return this->node_host_; }
+const string& NodeInfo::GetHost() const { return node_host_; }
 
-const uint32_t NodeInfo::GetPort() const { return this->node_port_; }
+const uint32_t NodeInfo::GetPort() const { return node_port_; }
 
-const string& NodeInfo::GetAddrInfo() const { return this->addr_info_; }
+const string& NodeInfo::GetAddrInfo() const { return addr_info_; }
 
-const string& NodeInfo::GetNodeInfo() const { return this->node_info_; }
+const string& NodeInfo::GetNodeInfo() const { return node_info_; }
 
 void NodeInfo::buildStrInfo() {
   stringstream ss1;
-  ss1 << this->node_host_;
+  ss1 << node_host_;
   ss1 << delimiter::kDelimiterColon;
-  ss1 << this->node_port_;
-  this->addr_info_ = ss1.str();
+  ss1 << node_port_;
+  addr_info_ = ss1.str();
 
   stringstream ss2;
-  ss2 << this->node_id_;
+  ss2 << node_id_;
   ss2 << delimiter::kDelimiterColon;
-  ss2 << this->addr_info_;
-  this->node_info_ = ss2.str();
+  ss2 << addr_info_;
+  node_info_ = ss2.str();
 }
 
 Partition::Partition() {
-  this->topic_ = " ";
-  this->partition_id_ = 0;
+  topic_ = " ";
+  partition_id_ = 0;
   buildPartitionKey();
 }
 
 // partition_info = broker_info#topic:partitionId
 Partition::Partition(const string& partition_info) {
   // initial process
-  this->topic_ = " ";
-  this->partition_id_ = 0;
+  topic_ = " ";
+  partition_id_ = 0;
   // parse partition_info string
   string::size_type pos = 0;
   string seg_key = delimiter::kDelimiterPound;
@@ -148,7 +149,8 @@ Partition::Partition(const string& partition_info) {
   if (pos != string::npos) {
     string broker_info = partition_info.substr(0, pos);
     broker_info = Utils::Trim(broker_info);
-    this->broker_info_ = NodeInfo(true, broker_info);
+    NodeInfo tmp_node(true, broker_info);
+    broker_info_ = tmp_node;
     string part_str = partition_info.substr(pos + seg_key.size(), partition_info.size());
     part_str = Utils::Trim(part_str);
     pos = part_str.find(token_key);
@@ -157,8 +159,8 @@ Partition::Partition(const string& partition_info) {
       string part_id_str = part_str.substr(pos + token_key.size(), part_str.size());
       topic_str = Utils::Trim(topic_str);
       part_id_str = Utils::Trim(part_id_str);
-      this->topic_ = topic_str;
-      this->partition_id_ = atoi(part_id_str.c_str());
+      topic_ = topic_str;
+      partition_id_ = atoi(part_id_str.c_str());
     }
   }
   buildPartitionKey();
@@ -167,21 +169,21 @@ Partition::Partition(const string& partition_info) {
 // part_str = topic:partition_id
 Partition::Partition(const NodeInfo& broker_info, const string& part_str) {
   vector<string> result;
-  this->topic_ = " ";
-  this->partition_id_ = 0;
-  this->broker_info_ = broker_info;
+  topic_ = " ";
+  partition_id_ = 0;
+  broker_info_ = broker_info;
   Utils::Split(part_str, result, delimiter::kDelimiterColon);
   if (result.size() >= 2) {
-    this->topic_ = result[0];
-    this->partition_id_ = atoi(result[1].c_str());
+    topic_ = result[0];
+    partition_id_ = atoi(result[1].c_str());
   }
   buildPartitionKey();
 }
 
 Partition::Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id) {
-  this->topic_ = topic;
-  this->partition_id_ = partition_id;
-  this->broker_info_ = broker_info;
+  topic_ = topic;
+  partition_id_ = partition_id;
+  broker_info_ = broker_info;
   buildPartitionKey();
 }
 
@@ -191,11 +193,12 @@ Partition::~Partition() {
 
 Partition& Partition::operator=(const Partition& target) {
   if (this != &target) {
-    this->topic_ = target.topic_;
-    this->partition_id_ = target.partition_id_;
-    this->broker_info_ = target.broker_info_;
-    this->partition_key_ = target.partition_key_;
-    this->partition_info_ = target.partition_info_;
+    topic_ = target.topic_;
+    partition_id_ = target.partition_id_;
+    broker_info_ = target.broker_info_;
+    partition_key_ = target.partition_key_;
+    partition_info_ = target.partition_info_;
+    buildPartitionKey();
   }
   return *this;
 }
@@ -204,44 +207,44 @@ bool Partition::operator==(const Partition& target) {
   if (this == &target) {
     return true;
   }
-  if (this->partition_info_ == target.partition_info_) {
+  if (partition_info_ == target.partition_info_) {
     return true;
   }
   return false;
 }
 
-const uint32_t Partition::GetBrokerId() const { return this->broker_info_.GetNodeId(); }
+const uint32_t Partition::GetBrokerId() const { return broker_info_.GetNodeId(); }
 
-const string& Partition::GetBrokerHost() const { return this->broker_info_.GetHost(); }
+const string& Partition::GetBrokerHost() const { return broker_info_.GetHost(); }
 
-const uint32_t Partition::GetBrokerPort() const { return this->broker_info_.GetPort(); }
+const uint32_t Partition::GetBrokerPort() const { return broker_info_.GetPort(); }
 
-const string& Partition::GetPartitionKey() const { return this->partition_key_; }
+const string& Partition::GetPartitionKey() const { return partition_key_; }
 
-const string& Partition::GetTopic() const { return this->topic_; }
+const string& Partition::GetTopic() const { return topic_; }
 
-const NodeInfo& Partition::GetBrokerInfo() const { return this->broker_info_; }
+const NodeInfo& Partition::GetBrokerInfo() const { return broker_info_; }
 
-const uint32_t Partition::GetPartitionId() const { return this->partition_id_; }
+const uint32_t Partition::GetPartitionId() const { return partition_id_; }
 
-const string& Partition::ToString() const { return this->partition_info_; }
+const string& Partition::ToString() const { return partition_info_; }
 
 void Partition::buildPartitionKey() {
   stringstream ss1;
-  ss1 << this->broker_info_.GetNodeId();
+  ss1 << broker_info_.GetNodeId();
   ss1 << delimiter::kDelimiterColon;
-  ss1 << this->topic_;
+  ss1 << topic_;
   ss1 << delimiter::kDelimiterColon;
-  ss1 << this->partition_id_;
-  this->partition_key_ = ss1.str();
+  ss1 << partition_id_;
+  partition_key_ = ss1.str();
 
   stringstream ss2;
-  ss2 << this->broker_info_.GetNodeInfo();
+  ss2 << broker_info_.GetNodeInfo();
   ss2 << delimiter::kDelimiterPound;
-  ss2 << this->topic_;
+  ss2 << topic_;
   ss2 << delimiter::kDelimiterColon;
-  ss2 << this->partition_id_;
-  this->partition_info_ = ss2.str();
+  ss2 << partition_id_;
+  partition_info_ = ss2.str();
 }
 
 PartitionExt::PartitionExt() : Partition() {
@@ -266,24 +269,24 @@ PartitionExt& PartitionExt::operator=(const PartitionExt& target) {
     // parent class
     Partition::operator=(target);
     // child class
-    this->is_last_consumed_ = target.is_last_consumed_;
-    this->cur_flowctrl_ = target.cur_flowctrl_;
-    this->cur_freqctrl_ = target.cur_freqctrl_;
-    this->next_stage_updtime_ = target.next_stage_updtime_;
-    this->next_slice_updtime_ = target.next_slice_updtime_;
-    this->limit_slice_msgsize_ = target.limit_slice_msgsize_;
-    this->cur_stage_msgsize_ = target.cur_stage_msgsize_;
-    this->cur_slice_msgsize_ = target.cur_slice_msgsize_;
-    this->total_zero_cnt_ = target.total_zero_cnt_;
-    this->booked_time_ = target.booked_time_;
-    this->booked_errcode_ = target.booked_errcode_;
-    this->booked_esc_limit_ = target.booked_esc_limit_;
-    this->booked_msgsize_ = target.booked_msgsize_;
-    this->booked_dlt_limit_ = target.booked_dlt_limit_;
-    this->booked_curdata_dlt_ = target.booked_curdata_dlt_;
-    this->booked_require_slow_ = target.booked_require_slow_;
-    this->booked_errcode_ = target.booked_errcode_;
-    this->booked_errcode_ = target.booked_errcode_;
+    is_last_consumed_ = target.is_last_consumed_;
+    cur_flowctrl_ = target.cur_flowctrl_;
+    cur_freqctrl_ = target.cur_freqctrl_;
+    next_stage_updtime_ = target.next_stage_updtime_;
+    next_slice_updtime_ = target.next_slice_updtime_;
+    limit_slice_msgsize_ = target.limit_slice_msgsize_;
+    cur_stage_msgsize_ = target.cur_stage_msgsize_;
+    cur_slice_msgsize_ = target.cur_slice_msgsize_;
+    total_zero_cnt_ = target.total_zero_cnt_;
+    booked_time_ = target.booked_time_;
+    booked_errcode_ = target.booked_errcode_;
+    booked_esc_limit_ = target.booked_esc_limit_;
+    booked_msgsize_ = target.booked_msgsize_;
+    booked_dlt_limit_ = target.booked_dlt_limit_;
+    booked_curdata_dlt_ = target.booked_curdata_dlt_;
+    booked_require_slow_ = target.booked_require_slow_;
+    booked_errcode_ = target.booked_errcode_;
+    booked_errcode_ = target.booked_errcode_;
   }
   return *this;
 }
@@ -291,21 +294,21 @@ PartitionExt& PartitionExt::operator=(const PartitionExt& target) {
 
 void PartitionExt::BookConsumeData(int32_t errcode, int32_t msg_size,
   bool req_esc_limit, int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow) {
-  this->booked_time_ = Utils::GetCurrentTimeMillis();
-  this->booked_errcode_ = errcode;
-  this->booked_esc_limit_ = req_esc_limit;
-  this->booked_msgsize_ = msg_size;
-  this->booked_dlt_limit_ = rsp_dlt_limit;
-  this->booked_curdata_dlt_ = last_datadlt;
-  this->booked_require_slow_ = require_slow;
+  booked_time_ = Utils::GetCurrentTimeMillis();
+  booked_errcode_ = errcode;
+  booked_esc_limit_ = req_esc_limit;
+  booked_msgsize_ = msg_size;
+  booked_dlt_limit_ = rsp_dlt_limit;
+  booked_curdata_dlt_ = last_datadlt;
+  booked_require_slow_ = require_slow;
 }
 
 int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
   const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed) {
-  int64_t dlt_time = Utils::GetCurrentTimeMillis() - this->booked_time_;
+  int64_t dlt_time = Utils::GetCurrentTimeMillis() - booked_time_;
   return ProcConsumeResult(def_flowctrl_handler, group_flowctrl_handler, filter_consume,
-    last_consumed, this->booked_errcode_, this->booked_msgsize_, this->booked_esc_limit_,
-    this->booked_dlt_limit_, this->booked_curdata_dlt_, this->booked_require_slow_) - dlt_time;
+    last_consumed, booked_errcode_, booked_msgsize_, booked_esc_limit_,
+    booked_dlt_limit_, booked_curdata_dlt_, booked_require_slow_) - dlt_time;
 }
 
 int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
@@ -314,7 +317,7 @@ int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_
   int64_t last_datadlt, bool require_slow) {
   // #lizard forgives
   // record consume status
-  this->is_last_consumed_ = last_consumed;
+  is_last_consumed_ = last_consumed;
   // Update strategy data values
   updateStrategyData(def_flowctrl_handler, group_flowctrl_handler, msg_size, last_datadlt);
   // Perform different strategies based on error codes
@@ -322,36 +325,36 @@ int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_
     case err_code::kErrNotFound:
     case err_code::kErrSuccess:
       if (msg_size == 0 && errcode != err_code::kErrSuccess) {
-        this->total_zero_cnt_ += 1;
+        total_zero_cnt_ += 1;
       } else {
-        this->total_zero_cnt_ = 0;
+        total_zero_cnt_ = 0;
       }
-      if (this->total_zero_cnt_ > 0) {
+      if (total_zero_cnt_ > 0) {
         if (group_flowctrl_handler.GetMinZeroCnt() != tb_config::kMaxIntValue) {
           return (int64_t)(group_flowctrl_handler.GetCurFreqLimitTime(
-            this->total_zero_cnt_, (int32_t)rsp_dlt_limit));
+            total_zero_cnt_, (int32_t)rsp_dlt_limit));
         } else {
           return (int64_t)def_flowctrl_handler.GetCurFreqLimitTime(
-            this->total_zero_cnt_, (int32_t)rsp_dlt_limit);
+            total_zero_cnt_, (int32_t)rsp_dlt_limit);
         }
       }
       if (req_esc_limit) {
         return 0;
       } else {
-        if (this->cur_stage_msgsize_ >= this->cur_flowctrl_.GetDataSizeLimit()
-          || this->cur_slice_msgsize_ >= this->limit_slice_msgsize_) {
-          return this->cur_flowctrl_.GetFreqMsLimit() > rsp_dlt_limit
-            ? this->cur_flowctrl_.GetFreqMsLimit() : rsp_dlt_limit;
+        if (cur_stage_msgsize_ >= cur_flowctrl_.GetDataSizeLimit()
+          || cur_slice_msgsize_ >= limit_slice_msgsize_) {
+          return cur_flowctrl_.GetFreqMsLimit() > rsp_dlt_limit
+            ? cur_flowctrl_.GetFreqMsLimit() : rsp_dlt_limit;
         }
         if (errcode == err_code::kErrSuccess) {
-          if (filter_consume && this->cur_freqctrl_.GetFreqMsLimit() >= 0) {
+          if (filter_consume && cur_freqctrl_.GetFreqMsLimit() >= 0) {
             if (require_slow) {
-              return this->cur_freqctrl_.GetZeroCnt();
+              return cur_freqctrl_.GetZeroCnt();
             } else {
-              return this->cur_freqctrl_.GetFreqMsLimit();
+              return cur_freqctrl_.GetFreqMsLimit();
             }
-          } else if (!filter_consume && this->cur_freqctrl_.GetDataSizeLimit() >=0) {
-            return this->cur_freqctrl_.GetDataSizeLimit();
+          } else if (!filter_consume && cur_freqctrl_.GetDataSizeLimit() >=0) {
+            return cur_freqctrl_.GetDataSizeLimit();
           }
         }
         return rsp_dlt_limit;
@@ -364,68 +367,68 @@ int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_
 }
 
 void PartitionExt::SetLastConsumed(bool last_consumed) {
-  this->is_last_consumed_ = last_consumed;
+  is_last_consumed_ = last_consumed;
 }
 
-bool PartitionExt::IsLastConsumed() {
-  return this->is_last_consumed_;
+bool PartitionExt::IsLastConsumed() const {
+  return is_last_consumed_;
 }
 
 void PartitionExt::resetParameters() {
-  this->is_last_consumed_ = false;
-  this->cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 20);
-  this->next_stage_updtime_ = 0;
-  this->next_slice_updtime_ = 0;
-  this->limit_slice_msgsize_ = 0;
-  this->cur_stage_msgsize_ = 0;
-  this->cur_slice_msgsize_ = 0;
-  this->total_zero_cnt_ = 0;
-  this->booked_time_ = 0;
-  this->booked_errcode_ = 0;
-  this->booked_esc_limit_ = false;
-  this->booked_msgsize_ = 0;
-  this->booked_dlt_limit_ = 0;
-  this->booked_curdata_dlt_ = 0;
-  this->booked_require_slow_ = false;
+  is_last_consumed_ = false;
+  cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 20);
+  next_stage_updtime_ = 0;
+  next_slice_updtime_ = 0;
+  limit_slice_msgsize_ = 0;
+  cur_stage_msgsize_ = 0;
+  cur_slice_msgsize_ = 0;
+  total_zero_cnt_ = 0;
+  booked_time_ = 0;
+  booked_errcode_ = 0;
+  booked_esc_limit_ = false;
+  booked_msgsize_ = 0;
+  booked_dlt_limit_ = 0;
+  booked_curdata_dlt_ = 0;
+  booked_require_slow_ = false;
 }
 
 void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
   const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt) {
   bool result = false;
   // Accumulated data received
-  this->cur_stage_msgsize_ += msg_size;
-  this->cur_slice_msgsize_ += msg_size;
+  cur_stage_msgsize_ += msg_size;
+  cur_slice_msgsize_ += msg_size;
   int64_t curr_time = Utils::GetCurrentTimeMillis();
   // Update strategy data values
-  if (curr_time > this->next_stage_updtime_) {
-    this->cur_stage_msgsize_ = 0;
-    this->cur_slice_msgsize_ = 0;
+  if (curr_time > next_stage_updtime_) {
+    cur_stage_msgsize_ = 0;
+    cur_slice_msgsize_ = 0;
     if (last_datadlt >= 0) {
-      result = group_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+      result = group_flowctrl_handler.GetCurDataLimit(last_datadlt, cur_flowctrl_);
       if (!result) {
-        result = def_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+        result = def_flowctrl_handler.GetCurDataLimit(last_datadlt, cur_flowctrl_);
         if (!result) {
-          this->cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 0);
+          cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 0);
         }
       }
-      group_flowctrl_handler.GetFilterCtrlItem(this->cur_freqctrl_);
-      if (this->cur_freqctrl_.GetFreqMsLimit() < 0) {
-        def_flowctrl_handler.GetFilterCtrlItem(this->cur_freqctrl_);
+      group_flowctrl_handler.GetFilterCtrlItem(cur_freqctrl_);
+      if (cur_freqctrl_.GetFreqMsLimit() < 0) {
+        def_flowctrl_handler.GetFilterCtrlItem(cur_freqctrl_);
       }
       curr_time = Utils::GetCurrentTimeMillis();
     }
-    this->limit_slice_msgsize_ = this->cur_flowctrl_.GetDataSizeLimit() / 12;
-    this->next_stage_updtime_ = curr_time + 60000;
-    this->next_slice_updtime_ = curr_time + 5000;
-  } else if (curr_time > this->next_slice_updtime_) {
-    this->cur_slice_msgsize_ = 0;
-    this->next_slice_updtime_ = curr_time + 5000;
+    limit_slice_msgsize_ = cur_flowctrl_.GetDataSizeLimit() / 12;
+    next_stage_updtime_ = curr_time + 60000;
+    next_slice_updtime_ = curr_time + 5000;
+  } else if (curr_time > next_slice_updtime_) {
+    cur_slice_msgsize_ = 0;
+    next_slice_updtime_ = curr_time + 5000;
   }
 }
 
 SubscribeInfo::SubscribeInfo() {
-  this->consumer_id_ = " ";
-  this->group_ = " ";
+  consumer_id_ = " ";
+  group_ = " ";
   buildSubInfo();
 }
 
@@ -434,8 +437,8 @@ SubscribeInfo::SubscribeInfo(const string& sub_info) {
   string::size_type pos = 0;
   string seg_key = delimiter::kDelimiterPound;
   string at_key = delimiter::kDelimiterAt;
-  this->consumer_id_ = " ";
-  this->group_ = " ";
+  consumer_id_ = " ";
+  group_ = " ";
   // parse sub_info
   pos = sub_info.find(seg_key);
   if (pos != string::npos) {
@@ -443,109 +446,111 @@ SubscribeInfo::SubscribeInfo(const string& sub_info) {
     consumer_info = Utils::Trim(consumer_info);
     string partition_info = sub_info.substr(pos + seg_key.size(), sub_info.size());
     partition_info = Utils::Trim(partition_info);
-    this->partitionext_ = PartitionExt(partition_info);
+    PartitionExt tmp_part(partition_info);
+    partitionext_ = tmp_part;
     pos = consumer_info.find(at_key);
-    this->consumer_id_ = consumer_info.substr(0, pos);
-    this->consumer_id_ = Utils::Trim(this->consumer_id_);
-    this->group_ = consumer_info.substr(pos + at_key.size(), consumer_info.size());
-    this->group_ = Utils::Trim(this->group_);
+    consumer_id_ = consumer_info.substr(0, pos);
+    consumer_id_ = Utils::Trim(consumer_id_);
+    group_ = consumer_info.substr(pos + at_key.size(), consumer_info.size());
+    group_ = Utils::Trim(group_);
   }
   buildSubInfo();
 }
 
 SubscribeInfo::SubscribeInfo(const string& consumer_id,
         const string& group_name, const PartitionExt& partition_ext) {
-  this->consumer_id_ = consumer_id;
-  this->group_ = group_name;
-  this->partitionext_ = partition_ext;
+  consumer_id_ = consumer_id;
+  group_ = group_name;
+  partitionext_ = partition_ext;
   buildSubInfo();
 }
 
 SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
   if (this != &target) {
-    this->consumer_id_ = target.consumer_id_;
-    this->group_ = target.group_;
-    this->partitionext_ = target.partitionext_;
+    consumer_id_ = target.consumer_id_;
+    group_ = target.group_;
+    partitionext_ = target.partitionext_;
+    buildSubInfo();
   }
   return *this;
 }
 
-const string& SubscribeInfo::GetConsumerId() const { return this->consumer_id_; }
+const string& SubscribeInfo::GetConsumerId() const { return consumer_id_; }
 
-const string& SubscribeInfo::GetGroup() const { return this->group_; }
+const string& SubscribeInfo::GetGroup() const { return group_; }
 
-const PartitionExt& SubscribeInfo::GetPartitionExt() const { return this->partitionext_; }
+const PartitionExt& SubscribeInfo::GetPartitionExt() const { return partitionext_; }
 
-const uint32_t SubscribeInfo::GgetBrokerId() const { return this->partitionext_.GetBrokerId(); }
+const uint32_t SubscribeInfo::GgetBrokerId() const { return partitionext_.GetBrokerId(); }
 
-const string& SubscribeInfo::GetBrokerHost() const { return this->partitionext_.GetBrokerHost(); }
+const string& SubscribeInfo::GetBrokerHost() const { return partitionext_.GetBrokerHost(); }
 
-const uint32_t SubscribeInfo::GetBrokerPort() const { return this->partitionext_.GetBrokerPort(); }
+const uint32_t SubscribeInfo::GetBrokerPort() const { return partitionext_.GetBrokerPort(); }
 
-const string& SubscribeInfo::GetTopic() const { return this->partitionext_.GetTopic(); }
+const string& SubscribeInfo::GetTopic() const { return partitionext_.GetTopic(); }
 
 const uint32_t SubscribeInfo::GetPartitionId() const {
-  return this->partitionext_.GetPartitionId();
+  return partitionext_.GetPartitionId();
 }
 
-const string& SubscribeInfo::ToString() const { return this->sub_info_; }
+const string& SubscribeInfo::ToString() const { return sub_info_; }
 
 void SubscribeInfo::buildSubInfo() {
   stringstream ss;
-  ss << this->consumer_id_;
+  ss << consumer_id_;
   ss << delimiter::kDelimiterAt;
-  ss << this->group_;
+  ss << group_;
   ss << delimiter::kDelimiterPound;
-  ss << this->partitionext_.ToString();
-  this->sub_info_ = ss.str();
+  ss << partitionext_.ToString();
+  sub_info_ = ss.str();
 }
 
 ConsumerEvent::ConsumerEvent() {
-  this->rebalance_id_ = tb_config::kInvalidValue;
-  this->event_type_ = tb_config::kInvalidValue;
-  this->event_status_ = tb_config::kInvalidValue;
+  rebalance_id_ = tb_config::kInvalidValue;
+  event_type_ = tb_config::kInvalidValue;
+  event_status_ = tb_config::kInvalidValue;
 }
 
 ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
-  this->rebalance_id_ = target.rebalance_id_;
-  this->event_type_ = target.event_type_;
-  this->event_status_ = target.event_status_;
-  this->subscribe_list_ = target.subscribe_list_;
+  rebalance_id_ = target.rebalance_id_;
+  event_type_ = target.event_type_;
+  event_status_ = target.event_status_;
+  subscribe_list_ = target.subscribe_list_;
 }
 
 ConsumerEvent::ConsumerEvent(int64_t rebalance_id, int32_t event_type,
                              const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
   list<SubscribeInfo>::const_iterator it;
-  this->rebalance_id_ = rebalance_id;
-  this->event_type_ = event_type;
-  this->event_status_ = event_status;
+  rebalance_id_ = rebalance_id;
+  event_type_ = event_type;
+  event_status_ = event_status;
   for (it = subscribeInfo_lst.begin(); it != subscribeInfo_lst.end(); ++it) {
-    this->subscribe_list_.push_back(*it);
+    subscribe_list_.push_back(*it);
   }
 }
 
 ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
   if (this != &target) {
-    this->rebalance_id_ = target.rebalance_id_;
-    this->event_type_ = target.event_type_;
-    this->event_status_ = target.event_status_;
-    this->subscribe_list_ = target.subscribe_list_;
+    rebalance_id_ = target.rebalance_id_;
+    event_type_ = target.event_type_;
+    event_status_ = target.event_status_;
+    subscribe_list_ = target.subscribe_list_;
   }
   return *this;
 }
 
-const int64_t ConsumerEvent::GetRebalanceId() const { return this->rebalance_id_; }
+const int64_t ConsumerEvent::GetRebalanceId() const { return rebalance_id_; }
 
-const int32_t ConsumerEvent::GetEventType() const { return this->event_type_; }
+const int32_t ConsumerEvent::GetEventType() const { return event_type_; }
 
-const int32_t ConsumerEvent::GetEventStatus() const { return this->event_status_; }
+const int32_t ConsumerEvent::GetEventStatus() const { return event_status_; }
 
-void ConsumerEvent::SetEventType(int32_t event_type) { this->event_type_ = event_type; }
+void ConsumerEvent::SetEventType(int32_t event_type) { event_type_ = event_type; }
 
-void ConsumerEvent::SetEventStatus(int32_t event_status) { this->event_status_ = event_status; }
+void ConsumerEvent::SetEventStatus(int32_t event_status) { event_status_ = event_status; }
 
 const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
-  return this->subscribe_list_;
+  return subscribe_list_;
 }
 
 string ConsumerEvent::ToString() {
@@ -553,13 +558,13 @@ string ConsumerEvent::ToString() {
   stringstream ss;
   list<SubscribeInfo>::const_iterator it;
   ss << "ConsumerEvent [rebalanceId=";
-  ss << this->rebalance_id_;
+  ss << rebalance_id_;
   ss << ", type=";
-  ss << this->event_type_;
+  ss << event_type_;
   ss << ", status=";
-  ss << this->event_status_;
+  ss << event_status_;
   ss << ", subscribeInfoList=[";
-  for (it = this->subscribe_list_.begin(); it != this->subscribe_list_.end(); ++it) {
+  for (it = subscribe_list_.begin(); it != subscribe_list_.end(); ++it) {
     if (count++ > 0) {
       ss << ",";
     }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.h
new file mode 100644
index 0000000..6117d65
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.h
@@ -0,0 +1,188 @@
+/**
+ * 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 TUBEMQ_CLIENT_META_INFO_H_
+#define TUBEMQ_CLIENT_META_INFO_H_
+
+#include <stdint.h>
+
+#include <list>
+#include <string>
+
+#include "flowctrl_def.h"
+
+namespace tubemq {
+
+using std::list;
+using std::map;
+using std::string;
+
+class NodeInfo {
+ public:
+  NodeInfo();
+  NodeInfo(bool is_broker, const string& node_info);
+  NodeInfo(const string& node_host, uint32_t node_port);
+  NodeInfo(int32_t node_id, const string& node_host, uint32_t node_port);
+  ~NodeInfo();
+  NodeInfo& operator=(const NodeInfo& target);
+  bool operator==(const NodeInfo& target);
+  bool operator<(const NodeInfo& target) const;
+  const uint32_t GetNodeId() const;
+  const string& GetHost() const;
+  const uint32_t GetPort() const;
+  const string& GetAddrInfo() const;
+  const string& GetNodeInfo() const;
+
+ private:
+  void buildStrInfo();
+
+ private:
+  uint32_t node_id_;
+  string node_host_;
+  uint32_t node_port_;
+  // ip:port
+  string addr_info_;
+  // id:ip:port
+  string node_info_;
+};
+
+class Partition {
+ public:
+  Partition();
+  explicit Partition(const string& partition_info);
+  Partition(const NodeInfo& broker_info, const string& part_str);
+  Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id);
+  ~Partition();
+  Partition& operator=(const Partition& target);
+  bool operator==(const Partition& target);
+  const uint32_t GetBrokerId() const;
+  const string& GetBrokerHost() const;
+  const uint32_t GetBrokerPort() const;
+  const string& GetPartitionKey() const;
+  const string& GetTopic() const;
+  const NodeInfo& GetBrokerInfo() const;
+  const uint32_t GetPartitionId() const;
+  const string& ToString() const;
+
+ private:
+  void buildPartitionKey();
+
+ private:
+  string topic_;
+  NodeInfo broker_info_;
+  uint32_t partition_id_;
+  string partition_key_;
+  string partition_info_;
+};
+
+class PartitionExt : public Partition {
+ public:
+  PartitionExt();
+  explicit PartitionExt(const string& partition_info);
+  PartitionExt(const NodeInfo& broker_info, const string& part_str);
+  ~PartitionExt();
+  PartitionExt& operator=(const PartitionExt& target);
+  void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
+    int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
+    int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
+    int64_t last_datadlt, bool require_slow);
+  void SetLastConsumed(bool last_consumed);
+  bool IsLastConsumed() const;
+
+ private:
+  void resetParameters();
+  void updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt);
+
+ private:
+  bool is_last_consumed_;
+  FlowCtrlResult cur_flowctrl_;
+  FlowCtrlItem cur_freqctrl_;
+  int64_t next_stage_updtime_;
+  int64_t next_slice_updtime_;
+  int64_t limit_slice_msgsize_;
+  int64_t cur_stage_msgsize_;
+  int64_t cur_slice_msgsize_;
+  int32_t total_zero_cnt_;
+  int64_t booked_time_;
+  int32_t booked_errcode_;
+  bool    booked_esc_limit_;
+  int32_t booked_msgsize_;
+  int64_t booked_dlt_limit_;
+  int64_t booked_curdata_dlt_;
+  bool    booked_require_slow_;
+};
+
+class SubscribeInfo {
+ public:
+  SubscribeInfo();
+  explicit SubscribeInfo(const string& sub_info);
+  SubscribeInfo(const string& consumer_id,
+        const string& group_name, const PartitionExt& partition_ext);
+  SubscribeInfo& operator=(const SubscribeInfo& target);
+  const string& GetConsumerId() const;
+  const string& GetGroup() const;
+  const PartitionExt& GetPartitionExt() const;
+  const uint32_t GgetBrokerId() const;
+  const string& GetBrokerHost() const;
+  const uint32_t GetBrokerPort() const;
+  const string& GetTopic() const;
+  const uint32_t GetPartitionId() const;
+  const string& ToString() const;
+
+ private:
+  void buildSubInfo();
+
+ private:
+  string consumer_id_;
+  string group_;
+  PartitionExt partitionext_;
+  string sub_info_;
+};
+
+class ConsumerEvent {
+ public:
+  ConsumerEvent();
+  ConsumerEvent(const ConsumerEvent& target);
+  ConsumerEvent(int64_t rebalance_id, int32_t event_type,
+                const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status);
+  ConsumerEvent& operator=(const ConsumerEvent& target);
+  const int64_t GetRebalanceId() const;
+  const int32_t GetEventType() const;
+  const int32_t GetEventStatus() const;
+  void SetEventType(int32_t event_type);
+  void SetEventStatus(int32_t event_status);
+  const list<SubscribeInfo>& GetSubscribeInfoList() const;
+  string ToString();
+
+ private:
+  int64_t rebalance_id_;
+  int32_t event_type_;
+  int32_t event_status_;
+  list<SubscribeInfo> subscribe_list_;
+};
+
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_META_INFO_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/noncopyable.h b/tubemq-client-twins/tubemq-client-cpp/src/noncopyable.h
new file mode 100644
index 0000000..9afbf52
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/noncopyable.h
@@ -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.
+ */
+
+#ifndef _TUBUMQ_NONCOPYABLE_H
+#define _TUBUMQ_NONCOPYABLE_H
+
+namespace tubemq {
+
+class noncopyable {
+ public:
+  noncopyable(const noncopyable&) = delete;
+  void operator=(const noncopyable&) = delete;
+
+ protected:
+  noncopyable() = default;
+  ~noncopyable() = default;
+};
+
+}  // namespace tubemq
+
+#endif  // _TUBUMQ_NONCOPYABLE_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index f97535c..8bc9067 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -17,75 +17,68 @@
  * under the License.
  */
 
-#include "tubemq/rmt_data_cache.h"
+#include "rmt_data_cache.h"
 
 #include <stdlib.h>
-#include <string>
-
-#include "tubemq/client_service.h"
-#include "tubemq/const_config.h"
-#include "tubemq/meta_info.h"
-#include "tubemq/utils.h"
 
+#include <string>
 
+#include "client_service.h"
+#include "const_config.h"
+#include "logger.h"
+#include "meta_info.h"
+#include "utils.h"
 
 namespace tubemq {
 
 using std::lock_guard;
 using std::unique_lock;
-using namespace std::placeholders;
-
 
 RmtDataCacheCsm::RmtDataCacheCsm() {
   under_groupctrl_.Set(false);
   last_checktime_.Set(0);
+  cur_part_cnt_.Set(0);
 }
 
 RmtDataCacheCsm::~RmtDataCacheCsm() {
-  // 
+  //
 }
 
-void RmtDataCacheCsm::SetConsumerInfo(const string& client_id,
-                                            const string& group_name) {
+void RmtDataCacheCsm::SetConsumerInfo(const string& client_id, const string& group_name) {
   consumer_id_ = client_id;
   group_name_ = group_name;
 }
 
-void RmtDataCacheCsm::UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
-                                                 const string& flowctrl_info) {
+void RmtDataCacheCsm::UpdateDefFlowCtrlInfo(int64_t flowctrl_id, const string& flowctrl_info) {
   if (flowctrl_id != def_flowctrl_handler_.GetFlowCtrlId()) {
-    def_flowctrl_handler_.UpdateDefFlowCtrlInfo(true,
-      tb_config::kInvalidValue, flowctrl_id, flowctrl_info);
+    def_flowctrl_handler_.UpdateDefFlowCtrlInfo(true, tb_config::kInvalidValue, flowctrl_id,
+                                                flowctrl_info);
   }
 }
-void RmtDataCacheCsm::UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
-                             int64_t flowctrl_id, const string& flowctrl_info) {
+void RmtDataCacheCsm::UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id, int64_t flowctrl_id,
+                                              const string& flowctrl_info) {
   if (flowctrl_id != group_flowctrl_handler_.GetFlowCtrlId()) {
-    group_flowctrl_handler_.UpdateDefFlowCtrlInfo(false,
-                qyrpriority_id, flowctrl_id, flowctrl_info);
+    group_flowctrl_handler_.UpdateDefFlowCtrlInfo(false, qyrpriority_id, flowctrl_id,
+                                                  flowctrl_info);
   }
   if (qyrpriority_id != group_flowctrl_handler_.GetQryPriorityId()) {
-    this->group_flowctrl_handler_.SetQryPriorityId(qyrpriority_id);
+    group_flowctrl_handler_.SetQryPriorityId(qyrpriority_id);
   }
   // update current if under group flowctrl
   int64_t cur_time = Utils::GetCurrentTimeMillis();
   if (cur_time - last_checktime_.Get() > 10000) {
     FlowCtrlResult flowctrl_result;
-    this->under_groupctrl_.Set(
-      group_flowctrl_handler_.GetCurDataLimit(
-        tb_config::kMaxLongValue, flowctrl_result));
+    under_groupctrl_.Set(
+        group_flowctrl_handler_.GetCurDataLimit(tb_config::kMaxLongValue, flowctrl_result));
     last_checktime_.Set(cur_time);
   }
 }
 
 const int64_t RmtDataCacheCsm::GetGroupQryPriorityId() const {
-  return this->group_flowctrl_handler_.GetQryPriorityId();
-}
-
-bool RmtDataCacheCsm::IsUnderGroupCtrl() {
-  return this->under_groupctrl_.Get();
+  return group_flowctrl_handler_.GetQryPriorityId();
 }
 
+bool RmtDataCacheCsm::IsUnderGroupCtrl() { return under_groupctrl_.Get(); }
 
 void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   //
@@ -99,6 +92,7 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   lock_guard<mutex> lck(meta_lock_);
   it_map = partitions_.find(partition_key);
   if (it_map == partitions_.end()) {
+    cur_part_cnt_.GetAndIncrement();
     partitions_[partition_key] = partition_ext;
     it_topic = topic_partition_.find(partition_ext.GetTopic());
     if (it_topic == topic_partition_.end()) {
@@ -126,8 +120,23 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   resetIdlePartition(partition_key, true);
 }
 
-bool RmtDataCacheCsm::SelectPartition(string &err_info,
-                        PartitionExt& partition_ext, string& confirm_context) {
+int32_t RmtDataCacheCsm::GetCurConsumeStatus() {
+  lock_guard<mutex> lck(meta_lock_);
+  if (partitions_.empty()) {
+    return err_code::kErrNoPartAssigned;
+  }
+  if (index_partitions_.empty()) {
+    if (partition_useds_.empty()) {
+      return err_code::kErrAllPartInUse;
+    } else {
+      return err_code::kErrAllPartWaiting;
+    }
+  }
+  return err_code::kErrSuccess;
+}
+
+bool RmtDataCacheCsm::SelectPartition(int32_t& error_code, string& err_info,
+                                      PartitionExt& partition_ext, string& confirm_context) {
   bool result = false;
   int64_t booked_time = 0;
   string partition_key;
@@ -135,14 +144,22 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
   // lock operate
   lock_guard<mutex> lck(meta_lock_);
   if (partitions_.empty()) {
+    error_code = err_code::kErrNoPartAssigned;
     err_info = "No partition info in local cache, please retry later!";
     result = false;
   } else {
     if (index_partitions_.empty()) {
-      err_info = "No idle partition to consume, please retry later!";
+      if (partition_useds_.empty()) {
+        error_code = err_code::kErrAllPartInUse;
+        err_info = "No idle partition to consume, please retry later!";
+      } else {
+        error_code = err_code::kErrAllPartWaiting;
+        err_info = "All partitions reach max position, please retry later!";
+      }
       result = false;
     } else {
       result = false;
+      error_code = err_code::kErrAllPartInUse;
       err_info = "No idle partition to consume data 2, please retry later!";
       booked_time = Utils::GetCurrentTimeMillis();
       partition_key = index_partitions_.front();
@@ -160,10 +177,19 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
   return result;
 }
 
-void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key,
-                     int64_t curr_offset, int32_t err_code, bool esc_limit,
-                  int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt,
-                     bool require_slow) {
+void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t curr_offset) {
+  map<string, PartitionExt>::iterator it_part;
+  // book partition offset info
+  if (curr_offset >= 0) {
+    lock_guard<mutex> lck1(data_book_mutex_);
+    partition_offset_[partition_key] = curr_offset;
+  }
+}
+
+void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t curr_offset,
+                                        int32_t error_code, bool esc_limit, int32_t msg_size,
+                                        int64_t limit_dlt, int64_t cur_data_dlt,
+                                        bool require_slow) {
   map<string, PartitionExt>::iterator it_part;
   // book partition offset info
   if (curr_offset >= 0) {
@@ -174,44 +200,53 @@ void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key,
   lock_guard<mutex> lck2(meta_lock_);
   it_part = partitions_.find(partition_key);
   if (it_part != partitions_.end()) {
-    it_part->second.BookConsumeData(err_code, msg_size,
-              esc_limit, limit_dlt, cur_data_dlt, require_slow);
+    it_part->second.BookConsumeData(error_code, msg_size, esc_limit, limit_dlt, cur_data_dlt,
+                                    require_slow);
+  }
+}
+
+bool RmtDataCacheCsm::IsPartitionInUse(string partition_key, int64_t used_time) {
+  map<string, int64_t>::iterator it_used;
+  lock_guard<mutex> lck(meta_lock_);
+  it_used = partition_useds_.find(partition_key);
+  if (it_used == partition_useds_.end() || it_used->second != used_time) {
+    return false;
   }
+  return true;
 }
 
 // success process release partition
-bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
-                                 const string& confirm_context, bool is_consumed) {
+bool RmtDataCacheCsm::RelPartition(string& err_info, bool filter_consume,
+                                   const string& confirm_context, bool is_consumed) {
   return inRelPartition(err_info, true, filter_consume, confirm_context, is_consumed);
 }
 
 // release partiton without response return
-bool RmtDataCacheCsm::RelPartition(string &err_info,
-                                 const string& confirm_context, bool is_consumed) {
+bool RmtDataCacheCsm::RelPartition(string& err_info, const string& confirm_context,
+                                   bool is_consumed) {
   return inRelPartition(err_info, true, false, confirm_context, is_consumed);
 }
 
 // release partiton with error response return
-bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
-                              const string& confirm_context, bool is_consumed,
-                              int64_t curr_offset, int32_t err_code, bool esc_limit,
-                              int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt) {
+bool RmtDataCacheCsm::RelPartition(string& err_info, bool filter_consume,
+                                   const string& confirm_context, bool is_consumed,
+                                   int64_t curr_offset, int32_t error_code, bool esc_limit,
+                                   int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt) {
   int64_t booked_time;
-  string  partition_key;
+  string partition_key;
   // parse confirm context
-  bool result = parseConfirmContext(err_info,
-                      confirm_context, partition_key, booked_time);
+  bool result = parseConfirmContext(err_info, confirm_context, partition_key, booked_time);
   if (!result) {
     return false;
   }
-  BookedPartionInfo(partition_key, curr_offset, err_code,
-            esc_limit, msg_size, limit_dlt, cur_data_dlt, false);
-  return inRelPartition(err_info, true,
-    filter_consume, confirm_context, is_consumed);
+  BookedPartionInfo(partition_key, curr_offset, error_code, esc_limit, msg_size, limit_dlt,
+                    cur_data_dlt, false);
+  return inRelPartition(err_info, true, filter_consume, confirm_context, is_consumed);
 }
 
 void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
-            list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions) {
+                                       list<PartitionExt>& subscribed_partitions,
+                                       list<PartitionExt>& unsub_partitions) {
   //
   map<string, PartitionExt>::iterator it_part;
   list<SubscribeInfo>::const_iterator it_lst;
@@ -244,8 +279,7 @@ void RmtDataCacheCsm::GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst)
   }
 }
 
-void RmtDataCacheCsm::GetAllBrokerPartitions(
-                    map<NodeInfo, list<PartitionExt> >& broker_parts) {
+void RmtDataCacheCsm::GetAllClosedBrokerParts(map<NodeInfo, list<PartitionExt> >& broker_parts) {
   map<string, PartitionExt>::iterator it_part;
   map<NodeInfo, list<PartitionExt> >::iterator it_broker;
 
@@ -287,7 +321,7 @@ void RmtDataCacheCsm::GetRegBrokers(list<NodeInfo>& brokers) {
 }
 
 void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
-                                            list<PartitionExt>& partition_list) {
+                                           list<PartitionExt>& partition_list) {
   set<string>::iterator it_key;
   map<NodeInfo, set<string> >::iterator it_broker;
   map<string, PartitionExt>::iterator it_part;
@@ -296,8 +330,7 @@ void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
   lock_guard<mutex> lck(meta_lock_);
   it_broker = broker_partition_.find(broker_info);
   if (it_broker != broker_partition_.end()) {
-    for (it_key = it_broker->second.begin();
-    it_key != it_broker->second.end(); it_key++) {
+    for (it_key = it_broker->second.begin(); it_key != it_broker->second.end(); it_key++) {
       it_part = partitions_.find(*it_key);
       if (it_part != partitions_.end()) {
         partition_list.push_back(it_part->second);
@@ -306,7 +339,6 @@ void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
   }
 }
 
-
 void RmtDataCacheCsm::GetCurPartitionOffsets(map<string, int64_t> part_offset_map) {
   map<string, int64_t>::iterator it;
 
@@ -317,16 +349,13 @@ void RmtDataCacheCsm::GetCurPartitionOffsets(map<string, int64_t> part_offset_ma
   }
 }
 
-
 //
-bool RmtDataCacheCsm::RemovePartition(string &err_info,
-                                  const string& confirm_context) {
+bool RmtDataCacheCsm::RemovePartition(string& err_info, const string& confirm_context) {
   int64_t booked_time;
-  string  partition_key;
+  string partition_key;
   set<string> partition_keys;
   // parse confirm context
-  bool result = parseConfirmContext(err_info,
-                      confirm_context, partition_key, booked_time);
+  bool result = parseConfirmContext(err_info, confirm_context, partition_key, booked_time);
   if (!result) {
     return false;
   }
@@ -360,7 +389,8 @@ void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
 }
 
 void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
-        bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts) {
+                                            bool is_processing_rollback,
+                                            map<NodeInfo, list<PartitionExt> >& broker_parts) {
   //
   string part_key;
   list<SubscribeInfo>::const_iterator it;
@@ -398,29 +428,56 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
   }
 }
 
+void RmtDataCacheCsm::handleExpiredPartitions(int64_t max_wait_period_ms) {
+  int64_t curr_time;
+  set<string> expired_keys;
+  set<string>::iterator it_lst;
+  map<string, int64_t>::iterator it_used;
+  map<string, PartitionExt>::iterator it_map;
+
+  lock_guard<mutex> lck(meta_lock_);
+  if (!partition_useds_.empty()) {
+    curr_time = Utils::GetCurrentTimeMillis();
+    for (it_used = partition_useds_.begin();
+      it_used != partition_useds_.end(); ++it_used) {
+      if (curr_time - it_used->second > max_wait_period_ms) {
+        expired_keys.insert(it_used->first);
+        it_map = partitions_.find(it_used->first);
+        if (it_map != partitions_.end()) {
+          it_map->second.SetLastConsumed(false);
+        }
+      }
+    }
+    if (!expired_keys.empty()) {
+      for (it_lst = expired_keys.begin();
+        it_lst != expired_keys.end(); it_lst++) {
+        resetIdlePartition(*it_lst, true);
+      }
+    }
+  }
+}
+
 
 
 bool RmtDataCacheCsm::IsPartitionFirstReg(const string& partition_key) {
-  bool result = false;
   map<string, bool>::iterator it;
-
   lock_guard<mutex> lck(data_book_mutex_);
   it = part_reg_booked_.find(partition_key);
   if (it == part_reg_booked_.end()) {
     part_reg_booked_[partition_key] = true;
   }
-  return result;
+  return part_reg_booked_[partition_key];
 }
 
 void RmtDataCacheCsm::OfferEvent(const ConsumerEvent& event) {
   unique_lock<mutex> lck(event_read_mutex_);
-  this->rebalance_events_.push_back(event);
+  rebalance_events_.push_back(event);
   event_read_cond_.notify_all();
 }
 
 void RmtDataCacheCsm::TakeEvent(ConsumerEvent& event) {
   unique_lock<mutex> lck(event_read_mutex_);
-  while (this->rebalance_events_.empty()) {
+  while (rebalance_events_.empty()) {
     event_read_cond_.wait(lck);
   }
   event = rebalance_events_.front();
@@ -434,41 +491,65 @@ void RmtDataCacheCsm::ClearEvent() {
 
 void RmtDataCacheCsm::OfferEventResult(const ConsumerEvent& event) {
   lock_guard<mutex> lck(event_write_mutex_);
-  this->rebalance_events_.push_back(event);
+  rebalance_results_.push_back(event);
 }
 
 bool RmtDataCacheCsm::PollEventResult(ConsumerEvent& event) {
   bool result = false;
   lock_guard<mutex> lck(event_write_mutex_);
-  if (!rebalance_events_.empty()) {
-    event = rebalance_events_.front();
-    rebalance_events_.pop_front();
+  if (!rebalance_results_.empty()) {
+    event = rebalance_results_.front();
+    rebalance_results_.pop_front();
     result = true;
   }
   return result;
 }
 
-void RmtDataCacheCsm::HandleTimeout(const string partition_key,
-                                          const asio::error_code& error) {
+void RmtDataCacheCsm::HandleTimeout(const string partition_key, const asio::error_code& error) {
   if (!error) {
     lock_guard<mutex> lck(meta_lock_);
     resetIdlePartition(partition_key, true);
   }
 }
 
+int RmtDataCacheCsm::IncrAndGetHBError(NodeInfo broker) {
+  int count = 0;
+  map<NodeInfo, int>::iterator it_map;
+  lock_guard<mutex> lck(status_mutex_);
+  it_map = broker_status_.find(broker);
+  if (it_map == broker_status_.end()) {
+    broker_status_[broker] = 1;
+    count = 1;
+  } else {
+    count = ++it_map->second;
+  }
+  return count;
+}
+
+void RmtDataCacheCsm::ResetHBError(NodeInfo broker) {
+  map<NodeInfo, int>::iterator it_map;
+  lock_guard<mutex> lck(status_mutex_);
+  it_map = broker_status_.find(broker);
+  if (it_map != broker_status_.end()) {
+    it_map->second = 0;
+  }
+}
+
+
 void RmtDataCacheCsm::addDelayTimer(const string& partition_key, int64_t delay_time) {
   // add timer
-  tuple<int64_t, SteadyTimerPtr> timer = 
-      std::make_tuple(Utils::GetCurrentTimeMillis(),
-      TubeMQService::Instance().GetTimerExecutorPool().Get()->CreateSteadyTimer());
+  tuple<int64_t, SteadyTimerPtr> timer = std::make_tuple(
+      Utils::GetCurrentTimeMillis(),
+      TubeMQService::Instance()->CreateTimer());
   std::get<1>(timer)->expires_after(std::chrono::milliseconds(delay_time));
-  std::get<1>(timer)->async_wait(std::bind(&RmtDataCacheCsm::HandleTimeout, this, partition_key, _1));
-  partition_timeouts_.insert(std::make_pair(partition_key, timer));          
+  std::get<1>(timer)->async_wait(
+      std::bind(&RmtDataCacheCsm::HandleTimeout, this, partition_key, std::placeholders::_1));
+  partition_timeouts_.insert(std::make_pair(partition_key, timer));
 }
 
 void RmtDataCacheCsm::resetIdlePartition(const string& partition_key, bool need_reuse) {
   map<string, PartitionExt>::iterator it_map;
-  map<string, tuple<int64_t, SteadyTimerPtr> >::iterator it_timeout; 
+  map<string, tuple<int64_t, SteadyTimerPtr> >::iterator it_timeout;
   partition_useds_.erase(partition_key);
   it_timeout = partition_timeouts_.find(partition_key);
   if (it_timeout != partition_timeouts_.end()) {
@@ -483,16 +564,16 @@ void RmtDataCacheCsm::resetIdlePartition(const string& partition_key, bool need_
   }
 }
 
-void RmtDataCacheCsm::buildConfirmContext(const string& partition_key,
-                                   int64_t booked_time, string& confirm_context) {
+void RmtDataCacheCsm::buildConfirmContext(const string& partition_key, int64_t booked_time,
+                                          string& confirm_context) {
   confirm_context.clear();
   confirm_context += partition_key;
   confirm_context += delimiter::kDelimiterAt;
   confirm_context += Utils::Long2str(booked_time);
 }
 
-bool RmtDataCacheCsm::parseConfirmContext(string &err_info,
-     const string& confirm_context, string& partition_key, int64_t& booked_time) {
+bool RmtDataCacheCsm::parseConfirmContext(string& err_info, const string& confirm_context,
+                                          string& partition_key, int64_t& booked_time) {
   //
   vector<string> result;
   Utils::Split(confirm_context, result, delimiter::kDelimiterAt);
@@ -528,19 +609,19 @@ void RmtDataCacheCsm::rmvMetaInfo(const string& partition_key) {
     }
     partitions_.erase(partition_key);
     part_subinfo_.erase(partition_key);
+    cur_part_cnt_.DecrementAndGet();
   }
 }
 
-bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
-                     bool filter_consume, const string& confirm_context, bool is_consumed) {
+bool RmtDataCacheCsm::inRelPartition(string& err_info, bool need_delay_check, bool filter_consume,
+                                     const string& confirm_context, bool is_consumed) {
   int64_t delay_time;
   int64_t booked_time;
-  string  partition_key;
+  string partition_key;
   map<string, PartitionExt>::iterator it_part;
   map<string, int64_t>::iterator it_used;
-  // parse confirm context  
-  bool result = parseConfirmContext(err_info,
-                      confirm_context, partition_key, booked_time);
+  // parse confirm context
+  bool result = parseConfirmContext(err_info, confirm_context, partition_key, booked_time);
   if (!result) {
     return false;
   }
@@ -565,8 +646,8 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
         index_partitions_.remove(partition_key);
         delay_time = 0;
         if (need_delay_check) {
-          delay_time = it_part->second.ProcConsumeResult(def_flowctrl_handler_,
-                        group_flowctrl_handler_, filter_consume, is_consumed);
+          delay_time = it_part->second.ProcConsumeResult(
+              def_flowctrl_handler_, group_flowctrl_handler_, filter_consume, is_consumed);
         }
         if (delay_time > 10) {
           addDelayTimer(partition_key, delay_time);
@@ -585,5 +666,4 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
   return result;
 }
 
-
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.h
new file mode 100644
index 0000000..5b77eb7
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.h
@@ -0,0 +1,166 @@
+/**
+ * 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 TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
+#define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
+
+#include <stdint.h>
+
+#include <condition_variable>
+#include <list>
+#include <map>
+#include <mutex>
+#include <set>
+#include <string>
+#include <tuple>
+
+#include "executor_pool.h"
+#include "flowctrl_def.h"
+#include "meta_info.h"
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_errcode.h"
+
+
+
+
+
+namespace tubemq {
+
+using std::condition_variable;
+using std::map;
+using std::set;
+using std::list;
+using std::mutex;
+using std::string;
+using std::tuple;
+
+
+
+// consumer remote data cache
+class RmtDataCacheCsm {
+ public:
+  RmtDataCacheCsm();
+  ~RmtDataCacheCsm();
+  void SetConsumerInfo(const string& client_id, const string& group_name);
+  void UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
+                                     const string& flowctrl_info);
+  void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
+                 int64_t flowctrl_id, const string& flowctrl_info);
+  const int64_t GetGroupQryPriorityId() const;
+  const int64_t GetDefFlowCtrlId() const { return def_flowctrl_handler_.GetFlowCtrlId(); }
+  const int64_t GetGroupFlowCtrlId() const { return group_flowctrl_handler_.GetFlowCtrlId(); }
+  bool IsUnderGroupCtrl();
+  int32_t GetCurConsumeStatus();
+  void handleExpiredPartitions(int64_t max_wait_period_ms);
+  int32_t GetCurPartCount() const { return cur_part_cnt_.Get(); }
+  bool IsPartitionInUse(string partition_key, int64_t used_time);
+  void AddNewPartition(const PartitionExt& partition_ext);
+  bool SelectPartition(int32_t& error_code, string &err_info,
+           PartitionExt& partition_ext, string& confirm_context);
+  void BookedPartionInfo(const string& partition_key, int64_t curr_offset);
+  void BookedPartionInfo(const string& partition_key, int64_t curr_offset,
+                            int32_t error_code, bool esc_limit, int32_t msg_size,
+                            int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow);
+  bool RelPartition(string &err_info, bool filter_consume,
+                         const string& confirm_context, bool is_consumed);
+  bool RelPartition(string &err_info, const string& confirm_context, bool is_consumed);
+  bool RelPartition(string &err_info, bool filter_consume,
+                         const string& confirm_context, bool is_consumed,
+                         int64_t curr_offset, int32_t error_code, bool esc_limit,
+                         int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt);
+  void FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
+          list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions);
+  void GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst);
+  bool GetPartitionExt(const string& part_key, PartitionExt& partition_ext);
+  void GetRegBrokers(list<NodeInfo>& brokers);
+  void GetPartitionByBroker(const NodeInfo& broker_info,
+                                    list<PartitionExt>& partition_list);
+  void GetCurPartitionOffsets(map<string, int64_t> part_offset_map);
+  void GetAllClosedBrokerParts(map<NodeInfo, list<PartitionExt> >& broker_parts);
+  void RemovePartition(const list<PartitionExt>& partition_list);
+  void RemovePartition(const set<string>& partition_keys);
+  bool RemovePartition(string &err_info, const string& confirm_context);
+  void RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
+        bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts);
+  bool IsPartitionFirstReg(const string& partition_key);
+  void OfferEvent(const ConsumerEvent& event);
+  void TakeEvent(ConsumerEvent& event);
+  void ClearEvent();
+  void OfferEventResult(const ConsumerEvent& event);
+  bool PollEventResult(ConsumerEvent& event);
+  void HandleTimeout(const string partition_key, const asio::error_code& error);
+  int IncrAndGetHBError(NodeInfo broker);
+  void ResetHBError(NodeInfo broker);
+
+ private:
+  void addDelayTimer(const string& part_key, int64_t delay_time);
+  void resetIdlePartition(const string& partition_key, bool need_reuse);
+  void rmvMetaInfo(const string& partition_key);
+  void buildConfirmContext(const string& partition_key,
+                    int64_t booked_time, string& confirm_context);
+  bool parseConfirmContext(string &err_info,
+    const string& confirm_context, string& partition_key, int64_t& booked_time);
+  bool inRelPartition(string &err_info, bool need_delay_check,
+    bool filter_consume, const string& confirm_context, bool is_consumed);
+
+ private:
+  //
+  string consumer_id_;
+  string group_name_;
+  // flow ctrl
+  AtomicInteger cur_part_cnt_;
+  FlowCtrlRuleHandler group_flowctrl_handler_;
+  FlowCtrlRuleHandler def_flowctrl_handler_;
+  AtomicBoolean under_groupctrl_;
+  AtomicLong last_checktime_;
+  // meta info
+  mutable mutex meta_lock_;
+  // partiton allocated map
+  map<string, PartitionExt> partitions_;
+  // topic partiton map
+  map<string, set<string> > topic_partition_;
+  // broker parition map
+  map<NodeInfo, set<string> > broker_partition_;
+  map<string, SubscribeInfo>  part_subinfo_;
+  // for idle partitions occupy
+  list<string> index_partitions_;
+  // for partition used map
+  map<string, int64_t> partition_useds_;
+  // for partiton timer map
+  map<string, tuple<int64_t, SteadyTimerPtr> > partition_timeouts_;
+  // data book
+  mutable mutex data_book_mutex_;
+  // for partition offset cache
+  map<string, int64_t> partition_offset_;
+  // for partiton register booked
+  map<string, bool> part_reg_booked_;
+  // event
+  mutable mutex event_read_mutex_;
+  condition_variable event_read_cond_;
+  list<ConsumerEvent> rebalance_events_;
+  mutable mutex event_write_mutex_;
+  list<ConsumerEvent> rebalance_results_;
+  // status check
+  mutable mutex status_mutex_;
+  map<NodeInfo, int> broker_status_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/singleton.h
similarity index 50%
copy from tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
copy to tubemq-client-twins/tubemq-client-cpp/src/singleton.h
index 1adcdf8..c945005 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/singleton.h
@@ -17,52 +17,46 @@
  * under the License.
  */
 
-#ifndef _TUBEMQ_THREAD_POOL_
-#define _TUBEMQ_THREAD_POOL_
+#ifndef _TUBEMQ_SINGLETON_H
+#define _TUBEMQ_SINGLETON_H
 
+#include <assert.h>
 #include <stdlib.h>
 
-#include <asio.hpp>
-#include <asio/ssl.hpp>
-#include <chrono>
-#include <functional>
-#include <memory>
 #include <mutex>
 #include <thread>
-#include <vector>
 
 #include "noncopyable.h"
 
 namespace tubemq {
-// ThreadPool use one io_context for thread pool
-class ThreadPool : noncopyable {
+
+template <typename T>
+class Singleton : noncopyable {
  public:
-  explicit ThreadPool(std::size_t size)
-      : io_context_(size), work_(asio::make_work_guard(io_context_)) {
-    for (size_t i = 0; i < size; ++i) {
-      workers_.emplace_back([this] { io_context_.run(); });
-    }
+  static T& Instance() {
+    std::call_once(once_, Singleton::init);
+    assert(value_ != nullptr);
+    return *value_;
   }
 
-  ~ThreadPool() {
-    work_.reset();
-    io_context_.stop();
-    for (std::thread &worker : workers_) {
-      worker.join();
-    }
-    workers_.clear();
-  }
+ protected:
+  Singleton() {}
+  ~Singleton() {}
 
-  template <class function>
-  void Post(function f) {
-    io_context_.post(f);
-  }
+ private:
+  static void init() { value_ = new T(); }
 
  private:
-  asio::io_context io_context_;
-  using io_context_work = asio::executor_work_guard<asio::io_context::executor_type>;
-  io_context_work work_;
-  std::vector<std::thread> workers_;
-};  // namespace tubemq
+  static std::once_flag once_;
+  static T* value_;
+};
+
+template <typename T>
+std::once_flag Singleton<T>::once_;
+
+template <typename T>
+T* Singleton<T>::value_ = nullptr;
+
 }  // namespace tubemq
-#endif  // _TUBEMQ_THREAD_POOL_
+
+#endif  // _TUBEMQ_SINGLETON_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
index 1adcdf8..803eb15 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
@@ -22,8 +22,7 @@
 
 #include <stdlib.h>
 
-#include <asio.hpp>
-#include <asio/ssl.hpp>
+
 #include <chrono>
 #include <functional>
 #include <memory>
@@ -31,6 +30,8 @@
 #include <thread>
 #include <vector>
 
+#include <asio.hpp>
+#include <asio/ssl.hpp>
 #include "noncopyable.h"
 
 namespace tubemq {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_client.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_client.cc
new file mode 100644
index 0000000..9ba1aef
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_client.cc
@@ -0,0 +1,184 @@
+/**
+ * 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 "tubemq/tubemq_client.h"
+
+#include <signal.h>
+#include <unistd.h>
+
+#include <sstream>
+
+#include "baseconsumer.h"
+#include "client_service.h"
+#include "const_config.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_errcode.h"
+
+
+
+namespace tubemq {
+
+using std::lock_guard;
+using std::stringstream;
+
+bool StartTubeMQService(string& err_info, const string& conf_file) {
+  signal(SIGPIPE, SIG_IGN);
+  return TubeMQService::Instance()->Start(err_info, conf_file);
+}
+
+bool StopTubeMQService(string& err_info) {
+  int32_t count = TubeMQService::Instance()->GetClientObjCnt();
+  if (count > 0) {
+    stringstream ss;
+    ss << "Check found ";
+    ss << count;
+    ss << " clients not shutdown, please shutdown clients first!";
+    err_info = ss.str();
+    return false;
+  }
+  return TubeMQService::Instance()->Stop(err_info);
+}
+
+
+TubeMQConsumer::TubeMQConsumer() {
+  client_id_ = tb_config::kInvalidValue;
+  status_.Set(0);
+}
+
+TubeMQConsumer::~TubeMQConsumer() {
+  ShutDown();
+}
+
+bool TubeMQConsumer::Start(string& err_info,
+  const ConsumerConfig& config) {
+  if (!TubeMQService::Instance()->IsRunning()) {
+    err_info = "TubeMQ Service not startted!";
+    return false;
+  }
+  // check status
+  if (!status_.CompareAndSet(0, 1)) {
+    err_info = "Duplicated call!";
+    return false;
+  }
+  BaseConsumer* rmt_client = new BaseConsumer();
+  if (rmt_client == NULL) {
+    err_info = "No memory for create CONSUMER remote object!";
+    return false;
+  }
+  if (!rmt_client->Start(err_info, config)) {
+    rmt_client->ShutDown();
+    delete rmt_client;
+    return false;
+  }
+  client_id_ = rmt_client->GetClientIndex();
+  status_.Set(2);
+  err_info = "Ok!";
+  return true;
+}
+
+void TubeMQConsumer::ShutDown() {
+  if (!status_.CompareAndSet(2, 0)) {
+    return;
+  }
+  if (client_id_ != tb_config::kInvalidValue) {
+    BaseConsumer* rmt_client =
+      (BaseConsumer *)TubeMQService::Instance()->GetClientObj(client_id_);
+    if ((rmt_client != NULL)
+      && (rmt_client->GetClientIndex() == client_id_)) {
+      rmt_client->ShutDown();
+      delete rmt_client;
+    }
+    client_id_ = tb_config::kInvalidValue;
+  }
+}
+
+bool TubeMQConsumer::GetMessage(ConsumerResult& result) {
+  if (!TubeMQService::Instance()->IsRunning()) {
+    result.SetFailureResult(err_code::kErrMQServiceStop,
+      "TubeMQ Service stopped!");
+    return false;
+  }
+  if (status_.Get() != 2) {
+    result.SetFailureResult(err_code::kErrClientStop,
+      "TubeMQ Service not startted!");
+    return false;
+  }
+  if (client_id_ == tb_config::kInvalidValue) {
+    result.SetFailureResult(err_code::kErrClientStop,
+      "Tube client not call init function, please initial first!");
+    return false;
+  }
+  BaseConsumer* rmt_client =
+    (BaseConsumer*)TubeMQService::Instance()->GetClientObj(client_id_);
+  if ((rmt_client == NULL)
+    || (rmt_client->GetClientIndex() != client_id_)) {
+    result.SetFailureResult(err_code::kErrBadRequest,
+      "Rmt client CB has been released, please re-start this client");
+    return false;
+  }
+  return rmt_client->GetMessage(result);
+}
+
+bool TubeMQConsumer::Confirm(const string& confirm_context,
+  bool is_consumed, ConsumerResult& result) {
+  if (!TubeMQService::Instance()->IsRunning()) {
+    result.SetFailureResult(err_code::kErrMQServiceStop,
+      "TubeMQ Service stopped!");
+    return false;
+  }
+  if (status_.Get() != 2) {
+    result.SetFailureResult(err_code::kErrClientStop,
+      "TubeMQ Service not startted!");
+    return false;
+  }
+  if (client_id_ == tb_config::kInvalidValue) {
+    result.SetFailureResult(err_code::kErrClientStop,
+      "Tube client not call init function, please initial first!");
+    return false;
+  }
+  BaseConsumer* rmt_client =
+    (BaseConsumer*)TubeMQService::Instance()->GetClientObj(client_id_);
+  if ((rmt_client == NULL)
+    || (rmt_client->GetClientIndex() != client_id_)) {
+    result.SetFailureResult(err_code::kErrBadRequest,
+      "Rmt client CB has been released, please re-start this client");
+    return false;
+  }
+  return rmt_client->Confirm(confirm_context, is_consumed, result);
+}
+
+bool TubeMQConsumer::GetCurConsumedInfo(map<string, ConsumeOffsetInfo>& consume_info_map) {
+  if (!TubeMQService::Instance()->IsRunning()) {
+    return false;
+  }
+  if ((status_.Get() != 2)
+    || (client_id_ == tb_config::kInvalidValue)) {
+    return false;
+  }
+  BaseConsumer* rmt_client =
+    (BaseConsumer*)TubeMQService::Instance()->GetClientObj(client_id_);
+  if ((rmt_client == NULL)
+    || (rmt_client->GetClientIndex() != client_id_)) {
+    return false;
+  }
+  return rmt_client->GetCurConsumedInfo(consume_info_map);
+}
+
+}  // namespace tubemq
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
index 89759df..c186701 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
@@ -185,9 +185,10 @@ class TubeMQCodec final : public CodecProtocol {
   // return code: -1 failed; 0-Unfinished; > 0 package buffer size
   virtual int32_t Check(BufferPtr &in, Any &out, uint32_t &request_id, bool &has_request_id,
                         size_t &package_length) {
+    LOG_TRACE("check in:%s", in->String().c_str());
     // check package is valid
     if (in->length() < 12) {
-      package_length = 12;
+      // package_length = 12;
       LOG_TRACE("Check: data's length < 12, is %ld, out", in->length());
       return 0;
     }
@@ -206,13 +207,11 @@ class TubeMQCodec final : public CodecProtocol {
     }
     // check data list
     uint32_t item_len = 0;
+    // package_length = 12;
     auto check_buf = in->Slice();
     for (uint32_t i = 0; i < list_size; i++) {
+      // package_length += 4;
       if (check_buf->length() < 4) {
-        package_length += 4;
-        if (i > 0) {
-          package_length += i * rpc_config::kRpcMaxBufferSize;
-        }
         LOG_TRACE("Check: buffer Remaining length < 4, is %ld, out", check_buf->length());
         return 0;
       }
@@ -226,11 +225,8 @@ class TubeMQCodec final : public CodecProtocol {
                   rpc_config::kRpcMaxBufferSize);
         return -1;
       }
+      // package_length += item_len;
       if (item_len > check_buf->length()) {
-        package_length += 4 + item_len;
-        if (i > 0) {
-          package_length += i * rpc_config::kRpcMaxBufferSize;
-        }
         LOG_TRACE("Check: item_len(%d) > remaining length(%ld), out", item_len,
                   check_buf->length());
         return 0;
@@ -248,8 +244,8 @@ class TubeMQCodec final : public CodecProtocol {
       in->Skip(item_len);
     }
     out = buf;
-    LOG_TRACE("Check: received message check finished, request_id=%d, readed_len:%d",
-      request_id, readed_len);
+    LOG_TRACE("Check: received message check finished, request_id=%d, readed_len:%d", request_id,
+              readed_len);
     return readed_len;
   }
 
@@ -310,8 +306,7 @@ class TubeMQCodec final : public CodecProtocol {
     }
     return block_cnt;
   }
-
- };
+};
 }  // namespace tubemq
 #endif
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
index b976add..a2534ff 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_config.cc
@@ -22,9 +22,9 @@
 #include <sstream>
 #include <vector>
 
-#include "tubemq/const_config.h"
-#include "tubemq/const_rpc.h"
-#include "tubemq/utils.h"
+#include "const_config.h"
+#include "const_rpc.h"
+#include "utils.h"
 
 namespace tubemq {
 
@@ -33,17 +33,17 @@ using std::stringstream;
 using std::vector;
 
 BaseConfig::BaseConfig() {
-  this->master_addrinfo_ = "";
-  this->auth_enable_ = false;
-  this->auth_usrname_ = "";
-  this->auth_usrpassword_ = "";
-  this->tls_enabled_ = false;
-  this->tls_trust_store_path_ = "";
-  this->tls_trust_store_password_ = "";
-  this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutDefSec;
-  this->heartbeat_period_sec_ = tb_config::kHeartBeatPeriodDef;
-  this->max_heartbeat_retry_times_ = tb_config::kHeartBeatFailRetryTimesDef;
-  this->heartbeat_period_afterfail_sec_ = tb_config::kHeartBeatSleepPeriodDef;
+  master_addrinfo_ = "";
+  auth_enable_ = false;
+  auth_usrname_ = "";
+  auth_usrpassword_ = "";
+  tls_enabled_ = false;
+  tls_trust_store_path_ = "";
+  tls_trust_store_password_ = "";
+  rpc_read_timeout_ms_ = tb_config::kRpcTimoutDefMs;
+  heartbeat_period_ms_ = tb_config::kHeartBeatPeriodDefMs;
+  max_heartbeat_retry_times_ = tb_config::kHeartBeatFailRetryTimesDef;
+  heartbeat_period_afterfail_ms_ = tb_config::kHeartBeatSleepPeriodDefMs;
 }
 
 BaseConfig::~BaseConfig() {
@@ -52,17 +52,17 @@ BaseConfig::~BaseConfig() {
 
 BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
   if (this != &target) {
-    this->master_addrinfo_ = target.master_addrinfo_;
-    this->auth_enable_ = target.auth_enable_;
-    this->auth_usrname_ = target.auth_usrname_;
-    this->auth_usrpassword_ = target.auth_usrpassword_;
-    this->tls_enabled_ = target.tls_enabled_;
-    this->tls_trust_store_path_ = target.tls_trust_store_path_;
-    this->tls_trust_store_password_ = target.tls_trust_store_password_;
-    this->rpc_read_timeout_sec_ = target.rpc_read_timeout_sec_;
-    this->heartbeat_period_sec_ = target.heartbeat_period_sec_;
-    this->max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
-    this->heartbeat_period_afterfail_sec_ = target.heartbeat_period_afterfail_sec_;
+    master_addrinfo_ = target.master_addrinfo_;
+    auth_enable_ = target.auth_enable_;
+    auth_usrname_ = target.auth_usrname_;
+    auth_usrpassword_ = target.auth_usrpassword_;
+    tls_enabled_ = target.tls_enabled_;
+    tls_trust_store_path_ = target.tls_trust_store_path_;
+    tls_trust_store_password_ = target.tls_trust_store_password_;
+    rpc_read_timeout_ms_ = target.rpc_read_timeout_ms_;
+    heartbeat_period_ms_ = target.heartbeat_period_ms_;
+    max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
+    heartbeat_period_afterfail_ms_ = target.heartbeat_period_afterfail_ms_;
   }
   return *this;
 }
@@ -91,14 +91,14 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
-  this->master_addrinfo_ = trimed_master_addr_info;
+  master_addrinfo_ = trimed_master_addr_info;
   err_info = "Ok";
   return true;
 }
 
 bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable, const string& trust_store_path,
                             const string& trust_store_password) {
-  this->tls_enabled_ = tls_enable;
+  tls_enabled_ = tls_enable;
   if (tls_enable) {
     string trimed_trust_store_path = Utils::Trim(trust_store_path);
     if (trimed_trust_store_path.empty()) {
@@ -110,11 +110,11 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable, const string& tru
       err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
     }
-    this->tls_trust_store_path_ = trimed_trust_store_path;
-    this->tls_trust_store_password_ = trimed_trust_store_password;
+    tls_trust_store_path_ = trimed_trust_store_path;
+    tls_trust_store_password_ = trimed_trust_store_password;
   } else {
-    this->tls_trust_store_path_ = "";
-    this->tls_trust_store_password_ = "";
+    tls_trust_store_path_ = "";
+    tls_trust_store_password_ = "";
   }
   err_info = "Ok";
   return true;
@@ -122,7 +122,7 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable, const string& tru
 
 bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, const string& usr_name,
                                   const string& usr_password) {
-  this->auth_enable_ = authentic_enable;
+  auth_enable_ = authentic_enable;
   if (authentic_enable) {
     string trimed_usr_name = Utils::Trim(usr_name);
     if (trimed_usr_name.empty()) {
@@ -134,102 +134,103 @@ bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, const
       err_info = "Illegal parameter: usr_password is empty!";
       return false;
     }
-    this->auth_usrname_ = trimed_usr_name;
-    this->auth_usrpassword_ = trimed_usr_password;
+    auth_usrname_ = trimed_usr_name;
+    auth_usrpassword_ = trimed_usr_password;
   } else {
-    this->auth_usrname_ = "";
-    this->auth_usrpassword_ = "";
+    auth_usrname_ = "";
+    auth_usrpassword_ = "";
   }
   err_info = "Ok";
   return true;
 }
 
-const string& BaseConfig::GetMasterAddrInfo() const { return this->master_addrinfo_; }
+const string& BaseConfig::GetMasterAddrInfo() const { return master_addrinfo_; }
 
-bool BaseConfig::IsTlsEnabled() { return this->tls_enabled_; }
+bool BaseConfig::IsTlsEnabled() { return tls_enabled_; }
 
-const string& BaseConfig::GetTrustStorePath() const { return this->tls_trust_store_path_; }
+const string& BaseConfig::GetTrustStorePath() const { return tls_trust_store_path_; }
 
-const string& BaseConfig::GetTrustStorePassword() const { return this->tls_trust_store_password_; }
+const string& BaseConfig::GetTrustStorePassword() const { return tls_trust_store_password_; }
 
-bool BaseConfig::IsAuthenticEnabled() { return this->auth_enable_; }
+bool BaseConfig::IsAuthenticEnabled() { return auth_enable_; }
 
-const string& BaseConfig::GetUsrName() const { return this->auth_usrname_; }
+const string& BaseConfig::GetUsrName() const { return auth_usrname_; }
 
-const string& BaseConfig::GetUsrPassWord() const { return this->auth_usrpassword_; }
+const string& BaseConfig::GetUsrPassWord() const { return auth_usrpassword_; }
 
-void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
-  if (rpc_read_timeout_sec >= rpc_config::kRpcTimoutMaxSec) {
-    this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutMaxSec;
-  } else if (rpc_read_timeout_sec <= rpc_config::kRpcTimoutMinSec) {
-    this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutMinSec;
+void BaseConfig::SetRpcReadTimeoutMs(int rpc_read_timeout_ms) {
+  if (rpc_read_timeout_ms >= tb_config::kRpcTimoutMaxMs) {
+    rpc_read_timeout_ms_ = tb_config::kRpcTimoutMaxMs;
+  } else if (rpc_read_timeout_ms <= tb_config::kRpcTimoutMinMs) {
+    rpc_read_timeout_ms_ = tb_config::kRpcTimoutMinMs;
   } else {
-    this->rpc_read_timeout_sec_ = rpc_read_timeout_sec;
+    rpc_read_timeout_ms_ = rpc_read_timeout_ms;
   }
 }
 
-int32_t BaseConfig::GetRpcReadTimeoutSec() { return this->rpc_read_timeout_sec_; }
+int32_t BaseConfig::GetRpcReadTimeoutMs() { return rpc_read_timeout_ms_; }
 
-void BaseConfig::SetHeartbeatPeriodSec(int32_t heartbeat_period_sec) {
-  this->heartbeat_period_sec_ = heartbeat_period_sec;
+void BaseConfig::SetHeartbeatPeriodMs(int32_t heartbeat_period_ms) {
+  heartbeat_period_ms_ = heartbeat_period_ms;
 }
 
-int32_t BaseConfig::GetHeartbeatPeriodSec() { return this->heartbeat_period_sec_; }
+int32_t BaseConfig::GetHeartbeatPeriodMs() { return heartbeat_period_ms_; }
 
 void BaseConfig::SetMaxHeartBeatRetryTimes(int32_t max_heartbeat_retry_times) {
-  this->max_heartbeat_retry_times_ = max_heartbeat_retry_times;
+  max_heartbeat_retry_times_ = max_heartbeat_retry_times;
 }
 
-int32_t BaseConfig::GetMaxHeartBeatRetryTimes() { return this->max_heartbeat_retry_times_; }
+int32_t BaseConfig::GetMaxHeartBeatRetryTimes() { return max_heartbeat_retry_times_; }
 
-void BaseConfig::SetHeartbeatPeriodAftFailSec(int32_t heartbeat_period_afterfail_sec) {
-  this->heartbeat_period_afterfail_sec_ = heartbeat_period_afterfail_sec;
+void BaseConfig::SetHeartbeatPeriodAftFailMs(int32_t heartbeat_period_afterfail_ms) {
+  heartbeat_period_afterfail_ms_ = heartbeat_period_afterfail_ms;
 }
 
-int32_t BaseConfig::GetHeartbeatPeriodAftFailSec() { return this->heartbeat_period_afterfail_sec_; }
+int32_t BaseConfig::GetHeartbeatPeriodAftFailMs() { return heartbeat_period_afterfail_ms_; }
 
 string BaseConfig::ToString() {
   stringstream ss;
   ss << "BaseConfig={master_addrinfo_='";
-  ss << this->master_addrinfo_;
+  ss << master_addrinfo_;
   ss << "', authEnable=";
-  ss << this->auth_enable_;
+  ss << auth_enable_;
   ss << ", auth_usrname_='";
-  ss << this->auth_usrname_;
+  ss << auth_usrname_;
   ss << "', auth_usrpassword_='";
-  ss << this->auth_usrpassword_;
+  ss << auth_usrpassword_;
   ss << "', tls_enabled_=";
-  ss << this->tls_enabled_;
+  ss << tls_enabled_;
   ss << ", tls_trust_store_path_='";
-  ss << this->tls_trust_store_path_;
+  ss << tls_trust_store_path_;
   ss << "', tls_trust_store_password_='";
-  ss << this->tls_trust_store_password_;
-  ss << "', rpc_read_timeout_sec_=";
-  ss << this->rpc_read_timeout_sec_;
-  ss << ", heartbeat_period_sec_=";
-  ss << this->heartbeat_period_sec_;
+  ss << tls_trust_store_password_;
+  ss << "', rpc_read_timeout_ms_=";
+  ss << rpc_read_timeout_ms_;
+  ss << ", heartbeat_period_ms_=";
+  ss << heartbeat_period_ms_;
   ss << ", max_heartbeat_retry_times_=";
-  ss << this->max_heartbeat_retry_times_;
-  ss << ", heartbeat_period_afterfail_sec_=";
-  ss << this->heartbeat_period_afterfail_sec_;
+  ss << max_heartbeat_retry_times_;
+  ss << ", heartbeat_period_afterfail_ms_=";
+  ss << heartbeat_period_afterfail_ms_;
   ss << "}";
   return ss.str();
 }
 
 ConsumerConfig::ConsumerConfig() {
-  this->group_name_ = "";
-  this->is_bound_consume_ = false;
-  this->session_key_ = "";
-  this->source_count_ = 0;
-  this->is_select_big_ = true;
-  this->consume_position_ = kConsumeFromLatestOffset;
-  this->is_confirm_in_local_ = false;
-  this->is_rollback_if_confirm_timout_ = true;
-  this->max_subinfo_report_intvl_ = tb_config::kSubInfoReportMaxIntervalTimes;
-  this->msg_notfound_wait_period_ms_ = tb_config::kMsgNotfoundWaitPeriodMsDef;
-  this->reb_confirm_wait_period_ms_ = tb_config::kRebConfirmWaitPeriodMsDef;
-  this->max_confirm_wait_period_ms_ = tb_config::kConfirmWaitPeriodMsMax;
-  this->shutdown_reb_wait_period_ms_ = tb_config::kRebWaitPeriodWhenShutdownMs;
+  group_name_ = "";
+  is_bound_consume_ = false;
+  session_key_ = "";
+  source_count_ = 0;
+  is_select_big_ = true;
+  consume_position_ = kConsumeFromLatestOffset;
+  is_rollback_if_confirm_timout_ = true;
+  max_subinfo_report_intvl_ = tb_config::kSubInfoReportMaxIntervalTimes;
+  max_part_check_period_ms_ = tb_config::kMaxPartCheckPeriodMsDef;
+  part_check_slice_ms_ = tb_config::kPartCheckSliceMsDef;
+  msg_notfound_wait_period_ms_ = tb_config::kMsgNotfoundWaitPeriodMsDef;
+  reb_confirm_wait_period_ms_ = tb_config::kRebConfirmWaitPeriodMsDef;
+  max_confirm_wait_period_ms_ = tb_config::kConfirmWaitPeriodMsMax;
+  shutdown_reb_wait_period_ms_ = tb_config::kRebWaitPeriodWhenShutdownMs;
 }
 
 ConsumerConfig::~ConsumerConfig() {
@@ -241,21 +242,20 @@ ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
     // parent class
     BaseConfig::operator=(target);
     // child class
-    this->group_name_ = target.group_name_;
-    this->sub_topic_and_filter_map_ = target.sub_topic_and_filter_map_;
-    this->is_bound_consume_ = target.is_bound_consume_;
-    this->session_key_ = target.session_key_;
-    this->source_count_ = target.source_count_;
-    this->is_select_big_ = target.is_select_big_;
-    this->part_offset_map_ = target.part_offset_map_;
-    this->consume_position_ = target.consume_position_;
-    this->max_subinfo_report_intvl_ = target.max_subinfo_report_intvl_;
-    this->msg_notfound_wait_period_ms_ = target.msg_notfound_wait_period_ms_;
-    this->is_confirm_in_local_ = target.is_confirm_in_local_;
-    this->is_rollback_if_confirm_timout_ = target.is_rollback_if_confirm_timout_;
-    this->reb_confirm_wait_period_ms_ = target.reb_confirm_wait_period_ms_;
-    this->max_confirm_wait_period_ms_ = target.max_confirm_wait_period_ms_;
-    this->shutdown_reb_wait_period_ms_ = target.shutdown_reb_wait_period_ms_;
+    group_name_ = target.group_name_;
+    sub_topic_and_filter_map_ = target.sub_topic_and_filter_map_;
+    is_bound_consume_ = target.is_bound_consume_;
+    session_key_ = target.session_key_;
+    source_count_ = target.source_count_;
+    is_select_big_ = target.is_select_big_;
+    part_offset_map_ = target.part_offset_map_;
+    consume_position_ = target.consume_position_;
+    max_subinfo_report_intvl_ = target.max_subinfo_report_intvl_;
+    msg_notfound_wait_period_ms_ = target.msg_notfound_wait_period_ms_;
+    is_rollback_if_confirm_timout_ = target.is_rollback_if_confirm_timout_;
+    reb_confirm_wait_period_ms_ = target.reb_confirm_wait_period_ms_;
+    max_confirm_wait_period_ms_ = target.max_confirm_wait_period_ms_;
+    shutdown_reb_wait_period_ms_ = target.shutdown_reb_wait_period_ms_;
   }
   return *this;
 }
@@ -285,9 +285,9 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, const string& group
     set<string> tmp_filters;
     tmp_sub_map[topic_name] = tmp_filters;
   }
-  this->is_bound_consume_ = false;
-  this->group_name_ = tgt_group_name;
-  this->sub_topic_and_filter_map_ = tmp_sub_map;
+  is_bound_consume_ = false;
+  group_name_ = tgt_group_name;
+  sub_topic_and_filter_map_ = tmp_sub_map;
   err_info = "Ok";
   return true;
 }
@@ -376,15 +376,16 @@ bool ConsumerConfig::setGroupConsumeTarget(
   }
   // check if bound consume
   if (!is_bound_consume) {
-    this->is_bound_consume_ = false;
-    this->group_name_ = tgt_group_name;
-    this->sub_topic_and_filter_map_ = tmp_sub_map;
+    is_bound_consume_ = false;
+    group_name_ = tgt_group_name;
+    sub_topic_and_filter_map_ = tmp_sub_map;
     err_info = "Ok";
     return true;
   }
   // check session_key
   string tgt_session_key = Utils::Trim(session_key);
-  if (tgt_session_key.length() == 0 || tgt_session_key.length() > tb_config::kSessionKeyMaxLength) {
+  if (tgt_session_key.length() == 0
+    || tgt_session_key.length() > tb_config::kSessionKeyMaxLength) {
     if (tgt_session_key.length() == 0) {
       err_info = "Illegal parameter: session_key is empty!";
     } else {
@@ -410,7 +411,7 @@ bool ConsumerConfig::setGroupConsumeTarget(
       err_info = ss.str();
       return false;
     }
-    if (tmp_sub_map.find(result[1]) != tmp_sub_map.end()) {
+    if (tmp_sub_map.find(result[1]) == tmp_sub_map.end()) {
       stringstream ss;
       ss << "Illegal parameter: ";
       ss << it_part->first;
@@ -443,77 +444,96 @@ bool ConsumerConfig::setGroupConsumeTarget(
     tmp_parts_map[part_key] = it_part->second;
   }
   // set verified data
-  this->is_bound_consume_ = true;
-  this->group_name_ = tgt_group_name;
-  this->sub_topic_and_filter_map_ = tmp_sub_map;
-  this->session_key_ = tgt_session_key;
-  this->source_count_ = source_count;
-  this->is_select_big_ = is_select_big;
-  this->part_offset_map_ = tmp_parts_map;
+  is_bound_consume_ = true;
+  group_name_ = tgt_group_name;
+  sub_topic_and_filter_map_ = tmp_sub_map;
+  session_key_ = tgt_session_key;
+  source_count_ = source_count;
+  is_select_big_ = is_select_big;
+  part_offset_map_ = tmp_parts_map;
   err_info = "Ok";
   return true;
 }
 
-const string& ConsumerConfig::GetGroupName() const { return this->group_name_; }
+const string& ConsumerConfig::GetGroupName() const { return group_name_; }
 
 const map<string, set<string> >& ConsumerConfig::GetSubTopicAndFilterMap() const {
-  return this->sub_topic_and_filter_map_;
+  return sub_topic_and_filter_map_;
 }
 
 void ConsumerConfig::SetConsumePosition(ConsumePosition consume_from_where) {
-  this->consume_position_ = consume_from_where;
+  consume_position_ = consume_from_where;
 }
 
-const ConsumePosition ConsumerConfig::GetConsumePosition() const { return this->consume_position_; }
+const ConsumePosition ConsumerConfig::GetConsumePosition() const { return consume_position_; }
 
-const int ConsumerConfig::GetMsgNotFoundWaitPeriodMs() const {
-  return this->msg_notfound_wait_period_ms_;
+const int32_t ConsumerConfig::GetMsgNotFoundWaitPeriodMs() const {
+  return msg_notfound_wait_period_ms_;
 }
 
-void ConsumerConfig::SetMsgNotFoundWaitPeriodMs(int msg_notfound_wait_period_ms) {
-  this->msg_notfound_wait_period_ms_ = msg_notfound_wait_period_ms;
+void ConsumerConfig::SetMsgNotFoundWaitPeriodMs(int32_t msg_notfound_wait_period_ms) {
+  msg_notfound_wait_period_ms_ = msg_notfound_wait_period_ms;
 }
 
-const int ConsumerConfig::GetMaxSubinfoReportIntvl() const {
-  return this->max_subinfo_report_intvl_;
+const uint32_t ConsumerConfig::GetPartCheckSliceMs() const {
+  return part_check_slice_ms_;
 }
 
-void ConsumerConfig::SetMaxSubinfoReportIntvl(int max_subinfo_report_intvl) {
-  this->max_subinfo_report_intvl_ = max_subinfo_report_intvl;
+void ConsumerConfig::SetPartCheckSliceMs(uint32_t part_check_slice_ms) {
+  if (part_check_slice_ms >= 0
+    && part_check_slice_ms <= 1000) {
+    part_check_slice_ms_ = part_check_slice_ms;
+  }
+}
+
+const int32_t ConsumerConfig::GetMaxPartCheckPeriodMs() const {
+  return max_part_check_period_ms_;
 }
 
-bool ConsumerConfig::IsConfirmInLocal() { return this->is_confirm_in_local_; }
+void ConsumerConfig::SetMaxPartCheckPeriodMs(int32_t max_part_check_period_ms) {
+  max_part_check_period_ms_ = max_part_check_period_ms;
+}
+
+const int32_t ConsumerConfig::GetMaxSubinfoReportIntvl() const {
+  return max_subinfo_report_intvl_;
+}
 
-void ConsumerConfig::SetConfirmInLocal(bool confirm_in_local) {
-  this->is_confirm_in_local_ = confirm_in_local;
+void ConsumerConfig::SetMaxSubinfoReportIntvl(int32_t max_subinfo_report_intvl) {
+  max_subinfo_report_intvl_ = max_subinfo_report_intvl;
 }
 
-bool ConsumerConfig::IsRollbackIfConfirmTimeout() { return this->is_rollback_if_confirm_timout_; }
+bool ConsumerConfig::IsRollbackIfConfirmTimeout() {
+  return is_rollback_if_confirm_timout_; }
 
-void ConsumerConfig::setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout) {
-  this->is_rollback_if_confirm_timout_ = is_rollback_if_confirm_timeout;
+void ConsumerConfig::setRollbackIfConfirmTimeout(
+  bool is_rollback_if_confirm_timeout) {
+  is_rollback_if_confirm_timout_ = is_rollback_if_confirm_timeout;
 }
 
 const int ConsumerConfig::GetWaitPeriodIfConfirmWaitRebalanceMs() const {
-  return this->reb_confirm_wait_period_ms_;
+  return reb_confirm_wait_period_ms_;
 }
 
-void ConsumerConfig::SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_period_ms) {
-  this->reb_confirm_wait_period_ms_ = reb_confirm_wait_period_ms;
+void ConsumerConfig::SetWaitPeriodIfConfirmWaitRebalanceMs(
+  int32_t reb_confirm_wait_period_ms) {
+  reb_confirm_wait_period_ms_ = reb_confirm_wait_period_ms;
 }
 
-const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const { return max_confirm_wait_period_ms_; }
+const int32_t ConsumerConfig::GetMaxConfirmWaitPeriodMs() const {
+  return max_confirm_wait_period_ms_; }
 
-void ConsumerConfig::SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms) {
-  this->max_confirm_wait_period_ms_ = max_confirm_wait_period_ms;
+void ConsumerConfig::SetMaxConfirmWaitPeriodMs(
+  int32_t max_confirm_wait_period_ms) {
+  max_confirm_wait_period_ms_ = max_confirm_wait_period_ms;
 }
 
-const int ConsumerConfig::GetShutdownRebWaitPeriodMs() const {
-  return this->shutdown_reb_wait_period_ms_;
+const int32_t ConsumerConfig::GetShutdownRebWaitPeriodMs() const {
+  return shutdown_reb_wait_period_ms_;
 }
 
-void ConsumerConfig::SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms) {
-  this->shutdown_reb_wait_period_ms_ = wait_period_when_shutdown_ms;
+void ConsumerConfig::SetShutdownRebWaitPeriodMs(
+  int32_t wait_period_when_shutdown_ms) {
+  shutdown_reb_wait_period_ms_ = wait_period_when_shutdown_ms;
 }
 
 string ConsumerConfig::ToString() {
@@ -526,10 +546,10 @@ string ConsumerConfig::ToString() {
   ss << "ConsumerConfig = {";
   ss << BaseConfig::ToString();
   ss << ", group_name_='";
-  ss << this->group_name_;
+  ss << group_name_;
   ss << "', sub_topic_and_filter_map_={";
-  for (it_map = this->sub_topic_and_filter_map_.begin();
-       it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
+  for (it_map = sub_topic_and_filter_map_.begin();
+       it_map != sub_topic_and_filter_map_.end(); ++it_map) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -549,16 +569,16 @@ string ConsumerConfig::ToString() {
     ss << "]";
   }
   ss << "}, is_bound_consume_=";
-  ss << this->is_bound_consume_;
+  ss << is_bound_consume_;
   ss << ", session_key_='";
-  ss << this->session_key_;
+  ss << session_key_;
   ss << "', source_count_=";
-  ss << this->source_count_;
+  ss << source_count_;
   ss << ", is_select_big_=";
-  ss << this->is_select_big_;
+  ss << is_select_big_;
   ss << ", part_offset_map_={";
   i = 0;
-  for (it = this->part_offset_map_.begin(); it != this->part_offset_map_.end(); ++it) {
+  for (it = part_offset_map_.begin(); it != part_offset_map_.end(); ++it) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -568,21 +588,23 @@ string ConsumerConfig::ToString() {
     ss << it->second;
   }
   ss << "}, consume_position_=";
-  ss << this->consume_position_;
+  ss << consume_position_;
   ss << ", max_subinfo_report_intvl_=";
-  ss << this->max_subinfo_report_intvl_;
+  ss << max_subinfo_report_intvl_;
   ss << ", msg_notfound_wait_period_ms_=";
-  ss << this->msg_notfound_wait_period_ms_;
-  ss << ", is_confirm_in_local_=";
-  ss << this->is_confirm_in_local_;
+  ss << msg_notfound_wait_period_ms_;
+  ss << ", max_part_check_period_ms_=";
+  ss << max_part_check_period_ms_;
+  ss << ", part_check_slice_ms_=";
+  ss << part_check_slice_ms_;
   ss << ", is_rollback_if_confirm_timout_=";
-  ss << this->is_rollback_if_confirm_timout_;
+  ss << is_rollback_if_confirm_timout_;
   ss << ", reb_confirm_wait_period_ms_=";
-  ss << this->reb_confirm_wait_period_ms_;
+  ss << reb_confirm_wait_period_ms_;
   ss << ", max_confirm_wait_period_ms_=";
-  ss << this->max_confirm_wait_period_ms_;
+  ss << max_confirm_wait_period_ms_;
   ss << ", shutdown_reb_wait_period_ms_=";
-  ss << this->shutdown_reb_wait_period_ms_;
+  ss << shutdown_reb_wait_period_ms_;
   ss << "}";
   return ss.str();
 }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
index f914741..a12b199 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_message.cc
@@ -20,89 +20,88 @@
 #include "tubemq/tubemq_message.h"
 
 #include <string.h>
-
 #include <sstream>
 
-#include "tubemq/const_config.h"
-#include "tubemq/utils.h"
+#include "const_config.h"
+#include "utils.h"
 
 namespace tubemq {
 
 using std::stringstream;
 
-
-// message flag's properties settings
-static const int32_t kMsgFlagIncProperties = 0x01;
-// reserved property key Filter Item
-static const char kRsvPropKeyFilterItem[] = "$msgType$";
-// reserved property key message send time
-static const char kRsvPropKeyMsgTime[] = "$msgTime$";
-
 Message::Message() {
-  this->topic_ = "";
-  this->flag_ = 0;
-  this->message_id_ = tb_config::kInvalidValue;
-  this->data_ = NULL;
-  this->datalen_ = 0;
-  this->properties_.clear();
+  flag_ = 0;
+  message_id_ = tb_config::kInvalidValue;
+  data_ = NULL;
+  datalen_ = 0;
 }
 
 Message::Message(const Message& target) {
-  this->topic_ = target.topic_;
-  this->message_id_ = target.message_id_;
+  topic_ = target.topic_;
+  message_id_ = target.message_id_;
   copyData(target.data_, target.datalen_);
   copyProperties(target.properties_);
-  this->flag_ = target.flag_;
+  flag_ = target.flag_;
 }
 
 Message::Message(const string& topic, const char* data, uint32_t datalen) {
-  this->topic_ = topic;
-  this->flag_ = 0;
-  this->message_id_ = tb_config::kInvalidValue;
+  topic_ = topic;
+  flag_ = 0;
+  message_id_ = tb_config::kInvalidValue;
   copyData(data, datalen);
-  this->properties_.clear();
+  properties_.clear();
+}
+
+Message::Message(const string& topic, int32_t flag,
+  int64_t message_id, const char* data, uint32_t datalen,
+  const map<string, string>& properties) {
+  topic_ = topic;
+  flag_ = flag;
+  message_id_ = message_id;
+  copyData(data, datalen);
+  copyProperties(properties);
 }
 
 Message::~Message() { clearData(); }
 
 Message& Message::operator=(const Message& target) {
   if (this == &target) return *this;
-  this->topic_ = target.topic_;
-  this->message_id_ = target.message_id_;
+  topic_ = target.topic_;
+  message_id_ = target.message_id_;
   clearData();
   copyData(target.data_, target.datalen_);
   copyProperties(target.properties_);
-  this->flag_ = target.flag_;
+  flag_ = target.flag_;
   return *this;
 }
 
-const int64_t Message::GetMessageId() const { return this->message_id_; }
+const int64_t Message::GetMessageId() const { return message_id_; }
 
-void Message::SetMessageId(int64_t message_id) { this->message_id_ = message_id; }
+void Message::SetMessageId(int64_t message_id) { message_id_ = message_id; }
 
-const string& Message::GetTopic() const { return this->topic_; }
+const string& Message::GetTopic() const { return topic_; }
 
-void Message::SetTopic(const string& topic) { this->topic_ = topic; }
+void Message::SetTopic(const string& topic) { topic_ = topic; }
 
-const char* Message::GetData() const { return this->data_; }
+const char* Message::GetData() const { return data_; }
 
-uint32_t Message::GetDataLength() const { return this->datalen_; }
+uint32_t Message::GetDataLength() const { return datalen_; }
 
 void Message::setData(const char* data, uint32_t datalen) {
   clearData();
   copyData(data, datalen);
 }
 
-const int32_t Message::GetFlag() const { return this->flag_; }
+const int32_t Message::GetFlag() const { return flag_; }
 
-void Message::SetFlag(int32_t flag) { this->flag_ = flag; }
+void Message::SetFlag(int32_t flag) { flag_ = flag; }
 
-const map<string, string>& Message::GetProperties() const { return this->properties_; }
+const map<string, string>& Message::GetProperties() const { return properties_; }
 
 int32_t Message::GetProperties(string& attribute) {
   attribute.clear();
   map<string, string>::iterator it_map;
-  for (it_map = this->properties_.begin(); it_map != this->properties_.end(); ++it_map) {
+  for (it_map = properties_.begin(); it_map != properties_.end(); ++it_map) {
     if (!attribute.empty()) {
       attribute += delimiter::kDelimiterComma;
     }
@@ -117,8 +116,8 @@ bool Message::HasProperty(const string& key) {
   map<string, string>::iterator it_map;
   string trimed_key = Utils::Trim(key);
   if (!trimed_key.empty()) {
-    it_map = this->properties_.find(trimed_key);
-    if (it_map != this->properties_.end()) {
+    it_map = properties_.find(trimed_key);
+    if (it_map != properties_.end()) {
       return true;
     }
   }
@@ -129,8 +128,8 @@ bool Message::GetProperty(const string& key, string& value) {
   map<string, string>::iterator it_map;
   string trimed_key = Utils::Trim(key);
   if (!trimed_key.empty()) {
-    it_map = this->properties_.find(trimed_key);
-    if (it_map != this->properties_.end()) {
+    it_map = properties_.find(trimed_key);
+    if (it_map != properties_.end()) {
       value = it_map->second;
       return true;
     }
@@ -138,7 +137,9 @@ bool Message::GetProperty(const string& key, string& value) {
   return false;
 }
 
-bool Message::GetFilterItem(string& value) { return GetProperty(kRsvPropKeyFilterItem, value); }
+bool Message::GetFilterItem(string& value) {
+  return GetProperty(tb_config::kRsvPropKeyFilterItem, value);
+}
 
 bool Message::AddProperty(string& err_info, const string& key, const string& value) {
   string trimed_key = Utils::Trim(key);
@@ -169,53 +170,54 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if (trimed_key == kRsvPropKeyFilterItem || trimed_key == kRsvPropKeyMsgTime) {
+  if (trimed_key == tb_config::kRsvPropKeyFilterItem
+    || trimed_key == tb_config::kRsvPropKeyMsgTime) {
     stringstream ss;
     ss << "Reserved token '";
-    ss << kRsvPropKeyFilterItem;
+    ss << tb_config::kRsvPropKeyFilterItem;
     ss << "' or '";
-    ss << kRsvPropKeyMsgTime;
+    ss << tb_config::kRsvPropKeyMsgTime;
     ss << "' must not be used in parmeter key!";
     err_info = ss.str();
     return false;
   }
   // add key and value
-  this->properties_[trimed_key] = trimed_value;
-  if (!this->properties_.empty()) {
-    this->flag_ |= kMsgFlagIncProperties;
+  properties_[trimed_key] = trimed_value;
+  if (!properties_.empty()) {
+    flag_ |= tb_config::kMsgFlagIncProperties;
   }
   err_info = "Ok";
   return true;
 }
 
 void Message::clearData() {
-  if (this->data_ != NULL) {
-    delete[] this->data_;
-    this->data_ = NULL;
-    this->datalen_ = 0;
+  if (data_ != NULL) {
+    delete[] data_;
+    data_ = NULL;
+    datalen_ = 0;
   }
 }
 
 void Message::copyData(const char* data, uint32_t datalen) {
   if (data == NULL) {
-    this->data_ = NULL;
-    this->datalen_ = 0;
+    data_ = NULL;
+    datalen_ = 0;
   } else {
-    this->datalen_ = datalen;
-    this->data_ = new char[datalen];
-    memset(this->data_, 0, datalen);
-    memcpy(this->data_, data, datalen);
+    datalen_ = datalen;
+    data_ = new char[datalen];
+    memset(data_, 0, datalen);
+    memcpy(data_, data, datalen);
   }
 }
 
 void Message::copyProperties(const map<string, string>& properties) {
-  this->properties_.clear();
+  properties_.clear();
   map<string, string>::const_iterator it_map;
   for (it_map = properties.begin(); it_map != properties.end(); ++it_map) {
-    this->properties_[it_map->first] = it_map->second;
+    properties_[it_map->first] = it_map->second;
   }
-  if (!this->properties_.empty()) {
-    this->flag_ |= kMsgFlagIncProperties;
+  if (!properties_.empty()) {
+    flag_ |= tb_config::kMsgFlagIncProperties;
   }
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
index 5298606..ef79440 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_return.cc
@@ -18,7 +18,7 @@
  */
 
 #include "tubemq/tubemq_return.h"
-#include "tubemq/const_config.h"
+#include "const_config.h"
 
 
 
@@ -32,27 +32,52 @@ PeerInfo::PeerInfo() {
   curr_offset_ = tb_config::kInvalidValue;
 }
 
-PeerInfo::PeerInfo(const Partition& partition, int64_t offset) {
-  SetMsgSourceInfo(partition, offset);
+
+PeerInfo::PeerInfo(const string& broker_host, uint32_t partition_id,
+  const string& partiton_key, int64_t offset) {
+  broker_host_ = broker_host;
+  partition_id_ = partition_id;
+  partition_key_ = partiton_key;
+  curr_offset_ = offset;
 }
 
 PeerInfo& PeerInfo::operator=(const PeerInfo& target) {
   if (this != &target) {
-    this->partition_id_ = target.partition_id_;
-    this->broker_host_ = target.broker_host_;
-    this->partition_key_ = target.partition_key_;
-    this->curr_offset_ = target.curr_offset_;
+    partition_id_ = target.partition_id_;
+    broker_host_ = target.broker_host_;
+    partition_key_ = target.partition_key_;
+    curr_offset_ = target.curr_offset_;
   }
   return *this;
 }
 
-void PeerInfo::SetMsgSourceInfo(const Partition& partition, int64_t offset) {
-  partition_id_ = partition.GetPartitionId();
-  broker_host_ = partition.GetBrokerHost();
-  partition_key_ = partition.GetPartitionKey();
-  curr_offset_ = offset;
+ConsumeOffsetInfo::ConsumeOffsetInfo() {
+  partition_key_ = "";
+  curr_offset_ = tb_config::kInvalidValue;
+}
+
+ConsumeOffsetInfo::ConsumeOffsetInfo(
+  const string& part_key, int64_t curr_offset) {
+  partition_key_ = part_key;
+  curr_offset_ = curr_offset;
+}
+
+void ConsumeOffsetInfo::SetConsumeOffsetInfo(
+  const string& part_key, int64_t curr_offset) {
+  partition_key_ = part_key;
+  curr_offset_ = curr_offset;
 }
 
+ConsumeOffsetInfo& ConsumeOffsetInfo::operator=(
+  const ConsumeOffsetInfo& target) {
+  if (this != &target) {
+    partition_key_ = target.partition_key_;
+    curr_offset_ = target.curr_offset_;
+  }
+  return *this;
+}
+
+
 ConsumerResult::ConsumerResult() {
   success_ = false;
   err_code_ = tb_config::kInvalidValue;
@@ -62,25 +87,25 @@ ConsumerResult::ConsumerResult() {
 }
 
 ConsumerResult::ConsumerResult(const ConsumerResult& target) {
-  this->success_ = target.success_;
-  this->err_code_ = target.err_code_;
-  this->err_msg_ = target.err_msg_;
-  this->topic_name_ = target.topic_name_;
-  this->peer_info_ = target.peer_info_;
-  this->confirm_context_ = target.confirm_context_;
-  this->message_list_ = target.message_list_;
+  success_ = target.success_;
+  err_code_ = target.err_code_;
+  err_msg_ = target.err_msg_;
+  topic_name_ = target.topic_name_;
+  peer_info_ = target.peer_info_;
+  confirm_context_ = target.confirm_context_;
+  message_list_ = target.message_list_;
 }
 
-ConsumerResult::ConsumerResult(int32_t err_code, string err_msg) {
+ConsumerResult::ConsumerResult(int32_t error_code, string err_msg) {
   success_ = false;
-  err_code_ = err_code;
+  err_code_ = error_code;
   err_msg_ = err_msg;
   topic_name_ = "";
   confirm_context_ = "";
 }
 
 ConsumerResult::~ConsumerResult() {
-  this->message_list_.clear();
+  message_list_.clear();
   success_ = false;
   err_code_ = tb_config::kInvalidValue;
   err_msg_ = "";
@@ -90,52 +115,64 @@ ConsumerResult::~ConsumerResult() {
 
 ConsumerResult& ConsumerResult::operator=(const ConsumerResult& target) {
   if (this != &target) {
-    this->success_ = target.success_;
-    this->err_code_ = target.err_code_;
-    this->err_msg_ = target.err_msg_;
-    this->topic_name_ = target.topic_name_;
-    this->peer_info_ = target.peer_info_;
-    this->confirm_context_ = target.confirm_context_;
-    this->message_list_ = target.message_list_;
+    success_ = target.success_;
+    err_code_ = target.err_code_;
+    err_msg_ = target.err_msg_;
+    topic_name_ = target.topic_name_;
+    peer_info_ = target.peer_info_;
+    confirm_context_ = target.confirm_context_;
+    message_list_ = target.message_list_;
   }
   return *this;
 }
 
-void ConsumerResult::SetFailureResult(int32_t err_code, string err_msg) {
+void ConsumerResult::SetFailureResult(int32_t error_code, string err_msg) {
   success_ = false;
-  err_code_ = err_code;
+  err_code_ = error_code;
   err_msg_ = err_msg;
 }
 
-void ConsumerResult::SetFailureResult(int32_t err_code, string err_msg,
+void ConsumerResult::SetFailureResult(int32_t error_code, string err_msg,
                             const string& topic_name, const PeerInfo& peer_info) {
   success_ = false;
-  err_code_ = err_code;
+  err_code_ = error_code;
   err_msg_ = err_msg;
   topic_name_ = topic_name;
   peer_info_ = peer_info;
 }
 
-void ConsumerResult::SetSuccessResult(int32_t err_code,
+void ConsumerResult::SetSuccessResult(int32_t error_code,
+                                             const string& topic_name,
+                                             const PeerInfo& peer_info) {
+  success_ = true;
+  err_code_ = error_code;
+  err_msg_ = "Ok";
+  topic_name_ = topic_name;
+  peer_info_ = peer_info;
+  confirm_context_ = "";
+  message_list_.clear();
+}
+
+void ConsumerResult::SetSuccessResult(int32_t error_code,
                                              const string& topic_name,
                                              const PeerInfo& peer_info,
                                              const string& confirm_context,
                                              const list<Message>& message_list) {
-  this->success_ = true;
-  this->err_code_ = err_code;
-  this->err_msg_ = "Ok";
-  this->topic_name_ = topic_name;
-  this->peer_info_ = peer_info;
-  this->confirm_context_ = confirm_context;
-  this->message_list_ = message_list;
+  success_ = true;
+  err_code_ = error_code;
+  err_msg_ = "Ok";
+  topic_name_ = topic_name;
+  peer_info_ = peer_info;
+  confirm_context_ = confirm_context;
+  message_list_ = message_list;
 }
 
 const string& ConsumerResult::GetPartitionKey() const {
-  return this->peer_info_.GetPartitionKey();
+  return peer_info_.GetPartitionKey();
 }
 
 const int64_t ConsumerResult::GetCurrOffset() const {
-  return this->peer_info_.GetCurrOffset();
+  return peer_info_.GetCurrOffset();
 }
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/unique_seq_id.h b/tubemq-client-twins/tubemq-client-cpp/src/unique_seq_id.h
new file mode 100644
index 0000000..b24b627
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/unique_seq_id.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 TUBEMQ_UNIQUESEQID_H
+#define TUBEMQ_UNIQUESEQID_H
+
+#include <stdint.h>
+
+#include <atomic>
+
+namespace tubemq {
+
+class UniqueSeqId {
+ public:
+  UniqueSeqId() : id_(0) {}
+
+  uint32_t Next() { return id_.fetch_add(1, std::memory_order_relaxed); }
+
+ protected:
+  std::atomic<uint32_t> id_;
+};
+
+}  // namespace tubemq
+
+#endif
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 75fcba5..81ef56e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -17,24 +17,27 @@
  * under the License.
  */
 
-#include "tubemq/utils.h"
+#include "utils.h"
 
 #include <arpa/inet.h>
+#include <ctype.h>
 #include <linux/if.h>
+#include <netdb.h>
 #include <netinet/in.h>
 #include <regex.h>
-#include <stdlib.h>
 #include <stdio.h>
+#include <stdlib.h>
 #include <string.h>
 #include <sys/ioctl.h>
 #include <sys/socket.h>
 #include <sys/time.h>
 #include <unistd.h>
-
 #include <sstream>
 #include <vector>
+#include "const_config.h"
+#include "const_rpc.h"
+
 
-#include "tubemq/const_config.h"
 
 namespace tubemq {
 
@@ -43,6 +46,64 @@ using std::stringstream;
 
 static const char kWhitespaceCharSet[] = " \n\r\t\f\v";
 
+/*
+ *  copy from https://web.mit.edu/freebsd/head/sys/libkern/crc32.c
+*/
+static const uint32_t crc32_tab[256] = {
+  0x00000000, 0x77073096, 0xee0e612c, 0x990951ba, 0x076dc419, 0x706af48f,
+  0xe963a535, 0x9e6495a3, 0x0edb8832, 0x79dcb8a4, 0xe0d5e91e, 0x97d2d988,
+  0x09b64c2b, 0x7eb17cbd, 0xe7b82d07, 0x90bf1d91, 0x1db71064, 0x6ab020f2,
+  0xf3b97148, 0x84be41de, 0x1adad47d, 0x6ddde4eb, 0xf4d4b551, 0x83d385c7,
+  0x136c9856, 0x646ba8c0, 0xfd62f97a, 0x8a65c9ec, 0x14015c4f, 0x63066cd9,
+  0xfa0f3d63, 0x8d080df5, 0x3b6e20c8, 0x4c69105e, 0xd56041e4, 0xa2677172,
+  0x3c03e4d1, 0x4b04d447, 0xd20d85fd, 0xa50ab56b, 0x35b5a8fa, 0x42b2986c,
+  0xdbbbc9d6, 0xacbcf940, 0x32d86ce3, 0x45df5c75, 0xdcd60dcf, 0xabd13d59,
+  0x26d930ac, 0x51de003a, 0xc8d75180, 0xbfd06116, 0x21b4f4b5, 0x56b3c423,
+  0xcfba9599, 0xb8bda50f, 0x2802b89e, 0x5f058808, 0xc60cd9b2, 0xb10be924,
+  0x2f6f7c87, 0x58684c11, 0xc1611dab, 0xb6662d3d, 0x76dc4190, 0x01db7106,
+  0x98d220bc, 0xefd5102a, 0x71b18589, 0x06b6b51f, 0x9fbfe4a5, 0xe8b8d433,
+  0x7807c9a2, 0x0f00f934, 0x9609a88e, 0xe10e9818, 0x7f6a0dbb, 0x086d3d2d,
+  0x91646c97, 0xe6635c01, 0x6b6b51f4, 0x1c6c6162, 0x856530d8, 0xf262004e,
+  0x6c0695ed, 0x1b01a57b, 0x8208f4c1, 0xf50fc457, 0x65b0d9c6, 0x12b7e950,
+  0x8bbeb8ea, 0xfcb9887c, 0x62dd1ddf, 0x15da2d49, 0x8cd37cf3, 0xfbd44c65,
+  0x4db26158, 0x3ab551ce, 0xa3bc0074, 0xd4bb30e2, 0x4adfa541, 0x3dd895d7,
+  0xa4d1c46d, 0xd3d6f4fb, 0x4369e96a, 0x346ed9fc, 0xad678846, 0xda60b8d0,
+  0x44042d73, 0x33031de5, 0xaa0a4c5f, 0xdd0d7cc9, 0x5005713c, 0x270241aa,
+  0xbe0b1010, 0xc90c2086, 0x5768b525, 0x206f85b3, 0xb966d409, 0xce61e49f,
+  0x5edef90e, 0x29d9c998, 0xb0d09822, 0xc7d7a8b4, 0x59b33d17, 0x2eb40d81,
+  0xb7bd5c3b, 0xc0ba6cad, 0xedb88320, 0x9abfb3b6, 0x03b6e20c, 0x74b1d29a,
+  0xead54739, 0x9dd277af, 0x04db2615, 0x73dc1683, 0xe3630b12, 0x94643b84,
+  0x0d6d6a3e, 0x7a6a5aa8, 0xe40ecf0b, 0x9309ff9d, 0x0a00ae27, 0x7d079eb1,
+  0xf00f9344, 0x8708a3d2, 0x1e01f268, 0x6906c2fe, 0xf762575d, 0x806567cb,
+  0x196c3671, 0x6e6b06e7, 0xfed41b76, 0x89d32be0, 0x10da7a5a, 0x67dd4acc,
+  0xf9b9df6f, 0x8ebeeff9, 0x17b7be43, 0x60b08ed5, 0xd6d6a3e8, 0xa1d1937e,
+  0x38d8c2c4, 0x4fdff252, 0xd1bb67f1, 0xa6bc5767, 0x3fb506dd, 0x48b2364b,
+  0xd80d2bda, 0xaf0a1b4c, 0x36034af6, 0x41047a60, 0xdf60efc3, 0xa867df55,
+  0x316e8eef, 0x4669be79, 0xcb61b38c, 0xbc66831a, 0x256fd2a0, 0x5268e236,
+  0xcc0c7795, 0xbb0b4703, 0x220216b9, 0x5505262f, 0xc5ba3bbe, 0xb2bd0b28,
+  0x2bb45a92, 0x5cb36a04, 0xc2d7ffa7, 0xb5d0cf31, 0x2cd99e8b, 0x5bdeae1d,
+  0x9b64c2b0, 0xec63f226, 0x756aa39c, 0x026d930a, 0x9c0906a9, 0xeb0e363f,
+  0x72076785, 0x05005713, 0x95bf4a82, 0xe2b87a14, 0x7bb12bae, 0x0cb61b38,
+  0x92d28e9b, 0xe5d5be0d, 0x7cdcefb7, 0x0bdbdf21, 0x86d3d2d4, 0xf1d4e242,
+  0x68ddb3f8, 0x1fda836e, 0x81be16cd, 0xf6b9265b, 0x6fb077e1, 0x18b74777,
+  0x88085ae6, 0xff0f6a70, 0x66063bca, 0x11010b5c, 0x8f659eff, 0xf862ae69,
+  0x616bffd3, 0x166ccf45, 0xa00ae278, 0xd70dd2ee, 0x4e048354, 0x3903b3c2,
+  0xa7672661, 0xd06016f7, 0x4969474d, 0x3e6e77db, 0xaed16a4a, 0xd9d65adc,
+  0x40df0b66, 0x37d83bf0, 0xa9bcae53, 0xdebb9ec5, 0x47b2cf7f, 0x30b5ffe9,
+  0xbdbdf21c, 0xcabac28a, 0x53b39330, 0x24b4a3a6, 0xbad03605, 0xcdd70693,
+  0x54de5729, 0x23d967bf, 0xb3667a2e, 0xc4614ab8, 0x5d681b02, 0x2a6f2b94,
+  0xb40bbe37, 0xc30c8ea1, 0x5a05df1b, 0x2d02ef8d
+};
+
+int32_t Utils::Crc32(const string &buf) {
+  uint32_t crc = ~0U;
+  for (uint32_t i = 0; i < buf.size(); ++i) {
+    unsigned char c_data = buf[i];
+    crc = crc32_tab[(crc ^ c_data) & 0xFF] ^ (crc >> 8);
+  }
+  return ((~crc)& 0x7FFFFFFF);
+}
+
 string Utils::Trim(const string& source) {
   string target = source;
   if (!target.empty()) {
@@ -132,6 +193,55 @@ void Utils::Split(const string& source, map<string, int32_t>& result, const stri
   }
 }
 
+void Utils::Split(const string& source, map<string, string>& result,
+  const string& delimiter_step1, const string& delimiter_step2) {
+  string item_str;
+  string key_str;
+  string val_str;
+  string::size_type pos1 = 0;
+  string::size_type pos2 = 0;
+  string::size_type pos3 = 0;
+  if (!source.empty()) {
+    pos1 = 0;
+    pos2 = source.find(delimiter_step1);
+    while (string::npos != pos2) {
+      item_str = source.substr(pos1, pos2 - pos1);
+      item_str = Utils::Trim(item_str);
+      pos1 = pos2 + delimiter_step1.length();
+      pos2 = source.find(delimiter_step1, pos1);
+      if (item_str.empty()) {
+        continue;
+      }
+      pos3 = item_str.find(delimiter_step2);
+      if (string::npos == pos3) {
+        continue;
+      }
+      key_str = item_str.substr(0, pos3);
+      val_str = item_str.substr(pos3 + delimiter_step2.length());
+      key_str = Utils::Trim(key_str);
+      val_str = Utils::Trim(val_str);
+      if (key_str.empty()) {
+        continue;
+      }
+      result[key_str] = val_str;
+    }
+    if (pos1 != source.length()) {
+      item_str = source.substr(pos1);
+      item_str = Utils::Trim(item_str);
+      pos3 = item_str.find(delimiter_step2);
+      if (string::npos != pos3) {
+        key_str = item_str.substr(0, pos3);
+        val_str = item_str.substr(pos3 + delimiter_step2.length());
+        key_str = Utils::Trim(key_str);
+        val_str = Utils::Trim(val_str);
+        if (!key_str.empty()) {
+          result[key_str] = val_str;
+        }
+      }
+    }
+  }
+}
+
 void Utils::Join(const vector<string>& vec, const string& delimiter, string& target) {
   vector<string>::const_iterator it;
   target.clear();
@@ -143,6 +253,19 @@ void Utils::Join(const vector<string>& vec, const string& delimiter, string& tar
   }
 }
 
+void Utils::Join(const map<string, string>& source, string& target,
+  const string& delimiter_step1, const string& delimiter_step2) {
+  map<string, string>::const_iterator it;
+  target.clear();
+  int count = 0;
+  for (it = source.begin(); it != source.end(); ++it) {
+    if (count++ > 0) {
+      target += delimiter_step1;
+    }
+    target += it->first + delimiter_step2 + it->second;
+  }
+}
+
 bool Utils::ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
                         bool check_max_length, unsigned int maxlen) {
   if (source.empty()) {
@@ -202,7 +325,8 @@ bool Utils::ValidGroupName(string& err_info, const string& group_name, string& t
   int cflags = REG_EXTENDED;
   regex_t reg;
   regmatch_t pmatch[1];
-  const char* patRule = "^[a-zA-Z][\\w-]+$";
+  //  const char* patRule = "^[a-zA-Z][\\w-]+$";
+  const char* patRule = "^[a-zA-Z]\\w+$";
   regcomp(&reg, patRule, cflags);
   int status = regexec(&reg, tgt_group_name.c_str(), 1, pmatch, 0);
   regfree(&reg);
@@ -210,8 +334,9 @@ bool Utils::ValidGroupName(string& err_info, const string& group_name, string& t
     stringstream ss;
     ss << "Illegal parameter: ";
     ss << group_name;
-    ss << " must begin with a letter,can only contain ";
-    ss << "characters,numbers,hyphen,and underscores";
+    ss << " must begin with a letter,can only contain characters,numbers,and underscores";
+    //  ss << " must begin with a letter,can only contain ";
+    //  ss << "characters,numbers,hyphen,and underscores";
     err_info = ss.str();
     return false;
   }
@@ -281,13 +406,13 @@ int64_t Utils::GetCurrentTimeMillis() {
 
 bool Utils::ValidConfigFile(string& err_info, const string& conf_file) {
   FILE *fp = NULL;
-  
+
   if (conf_file.length() == 0) {
     err_info = "Configure file is blank";
     return false;
-  }  
-  fp = fopen(conf_file.c_str(),"r");
-  if(fp == NULL) {
+  }
+  fp = fopen(conf_file.c_str(), "r");
+  if (fp == NULL) {
     err_info = "Open configure file Failed!";
     return false;
   }
@@ -321,7 +446,7 @@ bool Utils::GetLocalIPV4Address(string& err_info, string& localhost) {
     if (0 == strncmp(&ifreq->ifr_name[0], "lo", sizeof("lo"))) {
       continue;
     }
-    memcpy(&if_flag.ifr_name[0],&ifreq->ifr_name[0],sizeof(ifreq->ifr_name));
+    memcpy(&if_flag.ifr_name[0], &ifreq->ifr_name[0], sizeof(ifreq->ifr_name));
     if ((ioctl(sockfd, SIOCGIFFLAGS, (char *) &if_flag)) < 0) {
       continue;
     }
@@ -329,8 +454,8 @@ bool Utils::GetLocalIPV4Address(string& err_info, string& localhost) {
       || !(if_flag.ifr_flags & IFF_UP)) {
       continue;
     }
-    
-    if (!strncmp(inet_ntoa(((struct sockaddr_in*)&(ifreq->ifr_addr))->sin_addr), 
+
+    if (!strncmp(inet_ntoa(((struct sockaddr_in*)&(ifreq->ifr_addr))->sin_addr),
       "127.0.0.1", 7)) {
       continue;
     }
@@ -344,6 +469,91 @@ bool Utils::GetLocalIPV4Address(string& err_info, string& localhost) {
   return false;
 }
 
+int32_t Utils::GetServiceTypeByMethodId(int32_t method_id) {
+  switch (method_id) {
+    // broker write service
+    case rpc_config::kBrokerMethoddProducerRegister:
+    case rpc_config::kBrokerMethoddProducerHeatbeat:
+    case rpc_config::kBrokerMethoddProducerSendMsg:
+    case rpc_config::kBrokerMethoddProducerClose: {
+      return rpc_config::kBrokerWriteService;
+    }
+    // broker read service
+    case rpc_config::kBrokerMethoddConsumerRegister:
+    case rpc_config::kBrokerMethoddConsumerHeatbeat:
+    case rpc_config::kBrokerMethoddConsumerGetMsg:
+    case rpc_config::kBrokerMethoddConsumerCommit:
+    case rpc_config::kBrokerMethoddConsumerClose: {
+      return rpc_config::kBrokerReadService;
+    }
+    // master service
+    case rpc_config::kMasterMethoddProducerRegister:
+    case rpc_config::kMasterMethoddProducerHeatbeat:
+    case rpc_config::kMasterMethoddProducerClose:
+    case rpc_config::kMasterMethoddConsumerRegister:
+    case rpc_config::kMasterMethoddConsumerHeatbeat:
+    case rpc_config::kMasterMethoddConsumerClose:
+    default: {
+      return rpc_config::kMasterService;
+    }
+  }
+}
+
+void Utils::XfsAddrByDns(const map<string, int32_t>& orig_addr_map,
+  map<string, string>& target_addr_map) {
+  hostent* host = NULL;
+  map<string, int32_t>::const_iterator it;
+  for (it = orig_addr_map.begin(); it != orig_addr_map.end(); it++) {
+    char first_char =  it->first.c_str()[0];
+    if (isalpha(first_char)) {
+      host = gethostbyname(it->first.c_str());
+      if (host != NULL) {
+        switch (host->h_addrtype) {
+          case AF_INET:
+          case AF_INET6: {
+            char **pptr = NULL;
+            unsigned int addr = 0;
+            char temp_str[32];
+            memset(temp_str, 0, 32);
+            pptr = host->h_addr_list;
+            addr = ((unsigned int *) host->h_addr_list[0])[0];
+            if ((addr & 0xffff) == 0x0a0a) {
+              pptr++;
+              addr = ((unsigned int *) host->h_addr_list[0])[1];
+            }
+            inet_ntop(host->h_addrtype, *pptr, temp_str, sizeof(temp_str));
+            string tempIpaddr = temp_str;
+            if (tempIpaddr.length() > 0) {
+              target_addr_map[it->first] = tempIpaddr;
+            }
+          }
+          break;
+
+          default:
+            break;
+        }
+      }
+    } else {
+      target_addr_map[it->first] = it->first;
+    }
+  }
+}
+
+bool Utils::NeedDnsXfs(const string& masteraddr) {
+  if (masteraddr.length() > 0) {
+    char first_char = masteraddr.c_str()[0];
+    if (isalpha(first_char)) {
+      return true;
+    }
+  }
+  return false;
+}
+
+string Utils::GenBrokerAuthenticateToken(const string& username,
+  const string& usrpassword) {
+  return "";
+}
+
 
 }  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.h b/tubemq-client-twins/tubemq-client-cpp/src/utils.h
new file mode 100644
index 0000000..c61043b
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.h
@@ -0,0 +1,71 @@
+/**
+ * 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 TUBEMQ_CLIENT_UTILS_H_
+#define TUBEMQ_CLIENT_UTILS_H_
+
+#include <stdint.h>
+
+#include <map>
+#include <string>
+#include <vector>
+
+namespace tubemq {
+
+using std::map;
+using std::string;
+using std::vector;
+
+class Utils {
+ public:
+  // trim string info
+  static int32_t Crc32(const string &buf);
+  static string Trim(const string& source);
+  // split string to vector
+  static void Split(const string& source, vector<string>& result, const string& delimiter);
+  // split string to map<string, int>
+  static void Split(const string& source, map<string, int32_t>& result,
+                    const string& delimiter_step1, const string& delimiter_step2);
+  static void Split(const string& source, map<string, string>& result,
+                    const string& delimiter_step1, const string& delimiter_step2);
+  static void Join(const vector<string>& vec, const string& delimiter, string& target);
+  static void Join(const map<string, string>& source, string& target,
+    const string& delimiter_step1, const string& delimiter_step2);
+  static bool ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
+                          bool check_max_length, unsigned int maxlen);
+  static bool ValidGroupName(string& err_info, const string& group_name, string& tgt_group_name);
+  static bool ValidFilterItem(string& err_info, const string& src_filteritem,
+                              string& tgt_filteritem);
+  static string Int2str(int32_t data);
+  static string Long2str(int64_t data);
+  static uint32_t IpToInt(const string& ipv4_addr);
+  static int64_t GetCurrentTimeMillis();
+  static bool ValidConfigFile(string& err_info, const string& conf_file);
+  static bool GetLocalIPV4Address(string& err_info, string& localhost);
+  static int32_t GetServiceTypeByMethodId(int32_t method_id);
+  static void XfsAddrByDns(const map<string, int32_t>& orig_addr_map,
+    map<string, string>& target_addr_map);
+  static bool NeedDnsXfs(const string& masteraddr);
+  static string GenBrokerAuthenticateToken(const string& username, const string& usrpassword);
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_UTILS_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/version.h b/tubemq-client-twins/tubemq-client-cpp/src/version.h
new file mode 100644
index 0000000..c479ede
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/version.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 TUBEMQ_CLIENT_VERSION_H_
+#define TUBEMQ_CLIENT_VERSION_H_
+
+#include <string>
+
+namespace tubemq {
+
+using std::string;
+
+static const char kTubeMQClientVersion[] = "0.1.0-0.5.0";
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_VERSION_H_


[incubator-tubemq] 43/50: [TUBEMQ-288]C++ SDK Codec interface (#254)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 09d2c9bf02aad92d1ecfbd41960f44eda24102fa
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Sat Sep 12 09:36:40 2020 +0800

    [TUBEMQ-288]C++ SDK Codec interface (#254)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/src/codec_protocol.h         | 58 ++++++++++++++++++++++
 1 file changed, 58 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/codec_protocol.h b/tubemq-client-twins/tubemq-client-cpp/src/codec_protocol.h
new file mode 100644
index 0000000..82e68e7
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/codec_protocol.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 _TUBEMQ_CODEC_PROTOCOL_H_
+#define _TUBEMQ_CODEC_PROTOCOL_H_
+
+#include <list>
+#include <memory>
+#include <string>
+
+#include "any.h"
+#include "buffer.h"
+
+namespace tubemq {
+
+class CodecProtocol {
+ public:
+  CodecProtocol() {}
+
+  virtual ~CodecProtocol() {}
+
+  virtual std::string Name() const = 0;
+
+  virtual bool Decode(const BufferPtr &buff, Any &out) = 0;
+
+  virtual bool Encode(const Any &in, BufferPtr &buff) = 0;
+
+  // return code: -1 failed; 0-Unfinished; > 0 package buffer size
+  virtual int32_t Check(BufferPtr &in, Any &out, uint32_t &request_id, bool &has_request_id,
+                        size_t &package_length) = 0;
+
+  // get protocol request id
+  virtual int32_t GetRequestId(uint32_t &request_id, const Any &rsp) const { return -1; }
+
+  // set protocol request request id
+  virtual int32_t SetRequestId(uint32_t request_id, Any &req) { return -1; }
+};
+
+using CodecProtocolPtr = std::shared_ptr<CodecProtocol>;
+
+}  // namespace tubemq
+#endif  // _TUBEMQ_CODEC_PROTOCOL_H_


[incubator-tubemq] 19/50: [TUBEMQ-268] C++ SDK log module (#190)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit e9ce60431a16acef88fd4295d2320265e1e505e8
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Mon Jul 6 18:57:06 2020 +0800

    [TUBEMQ-268] C++ SDK log module (#190)
---
 .gitmodules                                        |   4 +
 .../tubemq-client-cpp/example/README.md            |  22 +++++
 .../tubemq-client-cpp/example/log/main.cc          |  46 +++++++++
 tubemq-client-twins/tubemq-client-cpp/inc/logger.h | 109 +++++++++++++++++++++
 .../tubemq-client-cpp/inc/noncopyable.h            |  37 +++++++
 .../tubemq-client-cpp/inc/singleton.h              |  60 ++++++++++++
 .../tubemq-client-cpp/inc/unique_seq_id.h          |  40 ++++++++
 .../tubemq-client-cpp/src/logger.cc                |  75 ++++++++++++++
 .../tubemq-client-cpp/third_party/log4cplus        |   1 +
 9 files changed, 394 insertions(+)

diff --git a/.gitmodules b/.gitmodules
index 417049c..c80f44d 100644
--- a/.gitmodules
+++ b/.gitmodules
@@ -2,3 +2,7 @@
 	path = tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
 	url = https://github.com/Tencent/rapidjson.git
 	tag = v1.1.0
+[submodule "tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus"]
+	path = tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
+	url = https://github.com/log4cplus/log4cplus
+	tag = REL_2_0_5
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/README.md b/tubemq-client-twins/tubemq-client-cpp/example/README.md
new file mode 100644
index 0000000..4998b5f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/README.md
@@ -0,0 +1,22 @@
+/**
+ * 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.
+ */
+
+#Example#
+
+tubemq-client-cpp example. 
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc b/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
new file mode 100644
index 0000000..f21d022
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <chrono>
+#include <exception>
+#include <iostream>
+#include <string>
+#include <thread>
+
+#include "logger.h"
+
+using namespace std;
+using namespace tubemq;
+
+void log() {
+  int i = 0;
+  while (1) {
+    LOG_ERROR("threadid:%ld, i:%d", std::this_thread::get_id(), i++);
+  }
+}
+
+int main() {
+  std::thread t1(log);
+  std::thread t2(log);
+  std::thread t3(log);
+  std::thread t4(log);
+  t1.join();
+  return 0;
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
new file mode 100644
index 0000000..44c28b7
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
@@ -0,0 +1,109 @@
+/**
+ * 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 _TUBEMQ_LOG_FILE_
+#define _TUBEMQ_LOG_FILE_
+
+#include <string>
+#include <vector>
+
+namespace tubemq {
+class Logger;
+
+Logger& GetLogger();
+
+#define LOG_LEVEL(level, fmt, ...)                                                                          \
+  {                                                                                                         \
+    if (tubemq::GetLogger().IsEnable(level)) {                                                              \
+      tubemq::GetLogger().Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), \
+                                ##__VA_ARGS__);                                                             \
+    }                                                                                                       \
+  }
+
+#define LOG_TRACE(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
+#define LOG_DEBUG(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
+#define LOG_INFO(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
+#define LOG_WARN(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
+#define LOG_ERROR(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
+
+#define LOG_TUBE(logger, level, fmt, ...)                                                                      \
+  {                                                                                                            \
+    if (logger.IsEnable(level)) {                                                                              \
+      logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
+    }                                                                                                          \
+  }
+
+class Logger {
+ public:
+  enum Level
+  {
+    kTrace = 0,
+    kDebug = 1,
+    kInfo = 2,
+    kWarn = 3,
+    kError = 4,
+  };
+
+  // size: MB
+  Logger() : file_max_size_(100), file_num_(10), level_(kError), base_path_("tubemq"), instance_("TubeMQ") { setup(); }
+
+  ~Logger(void) {}
+
+  // path example: ../log/tubemq
+  // size: MB
+  bool Init(const std::string& path, Level level, uint32_t file_max_size = 100, uint32_t file_num = 10);
+
+  bool Write(const char* sFormat, ...) __attribute__((format(printf, 2, 3)));
+  inline bool WriteStream(const std::string& msg) { return writeStream(msg.c_str()); }
+
+  inline void SetInstance(const std::string& instance) { instance_ = instance; }
+  inline bool IsEnable(Level level) {
+    if (level_ <= level) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  static const char* Level2String(Level level) {
+    static const char* level_names[] = {
+        "TRACE",
+        "DEBUG"
+        "INFO",
+        "WARN",
+        "ERROR",
+    };
+    return level_names[level];
+  }
+
+ private:
+  void setup();
+  bool writeStream(const char* msg);
+
+ private:
+  uint32_t file_max_size_;
+  uint16_t file_num_;
+  uint8_t level_;
+
+  std::string base_path_;
+  std::string instance_;
+  std::string err_msg_;
+};
+}  // namespace tubemq
+#endif  // _TUBEMQ_LOG_FILE_
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/noncopyable.h b/tubemq-client-twins/tubemq-client-cpp/inc/noncopyable.h
new file mode 100644
index 0000000..9afbf52
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/noncopyable.h
@@ -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.
+ */
+
+#ifndef _TUBUMQ_NONCOPYABLE_H
+#define _TUBUMQ_NONCOPYABLE_H
+
+namespace tubemq {
+
+class noncopyable {
+ public:
+  noncopyable(const noncopyable&) = delete;
+  void operator=(const noncopyable&) = delete;
+
+ protected:
+  noncopyable() = default;
+  ~noncopyable() = default;
+};
+
+}  // namespace tubemq
+
+#endif  // _TUBUMQ_NONCOPYABLE_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h b/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
new file mode 100644
index 0000000..af3b0d4
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
@@ -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.
+ */
+
+#ifndef _TUBEMQ_SINGLETON_H
+#define _TUBEMQ_SINGLETON_H
+
+#include <thread>
+#include <mutex>
+#include <assert.h>
+#include <stdlib.h>
+
+#include "noncopyable.h"
+
+namespace tubemq {
+
+template <typename T>
+class Singleton : noncopyable {
+ public:
+  Singleton() = delete;
+  ~Singleton() = delete;
+
+  static T& instance() {
+    std::call_once(once_, Singleton::init);
+    assert(value_ != nullptr);
+    return *value_;
+  }
+
+ private:
+  static void init() { value_ = new T(); }
+
+ private:
+  static std::once_flag once_;
+  static T* value_;
+};
+
+template <typename T>
+std::once_flag Singleton<T>::once_;
+
+template <typename T>
+T* Singleton<T>::value_ = nullptr;
+
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_SINGLETON_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h b/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
new file mode 100644
index 0000000..d836be1
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
@@ -0,0 +1,40 @@
+/**
+ * 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 TUBEMQ_UNIQUESEQID_H
+#define TUBEMQ_UNIQUESEQID_H
+
+#include <atomic>
+#include <stdint.h>
+
+namespace tubemq {
+
+class UniqueSeqId {
+ public:
+  UniqueSeqId() : id(0) {}
+
+  uint32_t Next() { return id.fetch_add(1, std::memory_order_relaxed); }
+
+ protected:
+  std::atomic<uint32_t> id;
+};
+
+}  // namespace tubemq
+
+#endif
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
new file mode 100644
index 0000000..c1860df
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -0,0 +1,75 @@
+/**
+ * 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 "logger.h"
+
+#include <log4cplus/fileappender.h>
+#include <log4cplus/layout.h>
+#include <log4cplus/logger.h>
+#include <log4cplus/loggingmacros.h>
+#include <stdarg.h>
+
+#include <string>
+
+namespace tubemq {
+
+Logger tubemq_logger;
+
+Logger& GetLogger() { return tubemq_logger; }
+
+bool Logger::Init(const std::string& path, Logger::Level level, uint32_t file_max_size, uint32_t file_num) {
+  base_path_ = path;
+  file_max_size_ = file_max_size;
+  file_num_ = file_num;
+  level_ = level;
+  setup();
+  return true;
+}
+
+bool Logger::Write(const char* format, ...) {
+  char buf[8192];
+  buf[sizeof(buf) - 1] = 0;
+  va_list ap;
+  va_start(ap, format);
+  vsnprintf(buf, sizeof(buf) - 1, format, ap);
+  va_end(ap);
+  return writeStream(buf);
+}
+
+bool Logger::writeStream(const char* log) {
+  auto logger = log4cplus::Logger::getInstance(instance_);
+  log4cplus::tostringstream _log4cplus_buf;
+  _log4cplus_buf << log;
+  logger.forcedLog(log4cplus::TRACE_LOG_LEVEL, _log4cplus_buf.str());
+  return true;
+}
+
+void Logger::setup() {
+  bool immediate_fush = true;
+  std::string pattern = "[%D{%Y-%m-%d %H:%M:%S.%q}][tid:%t]%m%n";
+  auto logger_d = log4cplus::Logger::getInstance(instance_);
+  logger_d.setLogLevel(log4cplus::TRACE_LOG_LEVEL);
+  log4cplus::helpers::SharedObjectPtr<log4cplus::Appender> append_d(
+      new log4cplus::RollingFileAppender(base_path_ + ".log", file_max_size_, file_num_, immediate_fush));
+  std::unique_ptr<log4cplus::Layout> layout_d(new log4cplus::PatternLayout(pattern));
+  append_d->setLayout(std::move(layout_d));
+  logger_d.addAppender(append_d);
+}
+
+}  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus b/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
new file mode 160000
index 0000000..76ff7e6
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
@@ -0,0 +1 @@
+Subproject commit 76ff7e68c35e277440d414ba782eceedad8db7b1


[incubator-tubemq] 17/50: [TUBEMQ-252] Create C/C++ Metadata classes (#186)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 318c8cc9bba147553f5b8205a1c3045419930cb9
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 6 01:40:20 2020 +0000

    [TUBEMQ-252] Create C/C++ Metadata classes (#186)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/const_config.h           |   7 +-
 .../tubemq-client-cpp/inc/meta_info.h              |  94 ++++++++++++++
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  |   5 +-
 .../tubemq-client-cpp/src/file_ini.cc              |   4 +-
 .../tubemq-client-cpp/src/meta_info.cc             | 139 +++++++++++++++++++++
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  15 ++-
 6 files changed, 256 insertions(+), 8 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index 1cc2b60..80d8b3b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -51,7 +51,6 @@ static const int kFilterItemMaxCount = 500;
 // max session key length
 static const int kSessionKeyMaxLength = 1024;
 
-
 // max subscribe info report times
 static const int kSubInfoReportMaxIntervalTimes = 6;
 // default message not found response wait period
@@ -68,6 +67,11 @@ static const int kMaxIntValue = 0x7fffffff;
 // max long value
 static const long kMaxLongValue = 0x7fffffffffffffffL;
 
+// default broker port
+static const int kBrokerPortDef = 8123;
+// default broker TLS port
+static const int kBrokerTlsPortDef = 8124;
+
 // invalid value
 static const int kInvalidValue = -2;
 
@@ -76,6 +80,7 @@ static const int kInvalidValue = -2;
 
 
 namespace delimiter {
+  static const string kDelimiterDot = ".";
   static const string kDelimiterEqual = "=";
   static const string kDelimiterAnd   = "&";
   static const string kDelimiterComma = ",";
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
new file mode 100644
index 0000000..55baa4c
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
@@ -0,0 +1,94 @@
+/**
+ * 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 _TUBEMQ_CLIENT_META_INFO_H_
+#define _TUBEMQ_CLIENT_META_INFO_H_
+
+#include <string>
+
+namespace tubemq {
+
+using namespace std;
+
+
+
+class NodeInfo {
+ public:
+  NodeInfo();
+  NodeInfo(bool is_broker, const string& node_info);
+  NodeInfo(const string& node_host, int node_port);
+  NodeInfo(int node_id, const string& node_host, int node_port);
+  ~NodeInfo();
+  NodeInfo& operator=(const NodeInfo& target);
+  bool operator== (const NodeInfo& target);
+  bool operator< (const NodeInfo& target) const;
+  const int GetNodeId() const;
+  const string& GetHost() const;
+  const int GetPort() const;
+  const string& GetAddrInfo() const;
+  const string& GetNodeInfo() const;
+      
+ private:
+  void buildStrInfo();
+
+ private: 
+  int    node_id_;
+  string node_host_;
+  int    node_port_;
+  // ip:port
+  string addr_info_;
+  // id:ip:port
+  string node_info_;
+};
+
+
+class Partition {
+ public:
+  Partition();
+  Partition(const string& partition_info);
+  Partition(const NodeInfo& broker_info, const string& partStr);
+  Partition(const NodeInfo& broker_info, const string& topic, int partition_id);
+  ~Partition();
+  Partition& operator=(const Partition& target);
+  bool operator== (const Partition& target);
+  const int GetBrokerId() const;
+  const string& GetBrokerHost() const;
+  const int GetBrokerPort() const;
+  const string& GetPartitionKey() const;
+  const string& GetTopic() const;
+  const NodeInfo& GetBrokerInfo() const;
+  const int GetPartitionId() const;
+  const string& ToString() const;
+
+ private:
+  void buildPartitionKey();
+
+ private:
+  string   topic_;
+  NodeInfo broker_info_;
+  int      partition_id_;   
+  string   partition_key_;
+  string   partition_info_;
+};
+
+
+}
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index 0709bb8..efb22d7 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -47,8 +47,9 @@ class Utils {
                    const string& group_name, string& tgt_group_name);
   static bool ValidFilterItem(string& err_info, 
                    const string& src_filteritem, string& tgt_filteritem);
-  static string int2str(int data);
-  static string long2str(long data);
+  static string Int2str(int data);
+  static string Long2str(long data);
+  static int IpToInt(const string& ipv4_addr);
   static long GetCurrentTimeMillis();
 
 };
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 822b803..df0cdf3 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -135,7 +135,7 @@ bool Fileini::GetValue(string& err_info, const string& sector,
 bool Fileini::GetValue(string& err_info, const string& sector, 
                 const string& key, int& value, const int def) {
   string val_str;
-  string def_str = Utils::int2str(def);
+  string def_str = Utils::Int2str(def);
   bool result = GetValue(err_info, sector, key, val_str, def_str);
   if (!result) {
     return result;
@@ -144,8 +144,6 @@ bool Fileini::GetValue(string& err_info, const string& sector,
   return true;
 }
 
-
-
 }
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
new file mode 100644
index 0000000..c744b47
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -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 <sstream>
+#include <stdlib.h>
+#include "utils.h"
+#include "meta_info.h"
+#include "const_config.h"
+
+
+namespace tubemq {
+
+
+NodeInfo::NodeInfo() {
+  this->node_id_   = config::kInvalidValue;
+  this->node_host_ = " ";
+  this->node_port_ = config::kInvalidValue;
+  buildStrInfo();
+}
+
+NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
+  vector<string> result;
+  Utils::Split(node_info, result, delimiter::kDelimiterColon);
+  if (is_broker) {
+    this->node_id_   = atoi(result[0].c_str());
+    this->node_host_ = result[1];
+    this->node_port_ = config::kBrokerPortDef;
+    if(result.size() >= 3){
+      this->node_port_ = atoi(result[2].c_str());
+    }
+  } else {
+    this->node_id_   = config::kInvalidValue;
+    this->node_host_ = result[0];
+    this->node_port_ = config::kBrokerPortDef;
+    if (result.size() >= 2) {
+      this->node_port_ = atoi(result[1].c_str());
+    }
+  }
+  buildStrInfo();
+}
+
+NodeInfo::NodeInfo(const string& node_host, int node_port) {
+  this->node_id_   = config::kInvalidValue;
+  this->node_host_ = node_host;
+  this->node_port_ = node_port;
+  buildStrInfo();
+
+}
+
+NodeInfo::NodeInfo(int node_id, const string& node_host, int node_port) {
+  this->node_id_   = node_id;
+  this->node_host_ = node_host;
+  this->node_port_ = node_port;
+  buildStrInfo();
+}
+
+NodeInfo::~NodeInfo() {
+
+}
+
+NodeInfo& NodeInfo::operator=(const NodeInfo& target) {
+  if (this != &target){
+    this->node_id_   = target.node_id_;
+    this->node_host_ = target.node_host_;
+    this->node_port_ = target.node_port_;
+    this->addr_info_ = target.addr_info_;
+    this->node_info_ = target.node_info_;
+  }
+  return *this;
+}
+
+bool NodeInfo::operator== (const NodeInfo& target) {
+  if (this == &target) {
+    return true;
+  }
+  if (this->node_info_ == target.node_info_) {
+    return true;
+  }
+  return false;
+
+}
+
+bool NodeInfo::operator< (const NodeInfo& target) const {
+  return this->node_id_ < target.node_id_;
+}
+  
+const int NodeInfo::GetNodeId() const {
+  return this->node_id_;
+}
+
+const string& NodeInfo::GetHost() const {
+  return this->node_host_;
+}
+
+const int NodeInfo::GetPort() const {
+  return this->node_port_;
+}
+  
+const string& NodeInfo::GetAddrInfo() const {
+  return this->addr_info_;
+}
+
+const string& NodeInfo::GetNodeInfo() const {
+  return this->node_info_;
+}
+
+void NodeInfo::buildStrInfo() {
+  stringstream ss1;
+  ss1 << this->node_host_;
+  ss1 << delimiter::kDelimiterColon;
+  ss1 << this->node_port_;
+  this->addr_info_ = ss1.str();
+
+  stringstream ss2;
+  ss2 << this->node_id_;
+  ss2 << delimiter::kDelimiterColon;
+  ss2 << this->addr_info_;
+  this->node_info_ = ss2.str();
+}
+
+
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index cd60982..d1fac8f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -239,18 +239,29 @@ bool Utils::ValidFilterItem(string& err_info,
 }
 
 
-string Utils::int2str(int data) {
+string Utils::Int2str(int data) {
   stringstream ss;
   ss<<data;
   return ss.str();
 }
 
-string Utils::long2str(long data) {
+string Utils::Long2str(long data) {
   stringstream ss;
   ss<<data;
   return ss.str();
 }
 
+int Utils::IpToInt(const string& ipv4_addr) {
+  int result = 0;
+  vector<string> result_vec;
+
+  Utils::Split(ipv4_addr, result_vec, delimiter::kDelimiterDot);
+  result = ((char) atoi(result_vec[3].c_str())) & 0xFF;
+  result |= ((char) atoi(result_vec[2].c_str()) << 8) & 0xFF00;
+  result |= ((char) atoi(result_vec[1].c_str()) << 16) & 0xFF0000;
+  result |= ((char) atoi(result_vec[0].c_str()) << 24) & 0xFF000000;
+  return result;
+}
 
 long Utils::GetCurrentTimeMillis() {
   struct timeval tv;


[incubator-tubemq] 30/50: [TUBEMQ-269]Create C/C++ RmtDataCache class (#204)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 325073c7b7577f972652e00a6e0714d8d6c4c36c
Author: gosonzhang <46...@qq.com>
AuthorDate: Tue Jul 14 01:54:45 2020 +0000

    [TUBEMQ-269]Create C/C++ RmtDataCache class (#204)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |   1 +
 .../include/tubemq/rmt_data_cache.h                |  25 +++-
 .../tubemq-client-cpp/src/meta_info.cc             |   6 +
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 159 +++++++++++++++++----
 4 files changed, 163 insertions(+), 28 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index 2d2796e..ccd62e9 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -135,6 +135,7 @@ class PartitionExt : public Partition {
 
 class SubscribeInfo {
  public:
+  SubscribeInfo();
   SubscribeInfo(const string& sub_info);
   SubscribeInfo(const string& consumer_id,
         const string& group_name, const PartitionExt& partition_ext);
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index 7c25757..2b79f13 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -26,13 +26,16 @@
 #include <list>
 #include <map>
 #include <set>
+#include <string>
 
+#include "tubemq/atomic_def.h"
 #include "tubemq/flowctrl_def.h"
 #include "tubemq/meta_info.h"
 
 
 
 
+
 namespace tubemq {
 
 using std::map;
@@ -45,6 +48,12 @@ class RmtDataCacheCsm {
  public:
   RmtDataCacheCsm(const string& client_id, const string& group_name);
   ~RmtDataCacheCsm();
+  void UpdateDefFlowCtrlInfo(int64_t flowctrl_id, 
+                                     const string& flowctrl_info);
+  void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
+                 int64_t flowctrl_id, const string& flowctrl_info);
+  const int64_t GetGroupQryPriorityId() const;
+  bool IsUnderGroupCtrl();
   void AddNewPartition(const PartitionExt& partition_ext);
   bool SelectPartition(string &err_info,
            PartitionExt& partition_ext, string& confirm_context);
@@ -54,20 +63,24 @@ class RmtDataCacheCsm {
   bool RelPartition(string &err_info, bool filter_consume,
                          const string& confirm_context, bool is_consumed);
   bool RelPartition(string &err_info, const string& confirm_context, bool is_consumed);
-  bool RelPartition(string &err_info, bool filter_consume, 
+  bool RelPartition(string &err_info, bool filter_consume,
                          const string& confirm_context, bool is_consumed,
                          int64_t curr_offset, int32_t err_code, bool esc_limit,
                          int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt);
   void FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
-                    list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions);
+          list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions);
   void GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst);
   bool GetPartitionExt(const string& part_key, PartitionExt& partition_ext);
   void GetRegBrokers(list<NodeInfo>& brokers);
+  void GetPartitionByBroker(const NodeInfo& broker_info,
+                                    list<PartitionExt>& partition_list);
   void GetCurPartitionOffsets(map<string, int64_t> part_offset_map);
   void GetAllBrokerPartitions(map<NodeInfo, list<PartitionExt> >& broker_parts);
   void RemovePartition(const list<PartitionExt>& partition_list);
   void RemovePartition(const set<string>& partition_keys);
   bool RemovePartition(string &err_info, const string& confirm_context);
+  void RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
+        bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts);
   bool BookPartition(const string& partition_key);
   void OfferEvent(const ConsumerEvent& event);
   void TakeEvent(ConsumerEvent& event);
@@ -84,15 +97,20 @@ class RmtDataCacheCsm {
   bool inRelPartition(string &err_info, bool need_delay_check,
     bool filter_consume, const string& confirm_context, bool is_consumed);
 
+
  private:
   // timer begin
 
   // timer end
   string consumer_id_;
-  string group_name_;  
+  string group_name_;
   // flow ctrl
   FlowCtrlRuleHandler group_flowctrl_handler_;
   FlowCtrlRuleHandler def_flowctrl_handler_;
+  AtomicBoolean under_groupctrl_;
+  AtomicLong last_checktime_;
+
+  // meta info
   pthread_rwlock_t meta_rw_lock_;
   // partiton allocated map
   map<string, PartitionExt> partitions_;
@@ -115,6 +133,7 @@ class RmtDataCacheCsm {
   map<string, int64_t> partition_offset_;
   // for partiton register booked
   map<string, bool> part_reg_booked_;
+
   // event
   pthread_mutex_t  event_read_mutex_;
   pthread_cond_t   event_read_cond_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index aea6239..7694c5b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -423,6 +423,12 @@ void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_ha
   }
 }
 
+SubscribeInfo::SubscribeInfo() {
+  this->consumer_id_ = " ";
+  this->group_ = " ";
+  this->partitionext_;
+  buildSubInfo();
+}
 
 // sub_info = consumerId@group#broker_info#topic:partitionId
 SubscribeInfo::SubscribeInfo(const string& sub_info) {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index 3b86b43..caa4236 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -38,6 +38,8 @@ RmtDataCacheCsm::RmtDataCacheCsm(const string& client_id,
                                       const string& group_name) {
   consumer_id_ = client_id;
   group_name_ = group_name;
+  under_groupctrl_.Set(false);
+  last_checktime_.Set(0);
   pthread_rwlock_init(&meta_rw_lock_, NULL);
   pthread_mutex_init(&part_mutex_, NULL);
   pthread_mutex_init(&data_book_mutex_, NULL);
@@ -55,6 +57,43 @@ RmtDataCacheCsm::~RmtDataCacheCsm() {
   pthread_rwlock_destroy(&meta_rw_lock_);
 }
 
+void RmtDataCacheCsm::UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
+                                                 const string& flowctrl_info) {
+  if (flowctrl_id != def_flowctrl_handler_.GetFlowCtrlId()) {
+    def_flowctrl_handler_.UpdateDefFlowCtrlInfo(true, 
+      tb_config::kInvalidValue, flowctrl_id, flowctrl_info);
+  }
+}
+void RmtDataCacheCsm::UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
+                             int64_t flowctrl_id, const string& flowctrl_info) {
+  if (flowctrl_id != group_flowctrl_handler_.GetFlowCtrlId()) {
+    group_flowctrl_handler_.UpdateDefFlowCtrlInfo(false, 
+                qyrpriority_id, flowctrl_id, flowctrl_info);
+  }
+  if (qyrpriority_id != group_flowctrl_handler_.GetQryPriorityId()) {
+    this->group_flowctrl_handler_.SetQryPriorityId(qyrpriority_id);
+
+  }
+  // update current if under group flowctrl 
+  int64_t cur_time = Utils::GetCurrentTimeMillis();
+  if (cur_time - last_checktime_.Get() > 10000) {
+    FlowCtrlResult flowctrl_result;
+    this->under_groupctrl_.Set(
+      group_flowctrl_handler_.GetCurDataLimit(
+        tb_config::kMaxLongValue, flowctrl_result));
+    last_checktime_.Set(cur_time);
+  }
+}
+
+const int64_t RmtDataCacheCsm::GetGroupQryPriorityId() const {
+  return this->group_flowctrl_handler_.GetQryPriorityId();
+}
+
+bool RmtDataCacheCsm::IsUnderGroupCtrl() {
+  return this->under_groupctrl_.Get();
+}
+
+
 void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   //
   map<string, PartitionExt>::iterator it_map;
@@ -119,7 +158,7 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
     } else {
       result = false;
       err_info = "No idle partition to consume data 2, please retry later!";
-      booked_time =Utils::GetCurrentTimeMillis();
+      booked_time = Utils::GetCurrentTimeMillis();
       partition_key = index_partitions_.front();
       index_partitions_.pop_front();
       buildConfirmContext(partition_key, booked_time, confirm_context);
@@ -138,7 +177,7 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
 }
 
 void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t curr_offset,
-                                             int32_t err_code, bool esc_limit, int32_t msg_size, 
+                                             int32_t err_code, bool esc_limit, int32_t msg_size,
                                              int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow) {
   map<string, PartitionExt>::iterator it_part;
   // book partition offset info
@@ -150,7 +189,7 @@ void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t cur
   // book partition temp info
   pthread_rwlock_rdlock(&meta_rw_lock_);
   it_part = partitions_.find(partition_key);
-  if(it_part != partitions_.end()) {
+  if (it_part != partitions_.end()) {
     it_part->second.BookConsumeData(err_code, msg_size,
               esc_limit, limit_dlt, cur_data_dlt, require_slow);
   }
@@ -172,11 +211,11 @@ bool RmtDataCacheCsm::RelPartition(string &err_info,
 // release partiton with error response return
 bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
                               const string& confirm_context, bool is_consumed,
-                              int64_t curr_offset, int32_t err_code, bool esc_limit, 
+                              int64_t curr_offset, int32_t err_code, bool esc_limit,
                               int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt) {
   int64_t booked_time;
   string  partition_key;
-  // parse confirm context  
+  // parse confirm context
   bool result = parseConfirmContext(err_info,
                       confirm_context, partition_key, booked_time);
   if (!result) {
@@ -184,12 +223,12 @@ bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
   }
   BookedPartionInfo(partition_key, curr_offset, err_code,
             esc_limit, msg_size, limit_dlt, cur_data_dlt, false);
-  return inRelPartition(err_info, true, 
+  return inRelPartition(err_info, true,
     filter_consume, confirm_context, is_consumed);
 }
 
 void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
-                    list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions) {
+            list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions) {
   //
   map<string, PartitionExt>::iterator it_part;
   list<SubscribeInfo>::const_iterator it_lst;
@@ -204,10 +243,10 @@ void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info
   } else {
     for (it_lst = subscribe_info_lst.begin(); it_lst != subscribe_info_lst.end(); it_lst++) {
       it_part = partitions_.find(it_lst->GetPartitionExt().GetPartitionKey());
-    	if (it_part == partitions_.end()) {
+      if (it_part == partitions_.end()) {
         unsub_partitions.push_back(it_lst->GetPartitionExt());
-    	} else {
-    		subscribed_partitions.push_back(it_lst->GetPartitionExt());
+      } else {
+        subscribed_partitions.push_back(it_lst->GetPartitionExt());
       }
     }
   }
@@ -216,12 +255,12 @@ void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info
 
 void RmtDataCacheCsm::GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst) {
   map<string, SubscribeInfo>::iterator it_sub;
-  subscribe_info_lst.clear();                                             
-	pthread_rwlock_rdlock(&meta_rw_lock_);
-	for (it_sub = part_subinfo_.begin(); it_sub != part_subinfo_.end(); ++it_sub) {
+  subscribe_info_lst.clear();
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  for (it_sub = part_subinfo_.begin(); it_sub != part_subinfo_.end(); ++it_sub) {
     subscribe_info_lst.push_back(it_sub->second);
-	}
-	pthread_rwlock_unlock(&meta_rw_lock_);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 void RmtDataCacheCsm::GetAllBrokerPartitions(
@@ -244,7 +283,6 @@ void RmtDataCacheCsm::GetAllBrokerPartitions(
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
-
 bool RmtDataCacheCsm::GetPartitionExt(const string& part_key, PartitionExt& partition_ext) {
   bool result = false;
   map<string, PartitionExt>::iterator it_map;
@@ -253,14 +291,14 @@ bool RmtDataCacheCsm::GetPartitionExt(const string& part_key, PartitionExt& part
   it_map = partitions_.find(part_key);
   if (it_map != partitions_.end()) {
     result = true;
-    partition_ext = it_map->second;  
+    partition_ext = it_map->second;
   }
   pthread_rwlock_unlock(&meta_rw_lock_);
   return result;
 }
 
 void RmtDataCacheCsm::GetRegBrokers(list<NodeInfo>& brokers) {
-  map<NodeInfo, set<string> >::iterator it;  
+  map<NodeInfo, set<string> >::iterator it;
 
   brokers.clear();
   pthread_rwlock_rdlock(&meta_rw_lock_);
@@ -270,6 +308,28 @@ void RmtDataCacheCsm::GetRegBrokers(list<NodeInfo>& brokers) {
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
+void RmtDataCacheCsm::GetPartitionByBroker(const NodeInfo& broker_info,
+                                            list<PartitionExt>& partition_list) {
+  set<string>::iterator it_key;
+  map<NodeInfo, set<string> >::iterator it_broker;
+  map<string, PartitionExt>::iterator it_part;
+  
+  partition_list.clear();
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  it_broker = broker_partition_.find(broker_info);
+  if (it_broker != broker_partition_.end()) {
+    for (it_key = it_broker->second.begin();
+    it_key != it_broker->second.end(); it_key++) {
+      it_part = partitions_.find(*it_key);
+      if (it_part != partitions_.end()) {
+        partition_list.push_back(it_part->second);
+      }
+    }
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+
 void RmtDataCacheCsm::GetCurPartitionOffsets(map<string, int64_t> part_offset_map) {
   map<string, int64_t>::iterator it;
 
@@ -290,7 +350,7 @@ bool RmtDataCacheCsm::RemovePartition(string &err_info,
   map<string, PartitionExt>::iterator it_part;
   map<string, set<string> >::iterator it_topic;
   map<NodeInfo, set<string> >::iterator it_broker;
-  // parse confirm context  
+  // parse confirm context
   bool result = parseConfirmContext(err_info,
                       confirm_context, partition_key, booked_time);
   if (!result) {
@@ -310,7 +370,9 @@ void RmtDataCacheCsm::RemovePartition(const list<PartitionExt>& partition_list)
   for (it_lst = partition_list.begin(); it_lst != partition_list.end(); it_lst++) {
     partition_keys.insert(it_lst->GetPartitionKey());
   }
-  RemovePartition(partition_keys);
+  if (!partition_keys.empty()) {
+    RemovePartition(partition_keys);
+  }
 }
 
 void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
@@ -331,6 +393,53 @@ void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
+void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
+        bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts) {
+  //
+  string part_key;
+  list<SubscribeInfo>::const_iterator it;
+  map<string, PartitionExt>::iterator it_part;
+  map<NodeInfo, list<PartitionExt> >::iterator it_broker;
+
+  broker_parts.clear();
+  // check if empty
+  if (subscribe_infos.empty()) {
+    return;
+  }
+  pthread_rwlock_wrlock(&meta_rw_lock_);
+  pthread_mutex_lock(&part_mutex_);
+  for (it = subscribe_infos.begin(); it != subscribe_infos.end(); ++it) {
+    part_key = it->GetPartitionExt().GetPartitionKey();
+    it_part = partitions_.find(part_key);
+    if (it_part != partitions_.end()) {
+      if (partition_useds_.find(part_key) != partition_useds_.end()) {
+        if (is_processing_rollback) {
+          it_part->second.SetLastConsumed(false);
+        } else {
+          it_part->second.SetLastConsumed(true);
+        }
+      }
+      it_broker = broker_parts.find(it_part->second.GetBrokerInfo());
+      if (it_broker == broker_parts.end()) {
+        list<PartitionExt> tmp_part_list;
+        tmp_part_list.push_back(it_part->second);
+        broker_parts[it_part->second.GetBrokerInfo()] = tmp_part_list;
+      } else {
+        it_broker->second.push_back(it_part->second);
+      }
+      rmvMetaInfo(part_key);  
+    }
+    partition_useds_.erase(part_key);
+    index_partitions_.remove(part_key);
+    // todo need modify if timer build finished
+    partition_timeouts_.erase(part_key);
+    // end todo
+  }
+  pthread_mutex_unlock(&part_mutex_);
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+
 
 bool RmtDataCacheCsm::BookPartition(const string& partition_key) {
   bool result = false;
@@ -397,8 +506,8 @@ bool RmtDataCacheCsm::parseConfirmContext(string &err_info,
      const string& confirm_context, string& partition_key, int64_t& booked_time) {
   //
   vector<string> result;
-  Utils::Split(confirm_context, result, delimiter::kDelimiterAt); 
-  if(result.empty()) {
+  Utils::Split(confirm_context, result, delimiter::kDelimiterAt);
+  if (result.empty()) {
     err_info = "Illegel confirmContext content: unregular value format!";
     return false;
   }
@@ -411,7 +520,7 @@ bool RmtDataCacheCsm::parseConfirmContext(string &err_info,
 void RmtDataCacheCsm::rmvMetaInfo(const string& partition_key) {
   map<string, PartitionExt>::iterator it_part;
   map<string, set<string> >::iterator it_topic;
-  map<NodeInfo, set<string> >::iterator it_broker;  
+  map<NodeInfo, set<string> >::iterator it_broker;
   it_part = partitions_.find(partition_key);
   if (it_part != partitions_.end()) {
     it_topic = topic_partition_.find(it_part->second.GetTopic());
@@ -472,7 +581,7 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
         if (need_delay_check) {
           wait_time = it_part->second.ProcConsumeResult(def_flowctrl_handler_,
                         group_flowctrl_handler_, filter_consume, is_consumed);
-        } 
+        }
         if (wait_time >= 10) {
           // todo add timer 
           // end todo
@@ -480,7 +589,7 @@ bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
           index_partitions_.push_back(partition_key);
         }
         err_info = "Ok";
-        result = true;    
+        result = true;
       } else {
         // partiton is used by other thread
         err_info = "Illegel confirmContext content: context not equal!";


[incubator-tubemq] 08/50: [TUBEMQ-262]Create C++ flow control handler

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 12f841256db2bd606d253638b7cd5295c1c0016d
Author: gosonzhang <go...@tencent.com>
AuthorDate: Thu Jul 2 16:56:38 2020 +0800

    [TUBEMQ-262]Create C++ flow control handler
---
 .../tubemq-client-cpp/inc/atomic_def.h             | 277 +++++++++++++++++++++
 .../tubemq-client-cpp/inc/const_config.h           |   9 +
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  |   1 +
 .../tubemq-client-cpp/src/client_config.cc         |  47 ++--
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |   6 +
 5 files changed, 317 insertions(+), 23 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
new file mode 100644
index 0000000..24df5f3
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.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 _TUBEMQ_CLIENT_ATOMIC_DEF_H_
+#define _TUBEMQ_CLIENT_ATOMIC_DEF_H_
+
+#include <stdlib.h>
+
+
+
+namespace tubemq {
+
+using namespace std;
+
+class AtomicInteger {
+ public:
+  AtomicInteger(){
+    this->counter_ = 0;
+  }
+            
+  AtomicInteger(int initial_value) {
+    this->counter_ = initial_value;
+  }
+
+  int Get() {
+    return this->counter_;
+  }
+
+  void Set(long new_value) {
+    this->counter_ = new_value;
+  }
+
+  long LongValue() {
+    return (long)this->counter_;
+  }
+
+  int GetAndSet(int new_value) {
+    for ( ; ; ) {
+      int current = this->counter_;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
+        return current;
+      }
+    }
+  }
+
+  bool CompareAndSet(int expect, int update) {
+    return __sync_bool_compare_and_swap(&this->counter_, expect, update);
+  }
+
+  int GetAndIncrement() {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current + 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }  
+    }
+  }
+
+  int GetAndDecrement() {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current - 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }  
+    }
+  }
+
+  int GetAndAdd(int delta) {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current + delta;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }  
+    }
+  }
+
+  int IncrementAndGet() {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current + 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+
+  int DecrementAndGet() {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current - 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+
+  int AddAndGet(int delta) {
+    for ( ; ; ) {
+      int current = this->counter_;
+      int next = current + delta;
+      if (__sync_bool_compare_and_swap (&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+            
+ private:
+  volatile int counter_;
+};
+
+
+class AtomicLong {
+ public:
+  AtomicLong() {
+    this->counter_ = 0;
+  }
+
+  AtomicLong(long initial_value) {
+    this->counter_ = initial_value;
+  }
+
+  long Get() {
+    return this->counter_;
+  }
+
+  void Set(long new_value) {
+    this->counter_ = new_value;
+  }
+
+  long IntValue() {
+    return (int)this->counter_;
+  }
+
+  long GetAndSet(long new_value) {
+    for ( ; ; ) {
+      long current = this->counter_;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
+        return current;
+      }
+    }
+  }
+
+  bool CompareAndSet(long expect, long update) {
+    return __sync_bool_compare_and_swap(&this->counter_, expect, update);
+  }
+
+  long GetAndIncrement() {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current + 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }
+    }
+  }
+
+  long GetAndDecrement() {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current - 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }
+    }
+  }
+
+  long GetAndAdd(long delta) {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current + delta;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return current;
+      }
+    }
+  }
+
+  long IncrementAndGet() {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current + 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+
+  long DecrementAndGet() {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current - 1;
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+
+  long AddAndGet(long delta) {
+    for ( ; ; ) {
+      long current = this->counter_;
+      long next = current + delta;
+      if (__sync_bool_compare_and_swap (&this->counter_, current, next)) {
+        return next;
+      }
+    }
+  }
+
+ private:
+  volatile long counter_;
+};
+
+
+class AtomicBoolean{
+ public:
+  AtomicBoolean() {
+    this->counter_ = 0;
+  }
+
+  AtomicBoolean(bool initial_value) {
+    this->counter_ = initial_value ? 1 : 0;
+  }
+
+  bool Get() {
+    return this->counter_ != 0;
+  }
+
+  void Set(bool new_value) {
+    this->counter_ = new_value ? 1 : 0;
+  }
+
+  bool GetAndSet(bool new_value) {
+    int u = new_value ? 1 : 0;
+    for (;;) {
+      int e = this->counter_ ? 1 : 0;
+      if (__sync_bool_compare_and_swap(&this->counter_, e, u)) {
+        return e != 0;
+      }
+    }
+  }
+
+  bool CompareAndSet(bool expect, bool update) {
+    int e = expect ? 1 : 0;
+    int u = update ? 1 : 0;
+    return __sync_bool_compare_and_swap(&this->counter_, e, u);
+  }
+
+ private:
+  volatile int counter_;
+};
+
+
+
+
+
+}
+
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index 1f2e655..5ec1709 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -63,6 +63,15 @@ static const int kConfirmWaitPeriodMsMax = 60000;
 // default rebalance wait if shutdown meeting
 static const int kRebWaitPeriodWhenShutdownMs = 10000;
 
+// max int value
+static const int kMaxIntValue = 0x7fffffff;
+// max long value
+static const int kMaxLongValue = 0x7fffffffffffffffL;
+
+// invalid value
+static const int kInvalidValue = -2;
+
+
 }  // namespace config
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index 9326449..de39683 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -47,6 +47,7 @@ class Utils {
                    const string& group_name, string& tgt_group_name);
   static bool ValidFilterItem(string& err_info, 
                    const string& src_filteritem, string& tgt_filteritem);
+  static long GetCurrentTimeMillis();
 
 };
  
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index 84d6721..f7fde92 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -344,26 +344,27 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     return false;
   }
   map<string, set<string> > tmp_sub_map;
-  map<string, set<string> >::const_iterator itMap;
-  for (itMap = subscribed_topic_and_filter_map.begin(); itMap != subscribed_topic_and_filter_map.end(); ++itMap) {
+  map<string, set<string> >::const_iterator it_map;
+  for (it_map = subscribed_topic_and_filter_map.begin(); 
+    it_map != subscribed_topic_and_filter_map.end(); ++it_map) {
     int count=0;
     string tmp_filteritem;
     set<string> tgt_filters;
     // check topic_name info
-    is_success = Utils::ValidString(err_info, itMap->first, 
+    is_success = Utils::ValidString(err_info, it_map->first, 
                          false, true, true, config::kTopicNameMaxLength);  
     if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
-      ss << itMap->first;
+      ss << it_map->first;
       ss << " ";
       ss << err_info;
       err_info = ss.str();
       return false;
     }
-    string topic_name = Utils::Trim(itMap->first);
+    string topic_name = Utils::Trim(it_map->first);
     // check filter info
-    set<string> subscribed_filters = itMap->second;
+    set<string> subscribed_filters = it_map->second;
     for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end(); ++it) {
       is_success = Utils::ValidFilterItem(err_info, *it, tmp_filteritem);
       if (!is_success) {
@@ -381,7 +382,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     if (count > config::kFilterItemMaxCount) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
-      ss << itMap->first;
+      ss << it_map->first;
       ss << "'s filter item over max item count : ";
       ss << config::kFilterItemMaxCount;
       err_info = ss.str();
@@ -419,14 +420,14 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   // check part_offset_map
   string part_key;
   map<string, long> tmp_parts_map;
-  map<string, long>::const_iterator itPart;
-  for (itPart = part_offset_map.begin(); itPart != part_offset_map.end(); ++itPart) {
+  map<string, long>::const_iterator it_part;
+  for (itPart = part_offset_map.begin(); it_part != part_offset_map.end(); ++it_part) {
     vector<string> result;
-    Utils::Split(itPart->first, result, delimiter::kDelimiterColon);
+    Utils::Split(it_part->first, result, delimiter::kDelimiterColon);
     if (result.size() != 3) {
       stringstream ss;
       ss << "Illegal parameter: part_offset_map's key ";
-      ss << itPart->first;
+      ss << it_part->first;
       ss << " format error, value must be aaaa:bbbb:cccc !";
       err_info = ss.str();
       return false;
@@ -434,34 +435,34 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     if (tmp_sub_map.find(result[1]) != tmp_sub_map.end()) {
       stringstream ss;
       ss << "Illegal parameter: ";
-      ss << itPart->first;
+      ss << it_part->first;
       ss << " subscribed topic ";
       ss << result[1];
       ss << " not included in subscribed_topic_and_filter_map's topic list!";
       err_info = ss.str();
       return false;
     }
-    if (itPart->first.find_first_of(delimiter::kDelimiterComma) != string::npos) {
+    if (it_part->first.find_first_of(delimiter::kDelimiterComma) != string::npos) {
       stringstream ss;
       ss << "Illegal parameter: key ";
-      ss << itPart->first;
+      ss << it_part->first;
       ss << " include ";
       ss << delimiter::kDelimiterComma;
       ss << " char!";
       err_info = ss.str();
       return false;
     }
-    if (itPart->second < 0) {
+    if (it_part->second < 0) {
       stringstream ss;
       ss << "Illegal parameter: ";
-      ss << itPart->first;
+      ss << it_part->first;
       ss << "'s offset must over or equal zero, value is ";
-      ss << itPart->second;
+      ss << it_part->second;
       err_info = ss.str();
       return false;
     }
     Utils::Join(result, delimiter::kDelimiterColon, part_key);
-    tmp_parts_map[part_key] = itPart->second;
+    tmp_parts_map[part_key] = it_part->second;
   }
   // set verified data
   this->is_bound_consume_ = true;
@@ -551,7 +552,7 @@ string ConsumerConfig::ToString() {
   int i = 0;
   stringstream ss;
   map<string, long>::iterator it;
-  map<string, set<string> >::iterator itMap;
+  map<string, set<string> >::iterator it_map;
 
   // print info
   ss << "ConsumerConfig = {";
@@ -559,8 +560,8 @@ string ConsumerConfig::ToString() {
   ss << ", group_name_='";
   ss << this->group_name_;
   ss << "', sub_topic_and_filter_map_={";
-  for (itMap = this->sub_topic_and_filter_map_.begin(); 
-      itMap != this->sub_topic_and_filter_map_.end(); ++itMap) {
+  for (it_map = this->sub_topic_and_filter_map_.begin(); 
+      it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -568,8 +569,8 @@ string ConsumerConfig::ToString() {
     ss << itMap->first;
     ss << "'=[";
     int j=0;
-    set<string> topicSet = itMap->second;
-    for (set<string>::iterator it = topicSet.begin(); it != topicSet.end(); ++it) {
+    set<string> topic_set = it_map->second;
+    for (set<string>::iterator it = topic_set.begin(); it != topic_set.end(); ++it) {
       if (j++ > 0) {
         ss << ",";
       }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index f9a1b1e..f8ee3c2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -237,6 +237,12 @@ bool Utils::ValidFilterItem(string& err_info,
 }
 
 
+long Utils::GetCurrentTimeMillis() {
+  struct timeval tv;
+  gettimeofday(&tv,NULL);
+  return tv.tv_sec * 1000 + tv.tv_usec /1000;
+}
+
 
 
 }


[incubator-tubemq] 12/50: [TUBEMQ-262]Create C++ flow control handler

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit d171c375b2d21ea238af087ba5e77b3e830201ec
Author: gosonzhang <go...@tencent.com>
AuthorDate: Fri Jul 3 09:51:28 2020 +0800

    [TUBEMQ-262]Create C++ flow control handler
---
 .gitmodules => tubemq-client-twins/tubemq-client-cpp/.gitmodules | 0
 1 file changed, 0 insertions(+), 0 deletions(-)

diff --git a/.gitmodules b/tubemq-client-twins/tubemq-client-cpp/.gitmodules
similarity index 100%
rename from .gitmodules
rename to tubemq-client-twins/tubemq-client-cpp/.gitmodules


[incubator-tubemq] 27/50: [TUBEMQ-269] Create C/C++ RmtDataCache class (#198)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 6a42851b41fd18cb0dc65cf34a19177e71f3c011
Author: gosonzhang <46...@qq.com>
AuthorDate: Fri Jul 10 01:01:47 2020 +0000

    [TUBEMQ-269] Create C/C++ RmtDataCache class (#198)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/atomic_def.h  |   1 -
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |  86 +++----
 .../include/tubemq/rmt_data_cache.h                |  62 ++++-
 .../tubemq-client-cpp/src/meta_info.cc             | 274 ++++++++++-----------
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 132 ++++++++++
 5 files changed, 369 insertions(+), 186 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
index c8030db..30830d9 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
@@ -24,7 +24,6 @@
 
 namespace tubemq {
 
-using namespace std;
 
 class AtomicInteger {
  public:
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index 02d5a63..813a9c5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -91,14 +91,54 @@ class Partition {
   string partition_info_;
 };
 
+class PartitionExt : public Partition {
+ public:
+  PartitionExt();
+  PartitionExt(const string& partition_info);
+  PartitionExt(const NodeInfo& broker_info, const string& part_str);
+  ~PartitionExt();
+  void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
+    int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
+    int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
+    int64_t last_datadlt, bool require_slow);
+  void SetLastConsumed(bool last_consumed);
+  bool IsLastConsumed();
+
+ private:
+  void resetParameters();
+  void updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt);
+
+ private:
+  bool is_last_consumed_;
+  FlowCtrlResult cur_flowctrl_;
+  FlowCtrlItem cur_freqctrl_;
+  int64_t next_stage_updtime_;
+  int64_t next_slice_updtime_;
+  int64_t limit_slice_msgsize_;
+  int64_t cur_stage_msgsize_;
+  int64_t cur_slice_msgsize_;
+  int32_t total_zero_cnt_;
+  int64_t booked_time_;
+  int32_t booked_errcode_;
+  bool    booked_esc_limit_;
+  int32_t booked_msgsize_;
+  int64_t booked_dlt_limit_;
+  int64_t booked_curdata_dlt_;
+  bool    booked_require_slow_;
+};
+
 class SubscribeInfo {
  public:
   SubscribeInfo(const string& sub_info);
-  SubscribeInfo(const string& consumer_id, const string& group, const Partition& partition);
   SubscribeInfo& operator=(const SubscribeInfo& target);
   const string& GetConsumerId() const;
   const string& GetGroup() const;
-  const Partition& GetPartition() const;
+  const PartitionExt& GetPartitionExt() const;
   const uint32_t GgetBrokerId() const;
   const string& GetBrokerHost() const;
   const uint32_t GetBrokerPort() const;
@@ -112,7 +152,7 @@ class SubscribeInfo {
  private:
   string consumer_id_;
   string group_;
-  Partition partition_;
+  PartitionExt partitionext_;
   string sub_info_;
 };
 
@@ -138,46 +178,6 @@ class ConsumerEvent {
   list<SubscribeInfo> subscribe_list_;
 };
 
-class PartitionExt : public Partition {
- public:
-  PartitionExt();
-  PartitionExt(const string& partition_info);
-  PartitionExt(const NodeInfo& broker_info, const string& part_str);
-  ~PartitionExt();
-  void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
-    int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
-  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
-  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
-    int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
-    int64_t last_datadlt, bool require_slow);
-  void SetLastConsumed(bool last_consumed);
-  bool IsLastConsumed();
-
- private:
-  void resetParameters();
-  void updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt);
-
- private:
-  bool is_last_consumed_;
-  FlowCtrlResult cur_flowctrl_;
-  FlowCtrlItem cur_freqctrl_;
-  int64_t next_stage_updtime_;
-  int64_t next_slice_updtime_;
-  int64_t limit_slice_msgsize_;
-  int64_t cur_stage_msgsize_;
-  int64_t cur_slice_msgsize_;
-  int32_t total_zero_cnt_;
-  int64_t booked_time_;
-  int32_t booked_errcode_;
-  bool    booked_esc_limit_;
-  int32_t booked_msgsize_;
-  int64_t booked_dlt_limit_;
-  int64_t booked_curdata_dlt_;
-  bool    booked_require_slow_;
-};
 
 }  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index d97cbac..11f9018 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -21,15 +21,73 @@
 #define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
 
 #include <stdint.h>
+#include <pthread.h>
 
 #include <atomic>
+#include <list>
+#include <map>
+#include <set>
+
+#include "tubemq/flowctrl_def.h"
+#include "tubemq/meta_info.h"
+
+
+
 
 namespace tubemq {
 
-using namespace std;
+using std::map;
+using std::set;
+using std::list;
+
 
 // consumer remote data cache
-class RmtDataCacheCsm {}
+class RmtDataCacheCsm {
+ public:
+  RmtDataCacheCsm();
+  ~RmtDataCacheCsm();
+  void AddNewPartition(const PartitionExt& partition_ext);
+  void OfferEvent(const ConsumerEvent& event);
+  void TakeEvent(ConsumerEvent& event);
+  void ClearEvent();
+  void OfferEventResult(const ConsumerEvent& event);
+  bool PollEventResult(ConsumerEvent& event);
+
+
+ private:
+  // timer begin
+
+  // timer end
+  // flow ctrl
+  FlowCtrlRuleHandler group_flowctrl_handler_;
+  FlowCtrlRuleHandler def_flowctrl_handler_;
+  pthread_rwlock_t meta_rw_lock_;
+  // partiton allocated map
+  map<string, PartitionExt> partitions_;
+  // topic partiton map
+  map<string, set<string> > topic_partition_;
+  // broker parition map
+  map<NodeInfo, set<string> > broker_partition_;
+  // for partiton idle map
+  list<string> index_partitions_;
+  // for partition used map
+  map<string, int64_t> partition_useds_;
+  // for partiton timer map
+  map<string, int64_t> partition_timeouts_;
+  // data book
+  pthread_mutex_t data_book_mutex_;
+  // for partition offset cache
+  map<string, int64_t> partition_offset_;
+  // for partiton register booked
+  map<string, bool> part_reg_booked_;
+  // event
+  pthread_mutex_t  event_read_mutex_;
+  pthread_cond_t   event_read_cond_;
+  list<ConsumerEvent> rebalance_events_;
+  pthread_mutex_t  event_write_mutex_;
+  list<ConsumerEvent> rebalance_results_;
+};
+
 
 }  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 5c4e66b..4f0f860 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -244,145 +244,6 @@ void Partition::buildPartitionKey() {
   this->partition_info_ = ss2.str();
 }
 
-// sub_info = consumerId@group#broker_info#topic:partitionId
-SubscribeInfo::SubscribeInfo(const string& sub_info) {
-  string::size_type pos = 0;
-  string seg_key = delimiter::kDelimiterPound;
-  string at_key = delimiter::kDelimiterAt;
-  this->consumer_id_ = " ";
-  this->group_ = " ";
-  // parse sub_info
-  pos = sub_info.find(seg_key);
-  if (pos != string::npos) {
-    string consumer_info = sub_info.substr(0, pos);
-    consumer_info = Utils::Trim(consumer_info);
-    string partition_info = sub_info.substr(pos + seg_key.size(), sub_info.size());
-    partition_info = Utils::Trim(partition_info);
-    this->partition_ = Partition(partition_info);
-    pos = consumer_info.find(at_key);
-    this->consumer_id_ = consumer_info.substr(0, pos);
-    this->consumer_id_ = Utils::Trim(this->consumer_id_);
-    this->group_ = consumer_info.substr(pos + at_key.size(), consumer_info.size());
-    this->group_ = Utils::Trim(this->group_);
-  }
-  buildSubInfo();
-}
-
-SubscribeInfo::SubscribeInfo(const string& consumer_id, const string& group,
-                             const Partition& partition) {
-  this->consumer_id_ = consumer_id;
-  this->group_ = group;
-  this->partition_ = partition;
-  buildSubInfo();
-}
-
-SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
-  if (this != &target) {
-    this->consumer_id_ = target.consumer_id_;
-    this->group_ = target.group_;
-    this->partition_ = target.partition_;
-  }
-  return *this;
-}
-
-const string& SubscribeInfo::GetConsumerId() const { return this->consumer_id_; }
-
-const string& SubscribeInfo::GetGroup() const { return this->group_; }
-
-const Partition& SubscribeInfo::GetPartition() const { return this->partition_; }
-
-const uint32_t SubscribeInfo::GgetBrokerId() const { return this->partition_.GetBrokerId(); }
-
-const string& SubscribeInfo::GetBrokerHost() const { return this->partition_.GetBrokerHost(); }
-
-const uint32_t SubscribeInfo::GetBrokerPort() const { return this->partition_.GetBrokerPort(); }
-
-const string& SubscribeInfo::GetTopic() const { return this->partition_.GetTopic(); }
-
-const uint32_t SubscribeInfo::GetPartitionId() const { return this->partition_.GetPartitionId(); }
-
-const string& SubscribeInfo::ToString() const { return this->sub_info_; }
-
-void SubscribeInfo::buildSubInfo() {
-  stringstream ss;
-  ss << this->consumer_id_;
-  ss << delimiter::kDelimiterAt;
-  ss << this->group_;
-  ss << delimiter::kDelimiterPound;
-  ss << this->partition_.ToString();
-  this->sub_info_ = ss.str();
-}
-
-ConsumerEvent::ConsumerEvent() {
-  this->rebalance_id_ = tb_config::kInvalidValue;
-  this->event_type_ = tb_config::kInvalidValue;
-  this->event_status_ = tb_config::kInvalidValue;
-}
-
-ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
-  this->rebalance_id_ = target.rebalance_id_;
-  this->event_type_ = target.event_type_;
-  this->event_status_ = target.event_status_;
-  this->subscribe_list_ = target.subscribe_list_;
-}
-
-ConsumerEvent::ConsumerEvent(int64_t rebalance_id, int32_t event_type,
-                             const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
-  list<SubscribeInfo>::const_iterator it;
-  this->rebalance_id_ = rebalance_id;
-  this->event_type_ = event_type;
-  this->event_status_ = event_status;
-  for (it = subscribeInfo_lst.begin(); it != subscribeInfo_lst.end(); ++it) {
-    this->subscribe_list_.push_back(*it);
-  }
-}
-
-ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
-  if (this != &target) {
-    this->rebalance_id_ = target.rebalance_id_;
-    this->event_type_ = target.event_type_;
-    this->event_status_ = target.event_status_;
-    this->subscribe_list_ = target.subscribe_list_;
-  }
-  return *this;
-}
-
-const int64_t ConsumerEvent::GetRebalanceId() const { return this->rebalance_id_; }
-
-const int32_t ConsumerEvent::GetEventType() const { return this->event_type_; }
-
-const int32_t ConsumerEvent::GetEventStatus() const { return this->event_status_; }
-
-void ConsumerEvent::SetEventType(int32_t event_type) { this->event_type_ = event_type; }
-
-void ConsumerEvent::SetEventStatus(int32_t event_status) { this->event_status_ = event_status; }
-
-const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
-  return this->subscribe_list_;
-}
-
-string ConsumerEvent::ToString() {
-  uint32_t count = 0;
-  stringstream ss;
-  list<SubscribeInfo>::const_iterator it;
-  ss << "ConsumerEvent [rebalanceId=";
-  ss << this->rebalance_id_;
-  ss << ", type=";
-  ss << this->event_type_;
-  ss << ", status=";
-  ss << this->event_status_;
-  ss << ", subscribeInfoList=[";
-  for (it = this->subscribe_list_.begin(); it != this->subscribe_list_.end(); ++it) {
-    if (count++ > 0) {
-      ss << ",";
-    }
-    ss << it->ToString();
-  }
-  ss << "]]";
-  return ss.str();
-}
-
-
 PartitionExt::PartitionExt() : Partition() {
   resetParameters();
 }
@@ -505,7 +366,7 @@ void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_ha
   bool result = false;
   // Accumulated data received
   this->cur_stage_msgsize_ += msg_size;
-  this->cur_slice_msgsize_ += msg_size;  
+  this->cur_slice_msgsize_ += msg_size;
   int64_t curr_time = Utils::GetCurrentTimeMillis();
   // Update strategy data values
   if (curr_time > this->next_stage_updtime_) {
@@ -535,4 +396,137 @@ void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_ha
 }
 
 
+// sub_info = consumerId@group#broker_info#topic:partitionId
+SubscribeInfo::SubscribeInfo(const string& sub_info) {
+  string::size_type pos = 0;
+  string seg_key = delimiter::kDelimiterPound;
+  string at_key = delimiter::kDelimiterAt;
+  this->consumer_id_ = " ";
+  this->group_ = " ";
+  // parse sub_info
+  pos = sub_info.find(seg_key);
+  if (pos != string::npos) {
+    string consumer_info = sub_info.substr(0, pos);
+    consumer_info = Utils::Trim(consumer_info);
+    string partition_info = sub_info.substr(pos + seg_key.size(), sub_info.size());
+    partition_info = Utils::Trim(partition_info);
+    this->partitionext_ = PartitionExt(partition_info);
+    pos = consumer_info.find(at_key);
+    this->consumer_id_ = consumer_info.substr(0, pos);
+    this->consumer_id_ = Utils::Trim(this->consumer_id_);
+    this->group_ = consumer_info.substr(pos + at_key.size(), consumer_info.size());
+    this->group_ = Utils::Trim(this->group_);
+  }
+  buildSubInfo();
+}
+
+SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
+  if (this != &target) {
+    this->consumer_id_ = target.consumer_id_;
+    this->group_ = target.group_;
+    this->partitionext_ = target.partitionext_;
+  }
+  return *this;
+}
+
+const string& SubscribeInfo::GetConsumerId() const { return this->consumer_id_; }
+
+const string& SubscribeInfo::GetGroup() const { return this->group_; }
+
+const PartitionExt& SubscribeInfo::GetPartitionExt() const { return this->partitionext_; }
+
+const uint32_t SubscribeInfo::GgetBrokerId() const { return this->partitionext_.GetBrokerId(); }
+
+const string& SubscribeInfo::GetBrokerHost() const { return this->partitionext_.GetBrokerHost(); }
+
+const uint32_t SubscribeInfo::GetBrokerPort() const { return this->partitionext_.GetBrokerPort(); }
+
+const string& SubscribeInfo::GetTopic() const { return this->partitionext_.GetTopic(); }
+
+const uint32_t SubscribeInfo::GetPartitionId() const { return this->partitionext_.GetPartitionId(); }
+
+const string& SubscribeInfo::ToString() const { return this->sub_info_; }
+
+void SubscribeInfo::buildSubInfo() {
+  stringstream ss;
+  ss << this->consumer_id_;
+  ss << delimiter::kDelimiterAt;
+  ss << this->group_;
+  ss << delimiter::kDelimiterPound;
+  ss << this->partitionext_.ToString();
+  this->sub_info_ = ss.str();
+}
+
+ConsumerEvent::ConsumerEvent() {
+  this->rebalance_id_ = tb_config::kInvalidValue;
+  this->event_type_ = tb_config::kInvalidValue;
+  this->event_status_ = tb_config::kInvalidValue;
+}
+
+ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
+  this->rebalance_id_ = target.rebalance_id_;
+  this->event_type_ = target.event_type_;
+  this->event_status_ = target.event_status_;
+  this->subscribe_list_ = target.subscribe_list_;
+}
+
+ConsumerEvent::ConsumerEvent(int64_t rebalance_id, int32_t event_type,
+                             const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
+  list<SubscribeInfo>::const_iterator it;
+  this->rebalance_id_ = rebalance_id;
+  this->event_type_ = event_type;
+  this->event_status_ = event_status;
+  for (it = subscribeInfo_lst.begin(); it != subscribeInfo_lst.end(); ++it) {
+    this->subscribe_list_.push_back(*it);
+  }
+}
+
+ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
+  if (this != &target) {
+    this->rebalance_id_ = target.rebalance_id_;
+    this->event_type_ = target.event_type_;
+    this->event_status_ = target.event_status_;
+    this->subscribe_list_ = target.subscribe_list_;
+  }
+  return *this;
+}
+
+const int64_t ConsumerEvent::GetRebalanceId() const { return this->rebalance_id_; }
+
+const int32_t ConsumerEvent::GetEventType() const { return this->event_type_; }
+
+const int32_t ConsumerEvent::GetEventStatus() const { return this->event_status_; }
+
+void ConsumerEvent::SetEventType(int32_t event_type) { this->event_type_ = event_type; }
+
+void ConsumerEvent::SetEventStatus(int32_t event_status) { this->event_status_ = event_status; }
+
+const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
+  return this->subscribe_list_;
+}
+
+string ConsumerEvent::ToString() {
+  uint32_t count = 0;
+  stringstream ss;
+  list<SubscribeInfo>::const_iterator it;
+  ss << "ConsumerEvent [rebalanceId=";
+  ss << this->rebalance_id_;
+  ss << ", type=";
+  ss << this->event_type_;
+  ss << ", status=";
+  ss << this->event_status_;
+  ss << ", subscribeInfoList=[";
+  for (it = this->subscribe_list_.begin(); it != this->subscribe_list_.end(); ++it) {
+    if (count++ > 0) {
+      ss << ",";
+    }
+    ss << it->ToString();
+  }
+  ss << "]]";
+  return ss.str();
+}
+
+
+
+
 };  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
new file mode 100644
index 0000000..d239d26
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -0,0 +1,132 @@
+/**
+ * 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 "tubemq/rmt_data_cache.h"
+#include "tubemq/meta_info.h"
+
+
+
+namespace tubemq {
+ 
+
+RmtDataCacheCsm::RmtDataCacheCsm() {
+  pthread_rwlock_init(&meta_rw_lock_, NULL);
+  pthread_mutex_init(&data_book_mutex_, NULL);
+  pthread_mutex_init(&event_read_mutex_, NULL);
+  pthread_cond_init(&event_read_cond_, NULL);
+  pthread_mutex_init(&event_write_mutex_, NULL);
+}
+
+RmtDataCacheCsm::~RmtDataCacheCsm() {
+  pthread_mutex_destroy(&event_write_mutex_);
+  pthread_mutex_destroy(&event_read_mutex_);
+  pthread_mutex_destroy(&data_book_mutex_);
+  pthread_cond_destroy(&event_read_cond_);
+  pthread_rwlock_destroy(&meta_rw_lock_);
+}
+
+void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
+  //
+  map<string, PartitionExt>::iterator it_map;
+  map<string, set<string> >::iterator it_topic;
+  map<NodeInfo, set<string> >::iterator it_broker;
+  //
+  string partition_key = partition_ext.GetPartitionKey();
+  pthread_rwlock_wrlock(&meta_rw_lock_);
+  it_map = partitions_.find(partition_key);
+  if (it_map == partitions_.end()) {
+    partitions_[partition_key] = partition_ext;
+    it_topic = topic_partition_.find(partition_ext.GetTopic());
+    if (it_topic == topic_partition_.end()) {
+      set<string> tmp_part_set;
+      tmp_part_set.insert(partition_key);
+      topic_partition_[partition_ext.GetTopic()] = tmp_part_set;
+    } else {
+      if (it_topic->second.find(partition_key) == it_topic->second.end()) {
+        it_topic->second.insert(partition_key);
+      }
+    }
+    it_broker = broker_partition_.find(partition_ext.GetBrokerInfo());
+    if (it_broker == broker_partition_.end()) {
+      set<string> tmp_part_set;
+      tmp_part_set.insert(partition_key);
+      broker_partition_[partition_ext.GetBrokerInfo()] = tmp_part_set;
+    } else {
+      if (it_broker->second.find(partition_key) == it_broker->second.end()) {
+        it_broker->second.insert(partition_key);
+      }
+    }
+  }
+  // check partition_key status
+  if (partition_useds_.find(partition_key) == partition_useds_.end() 
+    && partition_timeouts_.find(partition_key) == partition_timeouts_.end()) {
+    index_partitions_.remove(partition_key);
+    index_partitions_.push_back(partition_key);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+void RmtDataCacheCsm::OfferEvent(const ConsumerEvent& event) {
+  pthread_mutex_lock(&event_read_mutex_);
+  this->rebalance_events_.push_back(event);
+  pthread_cond_broadcast(&event_read_cond_);
+  pthread_mutex_unlock(&event_read_mutex_);
+}
+
+void RmtDataCacheCsm::TakeEvent(ConsumerEvent& event) {
+  pthread_mutex_lock(&event_read_mutex_);
+  while (this->rebalance_events_.empty()) {
+    pthread_cond_wait(&event_read_cond_, &event_read_mutex_);
+  }
+  event = rebalance_events_.front();
+  rebalance_events_.pop_front();
+  pthread_mutex_unlock(&event_read_mutex_);
+}
+
+void RmtDataCacheCsm::ClearEvent() {
+  pthread_mutex_lock(&event_read_mutex_);
+  rebalance_events_.clear();
+  pthread_mutex_unlock(&event_read_mutex_);
+}
+
+void RmtDataCacheCsm::OfferEventResult(const ConsumerEvent& event) {
+  pthread_mutex_lock(&event_write_mutex_);
+  this->rebalance_events_.push_back(event);
+  pthread_mutex_unlock(&event_write_mutex_);
+}
+
+bool RmtDataCacheCsm::PollEventResult(ConsumerEvent& event) {
+  bool result = false;
+  pthread_mutex_lock(&event_write_mutex_);
+  if (!rebalance_events_.empty()) {
+    event = rebalance_events_.front();
+    rebalance_events_.pop_front();
+    result = true;
+  }
+  pthread_mutex_unlock(&event_write_mutex_);
+  return result;
+}
+
+
+
+
+
+
+
+}  // namespace tubemq


[incubator-tubemq] 46/50: [TUBEMQ-347]C++ SDK Create client API (#258)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 99c9630d5682173f4d0f8c4dcd2ea6752090011c
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Mon Sep 14 17:22:17 2020 +0800

    [TUBEMQ-347]C++ SDK Create client API (#258)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../include/tubemq/client_service.h                |  95 -----------
 .../include/tubemq/client_subinfo.h                |  91 ----------
 .../include/tubemq/const_config.h                  | 103 -----------
 .../tubemq-client-cpp/include/tubemq/const_rpc.h   |  84 ---------
 .../include/tubemq/executor_pool.h                 |  97 -----------
 .../include/tubemq/flowctrl_def.h                  | 144 ----------------
 .../tubemq-client-cpp/include/tubemq/logger.h      | 120 -------------
 .../tubemq-client-cpp/include/tubemq/meta_info.h   | 188 ---------------------
 .../tubemq-client-cpp/include/tubemq/noncopyable.h |  37 ----
 .../include/tubemq/rmt_data_cache.h                | 158 -----------------
 .../tubemq-client-cpp/include/tubemq/singleton.h   |  62 -------
 .../tubemq/{atomic_def.h => tubemq_atomic.h}       |   2 +-
 .../include/tubemq/{file_ini.h => tubemq_client.h} |  42 +++--
 .../include/tubemq/tubemq_config.h                 |  35 ++--
 .../include/tubemq/tubemq_errcode.h                | 110 +++++++++---
 .../include/tubemq/tubemq_message.h                |   3 +-
 .../include/tubemq/tubemq_return.h                 |  36 +++-
 .../include/tubemq/unique_seq_id.h                 |  41 -----
 .../tubemq-client-cpp/include/tubemq/utils.h       |  60 -------
 .../tubemq-client-cpp/include/tubemq/version.h     |  33 ----
 20 files changed, 163 insertions(+), 1378 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
deleted file mode 100644
index ef4ae78..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_BASE_CLIENT_H_
-#define TUBEMQ_CLIENT_BASE_CLIENT_H_
-
-#include <mutex>
-#include <stdint.h>
-#include <string>
-#include <thread>
-
-#include "tubemq/atomic_def.h"
-#include "tubemq/file_ini.h"
-#include "tubemq/rmt_data_cache.h"
-#include "tubemq/singleton.h"
-#include "tubemq/tubemq_message.h"
-#include "tubemq/tubemq_config.h"
-#include "tubemq/tubemq_return.h"
-
-
-
-
-namespace tubemq {
-
-using std::map;
-using std::mutex;
-using std::string;
-using std::thread;
-
-
-class BaseClient {
- public:
-  BaseClient(bool is_producer);
-  virtual ~BaseClient();
-  virtual void ShutDown();
-  void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
-  bool IsProducer() { return is_producer_; }
-  const int32_t GetClientIndex() { return client_index_; }
-
- private:
-  bool  is_producer_;
-  int32_t client_index_;
-};
-
-
-class TubeMQService : public Singleton<TubeMQService> {
- public:
-  // TubeMQService();
-  // ~TubeMQService();
-  bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
-  bool Stop(string& err_info);
-  bool IsRunning();
-  const int32_t  GetServiceStatus() const { return service_status_.Get(); }
-  int32_t GetClientObjCnt();
-  bool AddClientObj(string& err_info, BaseClient* client_obj);
-  BaseClient* GetClientObj(int32_t client_index) const;
-  BaseClient* RmvClientObj(int32_t client_index);
-  const string& GetLocalHost() const { return local_host_; }
-  const ExecutorPool& GetTimerExecutorPool() const { return timer_executor_; }
-  const ExecutorPool& GetNetWorkExecutorPool() const { return network_executor_; }
-
- private:
-  void iniLogger(const Fileini& fileini, const string& sector);
-  void shutDownClinets() const;
-
- private:
-  string local_host_;  
-  AtomicInteger service_status_;
-  AtomicInteger client_index_base_;
-  mutable mutex mutex_;
-  map<int32_t, BaseClient*> clients_map_;
-  ExecutorPool timer_executor_;
-  ExecutorPool network_executor_;
-};
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_BASE_CLIENT_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
deleted file mode 100644
index 14bcbbd..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_SUBINFO_H_
-#define TUBEMQ_CLIENT_SUBINFO_H_
-
-#include <stdint.h>
-#include <list>
-#include <map>
-#include <set>
-#include <string>
-#include "tubemq/atomic_def.h"
-#include "tubemq/tubemq_config.h"
-
-namespace tubemq {
-
-using std::list;
-using std::map;
-using std::set;
-using std::string;
-
-
-class MasterAddrInfo {
- public:
-  MasterAddrInfo();
-  bool InitMasterAddress(string& err_info, const string& master_info);
-  void GetNextMasterAddr(string& ipaddr, int32_t& port);
-  void GetCurrentMasterAddr(string& ipaddr, int32_t& port);
-  int32_t GetTotalMasterAddrCnt() { return master_addr_.size(); } 
-
- private:
-   string curr_master_addr_;
-   map<string, int32_t> master_addr_;
-};
-
-class ClientSubInfo {
- public:
-  ClientSubInfo();
-  void SetConsumeTarget(const ConsumerConfig& config);
-  bool CompAndSetNotAllocated(bool expect, bool update);
-  void BookFstRegistered() { first_registered_.Set(true); }
-  bool IsBoundConsume() const { return bound_consume_; }
-  bool IsNotAllocated() const { return not_allocated_.Get(); }
-  const int64_t GetSubscribedTime() const { return subscribed_time_; }
-  const string& GetSessionKey() const { return session_key_; }
-  const uint32_t GetSourceCnt() const { return source_count_; }
-  bool SelectBig() { return select_big_; }
-  bool IsFilterConsume(const string& topic);
-  void GetAssignedPartOffset(const string& partition_key, int64_t& offset);
-  const string& GetBoundPartInfo() const { return bound_partions_; }
-  const list<string>& GetSubTopics() const { return topics_; }
-  const list<string>& GetTopicConds() const { return topic_conds_; }
-  const map<string, set<string> >& GetTopicFilterMap() const;
-
- private:
-  bool bound_consume_;
-  AtomicBoolean first_registered_;
-  AtomicBoolean not_allocated_;
-  int64_t  subscribed_time_;
-  map<string, set<string> > topic_and_filter_map_;
-  list<string> topics_;
-  list<string> topic_conds_;
-  map<string, bool> topic_filter_map_;
-  // bound info
-  string session_key_;
-  uint32_t source_count_;
-  bool select_big_;
-  map<string, int64_t> assigned_part_map_;
-  string bound_partions_;
-};
-
-}  // namespace tubemq
-
-
-#endif  // TUBEMQ_CLIENT_SUBINFO_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
deleted file mode 100644
index 1cfb962..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_CONST_CONFIG_H_
-#define TUBEMQ_CLIENT_CONST_CONFIG_H_
-
-#include <stdint.h>
-
-#include <map>
-#include <string>
-
-namespace tubemq {
-
-using std::string;
-
-#define TUBEMQ_MAX(a, b) ( ((a)>(b))?(a):(b) )
-#define TUBEMQ_MIN(a, b) ( ((a)>(b))?(b):(a) )
-#define TUBEMQ_MID(data, max, min)   TUBEMQ_MAX(min,TUBEMQ_MIN((max),(data)))
-
-
-// configuration value setting
-namespace tb_config {
-// heartbeat period define
-static const int32_t kHeartBeatPeriodDef = 10;
-static const int32_t kHeartBeatFailRetryTimesDef = 5;
-static const int32_t kHeartBeatSleepPeriodDef = 60;
-// max masterAddrInfo length
-static const int32_t kMasterAddrInfoMaxLength = 1024;
-
-// max TopicName length
-static const int32_t kTopicNameMaxLength = 64;
-// max Consume GroupName length
-static const int32_t kGroupNameMaxLength = 1024;
-// max filter item length
-static const int32_t kFilterItemMaxLength = 256;
-// max allowed filter item count
-static const int32_t kFilterItemMaxCount = 500;
-// max session key length
-static const int32_t kSessionKeyMaxLength = 1024;
-
-// max subscribe info report times
-static const int32_t kSubInfoReportMaxIntervalTimes = 6;
-// default message not found response wait period
-static const int32_t kMsgNotfoundWaitPeriodMsDef = 200;
-// default confirm wait period if rebalance meeting
-static const int32_t kRebConfirmWaitPeriodMsDef = 3000;
-// max confirm wait period anyway
-static const int32_t kConfirmWaitPeriodMsMax = 60000;
-// default rebalance wait if shutdown meeting
-static const int32_t kRebWaitPeriodWhenShutdownMs = 10000;
-
-// max int value
-static const int32_t kMaxIntValue = 0x7fffffff;
-// max long value
-static const int64_t kMaxLongValue = 0x7fffffffffffffffL;
-
-// default broker port
-static const uint32_t kBrokerPortDef = 8123;
-// default broker TLS port
-static const uint32_t kBrokerTlsPortDef = 8124;
-
-// invalid value
-static const int32_t kInvalidValue = -2;
-
-}  // namespace tb_config
-
-namespace delimiter {
-static const char kDelimiterDot[] = ".";
-static const char kDelimiterEqual[] = "=";
-static const char kDelimiterAnd[] = "&";
-static const char kDelimiterComma[] = ",";
-static const char kDelimiterColon[] = ":";
-static const char kDelimiterAt[] = "@";
-static const char kDelimiterPound[] = "#";
-static const char kDelimiterSemicolon[] = ";";
-// Double slash
-static const char kDelimiterDbSlash[] = "//";
-// left square bracket
-static const char kDelimiterLftSB[] = "[";
-// right square bracket
-static const char kDelimiterRgtSB[] = "]";
-
-}  // namespace delimiter
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_CONST_CONFIG_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.h
deleted file mode 100644
index c402477..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.h
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_CONST_RPC_H_
-#define TUBEMQ_CLIENT_CONST_RPC_H_
-
-namespace tubemq {
-
-#include <stdint.h>
-
-namespace rpc_config {
-
-// constant define
-static const uint32_t kRpcPrtBeginToken = 0xFF7FF4FE;
-static const uint32_t kRpcMaxBufferSize = 8192;
-static const uint32_t kRpcMaxFrameListCnt = (uint32_t)((1024 * 1024 * 8) / kRpcMaxBufferSize);
-
-// rpc protocol version
-static const uint32_t kRpcProtocolVersion = 2;
-
-// msg type flag
-static const int32_t kRpcFlagMsgRequest = 0x0;
-static const int32_t kRpcFlagMsgResponse = 0x1;
-
-// service type
-static const int32_t kMasterService = 1;
-static const int32_t kBrokerReadService = 2;
-static const int32_t kBrokerWriteService = 3;
-static const int32_t kBrokerAdminService = 4;
-static const int32_t kMasterAdminService = 5;
-
-// request method
-// master rpc method
-static const int32_t kMasterMethoddProducerRegister = 1;
-static const int32_t kMasterMethoddProducerHeatbeat = 2;
-static const int32_t kMasterMethoddProducerClose = 3;
-static const int32_t kMasterMethoddConsumerRegister = 4;
-static const int32_t kMasterMethoddConsumerHeatbeat = 5;
-static const int32_t kMasterMethoddConsumerClose = 6;
-
-// broker rpc method
-static const int32_t kBrokerMethoddProducerRegister = 11;
-static const int32_t kBrokerMethoddProducerHeatbeat = 12;
-static const int32_t kBrokerMethoddProducerSendMsg = 13;
-static const int32_t kBrokerMethoddProducerClose = 14;
-static const int32_t kBrokerMethoddConsumerRegister = 15;
-static const int32_t kBrokerMethoddConsumerHeatbeat = 16;
-static const int32_t kBrokerMethoddConsumerGetMsg = 17;
-static const int32_t kBrokerMethoddConsumerCommit = 18;
-static const int32_t kBrokerMethoddConsumerClose = 19;
-
-// register operate type
-static const int32_t kRegOpTypeRegister = 31;
-static const int32_t kRegOpTypeUnReg = 32;
-
-// rpc connect node timeout
-static const int32_t kRpcConnectTimeoutMs = 3000;
-
-// rpc timeout define
-static const int32_t kRpcTimoutDefSec = 15;
-static const int32_t kRpcTimoutMaxSec = 300;
-static const int32_t kRpcTimoutMinSec = 8;
-
-}  // namespace rpc_config
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_CONST_RPC_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
deleted file mode 100644
index 0b20c4f..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef _TUBEMQ_EXECUTOR_POOL_
-#define _TUBEMQ_EXECUTOR_POOL_
-
-#include <stdlib.h>
-
-#include <asio.hpp>
-#include <asio/ssl.hpp>
-#include <functional>
-#include <memory>
-#include <mutex>
-#include <thread>
-#include <vector>
-
-
-#include "tubemq/noncopyable.h"
-
-namespace tubemq {
-
-typedef std::shared_ptr<asio::ip::tcp::socket> SocketPtr;
-typedef std::shared_ptr<asio::ssl::stream<asio::ip::tcp::socket &> > TlsSocketPtr;
-typedef std::shared_ptr<asio::ip::tcp::resolver> TcpResolverPtr;
-typedef std::shared_ptr<asio::steady_timer> SteadyTimerPtr;
-
-class Executor : noncopyable {
- public:
-  Executor();
-  ~Executor();
-
-  SocketPtr CreateSocket();
-  TlsSocketPtr CreateTlsSocket(SocketPtr &socket, asio::ssl::context &ctx);
-  TcpResolverPtr CreateTcpResolver();
-  SteadyTimerPtr CreateSteadyTimer();
-  using func = std::function<void(void)>;
-
-  void Post(func task);
-
-  std::shared_ptr<asio::io_context> GetIoContext() { return io_context_; }
-
-  // Close executor and exit thread
-  void Close();
-
- private:
-  void StartWorker(std::shared_ptr<asio::io_context> io_context);
-  std::shared_ptr<asio::io_context> io_context_;
-  using io_context_work = asio::executor_work_guard<asio::io_context::executor_type>;
-  io_context_work work_;
-  std::thread worker_;
-};
-
-typedef std::shared_ptr<Executor> ExecutorPtr;
-
-class ExecutorPool : noncopyable {
- public:
-  explicit ExecutorPool(int nthreads = 2);
-
-  ExecutorPtr Get();
-
-  // Resize executor thread
-  void Resize(int nthreads) {
-    Lock lock(mutex_);
-    executors_.resize(nthreads);
-  }
-
-  void Close();
-
- private:
-  typedef std::vector<ExecutorPtr> ExecutorList;
-  ExecutorList executors_;
-  uint32_t executorIdx_;
-  std::mutex mutex_;
-  typedef std::unique_lock<std::mutex> Lock;
-};
-
-typedef std::shared_ptr<ExecutorPool> ExecutorPoolPtr;
-
-}  // namespace tubemq
-
-#endif  // _TUBEMQ_EXECUTOR_POOL_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
deleted file mode 100644
index 66b5aba..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_FLOW_CONTROL_H_
-#define TUBEMQ_CLIENT_FLOW_CONTROL_H_
-
-#include <rapidjson/document.h>
-#include <stdint.h>
-
-#include <algorithm>
-#include <list>
-#include <map>
-#include <mutex>
-#include <string>
-#include <vector>
-
-#include "tubemq/atomic_def.h"
-
-namespace tubemq {
-
-using std::map;
-using std::mutex;
-using std::string;
-using std::vector;
-
-class FlowCtrlResult {
- public:
-  FlowCtrlResult();
-  FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit);
-  FlowCtrlResult& operator=(const FlowCtrlResult& target);
-  void SetDataDltAndFreqLimit(int64_t datasize_limit, int32_t freqms_limit);
-  void SetDataSizeLimit(int64_t datasize_limit);
-  void SetFreqMsLimit(int32_t freqms_limit);
-  int64_t GetDataSizeLimit();
-  int32_t GetFreqMsLimit();
-
- private:
-  int64_t datasize_limit_;
-  int32_t freqms_limit_;
-};
-
-class FlowCtrlItem {
- public:
-  FlowCtrlItem();
-  FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit);
-  FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
-               int32_t min_data_filter_freqms);
-  FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, int64_t datadlt_m,
-               int64_t datasize_limit, int32_t freqms_limit);
-  FlowCtrlItem& operator=(const FlowCtrlItem& target);
-  void Clear();
-  void ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
-                          int32_t min_data_filter_freqms);
-  int32_t GetFreLimit(int32_t msg_zero_cnt) const;
-  bool GetDataLimit(int64_t datadlt_m,
-    int32_t curr_time, FlowCtrlResult& flowctrl_result) const;
-  const int32_t GetType() const { return type_; }
-  const int32_t GetZeroCnt() const { return zero_cnt_; }
-  const int32_t GetStartTime() const { return start_time_; }
-  const int32_t GetEndTime() const { return end_time_; }
-  const int64_t GetDataSizeLimit() const { return datasize_limit_; }
-  const int32_t GetFreqMsLimit() const { return freqms_limit_; }
-  const int64_t GetDltInM() const { return datadlt_m_; }
-
- private:
-  int32_t type_;
-  int32_t start_time_;
-  int32_t end_time_;
-  int64_t datadlt_m_;
-  int64_t datasize_limit_;
-  int32_t freqms_limit_;
-  int32_t zero_cnt_;
-};
-
-class FlowCtrlRuleHandler {
- public:
-  FlowCtrlRuleHandler();
-  ~FlowCtrlRuleHandler();
-  void UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id, int64_t flowctrl_id,
-                             const string& flowctrl_info);
-  bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const;
-  int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const;
-  void GetFilterCtrlItem(FlowCtrlItem& result) const;
-  void GetFlowCtrlInfo(string& flowctrl_info) const;
-  int32_t GetMinZeroCnt() const { return this->min_zero_cnt_.Get(); }
-  int32_t GetQryPriorityId() const { return this->qrypriority_id_.Get(); }
-  void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
-  const int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
-
- private:
-  void initialStatisData();
-  void clearStatisData();
-  static bool compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2);
-  static bool compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2);
-  bool parseStringMember(string& err_info, const rapidjson::Value& root, const char* key,
-                         string& value, bool compare_value, string required_val);
-  bool parseLongMember(string& err_info, const rapidjson::Value& root, const char* key,
-                       int64_t& value, bool compare_value, int64_t required_val);
-  bool parseIntMember(string& err_info, const rapidjson::Value& root, const char* key,
-                      int32_t& value, bool compare_value, int32_t required_val);
-  bool parseFlowCtrlInfo(const string& flowctrl_info,
-                         map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map);
-  bool parseDataLimit(string& err_info, const rapidjson::Value& root,
-                      vector<FlowCtrlItem>& flowCtrlItems);
-  bool parseFreqLimit(string& err_info, const rapidjson::Value& root,
-                      vector<FlowCtrlItem>& flowctrl_items);
-  bool parseLowFetchLimit(string& err_info, const rapidjson::Value& root,
-                          vector<FlowCtrlItem>& flowctrl_items);
-  bool parseTimeMember(string& err_info, const rapidjson::Value& root, const char* key,
-                       int32_t& value);
-
- private:
-  mutable mutex config_lock_;
-  string flowctrl_info_;
-  FlowCtrlItem filter_ctrl_item_;
-  map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
-  int64_t last_update_time_;
-  AtomicLong flowctrl_id_;
-  AtomicInteger qrypriority_id_;
-  AtomicInteger min_zero_cnt_;
-  AtomicLong min_datadlt_limt_;
-  AtomicInteger datalimit_start_time_;
-  AtomicInteger datalimit_end_time_;
-};
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_FLOW_CONTROL_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
deleted file mode 100644
index 74d083c..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef _TUBEMQ_LOG_FILE_
-#define _TUBEMQ_LOG_FILE_
-
-#include <stdint.h>
-
-#include <string>
-#include <vector>
-
-namespace tubemq {
-class Logger;
-
-Logger& GetLogger();
-
-#define LOG_LEVEL(level, fmt, ...)                                                   \
-  {                                                                                  \
-    if (tubemq::GetLogger().IsEnable(level)) {                                       \
-      tubemq::GetLogger().Write("[%s:%d][%s]" fmt, __func__, __LINE__,               \
-                                tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
-    }                                                                                \
-  }
-
-#define LOG_TRACE(fmt, ...) \
-  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
-#define LOG_DEBUG(fmt, ...) \
-  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
-#define LOG_INFO(fmt, ...) \
-  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
-#define LOG_WARN(fmt, ...) \
-  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
-#define LOG_ERROR(fmt, ...) \
-  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
-
-#define LOG_TUBEMQ(logger, level, fmt, ...)                                                    \
-  {                                                                                            \
-    if (logger.IsEnable(level)) {                                                              \
-      logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), \
-                   ##__VA_ARGS__);                                                             \
-    }                                                                                          \
-  }
-
-class Logger {
- public:
-  enum Level {
-    kTrace = 0,
-    kDebug = 1,
-    kInfo = 2,
-    kWarn = 3,
-    kError = 4,
-  };
-
-  // size: MB
-  Logger()
-      : file_max_size_(100),
-        file_num_(10),
-        level_(kError),
-        base_path_("tubemq"),
-        instance_("TubeMQ") {
-    setup();
-  }
-
-  ~Logger(void) {}
-
-  // path example: ../log/tubemq
-  // size: MB
-  bool Init(const std::string& path, Level level, uint32_t file_max_size = 100,
-            uint32_t file_num = 10);
-
-  bool Write(const char* sFormat, ...) __attribute__((format(printf, 2, 3)));
-  inline bool WriteStream(const std::string& msg) { return writeStream(msg.c_str()); }
-
-  inline void SetInstance(const std::string& instance) { instance_ = instance; }
-  inline bool IsEnable(Level level) {
-    if (level_ <= level) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  static const char* Level2String(Level level) {
-    static const char* level_names[] = {
-        "TRACE", "DEBUG", "INFO", "WARN", "ERROR",
-    };
-    return level_names[level];
-  }
-
- private:
-  void setup();
-  bool writeStream(const char* msg);
-
- private:
-  uint32_t file_max_size_;
-  uint16_t file_num_;
-  uint8_t level_;
-
-  std::string base_path_;
-  std::string instance_;
-  std::string err_msg_;
-};
-}  // namespace tubemq
-#endif  // _TUBEMQ_LOG_FILE_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
deleted file mode 100644
index 3091347..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ /dev/null
@@ -1,188 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_META_INFO_H_
-#define TUBEMQ_CLIENT_META_INFO_H_
-
-#include <stdint.h>
-
-#include <list>
-#include <string>
-
-#include "tubemq/flowctrl_def.h"
-
-namespace tubemq {
-
-using std::list;
-using std::map;
-using std::string;
-
-class NodeInfo {
- public:
-  NodeInfo();
-  NodeInfo(bool is_broker, const string& node_info);
-  NodeInfo(const string& node_host, uint32_t node_port);
-  NodeInfo(int32_t node_id, const string& node_host, uint32_t node_port);
-  ~NodeInfo();
-  NodeInfo& operator=(const NodeInfo& target);
-  bool operator==(const NodeInfo& target);
-  bool operator<(const NodeInfo& target) const;
-  const uint32_t GetNodeId() const;
-  const string& GetHost() const;
-  const uint32_t GetPort() const;
-  const string& GetAddrInfo() const;
-  const string& GetNodeInfo() const;
-
- private:
-  void buildStrInfo();
-
- private:
-  uint32_t node_id_;
-  string node_host_;
-  uint32_t node_port_;
-  // ip:port
-  string addr_info_;
-  // id:ip:port
-  string node_info_;
-};
-
-class Partition {
- public:
-  Partition();
-  explicit Partition(const string& partition_info);
-  Partition(const NodeInfo& broker_info, const string& part_str);
-  Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id);
-  ~Partition();
-  Partition& operator=(const Partition& target);
-  bool operator==(const Partition& target);
-  const uint32_t GetBrokerId() const;
-  const string& GetBrokerHost() const;
-  const uint32_t GetBrokerPort() const;
-  const string& GetPartitionKey() const;
-  const string& GetTopic() const;
-  const NodeInfo& GetBrokerInfo() const;
-  const uint32_t GetPartitionId() const;
-  const string& ToString() const;
-
- private:
-  void buildPartitionKey();
-
- private:
-  string topic_;
-  NodeInfo broker_info_;
-  uint32_t partition_id_;
-  string partition_key_;
-  string partition_info_;
-};
-
-class PartitionExt : public Partition {
- public:
-  PartitionExt();
-  explicit PartitionExt(const string& partition_info);
-  PartitionExt(const NodeInfo& broker_info, const string& part_str);
-  ~PartitionExt();
-  PartitionExt& operator=(const PartitionExt& target);
-  void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
-    int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
-  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
-  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
-    int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
-    int64_t last_datadlt, bool require_slow);
-  void SetLastConsumed(bool last_consumed);
-  bool IsLastConsumed();
-
- private:
-  void resetParameters();
-  void updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
-    const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt);
-
- private:
-  bool is_last_consumed_;
-  FlowCtrlResult cur_flowctrl_;
-  FlowCtrlItem cur_freqctrl_;
-  int64_t next_stage_updtime_;
-  int64_t next_slice_updtime_;
-  int64_t limit_slice_msgsize_;
-  int64_t cur_stage_msgsize_;
-  int64_t cur_slice_msgsize_;
-  int32_t total_zero_cnt_;
-  int64_t booked_time_;
-  int32_t booked_errcode_;
-  bool    booked_esc_limit_;
-  int32_t booked_msgsize_;
-  int64_t booked_dlt_limit_;
-  int64_t booked_curdata_dlt_;
-  bool    booked_require_slow_;
-};
-
-class SubscribeInfo {
- public:
-  SubscribeInfo();
-  explicit SubscribeInfo(const string& sub_info);
-  SubscribeInfo(const string& consumer_id,
-        const string& group_name, const PartitionExt& partition_ext);
-  SubscribeInfo& operator=(const SubscribeInfo& target);
-  const string& GetConsumerId() const;
-  const string& GetGroup() const;
-  const PartitionExt& GetPartitionExt() const;
-  const uint32_t GgetBrokerId() const;
-  const string& GetBrokerHost() const;
-  const uint32_t GetBrokerPort() const;
-  const string& GetTopic() const;
-  const uint32_t GetPartitionId() const;
-  const string& ToString() const;
-
- private:
-  void buildSubInfo();
-
- private:
-  string consumer_id_;
-  string group_;
-  PartitionExt partitionext_;
-  string sub_info_;
-};
-
-class ConsumerEvent {
- public:
-  ConsumerEvent();
-  ConsumerEvent(const ConsumerEvent& target);
-  ConsumerEvent(int64_t rebalance_id, int32_t event_type,
-                const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status);
-  ConsumerEvent& operator=(const ConsumerEvent& target);
-  const int64_t GetRebalanceId() const;
-  const int32_t GetEventType() const;
-  const int32_t GetEventStatus() const;
-  void SetEventType(int32_t event_type);
-  void SetEventStatus(int32_t event_status);
-  const list<SubscribeInfo>& GetSubscribeInfoList() const;
-  string ToString();
-
- private:
-  int64_t rebalance_id_;
-  int32_t event_type_;
-  int32_t event_status_;
-  list<SubscribeInfo> subscribe_list_;
-};
-
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_META_INFO_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/noncopyable.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/noncopyable.h
deleted file mode 100644
index 9afbf52..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/noncopyable.h
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef _TUBUMQ_NONCOPYABLE_H
-#define _TUBUMQ_NONCOPYABLE_H
-
-namespace tubemq {
-
-class noncopyable {
- public:
-  noncopyable(const noncopyable&) = delete;
-  void operator=(const noncopyable&) = delete;
-
- protected:
-  noncopyable() = default;
-  ~noncopyable() = default;
-};
-
-}  // namespace tubemq
-
-#endif  // _TUBUMQ_NONCOPYABLE_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
deleted file mode 100644
index 3b85462..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
-#define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
-
-#include <stdint.h>
-
-#include <condition_variable>
-#include <list>
-#include <map>
-#include <mutex>
-#include <set>
-#include <string>
-#include <tuple>
-
-#include "tubemq/atomic_def.h"
-#include "tubemq/flowctrl_def.h"
-#include "tubemq/meta_info.h"
-#include "tubemq/executor_pool.h"
-#include "tubemq/tubemq_errcode.h"
-
-
-
-
-
-namespace tubemq {
-
-using std::condition_variable;
-using std::map;
-using std::set;
-using std::list;
-using std::mutex;
-using std::string;
-using std::tuple;
-
-
-
-// consumer remote data cache
-class RmtDataCacheCsm {
- public:
-  RmtDataCacheCsm();
-  ~RmtDataCacheCsm();
-  void SetConsumerInfo(const string& client_id, const string& group_name);
-  void UpdateDefFlowCtrlInfo(int64_t flowctrl_id,
-                                     const string& flowctrl_info);
-  void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
-                 int64_t flowctrl_id, const string& flowctrl_info);
-  const int64_t GetGroupQryPriorityId() const;
-  const int64_t GetDefFlowCtrlId() const { return def_flowctrl_handler_.GetFlowCtrlId(); }
-  const int64_t GetGroupFlowCtrlId() const { return group_flowctrl_handler_.GetFlowCtrlId(); }
-  bool IsUnderGroupCtrl();
-  void AddNewPartition(const PartitionExt& partition_ext);
-  bool SelectPartition(string &err_info,
-           PartitionExt& partition_ext, string& confirm_context);
-  void BookedPartionInfo(const string& partition_key, int64_t curr_offset,
-                            int32_t err_code, bool esc_limit, int32_t msg_size,
-                            int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow);
-  bool RelPartition(string &err_info, bool filter_consume,
-                         const string& confirm_context, bool is_consumed);
-  bool RelPartition(string &err_info, const string& confirm_context, bool is_consumed);
-  bool RelPartition(string &err_info, bool filter_consume,
-                         const string& confirm_context, bool is_consumed,
-                         int64_t curr_offset, int32_t err_code, bool esc_limit,
-                         int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt);
-  void FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
-          list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions);
-  void GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst);
-  bool GetPartitionExt(const string& part_key, PartitionExt& partition_ext);
-  void GetRegBrokers(list<NodeInfo>& brokers);
-  void GetPartitionByBroker(const NodeInfo& broker_info,
-                                    list<PartitionExt>& partition_list);
-  void GetCurPartitionOffsets(map<string, int64_t> part_offset_map);
-  void GetAllBrokerPartitions(map<NodeInfo, list<PartitionExt> >& broker_parts);
-  void RemovePartition(const list<PartitionExt>& partition_list);
-  void RemovePartition(const set<string>& partition_keys);
-  bool RemovePartition(string &err_info, const string& confirm_context);
-  void RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
-        bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts);
-  bool IsPartitionFirstReg(const string& partition_key);
-  void OfferEvent(const ConsumerEvent& event);
-  void TakeEvent(ConsumerEvent& event);
-  void ClearEvent();
-  void OfferEventResult(const ConsumerEvent& event);
-  bool PollEventResult(ConsumerEvent& event);
-  void HandleTimeout(const string partition_key, const asio::error_code& error);
-
- private:
-  void addDelayTimer(const string& part_key, int64_t delay_time);
-  void resetIdlePartition(const string& partition_key, bool need_reuse);
-  void rmvMetaInfo(const string& partition_key);
-  void buildConfirmContext(const string& partition_key,
-                    int64_t booked_time, string& confirm_context);
-  bool parseConfirmContext(string &err_info,
-    const string& confirm_context, string& partition_key, int64_t& booked_time);
-  bool inRelPartition(string &err_info, bool need_delay_check,
-    bool filter_consume, const string& confirm_context, bool is_consumed);
-
-
- private:
-  // 
-  string consumer_id_;
-  string group_name_;
-  // flow ctrl
-  FlowCtrlRuleHandler group_flowctrl_handler_;
-  FlowCtrlRuleHandler def_flowctrl_handler_;
-  AtomicBoolean under_groupctrl_;
-  AtomicLong last_checktime_;
-  // meta info
-  mutable mutex meta_lock_;
-  // partiton allocated map
-  map<string, PartitionExt> partitions_;
-  // topic partiton map
-  map<string, set<string> > topic_partition_;
-  // broker parition map
-  map<NodeInfo, set<string> > broker_partition_;
-  map<string, SubscribeInfo>  part_subinfo_;
-  // for idle partitions occupy
-  list<string> index_partitions_;
-  // for partition used map
-  map<string, int64_t> partition_useds_;
-  // for partiton timer map
-  map<string, tuple<int64_t, SteadyTimerPtr> > partition_timeouts_;
-  // data book
-  mutable mutex data_book_mutex_;
-  // for partition offset cache
-  map<string, int64_t> partition_offset_;
-  // for partiton register booked
-  map<string, bool> part_reg_booked_;
-
-  // event
-  mutable mutex event_read_mutex_;
-  condition_variable event_read_cond_;
-  list<ConsumerEvent> rebalance_events_;
-  mutable mutex event_write_mutex_;
-  list<ConsumerEvent> rebalance_results_;
-};
-
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
deleted file mode 100644
index 1978da6..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef _TUBEMQ_SINGLETON_H
-#define _TUBEMQ_SINGLETON_H
-
-#include <assert.h>
-#include <stdlib.h>
-
-#include <mutex>
-#include <thread>
-
-#include "tubemq/noncopyable.h"
-
-namespace tubemq {
-
-template <typename T>
-class Singleton : noncopyable {
- public:
-  static T& Instance() {
-    std::call_once(once_, Singleton::init);
-    assert(value_ != nullptr);
-    return *value_;
-  }
-
- protected:
-  Singleton() {}
-  ~Singleton() {}
-
- private:
-  static void init() { value_ = new T(); }
-
- private:
-  static std::once_flag once_;
-  static T* value_;
-};
-
-template <typename T>
-std::once_flag Singleton<T>::once_;
-
-template <typename T>
-T* Singleton<T>::value_ = nullptr;
-
-}  // namespace tubemq
-
-#endif  // _TUBEMQ_SINGLETON_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_atomic.h
similarity index 98%
rename from tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_atomic.h
index f991269..c8c9718 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_atomic.h
@@ -83,7 +83,7 @@ class AtomicBoolean {
   inline bool GetAndSet(bool new_value) { return counter_.exchange(new_value); }
 
   // CAS SET
-  inline bool CompareAndSet(bool& expect, bool update) {
+  inline bool CompareAndSet(bool expect, bool update) {
     return counter_.compare_exchange_strong(expect, update, std::memory_order_relaxed);
   }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_client.h
similarity index 50%
rename from tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_client.h
index 1bede63..43e9fc8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_client.h
@@ -17,35 +17,43 @@
  * under the License.
  */
 
-#ifndef TUBEMQ_CLIENT_FILE_INI_H_
-#define TUBEMQ_CLIENT_FILE_INI_H_
+#ifndef TUBEMQ_CLIENT_HEADER_H_
+#define TUBEMQ_CLIENT_HEADER_H_
 
 #include <stdint.h>
-
 #include <map>
 #include <string>
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
 
 namespace tubemq {
 
-using std::map;
-using std::string;
+bool StartTubeMQService(string& err_info,
+  const string& conf_file = "../conf/client.conf");
+bool StopTubeMQService(string& err_info);
+
 
-class Fileini {
+class TubeMQConsumer {
  public:
-  Fileini();
-  ~Fileini();
-  bool Loadini(string& err_info, const string& file_name);
-  bool GetValue(string& err_info, const string& sector, const string& key,
-                    string& value, const string& def) const;
-  bool GetValue(string& err_info, const string& sector, const string& key,
-                   int32_t& value, int32_t def) const;
+  TubeMQConsumer();
+  ~TubeMQConsumer();
+  bool Start(string& err_info, const ConsumerConfig& config);
+  virtual void ShutDown();
+  const int32_t GetClientId() const { return client_id_; }
+  bool GetMessage(ConsumerResult& result);
+  bool Confirm(const string& confirm_context, bool is_consumed, ConsumerResult& result);
+  bool GetCurConsumedInfo(map<string, ConsumeOffsetInfo>& consume_info_map);
 
  private:
-  bool init_flag_;
-  // sector        key    value
-  map<string, map<string, string> > ini_map_;
+  int32_t client_id_;
+  AtomicInteger status_;
 };
 
+
 }  // namespace tubemq
 
-#endif  // TUBEMQ_CLIENT_FILE_INI_H_
+#endif  // TUBEMQ_CLIENT_HEADER_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
index f2b9952..03e5424 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
@@ -51,16 +51,18 @@ class BaseConfig {
   bool IsAuthenticEnabled();
   const string& GetUsrName() const;
   const string& GetUsrPassWord() const;
-  // set the rpc timout, unit second, duration [8, 300], default 15 seconds;
-  void SetRpcReadTimeoutSec(int32_t rpc_read_timeout_sec);
-  int32_t GetRpcReadTimeoutSec();
-  // Set the duration of the client's heartbeat cycle, in seconds, the default is 10 seconds
-  void SetHeartbeatPeriodSec(int32_t heartbeat_period_sec);
-  int32_t GetHeartbeatPeriodSec();
+  // set the rpc timout, unit Millis-second, duration [8000, 300000],
+  // default 15000 Millis-seconds;
+  void SetRpcReadTimeoutMs(int32_t rpc_read_timeout_ms);
+  int32_t GetRpcReadTimeoutMs();
+  // Set the duration of the client's heartbeat cycle, in Millis-seconds,
+  // the default is 10000 Millis-seconds
+  void SetHeartbeatPeriodMs(int32_t heartbeat_period_ms);
+  int32_t GetHeartbeatPeriodMs();
   void SetMaxHeartBeatRetryTimes(int32_t max_heartbeat_retry_times);
   int32_t GetMaxHeartBeatRetryTimes();
-  void SetHeartbeatPeriodAftFailSec(int32_t heartbeat_period_afterfail_sec);
-  int32_t GetHeartbeatPeriodAftFailSec();
+  void SetHeartbeatPeriodAftFailMs(int32_t heartbeat_period_afterfail_ms);
+  int32_t GetHeartbeatPeriodAftFailMs();
   string ToString();
 
  private:
@@ -74,10 +76,10 @@ class BaseConfig {
   string tls_trust_store_path_;
   string tls_trust_store_password_;
   // other setting
-  int32_t rpc_read_timeout_sec_;
-  int32_t heartbeat_period_sec_;
+  int32_t rpc_read_timeout_ms_;
+  int32_t heartbeat_period_ms_;
   int32_t max_heartbeat_retry_times_;
-  int32_t heartbeat_period_afterfail_sec_;
+  int32_t heartbeat_period_afterfail_ms_;
 };
 
 enum ConsumePosition {
@@ -109,11 +111,13 @@ class ConsumerConfig : public BaseConfig {
   void SetConsumePosition(ConsumePosition consume_from_where);
   const ConsumePosition GetConsumePosition() const;
   const int32_t GetMsgNotFoundWaitPeriodMs() const;
-  void SetMsgNotFoundWaitPeriodMs(int msg_notfound_wait_period_ms);
+  const int32_t GetMaxPartCheckPeriodMs() const;
+  const uint32_t GetPartCheckSliceMs() const;
+  void SetMsgNotFoundWaitPeriodMs(int32_t msg_notfound_wait_period_ms);
+  void SetMaxPartCheckPeriodMs(int32_t max_part_check_period_ms);
+  void SetPartCheckSliceMs(uint32_t part_check_slice_ms);
   const int32_t GetMaxSubinfoReportIntvl() const;
   void SetMaxSubinfoReportIntvl(int32_t max_subinfo_report_intvl);
-  bool IsConfirmInLocal();
-  void SetConfirmInLocal(bool confirm_in_local);
   bool IsRollbackIfConfirmTimeout();
   void setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout);
   const int32_t GetWaitPeriodIfConfirmWaitRebalanceMs() const;
@@ -140,8 +144,9 @@ class ConsumerConfig : public BaseConfig {
   map<string, int64_t> part_offset_map_;
   ConsumePosition consume_position_;
   int32_t max_subinfo_report_intvl_;
+  int32_t max_part_check_period_ms_;
+  uint32_t part_check_slice_ms_;
   int32_t msg_notfound_wait_period_ms_;
-  bool is_confirm_in_local_;
   bool is_rollback_if_confirm_timout_;
   int32_t reb_confirm_wait_period_ms_;
   int32_t max_confirm_wait_period_ms_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
index c4d625d..dd18e86 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_errcode.h
@@ -20,32 +20,98 @@
 #ifndef TUBEMQ_CLIENT_CONST_ERR_CODE_H_
 #define TUBEMQ_CLIENT_CONST_ERR_CODE_H_
 
+#include <string>
+
 namespace tubemq {
 
 namespace err_code {
-  static const int32_t kErrSuccess = 200;
-  static const int32_t kErrNotReady = 201;
-  static const int32_t kErrMoved = 301;
-
-  static const int32_t kErrBadRequest = 400;
-  static const int32_t kErrUnAuthorized = 401;
-  static const int32_t kErrForbidden = 403;
-  static const int32_t kErrNotFound = 404;
-  static const int32_t kErrPartitionOccupied = 410;
-  static const int32_t kErrHbNoNode = 411;
-  static const int32_t kErrDuplicatePartition = 412;
-  static const int32_t kErrCertificateFailure = 415;
-  static const int32_t kErrServerOverflow = 419;
-  static const int32_t kErrConsumeGroupForbidden = 450;
-  static const int32_t kErrConsumeSpeedLimit = 452;
-  static const int32_t kErrConsumeContentForbidden = 455;
-
-  static const int32_t kErrServerError = 500;
-  static const int32_t kErrServiceUnavilable = 503;
-  static const int32_t kErrServerMsgsetNullError = 510;
-  static const int32_t kErrWaitServerRspTimeout = 550;
+
+/**
+ * Collection of return codes
+ */
+enum Result {
+  kErrSuccess = 200,
+  kErrNotReady = 201,
+  kErrMoved = 301,
+  kErrBadRequest = 400,
+  kErrUnAuthorized = 401,
+  kErrForbidden = 403,
+  kErrNotFound = 404,
+  kErrNoPartAssigned = 406,
+  kErrAllPartWaiting = 407,
+  kErrAllPartInUse = 408,
+  kErrPartitionOccupied = 410,
+  kErrHbNoNode = 411,
+  kErrDuplicatePartition = 412,
+  kErrCertificateFailure = 415,
+  kErrServerOverflow = 419,
+  kErrConsumeGroupForbidden = 450,
+  kErrConsumeSpeedLimit = 452,
+  kErrConsumeContentForbidden = 455,
+
+  kErrServerError = 500,
+  kErrRcvThrowError = 501,
+  kErrServiceUnavilable = 503,
+  kErrServerMsgsetNullError = 510,
+  kErrWaitServerRspTimeout = 550,
+  kErrNetWorkTimeout = 1000,
+  kErrNetworkError = 1001,
+  kErrServerStop = 2001,
+  kErrMQServiceStop = 2002,
+  kErrClientStop = 2003,
+  kErrConfirmTimeout = 2004,
+  kErrParseFailure = 5001,
+};
 }  // namespace err_code
 
+// Class to represent an error code value.
+class ErrorCode {
+ public:
+  // Default constructor.
+  ErrorCode() : value_(err_code::kErrSuccess), error_msg_("") {}
+
+  // Construct with specific error code and category.
+  ErrorCode(int v, const std::string& c) : value_(v), error_msg_(c) {}
+
+  // Clear the error value to the default.
+  void Clear() {
+    value_ = 0;
+    error_msg_.clear();
+  }
+
+  // Assign a new error value.
+  void Assign(int v, const std::string& c) {
+    value_ = v;
+    error_msg_ = c;
+  }
+
+  /// Get the error value.
+  int Value() const { return value_; }
+
+  // Get the error message.
+  const std::string& Message() const { return error_msg_; }
+
+  // Operator to test if the error represents success.
+  bool operator!() const { return value_ == 0; }
+
+  // Equality operator to compare two error objects.
+  friend bool operator==(const ErrorCode& e1, const ErrorCode& e2) {
+    return e1.value_ == e2.value_;
+  }
+
+  // Inequality operator to compare two error objects.
+  friend bool operator!=(const ErrorCode& e1, const ErrorCode& e2) {
+    return e1.value_ != e2.value_;
+  }
+
+ private:
+  // The value associated with the error code.
+  int value_;
+
+  // The error msg associated with the error code.
+  std::string error_msg_;
+};
+
 }  // namespace tubemq
 
-#endif  // TUBEMQ_CLIENT_CONST_ERR_CODE_H_
\ No newline at end of file
+#endif  // TUBEMQ_CLIENT_CONST_ERR_CODE_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
index 8d68201..f418d36 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_message.h
@@ -23,7 +23,6 @@
 #include <stdint.h>
 #include <stdio.h>
 
-#include <list>
 #include <map>
 #include <string>
 
@@ -37,6 +36,8 @@ class Message {
   Message();
   Message(const Message& target);
   Message(const string& topic, const char* data, uint32_t datalen);
+  Message(const string& topic, int32_t flag, int64_t message_id,
+    const char* data, uint32_t datalen, const map<string, string>& properties);
   virtual ~Message();
   Message& operator=(const Message& target);
   const int64_t GetMessageId() const;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
index 2f35b22..5838fe6 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_return.h
@@ -26,26 +26,28 @@
 #include <string>
 
 #include "tubemq/tubemq_message.h"
-#include "tubemq/meta_info.h"
 
 
 
 
 namespace tubemq {
 
+using std::list;
 using std::string;
 
 
+
 class PeerInfo {
  public:
   PeerInfo();
-  PeerInfo(const Partition& partition, int64_t offset);
+  PeerInfo(const string& broker_host, uint32_t partition_id,
+    const string& partiton_key, int64_t offset);
   PeerInfo& operator=(const PeerInfo& target);
-  void SetMsgSourceInfo(const Partition& partition, int64_t offset);
   const uint32_t GetPartitionId() const { return partition_id_; }
   const string& GetBrokerHost() const { return broker_host_; }
   const string& GetPartitionKey() const { return partition_key_; }
   const int64_t GetCurrOffset() const { return curr_offset_; }
+  void SetCurrOffset(int64_t offset) { curr_offset_ = offset; }
 
  private:
   uint32_t partition_id_;
@@ -55,18 +57,34 @@ class PeerInfo {
 };
 
 
+class ConsumeOffsetInfo {
+ public:
+  ConsumeOffsetInfo();
+  ConsumeOffsetInfo(const string& part_key, int64_t curr_offset);
+  void SetConsumeOffsetInfo(const string& part_key, int64_t curr_offset);
+  ConsumeOffsetInfo& operator=(const ConsumeOffsetInfo& target);
+  const string& GetPartitonKey() const { return partition_key_; }
+  const int64_t& GetCurrOffset() const { return curr_offset_; }
+
+ private:
+  string partition_key_;
+  int64_t curr_offset_;
+};
+
 
 class ConsumerResult {
  public:
   ConsumerResult();
   ConsumerResult(const ConsumerResult& target);
-  ConsumerResult(int32_t err_code, string err_msg);
+  ConsumerResult(int32_t error_code, string err_msg);
   ~ConsumerResult();
   ConsumerResult& operator=(const ConsumerResult& target);
-  void SetFailureResult(int32_t err_code, string err_msg);
-  void SetFailureResult(int32_t err_code, string err_msg,
-              const string& topic_name, const PeerInfo& peer_info);
-  void SetSuccessResult(int32_t err_code, const string& topic_name,
+  void SetFailureResult(int32_t error_code, string err_msg);
+  void SetFailureResult(int32_t error_code, string err_msg,
+    const string& topic_name, const PeerInfo& peer_info);
+  void SetSuccessResult(int32_t error_code,
+    const string& topic_name, const PeerInfo& peer_info);
+  void SetSuccessResult(int32_t error_code, const string& topic_name,
                   const PeerInfo& peer_info, const string& confirm_context,
                   const list<Message>& message_list);
   bool IsSuccess() { return success_; }
@@ -89,7 +107,7 @@ class ConsumerResult {
   list<Message> message_list_;
 };
 
-}
+}  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_RETURN_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h
deleted file mode 100644
index bd04da0..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_UNIQUESEQID_H
-#define TUBEMQ_UNIQUESEQID_H
-
-#include <stdint.h>
-
-#include <atomic>
-
-namespace tubemq {
-
-class UniqueSeqId {
- public:
-  UniqueSeqId() : id(0) {}
-
-  uint32_t Next() { return id.fetch_add(1, std::memory_order_relaxed); }
-
- protected:
-  std::atomic<uint32_t> id;
-};
-
-}  // namespace tubemq
-
-#endif
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
deleted file mode 100644
index 3bd3068..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_UTILS_H_
-#define TUBEMQ_CLIENT_UTILS_H_
-
-#include <stdint.h>
-
-#include <map>
-#include <string>
-#include <vector>
-
-namespace tubemq {
-
-using std::map;
-using std::string;
-using std::vector;
-
-class Utils {
- public:
-  // trim string info
-  static string Trim(const string& source);
-  // split string to vector
-  static void Split(const string& source, vector<string>& result, const string& delimiter);
-  // split string to map<string, int>
-  static void Split(const string& source, map<string, int32_t>& result, const string& delimiter_step1,
-                    const string& delimiter_step2);
-  static void Join(const vector<string>& vec, const string& delimiter, string& target);
-  static bool ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
-                          bool check_max_length, unsigned int maxlen);
-  static bool ValidGroupName(string& err_info, const string& group_name, string& tgt_group_name);
-  static bool ValidFilterItem(string& err_info, const string& src_filteritem,
-                              string& tgt_filteritem);
-  static string Int2str(int32_t data);
-  static string Long2str(int64_t data);
-  static uint32_t IpToInt(const string& ipv4_addr);
-  static int64_t GetCurrentTimeMillis();
-  static bool ValidConfigFile(string& err_info, const string& conf_file);
-  static bool GetLocalIPV4Address(string& err_info, string& localhost);
-};
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_UTILS_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
deleted file mode 100644
index c479ede..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_VERSION_H_
-#define TUBEMQ_CLIENT_VERSION_H_
-
-#include <string>
-
-namespace tubemq {
-
-using std::string;
-
-static const char kTubeMQClientVersion[] = "0.1.0-0.5.0";
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_VERSION_H_


[incubator-tubemq] 04/50: [TUBEMQ-250] Create C/C++ configure files

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 94899ddd034599429736d788c8f43cdcc4b406b0
Author: gosonzhang <go...@tencent.com>
AuthorDate: Wed Jul 1 11:24:01 2020 +0800

    [TUBEMQ-250] Create C/C++ configure files
---
 .../tubemq-client-cpp/inc/client_config.h          | 72 +++++++++++++++++++++-
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  | 17 +++--
 .../tubemq-client-cpp/inc/version.h                |  2 +-
 .../tubemq-client-cpp/src/client_config.cc         | 67 ++++++++++++++++++--
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc | 21 ++++---
 5 files changed, 151 insertions(+), 28 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
index 71f2c0e..576a8e1 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
@@ -23,6 +23,7 @@
 #include <string>
 #include <stdio.h>
 #include <map>
+#include <set>
 
 
 
@@ -47,6 +48,16 @@ static const int kMasterAddrInfoMaxLength = 1024;
 static const int kTopicNameMaxLength = 64;
 // max Consume GroupName length
 static const int kGroupNameMaxLength = 1024;
+// max subscribe info report times
+static const int kSubInfoReportMaxIntervalTimes = 6;
+// default message not found response wait period
+static const int kMsgNotfoundWaitPeriodMsDef = 200;
+// default confirm wait period if rebalance meeting
+static const int kRebConfirmWaitPeriodMsDef = 3000;
+// max confirm wait period anyway
+static const int kConfirmWaitPeriodMsMax = 60000;
+// default rebalance wait if shutdown meeting
+static const int kRebWaitPeriodWhenShutdownMs = 10000;
 }  // namespace config
 
 
@@ -97,11 +108,68 @@ class BaseConfig {
 };
 
 
-class ConsumerConfig {
- public:
+enum ConsumePosition {
+  kConsumeFromFirstOffset = -1,
+  kConsumeFromLatestOffset = 0,
+  kComsumeFromMaxOffsetAlways = 1
+};
+
+
+
+class ConsumerConfig : public BaseConfig {
+ public: 
   ConsumerConfig();
+  ~ConsumerConfig();
+  ConsumerConfig& operator=(const ConsumerConfig& target); 
+  bool SetGroupConsumeTarget(string& err_info, 
+    const string& group_name, const set<string>& subscribed_topicset);
+  bool SetGroupConsumeTarget(string& err_info, 
+    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map);
+  bool SetGroupConsumeTarget(string& err_info, 
+    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
+    const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map);
+  const string& GetGroupName() const;
+  const map<string, set<string> >& GetSubTopicAndFilterMap() const;
+  void SetConsumePosition(ConsumePosition consume_from_where);
+  const ConsumePosition GetConsumePosition() const;
+  const int GetMsgNotFoundWaitPeriodMs() const;
+  void SetMsgNotFoundWaitPeriodMs(int msg_notfound_wait_period_ms);
+  const int GetMaxSubinfoReportIntvl() const;
+  void SetMaxSubinfoReportIntvl(int max_subinfo_report_intvl);
+  bool IsConfirmInLocal();
+  void SetConfirmInLocal(bool confirm_in_local);
+  bool IsRollbackIfConfirmTimeout();
+  void setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout);
+  const int GetWaitPeriodIfConfirmWaitRebalanceMs() const;
+  void SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_period_ms);
+  const int GetMaxConfirmWaitPeriodMs() const;
+  void SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms);
+  const int GetShutdownRebWaitPeriodMs() const;
+  void SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms);
+  string ToString();
+     
+  
+ private: 
+  string group_name_;
+  map<string, set<string> > sub_topic_and_filter_map_;
+  bool is_bound_consume_;
+  string session_key_;
+  int source_count_;
+  bool is_select_big_;
+  map<string, long> part_offset_map_;
+  ConsumePosition consume_position_;
+  int max_subinfo_report_intvl_;
+  int msg_notfound_wait_period_ms_;
+  bool is_confirm_in_local_;
+  bool is_rollback_if_confirm_timout_;
+  int reb_confirm_wait_period_ms_;
+  int max_confirm_wait_period_ms_;
+  int shutdown_reb_wait_period_ms_;
 };
 
+
+
+
 }
 
 #endif
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index 937eb37..e8df420 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -27,23 +27,22 @@ namespace tubemq {
 
 using namespace std;
 
-static const string tWhitespaceCharSet = " \n\r\t\f\v";
 
 namespace delimiter {
-  static const string tDelimiterEqual = "=";
-  static const string tDelimiterAnd   = "&";
-  static const string tDelimiterComma = ",";
-  static const string tDelimiterColon = ":";
-  static const string tDelimiterAt    = "@";
-  static const string tDelimiterPound = "#";
+  static const string kDelimiterEqual = "=";
+  static const string kDelimiterAnd   = "&";
+  static const string kDelimiterComma = ",";
+  static const string kDelimiterColon = ":";
+  static const string kDelimiterAt    = "@";
+  static const string kDelimiterPound = "#";
 }
 
 class Utils {
  public:
   // trim string info
-  static string trim(const string& source);
+  static string Trim(const string& source);
   // split string to vector
-  static void split(const string& source, map<string, int>& result, 
+  static void Split(const string& source, map<string, int>& result, 
                    const string& delimiter_step1, const string& delimiter_step2);
 
 };
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/version.h b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
index cdf1444..e7ccc62 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/version.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
@@ -26,7 +26,7 @@ namespace tubemq {
 
 using namespace std;
 
-static const String tTubeMQClientVersion = "0.5.0";
+static const String kTubeMQClientVersion = "0.1.0-0.5.0";
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index 2f540fa..822c0e5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -62,7 +62,7 @@ BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
 
 bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrinfo) {
   // check parameter masterAddrInfo
-  string trimed_master_addr_info = Utils::trim(master_addrinfo);
+  string trimed_master_addr_info = Utils::Trim(master_addrinfo);
   if(trimed_master_addr_info.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
@@ -78,8 +78,8 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
   // parse and verify master address info
   // master_addrinfo's format like ip1:port1,ip2:port2,ip3:port3
   map<string, int> tgt_address_map;
-  Utils::split(master_addrinfo, tgt_address_map, 
-    delimiter::tDelimiterComma, delimiter::tDelimiterColon);
+  Utils::Split(master_addrinfo, tgt_address_map, 
+    delimiter::kDelimiterComma, delimiter::kDelimiterColon);
   if(tgt_address_map.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
@@ -93,12 +93,12 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
                 const string& trust_store_path, const string& trust_store_password) {
   this->tls_enabled_ = tls_enable;
   if(tls_enable) {
-    string trimed_trust_store_path = Utils::trim(trust_store_path);  
+    string trimed_trust_store_path = Utils::Trim(trust_store_path);  
     if(trimed_trust_store_path.empty()) {
       err_info = "Illegal parameter: trust_store_path is empty!";
       return false;
     }
-    string trimed_trust_store_password = Utils::trim(trust_store_password);  
+    string trimed_trust_store_password = Utils::Trim(trust_store_password);  
     if(trimed_trust_store_password.empty()) {
       err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
@@ -117,7 +117,7 @@ bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable,
                                        const string& usr_name, const string& usr_password) {
   this->auth_enable_ = authentic_enable;
   if(authentic_enable) {
-    string trimed_usr_name = Utils::trim(usr_name);
+    string trimed_usr_name = Utils::Trim(usr_name);
     if(trimed_usr_name.empty()) {
       err_info = "Illegal parameter: usr_name is empty!";
       return false;
@@ -232,6 +232,61 @@ string BaseConfig::ToString() {
 }
 
 
+ ConsumerConfig::ConsumerConfig() {
+   this->group_name_ = "";
+   this->is_bound_consume_ = false;
+   this->session_key_ = "";
+   this->source_count_ = -1;
+   this->is_select_big_ = true;
+   this->consume_position_ = kConsumeFromLatestOffset;
+   this->is_confirm_in_local_ = false;
+   this->is_rollback_if_confirm_timout_ = true;
+   this->max_subinfo_report_intvl_ = config::kSubInfoReportMaxIntervalTimes;
+   this->msg_notfound_wait_period_ms_ = config::kMsgNotfoundWaitPeriodMsDef;
+   this->reb_confirm_wait_period_ms_ = config::kRebConfirmWaitPeriodMsDef;
+   this->max_confirm_wait_period_ms_ = config::kConfirmWaitPeriodMsMax;
+   this->shutdown_reb_wait_period_ms_ = config::kRebWaitPeriodWhenShutdownMs;
+ }
+ 
+ ConsumerConfig::~ConsumerConfig() {
+
+ }
+
+ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
+  if(this != &target) {
+    // parent class 
+    this->master_addrinfo_ = target.master_addrinfo_;
+    this->auth_enable_    = target.auth_enable_;
+    this->auth_usrname_   = target.auth_usrname_;
+    this->auth_usrpassword_ = target.auth_usrpassword_;
+    this->tls_enabled_      = target.tls_enabled_;
+    this->tls_trust_store_path_      = target.tls_trust_store_path_;
+    this->tls_trust_store_password_  = target.tls_trust_store_password_;
+    this->rpc_read_timeout_sec_      = target.rpc_read_timeout_sec_;
+    this->heartbeat_period_sec_     = target.heartbeat_period_sec_;
+    this->max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
+    this->heartbeat_period_afterfail_sec_ = target.heartbeat_period_afterfail_sec_;    
+    // child class
+    this->group_name_ = target.group_name_;
+    this->sub_topic_and_filter_map_ = target.sub_topic_and_filter_map_;
+    this->is_bound_consume_ = target.is_bound_consume_;
+    this->session_key_ = target.session_key_;
+    this->source_count_ = target.source_count_;
+    this->is_select_big_ = target.is_select_big_;
+    this->part_offset_map_ = target.part_offset_map_;
+    this->consume_position_ = target.consume_position_;
+    this->max_subinfo_report_intvl_ = target.max_subinfo_report_intvl_;
+    this->msg_notfound_wait_period_ms_ = target.msg_notfound_wait_period_ms_;
+    this->is_confirm_in_local_ = target.is_confirm_in_local_;
+    this->is_rollback_if_confirm_timout_ = target.is_rollback_if_confirm_timout_;
+    this->reb_confirm_wait_period_ms_ = target.reb_confirm_wait_period_ms_;
+    this->max_confirm_wait_period_ms_ = target.max_confirm_wait_period_ms_;
+    this->shutdown_reb_wait_period_ms_ = target.shutdown_reb_wait_period_ms_;
+    
+  }
+  return *this;
+}
+
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index c3d1ca7..8b2d08f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -23,15 +23,16 @@
 
 namespace tubemq {
 
+static const string kWhitespaceCharSet = " \n\r\t\f\v";
 
-string Utils::trim(const string& source) {
+string Utils::Trim(const string& source) {
   string target = source;
   if(!target.empty()) {
-    size_t foud_pos = target.find_first_not_of(tWhitespaceCharSet);
+    size_t foud_pos = target.find_first_not_of(kWhitespaceCharSet);
     if (foud_pos != string::npos) {
       target = target.substr(foud_pos);
     }
-    foud_pos = target.find_last_not_of(tWhitespaceCharSet);
+    foud_pos = target.find_last_not_of(kWhitespaceCharSet);
     if(foud_pos != string::npos) {
       target = target.substr(0, foud_pos + 1);
     }
@@ -39,7 +40,7 @@ string Utils::trim(const string& source) {
   return target;
 }
 
-void Utils::split(const string& source, map<string, int>& result, 
+void Utils::Split(const string& source, map<string, int>& result, 
       const string& delimiter_step1, const string& delimiter_step2) {
   string item_str;
   string key_str;
@@ -50,7 +51,7 @@ void Utils::split(const string& source, map<string, int>& result,
     pos2 = source.find(delimiter_step1);
     while(string::npos != pos2) {
       item_str = source.substr(pos1, pos2-pos1);
-      item_str = Utils::trim(item_str);
+      item_str = Utils::Trim(item_str);
       pos1 = pos2 + delimiter_step1.length();
       pos2 = source.find(delimiter_step1, pos1);
       if(item_str.empty()) {
@@ -62,8 +63,8 @@ void Utils::split(const string& source, map<string, int>& result,
       }
       key_str = item_str.substr(0, pos3);
       val_str = item_str.substr(pos3+delimiter_step2.length());
-      key_str = Utils::trim(key_str);
-      val_str = Utils::trim(val_str);
+      key_str = Utils::Trim(key_str);
+      val_str = Utils::Trim(val_str);
       if(key_str.empty()) {
         continue;
       }
@@ -71,13 +72,13 @@ void Utils::split(const string& source, map<string, int>& result,
     }
     if(pos1 != source.length()) {
       item_str = source.substr(pos1);
-      item_str = Utils::trim(item_str);
+      item_str = Utils::Trim(item_str);
       pos3 = item_str.find(delimiter_step2);
       if(string::npos != pos3) {
         key_str = item_str.substr(0, pos3);
         val_str = item_str.substr(pos3+delimiter_step2.length());
-        key_str = Utils::trim(key_str);
-        val_str = Utils::trim(val_str);
+        key_str = Utils::Trim(key_str);
+        val_str = Utils::Trim(val_str);
         if(!key_str.empty()){
           result[key_str] = atoi(val_str.c_str());
         }


[incubator-tubemq] 14/50: [TUBEMQ-266] Add Tencent/rapidjson as submodule

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 28d5ec35902007f559b674d9b4510871880ce405
Author: Guangxu Cheng <gx...@apache.org>
AuthorDate: Fri Jul 3 21:41:55 2020 +0800

    [TUBEMQ-266] Add Tencent/rapidjson as submodule
---
 .gitmodules                                                 | 4 ++++
 pom.xml                                                     | 2 ++
 tubemq-client-twins/tubemq-client-cpp/.gitmodules           | 3 ---
 tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson | 1 +
 4 files changed, 7 insertions(+), 3 deletions(-)

diff --git a/.gitmodules b/.gitmodules
new file mode 100644
index 0000000..417049c
--- /dev/null
+++ b/.gitmodules
@@ -0,0 +1,4 @@
+[submodule "tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson"]
+	path = tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
+	url = https://github.com/Tencent/rapidjson.git
+	tag = v1.1.0
diff --git a/pom.xml b/pom.xml
index afcdb83..296ce8a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -264,6 +264,8 @@
                         <exclude>resources/assets/lib/**</exclude>
                         <exclude>resources/assets/public/**</exclude>
                         <exclude>DISCLAIMER-WIP</exclude>
+
+                        <exclude>**/tubemq-client-twins/tubemq-client-cpp/third_party/**</exclude>
                     </excludes>
                 </configuration>
             </plugin>
diff --git a/tubemq-client-twins/tubemq-client-cpp/.gitmodules b/tubemq-client-twins/tubemq-client-cpp/.gitmodules
deleted file mode 100644
index b9a6c24..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/.gitmodules
+++ /dev/null
@@ -1,3 +0,0 @@
-[submodule "third_party/jsoncpp"]
-	path = third_party/jsoncpp
-	url = https://github.com/open-source-parsers/jsoncpp.git
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson b/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
new file mode 160000
index 0000000..f54b0e4
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
@@ -0,0 +1 @@
+Subproject commit f54b0e47a08782a6131cc3d60f94d038fa6e0a51


[incubator-tubemq] 09/50: [TUBEMQ-262]Create C++ flow control handler

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 3cf5c56aaba6692eab45e5bef280d6fbaac10996
Author: gosonzhang <go...@tencent.com>
AuthorDate: Thu Jul 2 17:23:39 2020 +0800

    [TUBEMQ-262]Create C++ flow control handler
---
 tubemq-client-twins/tubemq-client-cpp/inc/const_config.h   | 2 +-
 tubemq-client-twins/tubemq-client-cpp/src/client_config.cc | 4 ++--
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc         | 2 ++
 3 files changed, 5 insertions(+), 3 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index 5ec1709..a48873d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -66,7 +66,7 @@ static const int kRebWaitPeriodWhenShutdownMs = 10000;
 // max int value
 static const int kMaxIntValue = 0x7fffffff;
 // max long value
-static const int kMaxLongValue = 0x7fffffffffffffffL;
+static const long kMaxLongValue = 0x7fffffffffffffffL;
 
 // invalid value
 static const int kInvalidValue = -2;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index f7fde92..e668f81 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -421,7 +421,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   string part_key;
   map<string, long> tmp_parts_map;
   map<string, long>::const_iterator it_part;
-  for (itPart = part_offset_map.begin(); it_part != part_offset_map.end(); ++it_part) {
+  for (it_part = part_offset_map.begin(); it_part != part_offset_map.end(); ++it_part) {
     vector<string> result;
     Utils::Split(it_part->first, result, delimiter::kDelimiterColon);
     if (result.size() != 3) {
@@ -566,7 +566,7 @@ string ConsumerConfig::ToString() {
       ss << ",";
     }
     ss << "'";
-    ss << itMap->first;
+    ss << it_map->first;
     ss << "'=[";
     int j=0;
     set<string> topic_set = it_map->second;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index f8ee3c2..9ddd3b0 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -21,6 +21,8 @@
 #include <sstream> 
 #include <regex.h>
 #include <stdlib.h>
+#include <unistd.h>
+#include <sys/time.h>
 #include "utils.h"
 #include "const_config.h"
 


[incubator-tubemq] 22/50: [TUBEMQ-272]Unified C/C++ files's code style (#193)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 87ab59ad021e3b9b21dac223d5c4f26d3f109d04
Author: gosonzhang <46...@qq.com>
AuthorDate: Wed Jul 8 03:08:15 2020 +0000

    [TUBEMQ-272]Unified C/C++ files's code style (#193)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/atomic_def.h             |  28 ++-
 .../tubemq-client-cpp/inc/client_config.h          | 108 +++++-----
 .../tubemq-client-cpp/inc/const_config.h           |  60 +++---
 .../tubemq-client-cpp/inc/const_rpc.h              |  87 ++++----
 .../tubemq-client-cpp/inc/file_ini.h               |  23 +-
 .../tubemq-client-cpp/inc/flowctrl_def.h           | 132 ++++++------
 tubemq-client-twins/tubemq-client-cpp/inc/logger.h |   3 +-
 .../tubemq-client-cpp/inc/message.h                |  44 ++--
 .../tubemq-client-cpp/inc/meta_info.h              |  81 ++++---
 .../tubemq-client-cpp/inc/singleton.h              |   2 +-
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  |  33 +--
 .../tubemq-client-cpp/inc/version.h                |  14 +-
 .../tubemq-client-cpp/src/client_config.cc         | 163 +++++++-------
 .../tubemq-client-cpp/src/file_ini.cc              |  25 +--
 .../tubemq-client-cpp/src/flowctrl_def.cc          | 237 +++++++++++----------
 .../tubemq-client-cpp/src/logger.cc                |   2 -
 .../tubemq-client-cpp/src/message.cc               |  44 ++--
 .../tubemq-client-cpp/src/meta_info.cc             |  94 ++++----
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  81 +++----
 19 files changed, 642 insertions(+), 619 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
index 24df5f3..00d0c24 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
@@ -17,23 +17,22 @@
  * under the License.
  */
 
-#ifndef _TUBEMQ_CLIENT_ATOMIC_DEF_H_
-#define _TUBEMQ_CLIENT_ATOMIC_DEF_H_
+#ifndef TUBEMQ_CLIENT_ATOMIC_DEF_H_
+#define TUBEMQ_CLIENT_ATOMIC_DEF_H_
 
 #include <stdlib.h>
 
 
-
 namespace tubemq {
 
 using namespace std;
 
 class AtomicInteger {
  public:
-  AtomicInteger(){
+  AtomicInteger() {
     this->counter_ = 0;
   }
-            
+
   AtomicInteger(int initial_value) {
     this->counter_ = initial_value;
   }
@@ -69,7 +68,7 @@ class AtomicInteger {
       int next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
-      }  
+      }
     }
   }
 
@@ -79,7 +78,7 @@ class AtomicInteger {
       int next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
-      }  
+      }
     }
   }
 
@@ -89,7 +88,7 @@ class AtomicInteger {
       int next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
-      }  
+      }
     }
   }
 
@@ -117,12 +116,12 @@ class AtomicInteger {
     for ( ; ; ) {
       int current = this->counter_;
       int next = current + delta;
-      if (__sync_bool_compare_and_swap (&this->counter_, current, next)) {
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
   }
-            
+
  private:
   volatile int counter_;
 };
@@ -217,7 +216,7 @@ class AtomicLong {
     for ( ; ; ) {
       long current = this->counter_;
       long next = current + delta;
-      if (__sync_bool_compare_and_swap (&this->counter_, current, next)) {
+      if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
@@ -267,11 +266,8 @@ class AtomicBoolean{
 };
 
 
+}  // namespace tubemq
 
 
-
-}
-
-
-#endif
+#endif  // TUBEMQ_CLIENT_ATOMIC_DEF_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
index 0dcf458..41610e1 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
@@ -16,20 +16,25 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-#ifndef _TUBEMQ_CLIENT_CONFIGURE_H_
-#define _TUBEMQ_CLIENT_CONFIGURE_H_
 
-#include <string>
+#ifndef TUBEMQ_CLIENT_CONFIGURE_H_
+#define TUBEMQ_CLIENT_CONFIGURE_H_
+
+#include <stdint.h>
 #include <stdio.h>
 #include <map>
 #include <set>
-
+#include <string>
 
 
 namespace tubemq {
 
-using namespace std;
+
+using std::map;
+using std::set;
+using std::string;
+using std::vector;
+
 
 
 class BaseConfig {
@@ -38,27 +43,27 @@ class BaseConfig {
   ~BaseConfig();
   BaseConfig& operator=(const BaseConfig& target);
   bool SetMasterAddrInfo(string& err_info, const string& master_addrinfo);
-  bool SetTlsInfo(string& err_info, bool tls_enable, 
+  bool SetTlsInfo(string& err_info, bool tls_enable,
                     const string& trust_store_path, const string& trust_store_password);
-  bool SetAuthenticInfo(string& err_info, bool authentic_enable, 
-                            const string& usr_name, const string& usr_password);
+  bool SetAuthenticInfo(string& err_info, bool authentic_enable,
+                              const string& usr_name, const string& usr_password);
   const string& GetMasterAddrInfo() const;
   bool IsTlsEnabled();
   const string& GetTrustStorePath() const;
   const string& GetTrustStorePassword() const;
   bool IsAuthenticEnabled();
   const string& GetUsrName() const;
-  const string& GetUsrPassWord() const;            
+  const string& GetUsrPassWord() const;
   // set the rpc timout, unit second, duration [8, 300], default 15 seconds;
-  void SetRpcReadTimeoutSec(int rpc_read_timeout_sec);
-  int GetRpcReadTimeoutSec();
+  void SetRpcReadTimeoutSec(int32_t rpc_read_timeout_sec);
+  int32_t GetRpcReadTimeoutSec();
   // Set the duration of the client's heartbeat cycle, in seconds, the default is 10 seconds
-  void SetHeartbeatPeriodSec(int heartbeat_period_sec);
-  int GetHeartbeatPeriodSec();
-  void SetMaxHeartBeatRetryTimes(int max_heartbeat_retry_times);
-  int GetMaxHeartBeatRetryTimes();
-  void SetHeartbeatPeriodAftFailSec(int heartbeat_period_afterfail_sec);
-  int GetHeartbeatPeriodAftFailSec();
+  void SetHeartbeatPeriodSec(int32_t heartbeat_period_sec);
+  int32_t GetHeartbeatPeriodSec();
+  void SetMaxHeartBeatRetryTimes(int32_t max_heartbeat_retry_times);
+  int32_t GetMaxHeartBeatRetryTimes();
+  void SetHeartbeatPeriodAftFailSec(int32_t heartbeat_period_afterfail_sec);
+  int32_t GetHeartbeatPeriodAftFailSec();
   string ToString();
 
  private:
@@ -72,10 +77,10 @@ class BaseConfig {
   string tls_trust_store_path_;
   string tls_trust_store_password_;
   // other setting
-  int   rpc_read_timeout_sec_;
-  int   heartbeat_period_sec_;
-  int   max_heartbeat_retry_times_;
-  int   heartbeat_period_afterfail_sec_;
+  int32_t rpc_read_timeout_sec_;
+  int32_t heartbeat_period_sec_;
+  int32_t max_heartbeat_retry_times_;
+  int32_t heartbeat_period_afterfail_sec_;
 };
 
 
@@ -83,72 +88,73 @@ enum ConsumePosition {
   kConsumeFromFirstOffset = -1,
   kConsumeFromLatestOffset = 0,
   kComsumeFromMaxOffsetAlways = 1
-};
+};  // enum ConsumePosition
 
 
 
 class ConsumerConfig : public BaseConfig {
- public: 
+ public:
   ConsumerConfig();
   ~ConsumerConfig();
-  ConsumerConfig& operator=(const ConsumerConfig& target); 
-  bool SetGroupConsumeTarget(string& err_info, 
+  ConsumerConfig& operator=(const ConsumerConfig& target);
+  bool SetGroupConsumeTarget(string& err_info,
     const string& group_name, const set<string>& subscribed_topicset);
-  bool SetGroupConsumeTarget(string& err_info, 
+  bool SetGroupConsumeTarget(string& err_info,
     const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map);
-  bool SetGroupConsumeTarget(string& err_info, 
+  bool SetGroupConsumeTarget(string& err_info,
     const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-    const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map);
+    const string& session_key, uint32_t source_count, bool is_select_big,
+    const map<string, int64_t>& part_offset_map);
   const string& GetGroupName() const;
   const map<string, set<string> >& GetSubTopicAndFilterMap() const;
   void SetConsumePosition(ConsumePosition consume_from_where);
   const ConsumePosition GetConsumePosition() const;
-  const int GetMsgNotFoundWaitPeriodMs() const;
+  const int32_t GetMsgNotFoundWaitPeriodMs() const;
   void SetMsgNotFoundWaitPeriodMs(int msg_notfound_wait_period_ms);
-  const int GetMaxSubinfoReportIntvl() const;
-  void SetMaxSubinfoReportIntvl(int max_subinfo_report_intvl);
+  const int32_t GetMaxSubinfoReportIntvl() const;
+  void SetMaxSubinfoReportIntvl(int32_t max_subinfo_report_intvl);
   bool IsConfirmInLocal();
   void SetConfirmInLocal(bool confirm_in_local);
   bool IsRollbackIfConfirmTimeout();
   void setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout);
-  const int GetWaitPeriodIfConfirmWaitRebalanceMs() const;
-  void SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_period_ms);
-  const int GetMaxConfirmWaitPeriodMs() const;
-  void SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms);
-  const int GetShutdownRebWaitPeriodMs() const;
-  void SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms);
+  const int32_t GetWaitPeriodIfConfirmWaitRebalanceMs() const;
+  void SetWaitPeriodIfConfirmWaitRebalanceMs(int32_t reb_confirm_wait_period_ms);
+  const int32_t GetMaxConfirmWaitPeriodMs() const;
+  void SetMaxConfirmWaitPeriodMs(int32_t max_confirm_wait_period_ms);
+  const int32_t GetShutdownRebWaitPeriodMs() const;
+  void SetShutdownRebWaitPeriodMs(int32_t wait_period_when_shutdown_ms);
   string ToString();
 
  private:
   bool setGroupConsumeTarget(string& err_info, bool is_bound_consume,
     const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-    const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map);
-    
-  
- private: 
+    const string& session_key, int32_t source_count, bool is_select_big,
+    const map<string, int64_t>& part_offset_map);
+
+ private:
   string group_name_;
   map<string, set<string> > sub_topic_and_filter_map_;
   bool is_bound_consume_;
   string session_key_;
-  int source_count_;
+  uint32_t source_count_;
   bool is_select_big_;
-  map<string, long> part_offset_map_;
+  map<string, int64_t> part_offset_map_;
   ConsumePosition consume_position_;
-  int max_subinfo_report_intvl_;
-  int msg_notfound_wait_period_ms_;
+  int32_t max_subinfo_report_intvl_;
+  int32_t msg_notfound_wait_period_ms_;
   bool is_confirm_in_local_;
   bool is_rollback_if_confirm_timout_;
-  int reb_confirm_wait_period_ms_;
-  int max_confirm_wait_period_ms_;
-  int shutdown_reb_wait_period_ms_;
+  int32_t reb_confirm_wait_period_ms_;
+  int32_t max_confirm_wait_period_ms_;
+  int32_t shutdown_reb_wait_period_ms_;
 };
 
 
 
 
-}
+}  // namespace tubemq
 
-#endif
+#endif  // TUBEMQ_CLIENT_CONFIGURE_H_
 
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index 4e04ec0..9e72aaa 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -16,61 +16,62 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_CONST_CONFIG_H_
-#define _TUBEMQ_CLIENT_CONST_CONFIG_H_
 
+#ifndef TUBEMQ_CLIENT_CONST_CONFIG_H_
+#define TUBEMQ_CLIENT_CONST_CONFIG_H_
+
+#include <stdint.h>
 #include <map>
 #include <string>
 
 namespace tubemq {
 
-using namespace std;
+using std::string;
+
 
 // configuration value setting
 namespace config {
 // heartbeat period define
-static const int kHeartBeatPeriodDef = 10;
-static const int kHeartBeatFailRetryTimesDef = 5;
-static const int kHeartBeatSleepPeriodDef = 60;
+static const int32_t kHeartBeatPeriodDef = 10;
+static const int32_t kHeartBeatFailRetryTimesDef = 5;
+static const int32_t kHeartBeatSleepPeriodDef = 60;
 // max masterAddrInfo length
-static const int kMasterAddrInfoMaxLength = 1024;
+static const int32_t kMasterAddrInfoMaxLength = 1024;
 
 // max TopicName length
-static const int kTopicNameMaxLength = 64;
+static const int32_t kTopicNameMaxLength = 64;
 // max Consume GroupName length
-static const int kGroupNameMaxLength = 1024;
+static const int32_t kGroupNameMaxLength = 1024;
 // max filter item length
-static const int kFilterItemMaxLength = 256;
+static const int32_t kFilterItemMaxLength = 256;
 // max allowed filter item count
-static const int kFilterItemMaxCount = 500;
+static const int32_t kFilterItemMaxCount = 500;
 // max session key length
-static const int kSessionKeyMaxLength = 1024;
+static const int32_t kSessionKeyMaxLength = 1024;
 
 // max subscribe info report times
-static const int kSubInfoReportMaxIntervalTimes = 6;
+static const int32_t kSubInfoReportMaxIntervalTimes = 6;
 // default message not found response wait period
-static const int kMsgNotfoundWaitPeriodMsDef = 200;
+static const int32_t kMsgNotfoundWaitPeriodMsDef = 200;
 // default confirm wait period if rebalance meeting
-static const int kRebConfirmWaitPeriodMsDef = 3000;
+static const int32_t kRebConfirmWaitPeriodMsDef = 3000;
 // max confirm wait period anyway
-static const int kConfirmWaitPeriodMsMax = 60000;
+static const int32_t kConfirmWaitPeriodMsMax = 60000;
 // default rebalance wait if shutdown meeting
-static const int kRebWaitPeriodWhenShutdownMs = 10000;
+static const int32_t kRebWaitPeriodWhenShutdownMs = 10000;
 
 // max int value
-static const int kMaxIntValue = 0x7fffffff;
+static const int32_t kMaxIntValue = 0x7fffffff;
 // max long value
-static const long kMaxLongValue = 0x7fffffffffffffffL;
+static const int64_t kMaxLongValue = 0x7fffffffffffffffL;
 
 // default broker port
-static const int kBrokerPortDef = 8123;
+static const uint32_t kBrokerPortDef = 8123;
 // default broker TLS port
-static const int kBrokerTlsPortDef = 8124;
+static const uint32_t kBrokerTlsPortDef = 8124;
 
 // invalid value
-static const int kInvalidValue = -2;
-
+static const int32_t kInvalidValue = -2;
 
 }  // namespace config
 
@@ -84,18 +85,17 @@ namespace delimiter {
   static const string kDelimiterAt    = "@";
   static const string kDelimiterPound = "#";
   static const string kDelimiterSemicolon = ";";
-  //Double slash
+  // Double slash
   static const string kDelimiterDbSlash = "//";
   // left square bracket
   static const string kDelimiterLftSB = "[";
   // right square bracket
   static const string kDelimiterRgtSB = "]";
-  
-} // namespace delimiter
+
+}  // namespace delimiter
 
 
- 
-}
+}  // namespace tubemq
 
-#endif
+#endif  // TUBEMQ_CLIENT_CONST_CONFIG_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
index cb89eba..2bc0f30 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
@@ -16,68 +16,73 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_CONST_RPC_H_
-#define _TUBEMQ_CLIENT_CONST_RPC_H_
+
+#ifndef TUBEMQ_CLIENT_CONST_RPC_H_
+#define TUBEMQ_CLIENT_CONST_RPC_H_
 
 namespace tubemq {
 
-using namespace std;
+#include <stdint.h>
 
 
 namespace rpc_config {
 
   // constant define
-  static const int kRpcPrtBeginToken    = 0xFF7FF4FE;
-  static const int kRpcMaxBufferSize    = 8192;
-  static const int kRpcMaxFrameListCnt  = (int) ((1024 * 1024 * 8) / kRpcMaxBufferSize);
+  static const uint32_t kRpcPrtBeginToken    = 0xFF7FF4FE;
+  static const uint32_t kRpcMaxBufferSize    = 8192;
+  static const uint32_t kRpcMaxFrameListCnt  = (uint32_t) ((1024 * 1024 * 8) / kRpcMaxBufferSize);
+
   // rpc protocol version
-  static const int kRpcProtocolVersion  = 2;
+  static const uint32_t kRpcProtocolVersion  = 2;
+
   // msg type flag
-  static const int kRpcFlagMsgRequest   = 0x0;
-  static const int kRpcFlagMsgResponse  = 0x1;
+  static const int32_t  kRpcFlagMsgRequest   = 0x0;
+  static const int32_t  kRpcFlagMsgResponse  = 0x1;
+
   // service type
-  static const int kMasterService      = 1;
-  static const int kBrokerReadService  = 2;
-  static const int kBrokerWriteService = 3;
-  static const int kBrokerAdminService = 4;
-  static const int kMasterAdminService = 5;
+  static const int32_t kMasterService      = 1;
+  static const int32_t kBrokerReadService  = 2;
+  static const int32_t kBrokerWriteService = 3;
+  static const int32_t kBrokerAdminService = 4;
+  static const int32_t kMasterAdminService = 5;
+
   // request method
   // master rpc method
-  static const int kMasterMethoddProducerRegister = 1;
-  static const int kMasterMethoddProducerHeatbeat = 2;
-  static const int kMasterMethoddProducerClose    = 3;
-  static const int kMasterMethoddConsumerRegister = 4;
-  static const int kMasterMethoddConsumerHeatbeat = 5;
-  static const int kMasterMethoddConsumerClose    = 6;
+  static const int32_t kMasterMethoddProducerRegister = 1;
+  static const int32_t kMasterMethoddProducerHeatbeat = 2;
+  static const int32_t kMasterMethoddProducerClose    = 3;
+  static const int32_t kMasterMethoddConsumerRegister = 4;
+  static const int32_t kMasterMethoddConsumerHeatbeat = 5;
+  static const int32_t kMasterMethoddConsumerClose    = 6;
+
   // broker rpc method
-  static const int kBrokerMethoddProducerRegister    = 11;
-  static const int kBrokerMethoddProducerHeatbeat    = 12;
-  static const int kBrokerMethoddProducerSendMsg     = 13;
-  static const int kBrokerMethoddProducerClose       = 14;
-  static const int kBrokerMethoddConsumerRegister    = 15;
-  static const int kBrokerMethoddConsumerHeatbeat    = 16;
-  static const int kBrokerMethoddConsumerGetMsg      = 17;
-  static const int kBrokerMethoddConsumerCommit      = 18;  
-  static const int kBrokerMethoddConsumerClose       = 19; 
+  static const int32_t kBrokerMethoddProducerRegister    = 11;
+  static const int32_t kBrokerMethoddProducerHeatbeat    = 12;
+  static const int32_t kBrokerMethoddProducerSendMsg     = 13;
+  static const int32_t kBrokerMethoddProducerClose       = 14;
+  static const int32_t kBrokerMethoddConsumerRegister    = 15;
+  static const int32_t kBrokerMethoddConsumerHeatbeat    = 16;
+  static const int32_t kBrokerMethoddConsumerGetMsg      = 17;
+  static const int32_t kBrokerMethoddConsumerCommit      = 18;
+  static const int32_t kBrokerMethoddConsumerClose       = 19;
 
   // register operate type
-  static const int kRegOpTypeRegister       = 31; 
-  static const int kRegOpTypeUnReg          = 32; 
+  static const int32_t kRegOpTypeRegister       = 31;
+  static const int32_t kRegOpTypeUnReg          = 32;
 
   // rpc connect node timeout
-  static const int kRpcConnectTimeoutMs    = 3000;
-  
-  // rpc timeout define  
-  static const int kRpcTimoutDefSec = 15;
-  static const int kRpcTimoutMaxSec = 300;
-  static const int kRpcTimoutMinSec = 8;
+  static const int32_t kRpcConnectTimeoutMs    = 3000;
+
+  // rpc timeout define
+  static const int32_t kRpcTimoutDefSec = 15;
+  static const int32_t kRpcTimoutMaxSec = 300;
+  static const int32_t kRpcTimoutMinSec = 8;
 
 
-}
+}  // namespace rpc_config
 
 
-}
+}  // namespace tubemq
 
-#endif
+#endif  // TUBEMQ_CLIENT_CONST_RPC_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
index cab7f84..39f9a36 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
@@ -16,26 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_FILE_INI_H_
-#define _TUBEMQ_CLIENT_FILE_INI_H_
 
+#ifndef TUBEMQ_CLIENT_FILE_INI_H_
+#define TUBEMQ_CLIENT_FILE_INI_H_
+
+#include <stdint.h>
 #include <map>
 #include <string>
 
+
 namespace tubemq {
 
-using namespace std;
+using std::string;
+using std::map;
+
 
 class Fileini {
  public:
   Fileini();
   ~Fileini();
   bool Loadini(string& err_info, const string& file_name);
-  bool GetValue(string& err_info, const string& sector, 
+  bool GetValue(string& err_info, const string& sector,
                    const string& key, string& value, const string& def);
-  bool GetValue(string& err_info, const string& sector, 
-                   const string& key, int& value, const int def);  
+  bool GetValue(string& err_info, const string& sector,
+                   const string& key, int32_t& value, const int32_t def);
 
  private:
   bool init_flag_;
@@ -43,8 +47,7 @@ class Fileini {
   map<string, map<string, string> > ini_map_;
 };
 
-  
-}
+}  // namespace tubemq
 
-#endif
+#endif  // TUBEMQ_CLIENT_FILE_INI_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
index cff7cc6..5c39352 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
@@ -17,108 +17,113 @@
  * under the License.
  */
 
-#ifndef _TUBEMQ_CLIENT_FLOW_CONTROL_H_
-#define _TUBEMQ_CLIENT_FLOW_CONTROL_H_
+#ifndef TUBEMQ_CLIENT_FLOW_CONTROL_H_
+#define TUBEMQ_CLIENT_FLOW_CONTROL_H_
 
-#include <map>
+#include <stdint.h>
+#include <algorithm>
 #include <list>
+#include <map>
 #include <string>
 #include <vector>
-#include <algorithm>
-#include <rapidjson/document.h>
 #include "atomic_def.h"
+#include <rapidjson/document.h>
 
 
 
 namespace tubemq {
 
-using namespace std;
+
+using std::map;
+using std::string;
+using std::vector;
+
 
 class FlowCtrlResult {
  public:
   FlowCtrlResult();
-  FlowCtrlResult(long datasize_limit, int freqms_limit);
+  FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit);
   FlowCtrlResult& operator=(const FlowCtrlResult& target);
-  void SetDataDltAndFreqLimit(long datasize_limit, int freqms_limit);
-  void SetDataSizeLimit(long datasize_limit);
-  void SetFreqMsLimit(int freqms_limit);
-  long GetDataSizeLimit();
-  int GetFreqMsLimit();
+  void SetDataDltAndFreqLimit(int64_t datasize_limit, int32_t freqms_limit);
+  void SetDataSizeLimit(int64_t datasize_limit);
+  void SetFreqMsLimit(int32_t freqms_limit);
+  int64_t GetDataSizeLimit();
+  int32_t GetFreqMsLimit();
 
  private:
-  long datasize_limit_;  
-  int  freqms_limit_;
+  int64_t datasize_limit_;
+  int32_t freqms_limit_;
 };
 
 
 class FlowCtrlItem {
  public:
   FlowCtrlItem();
-  FlowCtrlItem(int type,int zero_cnt,int freqms_limit);
-  FlowCtrlItem(int type, 
-    int datasize_limit,int freqms_limit,int min_data_filter_freqms);
-  FlowCtrlItem(int type, int start_time, 
-    int end_time, long datadlt_m, long datasize_limit, int freqms_limit);
+  FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit);
+  FlowCtrlItem(int32_t type, int32_t datasize_limit,
+    int32_t freqms_limit, int32_t min_data_filter_freqms);
+  FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time,
+    int64_t datadlt_m, int64_t datasize_limit, int32_t freqms_limit);
   FlowCtrlItem& operator=(const FlowCtrlItem& target);
   void Clear();
-  void ResetFlowCtrlValue(int type, 
-    int datasize_limit,int freqms_limit,int min_data_filter_freqms);
-  int GetFreLimit(int msg_zero_cnt);
-  bool GetDataLimit(long datadlt_m, int curr_time, FlowCtrlResult& flowctrl_result);
-  const int GetType() const {
+  void ResetFlowCtrlValue(int32_t type,
+    int32_t datasize_limit, int32_t freqms_limit, int32_t min_data_filter_freqms);
+  int32_t GetFreLimit(int32_t msg_zero_cnt);
+  bool GetDataLimit(int64_t datadlt_m, int32_t curr_time, FlowCtrlResult& flowctrl_result);
+  const int32_t GetType() const {
     return type_;
   }
-  const int GetZeroCnt() const {
+  const int32_t GetZeroCnt() const {
     return zero_cnt_;
   }
-  const int GetStartTime() const {
+  const int32_t GetStartTime() const {
     return start_time_;
   }
-  const int GetEndTime() const {
+  const int32_t GetEndTime() const {
     return end_time_;
   }
-  const long GetDataSizeLimit() const { 
+  const int64_t GetDataSizeLimit() const {
     return datasize_limit_;
   }
-  const int GetFreqMsLimit() const {
+  const int32_t GetFreqMsLimit() const {
     return freqms_limit_;
   }
-  const long GetDltInM() const {
+  const int64_t GetDltInM() const {
     return datadlt_m_;
   }
 
  private:
-  int  type_;
-  int  start_time_;
-  int  end_time_;
-  long datadlt_m_;
-  long datasize_limit_;
-  int  freqms_limit_;
-  int  zero_cnt_;
+  int32_t  type_;
+  int32_t  start_time_;
+  int32_t  end_time_;
+  int64_t datadlt_m_;
+  int64_t datasize_limit_;
+  int32_t  freqms_limit_;
+  int32_t  zero_cnt_;
 };
 
 class FlowCtrlRuleHandler {
  public:
   FlowCtrlRuleHandler();
   ~FlowCtrlRuleHandler();
-  void UpdateDefFlowCtrlInfo(bool is_default, 
-    int qrypriority_id, long flowctrl_id, const string& flowctrl_info);
-  bool GetCurDataLimit(long last_datadlt,FlowCtrlResult& flowctrl_result);
-  int GetCurFreqLimitTime(int msg_zero_cnt, int received_limit);
-  int GetMinZeroCnt() { return this->min_zero_cnt_.Get();}
-  int GetQryPriorityId() { 
+  void UpdateDefFlowCtrlInfo(bool is_default,
+    int32_t qrypriority_id, int64_t flowctrl_id, const string& flowctrl_info);
+  bool GetCurDataLimit(int32_t last_datadlt, FlowCtrlResult& flowctrl_result);
+  int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit);
+  int32_t GetMinZeroCnt() { return this->min_zero_cnt_.Get();}
+  int32_t GetQryPriorityId() {
     return this->qrypriority_id_.Get();
   }
-  void SetQryPriorityId(int qrypriority_id) { 
+  void SetQryPriorityId(int32_t qrypriority_id) {
     this->qrypriority_id_.Set(qrypriority_id);
   }
-  long GetFlowCtrlId() { 
+  int64_t GetFlowCtrlId() {
     return this->flowctrl_id_.Get();
   }
   const FlowCtrlItem& GetFilterCtrlItem() const {
     return this->filter_ctrl_item_;
   }
-  const string& GetFlowCtrlInfo() const { 
+  const string& GetFlowCtrlInfo() const {
     return this->flowctrl_info_;
   }
 
@@ -127,17 +132,22 @@ class FlowCtrlRuleHandler {
   void clearStatisData();
   static bool compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2);
   static bool compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2);
-  bool parseStringMember(string &err_info, const rapidjson::Value& root, 
+  bool parseStringMember(string &err_info, const rapidjson::Value& root,
     const char* key, string& value, bool compare_value, string required_val);
-  bool parseLongMember(string &err_info, const rapidjson::Value& root, 
-    const char* key, long& value, bool compare_value, long required_val);
-  bool parseIntMember(string &err_info, const rapidjson::Value& root, 
-    const char* key, int& value, bool compare_value, int required_val);
-  bool parseFlowCtrlInfo(const string& flowctrl_info, map<int,vector<FlowCtrlItem> >& flowctrl_info_map);
-  bool parseDataLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowCtrlItems);
-  bool parseFreqLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
-  bool parseLowFetchLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
-  bool parseTimeMember(string& err_info, const rapidjson::Value& root, const char* key, int& value);
+  bool parseLongMember(string &err_info, const rapidjson::Value& root,
+    const char* key, int64_t& value, bool compare_value, int64_t required_val);
+  bool parseIntMember(string &err_info, const rapidjson::Value& root,
+    const char* key, int32_t& value, bool compare_value, int32_t required_val);
+  bool parseFlowCtrlInfo(const string& flowctrl_info,
+                      map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map);
+  bool parseDataLimit(string& err_info,
+            const rapidjson::Value& root, vector<FlowCtrlItem>& flowCtrlItems);
+  bool parseFreqLimit(string& err_info,
+            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
+  bool parseLowFetchLimit(string& err_info,
+            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
+  bool parseTimeMember(string& err_info,
+            const rapidjson::Value& root, const char* key, int32_t& value);
 
  private:
   AtomicLong    flowctrl_id_;
@@ -148,15 +158,13 @@ class FlowCtrlRuleHandler {
   AtomicInteger datalimit_start_time_;
   AtomicInteger datalimit_end_time_;
   FlowCtrlItem  filter_ctrl_item_;
-  map<int, vector<FlowCtrlItem> > flowctrl_rules_;
+  map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
   pthread_rwlock_t configrw_lock_;
-  long last_update_time_;
+  int64_t last_update_time_;
 };
 
-  
-
-}
+}  // namespace tubemq
 
 
-#endif
+#endif  // TUBEMQ_CLIENT_FLOW_CONTROL_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
index 8c5f9b3..c2f2e7d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
@@ -52,8 +52,7 @@ Logger& GetLogger();
 
 class Logger {
  public:
-  enum Level
-  {
+  enum Level {
     kTrace = 0,
     kDebug = 1,
     kInfo = 2,
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/message.h b/tubemq-client-twins/tubemq-client-cpp/inc/message.h
index 20d7fcf..5c8be2c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/message.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/message.h
@@ -16,63 +16,63 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_MESSAGE_H_
-#define _TUBEMQ_CLIENT_MESSAGE_H_
 
+#ifndef TUBEMQ_CLIENT_MESSAGE_H_
+#define TUBEMQ_CLIENT_MESSAGE_H_
 
+#include <stdint.h>
+#include <stdio.h>
 #include <list>
 #include <map>
 #include <string>
-#include <stdio.h>
 
 
 namespace tubemq {
 
-using namespace std;
+using std::map;
+using std::string;
+
+
 
 class Message {
  public:
   Message();
   Message(const Message& target);
-  Message(const string& topic, const char* data, int datalen);
+  Message(const string& topic, const char* data, uint32_t datalen);
   virtual ~Message();
   Message& operator=(const Message& target);
-  const long GetMessageId() const;
-  void SetMessageId(long message_id);
+  const int64_t GetMessageId() const;
+  void SetMessageId(int64_t message_id);
   const string& GetTopic() const;
   void SetTopic(const string& topic);
   const char* GetData() const;
-  int GetDataLength() const;
+  uint32_t GetDataLength() const;
   void setData(const char* data, int datalen);
-  const int GetFlag() const;
-  void SetFlag(int flag);
+  const int32_t GetFlag() const;
+  void SetFlag(int32_t flag);
   const map<string, string>& GetProperties() const;
-  int GetProperties(string& attribute);
+  int32_t GetProperties(string& attribute);
   bool HasProperty(const string& key);
   bool GetProperty(const string& key, string& value);
   bool GetFilterItem(string& value);
-  bool AddProperty(string& err_info, const string& key, const string& value);  
+  bool AddProperty(string& err_info, const string& key, const string& value);
 
  private:
   void clearData();
-  void copyData(const char* data, int datalen);  
+  void copyData(const char* data, uint32_t datalen);
   void copyProperties(const map<string, string>& properties);
 
-  
  private:
   string topic_;
   char* data_;
-  int   datalen_;
-  long  message_id_;
-  int   flag_;  
+  uint32_t datalen_;
+  int64_t  message_id_;
+  int32_t  flag_;
   map<string, string> properties_;
 };
 
-}
-
-
+}  // namespace tubemq
 
 
-#endif
+#endif  // TUBEMQ_CLIENT_MESSAGE_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
index 6264657..dcc34b0 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
@@ -16,46 +16,50 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_META_INFO_H_
-#define _TUBEMQ_CLIENT_META_INFO_H_
 
+#ifndef TUBEMQ_CLIENT_META_INFO_H_
+#define TUBEMQ_CLIENT_META_INFO_H_
+
+#include <stdint.h>
 #include <list>
 #include <string>
+#include "flowctrl_def.h"
 
-namespace tubemq {
 
-using namespace std;
+namespace tubemq {
 
+using std::list;
+using std::map;
+using std::string;
 
 
 class NodeInfo {
  public:
   NodeInfo();
   NodeInfo(bool is_broker, const string& node_info);
-  NodeInfo(const string& node_host, int node_port);
-  NodeInfo(int node_id, const string& node_host, int node_port);
+  NodeInfo(const string& node_host, uint32_t node_port);
+  NodeInfo(int32_t node_id, const string& node_host, uint32_t node_port);
   ~NodeInfo();
   NodeInfo& operator=(const NodeInfo& target);
   bool operator== (const NodeInfo& target);
   bool operator< (const NodeInfo& target) const;
-  const int GetNodeId() const;
+  const uint32_t GetNodeId() const;
   const string& GetHost() const;
-  const int GetPort() const;
+  const uint32_t GetPort() const;
   const string& GetAddrInfo() const;
   const string& GetNodeInfo() const;
-      
+
  private:
   void buildStrInfo();
 
- private: 
-  int    node_id_;
-  string node_host_;
-  int    node_port_;
+ private:
+  uint32_t node_id_;
+  string   node_host_;
+  uint32_t node_port_;
   // ip:port
-  string addr_info_;
+  string  addr_info_;
   // id:ip:port
-  string node_info_;
+  string  node_info_;
 };
 
 
@@ -64,17 +68,17 @@ class Partition {
   Partition();
   Partition(const string& partition_info);
   Partition(const NodeInfo& broker_info, const string& part_str);
-  Partition(const NodeInfo& broker_info, const string& topic, int partition_id);
+  Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id);
   ~Partition();
   Partition& operator=(const Partition& target);
   bool operator== (const Partition& target);
-  const int GetBrokerId() const;
+  const uint32_t GetBrokerId() const;
   const string& GetBrokerHost() const;
-  const int GetBrokerPort() const;
+  const uint32_t GetBrokerPort() const;
   const string& GetPartitionKey() const;
   const string& GetTopic() const;
   const NodeInfo& GetBrokerInfo() const;
-  const int GetPartitionId() const;
+  const uint32_t GetPartitionId() const;
   const string& ToString() const;
 
  private:
@@ -83,7 +87,7 @@ class Partition {
  private:
   string   topic_;
   NodeInfo broker_info_;
-  int      partition_id_;   
+  uint32_t partition_id_;
   string   partition_key_;
   string   partition_info_;
 };
@@ -97,11 +101,11 @@ class SubscribeInfo {
   const string& GetConsumerId() const;
   const string& GetGroup() const;
   const Partition& GetPartition() const;
-  const int GgetBrokerId() const;
+  const uint32_t GgetBrokerId() const;
   const string& GetBrokerHost() const;
-  const int GetBrokerPort() const;
+  const uint32_t GetBrokerPort() const;
   const string& GetTopic() const;
-  const int GetPartitionId() const;
+  const uint32_t GetPartitionId() const;
   const string& ToString() const;
 
  private:
@@ -119,36 +123,31 @@ class ConsumerEvent {
  public:
   ConsumerEvent();
   ConsumerEvent(const ConsumerEvent& target);
-  ConsumerEvent(long rebalance_id,int event_type, 
-    const list<SubscribeInfo>& subscribeInfo_lst, int event_status);
+  ConsumerEvent(int64_t rebalance_id, int32_t event_type,
+    const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status);
   ConsumerEvent& operator=(const ConsumerEvent& target);
-  const long GetRebalanceId() const;
-  const int  GetEventType() const;
-  const int  GetEventStatus() const;
-  void SetEventType(int event_type);
-  void SetEventStatus(int event_status);
+  const int64_t GetRebalanceId() const;
+  const int32_t GetEventType() const;
+  const int32_t  GetEventStatus() const;
+  void SetEventType(int32_t event_type);
+  void SetEventStatus(int32_t event_status);
   const list<SubscribeInfo>& GetSubscribeInfoList() const;
   string ToString();
 
  private:
-  long rebalance_id_;
-  int  event_type_;
-  int  event_status_;
+  int64_t rebalance_id_;
+  int32_t event_type_;
+  int32_t event_status_;
   list<SubscribeInfo> subscribe_list_;
 };
 
-
 class PartitionExt : public Partition {
   PartitionExt();
   
 
 };
 
+}  // namespace tubemq
 
-
-
-
-}
-
-#endif
+#endif  // TUBEMQ_CLIENT_META_INFO_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h b/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
index af3b0d4..734f4fb 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
@@ -20,8 +20,8 @@
 #ifndef _TUBEMQ_SINGLETON_H
 #define _TUBEMQ_SINGLETON_H
 
-#include <thread>
 #include <mutex>
+#include <thread>
 #include <assert.h>
 #include <stdlib.h>
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index efb22d7..707c1fd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -16,10 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_UTILS_H_
-#define _TUBEMQ_CLIENT_UTILS_H_
 
+#ifndef TUBEMQ_CLIENT_UTILS_H_
+#define TUBEMQ_CLIENT_UTILS_H_
+
+#include <stdint.h>
 #include <map>
 #include <string>
 #include <vector>
@@ -27,8 +28,9 @@
 
 namespace tubemq {
 
-using namespace std;
-
+using std::map;
+using std::string;
+using std::vector;
 
 
 class Utils {
@@ -38,23 +40,22 @@ class Utils {
   // split string to vector
   static void Split(const string& source, vector<string>& result, const string& delimiter);
   // split string to map<string, int>
-  static void Split(const string& source, map<string, int>& result, 
+  static void Split(const string& source, map<string, int>& result,
                    const string& delimiter_step1, const string& delimiter_step2);
   static void Join(const vector<string>& vec, const string& delimiter, string& target);
   static bool ValidString(string& err_info, const string& source,
                    bool allow_empty, bool pat_match, bool check_max_length, unsigned int maxlen);
-  static bool ValidGroupName(string &err_info, 
+  static bool ValidGroupName(string &err_info,
                    const string& group_name, string& tgt_group_name);
-  static bool ValidFilterItem(string& err_info, 
+  static bool ValidFilterItem(string& err_info,
                    const string& src_filteritem, string& tgt_filteritem);
-  static string Int2str(int data);
-  static string Long2str(long data);
-  static int IpToInt(const string& ipv4_addr);
-  static long GetCurrentTimeMillis();
-
+  static string Int2str(int32_t data);
+  static string Long2str(int64_t data);
+  static uint32_t IpToInt(const string& ipv4_addr);
+  static int64_t GetCurrentTimeMillis();
 };
- 
-}
 
-#endif
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_UTILS_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/version.h b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
index e7ccc62..848a5c8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/version.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
@@ -16,19 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-      
-#ifndef _TUBEMQ_CLIENT_VERSION_H_
-#define _TUBEMQ_CLIENT_VERSION_H_
+
+#ifndef TUBEMQ_CLIENT_VERSION_H_
+#define TUBEMQ_CLIENT_VERSION_H_
 
 #include <string>
 
 namespace tubemq {
 
-using namespace std;
+using std::string;
 
-static const String kTubeMQClientVersion = "0.1.0-0.5.0";
+static const string kTubeMQClientVersion = "0.1.0-0.5.0";
 
-}
+}  // namespace tubemq
 
-#endif
+#endif  // TUBEMQ_CLIENT_VERSION_H_
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index d3a03ba..3c88eb2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -17,12 +17,11 @@
  * under the License.
  */
 
-#include <sstream> 
-#include <vector>
 #include "client_config.h"
+#include <sstream>
+#include <vector>
 #include "const_config.h"
 #include "const_rpc.h"
-
 #include "utils.h"
 
 
@@ -44,7 +43,7 @@ BaseConfig::BaseConfig() {
 }
 
 BaseConfig::~BaseConfig() {
-
+  //
 }
 
 BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
@@ -75,14 +74,14 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
     stringstream ss;
     ss << "Illegal parameter: over max ";
     ss << config::kMasterAddrInfoMaxLength;
-    ss << " length of master_addrinfo parameter!";   
+    ss << " length of master_addrinfo parameter!";
     err_info = ss.str();
     return false;
   }
   // parse and verify master address info
   // master_addrinfo's format like ip1:port1,ip2:port2,ip3:port3
   map<string, int> tgt_address_map;
-  Utils::Split(master_addrinfo, tgt_address_map, 
+  Utils::Split(master_addrinfo, tgt_address_map,
     delimiter::kDelimiterComma, delimiter::kDelimiterColon);
   if (tgt_address_map.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
@@ -97,27 +96,27 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
                 const string& trust_store_path, const string& trust_store_password) {
   this->tls_enabled_ = tls_enable;
   if (tls_enable) {
-    string trimed_trust_store_path = Utils::Trim(trust_store_path);  
+    string trimed_trust_store_path = Utils::Trim(trust_store_path);
     if (trimed_trust_store_path.empty()) {
       err_info = "Illegal parameter: trust_store_path is empty!";
       return false;
     }
-    string trimed_trust_store_password = Utils::Trim(trust_store_password);  
+    string trimed_trust_store_password = Utils::Trim(trust_store_password);
     if (trimed_trust_store_password.empty()) {
       err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
     }
-      this->tls_trust_store_path_= trimed_trust_store_path;
-      this->tls_trust_store_password_= trimed_trust_store_password;    
+      this->tls_trust_store_path_ = trimed_trust_store_path;
+      this->tls_trust_store_password_ = trimed_trust_store_password;
   } else {
     this->tls_trust_store_path_ = "";
     this->tls_trust_store_password_ = "";
   }
   err_info = "Ok";
-  return true;  
+  return true;
 }
 
-bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, 
+bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable,
                 const string& usr_name, const string& usr_password) {
   this->auth_enable_ = authentic_enable;
   if (authentic_enable) {
@@ -126,7 +125,7 @@ bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable,
       err_info = "Illegal parameter: usr_name is empty!";
       return false;
     }
-    string trimed_usr_password = Utils::Trim(usr_password);  
+    string trimed_usr_password = Utils::Trim(usr_password);
     if (trimed_usr_password.empty()) {
       err_info = "Illegal parameter: usr_password is empty!";
       return false;
@@ -179,31 +178,31 @@ void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
   }
 }
 
-int BaseConfig::GetRpcReadTimeoutSec() {
+int32_t BaseConfig::GetRpcReadTimeoutSec() {
   return this->rpc_read_timeout_sec_;
 }
 
-void BaseConfig::SetHeartbeatPeriodSec(int heartbeat_period_sec) {
+void BaseConfig::SetHeartbeatPeriodSec(int32_t heartbeat_period_sec) {
   this->heartbeat_period_sec_ = heartbeat_period_sec;
 }
 
-int BaseConfig::GetHeartbeatPeriodSec() {
+int32_t BaseConfig::GetHeartbeatPeriodSec() {
   return this->heartbeat_period_sec_;
 }
 
-void BaseConfig::SetMaxHeartBeatRetryTimes(int max_heartbeat_retry_times) {
+void BaseConfig::SetMaxHeartBeatRetryTimes(int32_t max_heartbeat_retry_times) {
   this->max_heartbeat_retry_times_ = max_heartbeat_retry_times;
 }
 
-int BaseConfig::GetMaxHeartBeatRetryTimes() {
+int32_t BaseConfig::GetMaxHeartBeatRetryTimes() {
   return this->max_heartbeat_retry_times_;
 }
 
-void BaseConfig::SetHeartbeatPeriodAftFailSec(int heartbeat_period_afterfail_sec) {
+void BaseConfig::SetHeartbeatPeriodAftFailSec(int32_t heartbeat_period_afterfail_sec) {
   this->heartbeat_period_afterfail_sec_ = heartbeat_period_afterfail_sec;
 }
 
-int BaseConfig::GetHeartbeatPeriodAftFailSec() {
+int32_t BaseConfig::GetHeartbeatPeriodAftFailSec() {
   return this->heartbeat_period_afterfail_sec_;
 }
 
@@ -235,30 +234,29 @@ string BaseConfig::ToString() {
   return ss.str();
 }
 
+ConsumerConfig::ConsumerConfig() {
+  this->group_name_ = "";
+  this->is_bound_consume_ = false;
+  this->session_key_ = "";
+  this->source_count_ = 0;
+  this->is_select_big_ = true;
+  this->consume_position_ = kConsumeFromLatestOffset;
+  this->is_confirm_in_local_ = false;
+  this->is_rollback_if_confirm_timout_ = true;
+  this->max_subinfo_report_intvl_ = config::kSubInfoReportMaxIntervalTimes;
+  this->msg_notfound_wait_period_ms_ = config::kMsgNotfoundWaitPeriodMsDef;
+  this->reb_confirm_wait_period_ms_ = config::kRebConfirmWaitPeriodMsDef;
+  this->max_confirm_wait_period_ms_ = config::kConfirmWaitPeriodMsMax;
+  this->shutdown_reb_wait_period_ms_ = config::kRebWaitPeriodWhenShutdownMs;
+}
 
- ConsumerConfig::ConsumerConfig() {
-   this->group_name_ = "";
-   this->is_bound_consume_ = false;
-   this->session_key_ = "";
-   this->source_count_ = -1;
-   this->is_select_big_ = true;
-   this->consume_position_ = kConsumeFromLatestOffset;
-   this->is_confirm_in_local_ = false;
-   this->is_rollback_if_confirm_timout_ = true;
-   this->max_subinfo_report_intvl_ = config::kSubInfoReportMaxIntervalTimes;
-   this->msg_notfound_wait_period_ms_ = config::kMsgNotfoundWaitPeriodMsDef;
-   this->reb_confirm_wait_period_ms_ = config::kRebConfirmWaitPeriodMsDef;
-   this->max_confirm_wait_period_ms_ = config::kConfirmWaitPeriodMsMax;
-   this->shutdown_reb_wait_period_ms_ = config::kRebWaitPeriodWhenShutdownMs;
- }
- 
- ConsumerConfig::~ConsumerConfig() {
-
- }
+ConsumerConfig::~ConsumerConfig() {
+  //
+}
 
 ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
   if (this != &target) {
-    // parent class 
+    // parent class
     BaseConfig::operator =(target);
     // child class
     this->group_name_ = target.group_name_;
@@ -281,9 +279,9 @@ ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
 }
 
 bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
-                       const string& group_name, const set<string>& subscribed_topicset) {
+                        const string& group_name, const set<string>& subscribed_topicset) {
   string tgt_group_name;
-  bool is_success = Utils::ValidGroupName(err_info,group_name, tgt_group_name);
+  bool is_success = Utils::ValidGroupName(err_info, group_name, tgt_group_name);
   if (!is_success) {
     return false;
   }
@@ -293,17 +291,17 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
   }
   string topic_name;
   map<string, set<string> > tmp_sub_map;
-  for (set<string>::iterator it = subscribed_topicset.begin(); 
+  for (set<string>::iterator it = subscribed_topicset.begin();
           it != subscribed_topicset.end(); ++it) {
     topic_name = Utils::Trim(*it);
-    is_success = Utils::ValidString(err_info, topic_name, 
+    is_success = Utils::ValidString(err_info, topic_name,
                          false, true, true, config::kTopicNameMaxLength);
     if (!is_success) {
       err_info = "Illegal parameter: subscribed_topicset's item error, " + err_info;
       return false;
     }
     set<string> tmp_filters;
-    tmp_sub_map[topic_name] = tmp_filters; 
+    tmp_sub_map[topic_name] = tmp_filters;
   }
   this->is_bound_consume_ = false;
   this->group_name_ = tgt_group_name;
@@ -312,31 +310,33 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
   return true;
 }
 
-bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, 
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
   const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map) {
   string session_key;
   int source_count = 0;
   bool is_select_big = false;
-  map<string, long> part_offset_map;
-  return setGroupConsumeTarget(err_info, false, 
-    group_name, subscribed_topic_and_filter_map, 
-    session_key, source_count, is_select_big, part_offset_map);
+  map<string, int64_t> part_offset_map;
+  return setGroupConsumeTarget(err_info, false,
+              group_name, subscribed_topic_and_filter_map,
+              session_key, source_count, is_select_big, part_offset_map);
 }
 
-bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, 
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
       const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-      const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map) {
-  return setGroupConsumeTarget(err_info, true, 
-    group_name, subscribed_topic_and_filter_map, 
-    session_key, source_count, is_select_big, part_offset_map);
+      const string& session_key, uint32_t source_count, bool is_select_big,
+      const map<string, int64_t>& part_offset_map) {
+  return setGroupConsumeTarget(err_info, true,
+              group_name, subscribed_topic_and_filter_map,
+              session_key, source_count, is_select_big, part_offset_map);
 }
 
 bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consume,
       const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-      const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map) {
+      const string& session_key, uint32_t source_count, bool is_select_big,
+      const map<string, int64_t>& part_offset_map) {
   // check parameter group_name
   string tgt_group_name;
-  bool is_success = Utils::ValidGroupName(err_info,group_name, tgt_group_name);
+  bool is_success = Utils::ValidGroupName(err_info, group_name, tgt_group_name);
   if (!is_success) {
     return false;
   }
@@ -347,14 +347,14 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   }
   map<string, set<string> > tmp_sub_map;
   map<string, set<string> >::const_iterator it_map;
-  for (it_map = subscribed_topic_and_filter_map.begin(); 
-    it_map != subscribed_topic_and_filter_map.end(); ++it_map) {
-    int count=0;
+  for (it_map = subscribed_topic_and_filter_map.begin();
+              it_map != subscribed_topic_and_filter_map.end(); ++it_map) {
+    uint32_t count = 0;
     string tmp_filteritem;
     set<string> tgt_filters;
     // check topic_name info
-    is_success = Utils::ValidString(err_info, it_map->first, 
-                         false, true, true, config::kTopicNameMaxLength);  
+    is_success = Utils::ValidString(err_info, it_map->first,
+                          false, true, true, config::kTopicNameMaxLength);
     if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
@@ -367,7 +367,8 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     string topic_name = Utils::Trim(it_map->first);
     // check filter info
     set<string> subscribed_filters = it_map->second;
-    for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end(); ++it) {
+    for (set<string>::iterator it = subscribed_filters.begin();
+          it != subscribed_filters.end(); ++it) {
       is_success = Utils::ValidFilterItem(err_info, *it, tmp_filteritem);
       if (!is_success) {
         stringstream ss;
@@ -392,7 +393,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     }
     tmp_sub_map[topic_name] = tgt_filters;
   }
-  // check if bound consume 
+  // check if bound consume
   if (!is_bound_consume) {
     this->is_bound_consume_ = false;
     this->group_name_ = tgt_group_name;
@@ -402,7 +403,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   }
   // check session_key
   string tgt_session_key = Utils::Trim(session_key);
-  if (tgt_session_key.length() == 0 
+  if (tgt_session_key.length() == 0
     || tgt_session_key.length() > config::kSessionKeyMaxLength) {
     if (tgt_session_key.length() == 0) {
       err_info = "Illegal parameter: session_key is empty!";
@@ -421,8 +422,8 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   }
   // check part_offset_map
   string part_key;
-  map<string, long> tmp_parts_map;
-  map<string, long>::const_iterator it_part;
+  map<string, int64_t> tmp_parts_map;
+  map<string, int64_t>::const_iterator it_part;
   for (it_part = part_offset_map.begin(); it_part != part_offset_map.end(); ++it_part) {
     vector<string> result;
     Utils::Split(it_part->first, result, delimiter::kDelimiterColon);
@@ -469,7 +470,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   // set verified data
   this->is_bound_consume_ = true;
   this->group_name_ = tgt_group_name;
-  this->sub_topic_and_filter_map_ = tmp_sub_map;    
+  this->sub_topic_and_filter_map_ = tmp_sub_map;
   this->session_key_ = tgt_session_key;
   this->source_count_ = source_count;
   this->is_select_big_ = is_select_big;
@@ -485,7 +486,7 @@ const string& ConsumerConfig::GetGroupName() const {
 const map<string, set<string> >& ConsumerConfig::GetSubTopicAndFilterMap() const {
   return this->sub_topic_and_filter_map_;
 }
-  
+
 void ConsumerConfig::SetConsumePosition(ConsumePosition consume_from_where) {
   this->consume_position_ = consume_from_where;
 }
@@ -551,9 +552,9 @@ void ConsumerConfig::SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms
 }
 
 string ConsumerConfig::ToString() {
-  int i = 0;
+  int32_t i = 0;
   stringstream ss;
-  map<string, long>::iterator it;
+  map<string, int64_t>::iterator it;
   map<string, set<string> >::iterator it_map;
 
   // print info
@@ -562,15 +563,15 @@ string ConsumerConfig::ToString() {
   ss << ", group_name_='";
   ss << this->group_name_;
   ss << "', sub_topic_and_filter_map_={";
-  for (it_map = this->sub_topic_and_filter_map_.begin(); 
-      it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
+  for (it_map = this->sub_topic_and_filter_map_.begin();
+              it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
     if (i++ > 0) {
       ss << ",";
     }
     ss << "'";
     ss << it_map->first;
     ss << "'=[";
-    int j=0;
+    int32_t j = 0;
     set<string> topic_set = it_map->second;
     for (set<string>::iterator it = topic_set.begin(); it != topic_set.end(); ++it) {
       if (j++ > 0) {
@@ -585,15 +586,15 @@ string ConsumerConfig::ToString() {
   ss << "}, is_bound_consume_=";
   ss << this->is_bound_consume_;
   ss << ", session_key_='";
-  ss << this->session_key_;  
+  ss << this->session_key_;
   ss << "', source_count_=";
-  ss << this->source_count_;  
+  ss << this->source_count_;
   ss << ", is_select_big_=";
-  ss << this->is_select_big_;  
-  ss << ", part_offset_map_={";  
+  ss << this->is_select_big_;
+  ss << ", part_offset_map_={";
   i = 0;
-  for (it = this->part_offset_map_.begin(); 
-      it != this->part_offset_map_.end(); ++it) {
+  for (it = this->part_offset_map_.begin();
+              it != this->part_offset_map_.end(); ++it) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -622,5 +623,5 @@ string ConsumerConfig::ToString() {
   return ss.str();
 }
 
-}
+}  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index df0cdf3..6400359 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -17,12 +17,13 @@
  * under the License.
  */
 
-#include <fstream>
-#include <sstream> 
-#include <stdlib.h>
-#include "utils.h"
 #include "file_ini.h"
+#include <stdlib.h>
+#include <fstream>
+#include <sstream>
 #include "const_config.h"
+#include "utils.h"
+
 
 namespace tubemq {
 
@@ -60,8 +61,8 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   while (getline(conf_file, line_str)) {
     // check if a comment
     line_str = Utils::Trim(line_str);
-    if (line_str.empty() 
-      || line_str.find(delimiter::kDelimiterDbSlash) == 0 
+    if (line_str.empty()
+      || line_str.find(delimiter::kDelimiterDbSlash) == 0
       || line_str.find(delimiter::kDelimiterSemicolon) == 0) {
       continue;
     }
@@ -69,8 +70,8 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
     lftsb_pos = line_str.find(delimiter::kDelimiterLftSB);
     rgtsb_pos = line_str.find(delimiter::kDelimiterRgtSB);
     if (lftsb_pos != string::npos && rgtsb_pos != string::npos) {
-      sector = line_str.substr(lftsb_pos + (delimiter::kDelimiterLftSB).size(), 
-        rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
+      sector = line_str.substr(lftsb_pos + (delimiter::kDelimiterLftSB).size(),
+                      rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
       sector = Utils::Trim(sector);
       continue;
     }
@@ -107,7 +108,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   return true;
 }
 
-bool Fileini::GetValue(string& err_info, const string& sector, 
+bool Fileini::GetValue(string& err_info, const string& sector,
                 const string& key, string& value, const string& def) {
   if (!this->init_flag_) {
     err_info = "Please load configure file first!";
@@ -132,8 +133,8 @@ bool Fileini::GetValue(string& err_info, const string& sector,
   return true;
 }
 
-bool Fileini::GetValue(string& err_info, const string& sector, 
-                const string& key, int& value, const int def) {
+bool Fileini::GetValue(string& err_info, const string& sector,
+                const string& key, int32_t& value, const int32_t def) {
   string val_str;
   string def_str = Utils::Int2str(def);
   bool result = GetValue(err_info, sector, key, val_str, def_str);
@@ -144,6 +145,6 @@ bool Fileini::GetValue(string& err_info, const string& sector,
   return true;
 }
 
-}
+}  // namespace tubemq
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index bdc8138..e59b606 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -17,14 +17,14 @@
  * under the License.
  */
 
- #include <stdio.h>
+#include "flowctrl_def.h"
+#include <stdio.h>
 #include <time.h>
 #include <unistd.h>
 #include <sstream>
-#include "utils.h"
-#include "logger.h"
 #include "const_config.h"
-#include "flowctrl_def.h"
+#include "logger.h"
+#include "utils.h"
 
 
 
@@ -32,41 +32,41 @@ namespace tubemq {
 
 
 FlowCtrlResult::FlowCtrlResult() {
-  this->datasize_limit_ = config::kMaxIntValue;  
+  this->datasize_limit_ = config::kMaxIntValue;
   this->freqms_limit_ = 0;
 }
 
-FlowCtrlResult::FlowCtrlResult(long datasize_limit, int freqms_limit) {
-  this->datasize_limit_ = datasize_limit;  
+FlowCtrlResult::FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit) {
+  this->datasize_limit_ = datasize_limit;
   this->freqms_limit_ = freqms_limit;
 }
 
 FlowCtrlResult& FlowCtrlResult::operator=(const FlowCtrlResult& target) {
-  if (this == &target) 
+  if (this == &target)
     return *this;
-  this->datasize_limit_ = target.datasize_limit_;  
+  this->datasize_limit_ = target.datasize_limit_;
   this->freqms_limit_ = target.freqms_limit_;
   return *this;
 }
 
-void FlowCtrlResult::SetDataDltAndFreqLimit(long datasize_limit, int freqms_limit) {
+void FlowCtrlResult::SetDataDltAndFreqLimit(int64_t datasize_limit, int32_t freqms_limit) {
   this->datasize_limit_ = datasize_limit;
   this->freqms_limit_ = freqms_limit;
 }
 
-void FlowCtrlResult::SetDataSizeLimit(long datasize_limit) {
+void FlowCtrlResult::SetDataSizeLimit(int64_t datasize_limit) {
   this->datasize_limit_ = datasize_limit;
 }
 
-void FlowCtrlResult::SetFreqMsLimit(int freqms_limit) {
+void FlowCtrlResult::SetFreqMsLimit(int32_t freqms_limit) {
   this->freqms_limit_ = freqms_limit;
 }
 
-long FlowCtrlResult::GetDataSizeLimit() {
+int64_t FlowCtrlResult::GetDataSizeLimit() {
   return this->datasize_limit_;
 }
 
-int FlowCtrlResult::GetFreqMsLimit() {
+int32_t FlowCtrlResult::GetFreqMsLimit() {
   return this->freqms_limit_;
 }
 
@@ -81,7 +81,8 @@ FlowCtrlItem::FlowCtrlItem() {
   this->zero_cnt_       = config::kInvalidValue;
 }
 
-FlowCtrlItem::FlowCtrlItem(int type,int zero_cnt,int freqms_limit) {
+FlowCtrlItem::FlowCtrlItem(int32_t type,
+                       int32_t zero_cnt, int32_t freqms_limit) {
   this->type_           = type;
   this->start_time_     = 2500;
   this->end_time_       = config::kInvalidValue;
@@ -91,8 +92,8 @@ FlowCtrlItem::FlowCtrlItem(int type,int zero_cnt,int freqms_limit) {
   this->zero_cnt_       = zero_cnt;
 }
 
-FlowCtrlItem::FlowCtrlItem(int type, 
-  int datasize_limit,int freqms_limit,int min_data_filter_freqms) {
+FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t datasize_limit,
+                      int32_t freqms_limit, int32_t min_data_filter_freqms) {
   this->type_           = type;
   this->start_time_     = 2500;
   this->end_time_       = config::kInvalidValue;
@@ -102,8 +103,8 @@ FlowCtrlItem::FlowCtrlItem(int type,
   this->zero_cnt_       = min_data_filter_freqms;
 }
 
-FlowCtrlItem::FlowCtrlItem(int type, int start_time, int end_time, 
-  long datadlt_m, long datasize_limit, int freqms_limit) {
+FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time,
+                      int64_t datadlt_m, int64_t datasize_limit, int32_t freqms_limit) {
   this->type_           = type;
   this->start_time_     = start_time;
   this->end_time_       = end_time;
@@ -114,7 +115,7 @@ FlowCtrlItem::FlowCtrlItem(int type, int start_time, int end_time,
 }
 
 FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
-  if (this == &target) 
+  if (this == &target)
     return *this;
   this->type_           = target.type_;
   this->start_time_     = target.start_time_;
@@ -126,7 +127,7 @@ FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
     return *this;
 }
 
-int FlowCtrlItem::GetFreLimit(int msg_zero_cnt) {
+int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) {
   if (this->type_ != 1) {
     return -1;
   }
@@ -136,8 +137,8 @@ int FlowCtrlItem::GetFreLimit(int msg_zero_cnt) {
   return -1;
 }
 
-void FlowCtrlItem::ResetFlowCtrlValue(int type, 
-  int datasize_limit,int freqms_limit,int min_data_filter_freqms) {
+void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit,
+                                  int32_t freqms_limit, int32_t min_data_filter_freqms) {
   this->type_           = type;
   this->start_time_     = 2500;
   this->end_time_       = config::kInvalidValue;
@@ -157,15 +158,16 @@ void FlowCtrlItem::Clear() {
   this->zero_cnt_       = config::kInvalidValue;
 }
 
-bool FlowCtrlItem::GetDataLimit(long datadlt_m, int curr_time, FlowCtrlResult& flowctrl_result) {
-  if (this->type_ != 0 || datadlt_m <= this->datadlt_m_ ) {
+bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m,
+                      int32_t curr_time, FlowCtrlResult& flowctrl_result) {
+  if (this->type_ != 0 || datadlt_m <= this->datadlt_m_) {
     return false;
   }
-  if (curr_time < this->start_time_ 
+  if (curr_time < this->start_time_
     || curr_time > this->end_time_) {
     return false;
   }
-  flowctrl_result.SetDataDltAndFreqLimit(this->datasize_limit_,this->freqms_limit_);
+  flowctrl_result.SetDataDltAndFreqLimit(this->datasize_limit_, this->freqms_limit_);
   return true;
 }
 
@@ -179,21 +181,21 @@ FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
   this->datalimit_start_time_.Set(2500);
   this->datalimit_end_time_.Set(config::kInvalidValue);
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
-  pthread_rwlock_init(&configrw_lock_, NULL);  
+  pthread_rwlock_init(&configrw_lock_, NULL);
 }
 
 FlowCtrlRuleHandler::~FlowCtrlRuleHandler() {
   pthread_rwlock_destroy(&configrw_lock_);
 }
 
-void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, 
-  int qrypriority_id, long flowctrl_id, const string& flowctrl_info) {
+void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default,
+                  int32_t qrypriority_id, int64_t flowctrl_id, const string& flowctrl_info) {
   bool result;
-  map<int, vector<FlowCtrlItem> > tmp_flowctrl_map;
+  map<int32_t, vector<FlowCtrlItem> > tmp_flowctrl_map;
   if (flowctrl_id == this->flowctrl_id_.Get()) {
       return;
   }
-  long curr_flowctrl_id = this->flowctrl_id_.Get();
+  int64_t curr_flowctrl_id = this->flowctrl_id_.Get();
   if (flowctrl_info.length() > 0) {
     result = parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
   }
@@ -212,12 +214,13 @@ void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default,
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
   pthread_rwlock_unlock(&this->configrw_lock_);
   if (is_default) {
-    LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id, flowctrl_id);
+    LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n",
+      curr_flowctrl_id, flowctrl_id);
   } else {
-    LOG_INFO("[Flow Ctrl] Group FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id, flowctrl_id);
+    LOG_INFO("[Flow Ctrl] Group FlowCtrl's flowctrl_id from %ld to %ld\n",
+      curr_flowctrl_id, flowctrl_id);
   }
   return;
-
 }
 
 
@@ -262,7 +265,7 @@ void FlowCtrlRuleHandler::initialStatisData() {
       }
       it_vec->GetDataSizeLimit();
       this->filter_ctrl_item_.ResetFlowCtrlValue(3,
-          (int)(it_vec->GetDataSizeLimit()),it_vec->GetFreqMsLimit(),it_vec->GetZeroCnt());
+          (int)(it_vec->GetDataSizeLimit()), it_vec->GetFreqMsLimit(), it_vec->GetZeroCnt());
     }
   }
 }
@@ -276,15 +279,15 @@ void FlowCtrlRuleHandler::clearStatisData() {
   this->filter_ctrl_item_.Clear();
 }
 
-bool FlowCtrlRuleHandler::GetCurDataLimit(long last_datadlt, FlowCtrlResult& flowctrl_result) {
+bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) {
   struct tm utc_tm;
   vector<FlowCtrlItem>::iterator it_vec;
   map<int, vector<FlowCtrlItem> >::iterator it_map;
-  time_t cur_time=time(NULL);
+  time_t cur_time = time(NULL);
 
-  gmtime_r(&cur_time,&utc_tm);
+  gmtime_r(&cur_time, &utc_tm);
   int curr_time = (utc_tm.tm_hour+8)%24 * 100 + utc_tm.tm_min;
-  if ((last_datadlt < this->min_datadlt_limt_.Get()) 
+  if ((last_datadlt < this->min_datadlt_limt_.Get())
     || (curr_time < this->datalimit_start_time_.Get())
     || (curr_time > this->datalimit_end_time_.Get())) {
     return false;
@@ -294,16 +297,15 @@ bool FlowCtrlRuleHandler::GetCurDataLimit(long last_datadlt, FlowCtrlResult& flo
     return false;
   }
   for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
-    if (it_vec->GetDataLimit(last_datadlt,curr_time,flowctrl_result)) {
+    if (it_vec->GetDataLimit(last_datadlt, curr_time, flowctrl_result)) {
       return true;
     }
   }
   return false;
 }
 
-int FlowCtrlRuleHandler::GetCurFreqLimitTime(int msg_zero_cnt, int received_limit)
-{
-  int rule_val = -2;
+int FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) {
+  int32_t rule_val = -2;
   vector<FlowCtrlItem>::iterator it_vec;
   map<int, vector<FlowCtrlItem> >::iterator it_map;
 
@@ -316,8 +318,7 @@ int FlowCtrlRuleHandler::GetCurFreqLimitTime(int msg_zero_cnt, int received_limi
   }
   for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
     rule_val = it_vec->GetFreLimit(msg_zero_cnt);
-    if (rule_val >= 0) 
-    {
+    if (rule_val >= 0) {
       return rule_val;
     }
   }
@@ -336,26 +337,26 @@ bool FlowCtrlRuleHandler::compareFeqQueue(const FlowCtrlItem& queue1, const Flow
     return (queue1.GetZeroCnt() < queue2.GetZeroCnt());
 }
 
-bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info, 
-                                map<int,vector<FlowCtrlItem> >& flowctrl_info_map) {
-  int type;
+bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
+                                map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map) {
+  int32_t type;
   string err_info;
   stringstream ss;
   rapidjson::Document doc;
   // check flowctrl info length
-  if (flowctrl_info.length() == 0){
+  if (flowctrl_info.length() == 0) {
     return false;
   }
   // parse flowctrl info
   if (doc.Parse(flowctrl_info.c_str()).HasParseError()) {
-    LOG_ERROR("Parsing flowCtrlInfo failure! flowctrl_info=%s\n",flowctrl_info.c_str()); 
+    LOG_ERROR("Parsing flowCtrlInfo failure! flowctrl_info=%s\n", flowctrl_info.c_str());
     return false;
   }
   if (!doc.IsArray()) {
-    LOG_ERROR("flowCtrlInfo's value must be array! flowctrl_info=%s\n",flowctrl_info.c_str()); 
+    LOG_ERROR("flowCtrlInfo's value must be array! flowctrl_info=%s\n", flowctrl_info.c_str());
     return false;
   }
-  for (unsigned int i = 0; i < doc.Size(); i++) {
+  for (uint32_t i = 0; i < doc.Size(); i++) {
     vector<FlowCtrlItem> flowctrl_item_vec;
     const rapidjson::Value& node_item = doc[i];
     if (!node_item.IsObject()) {
@@ -366,42 +367,42 @@ bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
       ss << err_info;
       ss << " of type field in parse flowctrl_info!";
       err_info = ss.str();
-      LOG_ERROR("parse flowCtrlInfo failure %s", err_info.c_str()); 
+      LOG_ERROR("parse flowCtrlInfo failure %s", err_info.c_str());
       return false;
-    }    
+    }
     if (type < 0 || type > 3) {
       ss << "type value must in [0,1,2,3] in index(";
       ss << i;
       ss << ") of flowctrl_info value!";
       err_info = ss.str();
-      LOG_ERROR("parse flowCtrlInfo failure %s",err_info.c_str()); 
+      LOG_ERROR("parse flowCtrlInfo failure %s", err_info.c_str());
       return false;
     }
-    
+
     switch (type) {
       case 1: {
         if (FlowCtrlRuleHandler::parseFreqLimit(err_info, node_item, flowctrl_item_vec)) {
-          flowctrl_info_map[1]=flowctrl_item_vec;
+          flowctrl_info_map[1] = flowctrl_item_vec;
         } else {
-          LOG_ERROR("parse flowCtrlInfo's freqLimit failure: %s",err_info.c_str()); 
+          LOG_ERROR("parse flowCtrlInfo's freqLimit failure: %s", err_info.c_str());
         }
       }
       break;
-       
+
       case 3: {
         if (FlowCtrlRuleHandler::parseLowFetchLimit(err_info, node_item, flowctrl_item_vec)) {
-            flowctrl_info_map[3]=flowctrl_item_vec;
+            flowctrl_info_map[3] = flowctrl_item_vec;
         } else {
-          LOG_ERROR("parse flowCtrlInfo's lowFetchLimit failure: %s",err_info.c_str()); 
+          LOG_ERROR("parse flowCtrlInfo's lowFetchLimit failure: %s", err_info.c_str());
         }
       }
       break;
 
       case 0: {
         if (FlowCtrlRuleHandler::parseDataLimit(err_info, node_item, flowctrl_item_vec)) {
-          flowctrl_info_map[0]=flowctrl_item_vec;
+          flowctrl_info_map[0] = flowctrl_item_vec;
         } else {
-          LOG_ERROR("parse flowCtrlInfo's dataLimit failure: %s",err_info.c_str()); 
+          LOG_ERROR("parse flowCtrlInfo's dataLimit failure: %s", err_info.c_str());
         }
       }
       break;
@@ -409,13 +410,13 @@ bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
       default:
         break;
     }
-  }  
+  }
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseDataLimit(string& err_info, 
+bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
                            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
-  int type_val;
+  int32_t type_val;
   stringstream ss;
   string attr_sep = delimiter::kDelimiterColon;
   string::size_type pos1;
@@ -437,12 +438,12 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& obj_set = root["rule"];
-  for (unsigned int index = 0 ; index < obj_set.Size() ; index++) {
-    int start_time = 0;
-    int end_time = 0;
-    long datadlt_m = 0;
-    long datasize_limit = 0;
-    int freqms_limit = 0;
+  for (uint32_t index = 0 ; index < obj_set.Size() ; index++) {
+    int32_t start_time = 0;
+    int32_t end_time = 0;
+    int64_t datadlt_m = 0;
+    int64_t datasize_limit = 0;
+    int32_t freqms_limit = 0;
     const rapidjson::Value& node_item = obj_set[index];
     if (!node_item.IsObject()) {
         err_info = "Illegal rule'value item, must be dict type";
@@ -489,7 +490,7 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
       ss << ") of data limit rule!";
       err_info = ss.str();
       return false;
-    }    
+    }
     datasize_limit = datasize_limit * 1024 * 1024;
     if (!parseIntMember(err_info, node_item, "freqInMs", freqms_limit, false, -1)) {
       ss << "freqInMs key is required in index(";
@@ -504,20 +505,21 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
       ss << ") of data limit rule!";
       err_info = ss.str();
       return false;
-    }    
-    FlowCtrlItem flowctrl_item(0, start_time, end_time, datadlt_m, datasize_limit, freqms_limit);
+    }
+    FlowCtrlItem flowctrl_item(0, start_time,
+               end_time, datadlt_m, datasize_limit, freqms_limit);
     flowctrl_items.push_back(flowctrl_item);
   }
   if (!flowctrl_items.empty()) {
     std::sort(flowctrl_items.begin(), flowctrl_items.end(), compareDataLimitQueue);
   }
   err_info = "Ok";
-  return true;  
+  return true;
 }
 
-bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info, 
+bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
                            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
-  int type_val;
+  int32_t type_val;
   stringstream ss;
 
   if (!parseIntMember(err_info, root, "type", type_val, true, 1)) {
@@ -537,9 +539,9 @@ bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& obj_set = root["rule"];
-  for (unsigned int i = 0 ; i < obj_set.Size() ; i++) {
-    int zeroCnt = -2;
-    int freqms_limit = -2;
+  for (uint32_t i = 0 ; i < obj_set.Size() ; i++) {
+    int32_t zeroCnt = -2;
+    int32_t freqms_limit = -2;
     const rapidjson::Value& node_item = obj_set[i];
     if (!node_item.IsObject()) {
       err_info = "Illegal rule'value item, must be dict type";
@@ -566,12 +568,12 @@ bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
     std::sort(flowctrl_items.begin(), flowctrl_items.end(), compareFeqQueue);
   }
   err_info = "Ok";
-  return true;  
+  return true;
 }
 
-bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info, 
+bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
                             const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
-  int type_val;
+  int32_t type_val;
   stringstream ss;
   if (!parseIntMember(err_info, root, "type", type_val, true, 3)) {
     ss << "Decode Failure: ";
@@ -590,10 +592,10 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& node_item = root["rule"];
-  for (unsigned int i = 0 ; i < node_item.Size() ; i++) {
-    int norm_freq_ms = 0;
-    int filter_freq_ms = 0;
-    int min_filter_freq_ms = 0;
+  for (uint32_t i = 0 ; i < node_item.Size() ; i++) {
+    int32_t norm_freq_ms = 0;
+    int32_t filter_freq_ms = 0;
+    int32_t min_filter_freq_ms = 0;
     FlowCtrlItem flowctrl_item;
     const rapidjson::Value& node_item = node_item[i];
     if (!node_item.IsObject()) {
@@ -602,16 +604,16 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
     }
     if (node_item.HasMember("filterFreqInMs") 
       || node_item.HasMember("minDataFilterFreqInMs")) {
-      if (!parseIntMember(err_info, node_item, 
-        "filterFreqInMs", filter_freq_ms, false, -1)) {
+      if (!parseIntMember(err_info, node_item,
+                "filterFreqInMs", filter_freq_ms, false, -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of filterFreqInMs field in parse low fetch limit!";
         err_info = ss.str();
         return false;
       }
-      if (!parseIntMember(err_info, node_item, 
-        "minDataFilterFreqInMs", min_filter_freq_ms, false, -1)) {
+      if (!parseIntMember(err_info, node_item,
+                "minDataFilterFreqInMs", min_filter_freq_ms, false, -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of minDataFilterFreqInMs field in parse low fetch limit!";
@@ -644,8 +646,8 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
       }
     }
     if (node_item.HasMember("normFreqInMs")) {
-      if (!parseIntMember(err_info, node_item, 
-        "normFreqInMs", norm_freq_ms, false, -1)) {
+      if (!parseIntMember(err_info, node_item,
+                "normFreqInMs", norm_freq_ms, false, -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of normFreqInMs field in parse low fetch limit!";
@@ -659,17 +661,19 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
         ss << ") of low fetch limit rule!";
         err_info = ss.str();
         return false;
-      }        
+      }
     }
-    flowctrl_item.ResetFlowCtrlValue(3,norm_freq_ms,filter_freq_ms,min_filter_freq_ms);
+    flowctrl_item.ResetFlowCtrlValue(3,
+                     norm_freq_ms, filter_freq_ms, min_filter_freq_ms);
     flowctrl_items.push_back(flowctrl_item);
   }
   err_info = "Ok";
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseStringMember(string& err_info, const rapidjson::Value& root, 
-                            const char* key, string& value, bool compare_value, string required_val) {
+bool FlowCtrlRuleHandler::parseStringMember(string& err_info,
+    const rapidjson::Value& root, const char* key, string& value,
+    bool compare_value, string required_val) {
   // check key if exist
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
@@ -679,7 +683,7 @@ bool FlowCtrlRuleHandler::parseStringMember(string& err_info, const rapidjson::V
     err_info = "Illegal value, must be string type";
     return false;
   }
-  
+
   if (compare_value) {
     if (root[key].GetString() != required_val) {
       err_info = "Illegal value, not required value content";
@@ -690,8 +694,9 @@ bool FlowCtrlRuleHandler::parseStringMember(string& err_info, const rapidjson::V
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseLongMember(string& err_info, const rapidjson::Value& root, 
-                            const char* key, long& value, bool compare_value, long required_val) {
+bool FlowCtrlRuleHandler::parseLongMember(string& err_info,
+      const rapidjson::Value& root, const char* key, int64_t& value,
+      bool compare_value, int64_t required_val) {
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
     return false;
@@ -701,17 +706,18 @@ bool FlowCtrlRuleHandler::parseLongMember(string& err_info, const rapidjson::Val
     return false;
   }
   if (compare_value) {
-    if ((long)root[key].GetInt64() != required_val) {
+    if ((int64_t)root[key].GetInt64() != required_val) {
       err_info = "Illegal value, not required value content";
       return false;
     }
   }
-  value = (long)root[key].GetInt64();
+  value = (int64_t)root[key].GetInt64();
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseIntMember(string& err_info, const rapidjson::Value& root, 
-                            const char* key, int& value, bool compare_value, int required_val) {
+bool FlowCtrlRuleHandler::parseIntMember(string& err_info,
+      const rapidjson::Value& root, const char* key, int32_t& value,
+      bool compare_value, int32_t required_val) {
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
     return false;
@@ -730,8 +736,8 @@ bool FlowCtrlRuleHandler::parseIntMember(string& err_info, const rapidjson::Valu
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseTimeMember(string& err_info, 
-                           const rapidjson::Value& root, const char* key, int& value) {
+bool FlowCtrlRuleHandler::parseTimeMember(string& err_info,
+              const rapidjson::Value& root, const char* key, int32_t& value) {
   // check key if exist
   stringstream ss;
   if (!root.HasMember(key)) {
@@ -759,29 +765,28 @@ bool FlowCtrlRuleHandler::parseTimeMember(string& err_info,
     err_info = ss.str();
     return false;
   }
-  string sub_str_1 = str_value.substr(0,pos1);
-  string sub_str_2 = 
-    str_value.substr(pos1+attr_sep.size(),str_value.size());
-  int in_hour = atoi(sub_str_1.c_str());
-  int in_minute = atoi(sub_str_2.c_str());
+  string sub_str_1 = str_value.substr(0, pos1);
+  string sub_str_2 = str_value.substr(pos1 + attr_sep.size(), str_value.size());
+  int32_t in_hour = atoi(sub_str_1.c_str());
+  int32_t in_minute = atoi(sub_str_2.c_str());
   if (in_hour < 0 || in_hour > 24) {
     ss << "field ";
     ss << key;
     ss << " -hour value must in [0,23]!";
     err_info = ss.str();
-    return false;      
+    return false;
   }
   if (in_minute < 0 || in_minute > 59) {
     ss << "field ";
     ss << key;
     ss << " -minute value must in [0,59]!";
     err_info = ss.str();
-    return false;      
+    return false;
   }
   value = in_hour * 100 + in_minute;
   return true;
 }
 
 
-}
+}  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
index c1860df..a9d5df3 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -18,13 +18,11 @@
  */
 
 #include "logger.h"
-
 #include <log4cplus/fileappender.h>
 #include <log4cplus/layout.h>
 #include <log4cplus/logger.h>
 #include <log4cplus/loggingmacros.h>
 #include <stdarg.h>
-
 #include <string>
 
 namespace tubemq {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index e737a41..d57fb5a 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -17,19 +17,19 @@
  * under the License.
  */
 
-#include <sstream> 
-#include <string.h> 
-
 #include "message.h"
-#include "utils.h"
+#include <string.h>
+#include <sstream>
 #include "const_config.h"
+#include "utils.h"
+
 
 
 namespace tubemq {
 
 
 // message flag's properties settings
-static const int kMsgFlagIncProperties = 0x01;
+static const int32_t kMsgFlagIncProperties = 0x01;
 // reserved property key Filter Item
 static const string kRsvPropKeyFilterItem = "$msgType$";
 // reserved property key message send time
@@ -53,7 +53,7 @@ Message::Message(const Message& target) {
   this->flag_       = target.flag_;
 }
 
-Message::Message(const string& topic, const char* data, int datalen) {
+Message::Message(const string& topic, const char* data, uint32_t datalen) {
   this->topic_      = topic;
   this->flag_       = 0;
   this->message_id_ = config::kInvalidValue;
@@ -66,7 +66,7 @@ Message::~Message() {
 }
 
 Message& Message::operator=(const Message& target) {
-  if (this == &target) 
+  if (this == &target)
     return *this;
   this->topic_      = target.topic_;
   this->message_id_ = target.message_id_;
@@ -77,11 +77,11 @@ Message& Message::operator=(const Message& target) {
   return *this;
 }
 
-const long Message::GetMessageId() const {
+const uint64_t Message::GetMessageId() const {
   return this->message_id_;
 }
 
-void Message::SetMessageId(long message_id) {
+void Message::SetMessageId(int64_t message_id) {
   this->message_id_ = message_id;
 }
 
@@ -97,20 +97,20 @@ const char* Message::GetData() const {
   return this->data_;
 }
 
-int Message::GetDataLength() const {
+uint32_t Message::GetDataLength() const {
   return this->datalen_;
 }
 
-void Message::setData(const char* data, int datalen) {
+void Message::setData(const char* data, uint32_t datalen) {
   clearData();
   copyData(data, datalen);
 }
 
-const int Message::GetFlag() const {
+const int32_t Message::GetFlag() const {
   return this->flag_;
 }
 
-void Message::SetFlag(int flag) {
+void Message::SetFlag(int32_t flag) {
   this->flag_ = flag;
 }
 
@@ -118,7 +118,7 @@ const map<string, string>& Message::GetProperties() const {
   return this->properties_;
 }
 
-int Message::GetProperties(string& attribute) {
+int32_t Message::GetProperties(string& attribute) {
   attribute.clear();
   map<string, string>::iterator it_map;
   for (it_map = this->properties_.begin(); it_map != this->properties_.end(); ++it_map) {
@@ -141,7 +141,7 @@ bool Message::HasProperty(const string& key) {
       return true;
     }
   }
-  return false;  
+  return false;
 }
 
 bool Message::GetProperty(const string& key, string& value) {
@@ -154,7 +154,7 @@ bool Message::GetProperty(const string& key, string& value) {
       return true;
     }
   }
-  return false;  
+  return false;
 }
 
 bool Message::GetFilterItem(string& value) {
@@ -168,7 +168,7 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = "Not allowed null value of parmeter key or value";
     return false;
   }
-  if ((string::npos != trimed_key.find(delimiter::kDelimiterComma)) 
+  if ((string::npos != trimed_key.find(delimiter::kDelimiterComma))
     ||(string::npos != trimed_key.find(delimiter::kDelimiterEqual))) {
     stringstream ss;
     ss << "Reserved token '";
@@ -179,7 +179,7 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if ((string::npos != trimed_value.find(delimiter::kDelimiterComma)) 
+  if ((string::npos != trimed_value.find(delimiter::kDelimiterComma))
     ||(string::npos != trimed_value.find(delimiter::kDelimiterEqual))) {
     stringstream ss;
     ss << "Reserved token '";
@@ -190,8 +190,8 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if (trimed_key == kRsvPropKeyFilterItem 
-    || trimed_key == kRsvPropKeyMsgTime) {
+  if (trimed_key == kRsvPropKeyFilterItem
+         || trimed_key == kRsvPropKeyMsgTime) {
     stringstream ss;
     ss << "Reserved token '";
     ss << kRsvPropKeyFilterItem;
@@ -218,7 +218,7 @@ void Message::clearData() {
   }
 }
 
-void Message::copyData(const char* data, int datalen) {
+void Message::copyData(const char* data, uint32_t datalen) {
   if (data == NULL) {
     this->data_ = NULL;
     this->datalen_ = 0;
@@ -242,7 +242,7 @@ void Message::copyProperties(const map<string, string>& properties) {
 }
 
 
-}
+}  // namespace tubemq
 
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 8024a6a..ebd75d1 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -17,21 +17,25 @@
  * under the License.
  */
 
+#include "meta_info.h"
+#include <stdlib.h>
 #include <sstream>
 #include <vector>
-#include <stdlib.h>
-#include "utils.h"
-#include "meta_info.h"
 #include "const_config.h"
+#include "utils.h"
 
 
 namespace tubemq {
 
+using std::vector;
+using std::sstream;
+
+
 
 NodeInfo::NodeInfo() {
-  this->node_id_   = config::kInvalidValue;
+  this->node_id_   = 0;
   this->node_host_ = " ";
-  this->node_port_ = config::kInvalidValue;
+  this->node_port_ = config::kBrokerPortDef;
   buildStrInfo();
 }
 
@@ -43,11 +47,11 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
     this->node_id_   = atoi(result[0].c_str());
     this->node_host_ = result[1];
     this->node_port_ = config::kBrokerPortDef;
-    if (result.size() >= 3){
+    if (result.size() >= 3) {
       this->node_port_ = atoi(result[2].c_str());
     }
   } else {
-    this->node_id_   = config::kInvalidValue;
+    this->node_id_   = 0;
     this->node_host_ = result[0];
     this->node_port_ = config::kBrokerPortDef;
     if (result.size() >= 2) {
@@ -57,15 +61,14 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
   buildStrInfo();
 }
 
-NodeInfo::NodeInfo(const string& node_host, int node_port) {
+NodeInfo::NodeInfo(const string& node_host, uint32_t node_port) {
   this->node_id_   = config::kInvalidValue;
   this->node_host_ = node_host;
   this->node_port_ = node_port;
   buildStrInfo();
-
 }
 
-NodeInfo::NodeInfo(int node_id, const string& node_host, int node_port) {
+NodeInfo::NodeInfo(int node_id, const string& node_host, uint32_t node_port) {
   this->node_id_   = node_id;
   this->node_host_ = node_host;
   this->node_port_ = node_port;
@@ -73,11 +76,11 @@ NodeInfo::NodeInfo(int node_id, const string& node_host, int node_port) {
 }
 
 NodeInfo::~NodeInfo() {
-
+  //
 }
 
 NodeInfo& NodeInfo::operator=(const NodeInfo& target) {
-  if (this != &target){
+  if (this != &target) {
     this->node_id_   = target.node_id_;
     this->node_host_ = target.node_host_;
     this->node_port_ = target.node_port_;
@@ -95,14 +98,12 @@ bool NodeInfo::operator== (const NodeInfo& target) {
     return true;
   }
   return false;
-
 }
 
 bool NodeInfo::operator< (const NodeInfo& target) const {
   return this->node_info_ < target.node_info_;
 }
-  
-const int NodeInfo::GetNodeId() const {
+const uint32_t NodeInfo::GetNodeId() const {
   return this->node_id_;
 }
 
@@ -110,10 +111,10 @@ const string& NodeInfo::GetHost() const {
   return this->node_host_;
 }
 
-const int NodeInfo::GetPort() const {
+const uint32_t NodeInfo::GetPort() const {
   return this->node_port_;
 }
-  
+
 const string& NodeInfo::GetAddrInfo() const {
   return this->addr_info_;
 }
@@ -139,7 +140,7 @@ void NodeInfo::buildStrInfo() {
 
 Partition::Partition() {
   this->topic_ = " ";
-  this->partition_id_ = config::kInvalidValue;
+  this->partition_id_ = 0;
   buildPartitionKey();
 }
 
@@ -147,14 +148,14 @@ Partition::Partition() {
 Partition::Partition(const string& partition_info) {
   // initial process
   this->topic_ = " ";
-  this->partition_id_ = config::kInvalidValue;
+  this->partition_id_ = 0;
   // parse partition_info string
-  string::size_type pos=0;
+  string::size_type pos = 0;
   string seg_key = delimiter::kDelimiterPound;
   string token_key = delimiter::kDelimiterColon;
   // parse broker_info
   pos = partition_info.find(seg_key);
-  if (pos != string::npos){
+  if (pos != string::npos) {
     string broker_info = partition_info.substr(0, pos);
     broker_info = Utils::Trim(broker_info);
     this->broker_info_ = NodeInfo(true, broker_info);
@@ -172,12 +173,12 @@ Partition::Partition(const string& partition_info) {
   }
   buildPartitionKey();
 }
-  
+
 // part_str = topic:partition_id
 Partition::Partition(const NodeInfo& broker_info, const string& part_str) {
   vector<string> result;
   this->topic_ = " ";
-  this->partition_id_ = config::kInvalidValue;
+  this->partition_id_ = 0;
   this->broker_info_ = broker_info;
   Utils::Split(part_str, result, delimiter::kDelimiterColon);
   if (result.size() >= 2) {
@@ -187,7 +188,7 @@ Partition::Partition(const NodeInfo& broker_info, const string& part_str) {
   buildPartitionKey();
 }
 
-Partition::Partition(const NodeInfo& broker_info, const string& topic, int partition_id) {
+Partition::Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id) {
   this->topic_ = topic;
   this->partition_id_ = partition_id;
   this->broker_info_ = broker_info;
@@ -217,10 +218,9 @@ bool Partition::operator== (const Partition& target) {
     return true;
   }
   return false;
-
 }
 
-const int Partition::GetBrokerId() const {
+const uint32_t Partition::GetBrokerId() const {
   return this->broker_info_.GetNodeId();
 }
 
@@ -228,7 +228,7 @@ const string& Partition::GetBrokerHost() const {
   return this->broker_info_.GetHost();
 }
 
-const int Partition::GetBrokerPort() const {
+const uint32_t Partition::GetBrokerPort() const {
   return this->broker_info_.GetPort();
 }
 
@@ -244,7 +244,7 @@ const NodeInfo& Partition::GetBrokerInfo() const {
   return this->broker_info_;
 }
 
-const int Partition::GetPartitionId() const {
+const uint32_t Partition::GetPartitionId() const {
   return this->partition_id_;
 }
 
@@ -273,13 +273,13 @@ void Partition::buildPartitionKey() {
 
 // sub_info = consumerId@group#broker_info#topic:partitionId
 SubscribeInfo::SubscribeInfo(const string& sub_info) {
-  string::size_type pos=0;
+  string::size_type pos = 0;
   string seg_key = delimiter::kDelimiterPound;
   string at_key = delimiter::kDelimiterAt;
   this->consumer_id_ = " ";
   this->group_ = " ";
   // parse sub_info
-  pos=sub_info.find(seg_key);
+  pos = sub_info.find(seg_key);
   if (pos != string::npos) {
     string consumer_info = sub_info.substr(0, pos);
     consumer_info = Utils::Trim(consumer_info);
@@ -295,7 +295,7 @@ SubscribeInfo::SubscribeInfo(const string& sub_info) {
   buildSubInfo();
 }
 
-SubscribeInfo::SubscribeInfo(const string& consumer_id, 
+SubscribeInfo::SubscribeInfo(const string& consumer_id,
                  const string& group, const Partition& partition) {
   this->consumer_id_ = consumer_id;
   this->group_       = group;
@@ -325,7 +325,7 @@ const Partition& SubscribeInfo::GetPartition() const {
   return this->partition_;
 }
 
-const int SubscribeInfo::GgetBrokerId() const {
+const uint32_t SubscribeInfo::GgetBrokerId() const {
   return this->partition_.GetBrokerId();
 }
 
@@ -333,7 +333,7 @@ const string& SubscribeInfo::GetBrokerHost() const {
   return this->partition_.GetBrokerHost();
 }
 
-const int SubscribeInfo::GetBrokerPort() const {
+const uint32_t SubscribeInfo::GetBrokerPort() const {
   return this->partition_.GetBrokerPort();
 }
 
@@ -341,7 +341,7 @@ const string& SubscribeInfo::GetTopic() const {
   return this->partition_.GetTopic();
 }
 
-const int SubscribeInfo::GetPartitionId() const {
+const uint32_t SubscribeInfo::GetPartitionId() const {
   return this->partition_.GetPartitionId();
 }
 
@@ -373,8 +373,8 @@ ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
   this->subscribe_list_ = target.subscribe_list_;
 }
 
-ConsumerEvent::ConsumerEvent(long rebalance_id,int event_type, 
-    const list<SubscribeInfo>& subscribeInfo_lst, int event_status) {
+ConsumerEvent::ConsumerEvent(int64_t rebalance_id, int32_t event_type,
+                 const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
   list<SubscribeInfo>::const_iterator it;
   this->rebalance_id_ = rebalance_id;
   this->event_type_   = event_type;
@@ -385,7 +385,7 @@ ConsumerEvent::ConsumerEvent(long rebalance_id,int event_type,
 }
 
 ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
-  if(this != &target){
+  if (this != &target) {
     this->rebalance_id_ = target.rebalance_id_;
     this->event_type_ = target.event_type_;
     this->event_status_ = target.event_status_;
@@ -394,23 +394,23 @@ ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
   return *this;
 }
 
-const long ConsumerEvent::GetRebalanceId() const {
+const int64_t ConsumerEvent::GetRebalanceId() const {
   return this->rebalance_id_;
 }
 
-const int ConsumerEvent::GetEventType() const {
+const int32_t ConsumerEvent::GetEventType() const {
   return this->event_type_;
 }
 
-const int ConsumerEvent::GetEventStatus() const {
+const int32_t ConsumerEvent::GetEventStatus() const {
   return this->event_status_;
 }
 
-void ConsumerEvent::SetEventType(int event_type) {
+void ConsumerEvent::SetEventType(int32_t event_type) {
   this->event_type_ = event_type;
 }
 
-void ConsumerEvent::SetEventStatus(int event_status) {
+void ConsumerEvent::SetEventStatus(int32_t event_status) {
   this->event_status_ = event_status;
 }
 
@@ -419,7 +419,7 @@ const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
 }
 
 string ConsumerEvent::ToString() {
-  int count = 0;
+  uint32_t count = 0;
   stringstream ss;
   list<SubscribeInfo>::const_iterator it;
   ss << "ConsumerEvent [rebalanceId=";
@@ -429,16 +429,16 @@ string ConsumerEvent::ToString() {
   ss << ", status=";
   ss << this->event_status_;
   ss << ", subscribeInfoList=[";
-  for (it = this->subscribe_list_.begin(); 
+  for (it = this->subscribe_list_.begin();
           it != this->subscribe_list_.end(); ++it) {
-    if(count++ > 0) {
+    if (count++ > 0) {
       ss << ",";
     }
     ss << it->ToString();
   }
   ss << "]]";
-  return ss.str();   
+  return ss.str();
 }
 
-};
+};  // namespace tubemq
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index d1fac8f..c699266 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -17,14 +17,14 @@
  * under the License.
  */
 
+#include "utils.h"
+#include <sstream>
 #include <vector>
-#include <sstream> 
+#include "const_config.h"
 #include <regex.h>
 #include <stdlib.h>
-#include <unistd.h>
 #include <sys/time.h>
-#include "utils.h"
-#include "const_config.h"
+#include <unistd.h>
 
 
 namespace tubemq {
@@ -48,21 +48,21 @@ string Utils::Trim(const string& source) {
 
 void Utils::Split(const string& source, vector<string>& result, const string& delimiter) {
   string item_str;
-  string::size_type pos1,pos2;
+  string::size_type pos1 = 0;
+  string::size_type pos2 = 0;
   result.clear();
   if (!source.empty()) {
     pos1 = 0;
     pos2 = source.find(delimiter);
     while (string::npos != pos2) {
-      item_str = Utils::Trim(source.substr(pos1, pos2-pos1));
+      item_str = Utils::Trim(source.substr(pos1, pos2 - pos1));
       pos1 = pos2 + delimiter.size();
       pos2 = source.find(delimiter, pos1);
       if (!item_str.empty()) {
         result.push_back(item_str);
       }
     }
-    if (pos1 != source.length())
-    {
+    if (pos1 != source.length()) {
       item_str = Utils::Trim(source.substr(pos1));
       if (!item_str.empty()) {
         result.push_back(item_str);
@@ -72,12 +72,14 @@ void Utils::Split(const string& source, vector<string>& result, const string& de
 }
 
 
-void Utils::Split(const string& source, map<string, int>& result, 
+void Utils::Split(const string& source, map<string, int>& result,
                 const string& delimiter_step1, const string& delimiter_step2) {
   string item_str;
   string key_str;
   string val_str;
-  string::size_type pos1,pos2,pos3;
+  string::size_type pos1 = 0;
+  string::size_type pos2 = 0;
+  string::size_type pos3 = 0;
   if (!source.empty()) {
     pos1 = 0;
     pos2 = source.find(delimiter_step1);
@@ -111,7 +113,7 @@ void Utils::Split(const string& source, map<string, int>& result,
         val_str = item_str.substr(pos3+delimiter_step2.length());
         key_str = Utils::Trim(key_str);
         val_str = Utils::Trim(val_str);
-        if (!key_str.empty()){
+        if (!key_str.empty()) {
           result[key_str] = atoi(val_str.c_str());
         }
       }
@@ -130,8 +132,8 @@ void Utils::Join(const vector<string>& vec, const string& delimiter, string& tar
   }
 }
 
-bool Utils::ValidString(string& err_info, const string& source, 
-                bool allow_empty, bool pat_match, bool check_max_length, 
+bool Utils::ValidString(string& err_info, const string& source,
+                bool allow_empty, bool pat_match, bool check_max_length,
                 unsigned int maxlen) {
   if (source.empty()) {
     if (allow_empty) {
@@ -153,11 +155,11 @@ bool Utils::ValidString(string& err_info, const string& source,
   }
 
   if (pat_match) {
-    int cflags =REG_EXTENDED;     
-    regex_t reg;    
+    int cflags = REG_EXTENDED;
+    regex_t reg;
     regmatch_t pmatch[1];
-    const char* patRule = "^[a-zA-Z]\\w+$";  
-    regcomp(&reg, patRule,cflags);
+    const char* patRule = "^[a-zA-Z]\\w+$";
+    regcomp(&reg, patRule, cflags);
     int status = regexec(&reg, source.c_str(), 1, pmatch, 0);
     regfree(&reg);
     if (status == REG_NOMATCH) {
@@ -169,10 +171,10 @@ bool Utils::ValidString(string& err_info, const string& source,
     }
   }
   err_info = "Ok";
-  return true;        
+  return true;
 }
 
-bool Utils::ValidGroupName(string& err_info, 
+bool Utils::ValidGroupName(string& err_info,
                 const string& group_name, string& tgt_group_name) {
   tgt_group_name = Utils::Trim(group_name);
   if (tgt_group_name.empty()) {
@@ -188,11 +190,11 @@ bool Utils::ValidGroupName(string& err_info,
     err_info = ss.str();
     return false;
   }
-  int cflags =REG_EXTENDED;     
-  regex_t reg;    
+  int cflags = REG_EXTENDED;
+  regex_t reg;
   regmatch_t pmatch[1];
-  const char* patRule = "^[a-zA-Z][\\w-]+$"; 
-  regcomp(&reg, patRule,cflags);
+  const char* patRule = "^[a-zA-Z][\\w-]+$";
+  regcomp(&reg, patRule, cflags);
   int status = regexec(&reg, tgt_group_name.c_str(), 1, pmatch, 0);
   regfree(&reg);
   if (status == REG_NOMATCH) {
@@ -205,10 +207,10 @@ bool Utils::ValidGroupName(string& err_info,
     return false;
   }
   err_info = "Ok";
-  return true;        
+  return true;
 }
 
-bool Utils::ValidFilterItem(string& err_info, 
+bool Utils::ValidFilterItem(string& err_info,
                 const string& src_filteritem, string& tgt_filteritem) {
   tgt_filteritem = Utils::Trim(src_filteritem);
   if (tgt_filteritem.empty()) {
@@ -223,11 +225,11 @@ bool Utils::ValidFilterItem(string& err_info,
     err_info = ss.str();
     return false;
   }
-  int cflags =REG_EXTENDED;    
-  regex_t reg;    
+  int cflags = REG_EXTENDED;
+  regex_t reg;
   regmatch_t pmatch[1];
-  const char* patRule = "^[_A-Za-z0-9]+$";  
-  regcomp(&reg, patRule,cflags);
+  const char* patRule = "^[_A-Za-z0-9]+$";
+  regcomp(&reg, patRule, cflags);
   int status = regexec(&reg, tgt_filteritem.c_str(), 1, pmatch, 0);
   regfree(&reg);
   if (status == REG_NOMATCH) {
@@ -235,24 +237,23 @@ bool Utils::ValidFilterItem(string& err_info,
     return false;
   }
   err_info = "Ok";
-  return true;      
+  return true;
 }
 
-
-string Utils::Int2str(int data) {
+string Utils::Int2str(int32_t data) {
   stringstream ss;
-  ss<<data;
+  ss << data;
   return ss.str();
 }
 
-string Utils::Long2str(long data) {
+string Utils::Long2str(int64_t data) {
   stringstream ss;
-  ss<<data;
+  ss << data;
   return ss.str();
 }
 
-int Utils::IpToInt(const string& ipv4_addr) {
-  int result = 0;
+uint32_t Utils::IpToInt(const string& ipv4_addr) {
+  uint32_t result = 0;
   vector<string> result_vec;
 
   Utils::Split(ipv4_addr, result_vec, delimiter::kDelimiterDot);
@@ -263,13 +264,13 @@ int Utils::IpToInt(const string& ipv4_addr) {
   return result;
 }
 
-long Utils::GetCurrentTimeMillis() {
+int64_t Utils::GetCurrentTimeMillis() {
   struct timeval tv;
-  gettimeofday(&tv,NULL);
+  gettimeofday(&tv, NULL);
   return tv.tv_sec * 1000 + tv.tv_usec /1000;
 }
 
 
 
-}
+}  // namespace tubemq
 


[incubator-tubemq] 38/50: [TUBEMQ-286]Create C/C++ SDK's manager class (#213)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 2922467698d58f700572b4bc5d1b7bd250d9323c
Author: gosonzhang <46...@qq.com>
AuthorDate: Tue Jul 21 01:44:58 2020 +0000

    [TUBEMQ-286]Create C/C++ SDK's manager class (#213)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/client_service.h                | 13 ++---
 .../include/tubemq/client_subinfo.h                | 10 ++--
 .../include/tubemq/flowctrl_def.h                  |  2 +-
 .../include/tubemq/rmt_data_cache.h                |  6 +--
 .../include/tubemq/tubemq_config.h                 |  4 +-
 .../tubemq-client-cpp/include/tubemq/utils.h       |  1 +
 .../tubemq-client-cpp/src/client_service.cc        | 18 ++++---
 .../tubemq-client-cpp/src/client_subinfo.cc        | 27 +++++------
 .../tubemq-client-cpp/src/meta_info.cc             |  1 -
 .../tubemq-client-cpp/src/rmt_data_cache.cc        |  8 ++--
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc | 56 +++++++++++++++++++++-
 11 files changed, 103 insertions(+), 43 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
index 5e0b113..ef4ae78 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
@@ -68,24 +68,25 @@ class TubeMQService : public Singleton<TubeMQService> {
   bool IsRunning();
   const int32_t  GetServiceStatus() const { return service_status_.Get(); }
   int32_t GetClientObjCnt();
-  bool AddClientObj(string& err_info,
-         BaseClient* client_obj, int32_t& client_index);
+  bool AddClientObj(string& err_info, BaseClient* client_obj);
   BaseClient* GetClientObj(int32_t client_index) const;
   BaseClient* RmvClientObj(int32_t client_index);
-  const ExecutorPoolPtr& GetTimerExecutor() const { return timer_executor_; }
-  const ExecutorPoolPtr& GetNetWorkExecutor() const { return network_executor_; }
+  const string& GetLocalHost() const { return local_host_; }
+  const ExecutorPool& GetTimerExecutorPool() const { return timer_executor_; }
+  const ExecutorPool& GetNetWorkExecutorPool() const { return network_executor_; }
 
  private:
   void iniLogger(const Fileini& fileini, const string& sector);
   void shutDownClinets() const;
 
  private:
+  string local_host_;  
   AtomicInteger service_status_;
   AtomicInteger client_index_base_;
   mutable mutex mutex_;
   map<int32_t, BaseClient*> clients_map_;
-  ExecutorPoolPtr timer_executor_;
-  ExecutorPoolPtr network_executor_;
+  ExecutorPool timer_executor_;
+  ExecutorPool network_executor_;
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
index 613f97c..14bcbbd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
@@ -26,6 +26,7 @@
 #include <set>
 #include <string>
 #include "tubemq/atomic_def.h"
+#include "tubemq/tubemq_config.h"
 
 namespace tubemq {
 
@@ -51,14 +52,11 @@ class MasterAddrInfo {
 class ClientSubInfo {
  public:
   ClientSubInfo();
-  void SetConsumeTarget(bool bound_consume,
-                               const map<string, set<string> >& topic_and_filter_map,
-                               const string& session_key, uint32_t source_count,
-                               bool select_big, const map<string, int64_t>& part_offset_map);
+  void SetConsumeTarget(const ConsumerConfig& config);
   bool CompAndSetNotAllocated(bool expect, bool update);
   void BookFstRegistered() { first_registered_.Set(true); }
-  bool IsBoundConsume() { return bound_consume_; }
-  bool IsNotAllocated() { return not_allocated_.Get(); }
+  bool IsBoundConsume() const { return bound_consume_; }
+  bool IsNotAllocated() const { return not_allocated_.Get(); }
   const int64_t GetSubscribedTime() const { return subscribed_time_; }
   const string& GetSessionKey() const { return session_key_; }
   const uint32_t GetSourceCnt() const { return source_count_; }
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
index 8b99cbd..66b5aba 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
@@ -101,7 +101,7 @@ class FlowCtrlRuleHandler {
   int32_t GetMinZeroCnt() const { return this->min_zero_cnt_.Get(); }
   int32_t GetQryPriorityId() const { return this->qrypriority_id_.Get(); }
   void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
-  int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
+  const int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
 
  private:
   void initialStatisData();
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index 98f192e..3b85462 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -63,6 +63,8 @@ class RmtDataCacheCsm {
   void UpdateGroupFlowCtrlInfo(int32_t qyrpriority_id,
                  int64_t flowctrl_id, const string& flowctrl_info);
   const int64_t GetGroupQryPriorityId() const;
+  const int64_t GetDefFlowCtrlId() const { return def_flowctrl_handler_.GetFlowCtrlId(); }
+  const int64_t GetGroupFlowCtrlId() const { return group_flowctrl_handler_.GetFlowCtrlId(); }
   bool IsUnderGroupCtrl();
   void AddNewPartition(const PartitionExt& partition_ext);
   bool SelectPartition(string &err_info,
@@ -91,7 +93,7 @@ class RmtDataCacheCsm {
   bool RemovePartition(string &err_info, const string& confirm_context);
   void RemoveAndGetPartition(const list<SubscribeInfo>& subscribe_infos,
         bool is_processing_rollback, map<NodeInfo, list<PartitionExt> >& broker_parts);
-  bool BookPartition(const string& partition_key);
+  bool IsPartitionFirstReg(const string& partition_key);
   void OfferEvent(const ConsumerEvent& event);
   void TakeEvent(ConsumerEvent& event);
   void ClearEvent();
@@ -112,8 +114,6 @@ class RmtDataCacheCsm {
 
 
  private:
-  // timer executor
-  ExecutorPool executor_;
   // 
   string consumer_id_;
   string group_name_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
index 3660cbb..f2b9952 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/tubemq_config.h
@@ -99,10 +99,10 @@ class ConsumerConfig : public BaseConfig {
                              const map<string, set<string> >& subscribed_topic_and_filter_map,
                              const string& session_key, uint32_t source_count, bool is_select_big,
                              const map<string, int64_t>& part_offset_map);
-  bool IsBoundConsume() { return is_bound_consume_; }
+  bool IsBoundConsume() const { return is_bound_consume_; }
   const string& GetSessionKey() const { return session_key_; }
   const uint32_t GetSourceCount() const { return source_count_; }
-  bool IsSelectBig() { return is_select_big_; }
+  bool IsSelectBig() const { return is_select_big_; }
   const map<string, int64_t>& GetPartOffsetInfo() const { return part_offset_map_; }
   const string& GetGroupName() const;
   const map<string, set<string> >& GetSubTopicAndFilterMap() const;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
index 3fa9d48..3bd3068 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
@@ -52,6 +52,7 @@ class Utils {
   static uint32_t IpToInt(const string& ipv4_addr);
   static int64_t GetCurrentTimeMillis();
   static bool ValidConfigFile(string& err_info, const string& conf_file);
+  static bool GetLocalIPV4Address(string& err_info, string& localhost);
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
index 1357123..8e001a7 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
@@ -67,19 +67,25 @@ bool TubeMQService::Start(string& err_info, string conf_file) {
   if (!result) {
     return result;
   }
+  result = Utils::GetLocalIPV4Address(err_info, local_host_);
+  if (!result) {
+    return result;
+  }
   if (!service_status_.CompareAndSet(0,1)) {
     err_info = "TubeMQ Service has startted or Stopped!";
     return false;
   }
   iniLogger(fileini, sector);
-  service_status_.set(2);
+  service_status_.Set(2);
+  err_info = "Ok!";
+  return true;
 }
 
 bool TubeMQService::Stop(string& err_info) {
   if (service_status_.CompareAndSet(2, -1)) {
     shutDownClinets();
-    timer_executor_->Close();
-    network_executor_->Close();
+    timer_executor_.Close();
+    network_executor_.Close();
   }
   err_info = "OK!";
   return true;
@@ -110,14 +116,14 @@ int32_t TubeMQService::GetClientObjCnt() {
 }
 
 
-bool TubeMQService::AddClientObj(string& err_info,
-           BaseClient* client_obj, int32_t& client_index) {
+bool TubeMQService::AddClientObj(string& err_info, BaseClient* client_obj) {
   if (service_status_.Get() != 0) {
     err_info = "Service not startted!";
     return false;
   }
-  client_index = client_index_base_.IncrementAndGet();
+  int32_t client_index = client_index_base_.IncrementAndGet();
   lock_guard<mutex> lck(mutex_);
+  client_obj->SetClientIndex(client_index);
   this->clients_map_[client_index] = client_obj;
   err_info = "Ok";
   return true;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
index e57950c..a6c0fc1 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
@@ -80,25 +80,23 @@ ClientSubInfo::ClientSubInfo() {
   bound_partions_ = "";
 }
 
-void ClientSubInfo::SetConsumeTarget(bool bound_consume,
-         const map<string, set<string> >& topic_and_filter_map,
-         const string& session_key, uint32_t source_count,
-         bool select_big, const map<string, int64_t>& part_offset_map) {
+
+void ClientSubInfo::SetConsumeTarget(const ConsumerConfig& config) {
   int32_t count = 0;
   string tmpstr = "";
   // book register time
   subscribed_time_ = Utils::GetCurrentTimeMillis();
   //
   first_registered_.Set(false);
-  bound_consume_ = bound_consume;
-  topic_and_filter_map_ = topic_and_filter_map;
+  bound_consume_ = config.IsBoundConsume();
+  topic_and_filter_map_ = config.GetSubTopicAndFilterMap();
   // build topic filter info
   topics_.clear();
   topic_conds_.clear();
   set<string>::iterator it_set;
   map<string, set<string> >::const_iterator it_topic;
-  for (it_topic = topic_and_filter_map.begin();
-      it_topic != topic_and_filter_map.end(); it_topic++) {
+  for (it_topic = topic_and_filter_map_.begin();
+      it_topic != topic_and_filter_map_.end(); it_topic++) {
     topics_.push_back(it_topic->first);
     if (it_topic->second.empty()) {
       topic_filter_map_[it_topic->first] = false;
@@ -121,15 +119,16 @@ void ClientSubInfo::SetConsumeTarget(bool bound_consume,
   }
 
   // build bound_partition info
-  if (bound_consume) {
-    session_key_ = session_key;
-    source_count_ = source_count;
-    select_big_ = select_big;
-    assigned_part_map_ = part_offset_map;
+  if (bound_consume_) {
+    session_key_ = config.GetSessionKey();
+    source_count_ = config.GetSourceCount();
+    select_big_ = config.IsSelectBig();
+    assigned_part_map_ = config.GetPartOffsetInfo();
     count = 0;
     bound_partions_ = "";
     map<string, int64_t>::const_iterator it;
-    for (it = part_offset_map.begin(); it != part_offset_map.end(); it++) {
+    for (it = assigned_part_map_.begin();
+      it != assigned_part_map_.end(); it++) {
       if (count++ > 0) {
         bound_partions_ += delimiter::kDelimiterComma;
       }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 9299f71..4f03dfc 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -426,7 +426,6 @@ void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_ha
 SubscribeInfo::SubscribeInfo() {
   this->consumer_id_ = " ";
   this->group_ = " ";
-  this->partitionext_;
   buildSubInfo();
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index c9c499d..f97535c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -22,6 +22,7 @@
 #include <stdlib.h>
 #include <string>
 
+#include "tubemq/client_service.h"
 #include "tubemq/const_config.h"
 #include "tubemq/meta_info.h"
 #include "tubemq/utils.h"
@@ -399,7 +400,7 @@ void RmtDataCacheCsm::RemoveAndGetPartition(const list<SubscribeInfo>& subscribe
 
 
 
-bool RmtDataCacheCsm::BookPartition(const string& partition_key) {
+bool RmtDataCacheCsm::IsPartitionFirstReg(const string& partition_key) {
   bool result = false;
   map<string, bool>::iterator it;
 
@@ -458,9 +459,10 @@ void RmtDataCacheCsm::HandleTimeout(const string partition_key,
 void RmtDataCacheCsm::addDelayTimer(const string& partition_key, int64_t delay_time) {
   // add timer
   tuple<int64_t, SteadyTimerPtr> timer = 
-      std::make_tuple(Utils::GetCurrentTimeMillis(), executor_.Get()->CreateSteadyTimer());
+      std::make_tuple(Utils::GetCurrentTimeMillis(),
+      TubeMQService::Instance().GetTimerExecutorPool().Get()->CreateSteadyTimer());
   std::get<1>(timer)->expires_after(std::chrono::milliseconds(delay_time));
-  std::get<1>(timer)->async_wait(std::bind(&RmtDataCacheCsm::HandleTimeout, partition_key, _1));
+  std::get<1>(timer)->async_wait(std::bind(&RmtDataCacheCsm::HandleTimeout, this, partition_key, _1));
   partition_timeouts_.insert(std::make_pair(partition_key, timer));          
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index ba6ac3a..75fcba5 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -19,9 +19,15 @@
 
 #include "tubemq/utils.h"
 
+#include <arpa/inet.h>
+#include <linux/if.h>
+#include <netinet/in.h>
 #include <regex.h>
 #include <stdlib.h>
 #include <stdio.h>
+#include <string.h>
+#include <sys/ioctl.h>
+#include <sys/socket.h>
 #include <sys/time.h>
 #include <unistd.h>
 
@@ -280,7 +286,7 @@ bool Utils::ValidConfigFile(string& err_info, const string& conf_file) {
     err_info = "Configure file is blank";
     return false;
   }  
-  fp = fopen(configFile.c_str(),"r");
+  fp = fopen(conf_file.c_str(),"r");
   if(fp == NULL) {
     err_info = "Open configure file Failed!";
     return false;
@@ -290,6 +296,54 @@ bool Utils::ValidConfigFile(string& err_info, const string& conf_file) {
   return true;
 }
 
+bool Utils::GetLocalIPV4Address(string& err_info, string& localhost) {
+  int32_t sockfd;
+  int32_t ip_num = 0;
+  char  buf[1024] = {0};
+  struct ifreq *ifreq;
+  struct ifreq if_flag;
+  struct ifconf ifconf;
+
+  ifconf.ifc_len = sizeof(buf);
+  ifconf.ifc_buf = buf;
+  if ((sockfd = socket(AF_INET, SOCK_DGRAM, 0)) < 0) {
+    err_info = "Open the local socket(AF_INET, SOCK_DGRAM) failure!";
+    return false;
+  }
+
+  ioctl(sockfd, SIOCGIFCONF, &ifconf);
+  ifreq  = (struct ifreq *)buf;
+  ip_num = ifconf.ifc_len / sizeof(struct ifreq);
+  for (int32_t i = 0; i < ip_num; i++, ifreq++) {
+    if (ifreq->ifr_flags != AF_INET) {
+      continue;
+    }
+    if (0 == strncmp(&ifreq->ifr_name[0], "lo", sizeof("lo"))) {
+      continue;
+    }
+    memcpy(&if_flag.ifr_name[0],&ifreq->ifr_name[0],sizeof(ifreq->ifr_name));
+    if ((ioctl(sockfd, SIOCGIFFLAGS, (char *) &if_flag)) < 0) {
+      continue;
+    }
+    if ((if_flag.ifr_flags & IFF_LOOPBACK)
+      || !(if_flag.ifr_flags & IFF_UP)) {
+      continue;
+    }
+    
+    if (!strncmp(inet_ntoa(((struct sockaddr_in*)&(ifreq->ifr_addr))->sin_addr), 
+      "127.0.0.1", 7)) {
+      continue;
+    }
+    localhost = inet_ntoa(((struct sockaddr_in*)&(ifreq->ifr_addr))->sin_addr);
+    close(sockfd);
+    err_info = "Ok";
+    return true;
+  }
+  close(sockfd);
+  err_info = "Not found the localHost in local OS";
+  return false;
+}
+
 
 }  // namespace tubemq
 


[incubator-tubemq] 18/50: [TUBEMQ-252] Create C/C++ Metadata classes (#187)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 78800691e16b05a75019c2a0c6c9b706c0462508
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 6 07:54:20 2020 +0000

    [TUBEMQ-252] Create C/C++ Metadata classes (#187)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/meta_info.h              |  62 ++++-
 .../tubemq-client-cpp/src/message.cc               |   2 +-
 .../tubemq-client-cpp/src/meta_info.cc             | 309 ++++++++++++++++++++-
 3 files changed, 369 insertions(+), 4 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
index 55baa4c..6264657 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
@@ -20,6 +20,7 @@
 #ifndef _TUBEMQ_CLIENT_META_INFO_H_
 #define _TUBEMQ_CLIENT_META_INFO_H_
 
+#include <list>
 #include <string>
 
 namespace tubemq {
@@ -62,7 +63,7 @@ class Partition {
  public:
   Partition();
   Partition(const string& partition_info);
-  Partition(const NodeInfo& broker_info, const string& partStr);
+  Partition(const NodeInfo& broker_info, const string& part_str);
   Partition(const NodeInfo& broker_info, const string& topic, int partition_id);
   ~Partition();
   Partition& operator=(const Partition& target);
@@ -88,6 +89,65 @@ class Partition {
 };
 
 
+class SubscribeInfo {
+ public:
+  SubscribeInfo(const string& sub_info);
+  SubscribeInfo(const string& consumer_id, const string& group, const Partition& partition);
+  SubscribeInfo& operator=(const SubscribeInfo& target);
+  const string& GetConsumerId() const;
+  const string& GetGroup() const;
+  const Partition& GetPartition() const;
+  const int GgetBrokerId() const;
+  const string& GetBrokerHost() const;
+  const int GetBrokerPort() const;
+  const string& GetTopic() const;
+  const int GetPartitionId() const;
+  const string& ToString() const;
+
+ private:
+  void buildSubInfo();
+
+ private:
+  string    consumer_id_;
+  string    group_;
+  Partition partition_;
+  string    sub_info_;
+};
+
+
+class ConsumerEvent {
+ public:
+  ConsumerEvent();
+  ConsumerEvent(const ConsumerEvent& target);
+  ConsumerEvent(long rebalance_id,int event_type, 
+    const list<SubscribeInfo>& subscribeInfo_lst, int event_status);
+  ConsumerEvent& operator=(const ConsumerEvent& target);
+  const long GetRebalanceId() const;
+  const int  GetEventType() const;
+  const int  GetEventStatus() const;
+  void SetEventType(int event_type);
+  void SetEventStatus(int event_status);
+  const list<SubscribeInfo>& GetSubscribeInfoList() const;
+  string ToString();
+
+ private:
+  long rebalance_id_;
+  int  event_type_;
+  int  event_status_;
+  list<SubscribeInfo> subscribe_list_;
+};
+
+
+class PartitionExt : public Partition {
+  PartitionExt();
+  
+
+};
+
+
+
+
+
 }
 
 #endif
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index c5162e3..e737a41 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -190,7 +190,7 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if(trimed_key == kRsvPropKeyFilterItem 
+  if (trimed_key == kRsvPropKeyFilterItem 
     || trimed_key == kRsvPropKeyMsgTime) {
     stringstream ss;
     ss << "Reserved token '";
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index c744b47..8024a6a 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -18,6 +18,7 @@
  */
 
 #include <sstream>
+#include <vector>
 #include <stdlib.h>
 #include "utils.h"
 #include "meta_info.h"
@@ -34,6 +35,7 @@ NodeInfo::NodeInfo() {
   buildStrInfo();
 }
 
+// node_info = node_id:host:port
 NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
   vector<string> result;
   Utils::Split(node_info, result, delimiter::kDelimiterColon);
@@ -41,7 +43,7 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
     this->node_id_   = atoi(result[0].c_str());
     this->node_host_ = result[1];
     this->node_port_ = config::kBrokerPortDef;
-    if(result.size() >= 3){
+    if (result.size() >= 3){
       this->node_port_ = atoi(result[2].c_str());
     }
   } else {
@@ -97,7 +99,7 @@ bool NodeInfo::operator== (const NodeInfo& target) {
 }
 
 bool NodeInfo::operator< (const NodeInfo& target) const {
-  return this->node_id_ < target.node_id_;
+  return this->node_info_ < target.node_info_;
 }
   
 const int NodeInfo::GetNodeId() const {
@@ -135,5 +137,308 @@ void NodeInfo::buildStrInfo() {
 }
 
 
+Partition::Partition() {
+  this->topic_ = " ";
+  this->partition_id_ = config::kInvalidValue;
+  buildPartitionKey();
 }
 
+// partition_info = broker_info#topic:partitionId
+Partition::Partition(const string& partition_info) {
+  // initial process
+  this->topic_ = " ";
+  this->partition_id_ = config::kInvalidValue;
+  // parse partition_info string
+  string::size_type pos=0;
+  string seg_key = delimiter::kDelimiterPound;
+  string token_key = delimiter::kDelimiterColon;
+  // parse broker_info
+  pos = partition_info.find(seg_key);
+  if (pos != string::npos){
+    string broker_info = partition_info.substr(0, pos);
+    broker_info = Utils::Trim(broker_info);
+    this->broker_info_ = NodeInfo(true, broker_info);
+    string part_str = partition_info.substr(pos + seg_key.size(), partition_info.size());
+    part_str = Utils::Trim(part_str);
+    pos = part_str.find(token_key);
+    if (pos != string::npos) {
+      string topic_str = part_str.substr(0, pos);
+      string part_id_str = part_str.substr(pos + token_key.size(), part_str.size());
+      topic_str = Utils::Trim(topic_str);
+      part_id_str = Utils::Trim(part_id_str);
+      this->topic_ = topic_str;
+      this->partition_id_ = atoi(part_id_str.c_str());
+    }
+  }
+  buildPartitionKey();
+}
+  
+// part_str = topic:partition_id
+Partition::Partition(const NodeInfo& broker_info, const string& part_str) {
+  vector<string> result;
+  this->topic_ = " ";
+  this->partition_id_ = config::kInvalidValue;
+  this->broker_info_ = broker_info;
+  Utils::Split(part_str, result, delimiter::kDelimiterColon);
+  if (result.size() >= 2) {
+    this->topic_ = result[0];
+    this->partition_id_ = atoi(result[1].c_str());
+  }
+  buildPartitionKey();
+}
+
+Partition::Partition(const NodeInfo& broker_info, const string& topic, int partition_id) {
+  this->topic_ = topic;
+  this->partition_id_ = partition_id;
+  this->broker_info_ = broker_info;
+  buildPartitionKey();
+}
+
+Partition::~Partition() {
+  //
+}
+
+Partition& Partition::operator=(const Partition& target) {
+  if (this != &target) {
+    this->topic_ = target.topic_;
+    this->partition_id_ = target.partition_id_;
+    this->broker_info_ = target.broker_info_;
+    this->partition_key_ = target.partition_key_;
+    this->partition_info_ = target.partition_info_;
+  }
+  return *this;
+}
+
+bool Partition::operator== (const Partition& target) {
+  if (this == &target) {
+    return true;
+  }
+  if (this->partition_info_ == target.partition_info_) {
+    return true;
+  }
+  return false;
+
+}
+
+const int Partition::GetBrokerId() const {
+  return this->broker_info_.GetNodeId();
+}
+
+const string& Partition::GetBrokerHost() const {
+  return this->broker_info_.GetHost();
+}
+
+const int Partition::GetBrokerPort() const {
+  return this->broker_info_.GetPort();
+}
+
+const string& Partition::GetPartitionKey() const {
+  return this->partition_key_;
+}
+
+const string& Partition::GetTopic() const {
+  return this->topic_;
+}
+
+const NodeInfo& Partition::GetBrokerInfo() const {
+  return this->broker_info_;
+}
+
+const int Partition::GetPartitionId() const {
+  return this->partition_id_;
+}
+
+const string& Partition::ToString() const {
+  return this->partition_info_;
+}
+
+void Partition::buildPartitionKey() {
+  stringstream ss1;
+  ss1 << this->broker_info_.GetNodeId();
+  ss1 << delimiter::kDelimiterColon;
+  ss1 << this->topic_;
+  ss1 << delimiter::kDelimiterColon;
+  ss1 << this->partition_id_;
+  this->partition_key_ = ss1.str();
+
+  stringstream ss2;
+  ss2 << this->broker_info_.GetNodeInfo();
+  ss2 << delimiter::kDelimiterPound;
+  ss2 << this->topic_;
+  ss2 << delimiter::kDelimiterColon;
+  ss2 << this->partition_id_;
+  this->partition_info_ = ss2.str();
+}
+
+
+// sub_info = consumerId@group#broker_info#topic:partitionId
+SubscribeInfo::SubscribeInfo(const string& sub_info) {
+  string::size_type pos=0;
+  string seg_key = delimiter::kDelimiterPound;
+  string at_key = delimiter::kDelimiterAt;
+  this->consumer_id_ = " ";
+  this->group_ = " ";
+  // parse sub_info
+  pos=sub_info.find(seg_key);
+  if (pos != string::npos) {
+    string consumer_info = sub_info.substr(0, pos);
+    consumer_info = Utils::Trim(consumer_info);
+    string partition_info = sub_info.substr(pos + seg_key.size(), sub_info.size());
+    partition_info = Utils::Trim(partition_info);
+    this->partition_ = Partition(partition_info);
+    pos = consumer_info.find(at_key);
+    this->consumer_id_ = consumer_info.substr(0, pos);
+    this->consumer_id_ = Utils::Trim(this->consumer_id_);
+    this->group_ = consumer_info.substr(pos + at_key.size(), consumer_info.size());
+    this->group_ = Utils::Trim(this->group_);
+  }
+  buildSubInfo();
+}
+
+SubscribeInfo::SubscribeInfo(const string& consumer_id, 
+                 const string& group, const Partition& partition) {
+  this->consumer_id_ = consumer_id;
+  this->group_       = group;
+  this->partition_   = partition;
+  buildSubInfo();
+}
+
+
+SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
+  if (this != &target) {
+    this->consumer_id_ = target.consumer_id_;
+    this->group_       = target.group_;
+    this->partition_   = target.partition_;
+  }
+  return *this;
+}
+
+const string& SubscribeInfo::GetConsumerId() const {
+  return this->consumer_id_;
+}
+
+const string& SubscribeInfo::GetGroup() const {
+  return this->group_;
+}
+
+const Partition& SubscribeInfo::GetPartition() const {
+  return this->partition_;
+}
+
+const int SubscribeInfo::GgetBrokerId() const {
+  return this->partition_.GetBrokerId();
+}
+
+const string& SubscribeInfo::GetBrokerHost() const {
+  return this->partition_.GetBrokerHost();
+}
+
+const int SubscribeInfo::GetBrokerPort() const {
+  return this->partition_.GetBrokerPort();
+}
+
+const string& SubscribeInfo::GetTopic() const {
+  return this->partition_.GetTopic();
+}
+
+const int SubscribeInfo::GetPartitionId() const {
+  return this->partition_.GetPartitionId();
+}
+
+const string& SubscribeInfo::ToString() const {
+  return this->sub_info_;
+}
+
+void SubscribeInfo::buildSubInfo() {
+  stringstream ss;
+  ss << this->consumer_id_;
+  ss << delimiter::kDelimiterAt;
+  ss << this->group_;
+  ss << delimiter::kDelimiterPound;
+  ss << this->partition_.ToString();
+  this->sub_info_ = ss.str();
+}
+
+
+ConsumerEvent::ConsumerEvent() {
+  this->rebalance_id_ = config::kInvalidValue;
+  this->event_type_   = config::kInvalidValue;
+  this->event_status_ = config::kInvalidValue;
+}
+
+ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
+  this->rebalance_id_ = target.rebalance_id_;
+  this->event_type_   = target.event_type_;
+  this->event_status_ = target.event_status_;
+  this->subscribe_list_ = target.subscribe_list_;
+}
+
+ConsumerEvent::ConsumerEvent(long rebalance_id,int event_type, 
+    const list<SubscribeInfo>& subscribeInfo_lst, int event_status) {
+  list<SubscribeInfo>::const_iterator it;
+  this->rebalance_id_ = rebalance_id;
+  this->event_type_   = event_type;
+  this->event_status_ = event_status;
+  for (it = subscribeInfo_lst.begin(); it != subscribeInfo_lst.end(); ++it) {
+    this->subscribe_list_.push_back(*it);
+  }
+}
+
+ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
+  if(this != &target){
+    this->rebalance_id_ = target.rebalance_id_;
+    this->event_type_ = target.event_type_;
+    this->event_status_ = target.event_status_;
+    this->subscribe_list_ = target.subscribe_list_;
+  }
+  return *this;
+}
+
+const long ConsumerEvent::GetRebalanceId() const {
+  return this->rebalance_id_;
+}
+
+const int ConsumerEvent::GetEventType() const {
+  return this->event_type_;
+}
+
+const int ConsumerEvent::GetEventStatus() const {
+  return this->event_status_;
+}
+
+void ConsumerEvent::SetEventType(int event_type) {
+  this->event_type_ = event_type;
+}
+
+void ConsumerEvent::SetEventStatus(int event_status) {
+  this->event_status_ = event_status;
+}
+
+const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
+  return this->subscribe_list_;
+}
+
+string ConsumerEvent::ToString() {
+  int count = 0;
+  stringstream ss;
+  list<SubscribeInfo>::const_iterator it;
+  ss << "ConsumerEvent [rebalanceId=";
+  ss << this->rebalance_id_;
+  ss << ", type=";
+  ss << this->event_type_;
+  ss << ", status=";
+  ss << this->event_status_;
+  ss << ", subscribeInfoList=[";
+  for (it = this->subscribe_list_.begin(); 
+          it != this->subscribe_list_.end(); ++it) {
+    if(count++ > 0) {
+      ss << ",";
+    }
+    ss << it->ToString();
+  }
+  ss << "]]";
+  return ss.str();   
+}
+
+};
+


[incubator-tubemq] 45/50: [TUBEMQ-293]C++ SDK Create Future class (#257)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 8d20dc3279aa43e7ff00a5f6750b31fb3b79d09e
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Mon Sep 14 17:21:42 2020 +0800

    [TUBEMQ-293]C++ SDK Create Future class (#257)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 tubemq-client-twins/tubemq-client-cpp/src/future.h | 143 +++++++++++++++++++++
 1 file changed, 143 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/future.h b/tubemq-client-twins/tubemq-client-cpp/src/future.h
new file mode 100644
index 0000000..181a0b4
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/future.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 _TUBEMQ_FUTURE_H_
+#define _TUBEMQ_FUTURE_H_
+
+#include <condition_variable>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <vector>
+
+#include "tubemq/tubemq_errcode.h"
+
+namespace tubemq {
+
+template <typename Value>
+struct FutureInnerState {
+  std::mutex mutex_;
+  std::condition_variable condition_;
+  ErrorCode error_code_;
+  Value value_;
+  bool ready_ = false;
+  bool failed_ = false;
+  using FutureCallBackFunc = std::function<void(ErrorCode, const Value&)>;
+  std::vector<FutureCallBackFunc> callbacks_;
+};
+
+template <typename Value>
+class Future {
+ public:
+  using FutureInnerStatePtr = std::shared_ptr<FutureInnerState<Value> >;
+  using FutureCallBackFunc = std::function<void(ErrorCode, const Value&)>;
+  Future& AddCallBack(FutureCallBackFunc callback) {
+    Lock lock(state_->mutex_);
+
+    if (state_->ready_) {
+      lock.unlock();
+      callback(state_->error_code_, state_->value_);
+    } else {
+      state_->callbacks_.push_back(callback);
+    }
+    return *this;
+  }
+
+  ErrorCode Get(Value& value) {
+    Lock lock(state_->mutex_);
+
+    if (!state_->ready_) {
+      // Wait for error_code_
+      while (!state_->ready_) {
+        state_->condition_.wait(lock);
+      }
+    }
+
+    value = state_->value_;
+    return state_->error_code_;
+  }
+
+ private:
+  using Lock = std::unique_lock<std::mutex>;
+  explicit Future(FutureInnerStatePtr state) : state_(state) {}
+  FutureInnerStatePtr state_;
+
+  template <typename V>
+  friend class Promise;
+};
+
+template <typename Value>
+class Promise {
+ public:
+  using FutureInnerStatePtr = std::shared_ptr<FutureInnerState<Value> >;
+  using FutureCallBackFunc = std::function<void(ErrorCode, const Value&)>;
+  Promise() : state_(std::make_shared<FutureInnerState<Value> >()) {}
+
+  bool SetValue(const Value& value) {
+    Lock lock(state_->mutex_);
+
+    if (state_->ready_) {
+      return false;
+    }
+
+    state_->value_ = value;
+    state_->ready_ = true;
+
+    callbackAndNotify();
+    return true;
+  }
+
+  bool SetFailed(const ErrorCode& error_code_) {
+    Lock lock(state_->mutex_);
+
+    if (state_->ready_) {
+      return false;
+    }
+
+    state_->error_code_ = error_code_;
+    state_->ready_ = true;
+    state_->failed_ = true;
+
+    callbackAndNotify();
+    return true;
+  }
+
+  bool IsReady() const { return state_->ready_; }
+
+  bool IsFailed() const { return state_->failed_; }
+
+  Future<Value> GetFuture() const { return Future<Value>(state_); }
+
+ private:
+  void callbackAndNotify() {
+    for (auto callback : state_->callbacks_) {
+      callback(state_->error_code_, state_->value_);
+    }
+    state_->callbacks_.clear();
+    state_->condition_.notify_all();
+  }
+
+ private:
+  using Lock = std::unique_lock<std::mutex>;
+  FutureInnerStatePtr state_;
+};
+
+} /* namespace tubemq */
+
+#endif /* _TUBEMQ_FUTURE_H_ */


[incubator-tubemq] 48/50: [TUBEMQ-348]C++SDK Client handler detail (#260)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit de4960d285c40479ccdb504b84d435a089ccc4c1
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Mon Sep 14 20:25:33 2020 +0800

    [TUBEMQ-348]C++SDK Client handler detail (#260)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/src/baseconsumer.cc          | 1619 ++++++++++++++++++++
 .../tubemq-client-cpp/src/baseconsumer.h           |  146 ++
 .../tubemq-client-cpp/src/client_service.cc        |  196 ++-
 .../tubemq-client-cpp/src/client_service.h         |  296 ++++
 4 files changed, 2224 insertions(+), 33 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
new file mode 100644
index 0000000..215892d
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
@@ -0,0 +1,1619 @@
+/**
+ * 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 "baseconsumer.h"
+
+#include <signal.h>
+#include <unistd.h>
+
+#include <sstream>
+
+#include "client_service.h"
+#include "const_config.h"
+#include "const_rpc.h"
+#include "logger.h"
+#include "singleton.h"
+#include "transport.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_errcode.h"
+#include "tubemq_transport.h"
+#include "utils.h"
+#include "version.h"
+
+namespace tubemq {
+
+using std::lock_guard;
+using std::stringstream;
+
+BaseConsumer::BaseConsumer() : BaseClient(false) {
+  status_.Set(0);
+  unreport_times_ = 0;
+  visit_token_.Set(tb_config::kInvalidValue);
+  nextauth_2_master.Set(false);
+  nextauth_2_broker.Set(false);
+  masters_map_.clear();
+  is_master_actived_.Set(false);
+  master_reg_status_.Set(0);
+  master_hb_status_.Set(0);
+  last_master_hbtime_ = 0;
+  master_sh_retry_cnt_ = 0;
+}
+
+BaseConsumer::~BaseConsumer() {
+  //
+}
+
+bool BaseConsumer::Start(string& err_info, const ConsumerConfig& config) {
+  ConsumerConfig tmp_config;
+  if (!status_.CompareAndSet(0, 1)) {
+    err_info = "Ok";
+    return true;
+  }
+  // check configure
+  if (config.GetGroupName().length() == 0 || config.GetMasterAddrInfo().length() == 0) {
+    err_info = "Parameter error: not set master address info or group name!";
+    return false;
+  }
+  //
+  if (!TubeMQService::Instance()->IsRunning()) {
+    err_info = "TubeMQ Service not startted!";
+    return false;
+  }
+  if (!TubeMQService::Instance()->AddClientObj(err_info, this)) {
+    client_index_ = tb_config::kInvalidValue;
+    status_.CompareAndSet(1, 0);
+    return false;
+  }
+  config_ = config;
+  if (!initMasterAddress(err_info, config.GetMasterAddrInfo())) {
+    return false;
+  }
+  client_uuid_ = buildUUID();
+  sub_info_.SetConsumeTarget(config_);
+  rmtdata_cache_.SetConsumerInfo(client_uuid_, config_.GetGroupName());
+  // initial resource
+
+  // register to master
+  int32_t error_code;
+  if (!register2Master(error_code, err_info, false)) {
+    status_.CompareAndSet(1, 0);
+    return false;
+  }
+  status_.CompareAndSet(1, 2);
+  heart_beat_timer_ = TubeMQService::Instance()->CreateTimer();
+  heart_beat_timer_->expires_after(std::chrono::milliseconds(config_.GetHeartbeatPeriodMs() / 2));
+  heart_beat_timer_->async_wait([this](const std::error_code& ec) {
+    if (ec) {
+      return;
+    }
+    heartBeat2Master();
+  });
+  rebalance_thread_ptr_ = std::make_shared<std::thread>([this]() { processRebalanceEvent(); });
+  LOG_INFO("[CONSUMER] start consumer success, client=%s", client_uuid_.c_str());
+  err_info = "Ok";
+  return true;
+}
+
+void BaseConsumer::ShutDown() {
+  if (!status_.CompareAndSet(2, 0)) {
+    return;
+  }
+  LOG_INFO("[CONSUMER] ShutDown consumer begin, client=%s", client_uuid_.c_str());
+  // 1. exist rebalance thread
+  ConsumerEvent empty_event;
+  rmtdata_cache_.OfferEvent(empty_event);
+  // 2. close to master
+  close2Master();
+  // 3. close all brokers
+  closeAllBrokers();
+  // 4. remove client stub
+  TubeMQService::Instance()->RmvClientObj(this);
+  client_index_ = tb_config::kInvalidValue;
+  // 5. join hb thread;
+  heart_beat_timer_ = nullptr;
+  rebalance_thread_ptr_->join();
+  rebalance_thread_ptr_ = nullptr;
+  LOG_INFO("[CONSUMER] ShutDown consumer finished, client=%s", client_uuid_.c_str());
+}
+
+bool BaseConsumer::GetMessage(ConsumerResult& result) {
+  int32_t error_code;
+  string err_info;
+  PartitionExt partition_ext;
+  string confirm_context;
+  
+  if (!IsConsumeReady(result)) {
+    return false;
+  }
+  if (!rmtdata_cache_.SelectPartition(error_code,
+    err_info, partition_ext, confirm_context)) {
+    result.SetFailureResult(error_code, err_info);
+    return false;
+  }
+  long curr_offset = tb_config::kInvalidValue;
+  bool filter_consume = sub_info_.IsFilterConsume(partition_ext.GetTopic());
+  PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
+    partition_ext.GetPartitionKey(), curr_offset);
+  auto request = std::make_shared<RequestContext>();
+  TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+  // build getmessage request
+  buidGetMessageC2B(partition_ext, req_protocol);
+  request->codec_ = std::make_shared<TubeMQCodec>();
+  request->ip_ = partition_ext.GetBrokerHost();
+  request->port_ = partition_ext.GetBrokerPort();
+  request->timeout_ = config_.GetRpcReadTimeoutMs();
+  request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+  req_protocol->request_id_ = request->request_id_;
+  req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+
+  LOG_TRACE("[CONSUMER] GetMessage select partition=%s, client=%s",
+    partition_ext.GetPartitionKey().c_str(), client_uuid_.c_str());
+
+  // send message to target
+  ResponseContext response_context;
+  ErrorCode error = SyncRequest(response_context, request, req_protocol);
+  LOG_TRACE("[CONSUMER] GetMessage received response, ret_code=%d, client=%s",
+    error.Value(), client_uuid_.c_str());
+  if (!TubeMQService::Instance()->IsRunning()) {
+    result.SetFailureResult(err_code::kErrMQServiceStop, "TubeMQ Service stopped!");
+    return false;
+  }
+  if (!isClientRunning()) {
+    result.SetFailureResult(err_code::kErrClientStop, "TubeMQ Client stopped!");
+    return false;
+  }
+  if (error.Value() == err_code::kErrSuccess) {
+    // process response
+    auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+    processGetMessageRspB2C(result, peer_info, filter_consume, partition_ext, confirm_context, rsp);
+    return result.IsSuccess();
+  } else {
+    rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false);
+    result.SetFailureResult(error.Value(), error.Message(), partition_ext.GetTopic(), peer_info);
+    return false;
+  }
+}
+
+bool BaseConsumer::IsConsumeReady(ConsumerResult& result) {
+  int32_t ret_code;
+  int64_t start_time = Utils::GetCurrentTimeMillis();
+  while (true) {
+    if (!TubeMQService::Instance()->IsRunning()) {
+      result.SetFailureResult(err_code::kErrMQServiceStop, "TubeMQ Service stopped!");
+      return false;
+    }
+    if (!isClientRunning()) {
+      result.SetFailureResult(err_code::kErrClientStop, "TubeMQ Client stopped!");
+      return false;
+    }
+    ret_code = rmtdata_cache_.GetCurConsumeStatus();
+    if (err_code::kErrSuccess == ret_code) {
+      return true;
+    }
+    if ((config_.GetMaxPartCheckPeriodMs() > 0)
+      && (Utils::GetCurrentTimeMillis() - start_time 
+      > config_.GetMaxPartCheckPeriodMs())) {
+      switch (ret_code) {
+        case err_code::kErrNoPartAssigned: {
+          result.SetFailureResult(ret_code,
+            "No partition info in local cache, please retry later!");
+        }
+        break;
+
+        case err_code::kErrAllPartInUse: {
+          result.SetFailureResult(ret_code,
+            "No idle partition to consume, please retry later!");
+        }
+        break;
+
+        case err_code::kErrAllPartWaiting:
+        default: {
+          result.SetFailureResult(ret_code,
+            "All partitions reach max position, please retry later!");
+        }
+        break;
+      }
+      return false;
+    }
+    std::this_thread::sleep_for(std::chrono::milliseconds(config_.GetPartCheckSliceMs()));
+  }
+  return true;
+}
+
+bool BaseConsumer::GetCurConsumedInfo(map<string, ConsumeOffsetInfo>& consume_info_map) {
+  bool has_data = false;
+  consume_info_map.clear();
+  map<string, int64_t> part_offset_map;
+  map<string, int64_t>::iterator it_part;
+  rmtdata_cache_.GetCurPartitionOffsets(part_offset_map);
+  for (it_part = part_offset_map.begin(); it_part != part_offset_map.end(); ++it_part) {
+    ConsumeOffsetInfo tmp_info(it_part->first, it_part->second);
+    consume_info_map[it_part->first] = tmp_info;
+    has_data = true;
+  }
+  return has_data;
+}
+
+bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
+                             ConsumerResult& result) {
+  if (!TubeMQService::Instance()->IsRunning()) {
+    result.SetFailureResult(err_code::kErrMQServiceStop, "TubeMQ Service stopped!");
+    return false;
+  }
+  if (!isClientRunning()) {
+    result.SetFailureResult(err_code::kErrClientStop, "TubeMQ Client stopped!");
+    return false;
+  }
+
+  LOG_TRACE("[CONSUMER] Confirm begin, confirm_context = %s, is_consumed =%d, client=%s",
+    confirm_context.c_str(), is_consumed, client_uuid_.c_str());
+
+  string token1 = delimiter::kDelimiterAt;
+  string token2 = delimiter::kDelimiterColon;
+  string::size_type pos1, pos2;
+  pos1 = confirm_context.find(token1);
+  if (string::npos == pos1) {
+    result.SetFailureResult(
+        err_code::kErrBadRequest,
+        "Illegel confirm_context content: unregular confirm_context value format!");
+    return false;
+  }
+  string part_key = Utils::Trim(confirm_context.substr(0, pos1));
+  string booked_time_str =
+      Utils::Trim(confirm_context.substr(pos1 + token1.size(), confirm_context.size()));
+  long booked_time = atol(booked_time_str.c_str());
+  pos1 = part_key.find(token2);
+  if (string::npos == pos1) {
+    result.SetFailureResult(err_code::kErrBadRequest,
+                            "Illegel confirm_context content: unregular index key value format!");
+    return false;
+  }
+  pos1 = pos1 + token1.size();
+  string topic_name = part_key.substr(pos1);
+  pos2 = topic_name.rfind(token2);
+  if (string::npos == pos2) {
+    result.SetFailureResult(
+        err_code::kErrBadRequest,
+        "Illegel confirm_context content: unregular index's topic key value format!");
+    return false;
+  }
+  topic_name = topic_name.substr(0, pos2);
+  if (!rmtdata_cache_.IsPartitionInUse(part_key, booked_time)) {
+    result.SetFailureResult(err_code::kErrConfirmTimeout, "The confirm_context's value invalid!");
+    return false;
+  }
+  PartitionExt partition_ext;
+  bool ret_result = rmtdata_cache_.GetPartitionExt(part_key, partition_ext);
+  if (!ret_result) {
+    result.SetFailureResult(err_code::kErrConfirmTimeout,
+                            "Not found the partition by confirm_context!");
+    return false;
+  }
+  long curr_offset = tb_config::kInvalidValue;
+  PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
+    partition_ext.GetPartitionKey(), curr_offset);
+  auto request = std::make_shared<RequestContext>();
+  TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+  // build CommitC2B request
+  buidCommitC2B(partition_ext, is_consumed, req_protocol);
+  request->codec_ = std::make_shared<TubeMQCodec>();
+  request->ip_ = partition_ext.GetBrokerHost();
+  request->port_ = partition_ext.GetBrokerPort();
+  request->timeout_ = config_.GetRpcReadTimeoutMs();
+  request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+  req_protocol->request_id_ = request->request_id_;
+  req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+
+  LOG_TRACE("[CONSUMER] Confirm to %s, client=%s",
+    partition_ext.GetPartitionKey().c_str(), client_uuid_.c_str());
+
+  // send message to target
+  ResponseContext response_context;
+  ErrorCode error = SyncRequest(response_context, request, req_protocol);
+  if (!TubeMQService::Instance()->IsRunning()) {
+    result.SetFailureResult(err_code::kErrMQServiceStop, "TubeMQ Service stopped!");
+    return false;
+  }
+  if (!isClientRunning()) {
+    result.SetFailureResult(err_code::kErrClientStop, "TubeMQ Client stopped!");
+    return false;
+  }
+
+  LOG_TRACE("[CONSUMER] Confirm response result=%d, client=%s",
+    error.Value(), client_uuid_.c_str());
+
+  if (error.Value() == err_code::kErrSuccess) {
+    // process response
+    auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+    if (rsp->success_) {
+      CommitOffsetResponseB2C rsp_b2c;
+      ret_result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
+                                          (int)(rsp->rsp_body_.data().length()));
+      if (ret_result) {
+        if (rsp_b2c.success()) {
+          curr_offset = rsp_b2c.curroffset();
+          peer_info.SetCurrOffset(curr_offset);
+          result.SetSuccessResult(err_code::kErrSuccess, topic_name, peer_info);
+        } else {
+          result.SetFailureResult(rsp_b2c.errcode(), rsp_b2c.errmsg(), topic_name, peer_info);
+        }
+      } else {
+        result.SetFailureResult(err_code::kErrParseFailure,
+                                "Parse CommitOffsetResponseB2C response failure!", topic_name,
+                                peer_info);
+      }
+    } else {
+      result.SetFailureResult(rsp->code_, rsp->error_msg_, topic_name, peer_info);
+    }
+  } else {
+    result.SetFailureResult(error.Value(), error.Message(), topic_name, peer_info);
+  }
+  string err_info;
+  rmtdata_cache_.BookedPartionInfo(part_key, curr_offset);
+  rmtdata_cache_.RelPartition(err_info, sub_info_.IsFilterConsume(topic_name), confirm_context,
+                              is_consumed);
+  LOG_TRACE("[CONSUMER] Confirm response finished, result=%d, client=%s",
+    result.IsSuccess(), client_uuid_.c_str());
+  return result.IsSuccess();
+}
+
+bool BaseConsumer::register2Master(int32_t& error_code, string& err_info, bool need_change) {
+  string target_ip;
+  int target_port;
+
+  // set regist process status to begin
+  if (!master_reg_status_.CompareAndSet(0, 1)) {
+    err_info = "register2Master process has began!";
+    return false;
+  }
+
+  LOG_INFO("register2Master process begin: ");
+  // check client status
+  if (status_.Get() == 0) {
+    master_reg_status_.CompareAndSet(1, 0);
+    err_info = "Consumer not startted!";
+    return false;
+  }
+  LOG_DEBUG("[CONSUMER], initial register2master request, clientId=%s", client_uuid_.c_str());
+  // get master address and port
+  if (need_change) {
+    getNextMasterAddr(target_ip, target_port);
+  } else {
+    getCurrentMasterAddr(target_ip, target_port);
+  }
+  bool result = false;
+  int retry_count = 0;
+  int maxRetrycount = masters_map_.size();
+  err_info = "Master register failure, no online master service!";
+  while (retry_count < maxRetrycount) {
+    if (!TubeMQService::Instance()->IsRunning()) {
+      err_info = "TubeMQ Service is stopped!";
+      master_reg_status_.CompareAndSet(1, 0);
+      return false;
+    }
+    auto request = std::make_shared<RequestContext>();
+    TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+    // build register request
+    buidRegisterRequestC2M(req_protocol);
+    // set parameters
+    request->codec_ = std::make_shared<TubeMQCodec>();
+    request->ip_ = target_ip;
+    request->port_ = target_port;
+    request->timeout_ = config_.GetRpcReadTimeoutMs();
+    request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+    req_protocol->request_id_ = request->request_id_;
+    req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+    // send message to target
+    ResponseContext response_context;
+    ErrorCode error = SyncRequest(response_context, request, req_protocol);
+    LOG_INFO("register2Master response come, error.value is %d", error.Value());
+    if (error.Value() == err_code::kErrSuccess) {
+      // process response
+      auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+      result = processRegisterResponseM2C(error_code, err_info, rsp);
+      if (result) {
+        err_info = "Ok";
+        is_master_actived_.Set(true);
+        last_master_hbtime_ = Utils::GetCurrentTimeMillis();
+        break;
+      } else {
+        is_master_actived_.Set(false);
+      }
+    } else {
+      error_code = error.Value();
+      err_info = error.Message();
+    }
+    if (error_code == err_code::kErrConsumeGroupForbidden 
+      || error_code == err_code::kErrConsumeContentForbidden) {
+      // set regist process status to existed
+      master_reg_status_.CompareAndSet(1, 0);
+      LOG_WARN("[CONSUMER] register2master(%s:%d) failure exist register, client=%s,reason:%s",
+               target_ip.c_str(), target_port, client_uuid_.c_str(), err_info.c_str());
+      return false;
+    } else {
+      LOG_WARN(
+          "[CONSUMER] register2master(%s:%d) failure, client=%s, retrycount=(%d-%d), reason:%s",
+          target_ip.c_str(), target_port, client_uuid_.c_str(), maxRetrycount, retry_count + 1,
+          err_info.c_str());
+    }
+    retry_count++;
+    getNextMasterAddr(target_ip, target_port);
+  }
+  // set regist process status to existed
+  master_reg_status_.CompareAndSet(1, 0);
+  LOG_INFO("[CONSUMER] register2Master finished, client=%s, result:%d, err_info:%s",
+           client_uuid_.c_str(), result, err_info.c_str());
+  return result;
+}
+
+void BaseConsumer::asyncRegister2Master(bool need_change) {
+  TubeMQService::Instance()->Post([this, need_change]() {
+    int32_t error_code;
+    string error_info;
+    if (!is_master_actived_.Get()) {
+      auto ret_result = register2Master(error_code, error_info, need_change);
+      LOG_INFO("[CONSUMER] asyncRegister2Master ret_result:%d, master_sh_retry_cnt_:%d", ret_result,
+               master_sh_retry_cnt_);
+      if (ret_result) {
+        is_master_actived_.Set(true);
+        master_sh_retry_cnt_ = 0;
+      } else {
+        master_sh_retry_cnt_++;
+      }
+    }
+    heart_beat_timer_->expires_after(std::chrono::milliseconds(nextHeartBeatPeriodms()));
+    heart_beat_timer_->async_wait([this](const std::error_code& ec) {
+      if (ec) {
+        return;
+      }
+      heartBeat2Master();
+    });
+  });
+}
+
+void BaseConsumer::heartBeat2Master() {
+  // timer task
+  // 1. check if need re-register, if true, first call register
+  // 2. call heartbeat to master
+  // 3. process response
+  // 4. call timer again
+  string target_ip;
+  int target_port;
+
+  // set heartbeat process status to begin
+  if (!master_hb_status_.CompareAndSet(0, 1)) {
+    LOG_INFO("check hb process status, heartBeat2Master process has began!");
+    return;
+  }
+
+  LOG_TRACE("heartBeat2Master process begin:");
+
+  if (!TubeMQService::Instance()->IsRunning()) {
+    master_hb_status_.CompareAndSet(1, 0);
+    LOG_INFO("[CONSUMER] heartBeat2Master failure: TubeMQ Service is stopped! client=%s",
+             client_uuid_.c_str());
+    return;
+  }
+
+  if (!isClientRunning()) {
+    master_hb_status_.CompareAndSet(1, 0);
+    LOG_INFO("[CONSUMER] heartBeat2Master failure: TubeMQ Client stopped! client=%s",
+             client_uuid_.c_str());
+    return;
+  }
+
+  // check status in master
+  // if not actived first register, or send heartbeat
+  if (!is_master_actived_.Get()) {
+    LOG_INFO("[CONSUMER] heartBeat2Master found master not active, re-register first! client=%s",
+             client_uuid_.c_str());
+    asyncRegister2Master(false);
+    master_hb_status_.CompareAndSet(1, 0);
+    return;
+  }
+  // check partition status
+  if (Utils::GetCurrentTimeMillis() - last_master_hbtime_ > 30000) {
+    rmtdata_cache_.handleExpiredPartitions(
+      config_.GetMaxConfirmWaitPeriodMs());
+  }
+  // select current master
+  getCurrentMasterAddr(target_ip, target_port);
+  auto request = std::make_shared<RequestContext>();
+  TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+  // build heartbeat 2 master request
+  buidHeartRequestC2M(req_protocol);
+  request->codec_ = std::make_shared<TubeMQCodec>();
+  request->ip_ = target_ip;
+  request->port_ = target_port;
+  request->timeout_ = config_.GetRpcReadTimeoutMs();
+  request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+  req_protocol->request_id_ = request->request_id_;
+  req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+  // send message to target
+  AsyncRequest(request, req_protocol)
+      .AddCallBack([=](ErrorCode error, const ResponseContext& response_context) {
+        if (error.Value() != err_code::kErrSuccess) {
+          master_sh_retry_cnt_++;
+          LOG_WARN("[CONSUMER] heartBeat2Master failue to (%s:%d) : %s, client=%s",
+                   target_ip.c_str(), target_port, error.Message().c_str(), client_uuid_.c_str());
+        } else {
+          // process response
+          auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+          int32_t error_code = 0;
+          std::string error_info;
+          auto ret_result = processHBResponseM2C(error_code, error_info, rsp);
+          LOG_TRACE("[CONSUMER] processHBResponseM2C return result = %d! client=%s",
+            ret_result, client_uuid_.c_str());
+          if (ret_result) {
+            is_master_actived_.Set(true);
+            master_sh_retry_cnt_ = 0;
+          } else {
+            master_sh_retry_cnt_++;
+            if (error_code == err_code::kErrHbNoNode ||
+                error_info.find("StandbyException") != string::npos) {
+              is_master_actived_.Set(false);
+              LOG_WARN("[CONSUMER] hb2master found no-node or standby, re-register, client=%s",
+                       client_uuid_.c_str());
+              asyncRegister2Master(!(error_code == err_code::kErrHbNoNode));
+              master_hb_status_.CompareAndSet(1, 0);
+              return;
+            }
+          }
+        }
+        heart_beat_timer_->expires_after(std::chrono::milliseconds(nextHeartBeatPeriodms()));
+        heart_beat_timer_->async_wait([this](const std::error_code& ec) {
+          if (ec) {
+            return;
+          }
+          heartBeat2Master();
+        });
+        master_hb_status_.CompareAndSet(1, 0);
+        LOG_TRACE("[CONSUMER] processHBResponseM2C process result finished, client=%s",
+          client_uuid_.c_str());
+      });
+  return;
+}
+
+int32_t BaseConsumer::nextHeartBeatPeriodms() {
+  int32_t next_hb_periodms = config_.GetHeartbeatPeriodMs();
+  if (master_sh_retry_cnt_ >= config_.GetMaxHeartBeatRetryTimes()) {
+    next_hb_periodms = config_.GetHeartbeatPeriodAftFailMs();
+  }
+  return next_hb_periodms;
+}
+
+void BaseConsumer::processRebalanceEvent() {
+  // thread wait until event come
+  LOG_INFO("[CONSUMER] rebalance event Handler startted!");
+  while (true) {
+    if (!TubeMQService::Instance()->IsRunning()) {
+      LOG_INFO("[CONSUMER] Rebalance found Service stopped, existed, client=%s",
+               client_uuid_.c_str());
+      break;
+    }
+    if (!isClientRunning()) {
+      LOG_INFO("[CONSUMER] Rebalance found Client stopped, existed, client=%s",
+               client_uuid_.c_str());
+      break;
+    }
+    ConsumerEvent event;
+    rmtdata_cache_.TakeEvent(event);
+    if (event.GetEventStatus() == tb_config::kInvalidValue &&
+        event.GetRebalanceId() == tb_config::kInvalidValue) {
+      LOG_INFO("[CONSUMER] Rebalance found Shutdown notify, existed, client=%s",
+               client_uuid_.c_str());
+      break;
+    }
+    rmtdata_cache_.ClearEvent();
+    switch (event.GetEventType()) {
+      case 2:
+      case 20: {
+        processDisConnect2Broker(event);
+        rmtdata_cache_.OfferEventResult(event);
+      } break;
+
+      case 1:
+      case 10: {
+        processConnect2Broker(event);
+        rmtdata_cache_.OfferEventResult(event);
+      } break;
+
+      default: {
+        //
+      } break;
+    }
+  }
+
+  LOG_INFO("[CONSUMER] rebalance event Handler stopped!");
+  return;
+}
+
+void BaseConsumer::close2Master() {
+  string target_ip;
+  int target_port;
+
+  LOG_INFO("[CONSUMER] close2Master begin, clientid=%s", client_uuid_.c_str());
+  getCurrentMasterAddr(target_ip, target_port);
+  auto request = std::make_shared<RequestContext>();
+  TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+  // build close2master request
+  buidCloseRequestC2M(req_protocol);
+  request->codec_ = std::make_shared<TubeMQCodec>();
+  request->ip_ = target_ip;
+  request->port_ = target_port;
+  request->timeout_ = config_.GetRpcReadTimeoutMs();
+  request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+  req_protocol->request_id_ = request->request_id_;
+  req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+  // send message to target
+  AsyncRequest(request, req_protocol);
+  LOG_INFO("[CONSUMER] close2Master finished, clientid=%s", client_uuid_.c_str());
+  // not need wait response
+  return;
+}
+
+void BaseConsumer::processConnect2Broker(ConsumerEvent& event) {
+  LOG_TRACE("[processConnect2Broker] begin to process connect event, clientid=%s",
+    client_uuid_.c_str());
+  if (!isClientRunning()) {
+    return;
+  }
+  bool ret_result;
+  int32_t error_code;
+  string error_info;
+  list<PartitionExt> subscribed_partitions;
+  list<PartitionExt> unsub_partitions;
+  list<PartitionExt>::iterator it;
+  list<SubscribeInfo> subscribe_info = event.GetSubscribeInfoList();
+  if (!subscribe_info.empty()) {
+    rmtdata_cache_.FilterPartitions(subscribe_info, subscribed_partitions, unsub_partitions);
+    if (!unsub_partitions.empty()) {
+      for (it = unsub_partitions.begin(); it != unsub_partitions.end(); it++) {
+        LOG_TRACE("[processConnect2Broker] connect to %s, clientid=%s",
+          it->GetPartitionKey().c_str(), client_uuid_.c_str());
+        auto request = std::make_shared<RequestContext>();
+        TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+        // build close2master request
+        buidRegisterRequestC2B(*it, req_protocol);
+        request->codec_ = std::make_shared<TubeMQCodec>();
+        request->ip_ = it->GetBrokerHost();
+        request->port_ = it->GetBrokerPort();
+        request->timeout_ = config_.GetRpcReadTimeoutMs();
+        request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+        req_protocol->request_id_ = request->request_id_;
+        req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+        // send message to target
+        ResponseContext response_context;
+        ErrorCode error = SyncRequest(response_context, request, req_protocol);
+        if (error.Value() != err_code::kErrSuccess) {
+          LOG_WARN("[Connect2Broker] request network failure to (%s:%d) : %s",
+                   it->GetBrokerHost().c_str(), it->GetBrokerPort(), error.Message().c_str());
+        } else {
+          // process response
+          auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+          ret_result = processRegResponseB2C(error_code, error_info, rsp);
+          if (ret_result) {
+            LOG_TRACE("[processConnect2Broker] add broker hb timer for %s, clientid=%s",
+              it->GetBrokerInfo().GetAddrInfo().c_str(), client_uuid_.c_str());
+            rmtdata_cache_.AddNewPartition(*it);
+            addBrokerHBTimer(it->GetBrokerInfo());
+          }
+        }
+      }
+    }
+  }
+  sub_info_.BookFstRegistered();
+  event.SetEventStatus(2);
+  LOG_TRACE("[processConnect2Broker] out connect event process, clientid=%s",
+    client_uuid_.c_str());
+}
+
+void BaseConsumer::processDisConnect2Broker(ConsumerEvent& event) {
+  LOG_TRACE("[processDisConnect2Broker] begin to process disConnect event, clientid=%s",
+    client_uuid_.c_str());
+  if (!isClientRunning()) {
+    return;
+  }
+  list<SubscribeInfo> subscribe_info = event.GetSubscribeInfoList();
+  if (!subscribe_info.empty()) {
+    map<NodeInfo, list<PartitionExt> > rmv_partitions;
+    rmtdata_cache_.RemoveAndGetPartition(subscribe_info,
+      config_.IsRollbackIfConfirmTimeout(), rmv_partitions);
+    if (!rmv_partitions.empty()) {
+      LOG_TRACE("[processDisConnect2Broker] unregister 2 broker process, clientid=%s",
+        client_uuid_.c_str());
+      unregister2Brokers(rmv_partitions, true);
+    }
+  }
+  event.SetEventStatus(2);
+  LOG_TRACE("[processDisConnect2Broker] out disConnect event process, clientid=%s",
+    client_uuid_.c_str());
+  return;
+}
+
+void BaseConsumer::closeAllBrokers() {
+  map<NodeInfo, list<PartitionExt> > broker_parts;
+  LOG_INFO("[CONSUMER] closeAllBrokers begin, clientid=%s", client_uuid_.c_str());
+  rmtdata_cache_.GetAllClosedBrokerParts(broker_parts);
+  if (!broker_parts.empty()) {
+     unregister2Brokers(broker_parts, false);
+  }
+  LOG_INFO("[CONSUMER] closeAllBrokers end, clientid=%s", client_uuid_.c_str());
+}
+
+
+void BaseConsumer::processHeartBeat2Broker(NodeInfo broker_info) {
+  LOG_TRACE("[Heartbeat2Broker] process hb to broker(%s) startted!",
+    broker_info.GetAddrInfo().c_str());
+  if (!isClientRunning()) {
+    return;
+  }
+  list<PartitionExt> partition_list;
+  list<PartitionExt>::iterator it;
+  rmtdata_cache_.GetPartitionByBroker(broker_info, partition_list);
+  if (partition_list.empty()) {
+    reSetBrokerHBTimer(broker_info);
+    LOG_TRACE("[Heartbeat2Broker] no alive partitions, for broker(%s), out!",
+      broker_info.GetAddrInfo().c_str());
+    return;
+  }
+
+  set<string> req_part_keys;
+  for (it = partition_list.begin(); it != partition_list.end(); ++it) {
+    req_part_keys.insert(it->GetPartitionKey());
+  }
+  auto request = std::make_shared<RequestContext>();
+  TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+  // build heartbeat2broker request
+  buidHeartBeatC2B(partition_list, req_protocol);
+  request->codec_ = std::make_shared<TubeMQCodec>();
+  request->ip_ = broker_info.GetHost();
+  request->port_ = broker_info.GetPort();
+  request->timeout_ = config_.GetRpcReadTimeoutMs();
+  request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+  req_protocol->request_id_ = request->request_id_;
+  req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+
+  LOG_TRACE("[Heartbeat2Broker] send hb request to (%s)!", broker_info.GetAddrInfo().c_str());
+
+  // send message to target
+  AsyncRequest(request, req_protocol)
+      .AddCallBack([=](ErrorCode error, const ResponseContext& response_context) {
+        if (error.Value() != err_code::kErrSuccess) {
+          LOG_WARN("[Heartbeat2Broker] request network  to failure (%s), ression is %s",
+                   broker_info.GetAddrInfo().c_str(), error.Message().c_str());
+        } else {
+          // process response
+          auto rsp = any_cast<TubeMQCodec::RspProtocolPtr>(response_context.rsp_);
+          LOG_TRACE("[Heartbeat2Broker] receive hb response!");
+          if (rsp->success_) {
+            HeartBeatResponseB2C rsp_b2c;
+            bool result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
+                                                 (int)(rsp->rsp_body_.data().length()));
+            if (result) {
+              set<string> partition_keys;
+              if (rsp_b2c.success()) {
+                if (rsp_b2c.has_haspartfailure() && rsp_b2c.haspartfailure()) {
+                  for (int tmp_i = 0; tmp_i < rsp_b2c.failureinfo_size(); tmp_i++) {
+                    string token_key = delimiter::kDelimiterColon;
+                    string fullpart_str = rsp_b2c.failureinfo(tmp_i);
+                    string::size_type pos1 = fullpart_str.find(token_key);
+                    if (pos1 == string::npos) {
+                      continue;
+                    }
+                    // int error_code = atoi(fullpart_str.substr(0, pos1).c_str());
+                    string part_str =
+                        fullpart_str.substr(pos1 + token_key.size(), fullpart_str.size());
+                    Partition part(part_str);
+                    partition_keys.insert(part.GetPartitionKey());
+                    LOG_TRACE("[Heartbeat2Broker] found partiton(%s) hb failure!",
+                      part.GetPartitionKey().c_str());
+                  }
+                }
+                rmtdata_cache_.RemovePartition(partition_keys);
+              } else {
+                if (rsp_b2c.errcode() == err_code::kErrCertificateFailure) {
+                  rmtdata_cache_.RemovePartition(req_part_keys);
+                  LOG_WARN("[Heartbeat2Broker] request (%s) CertificateFailure",
+                           broker_info.GetAddrInfo().c_str());
+                }
+              }
+            }
+          }
+        }
+        LOG_TRACE("[Heartbeat2Broker] out hb response process, add broker(%s) timer!",
+          broker_info.GetAddrInfo().c_str());
+        reSetBrokerHBTimer(broker_info);
+      });
+}
+
+void BaseConsumer::buidRegisterRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string reg_msg;
+  RegisterRequestC2M c2m_request;
+  list<string>::iterator it_topics;
+  list<SubscribeInfo>::iterator it_sub;
+  c2m_request.set_clientid(client_uuid_);
+  c2m_request.set_hostname(TubeMQService::Instance()->GetLocalHost());
+  c2m_request.set_requirebound(sub_info_.IsBoundConsume());
+  c2m_request.set_groupname(config_.GetGroupName());
+  c2m_request.set_sessiontime(sub_info_.GetSubscribedTime());
+  // subscribed topic list
+  list<string> sub_topics = sub_info_.GetSubTopics();
+  for (it_topics = sub_topics.begin(); it_topics != sub_topics.end(); ++it_topics) {
+    c2m_request.add_topiclist(*it_topics);
+  }
+  c2m_request.set_defflowcheckid(rmtdata_cache_.GetDefFlowCtrlId());
+  c2m_request.set_groupflowcheckid(rmtdata_cache_.GetGroupFlowCtrlId());
+  c2m_request.set_qrypriorityid(rmtdata_cache_.GetGroupQryPriorityId());
+  // reported subscribed info
+  list<SubscribeInfo> subscribe_lst;
+  rmtdata_cache_.GetSubscribedInfo(subscribe_lst);
+  for (it_sub = subscribe_lst.begin(); it_sub != subscribe_lst.end(); ++it_sub) {
+    c2m_request.add_subscribeinfo(it_sub->ToString());
+  }
+  // get topic conditions
+  list<string> topic_conds = sub_info_.GetTopicConds();
+  for (it_topics = topic_conds.begin(); it_topics != topic_conds.end(); ++it_topics) {
+    c2m_request.add_topiccondition(*it_topics);
+  }
+  // add bound consume info
+  if (sub_info_.IsBoundConsume()) {
+    c2m_request.set_sessionkey(sub_info_.GetSessionKey());
+    c2m_request.set_selectbig(sub_info_.SelectBig());
+    c2m_request.set_totalcount(sub_info_.GetSourceCnt());
+    c2m_request.set_requiredpartition(sub_info_.GetBoundPartInfo());
+    c2m_request.set_notallocated(sub_info_.IsNotAllocated());
+  }
+  // authenticate info
+  if (needGenMasterCertificateInfo(true)) {
+    MasterCertificateInfo* pmst_certinfo = c2m_request.mutable_authinfo();
+    AuthenticateInfo* pauthinfo = pmst_certinfo->mutable_authinfo();
+    genMasterAuthenticateToken(pauthinfo, config_.GetUsrName(), config_.GetUsrPassWord());
+  }
+  //
+  c2m_request.SerializeToString(&reg_msg);
+  req_protocol->method_id_ = rpc_config::kMasterMethoddConsumerRegister;
+  req_protocol->prot_msg_ = reg_msg;
+}
+
+void BaseConsumer::buidHeartRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string hb_msg;
+  HeartRequestC2M c2m_request;
+  list<string>::iterator it_topics;
+  list<SubscribeInfo>::iterator it_sub;
+  c2m_request.set_clientid(client_uuid_);
+  c2m_request.set_groupname(config_.GetGroupName());
+  c2m_request.set_defflowcheckid(rmtdata_cache_.GetDefFlowCtrlId());
+  c2m_request.set_groupflowcheckid(rmtdata_cache_.GetGroupFlowCtrlId());
+  c2m_request.set_qrypriorityid(rmtdata_cache_.GetGroupQryPriorityId());
+  c2m_request.set_reportsubscribeinfo(false);
+  ConsumerEvent event;
+  list<SubscribeInfo>::iterator it;
+  list<SubscribeInfo> subscribe_info_lst;
+  bool has_event = rmtdata_cache_.PollEventResult(event);
+  // judge if report subscribe info
+  if ((has_event) || (++unreport_times_ > config_.GetMaxSubinfoReportIntvl())) {
+    unreport_times_ = 0;
+    c2m_request.set_reportsubscribeinfo(true);
+    rmtdata_cache_.GetSubscribedInfo(subscribe_info_lst);
+    if (has_event) {
+      EventProto* event_proto = c2m_request.mutable_event();
+      event_proto->set_rebalanceid(event.GetRebalanceId());
+      event_proto->set_optype(event.GetEventType());
+      event_proto->set_status(event.GetEventStatus());
+      list<SubscribeInfo> event_sub = event.GetSubscribeInfoList();
+      for (it = event_sub.begin(); it != event_sub.end(); it++) {
+        event_proto->add_subscribeinfo(it->ToString());
+      }
+    }
+    if (!subscribe_info_lst.empty()) {
+      for (it = subscribe_info_lst.begin(); it != subscribe_info_lst.end(); it++) {
+        c2m_request.add_subscribeinfo(it->ToString());
+      }
+    }
+  }
+  if (needGenMasterCertificateInfo(true)) {
+    MasterCertificateInfo* pmst_certinfo = c2m_request.mutable_authinfo();
+    AuthenticateInfo* pauthinfo = pmst_certinfo->mutable_authinfo();
+    genMasterAuthenticateToken(pauthinfo, config_.GetUsrName(), config_.GetUsrPassWord());
+  }
+  c2m_request.SerializeToString(&hb_msg);
+  req_protocol->method_id_ = rpc_config::kMasterMethoddConsumerHeatbeat;
+  req_protocol->prot_msg_ = hb_msg;
+}
+
+void BaseConsumer::buidCloseRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string close_msg;
+  CloseRequestC2M c2m_request;
+  c2m_request.set_clientid(client_uuid_);
+  c2m_request.set_groupname(config_.GetGroupName());
+  if (needGenMasterCertificateInfo(true)) {
+    MasterCertificateInfo* pmst_certinfo = c2m_request.mutable_authinfo();
+    AuthenticateInfo* pauthinfo = pmst_certinfo->mutable_authinfo();
+    genMasterAuthenticateToken(pauthinfo, config_.GetUsrName(), config_.GetUsrPassWord());
+  }
+  c2m_request.SerializeToString(&close_msg);
+  req_protocol->method_id_ = rpc_config::kMasterMethoddConsumerClose;
+  req_protocol->prot_msg_ = close_msg;
+}
+
+void BaseConsumer::buidRegisterRequestC2B(const PartitionExt& partition,
+                                            TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string register_msg;
+  set<string> filter_cond_set;
+  map<string, set<string> > filter_map;
+  RegisterRequestC2B c2b_request;
+  c2b_request.set_clientid(client_uuid_);
+  c2b_request.set_groupname(config_.GetGroupName());
+  c2b_request.set_optype(rpc_config::kRegOpTypeRegister);
+  c2b_request.set_topicname(partition.GetTopic());
+  c2b_request.set_partitionid(partition.GetPartitionId());
+  c2b_request.set_qrypriorityid(rmtdata_cache_.GetGroupQryPriorityId());
+  bool is_first_reg = rmtdata_cache_.IsPartitionFirstReg(partition.GetPartitionKey());
+  c2b_request.set_readstatus(getConsumeReadStatus(is_first_reg));
+  if (sub_info_.IsFilterConsume(partition.GetTopic())) {
+    filter_map = sub_info_.GetTopicFilterMap();
+    if (filter_map.find(partition.GetTopic()) != filter_map.end()) {
+      filter_cond_set = filter_map[partition.GetTopic()];
+      for (set<string>::iterator it_cond = filter_cond_set.begin();
+           it_cond != filter_cond_set.end(); it_cond++) {
+        c2b_request.add_filtercondstr(*it_cond);
+      }
+    }
+  }
+  int64_t part_offset = tb_config::kInvalidValue;
+  sub_info_.GetAssignedPartOffset(partition.GetPartitionKey(), part_offset);
+  if (part_offset != tb_config::kInvalidValue) {
+    c2b_request.set_curroffset(part_offset);
+  }
+  AuthorizedInfo* p_authInfo = c2b_request.mutable_authinfo();
+  genBrokerAuthenticInfo(p_authInfo, true);
+  c2b_request.SerializeToString(&register_msg);
+  req_protocol->method_id_ = rpc_config::kBrokerMethoddConsumerRegister;
+  req_protocol->prot_msg_ = register_msg;
+}
+
+void BaseConsumer::buidUnRegRequestC2B(const PartitionExt& partition,
+                                          TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string unreg_msg;
+  RegisterRequestC2B c2b_request;
+  c2b_request.set_clientid(client_uuid_);
+  c2b_request.set_groupname(config_.GetGroupName());
+  c2b_request.set_optype(rpc_config::kRegOpTypeUnReg);
+  c2b_request.set_topicname(partition.GetTopic());
+  c2b_request.set_partitionid(partition.GetPartitionId());
+  c2b_request.set_readstatus(1);
+  AuthorizedInfo* p_authInfo = c2b_request.mutable_authinfo();
+  genBrokerAuthenticInfo(p_authInfo, true);
+  c2b_request.SerializeToString(&unreg_msg);
+  req_protocol->method_id_ = rpc_config::kBrokerMethoddConsumerRegister;
+  req_protocol->prot_msg_ = unreg_msg;
+}
+
+void BaseConsumer::buidHeartBeatC2B(const list<PartitionExt>& partitions,
+                                      TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string hb_msg;
+  HeartBeatRequestC2B c2b_request;
+  list<PartitionExt>::const_iterator it_part;
+  c2b_request.set_clientid(client_uuid_);
+  c2b_request.set_groupname(config_.GetGroupName());
+  c2b_request.set_readstatus(getConsumeReadStatus(false));
+  c2b_request.set_qrypriorityid(rmtdata_cache_.GetGroupQryPriorityId());
+  for (it_part = partitions.begin(); it_part != partitions.end(); ++it_part) {
+    c2b_request.add_partitioninfo(it_part->ToString());
+  }
+  AuthorizedInfo* p_authInfo = c2b_request.mutable_authinfo();
+  genBrokerAuthenticInfo(p_authInfo, true);
+  c2b_request.SerializeToString(&hb_msg);
+  req_protocol->method_id_ = rpc_config::kBrokerMethoddConsumerHeatbeat;
+  req_protocol->prot_msg_ = hb_msg;
+}
+
+void BaseConsumer::buidGetMessageC2B(const PartitionExt& partition,
+                                       TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string get_msg;
+  GetMessageRequestC2B c2b_request;
+  c2b_request.set_clientid(client_uuid_);
+  c2b_request.set_groupname(config_.GetGroupName());
+  c2b_request.set_topicname(partition.GetTopic());
+  c2b_request.set_escflowctrl(rmtdata_cache_.IsUnderGroupCtrl());
+  c2b_request.set_partitionid(partition.GetPartitionId());
+  c2b_request.set_lastpackconsumed(partition.IsLastConsumed());
+  c2b_request.set_manualcommitoffset(false);
+  c2b_request.SerializeToString(&get_msg);
+  req_protocol->method_id_ = rpc_config::kBrokerMethoddConsumerGetMsg;
+  req_protocol->prot_msg_ = get_msg;
+}
+
+void BaseConsumer::buidCommitC2B(const PartitionExt& partition, bool is_last_consumed,
+                                   TubeMQCodec::ReqProtocolPtr& req_protocol) {
+  string commit_msg;
+  CommitOffsetRequestC2B c2b_request;
+  c2b_request.set_clientid(client_uuid_);
+  c2b_request.set_groupname(config_.GetGroupName());
+  c2b_request.set_topicname(partition.GetTopic());
+  c2b_request.set_partitionid(partition.GetPartitionId());
+  c2b_request.set_lastpackconsumed(is_last_consumed);
+  c2b_request.SerializeToString(&commit_msg);
+  req_protocol->method_id_ = rpc_config::kBrokerMethoddConsumerCommit;
+  req_protocol->prot_msg_ = commit_msg;
+}
+
+bool BaseConsumer::processRegisterResponseM2C(int32_t& error_code, string& err_info,
+                                                const TubeMQCodec::RspProtocolPtr& rsp_protocol) {
+  LOG_TRACE("processRegisterResponseM2C, process message begin");
+  if (!rsp_protocol->success_) {
+    error_code = rsp_protocol->code_;
+    err_info = rsp_protocol->error_msg_;
+    LOG_TRACE("processRegisterResponseM2C, rsp_protocol->success_ not true, out");
+    return false;
+  }
+  RegisterResponseM2C rsp_m2c;
+  bool result = rsp_m2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
+                                       (int)(rsp_protocol->rsp_body_.data().length()));
+  if (!result) {
+    error_code = err_code::kErrParseFailure;
+    err_info = "Parse RegisterResponseM2C response failure!";
+
+    LOG_TRACE("processRegisterResponseM2C come, parse message failure!");
+    return false;
+  }
+  if (!rsp_m2c.success()) {
+    error_code = rsp_m2c.errcode();
+    err_info = rsp_m2c.errmsg();
+
+    LOG_TRACE("processRegisterResponseM2C come, return failure, errorcode = %d!", error_code);
+    return false;
+  }
+  // update policy
+  if (rsp_m2c.has_notallocated() && !rsp_m2c.notallocated()) {
+    sub_info_.CompAndSetNotAllocated(true, false);
+  }
+  if (rsp_m2c.has_defflowcheckid() || rsp_m2c.has_groupflowcheckid()) {
+    if (rsp_m2c.has_defflowcheckid()) {
+      rmtdata_cache_.UpdateDefFlowCtrlInfo(rsp_m2c.defflowcheckid(), rsp_m2c.defflowcontrolinfo());
+    }
+    int qryPriorityId = rsp_m2c.has_qrypriorityid() ? rsp_m2c.qrypriorityid()
+                                                    : rmtdata_cache_.GetGroupQryPriorityId();
+    rmtdata_cache_.UpdateGroupFlowCtrlInfo(qryPriorityId, rsp_m2c.groupflowcheckid(),
+                                           rsp_m2c.groupflowcontrolinfo());
+  }
+  if (rsp_m2c.has_authorizedinfo()) {
+    processAuthorizedToken(rsp_m2c.authorizedinfo());
+  }
+  error_code = err_code::kErrSuccess;
+  err_info = "Ok";
+  LOG_TRACE("processRegisterResponseM2C, process finished, out");
+
+  return true;
+}
+
+bool BaseConsumer::processHBResponseM2C(int32_t& error_code, string& err_info,
+                                          const TubeMQCodec::RspProtocolPtr& rsp_protocol) {
+  LOG_TRACE("processHBResponseM2C, process message begin");
+  if (!rsp_protocol->success_) {
+    error_code = rsp_protocol->code_;
+    err_info = rsp_protocol->error_msg_;
+    LOG_TRACE("processHBResponseM2C success_ is false, errcode=%d, errinfo=%s",
+      error_code, err_info.c_str());
+    return false;
+  }
+  HeartResponseM2C rsp_m2c;
+  bool result = rsp_m2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
+                                       (int32_t)(rsp_protocol->rsp_body_.data().length()));
+  if (!result) {
+    error_code = err_code::kErrParseFailure;
+    err_info = "Parse HeartResponseM2C response failure!";
+
+    LOG_TRACE("processHBResponseM2C Parse result failure, out");
+    return false;
+  }
+  if (!rsp_m2c.success()) {
+    error_code = rsp_m2c.errcode();
+    err_info = rsp_m2c.errmsg();
+    LOG_TRACE("processHBResponseM2C response is false, errorInfo=%s, out", err_info.c_str());
+    return false;
+  }
+  // update policy
+  if (rsp_m2c.has_notallocated() && !rsp_m2c.notallocated()) {
+    sub_info_.CompAndSetNotAllocated(true, false);
+  }
+  if (rsp_m2c.has_defflowcheckid() || rsp_m2c.has_groupflowcheckid()) {
+    if (rsp_m2c.has_defflowcheckid()) {
+      rmtdata_cache_.UpdateDefFlowCtrlInfo(rsp_m2c.defflowcheckid(), rsp_m2c.defflowcontrolinfo());
+    }
+    int qryPriorityId = rsp_m2c.has_qrypriorityid() ? rsp_m2c.qrypriorityid()
+                                                    : rmtdata_cache_.GetGroupQryPriorityId();
+    rmtdata_cache_.UpdateGroupFlowCtrlInfo(qryPriorityId, rsp_m2c.groupflowcheckid(),
+                                           rsp_m2c.groupflowcontrolinfo());
+  }
+  if (rsp_m2c.has_authorizedinfo()) {
+    processAuthorizedToken(rsp_m2c.authorizedinfo());
+  }
+  if (rsp_m2c.has_requireauth()) {
+    nextauth_2_master.Set(rsp_m2c.requireauth());
+  }
+  // Get the latest rebalance task
+  if (rsp_m2c.has_event()) {
+    EventProto eventProto = rsp_m2c.event();
+    if (eventProto.rebalanceid() > 0) {
+      list<SubscribeInfo> subcribe_infos;
+      for (int i = 0; i < eventProto.subscribeinfo_size(); i++) {
+        SubscribeInfo sub_info(eventProto.subscribeinfo(i));
+        subcribe_infos.push_back(sub_info);
+      }
+      ConsumerEvent new_event(eventProto.rebalanceid(), eventProto.optype(), subcribe_infos, 0);
+      rmtdata_cache_.OfferEvent(new_event);
+    }
+  }
+  last_master_hbtime_ = Utils::GetCurrentTimeMillis();
+  error_code = err_code::kErrSuccess;
+  err_info = "Ok";
+  LOG_TRACE("processHBResponseM2C, process finished, out");
+
+  return true;
+}
+
+void BaseConsumer::unregister2Brokers(map<NodeInfo, list<PartitionExt> >& unreg_partitions,
+                                        bool wait_rsp) {
+  string err_info;
+  map<NodeInfo, list<PartitionExt> >::iterator it;
+  list<PartitionExt>::iterator it_part;
+
+  LOG_TRACE("unregister2Brokers, process begin");
+
+  if (unreg_partitions.empty()) {
+    LOG_TRACE("unregister2Brokers, unreg partitions empty, out");
+    return;
+  }
+  for (it = unreg_partitions.begin(); it != unreg_partitions.end(); ++it) {
+    list<PartitionExt> part_list = it->second;
+    for (it_part = part_list.begin(); it_part != part_list.end(); ++it_part) {
+      LOG_TRACE("unregister2Brokers, partitionkey=%s",
+        it_part->GetPartitionKey().c_str());
+      auto request = std::make_shared<RequestContext>();
+      TubeMQCodec::ReqProtocolPtr req_protocol = TubeMQCodec::GetReqProtocol();
+      // build unregister 2 broker request
+      buidUnRegRequestC2B(*it_part, req_protocol);
+      request->codec_ = std::make_shared<TubeMQCodec>();
+      request->ip_ = it_part->GetBrokerHost();
+      request->port_ = it_part->GetBrokerPort();
+      request->timeout_ = config_.GetRpcReadTimeoutMs();
+      request->request_id_ = Singleton<UniqueSeqId>::Instance().Next();
+      req_protocol->request_id_ = request->request_id_;
+      req_protocol->rpc_read_timeout_ = config_.GetRpcReadTimeoutMs() - 500;
+      // send message to target
+      // not need process
+      ResponseContext response_context;
+      if (wait_rsp) {
+        SyncRequest(response_context, request, req_protocol);
+      } else {
+        AsyncRequest(request, req_protocol);
+      }
+      LOG_TRACE("unregister2Brokers, partitionkey=%s return come",
+        it_part->GetPartitionKey().c_str());
+    }
+  }
+}
+
+bool BaseConsumer::processRegResponseB2C(int32_t& error_code, string& err_info,
+                                           const TubeMQCodec::RspProtocolPtr& rsp_protocol) {
+  LOG_TRACE("processRegResponseB2C, process begin");
+  if (!rsp_protocol->success_) {
+    error_code = rsp_protocol->code_;
+    err_info = rsp_protocol->error_msg_;
+    LOG_TRACE("processRegResponseB2C, not success, out");
+    return false;
+  }
+  RegisterResponseB2C rsp_b2c;
+  bool result = rsp_b2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
+                                       (int)(rsp_protocol->rsp_body_.data().length()));
+  if (!result) {
+    error_code = err_code::kErrParseFailure;
+    err_info = "Parse RegisterResponseB2C response failure!";
+    LOG_TRACE("processRegResponseB2C, parse body failure, out");
+    return false;
+  }
+  if (!rsp_b2c.success()) {
+    error_code = rsp_b2c.errcode();
+    err_info = rsp_b2c.errmsg();
+    LOG_TRACE("processRegResponseB2C, return failure, error is %s, out", err_info.c_str());
+    return false;
+  }
+  error_code = err_code::kErrSuccess;
+  err_info = "Ok";
+  LOG_TRACE("processRegResponseB2C, success, finished");
+  return true;
+}
+
+void BaseConsumer::convertMessages(int32_t& msg_size, list<Message>& message_list,
+                                     bool filter_consume, const string& topic_name,
+                                     GetMessageResponseB2C& rsp_b2c) {
+  // #lizard forgives
+  msg_size = 0;
+  message_list.clear();
+  if (rsp_b2c.messages_size() == 0) {
+    return;
+  }
+  for (int i = 0; i < rsp_b2c.messages_size(); i++) {
+    TransferedMessage tsfMsg = rsp_b2c.messages(i);
+    int32_t flag = tsfMsg.flag();
+    int64_t message_id = tsfMsg.messageid();
+    int32_t in_check_sum = tsfMsg.checksum();
+    int32_t payload_length = tsfMsg.payloaddata().length();
+    int32_t calc_checksum = Utils::Crc32(tsfMsg.payloaddata());
+    if (in_check_sum != calc_checksum) {
+
+      LOG_TRACE("[CONSUMER] convertMessages [%d], Crc32 failure, in=%d, calc=%d, client=%s",
+        i, in_check_sum, calc_checksum, client_uuid_.c_str());
+
+      continue;
+    }
+    int read_pos = 0;
+    int data_len = payload_length;
+    map<string, string> properties;
+    std::unique_ptr<char[]> payload_data(new char[payload_length]);
+    memcpy(&payload_data[0], tsfMsg.payloaddata().c_str(), payload_length);
+    if ((flag & tb_config::kMsgFlagIncProperties) == 1) {
+      if (payload_length < 4) {
+
+        LOG_TRACE("[CONSUMER] convertMessages [%d], payload_length(%d) < 4, client=%s",
+          i, payload_length, client_uuid_.c_str());
+
+        continue;
+      }
+      int32_t attr_len = ntohl(*(int*)(&payload_data[0]));
+      read_pos += 4;
+      data_len -= 4;
+      if (attr_len > data_len) {
+        LOG_TRACE("[CONSUMER] convertMessages [%d], attr_len(%d) > data_len(%d), client=%s",
+          i, attr_len, data_len, client_uuid_.c_str());
+        continue;
+      }
+      string attribute(&payload_data[0] + read_pos, attr_len);
+      read_pos += attr_len;
+      data_len -= attr_len;
+      Utils::Split(attribute, properties, delimiter::kDelimiterComma, delimiter::kDelimiterEqual);
+      if (filter_consume) {
+        map<string, set<string> > topic_filter_map = sub_info_.GetTopicFilterMap();
+        map<string, set<string> >::const_iterator it = topic_filter_map.find(topic_name);
+        if (properties.find(tb_config::kRsvPropKeyFilterItem) != properties.end()) {
+          string msg_key = properties[tb_config::kRsvPropKeyFilterItem];
+          if (it != topic_filter_map.end()) {
+            set<string> filters = it->second;
+            if (filters.find(msg_key) == filters.end()) {
+              LOG_TRACE("[CONSUMER] convertMessages [%d], filter consume, not matched, client=%s",
+                i, client_uuid_.c_str());
+              continue;
+            }
+          }
+        }
+      }
+    }
+    Message message(topic_name, flag, message_id, &payload_data[0] + read_pos, data_len,
+                    properties);
+    message_list.push_back(message);
+    msg_size += data_len;
+  }
+
+  LOG_TRACE("[CONSUMER] convertMessages finished, count=%ld, client=%s",
+    message_list.size(), client_uuid_.c_str());
+
+  return;
+}
+
+bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& peer_info,
+                                             bool filter_consume, const PartitionExt& partition_ext,
+                                             const string& confirm_context,
+                                             const TubeMQCodec::RspProtocolPtr& rsp) {
+  string err_info;
+
+  if (!rsp->success_) {
+    rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false);
+    result.SetFailureResult(rsp->code_, rsp->error_msg_, partition_ext.GetTopic(), peer_info);
+
+    LOG_TRACE("[CONSUMER] processGetMessageRspB2C failure, code_=%d, error_msg_=%s, client=%s",
+      rsp->code_, rsp->error_msg_.c_str(), client_uuid_.c_str());
+
+    return false;
+  }
+  GetMessageResponseB2C rsp_b2c;
+  bool ret_result =
+      rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(), (int)(rsp->rsp_body_.data().length()));
+  if (!ret_result) {
+    rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false);
+    result.SetFailureResult(err_code::kErrServerError,
+                            "Parse GetMessageResponseB2C response failure!",
+                            partition_ext.GetTopic(), peer_info);
+
+    LOG_TRACE("[CONSUMER] processGetMessageRspB2C parse failure, client=%s", client_uuid_.c_str());
+
+    return false;
+  }
+
+  switch (rsp_b2c.errcode()) {
+    case err_code::kErrSuccess: {
+      bool esc_limit = (rsp_b2c.has_escflowctrl() && rsp_b2c.escflowctrl());
+      long data_dltval =
+          rsp_b2c.has_currdatadlt() ? rsp_b2c.currdatadlt() : tb_config::kInvalidValue;
+      long curr_offset = rsp_b2c.has_curroffset() ? rsp_b2c.curroffset() : tb_config::kInvalidValue;
+      bool req_slow = rsp_b2c.has_requireslow() ? rsp_b2c.requireslow() : false;
+      int msg_size = 0;
+      list<Message> message_list;
+      convertMessages(msg_size, message_list, filter_consume, partition_ext.GetTopic(), rsp_b2c);
+      rmtdata_cache_.BookedPartionInfo(partition_ext.GetPartitionKey(), curr_offset,
+                                       err_code::kErrSuccess, esc_limit, msg_size, 0, data_dltval,
+                                       req_slow);
+      peer_info.SetCurrOffset(curr_offset);
+      result.SetSuccessResult(err_code::kErrSuccess, partition_ext.GetTopic(), peer_info,
+                              confirm_context, message_list);
+      return true;
+    }
+
+    case err_code::kErrHbNoNode:
+    case err_code::kErrCertificateFailure:
+    case err_code::kErrDuplicatePartition: {
+      rmtdata_cache_.RemovePartition(err_info, confirm_context);
+      result.SetFailureResult(rsp_b2c.errcode(), rsp_b2c.errmsg(), partition_ext.GetTopic(),
+                              peer_info);
+      return false;
+    }
+
+    case err_code::kErrConsumeSpeedLimit: {
+      // Process with server side speed limit
+      long def_dlttime = rsp_b2c.has_minlimittime() ? rsp_b2c.minlimittime()
+                                                    : config_.GetMsgNotFoundWaitPeriodMs();
+      rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false,
+                                  tb_config::kInvalidValue, rsp_b2c.errcode(), false, 0,
+                                  def_dlttime, tb_config::kInvalidValue);
+      result.SetFailureResult(rsp_b2c.errcode(), rsp_b2c.errmsg(), partition_ext.GetTopic(),
+                              peer_info);
+      return false;
+    }
+
+    case err_code::kErrNotFound:
+    case err_code::kErrForbidden:
+    case err_code::kErrMoved:
+    case err_code::kErrServiceUnavilable:
+    default: {
+      // Slow down the request based on the limitation configuration when meet these errors
+      long limit_dlt = 300;
+      switch (rsp_b2c.errcode()) {
+        case err_code::kErrForbidden: {
+          limit_dlt = 2000;
+          break;
+        }
+        case err_code::kErrServiceUnavilable: {
+          limit_dlt = 300;
+          break;
+        }
+        case err_code::kErrMoved: {
+          limit_dlt = 200;
+          break;
+        }
+        case err_code::kErrNotFound: {
+          limit_dlt = config_.GetMsgNotFoundWaitPeriodMs();
+          break;
+        }
+        default: {
+          //
+        }
+      }
+      rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false,
+                                  tb_config::kInvalidValue, rsp_b2c.errcode(), false, 0, limit_dlt,
+                                  tb_config::kInvalidValue);
+      result.SetFailureResult(rsp_b2c.errcode(), rsp_b2c.errmsg(), partition_ext.GetTopic(),
+                              peer_info);
+      return false;
+    }
+  }
+  return true;
+}
+
+bool BaseConsumer::isClientRunning() { return (status_.Get() == 2); }
+
+string BaseConsumer::buildUUID() {
+  stringstream ss;
+  ss << config_.GetGroupName();
+  ss << "_";
+  ss << TubeMQService::Instance()->GetLocalHost();
+  ss << "-";
+  ss << getpid();
+  ss << "-";
+  ss << Utils::GetCurrentTimeMillis();
+  ss << "-";
+  ss << GetClientIndex();
+  ss << "-";
+  ss << kTubeMQClientVersion;
+  return ss.str();
+}
+
+int32_t BaseConsumer::getConsumeReadStatus(bool is_first_reg) {
+  int32_t readStatus = rpc_config::kConsumeStatusNormal;
+  if (is_first_reg) {
+    if (config_.GetConsumePosition() == 0) {
+      readStatus = rpc_config::kConsumeStatusFromMax;
+      LOG_INFO("[Consumer From Max Offset], clientId=%s", client_uuid_.c_str());
+    } else if (config_.GetConsumePosition() > 0) {
+      readStatus = rpc_config::kConsumeStatusFromMaxAlways;
+      LOG_INFO("[Consumer From Max Offset Always], clientId=%s", client_uuid_.c_str());
+    }
+  }
+  LOG_INFO("[getConsumeReadStatus], readStatus=%d, is_first_reg=%d, config_.GetConsumePosition()=%d",
+    readStatus, is_first_reg, config_.GetConsumePosition());
+  return readStatus;
+}
+
+bool BaseConsumer::initMasterAddress(string& err_info, const string& master_info) {
+  masters_map_.clear();
+  Utils::Split(master_info, masters_map_, delimiter::kDelimiterComma, delimiter::kDelimiterColon);
+  if (masters_map_.empty()) {
+    err_info = "Illegal parameter: master_info is blank!";
+    return false;
+  }
+  bool needXfs = false;
+  map<string, int32_t>::iterator it;
+  for (it = masters_map_.begin(); it != masters_map_.end(); it++) {
+    if (Utils::NeedDnsXfs(it->first)) {
+      needXfs = true;
+      break;
+    }
+  }
+  it = masters_map_.begin();
+  curr_master_addr_ = it->first;
+  if (needXfs) {
+    TubeMQService::Instance()->AddMasterAddress(err_info, master_info);
+  }
+  err_info = "Ok";
+  return true;
+}
+
+void BaseConsumer::getNextMasterAddr(string& ipaddr, int32_t& port) {
+  map<string, int32_t>::iterator it;
+  it = masters_map_.find(curr_master_addr_);
+  if (it != masters_map_.end()) {
+    it++;
+    if (it == masters_map_.end()) {
+      it = masters_map_.begin();
+    }
+  } else {
+    it = masters_map_.begin();
+  }
+  ipaddr = it->first;
+  port = it->second;
+  curr_master_addr_ = it->first;
+  if (Utils::NeedDnsXfs(ipaddr)) {
+    TubeMQService::Instance()->GetXfsMasterAddress(curr_master_addr_, ipaddr);
+  }
+  LOG_TRACE("getNextMasterAddr address is %s:%d", ipaddr.c_str(), port);
+}
+
+void BaseConsumer::getCurrentMasterAddr(string& ipaddr, int32_t& port) {
+  ipaddr = curr_master_addr_;
+  port = masters_map_[curr_master_addr_];
+  if (Utils::NeedDnsXfs(ipaddr)) {
+    TubeMQService::Instance()->GetXfsMasterAddress(curr_master_addr_, ipaddr);
+  }
+  LOG_TRACE("getCurrentMasterAddr address is %s:%d", ipaddr.c_str(), port);
+}
+
+bool BaseConsumer::needGenMasterCertificateInfo(bool force) {
+  bool needAdd = false;
+  if (config_.IsAuthenticEnabled()) {
+    if (force) {
+      needAdd = true;
+      nextauth_2_master.Set(false);
+    } else if (nextauth_2_master.Get()) {
+      if (nextauth_2_master.CompareAndSet(true, false)) {
+        needAdd = true;
+      }
+    }
+  }
+  return needAdd;
+}
+
+void BaseConsumer::genBrokerAuthenticInfo(AuthorizedInfo* p_authInfo, bool force) {
+  bool needAdd = false;
+  p_authInfo->set_visitauthorizedtoken(visit_token_.Get());
+  if (config_.IsAuthenticEnabled()) {
+    if (force) {
+      needAdd = true;
+      nextauth_2_broker.Set(false);
+    } else if (nextauth_2_broker.Get()) {
+      if (nextauth_2_broker.CompareAndSet(true, false)) {
+        needAdd = true;
+      }
+    }
+    if (needAdd) {
+      string auth_token =
+          Utils::GenBrokerAuthenticateToken(config_.GetUsrName(), config_.GetUsrPassWord());
+      p_authInfo->set_authauthorizedtoken(auth_token);
+    }
+  }
+}
+
+void BaseConsumer::genMasterAuthenticateToken(AuthenticateInfo* pauthinfo, const string& username,
+                                                const string usrpassword) {
+  //
+}
+
+void BaseConsumer::processAuthorizedToken(const MasterAuthorizedInfo& authorized_token_info) {
+  visit_token_.Set(authorized_token_info.visitauthorizedtoken());
+  if (authorized_token_info.has_authauthorizedtoken()) {
+    lock_guard<mutex> lck(auth_lock_);
+
+    if (authorized_info_ != authorized_token_info.authauthorizedtoken()) {
+      authorized_info_ = authorized_token_info.authauthorizedtoken();
+    }
+  }
+}
+
+void BaseConsumer::addBrokerHBTimer(const NodeInfo broker) {
+  SteadyTimerPtr timer;
+  int32_t hb_periodms = config_.GetHeartbeatPeriodMs();
+  LOG_TRACE("[addBrokerHBTimer] add hb timer for broker(%s), in!",
+    broker.GetAddrInfo().c_str());
+  lock_guard<mutex> lck(broker_timer_lock_);
+  if (broker_timer_map_.find(broker) == broker_timer_map_.end()) {
+    LOG_TRACE("[addBrokerHBTimer] found no hb timer for broker(%s), add!",
+      broker.GetAddrInfo().c_str());
+    timer = TubeMQService::Instance()->CreateTimer();
+    broker_timer_map_[broker] = timer;
+    timer->expires_after(std::chrono::milliseconds(hb_periodms / 2));
+    timer->async_wait([this, broker](const std::error_code& ec) {
+      if (ec) {
+        return;
+      }
+      processHeartBeat2Broker(broker);
+    });
+  } else {
+    LOG_TRACE("[addBrokerHBTimer] found have hb timer for broker(%s), not add!",
+      broker.GetAddrInfo().c_str());
+  }
+}
+
+void BaseConsumer::reSetBrokerHBTimer(const NodeInfo broker) {
+  SteadyTimerPtr timer;
+  list<PartitionExt> partition_list;
+  int32_t hb_periodms = config_.GetHeartbeatPeriodMs();
+  LOG_TRACE("[reSetBrokerHBTimer] reset hb timer for broker(%s), in!",
+    broker.GetAddrInfo().c_str());
+  lock_guard<mutex> lck(broker_timer_lock_);
+  rmtdata_cache_.GetPartitionByBroker(broker, partition_list);
+  if (partition_list.empty()) {
+    broker_timer_map_.erase(broker);
+    LOG_TRACE("[reSetBrokerHBTimer] no alive partitions for broker(%s), clear timer!",
+      broker.GetAddrInfo().c_str());
+  } else {
+    if (broker_timer_map_.find(broker) != broker_timer_map_.end()) {
+      timer = broker_timer_map_[broker];
+      timer->expires_after(std::chrono::milliseconds(hb_periodms));
+      timer->async_wait([this, broker](const std::error_code& ec) {
+        if (ec) {
+          return;
+        }
+        processHeartBeat2Broker(broker);
+      });
+      LOG_TRACE("[reSetBrokerHBTimer] have alive partitions for broker(%s), reset timer!",
+        broker.GetAddrInfo().c_str());
+    }
+  }
+}
+}  // namespace tubemq
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.h b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.h
new file mode 100644
index 0000000..287532d
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.h
@@ -0,0 +1,146 @@
+/**
+ * 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 TUBEMQ_CLIENT_IMP_CONSUMER_API_H_
+#define TUBEMQ_CLIENT_IMP_CONSUMER_API_H_
+
+#include <stdlib.h>
+
+#include <list>
+#include <mutex>
+#include <string>
+
+#include "BrokerService.pb.h"
+#include "MasterService.pb.h"
+#include "RPC.pb.h"
+#include "client_service.h"
+#include "client_subinfo.h"
+#include "meta_info.h"
+#include "rmt_data_cache.h"
+#include "tubemq_codec.h"
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
+
+namespace tubemq {
+
+using std::mutex;
+using std::string;
+
+
+class BaseConsumer : public BaseClient, public std::enable_shared_from_this<BaseConsumer> {
+ public:
+  BaseConsumer();
+  ~BaseConsumer();
+  bool Start(string& err_info, const ConsumerConfig& config);
+  virtual void ShutDown();
+  bool GetMessage(ConsumerResult& result);
+  bool Confirm(const string& confirm_context, bool is_consumed, ConsumerResult& result);
+  bool GetCurConsumedInfo(map<string, ConsumeOffsetInfo>& consume_info_map);
+
+ private:
+  bool register2Master(int32_t& error_code, string& err_info, bool need_change);
+  void heartBeat2Master();
+  void processRebalanceEvent();
+  void close2Master();
+  void closeAllBrokers();
+
+ private:
+  string buildUUID();
+  bool isClientRunning();
+  bool IsConsumeReady(ConsumerResult& result);
+  int32_t getConsumeReadStatus(bool is_first_reg);
+  bool initMasterAddress(string& err_info, const string& master_info);
+  void getNextMasterAddr(string& ipaddr, int32_t& port);
+  void getCurrentMasterAddr(string& ipaddr, int32_t& port);
+  bool needGenMasterCertificateInfo(bool force);
+  void genBrokerAuthenticInfo(AuthorizedInfo* p_authInfo, bool force);
+  void processAuthorizedToken(const MasterAuthorizedInfo& authorized_token_info);
+  void addBrokerHBTimer(const NodeInfo broker);
+  void reSetBrokerHBTimer(const NodeInfo broker);
+
+ private:
+  void buidRegisterRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidHeartRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidCloseRequestC2M(TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidRegisterRequestC2B(const PartitionExt& partition,
+    TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidUnRegRequestC2B(const PartitionExt& partition,
+    TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidHeartBeatC2B(const list<PartitionExt>& partitions,
+    TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidGetMessageC2B(const PartitionExt& partition,
+    TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void buidCommitC2B(const PartitionExt& partition, bool is_last_consumed,
+    TubeMQCodec::ReqProtocolPtr& req_protocol);
+  void genMasterAuthenticateToken(AuthenticateInfo* pauthinfo,
+    const string& username, const string usrpassword);
+  bool processRegisterResponseM2C(int32_t& error_code, string& err_info,
+    const TubeMQCodec::RspProtocolPtr& rsp_protocol);
+  bool processHBResponseM2C(int32_t& error_code, string& err_info,
+    const TubeMQCodec::RspProtocolPtr& rsp_protocol);
+  void processDisConnect2Broker(ConsumerEvent& event);
+  void processConnect2Broker(ConsumerEvent& event);
+  void unregister2Brokers(map<NodeInfo, list<PartitionExt> >& unreg_partitions, bool wait_rsp);
+  bool processRegResponseB2C(int32_t& error_code, string& err_info,
+    const TubeMQCodec::RspProtocolPtr& rsp_protocol);
+  void processHeartBeat2Broker(NodeInfo broker_info);
+  bool processGetMessageRspB2C(ConsumerResult& result, PeerInfo& peer_info,
+    bool filter_consume, const PartitionExt& partition_ext, const string& confirm_context,
+    const TubeMQCodec::RspProtocolPtr& rsp_protocol);
+  void convertMessages(int32_t& msg_size, list<Message>& message_list,
+    bool filter_consume, const string& topic_name, GetMessageResponseB2C& rsp_b2c);
+  inline int32_t nextHeartBeatPeriodms();
+  void asyncRegister2Master(bool need_change);
+
+ private:
+  int32_t client_indexid_;
+  string client_uuid_;
+  AtomicInteger status_;
+  ConsumerConfig config_;
+  ClientSubInfo sub_info_;
+  RmtDataCacheCsm rmtdata_cache_;
+  AtomicLong visit_token_;
+  mutable mutex auth_lock_;
+  string authorized_info_;
+  AtomicBoolean nextauth_2_master;
+  AtomicBoolean nextauth_2_broker;
+  string curr_master_addr_;
+  map<string, int32_t> masters_map_;
+  AtomicBoolean is_master_actived_;
+  AtomicInteger master_reg_status_;
+  int32_t master_sh_retry_cnt_;
+  int64_t last_master_hbtime_;
+  int32_t unreport_times_;
+  // master heartbeat timer
+  SteadyTimerPtr heart_beat_timer_;
+  AtomicInteger master_hb_status_;
+  std::shared_ptr<std::thread> rebalance_thread_ptr_;
+  // broker heartbeat timer
+  mutable mutex broker_timer_lock_;
+  map<NodeInfo, SteadyTimerPtr> broker_timer_map_;
+};
+
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_IMP_CONSUMER_API_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
index 8e001a7..c6e0aaa 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
@@ -17,47 +17,60 @@
  * under the License.
  */
 
-#include "tubemq/client_service.h"
+#include "client_service.h"
 
 #include <sstream>
 
-#include "tubemq/const_config.h"
-#include "tubemq/logger.h"
-#include "tubemq/utils.h"
-
-
+#include "const_config.h"
+#include "logger.h"
+#include "utils.h"
 
 namespace tubemq {
 
 using std::lock_guard;
 using std::stringstream;
 
-
 BaseClient::BaseClient(bool is_producer) {
-  this->is_producer_ = is_producer;
+  is_producer_ = is_producer;
+  client_index_ = tb_config::kInvalidValue;
 }
 
 BaseClient::~BaseClient() {
   // no code
 }
 
-/*
-TubeMQService::TubeMQService() {
+TubeMQService* TubeMQService::_instance = NULL;
+
+static mutex tubemq_mutex_service_;
+
+TubeMQService* TubeMQService::Instance() {
+  if (NULL == _instance) {
+    lock_guard<mutex> lck(tubemq_mutex_service_);
+    if (NULL == _instance) {
+      _instance = new TubeMQService;
+    }
+  }
+  return _instance;
+}
+
+TubeMQService::TubeMQService()
+    : timer_executor_(std::make_shared<ExecutorPool>(2)),
+      network_executor_(std::make_shared<ExecutorPool>(4)) {
   service_status_.Set(0);
   client_index_base_.Set(0);
+  last_check_time_ = 0;
 }
 
 TubeMQService::~TubeMQService() {
   string err_info;
   Stop(err_info);
 }
-*/
 
 bool TubeMQService::Start(string& err_info, string conf_file) {
   // check configure file
   bool result = false;
   Fileini fileini;
-  string sector = "TubeMQ";  
+  string sector = "TubeMQ";
 
   result = Utils::ValidConfigFile(err_info, conf_file);
   if (!result) {
@@ -71,60 +84,92 @@ bool TubeMQService::Start(string& err_info, string conf_file) {
   if (!result) {
     return result;
   }
-  if (!service_status_.CompareAndSet(0,1)) {
+  if (!service_status_.CompareAndSet(0, 1)) {
     err_info = "TubeMQ Service has startted or Stopped!";
     return false;
   }
   iniLogger(fileini, sector);
+  iniPoolThreads(fileini, sector);
+  iniXfsThread(fileini, sector);
   service_status_.Set(2);
   err_info = "Ok!";
+  LOG_INFO("[TubeMQService] TubeMQ service startted!");
+
   return true;
 }
 
 bool TubeMQService::Stop(string& err_info) {
   if (service_status_.CompareAndSet(2, -1)) {
+    LOG_INFO("[TubeMQService] TubeMQ service begin to stop!");
+    if (dns_xfs_thread_.joinable()) {
+      dns_xfs_thread_.join();
+    }
     shutDownClinets();
-    timer_executor_.Close();
-    network_executor_.Close();
+    timer_executor_->Close();
+    network_executor_->Close();
+    connection_pool_ = nullptr;
+    thread_pool_ = nullptr;
+    LOG_INFO("[TubeMQService] TubeMQ service stopped!");
   }
   err_info = "OK!";
   return true;
 }
 
-bool TubeMQService::IsRunning() {
-  return (service_status_.Get() == 2);
-}
+bool TubeMQService::IsRunning() { return (service_status_.Get() == 2); }
 
 void TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
   string err_info;
   int32_t log_num = 10;
   int32_t log_size = 10;
   int32_t log_level = 4;
-  string log_path = "../log/";
+  string log_path = "../log/tubemq";
   fileini.GetValue(err_info, sector, "log_num", log_num, 10);
   fileini.GetValue(err_info, sector, "log_size", log_size, 100);
-  fileini.GetValue(err_info, sector, "log_path", log_path, "../log/");
+  fileini.GetValue(err_info, sector, "log_path", log_path, "../log/tubemq");
   fileini.GetValue(err_info, sector, "log_level", log_level, 4);
-  log_level = TUBEMQ_MID(log_level, 0, 4);
+  log_level = TUBEMQ_MID(log_level, 4, 0);
   GetLogger().Init(log_path, Logger::Level(log_level), log_size, log_num);
 }
 
+void TubeMQService::iniXfsThread(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t dns_xfs_period_ms = 30 * 1000;
+  fileini.GetValue(err_info, sector, "dns_xfs_period_ms", dns_xfs_period_ms, 30 * 1000);
+  TUBEMQ_MID(dns_xfs_period_ms, tb_config::kMaxIntValue, 10000);
+  dns_xfs_thread_ = std::thread(&TubeMQService::thread_task_dnsxfs, this, dns_xfs_period_ms);
+}
+
+void TubeMQService::iniPoolThreads(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t timer_threads = 2;
+  int32_t network_threads = 4;
+  int32_t signal_threads = 8;
+  fileini.GetValue(err_info, sector, "timer_threads", timer_threads, 2);
+  TUBEMQ_MID(timer_threads, 50, 2);
+  fileini.GetValue(err_info, sector, "network_threads", network_threads, 4);
+  TUBEMQ_MID(network_threads, 50, 4);
+  fileini.GetValue(err_info, sector, "signal_threads", signal_threads, 8);
+  TUBEMQ_MID(signal_threads, 50, 4);
+  timer_executor_->Resize(timer_threads);
+  network_executor_->Resize(network_threads);
+  thread_pool_ = std::make_shared<ThreadPool>(signal_threads);
+  connection_pool_ = std::make_shared<ConnectionPool>(network_executor_);
+}
 
 int32_t TubeMQService::GetClientObjCnt() {
   lock_guard<mutex> lck(mutex_);
   return clients_map_.size();
 }
 
-
 bool TubeMQService::AddClientObj(string& err_info, BaseClient* client_obj) {
-  if (service_status_.Get() != 0) {
+  if (!IsRunning()) {
     err_info = "Service not startted!";
     return false;
   }
   int32_t client_index = client_index_base_.IncrementAndGet();
   lock_guard<mutex> lck(mutex_);
+  clients_map_[client_index] = client_obj;
   client_obj->SetClientIndex(client_index);
-  this->clients_map_[client_index] = client_obj;
   err_info = "Ok";
   return true;
 }
@@ -132,7 +177,7 @@ bool TubeMQService::AddClientObj(string& err_info, BaseClient* client_obj) {
 BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
   BaseClient* client_obj = NULL;
   map<int32_t, BaseClient*>::const_iterator it;
-  
+
   lock_guard<mutex> lck(mutex_);
   it = clients_map_.find(client_index);
   if (it != clients_map_.end()) {
@@ -141,17 +186,14 @@ BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
   return client_obj;
 }
 
-BaseClient* TubeMQService::RmvClientObj(int32_t client_index) {
-  BaseClient* client_obj = NULL;
+void TubeMQService::RmvClientObj(BaseClient* client_obj) {
   map<int32_t, BaseClient*>::iterator it;
-  
-  lock_guard<mutex> lck(mutex_);
-  it = clients_map_.find(client_index);
-  if (it != clients_map_.end()) {
-    client_obj = it->second;
+  if (client_obj != NULL) {
+    lock_guard<mutex> lck(mutex_);
+    int32_t client_index = client_obj->GetClientIndex();
     clients_map_.erase(client_index);
+    client_obj->SetClientIndex(tb_config::kInvalidValue);
   }
-  return client_obj;
 }
 
 void TubeMQService::shutDownClinets() const {
@@ -162,5 +204,93 @@ void TubeMQService::shutDownClinets() const {
   }
 }
 
+bool TubeMQService::AddMasterAddress(string& err_info, const string& master_info) {
+  map<string, int32_t>::iterator it;
+  map<string, int32_t> tmp_addr_map;
+  Utils::Split(master_info, tmp_addr_map, delimiter::kDelimiterComma, delimiter::kDelimiterColon);
+  if (tmp_addr_map.empty()) {
+    err_info = "Illegal parameter: master_info is blank!";
+    return false;
+  }
+  for (it = tmp_addr_map.begin(); it != tmp_addr_map.end();) {
+    if (!Utils::NeedDnsXfs(it->first)) {
+      tmp_addr_map.erase(it++);
+    }
+  }
+  if (tmp_addr_map.empty()) {
+    err_info = "Ok";
+    return true;
+  }
+  if (addNeedDnsXfsAddr(tmp_addr_map)) {
+    updMasterAddrByDns();
+  }
+  err_info = "Ok";
+  return true;
+}
+
+void TubeMQService::GetXfsMasterAddress(const string& source, string& target) {
+  target = source;
+  lock_guard<mutex> lck(mutex_);
+  if (master_source_.find(source) != master_source_.end()) {
+    target = master_target_[source];
+  }
+}
+
+void TubeMQService::thread_task_dnsxfs(int dns_xfs_period_ms) {
+  LOG_INFO("[TubeMQService] DSN transfer thread startted!");
+  while (true) {
+    if (TubeMQService::Instance()->GetServiceStatus() <= 0) {
+      break;
+    }
+    if ((Utils::GetCurrentTimeMillis() - last_check_time_) >= dns_xfs_period_ms) {
+      TubeMQService::Instance()->updMasterAddrByDns();
+      last_check_time_ = Utils::GetCurrentTimeMillis();
+    }
+    std::this_thread::sleep_for(std::chrono::milliseconds(500));
+  }
+  LOG_INFO("[TubeMQService] DSN transfer thread stopped!");
+}
+
+bool TubeMQService::hasXfsTask(map<string, int32_t>& src_addr_map) {
+  lock_guard<mutex> lck(mutex_);
+  if (!master_source_.empty()) {
+    src_addr_map = master_source_;
+    return true;
+  }
+  return false;
+}
+
+bool TubeMQService::addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map) {
+  bool added = false;
+  map<string, int32_t>::iterator it;
+  if (!src_addr_map.empty()) {
+    lock_guard<mutex> lck(mutex_);
+    for (it = src_addr_map.begin(); it != src_addr_map.end(); it++) {
+      if (master_source_.find(it->first) == master_source_.end()) {
+        added = true;
+        master_source_[it->first] = it->second;
+      }
+    }
+  }
+  return added;
+}
+
+void TubeMQService::updMasterAddrByDns() {
+  map<string, int32_t> tmp_src_addr_map;
+  map<string, string> tmp_tgt_addr_map;
+  map<string, int32_t>::iterator it;
+  if (!hasXfsTask(tmp_src_addr_map)) {
+    return;
+  }
+  Utils::XfsAddrByDns(tmp_src_addr_map, tmp_tgt_addr_map);
+  lock_guard<mutex> lck(mutex_);
+  if (tmp_tgt_addr_map.empty()) {
+    for (it = tmp_src_addr_map.begin(); it != tmp_src_addr_map.end(); it++) {
+      master_target_[it->first] = it->first;
+    }
+  } else {
+    master_target_ = tmp_tgt_addr_map;
+  }
+}
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.h b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
new file mode 100644
index 0000000..c6e0aaa
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
@@ -0,0 +1,296 @@
+/**
+ * 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 "client_service.h"
+
+#include <sstream>
+
+#include "const_config.h"
+#include "logger.h"
+#include "utils.h"
+
+namespace tubemq {
+
+using std::lock_guard;
+using std::stringstream;
+
+BaseClient::BaseClient(bool is_producer) {
+  is_producer_ = is_producer;
+  client_index_ = tb_config::kInvalidValue;
+}
+
+BaseClient::~BaseClient() {
+  // no code
+}
+
+TubeMQService* TubeMQService::_instance = NULL;
+
+static mutex tubemq_mutex_service_;
+
+TubeMQService* TubeMQService::Instance() {
+  if (NULL == _instance) {
+    lock_guard<mutex> lck(tubemq_mutex_service_);
+    if (NULL == _instance) {
+      _instance = new TubeMQService;
+    }
+  }
+  return _instance;
+}
+
+TubeMQService::TubeMQService()
+    : timer_executor_(std::make_shared<ExecutorPool>(2)),
+      network_executor_(std::make_shared<ExecutorPool>(4)) {
+  service_status_.Set(0);
+  client_index_base_.Set(0);
+  last_check_time_ = 0;
+}
+
+TubeMQService::~TubeMQService() {
+  string err_info;
+  Stop(err_info);
+}
+
+bool TubeMQService::Start(string& err_info, string conf_file) {
+  // check configure file
+  bool result = false;
+  Fileini fileini;
+  string sector = "TubeMQ";
+
+  result = Utils::ValidConfigFile(err_info, conf_file);
+  if (!result) {
+    return result;
+  }
+  result = fileini.Loadini(err_info, conf_file);
+  if (!result) {
+    return result;
+  }
+  result = Utils::GetLocalIPV4Address(err_info, local_host_);
+  if (!result) {
+    return result;
+  }
+  if (!service_status_.CompareAndSet(0, 1)) {
+    err_info = "TubeMQ Service has startted or Stopped!";
+    return false;
+  }
+  iniLogger(fileini, sector);
+  iniPoolThreads(fileini, sector);
+  iniXfsThread(fileini, sector);
+  service_status_.Set(2);
+  err_info = "Ok!";
+  LOG_INFO("[TubeMQService] TubeMQ service startted!");
+
+  return true;
+}
+
+bool TubeMQService::Stop(string& err_info) {
+  if (service_status_.CompareAndSet(2, -1)) {
+    LOG_INFO("[TubeMQService] TubeMQ service begin to stop!");
+    if (dns_xfs_thread_.joinable()) {
+      dns_xfs_thread_.join();
+    }
+    shutDownClinets();
+    timer_executor_->Close();
+    network_executor_->Close();
+    connection_pool_ = nullptr;
+    thread_pool_ = nullptr;
+    LOG_INFO("[TubeMQService] TubeMQ service stopped!");
+  }
+  err_info = "OK!";
+  return true;
+}
+
+bool TubeMQService::IsRunning() { return (service_status_.Get() == 2); }
+
+void TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t log_num = 10;
+  int32_t log_size = 10;
+  int32_t log_level = 4;
+  string log_path = "../log/tubemq";
+  fileini.GetValue(err_info, sector, "log_num", log_num, 10);
+  fileini.GetValue(err_info, sector, "log_size", log_size, 100);
+  fileini.GetValue(err_info, sector, "log_path", log_path, "../log/tubemq");
+  fileini.GetValue(err_info, sector, "log_level", log_level, 4);
+  log_level = TUBEMQ_MID(log_level, 4, 0);
+  GetLogger().Init(log_path, Logger::Level(log_level), log_size, log_num);
+}
+
+void TubeMQService::iniXfsThread(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t dns_xfs_period_ms = 30 * 1000;
+  fileini.GetValue(err_info, sector, "dns_xfs_period_ms", dns_xfs_period_ms, 30 * 1000);
+  TUBEMQ_MID(dns_xfs_period_ms, tb_config::kMaxIntValue, 10000);
+  dns_xfs_thread_ = std::thread(&TubeMQService::thread_task_dnsxfs, this, dns_xfs_period_ms);
+}
+
+void TubeMQService::iniPoolThreads(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t timer_threads = 2;
+  int32_t network_threads = 4;
+  int32_t signal_threads = 8;
+  fileini.GetValue(err_info, sector, "timer_threads", timer_threads, 2);
+  TUBEMQ_MID(timer_threads, 50, 2);
+  fileini.GetValue(err_info, sector, "network_threads", network_threads, 4);
+  TUBEMQ_MID(network_threads, 50, 4);
+  fileini.GetValue(err_info, sector, "signal_threads", signal_threads, 8);
+  TUBEMQ_MID(signal_threads, 50, 4);
+  timer_executor_->Resize(timer_threads);
+  network_executor_->Resize(network_threads);
+  thread_pool_ = std::make_shared<ThreadPool>(signal_threads);
+  connection_pool_ = std::make_shared<ConnectionPool>(network_executor_);
+}
+
+int32_t TubeMQService::GetClientObjCnt() {
+  lock_guard<mutex> lck(mutex_);
+  return clients_map_.size();
+}
+
+bool TubeMQService::AddClientObj(string& err_info, BaseClient* client_obj) {
+  if (!IsRunning()) {
+    err_info = "Service not startted!";
+    return false;
+  }
+  int32_t client_index = client_index_base_.IncrementAndGet();
+  lock_guard<mutex> lck(mutex_);
+  clients_map_[client_index] = client_obj;
+  client_obj->SetClientIndex(client_index);
+  err_info = "Ok";
+  return true;
+}
+
+BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
+  BaseClient* client_obj = NULL;
+  map<int32_t, BaseClient*>::const_iterator it;
+
+  lock_guard<mutex> lck(mutex_);
+  it = clients_map_.find(client_index);
+  if (it != clients_map_.end()) {
+    client_obj = it->second;
+  }
+  return client_obj;
+}
+
+void TubeMQService::RmvClientObj(BaseClient* client_obj) {
+  map<int32_t, BaseClient*>::iterator it;
+  if (client_obj != NULL) {
+    lock_guard<mutex> lck(mutex_);
+    int32_t client_index = client_obj->GetClientIndex();
+    clients_map_.erase(client_index);
+    client_obj->SetClientIndex(tb_config::kInvalidValue);
+  }
+}
+
+void TubeMQService::shutDownClinets() const {
+  map<int32_t, BaseClient*>::const_iterator it;
+  lock_guard<mutex> lck(mutex_);
+  for (it = clients_map_.begin(); it != clients_map_.end(); it++) {
+    it->second->ShutDown();
+  }
+}
+
+bool TubeMQService::AddMasterAddress(string& err_info, const string& master_info) {
+  map<string, int32_t>::iterator it;
+  map<string, int32_t> tmp_addr_map;
+  Utils::Split(master_info, tmp_addr_map, delimiter::kDelimiterComma, delimiter::kDelimiterColon);
+  if (tmp_addr_map.empty()) {
+    err_info = "Illegal parameter: master_info is blank!";
+    return false;
+  }
+  for (it = tmp_addr_map.begin(); it != tmp_addr_map.end();) {
+    if (!Utils::NeedDnsXfs(it->first)) {
+      tmp_addr_map.erase(it++);
+    }
+  }
+  if (tmp_addr_map.empty()) {
+    err_info = "Ok";
+    return true;
+  }
+  if (addNeedDnsXfsAddr(tmp_addr_map)) {
+    updMasterAddrByDns();
+  }
+  err_info = "Ok";
+  return true;
+}
+
+void TubeMQService::GetXfsMasterAddress(const string& source, string& target) {
+  target = source;
+  lock_guard<mutex> lck(mutex_);
+  if (master_source_.find(source) != master_source_.end()) {
+    target = master_target_[source];
+  }
+}
+
+void TubeMQService::thread_task_dnsxfs(int dns_xfs_period_ms) {
+  LOG_INFO("[TubeMQService] DSN transfer thread startted!");
+  while (true) {
+    if (TubeMQService::Instance()->GetServiceStatus() <= 0) {
+      break;
+    }
+    if ((Utils::GetCurrentTimeMillis() - last_check_time_) >= dns_xfs_period_ms) {
+      TubeMQService::Instance()->updMasterAddrByDns();
+      last_check_time_ = Utils::GetCurrentTimeMillis();
+    }
+    std::this_thread::sleep_for(std::chrono::milliseconds(500));
+  }
+  LOG_INFO("[TubeMQService] DSN transfer thread stopped!");
+}
+
+bool TubeMQService::hasXfsTask(map<string, int32_t>& src_addr_map) {
+  lock_guard<mutex> lck(mutex_);
+  if (!master_source_.empty()) {
+    src_addr_map = master_source_;
+    return true;
+  }
+  return false;
+}
+
+bool TubeMQService::addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map) {
+  bool added = false;
+  map<string, int32_t>::iterator it;
+  if (!src_addr_map.empty()) {
+    lock_guard<mutex> lck(mutex_);
+    for (it = src_addr_map.begin(); it != src_addr_map.end(); it++) {
+      if (master_source_.find(it->first) == master_source_.end()) {
+        added = true;
+        master_source_[it->first] = it->second;
+      }
+    }
+  }
+  return added;
+}
+
+void TubeMQService::updMasterAddrByDns() {
+  map<string, int32_t> tmp_src_addr_map;
+  map<string, string> tmp_tgt_addr_map;
+  map<string, int32_t>::iterator it;
+  if (!hasXfsTask(tmp_src_addr_map)) {
+    return;
+  }
+  Utils::XfsAddrByDns(tmp_src_addr_map, tmp_tgt_addr_map);
+  lock_guard<mutex> lck(mutex_);
+  if (tmp_tgt_addr_map.empty()) {
+    for (it = tmp_src_addr_map.begin(); it != tmp_src_addr_map.end(); it++) {
+      master_target_[it->first] = it->first;
+    }
+  } else {
+    master_target_ = tmp_tgt_addr_map;
+  }
+}
+
+}  // namespace tubemq


[incubator-tubemq] 20/50: [TUBEMQ-251] Create C/C++ Codec utils (#191)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 881e4be877bf1c75aa5896e2ab0857f295cd17f4
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 6 13:03:31 2020 +0000

    [TUBEMQ-251] Create C/C++ Codec utils (#191)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/const_config.h           |  4 --
 .../tubemq-client-cpp/inc/const_rpc.h              | 83 ++++++++++++++++++++++
 .../tubemq-client-cpp/src/client_config.cc         | 12 ++--
 3 files changed, 90 insertions(+), 9 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index 80d8b3b..4e04ec0 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -29,10 +29,6 @@ using namespace std;
 
 // configuration value setting
 namespace config {
-// rpc timeout define  
-static const int kRpcTimoutDef = 15;
-static const int kRpcTimoutMax = 300;
-static const int kRpcTimoutMin = 8;
 // heartbeat period define
 static const int kHeartBeatPeriodDef = 10;
 static const int kHeartBeatFailRetryTimesDef = 5;
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
new file mode 100644
index 0000000..cb89eba
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.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 _TUBEMQ_CLIENT_CONST_RPC_H_
+#define _TUBEMQ_CLIENT_CONST_RPC_H_
+
+namespace tubemq {
+
+using namespace std;
+
+
+namespace rpc_config {
+
+  // constant define
+  static const int kRpcPrtBeginToken    = 0xFF7FF4FE;
+  static const int kRpcMaxBufferSize    = 8192;
+  static const int kRpcMaxFrameListCnt  = (int) ((1024 * 1024 * 8) / kRpcMaxBufferSize);
+  // rpc protocol version
+  static const int kRpcProtocolVersion  = 2;
+  // msg type flag
+  static const int kRpcFlagMsgRequest   = 0x0;
+  static const int kRpcFlagMsgResponse  = 0x1;
+  // service type
+  static const int kMasterService      = 1;
+  static const int kBrokerReadService  = 2;
+  static const int kBrokerWriteService = 3;
+  static const int kBrokerAdminService = 4;
+  static const int kMasterAdminService = 5;
+  // request method
+  // master rpc method
+  static const int kMasterMethoddProducerRegister = 1;
+  static const int kMasterMethoddProducerHeatbeat = 2;
+  static const int kMasterMethoddProducerClose    = 3;
+  static const int kMasterMethoddConsumerRegister = 4;
+  static const int kMasterMethoddConsumerHeatbeat = 5;
+  static const int kMasterMethoddConsumerClose    = 6;
+  // broker rpc method
+  static const int kBrokerMethoddProducerRegister    = 11;
+  static const int kBrokerMethoddProducerHeatbeat    = 12;
+  static const int kBrokerMethoddProducerSendMsg     = 13;
+  static const int kBrokerMethoddProducerClose       = 14;
+  static const int kBrokerMethoddConsumerRegister    = 15;
+  static const int kBrokerMethoddConsumerHeatbeat    = 16;
+  static const int kBrokerMethoddConsumerGetMsg      = 17;
+  static const int kBrokerMethoddConsumerCommit      = 18;  
+  static const int kBrokerMethoddConsumerClose       = 19; 
+
+  // register operate type
+  static const int kRegOpTypeRegister       = 31; 
+  static const int kRegOpTypeUnReg          = 32; 
+
+  // rpc connect node timeout
+  static const int kRpcConnectTimeoutMs    = 3000;
+  
+  // rpc timeout define  
+  static const int kRpcTimoutDefSec = 15;
+  static const int kRpcTimoutMaxSec = 300;
+  static const int kRpcTimoutMinSec = 8;
+
+
+}
+
+
+}
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index e668f81..d3a03ba 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -21,6 +21,8 @@
 #include <vector>
 #include "client_config.h"
 #include "const_config.h"
+#include "const_rpc.h"
+
 #include "utils.h"
 
 
@@ -35,7 +37,7 @@ BaseConfig::BaseConfig() {
   this->tls_enabled_ = false;
   this->tls_trust_store_path_ = "";
   this->tls_trust_store_password_ = "";
-  this->rpc_read_timeout_sec_ = config::kRpcTimoutDef;
+  this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutDefSec;
   this->heartbeat_period_sec_ = config::kHeartBeatPeriodDef;
   this->max_heartbeat_retry_times_ = config::kHeartBeatFailRetryTimesDef;
   this->heartbeat_period_afterfail_sec_ = config::kHeartBeatSleepPeriodDef;
@@ -168,10 +170,10 @@ const string& BaseConfig::GetUsrPassWord() const {
 }
 
 void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
-  if (rpc_read_timeout_sec >= config::kRpcTimoutMax) {
-    this->rpc_read_timeout_sec_ = config::kRpcTimoutMax;
-  } else if (rpc_read_timeout_sec <= config::kRpcTimoutMin) {
-    this->rpc_read_timeout_sec_ = config::kRpcTimoutMin;
+  if (rpc_read_timeout_sec >= rpc_config::kRpcTimoutMaxSec) {
+    this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutMaxSec;
+  } else if (rpc_read_timeout_sec <= rpc_config::kRpcTimoutMinSec) {
+    this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutMinSec;
   } else {
     this->rpc_read_timeout_sec_ = rpc_read_timeout_sec;
   }


[incubator-tubemq] 15/50: [TUBEMQ-263] Create C/C++ ini file read utils (#184)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit eee2db00124fe1be8e354fe5bea48e0f351f1c38
Author: gosonzhang <46...@qq.com>
AuthorDate: Sat Jul 4 16:16:52 2020 +0000

    [TUBEMQ-263] Create C/C++ ini file read utils (#184)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/const_config.h           |   8 ++
 .../tubemq-client-cpp/inc/file_ini.h               |  48 ++++++++
 .../tubemq-client-cpp/src/file_ini.cc              | 136 +++++++++++++++++++++
 3 files changed, 192 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
index a48873d..1cc2b60 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
@@ -82,6 +82,14 @@ namespace delimiter {
   static const string kDelimiterColon = ":";
   static const string kDelimiterAt    = "@";
   static const string kDelimiterPound = "#";
+  static const string kDelimiterSemicolon = ";";
+  //Double slash
+  static const string kDelimiterDbSlash = "//";
+  // left square bracket
+  static const string kDelimiterLftSB = "[";
+  // right square bracket
+  static const string kDelimiterRgtSB = "]";
+  
 } // namespace delimiter
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
new file mode 100644
index 0000000..ecc7f1f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.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 _TUBEMQ_CLIENT_FILE_INI_H_
+#define _TUBEMQ_CLIENT_FILE_INI_H_
+
+#include <map>
+#include <string>
+
+namespace tubemq {
+
+using namespace std;
+
+class Fileini {
+ public:
+  Fileini();
+  ~Fileini();
+  bool Loadini(string& err_info, const string& file_name);
+  bool GetValue(string& err_info, const string& sector, 
+                   const string& key, string& value, const string& def);
+
+ private:
+  bool init_flag_;
+  // sector        key    value
+  map<string, map<string, string> > ini_map_;
+};
+
+  
+}
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
new file mode 100644
index 0000000..84a847f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -0,0 +1,136 @@
+/**
+ * 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 <fstream>
+#include <sstream> 
+#include "utils.h"
+#include "file_ini.h"
+#include "const_config.h"
+
+namespace tubemq {
+
+
+Fileini::Fileini() {
+  this->init_flag_ = false;
+  this->ini_map_.clear();
+}
+
+Fileini::~Fileini() {
+  this->init_flag_ = false;
+  this->ini_map_.clear();
+}
+
+bool Fileini::Loadini(string& err_info, const string& file_name) {
+  // check paramter
+  if (file_name.empty()) {
+    err_info = "Ini configure file is null!";
+    return false;
+  }
+  // open configure file and check
+  ifstream conf_file(file_name.c_str());
+  if (!conf_file.is_open()) {
+    err_info = "Open file " + file_name + " failure!";
+    return false;
+  }
+  string line_str = "";
+  string sector = "";
+  string key = "";
+  string value = "";
+  string::size_type lftsb_pos = 0;
+  string::size_type rgtsb_pos = 0;
+  string::size_type equal_pos = 0;
+  // read ini file and parse content
+  while (getline(conf_file, line_str)) {
+    // check if a comment
+    line_str = Utils::Trim(line_str);
+    if (line_str.empty() 
+      || line_str.find(delimiter::kDelimiterDbSlash) == 0 
+      || line_str.find(delimiter::kDelimiterSemicolon) == 0) {
+      continue;
+    }
+    // check if a sector head
+    lftsb_pos = line_str.find(delimiter::kDelimiterLftSB);
+    rgtsb_pos = line_str.find(delimiter::kDelimiterRgtSB);
+    if (lftsb_pos != string::npos && rgtsb_pos != string::npos) {
+      sector = line_str.substr(lftsb_pos + (delimiter::kDelimiterLftSB).size(), 
+        rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
+      sector = Utils::Trim(sector);
+      continue;
+    }
+    // check if a key=value string
+    equal_pos = line_str.find(delimiter::kDelimiterEqual);
+    if (equal_pos == string::npos) {
+      continue;
+    }
+    key = line_str.substr(0, equal_pos);
+    value = line_str.substr(equal_pos + (delimiter::kDelimiterEqual).size(), line_str.size() - 1);
+    key = Utils::Trim(key);
+    value = Utils::Trim(value);
+    // get data from file to memory
+    if (sector.empty() && key.empty() && value.empty()) {
+      continue;
+    }
+    map<string, map<string, string> >::iterator it_sec;
+    it_sec = this->ini_map_.find(sector);
+    if (it_sec == this->ini_map_.end()) {
+      map<string, string> tmp_key_val_map;
+      tmp_key_val_map[key] = value;
+      ini_map_[sector] = tmp_key_val_map;
+    } else {
+      it_sec->second[key] = value;
+    }
+  }
+  // close configure file and clear status
+  conf_file.close();
+  conf_file.clear();
+  // set parser status
+  this->init_flag_ = true;
+  // end
+  err_info = "Ok";
+  return true;
+}
+
+bool Fileini::GetValue(string& err_info, const string& sector, 
+                const string& key, string& value, const string& def) {
+  if (!this->init_flag_) {
+    err_info = "Please load configure file first!";
+    return false;
+  }
+  err_info = "Ok";
+  // search key's value in sector
+  map<string, map<string, string> >::iterator it_sec;
+  map<string, string>::iterator it_keyval;
+  it_sec = this->ini_map_.find(sector);
+  if (it_sec == this->ini_map_.end()) {
+    value = def;
+    return true;
+  }
+  it_keyval = it_sec->second.find(key);
+  if (it_keyval == it_sec->second.end()) {
+    value = def;
+    return true;
+  }
+  value = it_keyval->second;
+  return true;
+}
+
+
+}
+
+


[incubator-tubemq] 40/50: [TUBEMQ-292]C++ SDK singleton & executor_pool optimization (#217)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 2d6b66b14b524e385ed0a07402792de32cbdf808
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Tue Jul 21 17:22:18 2020 +0800

    [TUBEMQ-292]C++ SDK singleton & executor_pool optimization (#217)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/executor_pool.h               | 6 ++++++
 tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h   | 7 ++++---
 tubemq-client-twins/tubemq-client-cpp/src/logger.cc                | 6 +++---
 3 files changed, 13 insertions(+), 6 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
index 404616b..0b20c4f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
@@ -74,6 +74,12 @@ class ExecutorPool : noncopyable {
 
   ExecutorPtr Get();
 
+  // Resize executor thread
+  void Resize(int nthreads) {
+    Lock lock(mutex_);
+    executors_.resize(nthreads);
+  }
+
   void Close();
 
  private:
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
index 2761bda..1978da6 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
@@ -33,15 +33,16 @@ namespace tubemq {
 template <typename T>
 class Singleton : noncopyable {
  public:
-  Singleton() = delete;
-  ~Singleton() = delete;
-
   static T& Instance() {
     std::call_once(once_, Singleton::init);
     assert(value_ != nullptr);
     return *value_;
   }
 
+ protected:
+  Singleton() {}
+  ~Singleton() {}
+
  private:
   static void init() { value_ = new T(); }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
index 0fd2fbf..f374d9b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -27,11 +27,11 @@
 
 #include <string>
 
-namespace tubemq {
+#include "tubemq/singleton.h"
 
-Logger tubemq_logger;
+namespace tubemq {
 
-Logger& GetLogger() { return tubemq_logger; }
+Logger& GetLogger() { return Singleton<Logger>::Instance(); }
 
 static const uint32_t kMBSize = 1024 * 1024;
 


[incubator-tubemq] 16/50: [TUBEMQ-263] Create C/C++ ini file read utils [addendum] (#185)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit ebf8d48034c98b410fcac29bfdc7974847c3ea0f
Author: gosonzhang <46...@qq.com>
AuthorDate: Sun Jul 5 12:05:32 2020 +0000

    [TUBEMQ-263] Create C/C++ ini file read utils [addendum] (#185)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h  |  2 ++
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h     |  2 ++
 tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc | 17 ++++++++++++++++-
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc    | 13 +++++++++++++
 4 files changed, 33 insertions(+), 1 deletion(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
index ecc7f1f..cab7f84 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
@@ -34,6 +34,8 @@ class Fileini {
   bool Loadini(string& err_info, const string& file_name);
   bool GetValue(string& err_info, const string& sector, 
                    const string& key, string& value, const string& def);
+  bool GetValue(string& err_info, const string& sector, 
+                   const string& key, int& value, const int def);  
 
  private:
   bool init_flag_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index de39683..0709bb8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -47,6 +47,8 @@ class Utils {
                    const string& group_name, string& tgt_group_name);
   static bool ValidFilterItem(string& err_info, 
                    const string& src_filteritem, string& tgt_filteritem);
+  static string int2str(int data);
+  static string long2str(long data);
   static long GetCurrentTimeMillis();
 
 };
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 84a847f..822b803 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -19,6 +19,7 @@
 
 #include <fstream>
 #include <sstream> 
+#include <stdlib.h>
 #include "utils.h"
 #include "file_ini.h"
 #include "const_config.h"
@@ -79,7 +80,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
       continue;
     }
     key = line_str.substr(0, equal_pos);
-    value = line_str.substr(equal_pos + (delimiter::kDelimiterEqual).size(), line_str.size() - 1);
+    value = line_str.substr(equal_pos + (delimiter::kDelimiterEqual).size(), line_str.size());
     key = Utils::Trim(key);
     value = Utils::Trim(value);
     // get data from file to memory
@@ -113,6 +114,7 @@ bool Fileini::GetValue(string& err_info, const string& sector,
     return false;
   }
   err_info = "Ok";
+  value.clear();
   // search key's value in sector
   map<string, map<string, string> >::iterator it_sec;
   map<string, string>::iterator it_keyval;
@@ -130,6 +132,19 @@ bool Fileini::GetValue(string& err_info, const string& sector,
   return true;
 }
 
+bool Fileini::GetValue(string& err_info, const string& sector, 
+                const string& key, int& value, const int def) {
+  string val_str;
+  string def_str = Utils::int2str(def);
+  bool result = GetValue(err_info, sector, key, val_str, def_str);
+  if (!result) {
+    return result;
+  }
+  value = atoi(val_str.c_str());
+  return true;
+}
+
+
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 9ddd3b0..cd60982 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -239,6 +239,19 @@ bool Utils::ValidFilterItem(string& err_info,
 }
 
 
+string Utils::int2str(int data) {
+  stringstream ss;
+  ss<<data;
+  return ss.str();
+}
+
+string Utils::long2str(long data) {
+  stringstream ss;
+  ss<<data;
+  return ss.str();
+}
+
+
 long Utils::GetCurrentTimeMillis() {
   struct timeval tv;
   gettimeofday(&tv,NULL);


[incubator-tubemq] 25/50: [TUBEMQ-272]Unified C/C++ files's code style (#196)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 38844d020905667d7913f0fda3c2bbb2ccabdc8b
Author: gosonzhang <46...@qq.com>
AuthorDate: Thu Jul 9 06:10:25 2020 +0000

    [TUBEMQ-272]Unified C/C++ files's code style (#196)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/atomic_def.h  | 116 ++++++++++-----------
 .../include/tubemq/const_config.h                  |   4 +-
 .../include/tubemq/const_errcode.h                 |   4 +-
 .../include/tubemq/flowctrl_def.h                  |  17 +--
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |   7 +-
 .../tubemq-client-cpp/include/tubemq/singleton.h   |   2 +-
 .../tubemq-client-cpp/src/client_config.cc         |  40 +++----
 .../tubemq-client-cpp/src/file_ini.cc              |   6 +-
 .../tubemq-client-cpp/src/flowctrl_def.cc          |  80 +++++++-------
 .../tubemq-client-cpp/src/logger.cc                |   2 +-
 .../tubemq-client-cpp/src/message.cc               |  10 +-
 .../tubemq-client-cpp/src/meta_info.cc             | 108 ++++++++++---------
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  12 +--
 13 files changed, 211 insertions(+), 197 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
index fefe325..c8030db 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
@@ -30,81 +30,81 @@ class AtomicInteger {
  public:
   AtomicInteger() { this->counter_ = 0; }
 
-  AtomicInteger(int initial_value) { this->counter_ = initial_value; }
+  AtomicInteger(int32_t initial_value) { this->counter_ = initial_value; }
 
-  int Get() { return this->counter_; }
+  int32_t Get() const { return this->counter_; }
 
-  void Set(long new_value) { this->counter_ = new_value; }
+  void Set(int32_t new_value) { this->counter_ = new_value; }
 
-  long LongValue() { return (long)this->counter_; }
+  int64_t LongValue() const { return (int64_t)this->counter_; }
 
-  int GetAndSet(int new_value) {
+  int32_t GetAndSet(int32_t new_value) {
     for (;;) {
-      int current = this->counter_;
+      int32_t current = this->counter_;
       if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
         return current;
       }
     }
   }
 
-  bool CompareAndSet(int expect, int update) {
+  bool CompareAndSet(int32_t expect, int32_t update) {
     return __sync_bool_compare_and_swap(&this->counter_, expect, update);
   }
 
-  int GetAndIncrement() {
+  int32_t GetAndIncrement() {
     for (;;) {
-      int current = this->counter_;
-      int next = current + 1;
+      int32_t current = this->counter_;
+      int32_t next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  int GetAndDecrement() {
+  int32_t GetAndDecrement() {
     for (;;) {
-      int current = this->counter_;
-      int next = current - 1;
+      int32_t current = this->counter_;
+      int32_t next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  int GetAndAdd(int delta) {
+  int32_t GetAndAdd(int32_t delta) {
     for (;;) {
-      int current = this->counter_;
-      int next = current + delta;
+      int32_t current = this->counter_;
+      int32_t next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  int IncrementAndGet() {
+  int32_t IncrementAndGet() {
     for (;;) {
-      int current = this->counter_;
-      int next = current + 1;
+      int32_t current = this->counter_;
+      int32_t next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
   }
 
-  int DecrementAndGet() {
+  int32_t DecrementAndGet() {
     for (;;) {
-      int current = this->counter_;
-      int next = current - 1;
+      int32_t current = this->counter_;
+      int32_t next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
   }
 
-  int AddAndGet(int delta) {
+  int32_t AddAndGet(int32_t delta) {
     for (;;) {
-      int current = this->counter_;
-      int next = current + delta;
+      int32_t current = this->counter_;
+      int32_t next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
@@ -112,88 +112,88 @@ class AtomicInteger {
   }
 
  private:
-  volatile int counter_;
+  volatile int32_t counter_;
 };
 
 class AtomicLong {
  public:
   AtomicLong() { this->counter_ = 0; }
 
-  AtomicLong(long initial_value) { this->counter_ = initial_value; }
+  AtomicLong(int64_t initial_value) { this->counter_ = initial_value; }
 
-  long Get() { return this->counter_; }
+  int64_t Get() const { return this->counter_; }
 
-  void Set(long new_value) { this->counter_ = new_value; }
+  void Set(int64_t new_value) { this->counter_ = new_value; }
 
-  long IntValue() { return (int)this->counter_; }
+  int32_t IntValue() const { return (int32_t)this->counter_; }
 
-  long GetAndSet(long new_value) {
+  int64_t GetAndSet(int64_t new_value) {
     for (;;) {
-      long current = this->counter_;
+      int64_t current = this->counter_;
       if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
         return current;
       }
     }
   }
 
-  bool CompareAndSet(long expect, long update) {
+  bool CompareAndSet(int64_t expect, int64_t update) {
     return __sync_bool_compare_and_swap(&this->counter_, expect, update);
   }
 
-  long GetAndIncrement() {
+  int64_t GetAndIncrement() {
     for (;;) {
-      long current = this->counter_;
-      long next = current + 1;
+      int64_t current = this->counter_;
+      int64_t next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  long GetAndDecrement() {
+  int64_t GetAndDecrement() {
     for (;;) {
-      long current = this->counter_;
-      long next = current - 1;
+      int64_t current = this->counter_;
+      int64_t next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  long GetAndAdd(long delta) {
+  int64_t GetAndAdd(int64_t delta) {
     for (;;) {
-      long current = this->counter_;
-      long next = current + delta;
+      int64_t current = this->counter_;
+      int64_t next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return current;
       }
     }
   }
 
-  long IncrementAndGet() {
+  int64_t IncrementAndGet() {
     for (;;) {
-      long current = this->counter_;
-      long next = current + 1;
+      int64_t current = this->counter_;
+      int64_t next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
   }
 
-  long DecrementAndGet() {
+  int64_t DecrementAndGet() {
     for (;;) {
-      long current = this->counter_;
-      long next = current - 1;
+      int64_t current = this->counter_;
+      int64_t next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
     }
   }
 
-  long AddAndGet(long delta) {
+  int64_t AddAndGet(int64_t delta) {
     for (;;) {
-      long current = this->counter_;
-      long next = current + delta;
+      int64_t current = this->counter_;
+      int64_t next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
         return next;
       }
@@ -201,7 +201,7 @@ class AtomicLong {
   }
 
  private:
-  volatile long counter_;
+  volatile int64_t counter_;
 };
 
 class AtomicBoolean {
@@ -210,14 +210,14 @@ class AtomicBoolean {
 
   AtomicBoolean(bool initial_value) { this->counter_ = initial_value ? 1 : 0; }
 
-  bool Get() { return this->counter_ != 0; }
+  bool Get() const { return this->counter_ != 0; }
 
   void Set(bool new_value) { this->counter_ = new_value ? 1 : 0; }
 
   bool GetAndSet(bool new_value) {
-    int u = new_value ? 1 : 0;
+    int32_t u = new_value ? 1 : 0;
     for (;;) {
-      int e = this->counter_ ? 1 : 0;
+      int32_t e = this->counter_ ? 1 : 0;
       if (__sync_bool_compare_and_swap(&this->counter_, e, u)) {
         return e != 0;
       }
@@ -225,13 +225,13 @@ class AtomicBoolean {
   }
 
   bool CompareAndSet(bool expect, bool update) {
-    int e = expect ? 1 : 0;
-    int u = update ? 1 : 0;
+    int32_t e = expect ? 1 : 0;
+    int32_t u = update ? 1 : 0;
     return __sync_bool_compare_and_swap(&this->counter_, e, u);
   }
 
  private:
-  volatile int counter_;
+  volatile int32_t counter_;
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
index 31a97ed..bc90194 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
@@ -30,7 +30,7 @@ namespace tubemq {
 using std::string;
 
 // configuration value setting
-namespace config {
+namespace tb_config {
 // heartbeat period define
 static const int32_t kHeartBeatPeriodDef = 10;
 static const int32_t kHeartBeatFailRetryTimesDef = 5;
@@ -73,7 +73,7 @@ static const uint32_t kBrokerTlsPortDef = 8124;
 // invalid value
 static const int32_t kInvalidValue = -2;
 
-}  // namespace config
+}  // namespace tb_config
 
 namespace delimiter {
 static const string kDelimiterDot = ".";
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h
index c4e6370..c4d625d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h
@@ -39,11 +39,13 @@ namespace err_code {
   static const int32_t kErrConsumeGroupForbidden = 450;
   static const int32_t kErrConsumeSpeedLimit = 452;
   static const int32_t kErrConsumeContentForbidden = 455;
-  
+
   static const int32_t kErrServerError = 500;
   static const int32_t kErrServiceUnavilable = 503;
   static const int32_t kErrServerMsgsetNullError = 510;
   static const int32_t kErrWaitServerRspTimeout = 550;
+}  // namespace err_code
+
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_CONST_ERR_CODE_H_
\ No newline at end of file
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
index 0f05bb3..c0289ca 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
@@ -29,7 +29,7 @@
 #include <string>
 #include <vector>
 
-#include "atomic_def.h"
+#include "tubemq/atomic_def.h"
 
 namespace tubemq {
 
@@ -65,8 +65,9 @@ class FlowCtrlItem {
   void Clear();
   void ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
                           int32_t min_data_filter_freqms);
-  int32_t GetFreLimit(int32_t msg_zero_cnt);
-  bool GetDataLimit(int64_t datadlt_m, int32_t curr_time, FlowCtrlResult& flowctrl_result);
+  int32_t GetFreLimit(int32_t msg_zero_cnt) const;
+  bool GetDataLimit(int64_t datadlt_m,
+    int32_t curr_time, FlowCtrlResult& flowctrl_result) const;
   const int32_t GetType() const { return type_; }
   const int32_t GetZeroCnt() const { return zero_cnt_; }
   const int32_t GetStartTime() const { return start_time_; }
@@ -91,12 +92,12 @@ class FlowCtrlRuleHandler {
   ~FlowCtrlRuleHandler();
   void UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id, int64_t flowctrl_id,
                              const string& flowctrl_info);
-  bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result);
-  int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit);
-  int32_t GetMinZeroCnt() { return this->min_zero_cnt_.Get(); }
-  int32_t GetQryPriorityId() { return this->qrypriority_id_.Get(); }
+  bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const;
+  int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const;
+  int32_t GetMinZeroCnt() const { return this->min_zero_cnt_.Get(); }
+  int32_t GetQryPriorityId() const { return this->qrypriority_id_.Get(); }
   void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
-  int64_t GetFlowCtrlId() { return this->flowctrl_id_.Get(); }
+  int64_t GetFlowCtrlId() const { return this->flowctrl_id_.Get(); }
   const FlowCtrlItem& GetFilterCtrlItem() const { return this->filter_ctrl_item_; }
   const string& GetFlowCtrlInfo() const { return this->flowctrl_info_; }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index 6dcd324..02d5a63 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -25,7 +25,7 @@
 #include <list>
 #include <string>
 
-#include "flowctrl_def.h"
+#include "tubemq/flowctrl_def.h"
 
 namespace tubemq {
 
@@ -145,7 +145,7 @@ class PartitionExt : public Partition {
   PartitionExt(const NodeInfo& broker_info, const string& part_str);
   ~PartitionExt();
   void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
-    int64_t rsp_dlt_limit, long last_datadlt, bool require_slow);
+    int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
   int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
     const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
   int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
@@ -154,8 +154,11 @@ class PartitionExt : public Partition {
     int64_t last_datadlt, bool require_slow);
   void SetLastConsumed(bool last_consumed);
   bool IsLastConsumed();
+
  private:
   void resetParameters();
+  void updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt);
 
  private:
   bool is_last_consumed_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
index 77eb414..fb22587 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
@@ -26,7 +26,7 @@
 #include <mutex>
 #include <thread>
 
-#include "noncopyable.h"
+#include "tubemq/noncopyable.h"
 
 namespace tubemq {
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index c6da13d..b623497 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-#include "client_config.h"
+#include "tubemq/client_config.h"
 
 #include <sstream>
 #include <vector>
 
-#include "const_config.h"
-#include "const_rpc.h"
-#include "utils.h"
+#include "tubemq/const_config.h"
+#include "tubemq/const_rpc.h"
+#include "tubemq/utils.h"
 
 namespace tubemq {
 
@@ -42,9 +42,9 @@ BaseConfig::BaseConfig() {
   this->tls_trust_store_path_ = "";
   this->tls_trust_store_password_ = "";
   this->rpc_read_timeout_sec_ = rpc_config::kRpcTimoutDefSec;
-  this->heartbeat_period_sec_ = config::kHeartBeatPeriodDef;
-  this->max_heartbeat_retry_times_ = config::kHeartBeatFailRetryTimesDef;
-  this->heartbeat_period_afterfail_sec_ = config::kHeartBeatSleepPeriodDef;
+  this->heartbeat_period_sec_ = tb_config::kHeartBeatPeriodDef;
+  this->max_heartbeat_retry_times_ = tb_config::kHeartBeatFailRetryTimesDef;
+  this->heartbeat_period_afterfail_sec_ = tb_config::kHeartBeatSleepPeriodDef;
 }
 
 BaseConfig::~BaseConfig() {
@@ -75,10 +75,10 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
-  if (trimed_master_addr_info.length() > config::kMasterAddrInfoMaxLength) {
+  if (trimed_master_addr_info.length() > tb_config::kMasterAddrInfoMaxLength) {
     stringstream ss;
     ss << "Illegal parameter: over max ";
-    ss << config::kMasterAddrInfoMaxLength;
+    ss << tb_config::kMasterAddrInfoMaxLength;
     ss << " length of master_addrinfo parameter!";
     err_info = ss.str();
     return false;
@@ -226,11 +226,11 @@ ConsumerConfig::ConsumerConfig() {
   this->consume_position_ = kConsumeFromLatestOffset;
   this->is_confirm_in_local_ = false;
   this->is_rollback_if_confirm_timout_ = true;
-  this->max_subinfo_report_intvl_ = config::kSubInfoReportMaxIntervalTimes;
-  this->msg_notfound_wait_period_ms_ = config::kMsgNotfoundWaitPeriodMsDef;
-  this->reb_confirm_wait_period_ms_ = config::kRebConfirmWaitPeriodMsDef;
-  this->max_confirm_wait_period_ms_ = config::kConfirmWaitPeriodMsMax;
-  this->shutdown_reb_wait_period_ms_ = config::kRebWaitPeriodWhenShutdownMs;
+  this->max_subinfo_report_intvl_ = tb_config::kSubInfoReportMaxIntervalTimes;
+  this->msg_notfound_wait_period_ms_ = tb_config::kMsgNotfoundWaitPeriodMsDef;
+  this->reb_confirm_wait_period_ms_ = tb_config::kRebConfirmWaitPeriodMsDef;
+  this->max_confirm_wait_period_ms_ = tb_config::kConfirmWaitPeriodMsMax;
+  this->shutdown_reb_wait_period_ms_ = tb_config::kRebWaitPeriodWhenShutdownMs;
 }
 
 ConsumerConfig::~ConsumerConfig() {
@@ -278,7 +278,7 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, const string& group
        ++it) {
     topic_name = Utils::Trim(*it);
     is_success =
-        Utils::ValidString(err_info, topic_name, false, true, true, config::kTopicNameMaxLength);
+        Utils::ValidString(err_info, topic_name, false, true, true, tb_config::kTopicNameMaxLength);
     if (!is_success) {
       err_info = "Illegal parameter: subscribed_topicset's item error, " + err_info;
       return false;
@@ -336,7 +336,7 @@ bool ConsumerConfig::setGroupConsumeTarget(
     set<string> tgt_filters;
     // check topic_name info
     is_success =
-        Utils::ValidString(err_info, it_map->first, false, true, true, config::kTopicNameMaxLength);
+        Utils::ValidString(err_info, it_map->first, false, true, true, tb_config::kTopicNameMaxLength);
     if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
@@ -364,12 +364,12 @@ bool ConsumerConfig::setGroupConsumeTarget(
       tgt_filters.insert(tmp_filteritem);
       count++;
     }
-    if (count > config::kFilterItemMaxCount) {
+    if (count > tb_config::kFilterItemMaxCount) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
       ss << it_map->first;
       ss << "'s filter item over max item count : ";
-      ss << config::kFilterItemMaxCount;
+      ss << tb_config::kFilterItemMaxCount;
       err_info = ss.str();
       return false;
     }
@@ -385,13 +385,13 @@ bool ConsumerConfig::setGroupConsumeTarget(
   }
   // check session_key
   string tgt_session_key = Utils::Trim(session_key);
-  if (tgt_session_key.length() == 0 || tgt_session_key.length() > config::kSessionKeyMaxLength) {
+  if (tgt_session_key.length() == 0 || tgt_session_key.length() > tb_config::kSessionKeyMaxLength) {
     if (tgt_session_key.length() == 0) {
       err_info = "Illegal parameter: session_key is empty!";
     } else {
       stringstream ss;
       ss << "Illegal parameter: session_key's length over max length ";
-      ss << config::kSessionKeyMaxLength;
+      ss << tb_config::kSessionKeyMaxLength;
       err_info = ss.str();
     }
     return false;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 97cf806..346dc10 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -17,15 +17,15 @@
  * under the License.
  */
 
-#include "file_ini.h"
+#include "tubemq/file_ini.h"
 
 #include <stdlib.h>
 
 #include <fstream>
 #include <sstream>
 
-#include "const_config.h"
-#include "utils.h"
+#include "tubemq/const_config.h"
+#include "tubemq/utils.h"
 
 namespace tubemq {
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index 27caba8..f5c90a8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "flowctrl_def.h"
+#include "tubemq/flowctrl_def.h"
 
 #include <stdio.h>
 #include <time.h>
@@ -25,16 +25,16 @@
 
 #include <sstream>
 
-#include "const_config.h"
-#include "logger.h"
-#include "utils.h"
+#include "tubemq/const_config.h"
+#include "tubemq/logger.h"
+#include "tubemq/utils.h"
 
 namespace tubemq {
 
 using std::stringstream;
 
 FlowCtrlResult::FlowCtrlResult() {
-  this->datasize_limit_ = config::kMaxIntValue;
+  this->datasize_limit_ = tb_config::kMaxIntValue;
   this->freqms_limit_ = 0;
 }
 
@@ -68,19 +68,19 @@ int32_t FlowCtrlResult::GetFreqMsLimit() { return this->freqms_limit_; }
 FlowCtrlItem::FlowCtrlItem() {
   this->type_ = 0;
   this->start_time_ = 2500;
-  this->end_time_ = config::kInvalidValue;
-  this->datadlt_m_ = config::kInvalidValue;
-  this->datasize_limit_ = config::kInvalidValue;
-  this->freqms_limit_ = config::kInvalidValue;
-  this->zero_cnt_ = config::kInvalidValue;
+  this->end_time_ = tb_config::kInvalidValue;
+  this->datadlt_m_ = tb_config::kInvalidValue;
+  this->datasize_limit_ = tb_config::kInvalidValue;
+  this->freqms_limit_ = tb_config::kInvalidValue;
+  this->zero_cnt_ = tb_config::kInvalidValue;
 }
 
 FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit) {
   this->type_ = type;
   this->start_time_ = 2500;
-  this->end_time_ = config::kInvalidValue;
-  this->datadlt_m_ = config::kInvalidValue;
-  this->datasize_limit_ = config::kInvalidValue;
+  this->end_time_ = tb_config::kInvalidValue;
+  this->datadlt_m_ = tb_config::kInvalidValue;
+  this->datasize_limit_ = tb_config::kInvalidValue;
   this->freqms_limit_ = freqms_limit;
   this->zero_cnt_ = zero_cnt;
 }
@@ -89,8 +89,8 @@ FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_
                            int32_t min_data_filter_freqms) {
   this->type_ = type;
   this->start_time_ = 2500;
-  this->end_time_ = config::kInvalidValue;
-  this->datadlt_m_ = config::kInvalidValue;
+  this->end_time_ = tb_config::kInvalidValue;
+  this->datadlt_m_ = tb_config::kInvalidValue;
   this->datasize_limit_ = datasize_limit;
   this->freqms_limit_ = freqms_limit;
   this->zero_cnt_ = min_data_filter_freqms;
@@ -104,7 +104,7 @@ FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, i
   this->datadlt_m_ = datadlt_m;
   this->datasize_limit_ = datasize_limit;
   this->freqms_limit_ = freqms_limit;
-  this->zero_cnt_ = config::kInvalidValue;
+  this->zero_cnt_ = tb_config::kInvalidValue;
 }
 
 FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
@@ -119,7 +119,7 @@ FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
   return *this;
 }
 
-int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) {
+int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) const {
   if (this->type_ != 1) {
     return -1;
   }
@@ -133,8 +133,8 @@ void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int3
                                       int32_t min_data_filter_freqms) {
   this->type_ = type;
   this->start_time_ = 2500;
-  this->end_time_ = config::kInvalidValue;
-  this->datadlt_m_ = config::kInvalidValue;
+  this->end_time_ = tb_config::kInvalidValue;
+  this->datadlt_m_ = tb_config::kInvalidValue;
   this->datasize_limit_ = datasize_limit;
   this->freqms_limit_ = freqms_limit;
   this->zero_cnt_ = min_data_filter_freqms;
@@ -143,15 +143,15 @@ void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int3
 void FlowCtrlItem::Clear() {
   this->type_ = 0;
   this->start_time_ = 2500;
-  this->end_time_ = config::kInvalidValue;
-  this->datadlt_m_ = config::kInvalidValue;
-  this->datasize_limit_ = config::kInvalidValue;
-  this->freqms_limit_ = config::kInvalidValue;
-  this->zero_cnt_ = config::kInvalidValue;
+  this->end_time_ = tb_config::kInvalidValue;
+  this->datadlt_m_ = tb_config::kInvalidValue;
+  this->datasize_limit_ = tb_config::kInvalidValue;
+  this->freqms_limit_ = tb_config::kInvalidValue;
+  this->zero_cnt_ = tb_config::kInvalidValue;
 }
 
 bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m, int32_t curr_time,
-                                FlowCtrlResult& flowctrl_result) {
+                                FlowCtrlResult& flowctrl_result) const {
   if (this->type_ != 0 || datadlt_m <= this->datadlt_m_) {
     return false;
   }
@@ -163,13 +163,13 @@ bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m, int32_t curr_time,
 }
 
 FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
-  this->flowctrl_id_.GetAndSet(config::kInvalidValue);
+  this->flowctrl_id_.GetAndSet(tb_config::kInvalidValue);
   this->flowctrl_info_ = "";
-  this->min_zero_cnt_.Set(config::kMaxIntValue);
-  this->qrypriority_id_.Set(config::kInvalidValue);
-  this->min_datadlt_limt_.Set(config::kMaxLongValue);
+  this->min_zero_cnt_.Set(tb_config::kMaxIntValue);
+  this->qrypriority_id_.Set(tb_config::kInvalidValue);
+  this->min_datadlt_limt_.Set(tb_config::kMaxLongValue);
   this->datalimit_start_time_.Set(2500);
-  this->datalimit_end_time_.Set(config::kInvalidValue);
+  this->datalimit_end_time_.Set(tb_config::kInvalidValue);
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
   pthread_rwlock_init(&configrw_lock_, NULL);
 }
@@ -258,18 +258,18 @@ void FlowCtrlRuleHandler::initialStatisData() {
 }
 
 void FlowCtrlRuleHandler::clearStatisData() {
-  this->min_zero_cnt_.GetAndSet(config::kMaxIntValue);
-  this->min_datadlt_limt_.GetAndSet(config::kMaxLongValue);
-  this->qrypriority_id_.Set(config::kInvalidValue);
+  this->min_zero_cnt_.GetAndSet(tb_config::kMaxIntValue);
+  this->min_datadlt_limt_.GetAndSet(tb_config::kMaxLongValue);
+  this->qrypriority_id_.Set(tb_config::kInvalidValue);
   this->datalimit_start_time_.Set(2500);
-  this->datalimit_end_time_.Set(config::kInvalidValue);
+  this->datalimit_end_time_.Set(tb_config::kInvalidValue);
   this->filter_ctrl_item_.Clear();
 }
 
-bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) {
+bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const {
   struct tm utc_tm;
-  vector<FlowCtrlItem>::iterator it_vec;
-  map<int, vector<FlowCtrlItem> >::iterator it_map;
+  vector<FlowCtrlItem>::const_iterator it_vec;
+  map<int, vector<FlowCtrlItem> >::const_iterator it_map;
   time_t cur_time = time(NULL);
 
   gmtime_r(&cur_time, &utc_tm);
@@ -291,10 +291,10 @@ bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult&
   return false;
 }
 
-int FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) {
+int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const {
   int32_t rule_val = -2;
-  vector<FlowCtrlItem>::iterator it_vec;
-  map<int, vector<FlowCtrlItem> >::iterator it_map;
+  vector<FlowCtrlItem>::const_iterator it_vec;
+  map<int, vector<FlowCtrlItem> >::const_iterator it_map;
 
   if (msg_zero_cnt < this->min_zero_cnt_.Get()) {
     return received_limit;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
index 7360360..0fd2fbf 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "logger.h"
+#include "tubemq/logger.h"
 
 #include <log4cplus/fileappender.h>
 #include <log4cplus/layout.h>
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index d0afacf..9d117cd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-#include "message.h"
+#include "tubemq/message.h"
 
 #include <string.h>
 
 #include <sstream>
 
-#include "const_config.h"
-#include "utils.h"
+#include "tubemq/const_config.h"
+#include "tubemq/utils.h"
 
 namespace tubemq {
 
@@ -41,7 +41,7 @@ static const string kRsvPropKeyMsgTime = "$msgTime$";
 Message::Message() {
   this->topic_ = "";
   this->flag_ = 0;
-  this->message_id_ = config::kInvalidValue;
+  this->message_id_ = tb_config::kInvalidValue;
   this->data_ = NULL;
   this->datalen_ = 0;
   this->properties_.clear();
@@ -58,7 +58,7 @@ Message::Message(const Message& target) {
 Message::Message(const string& topic, const char* data, uint32_t datalen) {
   this->topic_ = topic;
   this->flag_ = 0;
-  this->message_id_ = config::kInvalidValue;
+  this->message_id_ = tb_config::kInvalidValue;
   copyData(data, datalen);
   this->properties_.clear();
 }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 81ecf79..5c4e66b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -17,16 +17,16 @@
  * under the License.
  */
 
-#include "meta_info.h"
+#include "tubemq/meta_info.h"
 
 #include <stdlib.h>
 
 #include <sstream>
 #include <vector>
 
-#include "const_config.h"
-#include "const_errcode.h"
-#include "utils.h"
+#include "tubemq/const_config.h"
+#include "tubemq/const_errcode.h"
+#include "tubemq/utils.h"
 
 namespace tubemq {
 
@@ -36,7 +36,7 @@ using std::vector;
 NodeInfo::NodeInfo() {
   this->node_id_ = 0;
   this->node_host_ = " ";
-  this->node_port_ = config::kBrokerPortDef;
+  this->node_port_ = tb_config::kBrokerPortDef;
   buildStrInfo();
 }
 
@@ -47,14 +47,14 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
   if (is_broker) {
     this->node_id_ = atoi(result[0].c_str());
     this->node_host_ = result[1];
-    this->node_port_ = config::kBrokerPortDef;
+    this->node_port_ = tb_config::kBrokerPortDef;
     if (result.size() >= 3) {
       this->node_port_ = atoi(result[2].c_str());
     }
   } else {
     this->node_id_ = 0;
     this->node_host_ = result[0];
-    this->node_port_ = config::kBrokerPortDef;
+    this->node_port_ = tb_config::kBrokerPortDef;
     if (result.size() >= 2) {
       this->node_port_ = atoi(result[1].c_str());
     }
@@ -63,7 +63,7 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
 }
 
 NodeInfo::NodeInfo(const string& node_host, uint32_t node_port) {
-  this->node_id_ = config::kInvalidValue;
+  this->node_id_ = tb_config::kInvalidValue;
   this->node_host_ = node_host;
   this->node_port_ = node_port;
   buildStrInfo();
@@ -314,9 +314,9 @@ void SubscribeInfo::buildSubInfo() {
 }
 
 ConsumerEvent::ConsumerEvent() {
-  this->rebalance_id_ = config::kInvalidValue;
-  this->event_type_ = config::kInvalidValue;
-  this->event_status_ = config::kInvalidValue;
+  this->rebalance_id_ = tb_config::kInvalidValue;
+  this->event_type_ = tb_config::kInvalidValue;
+  this->event_status_ = tb_config::kInvalidValue;
 }
 
 ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
@@ -401,10 +401,10 @@ PartitionExt::~PartitionExt() {
 }
 
 void PartitionExt::BookConsumeData(int32_t errcode, int32_t msg_size,
-  bool req_esc_limit, int64_t rsp_dlt_limit, long last_datadlt, bool require_slow) {
-  this->booked_time_ =Utils::GetCurrentTimeMillis();
+  bool req_esc_limit, int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow) {
+  this->booked_time_ = Utils::GetCurrentTimeMillis();
   this->booked_errcode_ = errcode;
-  this->booked_esc_limit_= req_esc_limit;
+  this->booked_esc_limit_ = req_esc_limit;
   this->booked_msgsize_ = msg_size;
   this->booked_dlt_limit_ = rsp_dlt_limit;
   this->booked_curdata_dlt_ = last_datadlt;
@@ -423,37 +423,11 @@ int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_
   const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
   int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
   int64_t last_datadlt, bool require_slow) {
-  bool result = false;
-  // Accumulated data received
-  this->_isLastConsumed = last_consumed;
-  this->cur_stage_msgsize_ += msg_size;
-  this->cur_slice_msgsize_ += msg_size;
+  // #lizard forgives
+  // record consume status
+  this->is_last_consumed_ = last_consumed;
   // Update strategy data values
-  int64_t curr_time = Utils::GetCurrentTimeMillis();
-  if (curr_time - this->next_stage_updtime_) {
-    this->cur_stage_msgsize_ = 0;
-    this->cur_slice_msgsize_ = 0;
-    if (last_datadlt >= 0) {
-      result = group_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
-      if (!result) {
-        result = def_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
-        if (!result) {
-          this->cur_flowctrl_.SetDataDltAndFreqLimit(config::kMaxLongValue, 0);
-        }
-      }
-      this->cur_freqctrl_ = group_flowctrl_handler.GetFilterCtrlItem();
-      if (this->cur_freqctrl_.getFreqLtInMs() < 0) {
-        this->cur_freqctrl_ = def_flowctrl_handler.GetFilterCtrlItem();
-      }
-      curr_time = Utils::GetCurrentTimeMillis();
-    }
-    this->limit_slice_msgsize_ = this->cur_flowctrl_.GetDataSizeLimit() / 12;
-    this->next_stage_updtime_ = curr_time + 60000;
-    this->next_slice_updtime_ = curr_time + 5000;
-  } else if(curr_time > this->next_slice_updtime_) {
-    this->cur_slice_msgsize_ = 0;
-    this->next_slice_updtime_ = curr_time + 5000;
-  }
+  updateStrategyData(def_flowctrl_handler, group_flowctrl_handler, msg_size, last_datadlt);
   // Perform different strategies based on error codes
   switch (errcode) {
     case err_code::kErrNotFound:
@@ -464,12 +438,12 @@ int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_
         this->total_zero_cnt_ = 0;
       }
       if (this->total_zero_cnt_ > 0) {
-        if (group_flowctrl_handler.GetMinZeroCnt() != config::kMaxIntValue) {
-          return (int64_t)group_flowctrl_handler.GetCurFreqLimitTime(
-            this->total_zero_cnt_, (int32_t)rsp_dlt_limit);
+        if (group_flowctrl_handler.GetMinZeroCnt() != tb_config::kMaxIntValue) {
+          return (int64_t)(group_flowctrl_handler.GetCurFreqLimitTime(
+            this->total_zero_cnt_, (int32_t)rsp_dlt_limit));
         } else {
           return (int64_t)def_flowctrl_handler.GetCurFreqLimitTime(
-            this->_totalRcvZeroCount, (int32_t)rsp_dlt_limit);
+            this->total_zero_cnt_, (int32_t)rsp_dlt_limit);
         }
       }
       if (req_esc_limit) {
@@ -510,7 +484,7 @@ bool PartitionExt::IsLastConsumed() {
 
 void PartitionExt::resetParameters() {
   this->is_last_consumed_ = false;
-  this->cur_flowctrl_.SetDataDltAndFreqLimit(config::kMaxLongValue, 20);
+  this->cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 20);
   this->next_stage_updtime_ = 0;
   this->next_slice_updtime_ = 0;
   this->limit_slice_msgsize_ = 0;
@@ -519,12 +493,46 @@ void PartitionExt::resetParameters() {
   this->total_zero_cnt_ = 0;
   this->booked_time_ = 0;
   this->booked_errcode_ = 0;
-  this->booked_esc_limit_= false;
+  this->booked_esc_limit_ = false;
   this->booked_msgsize_ = 0;
   this->booked_dlt_limit_ = 0;
   this->booked_curdata_dlt_ = 0;
   this->booked_require_slow_ = false;
 }
 
+void PartitionExt::updateStrategyData(const FlowCtrlRuleHandler& def_flowctrl_handler,
+  const FlowCtrlRuleHandler& group_flowctrl_handler, int32_t msg_size, int64_t last_datadlt) {
+  bool result = false;
+  // Accumulated data received
+  this->cur_stage_msgsize_ += msg_size;
+  this->cur_slice_msgsize_ += msg_size;  
+  int64_t curr_time = Utils::GetCurrentTimeMillis();
+  // Update strategy data values
+  if (curr_time > this->next_stage_updtime_) {
+    this->cur_stage_msgsize_ = 0;
+    this->cur_slice_msgsize_ = 0;
+    if (last_datadlt >= 0) {
+      result = group_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+      if (!result) {
+        result = def_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+        if (!result) {
+          this->cur_flowctrl_.SetDataDltAndFreqLimit(tb_config::kMaxLongValue, 0);
+        }
+      }
+      this->cur_freqctrl_ = group_flowctrl_handler.GetFilterCtrlItem();
+      if (this->cur_freqctrl_.GetFreqMsLimit() < 0) {
+        this->cur_freqctrl_ = def_flowctrl_handler.GetFilterCtrlItem();
+      }
+      curr_time = Utils::GetCurrentTimeMillis();
+    }
+    this->limit_slice_msgsize_ = this->cur_flowctrl_.GetDataSizeLimit() / 12;
+    this->next_stage_updtime_ = curr_time + 60000;
+    this->next_slice_updtime_ = curr_time + 5000;
+  } else if (curr_time > this->next_slice_updtime_) {
+    this->cur_slice_msgsize_ = 0;
+    this->next_slice_updtime_ = curr_time + 5000;
+  }
+}
+
 
 };  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index bbfba96..e48c8b6 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#include "utils.h"
+#include "tubemq/utils.h"
 
 #include <regex.h>
 #include <stdlib.h>
@@ -27,7 +27,7 @@
 #include <sstream>
 #include <vector>
 
-#include "const_config.h"
+#include "tubemq/const_config.h"
 
 namespace tubemq {
 
@@ -183,12 +183,12 @@ bool Utils::ValidGroupName(string& err_info, const string& group_name, string& t
     err_info = "Illegal parameter: group_name is blank!";
     return false;
   }
-  if (tgt_group_name.length() > config::kGroupNameMaxLength) {
+  if (tgt_group_name.length() > tb_config::kGroupNameMaxLength) {
     stringstream ss;
     ss << "Illegal parameter: ";
     ss << group_name;
     ss << " over max length, the max allowed length is ";
-    ss << config::kGroupNameMaxLength;
+    ss << tb_config::kGroupNameMaxLength;
     err_info = ss.str();
     return false;
   }
@@ -220,10 +220,10 @@ bool Utils::ValidFilterItem(string& err_info, const string& src_filteritem,
     return false;
   }
 
-  if (tgt_filteritem.length() > config::kFilterItemMaxLength) {
+  if (tgt_filteritem.length() > tb_config::kFilterItemMaxLength) {
     stringstream ss;
     ss << "value over max length ";
-    ss << config::kFilterItemMaxLength;
+    ss << tb_config::kFilterItemMaxLength;
     err_info = ss.str();
     return false;
   }


[incubator-tubemq] 23/50: [TUBEMQ-272]Unified C/C++ files's code style (#194)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit f4356b376da595a31cc217f013bc7bf61bbca231
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Wed Jul 8 15:41:01 2020 +0800

    [TUBEMQ-272]Unified C/C++ files's code style (#194)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/.clang-format                |  25 ++
 .../tubemq-client-cpp/inc/const_rpc.h              |  88 -------
 .../{inc => include/tubemq}/atomic_def.h           |  92 +++-----
 .../{inc => include/tubemq}/client_config.h        |  62 ++---
 .../{inc => include/tubemq}/const_config.h         |  33 ++-
 .../tubemq-client-cpp/include/tubemq/const_rpc.h   |  84 +++++++
 .../{inc => include/tubemq}/file_ini.h             |  14 +-
 .../{inc => include/tubemq}/flowctrl_def.h         | 125 ++++------
 .../{inc => include/tubemq}/logger.h               |  55 +++--
 .../{inc => include/tubemq}/message.h              |  10 +-
 .../{inc => include/tubemq}/meta_info.h            |  39 ++--
 .../{inc => include/tubemq}/noncopyable.h          |   0
 .../tubemq/rmt_data_cache.h}                       |  76 +++---
 .../{inc => include/tubemq}/singleton.h            |   8 +-
 .../{inc => include/tubemq}/unique_seq_id.h        |   3 +-
 .../{inc => include/tubemq}/utils.h                |  19 +-
 .../{inc => include/tubemq}/version.h              |   1 -
 .../tubemq-client-cpp/src/client_config.cc         | 164 ++++++-------
 .../tubemq-client-cpp/src/file_ini.cc              |  22 +-
 .../tubemq-client-cpp/src/flowctrl_def.cc          | 254 +++++++++------------
 .../tubemq-client-cpp/src/logger.cc                |  10 +-
 .../tubemq-client-cpp/src/message.cc               |  92 +++-----
 .../tubemq-client-cpp/src/meta_info.cc             | 162 +++++--------
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  43 ++--
 24 files changed, 628 insertions(+), 853 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/.clang-format b/tubemq-client-twins/tubemq-client-cpp/.clang-format
new file mode 100644
index 0000000..1adba9c
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/.clang-format
@@ -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.
+
+# Run manually to reformat a file:
+# clang-format -i --style=file <file>
+# find . -iname '*.cc' -o -iname '*.h' -o -iname '*.h.in' | xargs clang-format -i --style=file
+BasedOnStyle: Google
+ColumnLimit: 100
+IndentWidth: 2
+SortIncludes: true
+BreakBeforeBraces: Custom
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
deleted file mode 100644
index 2bc0f30..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_rpc.h
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#ifndef TUBEMQ_CLIENT_CONST_RPC_H_
-#define TUBEMQ_CLIENT_CONST_RPC_H_
-
-namespace tubemq {
-
-#include <stdint.h>
-
-
-namespace rpc_config {
-
-  // constant define
-  static const uint32_t kRpcPrtBeginToken    = 0xFF7FF4FE;
-  static const uint32_t kRpcMaxBufferSize    = 8192;
-  static const uint32_t kRpcMaxFrameListCnt  = (uint32_t) ((1024 * 1024 * 8) / kRpcMaxBufferSize);
-
-  // rpc protocol version
-  static const uint32_t kRpcProtocolVersion  = 2;
-
-  // msg type flag
-  static const int32_t  kRpcFlagMsgRequest   = 0x0;
-  static const int32_t  kRpcFlagMsgResponse  = 0x1;
-
-  // service type
-  static const int32_t kMasterService      = 1;
-  static const int32_t kBrokerReadService  = 2;
-  static const int32_t kBrokerWriteService = 3;
-  static const int32_t kBrokerAdminService = 4;
-  static const int32_t kMasterAdminService = 5;
-
-  // request method
-  // master rpc method
-  static const int32_t kMasterMethoddProducerRegister = 1;
-  static const int32_t kMasterMethoddProducerHeatbeat = 2;
-  static const int32_t kMasterMethoddProducerClose    = 3;
-  static const int32_t kMasterMethoddConsumerRegister = 4;
-  static const int32_t kMasterMethoddConsumerHeatbeat = 5;
-  static const int32_t kMasterMethoddConsumerClose    = 6;
-
-  // broker rpc method
-  static const int32_t kBrokerMethoddProducerRegister    = 11;
-  static const int32_t kBrokerMethoddProducerHeatbeat    = 12;
-  static const int32_t kBrokerMethoddProducerSendMsg     = 13;
-  static const int32_t kBrokerMethoddProducerClose       = 14;
-  static const int32_t kBrokerMethoddConsumerRegister    = 15;
-  static const int32_t kBrokerMethoddConsumerHeatbeat    = 16;
-  static const int32_t kBrokerMethoddConsumerGetMsg      = 17;
-  static const int32_t kBrokerMethoddConsumerCommit      = 18;
-  static const int32_t kBrokerMethoddConsumerClose       = 19;
-
-  // register operate type
-  static const int32_t kRegOpTypeRegister       = 31;
-  static const int32_t kRegOpTypeUnReg          = 32;
-
-  // rpc connect node timeout
-  static const int32_t kRpcConnectTimeoutMs    = 3000;
-
-  // rpc timeout define
-  static const int32_t kRpcTimoutDefSec = 15;
-  static const int32_t kRpcTimoutMaxSec = 300;
-  static const int32_t kRpcTimoutMinSec = 8;
-
-
-}  // namespace rpc_config
-
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_CONST_RPC_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
similarity index 77%
rename from tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
index 00d0c24..fefe325 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
@@ -22,35 +22,24 @@
 
 #include <stdlib.h>
 
-
 namespace tubemq {
 
 using namespace std;
 
 class AtomicInteger {
  public:
-  AtomicInteger() {
-    this->counter_ = 0;
-  }
+  AtomicInteger() { this->counter_ = 0; }
 
-  AtomicInteger(int initial_value) {
-    this->counter_ = initial_value;
-  }
+  AtomicInteger(int initial_value) { this->counter_ = initial_value; }
 
-  int Get() {
-    return this->counter_;
-  }
+  int Get() { return this->counter_; }
 
-  void Set(long new_value) {
-    this->counter_ = new_value;
-  }
+  void Set(long new_value) { this->counter_ = new_value; }
 
-  long LongValue() {
-    return (long)this->counter_;
-  }
+  long LongValue() { return (long)this->counter_; }
 
   int GetAndSet(int new_value) {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
         return current;
@@ -63,7 +52,7 @@ class AtomicInteger {
   }
 
   int GetAndIncrement() {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -73,7 +62,7 @@ class AtomicInteger {
   }
 
   int GetAndDecrement() {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -83,7 +72,7 @@ class AtomicInteger {
   }
 
   int GetAndAdd(int delta) {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -93,7 +82,7 @@ class AtomicInteger {
   }
 
   int IncrementAndGet() {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -103,7 +92,7 @@ class AtomicInteger {
   }
 
   int DecrementAndGet() {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -113,7 +102,7 @@ class AtomicInteger {
   }
 
   int AddAndGet(int delta) {
-    for ( ; ; ) {
+    for (;;) {
       int current = this->counter_;
       int next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -126,31 +115,20 @@ class AtomicInteger {
   volatile int counter_;
 };
 
-
 class AtomicLong {
  public:
-  AtomicLong() {
-    this->counter_ = 0;
-  }
+  AtomicLong() { this->counter_ = 0; }
 
-  AtomicLong(long initial_value) {
-    this->counter_ = initial_value;
-  }
+  AtomicLong(long initial_value) { this->counter_ = initial_value; }
 
-  long Get() {
-    return this->counter_;
-  }
+  long Get() { return this->counter_; }
 
-  void Set(long new_value) {
-    this->counter_ = new_value;
-  }
+  void Set(long new_value) { this->counter_ = new_value; }
 
-  long IntValue() {
-    return (int)this->counter_;
-  }
+  long IntValue() { return (int)this->counter_; }
 
   long GetAndSet(long new_value) {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       if (__sync_bool_compare_and_swap(&this->counter_, current, new_value)) {
         return current;
@@ -163,7 +141,7 @@ class AtomicLong {
   }
 
   long GetAndIncrement() {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -173,7 +151,7 @@ class AtomicLong {
   }
 
   long GetAndDecrement() {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -183,7 +161,7 @@ class AtomicLong {
   }
 
   long GetAndAdd(long delta) {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -193,7 +171,7 @@ class AtomicLong {
   }
 
   long IncrementAndGet() {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current + 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -203,7 +181,7 @@ class AtomicLong {
   }
 
   long DecrementAndGet() {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current - 1;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -213,7 +191,7 @@ class AtomicLong {
   }
 
   long AddAndGet(long delta) {
-    for ( ; ; ) {
+    for (;;) {
       long current = this->counter_;
       long next = current + delta;
       if (__sync_bool_compare_and_swap(&this->counter_, current, next)) {
@@ -226,24 +204,15 @@ class AtomicLong {
   volatile long counter_;
 };
 
-
-class AtomicBoolean{
+class AtomicBoolean {
  public:
-  AtomicBoolean() {
-    this->counter_ = 0;
-  }
+  AtomicBoolean() { this->counter_ = 0; }
 
-  AtomicBoolean(bool initial_value) {
-    this->counter_ = initial_value ? 1 : 0;
-  }
+  AtomicBoolean(bool initial_value) { this->counter_ = initial_value ? 1 : 0; }
 
-  bool Get() {
-    return this->counter_ != 0;
-  }
+  bool Get() { return this->counter_ != 0; }
 
-  void Set(bool new_value) {
-    this->counter_ = new_value ? 1 : 0;
-  }
+  void Set(bool new_value) { this->counter_ = new_value ? 1 : 0; }
 
   bool GetAndSet(bool new_value) {
     int u = new_value ? 1 : 0;
@@ -265,9 +234,6 @@ class AtomicBoolean{
   volatile int counter_;
 };
 
-
 }  // namespace tubemq
 
-
 #endif  // TUBEMQ_CLIENT_ATOMIC_DEF_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
similarity index 79%
rename from tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
index 41610e1..95bef67 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
@@ -22,31 +22,28 @@
 
 #include <stdint.h>
 #include <stdio.h>
+
 #include <map>
 #include <set>
 #include <string>
 
-
 namespace tubemq {
 
-
 using std::map;
 using std::set;
 using std::string;
 using std::vector;
 
-
-
 class BaseConfig {
  public:
   BaseConfig();
   ~BaseConfig();
   BaseConfig& operator=(const BaseConfig& target);
   bool SetMasterAddrInfo(string& err_info, const string& master_addrinfo);
-  bool SetTlsInfo(string& err_info, bool tls_enable,
-                    const string& trust_store_path, const string& trust_store_password);
-  bool SetAuthenticInfo(string& err_info, bool authentic_enable,
-                              const string& usr_name, const string& usr_password);
+  bool SetTlsInfo(string& err_info, bool tls_enable, const string& trust_store_path,
+                  const string& trust_store_password);
+  bool SetAuthenticInfo(string& err_info, bool authentic_enable, const string& usr_name,
+                        const string& usr_password);
   const string& GetMasterAddrInfo() const;
   bool IsTlsEnabled();
   const string& GetTrustStorePath() const;
@@ -69,11 +66,11 @@ class BaseConfig {
  private:
   string master_addrinfo_;
   // user authenticate
-  bool   auth_enable_;
+  bool auth_enable_;
   string auth_usrname_;
   string auth_usrpassword_;
   // TLS configuration
-  bool   tls_enabled_;
+  bool tls_enabled_;
   string tls_trust_store_path_;
   string tls_trust_store_password_;
   // other setting
@@ -83,28 +80,25 @@ class BaseConfig {
   int32_t heartbeat_period_afterfail_sec_;
 };
 
-
 enum ConsumePosition {
   kConsumeFromFirstOffset = -1,
   kConsumeFromLatestOffset = 0,
   kComsumeFromMaxOffsetAlways = 1
 };  // enum ConsumePosition
 
-
-
 class ConsumerConfig : public BaseConfig {
  public:
   ConsumerConfig();
   ~ConsumerConfig();
   ConsumerConfig& operator=(const ConsumerConfig& target);
-  bool SetGroupConsumeTarget(string& err_info,
-    const string& group_name, const set<string>& subscribed_topicset);
-  bool SetGroupConsumeTarget(string& err_info,
-    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map);
-  bool SetGroupConsumeTarget(string& err_info,
-    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-    const string& session_key, uint32_t source_count, bool is_select_big,
-    const map<string, int64_t>& part_offset_map);
+  bool SetGroupConsumeTarget(string& err_info, const string& group_name,
+                             const set<string>& subscribed_topicset);
+  bool SetGroupConsumeTarget(string& err_info, const string& group_name,
+                             const map<string, set<string> >& subscribed_topic_and_filter_map);
+  bool SetGroupConsumeTarget(string& err_info, const string& group_name,
+                             const map<string, set<string> >& subscribed_topic_and_filter_map,
+                             const string& session_key, uint32_t source_count, bool is_select_big,
+                             const map<string, int64_t>& part_offset_map);
   const string& GetGroupName() const;
   const map<string, set<string> >& GetSubTopicAndFilterMap() const;
   void SetConsumePosition(ConsumePosition consume_from_where);
@@ -126,10 +120,10 @@ class ConsumerConfig : public BaseConfig {
   string ToString();
 
  private:
-  bool setGroupConsumeTarget(string& err_info, bool is_bound_consume,
-    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-    const string& session_key, int32_t source_count, bool is_select_big,
-    const map<string, int64_t>& part_offset_map);
+  bool setGroupConsumeTarget(string& err_info, bool is_bound_consume, const string& group_name,
+                             const map<string, set<string> >& subscribed_topic_and_filter_map,
+                             const string& session_key, int32_t source_count, bool is_select_big,
+                             const map<string, int64_t>& part_offset_map);
 
  private:
   string group_name_;
@@ -149,24 +143,6 @@ class ConsumerConfig : public BaseConfig {
   int32_t shutdown_reb_wait_period_ms_;
 };
 
-
-
-
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_CONFIGURE_H_
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
similarity index 79%
rename from tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
index 9e72aaa..31a97ed 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
@@ -21,6 +21,7 @@
 #define TUBEMQ_CLIENT_CONST_CONFIG_H_
 
 #include <stdint.h>
+
 #include <map>
 #include <string>
 
@@ -28,7 +29,6 @@ namespace tubemq {
 
 using std::string;
 
-
 // configuration value setting
 namespace config {
 // heartbeat period define
@@ -75,27 +75,24 @@ static const int32_t kInvalidValue = -2;
 
 }  // namespace config
 
-
 namespace delimiter {
-  static const string kDelimiterDot = ".";
-  static const string kDelimiterEqual = "=";
-  static const string kDelimiterAnd   = "&";
-  static const string kDelimiterComma = ",";
-  static const string kDelimiterColon = ":";
-  static const string kDelimiterAt    = "@";
-  static const string kDelimiterPound = "#";
-  static const string kDelimiterSemicolon = ";";
-  // Double slash
-  static const string kDelimiterDbSlash = "//";
-  // left square bracket
-  static const string kDelimiterLftSB = "[";
-  // right square bracket
-  static const string kDelimiterRgtSB = "]";
+static const string kDelimiterDot = ".";
+static const string kDelimiterEqual = "=";
+static const string kDelimiterAnd = "&";
+static const string kDelimiterComma = ",";
+static const string kDelimiterColon = ":";
+static const string kDelimiterAt = "@";
+static const string kDelimiterPound = "#";
+static const string kDelimiterSemicolon = ";";
+// Double slash
+static const string kDelimiterDbSlash = "//";
+// left square bracket
+static const string kDelimiterLftSB = "[";
+// right square bracket
+static const string kDelimiterRgtSB = "]";
 
 }  // namespace delimiter
 
-
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_CONST_CONFIG_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.h
new file mode 100644
index 0000000..c402477
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_rpc.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 TUBEMQ_CLIENT_CONST_RPC_H_
+#define TUBEMQ_CLIENT_CONST_RPC_H_
+
+namespace tubemq {
+
+#include <stdint.h>
+
+namespace rpc_config {
+
+// constant define
+static const uint32_t kRpcPrtBeginToken = 0xFF7FF4FE;
+static const uint32_t kRpcMaxBufferSize = 8192;
+static const uint32_t kRpcMaxFrameListCnt = (uint32_t)((1024 * 1024 * 8) / kRpcMaxBufferSize);
+
+// rpc protocol version
+static const uint32_t kRpcProtocolVersion = 2;
+
+// msg type flag
+static const int32_t kRpcFlagMsgRequest = 0x0;
+static const int32_t kRpcFlagMsgResponse = 0x1;
+
+// service type
+static const int32_t kMasterService = 1;
+static const int32_t kBrokerReadService = 2;
+static const int32_t kBrokerWriteService = 3;
+static const int32_t kBrokerAdminService = 4;
+static const int32_t kMasterAdminService = 5;
+
+// request method
+// master rpc method
+static const int32_t kMasterMethoddProducerRegister = 1;
+static const int32_t kMasterMethoddProducerHeatbeat = 2;
+static const int32_t kMasterMethoddProducerClose = 3;
+static const int32_t kMasterMethoddConsumerRegister = 4;
+static const int32_t kMasterMethoddConsumerHeatbeat = 5;
+static const int32_t kMasterMethoddConsumerClose = 6;
+
+// broker rpc method
+static const int32_t kBrokerMethoddProducerRegister = 11;
+static const int32_t kBrokerMethoddProducerHeatbeat = 12;
+static const int32_t kBrokerMethoddProducerSendMsg = 13;
+static const int32_t kBrokerMethoddProducerClose = 14;
+static const int32_t kBrokerMethoddConsumerRegister = 15;
+static const int32_t kBrokerMethoddConsumerHeatbeat = 16;
+static const int32_t kBrokerMethoddConsumerGetMsg = 17;
+static const int32_t kBrokerMethoddConsumerCommit = 18;
+static const int32_t kBrokerMethoddConsumerClose = 19;
+
+// register operate type
+static const int32_t kRegOpTypeRegister = 31;
+static const int32_t kRegOpTypeUnReg = 32;
+
+// rpc connect node timeout
+static const int32_t kRpcConnectTimeoutMs = 3000;
+
+// rpc timeout define
+static const int32_t kRpcTimoutDefSec = 15;
+static const int32_t kRpcTimoutMaxSec = 300;
+static const int32_t kRpcTimoutMinSec = 8;
+
+}  // namespace rpc_config
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_CONST_RPC_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
similarity index 80%
rename from tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
index 39f9a36..2fb81e8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/file_ini.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
@@ -21,25 +21,24 @@
 #define TUBEMQ_CLIENT_FILE_INI_H_
 
 #include <stdint.h>
+
 #include <map>
 #include <string>
 
-
 namespace tubemq {
 
-using std::string;
 using std::map;
-
+using std::string;
 
 class Fileini {
  public:
   Fileini();
   ~Fileini();
   bool Loadini(string& err_info, const string& file_name);
-  bool GetValue(string& err_info, const string& sector,
-                   const string& key, string& value, const string& def);
-  bool GetValue(string& err_info, const string& sector,
-                   const string& key, int32_t& value, const int32_t def);
+  bool GetValue(string& err_info, const string& sector, const string& key, string& value,
+                const string& def);
+  bool GetValue(string& err_info, const string& sector, const string& key, int32_t& value,
+                const int32_t def);
 
  private:
   bool init_flag_;
@@ -50,4 +49,3 @@ class Fileini {
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_FILE_INI_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
similarity index 52%
rename from tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
index 5c39352..98d8be4 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
@@ -20,25 +20,23 @@
 #ifndef TUBEMQ_CLIENT_FLOW_CONTROL_H_
 #define TUBEMQ_CLIENT_FLOW_CONTROL_H_
 
+#include <rapidjson/document.h>
 #include <stdint.h>
+
 #include <algorithm>
 #include <list>
 #include <map>
 #include <string>
 #include <vector>
-#include "atomic_def.h"
-#include <rapidjson/document.h>
-
 
+#include "atomic_def.h"
 
 namespace tubemq {
 
-
 using std::map;
 using std::string;
 using std::vector;
 
-
 class FlowCtrlResult {
  public:
   FlowCtrlResult();
@@ -55,109 +53,84 @@ class FlowCtrlResult {
   int32_t freqms_limit_;
 };
 
-
 class FlowCtrlItem {
  public:
   FlowCtrlItem();
   FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit);
-  FlowCtrlItem(int32_t type, int32_t datasize_limit,
-    int32_t freqms_limit, int32_t min_data_filter_freqms);
-  FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time,
-    int64_t datadlt_m, int64_t datasize_limit, int32_t freqms_limit);
+  FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+               int32_t min_data_filter_freqms);
+  FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, int64_t datadlt_m,
+               int64_t datasize_limit, int32_t freqms_limit);
   FlowCtrlItem& operator=(const FlowCtrlItem& target);
   void Clear();
-  void ResetFlowCtrlValue(int32_t type,
-    int32_t datasize_limit, int32_t freqms_limit, int32_t min_data_filter_freqms);
+  void ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+                          int32_t min_data_filter_freqms);
   int32_t GetFreLimit(int32_t msg_zero_cnt);
   bool GetDataLimit(int64_t datadlt_m, int32_t curr_time, FlowCtrlResult& flowctrl_result);
-  const int32_t GetType() const {
-    return type_;
-  }
-  const int32_t GetZeroCnt() const {
-    return zero_cnt_;
-  }
-  const int32_t GetStartTime() const {
-    return start_time_;
-  }
-  const int32_t GetEndTime() const {
-    return end_time_;
-  }
-  const int64_t GetDataSizeLimit() const {
-    return datasize_limit_;
-  }
-  const int32_t GetFreqMsLimit() const {
-    return freqms_limit_;
-  }
-  const int64_t GetDltInM() const {
-    return datadlt_m_;
-  }
+  const int32_t GetType() const { return type_; }
+  const int32_t GetZeroCnt() const { return zero_cnt_; }
+  const int32_t GetStartTime() const { return start_time_; }
+  const int32_t GetEndTime() const { return end_time_; }
+  const int64_t GetDataSizeLimit() const { return datasize_limit_; }
+  const int32_t GetFreqMsLimit() const { return freqms_limit_; }
+  const int64_t GetDltInM() const { return datadlt_m_; }
 
  private:
-  int32_t  type_;
-  int32_t  start_time_;
-  int32_t  end_time_;
+  int32_t type_;
+  int32_t start_time_;
+  int32_t end_time_;
   int64_t datadlt_m_;
   int64_t datasize_limit_;
-  int32_t  freqms_limit_;
-  int32_t  zero_cnt_;
+  int32_t freqms_limit_;
+  int32_t zero_cnt_;
 };
 
 class FlowCtrlRuleHandler {
  public:
   FlowCtrlRuleHandler();
   ~FlowCtrlRuleHandler();
-  void UpdateDefFlowCtrlInfo(bool is_default,
-    int32_t qrypriority_id, int64_t flowctrl_id, const string& flowctrl_info);
+  void UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id, int64_t flowctrl_id,
+                             const string& flowctrl_info);
   bool GetCurDataLimit(int32_t last_datadlt, FlowCtrlResult& flowctrl_result);
   int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit);
-  int32_t GetMinZeroCnt() { return this->min_zero_cnt_.Get();}
-  int32_t GetQryPriorityId() {
-    return this->qrypriority_id_.Get();
-  }
-  void SetQryPriorityId(int32_t qrypriority_id) {
-    this->qrypriority_id_.Set(qrypriority_id);
-  }
-  int64_t GetFlowCtrlId() {
-    return this->flowctrl_id_.Get();
-  }
-  const FlowCtrlItem& GetFilterCtrlItem() const {
-    return this->filter_ctrl_item_;
-  }
-  const string& GetFlowCtrlInfo() const {
-    return this->flowctrl_info_;
-  }
+  int32_t GetMinZeroCnt() { return this->min_zero_cnt_.Get(); }
+  int32_t GetQryPriorityId() { return this->qrypriority_id_.Get(); }
+  void SetQryPriorityId(int32_t qrypriority_id) { this->qrypriority_id_.Set(qrypriority_id); }
+  int64_t GetFlowCtrlId() { return this->flowctrl_id_.Get(); }
+  const FlowCtrlItem& GetFilterCtrlItem() const { return this->filter_ctrl_item_; }
+  const string& GetFlowCtrlInfo() const { return this->flowctrl_info_; }
 
  private:
   void initialStatisData();
   void clearStatisData();
   static bool compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2);
   static bool compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2);
-  bool parseStringMember(string &err_info, const rapidjson::Value& root,
-    const char* key, string& value, bool compare_value, string required_val);
-  bool parseLongMember(string &err_info, const rapidjson::Value& root,
-    const char* key, int64_t& value, bool compare_value, int64_t required_val);
-  bool parseIntMember(string &err_info, const rapidjson::Value& root,
-    const char* key, int32_t& value, bool compare_value, int32_t required_val);
+  bool parseStringMember(string& err_info, const rapidjson::Value& root, const char* key,
+                         string& value, bool compare_value, string required_val);
+  bool parseLongMember(string& err_info, const rapidjson::Value& root, const char* key,
+                       int64_t& value, bool compare_value, int64_t required_val);
+  bool parseIntMember(string& err_info, const rapidjson::Value& root, const char* key,
+                      int32_t& value, bool compare_value, int32_t required_val);
   bool parseFlowCtrlInfo(const string& flowctrl_info,
-                      map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map);
-  bool parseDataLimit(string& err_info,
-            const rapidjson::Value& root, vector<FlowCtrlItem>& flowCtrlItems);
-  bool parseFreqLimit(string& err_info,
-            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
-  bool parseLowFetchLimit(string& err_info,
-            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
-  bool parseTimeMember(string& err_info,
-            const rapidjson::Value& root, const char* key, int32_t& value);
+                         map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map);
+  bool parseDataLimit(string& err_info, const rapidjson::Value& root,
+                      vector<FlowCtrlItem>& flowCtrlItems);
+  bool parseFreqLimit(string& err_info, const rapidjson::Value& root,
+                      vector<FlowCtrlItem>& flowctrl_items);
+  bool parseLowFetchLimit(string& err_info, const rapidjson::Value& root,
+                          vector<FlowCtrlItem>& flowctrl_items);
+  bool parseTimeMember(string& err_info, const rapidjson::Value& root, const char* key,
+                       int32_t& value);
 
  private:
-  AtomicLong    flowctrl_id_;
+  AtomicLong flowctrl_id_;
   AtomicInteger qrypriority_id_;
-  string        flowctrl_info_;
+  string flowctrl_info_;
   AtomicInteger min_zero_cnt_;
-  AtomicLong    min_datadlt_limt_;
+  AtomicLong min_datadlt_limt_;
   AtomicInteger datalimit_start_time_;
   AtomicInteger datalimit_end_time_;
-  FlowCtrlItem  filter_ctrl_item_;
+  FlowCtrlItem filter_ctrl_item_;
   map<int32_t, vector<FlowCtrlItem> > flowctrl_rules_;
   pthread_rwlock_t configrw_lock_;
   int64_t last_update_time_;
@@ -165,6 +138,4 @@ class FlowCtrlRuleHandler {
 
 }  // namespace tubemq
 
-
 #endif  // TUBEMQ_CLIENT_FLOW_CONTROL_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
similarity index 68%
rename from tubemq-client-twins/tubemq-client-cpp/inc/logger.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
index c2f2e7d..eb35075 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
@@ -20,34 +20,41 @@
 #ifndef _TUBEMQ_LOG_FILE_
 #define _TUBEMQ_LOG_FILE_
 
+#include <stdint.h>
+
 #include <string>
 #include <vector>
-#include <stdint.h>
 
 namespace tubemq {
 class Logger;
 
 Logger& GetLogger();
 
-#define LOG_LEVEL(level, fmt, ...)                                                                          \
-  {                                                                                                         \
-    if (tubemq::GetLogger().IsEnable(level)) {                                                              \
-      tubemq::GetLogger().Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), \
-                                ##__VA_ARGS__);                                                             \
-    }                                                                                                       \
+#define LOG_LEVEL(level, fmt, ...)                                                   \
+  {                                                                                  \
+    if (tubemq::GetLogger().IsEnable(level)) {                                       \
+      tubemq::GetLogger().Write("[%s:%d][%s]" fmt, __func__, __LINE__,               \
+                                tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
+    }                                                                                \
   }
 
-#define LOG_TRACE(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
-#define LOG_DEBUG(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
-#define LOG_INFO(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
-#define LOG_WARN(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
-#define LOG_ERROR(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
-
-#define LOG_TUBEMQ(logger, level, fmt, ...)                                                                      \
-  {                                                                                                            \
-    if (logger.IsEnable(level)) {                                                                              \
-      logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
-    }                                                                                                          \
+#define LOG_TRACE(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
+#define LOG_DEBUG(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
+#define LOG_INFO(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
+#define LOG_WARN(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
+#define LOG_ERROR(fmt, ...) \
+  LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
+
+#define LOG_TUBEMQ(logger, level, fmt, ...)                                                    \
+  {                                                                                            \
+    if (logger.IsEnable(level)) {                                                              \
+      logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), \
+                   ##__VA_ARGS__);                                                             \
+    }                                                                                          \
   }
 
 class Logger {
@@ -61,13 +68,21 @@ class Logger {
   };
 
   // size: MB
-  Logger() : file_max_size_(100), file_num_(10), level_(kError), base_path_("tubemq"), instance_("TubeMQ") { setup(); }
+  Logger()
+      : file_max_size_(100),
+        file_num_(10),
+        level_(kError),
+        base_path_("tubemq"),
+        instance_("TubeMQ") {
+    setup();
+  }
 
   ~Logger(void) {}
 
   // path example: ../log/tubemq
   // size: MB
-  bool Init(const std::string& path, Level level, uint32_t file_max_size = 100, uint32_t file_num = 10);
+  bool Init(const std::string& path, Level level, uint32_t file_max_size = 100,
+            uint32_t file_num = 10);
 
   bool Write(const char* sFormat, ...) __attribute__((format(printf, 2, 3)));
   inline bool WriteStream(const std::string& msg) { return writeStream(msg.c_str()); }
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/message.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
similarity index 94%
rename from tubemq-client-twins/tubemq-client-cpp/inc/message.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
index 5c8be2c..945b59e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/message.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
@@ -22,18 +22,16 @@
 
 #include <stdint.h>
 #include <stdio.h>
+
 #include <list>
 #include <map>
 #include <string>
 
-
 namespace tubemq {
 
 using std::map;
 using std::string;
 
-
-
 class Message {
  public:
   Message();
@@ -66,13 +64,11 @@ class Message {
   string topic_;
   char* data_;
   uint32_t datalen_;
-  int64_t  message_id_;
-  int32_t  flag_;
+  int64_t message_id_;
+  int32_t flag_;
   map<string, string> properties_;
 };
 
 }  // namespace tubemq
 
-
 #endif  // TUBEMQ_CLIENT_MESSAGE_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
similarity index 85%
rename from tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index dcc34b0..e99939d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -21,10 +21,11 @@
 #define TUBEMQ_CLIENT_META_INFO_H_
 
 #include <stdint.h>
+
 #include <list>
 #include <string>
-#include "flowctrl_def.h"
 
+#include "flowctrl_def.h"
 
 namespace tubemq {
 
@@ -32,7 +33,6 @@ using std::list;
 using std::map;
 using std::string;
 
-
 class NodeInfo {
  public:
   NodeInfo();
@@ -41,8 +41,8 @@ class NodeInfo {
   NodeInfo(int32_t node_id, const string& node_host, uint32_t node_port);
   ~NodeInfo();
   NodeInfo& operator=(const NodeInfo& target);
-  bool operator== (const NodeInfo& target);
-  bool operator< (const NodeInfo& target) const;
+  bool operator==(const NodeInfo& target);
+  bool operator<(const NodeInfo& target) const;
   const uint32_t GetNodeId() const;
   const string& GetHost() const;
   const uint32_t GetPort() const;
@@ -54,15 +54,14 @@ class NodeInfo {
 
  private:
   uint32_t node_id_;
-  string   node_host_;
+  string node_host_;
   uint32_t node_port_;
   // ip:port
-  string  addr_info_;
+  string addr_info_;
   // id:ip:port
-  string  node_info_;
+  string node_info_;
 };
 
-
 class Partition {
  public:
   Partition();
@@ -71,7 +70,7 @@ class Partition {
   Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id);
   ~Partition();
   Partition& operator=(const Partition& target);
-  bool operator== (const Partition& target);
+  bool operator==(const Partition& target);
   const uint32_t GetBrokerId() const;
   const string& GetBrokerHost() const;
   const uint32_t GetBrokerPort() const;
@@ -85,14 +84,13 @@ class Partition {
   void buildPartitionKey();
 
  private:
-  string   topic_;
+  string topic_;
   NodeInfo broker_info_;
   uint32_t partition_id_;
-  string   partition_key_;
-  string   partition_info_;
+  string partition_key_;
+  string partition_info_;
 };
 
-
 class SubscribeInfo {
  public:
   SubscribeInfo(const string& sub_info);
@@ -112,23 +110,22 @@ class SubscribeInfo {
   void buildSubInfo();
 
  private:
-  string    consumer_id_;
-  string    group_;
+  string consumer_id_;
+  string group_;
   Partition partition_;
-  string    sub_info_;
+  string sub_info_;
 };
 
-
 class ConsumerEvent {
  public:
   ConsumerEvent();
   ConsumerEvent(const ConsumerEvent& target);
   ConsumerEvent(int64_t rebalance_id, int32_t event_type,
-    const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status);
+                const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status);
   ConsumerEvent& operator=(const ConsumerEvent& target);
   const int64_t GetRebalanceId() const;
   const int32_t GetEventType() const;
-  const int32_t  GetEventStatus() const;
+  const int32_t GetEventStatus() const;
   void SetEventType(int32_t event_type);
   void SetEventStatus(int32_t event_status);
   const list<SubscribeInfo>& GetSubscribeInfoList() const;
@@ -142,12 +139,10 @@ class ConsumerEvent {
 };
 
 class PartitionExt : public Partition {
+ public:
   PartitionExt();
-  
-
 };
 
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_META_INFO_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/noncopyable.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/noncopyable.h
similarity index 100%
rename from tubemq-client-twins/tubemq-client-cpp/inc/noncopyable.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/noncopyable.h
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
similarity index 79%
copy from tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
copy to tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index d836be1..d97cbac 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -1,40 +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 TUBEMQ_UNIQUESEQID_H
-#define TUBEMQ_UNIQUESEQID_H
-
-#include <atomic>
-#include <stdint.h>
-
-namespace tubemq {
-
-class UniqueSeqId {
- public:
-  UniqueSeqId() : id(0) {}
-
-  uint32_t Next() { return id.fetch_add(1, std::memory_order_relaxed); }
-
- protected:
-  std::atomic<uint32_t> id;
-};
-
-}  // namespace tubemq
-
-#endif
+/**
+ * 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 TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
+#define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
+
+#include <stdint.h>
+
+#include <atomic>
+
+namespace tubemq {
+
+using namespace std;
+
+// consumer remote data cache
+class RmtDataCacheCsm {}
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
similarity index 96%
rename from tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
index 734f4fb..77eb414 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
@@ -20,11 +20,12 @@
 #ifndef _TUBEMQ_SINGLETON_H
 #define _TUBEMQ_SINGLETON_H
 
-#include <mutex>
-#include <thread>
 #include <assert.h>
 #include <stdlib.h>
 
+#include <mutex>
+#include <thread>
+
 #include "noncopyable.h"
 
 namespace tubemq {
@@ -50,9 +51,6 @@ class Singleton : noncopyable {
 };
 
 template <typename T>
-std::once_flag Singleton<T>::once_;
-
-template <typename T>
 T* Singleton<T>::value_ = nullptr;
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h
similarity index 99%
rename from tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h
index d836be1..bd04da0 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/unique_seq_id.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/unique_seq_id.h
@@ -20,9 +20,10 @@
 #ifndef TUBEMQ_UNIQUESEQID_H
 #define TUBEMQ_UNIQUESEQID_H
 
-#include <atomic>
 #include <stdint.h>
 
+#include <atomic>
+
 namespace tubemq {
 
 class UniqueSeqId {
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
similarity index 77%
rename from tubemq-client-twins/tubemq-client-cpp/inc/utils.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
index 707c1fd..b4e0aeb 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
@@ -21,18 +21,17 @@
 #define TUBEMQ_CLIENT_UTILS_H_
 
 #include <stdint.h>
+
 #include <map>
 #include <string>
 #include <vector>
 
-
 namespace tubemq {
 
 using std::map;
 using std::string;
 using std::vector;
 
-
 class Utils {
  public:
   // trim string info
@@ -40,15 +39,14 @@ class Utils {
   // split string to vector
   static void Split(const string& source, vector<string>& result, const string& delimiter);
   // split string to map<string, int>
-  static void Split(const string& source, map<string, int>& result,
-                   const string& delimiter_step1, const string& delimiter_step2);
+  static void Split(const string& source, map<string, int>& result, const string& delimiter_step1,
+                    const string& delimiter_step2);
   static void Join(const vector<string>& vec, const string& delimiter, string& target);
-  static bool ValidString(string& err_info, const string& source,
-                   bool allow_empty, bool pat_match, bool check_max_length, unsigned int maxlen);
-  static bool ValidGroupName(string &err_info,
-                   const string& group_name, string& tgt_group_name);
-  static bool ValidFilterItem(string& err_info,
-                   const string& src_filteritem, string& tgt_filteritem);
+  static bool ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
+                          bool check_max_length, unsigned int maxlen);
+  static bool ValidGroupName(string& err_info, const string& group_name, string& tgt_group_name);
+  static bool ValidFilterItem(string& err_info, const string& src_filteritem,
+                              string& tgt_filteritem);
   static string Int2str(int32_t data);
   static string Long2str(int64_t data);
   static uint32_t IpToInt(const string& ipv4_addr);
@@ -58,4 +56,3 @@ class Utils {
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_UTILS_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/version.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
similarity index 96%
rename from tubemq-client-twins/tubemq-client-cpp/inc/version.h
rename to tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
index 848a5c8..4aede43 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/version.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/version.h
@@ -31,4 +31,3 @@ static const string kTubeMQClientVersion = "0.1.0-0.5.0";
 }  // namespace tubemq
 
 #endif  // TUBEMQ_CLIENT_VERSION_H_
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index 3c88eb2..6c1f757 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -18,16 +18,16 @@
  */
 
 #include "client_config.h"
+
 #include <sstream>
 #include <vector>
+
 #include "const_config.h"
 #include "const_rpc.h"
 #include "utils.h"
 
-
 namespace tubemq {
 
-
 BaseConfig::BaseConfig() {
   this->master_addrinfo_ = "";
   this->auth_enable_ = false;
@@ -49,14 +49,14 @@ BaseConfig::~BaseConfig() {
 BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
   if (this != &target) {
     this->master_addrinfo_ = target.master_addrinfo_;
-    this->auth_enable_    = target.auth_enable_;
-    this->auth_usrname_   = target.auth_usrname_;
+    this->auth_enable_ = target.auth_enable_;
+    this->auth_usrname_ = target.auth_usrname_;
     this->auth_usrpassword_ = target.auth_usrpassword_;
-    this->tls_enabled_      = target.tls_enabled_;
-    this->tls_trust_store_path_      = target.tls_trust_store_path_;
-    this->tls_trust_store_password_  = target.tls_trust_store_password_;
-    this->rpc_read_timeout_sec_      = target.rpc_read_timeout_sec_;
-    this->heartbeat_period_sec_     = target.heartbeat_period_sec_;
+    this->tls_enabled_ = target.tls_enabled_;
+    this->tls_trust_store_path_ = target.tls_trust_store_path_;
+    this->tls_trust_store_password_ = target.tls_trust_store_password_;
+    this->rpc_read_timeout_sec_ = target.rpc_read_timeout_sec_;
+    this->heartbeat_period_sec_ = target.heartbeat_period_sec_;
     this->max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
     this->heartbeat_period_afterfail_sec_ = target.heartbeat_period_afterfail_sec_;
   }
@@ -81,8 +81,8 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
   // parse and verify master address info
   // master_addrinfo's format like ip1:port1,ip2:port2,ip3:port3
   map<string, int> tgt_address_map;
-  Utils::Split(master_addrinfo, tgt_address_map,
-    delimiter::kDelimiterComma, delimiter::kDelimiterColon);
+  Utils::Split(master_addrinfo, tgt_address_map, delimiter::kDelimiterComma,
+               delimiter::kDelimiterColon);
   if (tgt_address_map.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
@@ -92,8 +92,8 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
   return true;
 }
 
-bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
-                const string& trust_store_path, const string& trust_store_password) {
+bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable, const string& trust_store_path,
+                            const string& trust_store_password) {
   this->tls_enabled_ = tls_enable;
   if (tls_enable) {
     string trimed_trust_store_path = Utils::Trim(trust_store_path);
@@ -106,8 +106,8 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
       err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
     }
-      this->tls_trust_store_path_ = trimed_trust_store_path;
-      this->tls_trust_store_password_ = trimed_trust_store_password;
+    this->tls_trust_store_path_ = trimed_trust_store_path;
+    this->tls_trust_store_password_ = trimed_trust_store_password;
   } else {
     this->tls_trust_store_path_ = "";
     this->tls_trust_store_password_ = "";
@@ -116,8 +116,8 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
   return true;
 }
 
-bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable,
-                const string& usr_name, const string& usr_password) {
+bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, const string& usr_name,
+                                  const string& usr_password) {
   this->auth_enable_ = authentic_enable;
   if (authentic_enable) {
     string trimed_usr_name = Utils::Trim(usr_name);
@@ -140,33 +140,19 @@ bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable,
   return true;
 }
 
-const string& BaseConfig::GetMasterAddrInfo() const {
-    return this->master_addrinfo_;
-}
+const string& BaseConfig::GetMasterAddrInfo() const { return this->master_addrinfo_; }
 
-bool BaseConfig::IsTlsEnabled() {
-  return this->tls_enabled_;
-}
+bool BaseConfig::IsTlsEnabled() { return this->tls_enabled_; }
 
-const string& BaseConfig::GetTrustStorePath() const {
-  return this->tls_trust_store_path_;
-}
+const string& BaseConfig::GetTrustStorePath() const { return this->tls_trust_store_path_; }
 
-const string& BaseConfig::GetTrustStorePassword() const {
-  return this->tls_trust_store_password_;
-}
+const string& BaseConfig::GetTrustStorePassword() const { return this->tls_trust_store_password_; }
 
-bool BaseConfig::IsAuthenticEnabled() {
-  return this->auth_enable_;
-}
+bool BaseConfig::IsAuthenticEnabled() { return this->auth_enable_; }
 
-const string& BaseConfig::GetUsrName() const {
-  return this->auth_usrname_;
-}
+const string& BaseConfig::GetUsrName() const { return this->auth_usrname_; }
 
-const string& BaseConfig::GetUsrPassWord() const {
-  return this->auth_usrpassword_;
-}
+const string& BaseConfig::GetUsrPassWord() const { return this->auth_usrpassword_; }
 
 void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
   if (rpc_read_timeout_sec >= rpc_config::kRpcTimoutMaxSec) {
@@ -178,33 +164,25 @@ void BaseConfig::SetRpcReadTimeoutSec(int rpc_read_timeout_sec) {
   }
 }
 
-int32_t BaseConfig::GetRpcReadTimeoutSec() {
-  return this->rpc_read_timeout_sec_;
-}
+int32_t BaseConfig::GetRpcReadTimeoutSec() { return this->rpc_read_timeout_sec_; }
 
 void BaseConfig::SetHeartbeatPeriodSec(int32_t heartbeat_period_sec) {
   this->heartbeat_period_sec_ = heartbeat_period_sec;
 }
 
-int32_t BaseConfig::GetHeartbeatPeriodSec() {
-  return this->heartbeat_period_sec_;
-}
+int32_t BaseConfig::GetHeartbeatPeriodSec() { return this->heartbeat_period_sec_; }
 
 void BaseConfig::SetMaxHeartBeatRetryTimes(int32_t max_heartbeat_retry_times) {
   this->max_heartbeat_retry_times_ = max_heartbeat_retry_times;
 }
 
-int32_t BaseConfig::GetMaxHeartBeatRetryTimes() {
-  return this->max_heartbeat_retry_times_;
-}
+int32_t BaseConfig::GetMaxHeartBeatRetryTimes() { return this->max_heartbeat_retry_times_; }
 
 void BaseConfig::SetHeartbeatPeriodAftFailSec(int32_t heartbeat_period_afterfail_sec) {
   this->heartbeat_period_afterfail_sec_ = heartbeat_period_afterfail_sec;
 }
 
-int32_t BaseConfig::GetHeartbeatPeriodAftFailSec() {
-  return this->heartbeat_period_afterfail_sec_;
-}
+int32_t BaseConfig::GetHeartbeatPeriodAftFailSec() { return this->heartbeat_period_afterfail_sec_; }
 
 string BaseConfig::ToString() {
   stringstream ss;
@@ -257,7 +235,7 @@ ConsumerConfig::~ConsumerConfig() {
 ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
   if (this != &target) {
     // parent class
-    BaseConfig::operator =(target);
+    BaseConfig::operator=(target);
     // child class
     this->group_name_ = target.group_name_;
     this->sub_topic_and_filter_map_ = target.sub_topic_and_filter_map_;
@@ -278,8 +256,8 @@ ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
   return *this;
 }
 
-bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
-                        const string& group_name, const set<string>& subscribed_topicset) {
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, const string& group_name,
+                                           const set<string>& subscribed_topicset) {
   string tgt_group_name;
   bool is_success = Utils::ValidGroupName(err_info, group_name, tgt_group_name);
   if (!is_success) {
@@ -291,11 +269,11 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
   }
   string topic_name;
   map<string, set<string> > tmp_sub_map;
-  for (set<string>::iterator it = subscribed_topicset.begin();
-          it != subscribed_topicset.end(); ++it) {
+  for (set<string>::iterator it = subscribed_topicset.begin(); it != subscribed_topicset.end();
+       ++it) {
     topic_name = Utils::Trim(*it);
-    is_success = Utils::ValidString(err_info, topic_name,
-                         false, true, true, config::kTopicNameMaxLength);
+    is_success =
+        Utils::ValidString(err_info, topic_name, false, true, true, config::kTopicNameMaxLength);
     if (!is_success) {
       err_info = "Illegal parameter: subscribed_topicset's item error, " + err_info;
       return false;
@@ -310,30 +288,29 @@ bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
   return true;
 }
 
-bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
-  const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map) {
+bool ConsumerConfig::SetGroupConsumeTarget(
+    string& err_info, const string& group_name,
+    const map<string, set<string> >& subscribed_topic_and_filter_map) {
   string session_key;
   int source_count = 0;
   bool is_select_big = false;
   map<string, int64_t> part_offset_map;
-  return setGroupConsumeTarget(err_info, false,
-              group_name, subscribed_topic_and_filter_map,
-              session_key, source_count, is_select_big, part_offset_map);
+  return setGroupConsumeTarget(err_info, false, group_name, subscribed_topic_and_filter_map,
+                               session_key, source_count, is_select_big, part_offset_map);
 }
 
-bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
-      const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-      const string& session_key, uint32_t source_count, bool is_select_big,
-      const map<string, int64_t>& part_offset_map) {
-  return setGroupConsumeTarget(err_info, true,
-              group_name, subscribed_topic_and_filter_map,
-              session_key, source_count, is_select_big, part_offset_map);
+bool ConsumerConfig::SetGroupConsumeTarget(
+    string& err_info, const string& group_name,
+    const map<string, set<string> >& subscribed_topic_and_filter_map, const string& session_key,
+    uint32_t source_count, bool is_select_big, const map<string, int64_t>& part_offset_map) {
+  return setGroupConsumeTarget(err_info, true, group_name, subscribed_topic_and_filter_map,
+                               session_key, source_count, is_select_big, part_offset_map);
 }
 
-bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consume,
-      const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
-      const string& session_key, uint32_t source_count, bool is_select_big,
-      const map<string, int64_t>& part_offset_map) {
+bool ConsumerConfig::setGroupConsumeTarget(
+    string& err_info, bool is_bound_consume, const string& group_name,
+    const map<string, set<string> >& subscribed_topic_and_filter_map, const string& session_key,
+    uint32_t source_count, bool is_select_big, const map<string, int64_t>& part_offset_map) {
   // check parameter group_name
   string tgt_group_name;
   bool is_success = Utils::ValidGroupName(err_info, group_name, tgt_group_name);
@@ -348,13 +325,13 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   map<string, set<string> > tmp_sub_map;
   map<string, set<string> >::const_iterator it_map;
   for (it_map = subscribed_topic_and_filter_map.begin();
-              it_map != subscribed_topic_and_filter_map.end(); ++it_map) {
+       it_map != subscribed_topic_and_filter_map.end(); ++it_map) {
     uint32_t count = 0;
     string tmp_filteritem;
     set<string> tgt_filters;
     // check topic_name info
-    is_success = Utils::ValidString(err_info, it_map->first,
-                          false, true, true, config::kTopicNameMaxLength);
+    is_success =
+        Utils::ValidString(err_info, it_map->first, false, true, true, config::kTopicNameMaxLength);
     if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
@@ -367,8 +344,8 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
     string topic_name = Utils::Trim(it_map->first);
     // check filter info
     set<string> subscribed_filters = it_map->second;
-    for (set<string>::iterator it = subscribed_filters.begin();
-          it != subscribed_filters.end(); ++it) {
+    for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end();
+         ++it) {
       is_success = Utils::ValidFilterItem(err_info, *it, tmp_filteritem);
       if (!is_success) {
         stringstream ss;
@@ -403,8 +380,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   }
   // check session_key
   string tgt_session_key = Utils::Trim(session_key);
-  if (tgt_session_key.length() == 0
-    || tgt_session_key.length() > config::kSessionKeyMaxLength) {
+  if (tgt_session_key.length() == 0 || tgt_session_key.length() > config::kSessionKeyMaxLength) {
     if (tgt_session_key.length() == 0) {
       err_info = "Illegal parameter: session_key is empty!";
     } else {
@@ -479,9 +455,7 @@ bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consu
   return true;
 }
 
-const string& ConsumerConfig::GetGroupName() const {
-  return this->group_name_;
-}
+const string& ConsumerConfig::GetGroupName() const { return this->group_name_; }
 
 const map<string, set<string> >& ConsumerConfig::GetSubTopicAndFilterMap() const {
   return this->sub_topic_and_filter_map_;
@@ -491,9 +465,7 @@ void ConsumerConfig::SetConsumePosition(ConsumePosition consume_from_where) {
   this->consume_position_ = consume_from_where;
 }
 
-const ConsumePosition ConsumerConfig::GetConsumePosition() const {
-  return this->consume_position_;
-}
+const ConsumePosition ConsumerConfig::GetConsumePosition() const { return this->consume_position_; }
 
 const int ConsumerConfig::GetMsgNotFoundWaitPeriodMs() const {
   return this->msg_notfound_wait_period_ms_;
@@ -511,17 +483,13 @@ void ConsumerConfig::SetMaxSubinfoReportIntvl(int max_subinfo_report_intvl) {
   this->max_subinfo_report_intvl_ = max_subinfo_report_intvl;
 }
 
-bool ConsumerConfig::IsConfirmInLocal() {
-  return this->is_confirm_in_local_;
-}
+bool ConsumerConfig::IsConfirmInLocal() { return this->is_confirm_in_local_; }
 
 void ConsumerConfig::SetConfirmInLocal(bool confirm_in_local) {
   this->is_confirm_in_local_ = confirm_in_local;
 }
 
-bool ConsumerConfig::IsRollbackIfConfirmTimeout() {
-  return this->is_rollback_if_confirm_timout_;
-}
+bool ConsumerConfig::IsRollbackIfConfirmTimeout() { return this->is_rollback_if_confirm_timout_; }
 
 void ConsumerConfig::setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout) {
   this->is_rollback_if_confirm_timout_ = is_rollback_if_confirm_timeout;
@@ -529,15 +497,13 @@ void ConsumerConfig::setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_tim
 
 const int ConsumerConfig::GetWaitPeriodIfConfirmWaitRebalanceMs() const {
   return this->reb_confirm_wait_period_ms_;
-}
+}
 
 void ConsumerConfig::SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_period_ms) {
   this->reb_confirm_wait_period_ms_ = reb_confirm_wait_period_ms;
 }
 
-const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const {
-  this->max_confirm_wait_period_ms_;
-}
+const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const { this->max_confirm_wait_period_ms_; }
 
 void ConsumerConfig::SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms) {
   this->max_confirm_wait_period_ms_ = max_confirm_wait_period_ms;
@@ -564,7 +530,7 @@ string ConsumerConfig::ToString() {
   ss << this->group_name_;
   ss << "', sub_topic_and_filter_map_={";
   for (it_map = this->sub_topic_and_filter_map_.begin();
-              it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
+       it_map != this->sub_topic_and_filter_map_.end(); ++it_map) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -593,8 +559,7 @@ string ConsumerConfig::ToString() {
   ss << this->is_select_big_;
   ss << ", part_offset_map_={";
   i = 0;
-  for (it = this->part_offset_map_.begin();
-              it != this->part_offset_map_.end(); ++it) {
+  for (it = this->part_offset_map_.begin(); it != this->part_offset_map_.end(); ++it) {
     if (i++ > 0) {
       ss << ",";
     }
@@ -624,4 +589,3 @@ string ConsumerConfig::ToString() {
 }
 
 }  // namespace tubemq
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index 6400359..fe8edcd 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -18,16 +18,17 @@
  */
 
 #include "file_ini.h"
+
 #include <stdlib.h>
+
 #include <fstream>
 #include <sstream>
+
 #include "const_config.h"
 #include "utils.h"
 
-
 namespace tubemq {
 
-
 Fileini::Fileini() {
   this->init_flag_ = false;
   this->ini_map_.clear();
@@ -61,9 +62,8 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   while (getline(conf_file, line_str)) {
     // check if a comment
     line_str = Utils::Trim(line_str);
-    if (line_str.empty()
-      || line_str.find(delimiter::kDelimiterDbSlash) == 0
-      || line_str.find(delimiter::kDelimiterSemicolon) == 0) {
+    if (line_str.empty() || line_str.find(delimiter::kDelimiterDbSlash) == 0 ||
+        line_str.find(delimiter::kDelimiterSemicolon) == 0) {
       continue;
     }
     // check if a sector head
@@ -71,7 +71,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
     rgtsb_pos = line_str.find(delimiter::kDelimiterRgtSB);
     if (lftsb_pos != string::npos && rgtsb_pos != string::npos) {
       sector = line_str.substr(lftsb_pos + (delimiter::kDelimiterLftSB).size(),
-                      rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
+                               rgtsb_pos - (delimiter::kDelimiterRgtSB).size());
       sector = Utils::Trim(sector);
       continue;
     }
@@ -108,8 +108,8 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
   return true;
 }
 
-bool Fileini::GetValue(string& err_info, const string& sector,
-                const string& key, string& value, const string& def) {
+bool Fileini::GetValue(string& err_info, const string& sector, const string& key, string& value,
+                       const string& def) {
   if (!this->init_flag_) {
     err_info = "Please load configure file first!";
     return false;
@@ -133,8 +133,8 @@ bool Fileini::GetValue(string& err_info, const string& sector,
   return true;
 }
 
-bool Fileini::GetValue(string& err_info, const string& sector,
-                const string& key, int32_t& value, const int32_t def) {
+bool Fileini::GetValue(string& err_info, const string& sector, const string& key, int32_t& value,
+                       const int32_t def) {
   string val_str;
   string def_str = Utils::Int2str(def);
   bool result = GetValue(err_info, sector, key, val_str, def_str);
@@ -146,5 +146,3 @@ bool Fileini::GetValue(string& err_info, const string& sector,
 }
 
 }  // namespace tubemq
-
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index e59b606..1975306 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -18,19 +18,19 @@
  */
 
 #include "flowctrl_def.h"
+
 #include <stdio.h>
 #include <time.h>
 #include <unistd.h>
+
 #include <sstream>
+
 #include "const_config.h"
 #include "logger.h"
 #include "utils.h"
 
-
-
 namespace tubemq {
 
-
 FlowCtrlResult::FlowCtrlResult() {
   this->datasize_limit_ = config::kMaxIntValue;
   this->freqms_limit_ = 0;
@@ -42,8 +42,7 @@ FlowCtrlResult::FlowCtrlResult(int64_t datasize_limit, int32_t freqms_limit) {
 }
 
 FlowCtrlResult& FlowCtrlResult::operator=(const FlowCtrlResult& target) {
-  if (this == &target)
-    return *this;
+  if (this == &target) return *this;
   this->datasize_limit_ = target.datasize_limit_;
   this->freqms_limit_ = target.freqms_limit_;
   return *this;
@@ -58,73 +57,64 @@ void FlowCtrlResult::SetDataSizeLimit(int64_t datasize_limit) {
   this->datasize_limit_ = datasize_limit;
 }
 
-void FlowCtrlResult::SetFreqMsLimit(int32_t freqms_limit) {
-  this->freqms_limit_ = freqms_limit;
-}
-
-int64_t FlowCtrlResult::GetDataSizeLimit() {
-  return this->datasize_limit_;
-}
+void FlowCtrlResult::SetFreqMsLimit(int32_t freqms_limit) { this->freqms_limit_ = freqms_limit; }
 
-int32_t FlowCtrlResult::GetFreqMsLimit() {
-  return this->freqms_limit_;
-}
+int64_t FlowCtrlResult::GetDataSizeLimit() { return this->datasize_limit_; }
 
+int32_t FlowCtrlResult::GetFreqMsLimit() { return this->freqms_limit_; }
 
 FlowCtrlItem::FlowCtrlItem() {
-  this->type_           = 0;
-  this->start_time_     = 2500;
-  this->end_time_       = config::kInvalidValue;
-  this->datadlt_m_      = config::kInvalidValue;
+  this->type_ = 0;
+  this->start_time_ = 2500;
+  this->end_time_ = config::kInvalidValue;
+  this->datadlt_m_ = config::kInvalidValue;
   this->datasize_limit_ = config::kInvalidValue;
-  this->freqms_limit_   = config::kInvalidValue;
-  this->zero_cnt_       = config::kInvalidValue;
+  this->freqms_limit_ = config::kInvalidValue;
+  this->zero_cnt_ = config::kInvalidValue;
 }
 
-FlowCtrlItem::FlowCtrlItem(int32_t type,
-                       int32_t zero_cnt, int32_t freqms_limit) {
-  this->type_           = type;
-  this->start_time_     = 2500;
-  this->end_time_       = config::kInvalidValue;
-  this->datadlt_m_      = config::kInvalidValue;
+FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t zero_cnt, int32_t freqms_limit) {
+  this->type_ = type;
+  this->start_time_ = 2500;
+  this->end_time_ = config::kInvalidValue;
+  this->datadlt_m_ = config::kInvalidValue;
   this->datasize_limit_ = config::kInvalidValue;
-  this->freqms_limit_   = freqms_limit;
-  this->zero_cnt_       = zero_cnt;
+  this->freqms_limit_ = freqms_limit;
+  this->zero_cnt_ = zero_cnt;
 }
 
-FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t datasize_limit,
-                      int32_t freqms_limit, int32_t min_data_filter_freqms) {
-  this->type_           = type;
-  this->start_time_     = 2500;
-  this->end_time_       = config::kInvalidValue;
-  this->datadlt_m_      = config::kInvalidValue;
+FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+                           int32_t min_data_filter_freqms) {
+  this->type_ = type;
+  this->start_time_ = 2500;
+  this->end_time_ = config::kInvalidValue;
+  this->datadlt_m_ = config::kInvalidValue;
   this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_   = freqms_limit;
-  this->zero_cnt_       = min_data_filter_freqms;
+  this->freqms_limit_ = freqms_limit;
+  this->zero_cnt_ = min_data_filter_freqms;
 }
 
-FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time,
-                      int64_t datadlt_m, int64_t datasize_limit, int32_t freqms_limit) {
-  this->type_           = type;
-  this->start_time_     = start_time;
-  this->end_time_       = end_time;
-  this->datadlt_m_      = datadlt_m;
+FlowCtrlItem::FlowCtrlItem(int32_t type, int32_t start_time, int32_t end_time, int64_t datadlt_m,
+                           int64_t datasize_limit, int32_t freqms_limit) {
+  this->type_ = type;
+  this->start_time_ = start_time;
+  this->end_time_ = end_time;
+  this->datadlt_m_ = datadlt_m;
   this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_   = freqms_limit;
-  this->zero_cnt_       = config::kInvalidValue;
+  this->freqms_limit_ = freqms_limit;
+  this->zero_cnt_ = config::kInvalidValue;
 }
 
 FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
-  if (this == &target)
-    return *this;
-  this->type_           = target.type_;
-  this->start_time_     = target.start_time_;
-  this->end_time_       = target.end_time_;
-  this->datadlt_m_      = target.datadlt_m_;
+  if (this == &target) return *this;
+  this->type_ = target.type_;
+  this->start_time_ = target.start_time_;
+  this->end_time_ = target.end_time_;
+  this->datadlt_m_ = target.datadlt_m_;
   this->datasize_limit_ = target.datasize_limit_;
-  this->freqms_limit_   = target.freqms_limit_;
-  this->zero_cnt_       = target.zero_cnt_;
-    return *this;
+  this->freqms_limit_ = target.freqms_limit_;
+  this->zero_cnt_ = target.zero_cnt_;
+  return *this;
 }
 
 int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) {
@@ -137,41 +127,39 @@ int32_t FlowCtrlItem::GetFreLimit(int32_t msg_zero_cnt) {
   return -1;
 }
 
-void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit,
-                                  int32_t freqms_limit, int32_t min_data_filter_freqms) {
-  this->type_           = type;
-  this->start_time_     = 2500;
-  this->end_time_       = config::kInvalidValue;
-  this->datadlt_m_      = config::kInvalidValue;
+void FlowCtrlItem::ResetFlowCtrlValue(int32_t type, int32_t datasize_limit, int32_t freqms_limit,
+                                      int32_t min_data_filter_freqms) {
+  this->type_ = type;
+  this->start_time_ = 2500;
+  this->end_time_ = config::kInvalidValue;
+  this->datadlt_m_ = config::kInvalidValue;
   this->datasize_limit_ = datasize_limit;
-  this->freqms_limit_   = freqms_limit;
-  this->zero_cnt_       = min_data_filter_freqms;
+  this->freqms_limit_ = freqms_limit;
+  this->zero_cnt_ = min_data_filter_freqms;
 }
 
 void FlowCtrlItem::Clear() {
-  this->type_           = 0;
-  this->start_time_     = 2500;
-  this->end_time_       = config::kInvalidValue;
-  this->datadlt_m_      = config::kInvalidValue;
+  this->type_ = 0;
+  this->start_time_ = 2500;
+  this->end_time_ = config::kInvalidValue;
+  this->datadlt_m_ = config::kInvalidValue;
   this->datasize_limit_ = config::kInvalidValue;
-  this->freqms_limit_   = config::kInvalidValue;
-  this->zero_cnt_       = config::kInvalidValue;
+  this->freqms_limit_ = config::kInvalidValue;
+  this->zero_cnt_ = config::kInvalidValue;
 }
 
-bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m,
-                      int32_t curr_time, FlowCtrlResult& flowctrl_result) {
+bool FlowCtrlItem::GetDataLimit(int64_t datadlt_m, int32_t curr_time,
+                                FlowCtrlResult& flowctrl_result) {
   if (this->type_ != 0 || datadlt_m <= this->datadlt_m_) {
     return false;
   }
-  if (curr_time < this->start_time_
-    || curr_time > this->end_time_) {
+  if (curr_time < this->start_time_ || curr_time > this->end_time_) {
     return false;
   }
   flowctrl_result.SetDataDltAndFreqLimit(this->datasize_limit_, this->freqms_limit_);
   return true;
 }
 
-
 FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
   this->flowctrl_id_.GetAndSet(config::kInvalidValue);
   this->flowctrl_info_ = "";
@@ -184,16 +172,14 @@ FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
   pthread_rwlock_init(&configrw_lock_, NULL);
 }
 
-FlowCtrlRuleHandler::~FlowCtrlRuleHandler() {
-  pthread_rwlock_destroy(&configrw_lock_);
-}
+FlowCtrlRuleHandler::~FlowCtrlRuleHandler() { pthread_rwlock_destroy(&configrw_lock_); }
 
-void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default,
-                  int32_t qrypriority_id, int64_t flowctrl_id, const string& flowctrl_info) {
+void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id,
+                                                int64_t flowctrl_id, const string& flowctrl_info) {
   bool result;
   map<int32_t, vector<FlowCtrlItem> > tmp_flowctrl_map;
   if (flowctrl_id == this->flowctrl_id_.Get()) {
-      return;
+    return;
   }
   int64_t curr_flowctrl_id = this->flowctrl_id_.Get();
   if (flowctrl_info.length() > 0) {
@@ -214,16 +200,15 @@ void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default,
   this->last_update_time_ = Utils::GetCurrentTimeMillis();
   pthread_rwlock_unlock(&this->configrw_lock_);
   if (is_default) {
-    LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n",
-      curr_flowctrl_id, flowctrl_id);
+    LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id,
+             flowctrl_id);
   } else {
-    LOG_INFO("[Flow Ctrl] Group FlowCtrl's flowctrl_id from %ld to %ld\n",
-      curr_flowctrl_id, flowctrl_id);
+    LOG_INFO("[Flow Ctrl] Group FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id,
+             flowctrl_id);
   }
   return;
 }
 
-
 void FlowCtrlRuleHandler::initialStatisData() {
   vector<FlowCtrlItem>::iterator it_vec;
   map<int, vector<FlowCtrlItem> >::iterator it_map;
@@ -250,7 +235,7 @@ void FlowCtrlRuleHandler::initialStatisData() {
   if (it_map != this->flowctrl_rules_.end()) {
     for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
       if (it_vec->GetType() != 1) {
-          continue;
+        continue;
       }
       if (it_vec->GetZeroCnt() < this->min_zero_cnt_.Get()) {
         this->min_zero_cnt_.Set(it_vec->GetZeroCnt());
@@ -264,8 +249,8 @@ void FlowCtrlRuleHandler::initialStatisData() {
         continue;
       }
       it_vec->GetDataSizeLimit();
-      this->filter_ctrl_item_.ResetFlowCtrlValue(3,
-          (int)(it_vec->GetDataSizeLimit()), it_vec->GetFreqMsLimit(), it_vec->GetZeroCnt());
+      this->filter_ctrl_item_.ResetFlowCtrlValue(3, (int)(it_vec->GetDataSizeLimit()),
+                                                 it_vec->GetFreqMsLimit(), it_vec->GetZeroCnt());
     }
   }
 }
@@ -286,10 +271,10 @@ bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult&
   time_t cur_time = time(NULL);
 
   gmtime_r(&cur_time, &utc_tm);
-  int curr_time = (utc_tm.tm_hour+8)%24 * 100 + utc_tm.tm_min;
-  if ((last_datadlt < this->min_datadlt_limt_.Get())
-    || (curr_time < this->datalimit_start_time_.Get())
-    || (curr_time > this->datalimit_end_time_.Get())) {
+  int curr_time = (utc_tm.tm_hour + 8) % 24 * 100 + utc_tm.tm_min;
+  if ((last_datadlt < this->min_datadlt_limt_.Get()) ||
+      (curr_time < this->datalimit_start_time_.Get()) ||
+      (curr_time > this->datalimit_end_time_.Get())) {
     return false;
   }
   it_map = this->flowctrl_rules_.find(0);
@@ -325,7 +310,6 @@ int FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t recei
   return received_limit;
 }
 
-
 bool FlowCtrlRuleHandler::compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2) {
   if (o1.GetStartTime() >= o2.GetStartTime()) {
     return true;
@@ -334,11 +318,11 @@ bool FlowCtrlRuleHandler::compareDataLimitQueue(const FlowCtrlItem& o1, const Fl
 }
 
 bool FlowCtrlRuleHandler::compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2) {
-    return (queue1.GetZeroCnt() < queue2.GetZeroCnt());
+  return (queue1.GetZeroCnt() < queue2.GetZeroCnt());
 }
 
-bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
-                                map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map) {
+bool FlowCtrlRuleHandler::parseFlowCtrlInfo(
+    const string& flowctrl_info, map<int32_t, vector<FlowCtrlItem> >& flowctrl_info_map) {
   int32_t type;
   string err_info;
   stringstream ss;
@@ -386,17 +370,15 @@ bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
         } else {
           LOG_ERROR("parse flowCtrlInfo's freqLimit failure: %s", err_info.c_str());
         }
-      }
-      break;
+      } break;
 
       case 3: {
         if (FlowCtrlRuleHandler::parseLowFetchLimit(err_info, node_item, flowctrl_item_vec)) {
-            flowctrl_info_map[3] = flowctrl_item_vec;
+          flowctrl_info_map[3] = flowctrl_item_vec;
         } else {
           LOG_ERROR("parse flowCtrlInfo's lowFetchLimit failure: %s", err_info.c_str());
         }
-      }
-      break;
+      } break;
 
       case 0: {
         if (FlowCtrlRuleHandler::parseDataLimit(err_info, node_item, flowctrl_item_vec)) {
@@ -404,8 +386,7 @@ bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
         } else {
           LOG_ERROR("parse flowCtrlInfo's dataLimit failure: %s", err_info.c_str());
         }
-      }
-      break;
+      } break;
 
       default:
         break;
@@ -414,8 +395,8 @@ bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
-                           const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+bool FlowCtrlRuleHandler::parseDataLimit(string& err_info, const rapidjson::Value& root,
+                                         vector<FlowCtrlItem>& flowctrl_items) {
   int32_t type_val;
   stringstream ss;
   string attr_sep = delimiter::kDelimiterColon;
@@ -438,7 +419,7 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& obj_set = root["rule"];
-  for (uint32_t index = 0 ; index < obj_set.Size() ; index++) {
+  for (uint32_t index = 0; index < obj_set.Size(); index++) {
     int32_t start_time = 0;
     int32_t end_time = 0;
     int64_t datadlt_m = 0;
@@ -446,8 +427,8 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
     int32_t freqms_limit = 0;
     const rapidjson::Value& node_item = obj_set[index];
     if (!node_item.IsObject()) {
-        err_info = "Illegal rule'value item, must be dict type";
-        return false;
+      err_info = "Illegal rule'value item, must be dict type";
+      return false;
     }
     if (!parseTimeMember(err_info, node_item, "start", start_time)) {
       return false;
@@ -506,8 +487,7 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
       err_info = ss.str();
       return false;
     }
-    FlowCtrlItem flowctrl_item(0, start_time,
-               end_time, datadlt_m, datasize_limit, freqms_limit);
+    FlowCtrlItem flowctrl_item(0, start_time, end_time, datadlt_m, datasize_limit, freqms_limit);
     flowctrl_items.push_back(flowctrl_item);
   }
   if (!flowctrl_items.empty()) {
@@ -517,8 +497,8 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
-                           const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info, const rapidjson::Value& root,
+                                         vector<FlowCtrlItem>& flowctrl_items) {
   int32_t type_val;
   stringstream ss;
 
@@ -539,7 +519,7 @@ bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& obj_set = root["rule"];
-  for (uint32_t i = 0 ; i < obj_set.Size() ; i++) {
+  for (uint32_t i = 0; i < obj_set.Size(); i++) {
     int32_t zeroCnt = -2;
     int32_t freqms_limit = -2;
     const rapidjson::Value& node_item = obj_set[i];
@@ -571,17 +551,17 @@ bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
-                            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info, const rapidjson::Value& root,
+                                             vector<FlowCtrlItem>& flowctrl_items) {
   int32_t type_val;
-  stringstream ss;
+  stringstream ss;
   if (!parseIntMember(err_info, root, "type", type_val, true, 3)) {
     ss << "Decode Failure: ";
     ss << err_info;
     ss << " of type field in parse low fetch limit!";
     err_info = ss.str();
     return false;
-  }
+  }
   if (!root.HasMember("rule")) {
     err_info = "rule field not existed";
     return false;
@@ -592,7 +572,7 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
   }
   // parse rule info
   const rapidjson::Value& node_item = root["rule"];
-  for (uint32_t i = 0 ; i < node_item.Size() ; i++) {
+  for (uint32_t i = 0; i < node_item.Size(); i++) {
     int32_t norm_freq_ms = 0;
     int32_t filter_freq_ms = 0;
     int32_t min_filter_freq_ms = 0;
@@ -602,18 +582,16 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
       err_info = "Illegal rule'value item, must be dict type";
       return false;
     }
-    if (node_item.HasMember("filterFreqInMs") 
-      || node_item.HasMember("minDataFilterFreqInMs")) {
-      if (!parseIntMember(err_info, node_item,
-                "filterFreqInMs", filter_freq_ms, false, -1)) {
+    if (node_item.HasMember("filterFreqInMs") || node_item.HasMember("minDataFilterFreqInMs")) {
+      if (!parseIntMember(err_info, node_item, "filterFreqInMs", filter_freq_ms, false, -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of filterFreqInMs field in parse low fetch limit!";
         err_info = ss.str();
         return false;
       }
-      if (!parseIntMember(err_info, node_item,
-                "minDataFilterFreqInMs", min_filter_freq_ms, false, -1)) {
+      if (!parseIntMember(err_info, node_item, "minDataFilterFreqInMs", min_filter_freq_ms, false,
+                          -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of minDataFilterFreqInMs field in parse low fetch limit!";
@@ -646,8 +624,7 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
       }
     }
     if (node_item.HasMember("normFreqInMs")) {
-      if (!parseIntMember(err_info, node_item,
-                "normFreqInMs", norm_freq_ms, false, -1)) {
+      if (!parseIntMember(err_info, node_item, "normFreqInMs", norm_freq_ms, false, -1)) {
         ss << "Decode Failure: ";
         ss << err_info;
         ss << " of normFreqInMs field in parse low fetch limit!";
@@ -663,17 +640,16 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info,
         return false;
       }
     }
-    flowctrl_item.ResetFlowCtrlValue(3,
-                     norm_freq_ms, filter_freq_ms, min_filter_freq_ms);
+    flowctrl_item.ResetFlowCtrlValue(3, norm_freq_ms, filter_freq_ms, min_filter_freq_ms);
     flowctrl_items.push_back(flowctrl_item);
   }
   err_info = "Ok";
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseStringMember(string& err_info,
-    const rapidjson::Value& root, const char* key, string& value,
-    bool compare_value, string required_val) {
+bool FlowCtrlRuleHandler::parseStringMember(string& err_info, const rapidjson::Value& root,
+                                            const char* key, string& value, bool compare_value,
+                                            string required_val) {
   // check key if exist
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
@@ -694,9 +670,9 @@ bool FlowCtrlRuleHandler::parseStringMember(string& err_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseLongMember(string& err_info,
-      const rapidjson::Value& root, const char* key, int64_t& value,
-      bool compare_value, int64_t required_val) {
+bool FlowCtrlRuleHandler::parseLongMember(string& err_info, const rapidjson::Value& root,
+                                          const char* key, int64_t& value, bool compare_value,
+                                          int64_t required_val) {
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
     return false;
@@ -715,9 +691,9 @@ bool FlowCtrlRuleHandler::parseLongMember(string& err_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseIntMember(string& err_info,
-      const rapidjson::Value& root, const char* key, int32_t& value,
-      bool compare_value, int32_t required_val) {
+bool FlowCtrlRuleHandler::parseIntMember(string& err_info, const rapidjson::Value& root,
+                                         const char* key, int32_t& value, bool compare_value,
+                                         int32_t required_val) {
   if (!root.HasMember(key)) {
     err_info = "Field not existed";
     return false;
@@ -736,8 +712,8 @@ bool FlowCtrlRuleHandler::parseIntMember(string& err_info,
   return true;
 }
 
-bool FlowCtrlRuleHandler::parseTimeMember(string& err_info,
-              const rapidjson::Value& root, const char* key, int32_t& value) {
+bool FlowCtrlRuleHandler::parseTimeMember(string& err_info, const rapidjson::Value& root,
+                                          const char* key, int32_t& value) {
   // check key if exist
   stringstream ss;
   if (!root.HasMember(key)) {
@@ -787,6 +763,4 @@ bool FlowCtrlRuleHandler::parseTimeMember(string& err_info,
   return true;
 }
 
-
 }  // namespace tubemq
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
index a9d5df3..7360360 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/logger.cc
@@ -18,11 +18,13 @@
  */
 
 #include "logger.h"
+
 #include <log4cplus/fileappender.h>
 #include <log4cplus/layout.h>
 #include <log4cplus/logger.h>
 #include <log4cplus/loggingmacros.h>
 #include <stdarg.h>
+
 #include <string>
 
 namespace tubemq {
@@ -31,7 +33,10 @@ Logger tubemq_logger;
 
 Logger& GetLogger() { return tubemq_logger; }
 
-bool Logger::Init(const std::string& path, Logger::Level level, uint32_t file_max_size, uint32_t file_num) {
+static const uint32_t kMBSize = 1024 * 1024;
+
+bool Logger::Init(const std::string& path, Logger::Level level, uint32_t file_max_size,
+                  uint32_t file_num) {
   base_path_ = path;
   file_max_size_ = file_max_size;
   file_num_ = file_num;
@@ -64,7 +69,8 @@ void Logger::setup() {
   auto logger_d = log4cplus::Logger::getInstance(instance_);
   logger_d.setLogLevel(log4cplus::TRACE_LOG_LEVEL);
   log4cplus::helpers::SharedObjectPtr<log4cplus::Appender> append_d(
-      new log4cplus::RollingFileAppender(base_path_ + ".log", file_max_size_, file_num_, immediate_fush));
+      new log4cplus::RollingFileAppender(base_path_ + ".log", file_max_size_ * kMBSize, file_num_,
+                                         immediate_fush));
   std::unique_ptr<log4cplus::Layout> layout_d(new log4cplus::PatternLayout(pattern));
   append_d->setLayout(std::move(layout_d));
   logger_d.addAppender(append_d);
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index d57fb5a..7fb83f8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -18,16 +18,16 @@
  */
 
 #include "message.h"
+
 #include <string.h>
+
 #include <sstream>
+
 #include "const_config.h"
 #include "utils.h"
 
-
-
 namespace tubemq {
 
-
 // message flag's properties settings
 static const int32_t kMsgFlagIncProperties = 0x01;
 // reserved property key Filter Item
@@ -35,88 +35,66 @@ static const string kRsvPropKeyFilterItem = "$msgType$";
 // reserved property key message send time
 static const string kRsvPropKeyMsgTime = "$msgTime$";
 
-
 Message::Message() {
-  this->topic_      = "";
-  this->flag_       = 0;
+  this->topic_ = "";
+  this->flag_ = 0;
   this->message_id_ = config::kInvalidValue;
-  this->data_       = NULL;
-  this->datalen_    = 0;
+  this->data_ = NULL;
+  this->datalen_ = 0;
   this->properties_.clear();
 }
 
 Message::Message(const Message& target) {
-  this->topic_      = target.topic_;
+  this->topic_ = target.topic_;
   this->message_id_ = target.message_id_;
   copyData(target.data_, target.datalen_);
   copyProperties(target.properties_);
-  this->flag_       = target.flag_;
+  this->flag_ = target.flag_;
 }
 
 Message::Message(const string& topic, const char* data, uint32_t datalen) {
-  this->topic_      = topic;
-  this->flag_       = 0;
+  this->topic_ = topic;
+  this->flag_ = 0;
   this->message_id_ = config::kInvalidValue;
   copyData(data, datalen);
   this->properties_.clear();
 }
 
-Message::~Message() {
-  clearData();
-}
+Message::~Message() { clearData(); }
 
 Message& Message::operator=(const Message& target) {
-  if (this == &target)
-    return *this;
-  this->topic_      = target.topic_;
+  if (this == &target) return *this;
+  this->topic_ = target.topic_;
   this->message_id_ = target.message_id_;
   clearData();
   copyData(target.data_, target.datalen_);
   copyProperties(target.properties_);
-  this->flag_       = target.flag_;
+  this->flag_ = target.flag_;
   return *this;
 }
 
-const uint64_t Message::GetMessageId() const {
-  return this->message_id_;
-}
+const uint64_t Message::GetMessageId() const { return this->message_id_; }
 
-void Message::SetMessageId(int64_t message_id) {
-  this->message_id_ = message_id;
-}
+void Message::SetMessageId(int64_t message_id) { this->message_id_ = message_id; }
 
-const string& Message::GetTopic() const {
-  return this->topic_;
-}
+const string& Message::GetTopic() const { return this->topic_; }
 
-void Message::SetTopic(const string& topic) {
-  this->topic_ = topic;
-}
+void Message::SetTopic(const string& topic) { this->topic_ = topic; }
 
-const char* Message::GetData() const {
-  return this->data_;
-}
+const char* Message::GetData() const { return this->data_; }
 
-uint32_t Message::GetDataLength() const {
-  return this->datalen_;
-}
+uint32_t Message::GetDataLength() const { return this->datalen_; }
 
 void Message::setData(const char* data, uint32_t datalen) {
   clearData();
   copyData(data, datalen);
 }
 
-const int32_t Message::GetFlag() const {
-  return this->flag_;
-}
+const int32_t Message::GetFlag() const { return this->flag_; }
 
-void Message::SetFlag(int32_t flag) {
-  this->flag_ = flag;
-}
+void Message::SetFlag(int32_t flag) { this->flag_ = flag; }
 
-const map<string, string>& Message::GetProperties() const {
-  return this->properties_;
-}
+const map<string, string>& Message::GetProperties() const { return this->properties_; }
 
 int32_t Message::GetProperties(string& attribute) {
   attribute.clear();
@@ -157,9 +135,7 @@ bool Message::GetProperty(const string& key, string& value) {
   return false;
 }
 
-bool Message::GetFilterItem(string& value) {
-  return GetProperty(kRsvPropKeyFilterItem, value);
-}
+bool Message::GetFilterItem(string& value) { return GetProperty(kRsvPropKeyFilterItem, value); }
 
 bool Message::AddProperty(string& err_info, const string& key, const string& value) {
   string trimed_key = Utils::Trim(key);
@@ -168,8 +144,8 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = "Not allowed null value of parmeter key or value";
     return false;
   }
-  if ((string::npos != trimed_key.find(delimiter::kDelimiterComma))
-    ||(string::npos != trimed_key.find(delimiter::kDelimiterEqual))) {
+  if ((string::npos != trimed_key.find(delimiter::kDelimiterComma)) ||
+      (string::npos != trimed_key.find(delimiter::kDelimiterEqual))) {
     stringstream ss;
     ss << "Reserved token '";
     ss << delimiter::kDelimiterComma;
@@ -179,8 +155,8 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if ((string::npos != trimed_value.find(delimiter::kDelimiterComma))
-    ||(string::npos != trimed_value.find(delimiter::kDelimiterEqual))) {
+  if ((string::npos != trimed_value.find(delimiter::kDelimiterComma)) ||
+      (string::npos != trimed_value.find(delimiter::kDelimiterEqual))) {
     stringstream ss;
     ss << "Reserved token '";
     ss << delimiter::kDelimiterComma;
@@ -190,8 +166,7 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
     err_info = ss.str();
     return false;
   }
-  if (trimed_key == kRsvPropKeyFilterItem
-         || trimed_key == kRsvPropKeyMsgTime) {
+  if (trimed_key == kRsvPropKeyFilterItem || trimed_key == kRsvPropKeyMsgTime) {
     stringstream ss;
     ss << "Reserved token '";
     ss << kRsvPropKeyFilterItem;
@@ -204,7 +179,7 @@ bool Message::AddProperty(string& err_info, const string& key, const string& val
   // add key and value
   this->properties_[trimed_key] = trimed_value;
   if (!this->properties_.empty()) {
-      this->flag_ |= kMsgFlagIncProperties;
+    this->flag_ |= kMsgFlagIncProperties;
   }
   err_info = "Ok";
   return true;
@@ -241,9 +216,4 @@ void Message::copyProperties(const map<string, string>& properties) {
   }
 }
 
-
 }  // namespace tubemq
-
-
-
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index ebd75d1..c1f428c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -18,22 +18,22 @@
  */
 
 #include "meta_info.h"
+
 #include <stdlib.h>
+
 #include <sstream>
 #include <vector>
+
 #include "const_config.h"
 #include "utils.h"
 
-
 namespace tubemq {
 
-using std::vector;
 using std::sstream;
-
-
+using std::vector;
 
 NodeInfo::NodeInfo() {
-  this->node_id_   = 0;
+  this->node_id_ = 0;
   this->node_host_ = " ";
   this->node_port_ = config::kBrokerPortDef;
   buildStrInfo();
@@ -44,14 +44,14 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
   vector<string> result;
   Utils::Split(node_info, result, delimiter::kDelimiterColon);
   if (is_broker) {
-    this->node_id_   = atoi(result[0].c_str());
+    this->node_id_ = atoi(result[0].c_str());
     this->node_host_ = result[1];
     this->node_port_ = config::kBrokerPortDef;
     if (result.size() >= 3) {
       this->node_port_ = atoi(result[2].c_str());
     }
   } else {
-    this->node_id_   = 0;
+    this->node_id_ = 0;
     this->node_host_ = result[0];
     this->node_port_ = config::kBrokerPortDef;
     if (result.size() >= 2) {
@@ -62,14 +62,14 @@ NodeInfo::NodeInfo(bool is_broker, const string& node_info) {
 }
 
 NodeInfo::NodeInfo(const string& node_host, uint32_t node_port) {
-  this->node_id_   = config::kInvalidValue;
+  this->node_id_ = config::kInvalidValue;
   this->node_host_ = node_host;
   this->node_port_ = node_port;
   buildStrInfo();
 }
 
 NodeInfo::NodeInfo(int node_id, const string& node_host, uint32_t node_port) {
-  this->node_id_   = node_id;
+  this->node_id_ = node_id;
   this->node_host_ = node_host;
   this->node_port_ = node_port;
   buildStrInfo();
@@ -81,7 +81,7 @@ NodeInfo::~NodeInfo() {
 
 NodeInfo& NodeInfo::operator=(const NodeInfo& target) {
   if (this != &target) {
-    this->node_id_   = target.node_id_;
+    this->node_id_ = target.node_id_;
     this->node_host_ = target.node_host_;
     this->node_port_ = target.node_port_;
     this->addr_info_ = target.addr_info_;
@@ -90,7 +90,7 @@ NodeInfo& NodeInfo::operator=(const NodeInfo& target) {
   return *this;
 }
 
-bool NodeInfo::operator== (const NodeInfo& target) {
+bool NodeInfo::operator==(const NodeInfo& target) {
   if (this == &target) {
     return true;
   }
@@ -100,28 +100,18 @@ bool NodeInfo::operator== (const NodeInfo& target) {
   return false;
 }
 
-bool NodeInfo::operator< (const NodeInfo& target) const {
+bool NodeInfo::operator<(const NodeInfo& target) const {
   return this->node_info_ < target.node_info_;
 }
-const uint32_t NodeInfo::GetNodeId() const {
-  return this->node_id_;
-}
+const uint32_t NodeInfo::GetNodeId() const { return this->node_id_; }
 
-const string& NodeInfo::GetHost() const {
-  return this->node_host_;
-}
+const string& NodeInfo::GetHost() const { return this->node_host_; }
 
-const uint32_t NodeInfo::GetPort() const {
-  return this->node_port_;
-}
+const uint32_t NodeInfo::GetPort() const { return this->node_port_; }
 
-const string& NodeInfo::GetAddrInfo() const {
-  return this->addr_info_;
-}
+const string& NodeInfo::GetAddrInfo() const { return this->addr_info_; }
 
-const string& NodeInfo::GetNodeInfo() const {
-  return this->node_info_;
-}
+const string& NodeInfo::GetNodeInfo() const { return this->node_info_; }
 
 void NodeInfo::buildStrInfo() {
   stringstream ss1;
@@ -137,7 +127,6 @@ void NodeInfo::buildStrInfo() {
   this->node_info_ = ss2.str();
 }
 
-
 Partition::Partition() {
   this->topic_ = " ";
   this->partition_id_ = 0;
@@ -210,7 +199,7 @@ Partition& Partition::operator=(const Partition& target) {
   return *this;
 }
 
-bool Partition::operator== (const Partition& target) {
+bool Partition::operator==(const Partition& target) {
   if (this == &target) {
     return true;
   }
@@ -220,37 +209,21 @@ bool Partition::operator== (const Partition& target) {
   return false;
 }
 
-const uint32_t Partition::GetBrokerId() const {
-  return this->broker_info_.GetNodeId();
-}
+const uint32_t Partition::GetBrokerId() const { return this->broker_info_.GetNodeId(); }
 
-const string& Partition::GetBrokerHost() const {
-  return this->broker_info_.GetHost();
-}
+const string& Partition::GetBrokerHost() const { return this->broker_info_.GetHost(); }
 
-const uint32_t Partition::GetBrokerPort() const {
-  return this->broker_info_.GetPort();
-}
+const uint32_t Partition::GetBrokerPort() const { return this->broker_info_.GetPort(); }
 
-const string& Partition::GetPartitionKey() const {
-  return this->partition_key_;
-}
+const string& Partition::GetPartitionKey() const { return this->partition_key_; }
 
-const string& Partition::GetTopic() const {
-  return this->topic_;
-}
+const string& Partition::GetTopic() const { return this->topic_; }
 
-const NodeInfo& Partition::GetBrokerInfo() const {
-  return this->broker_info_;
-}
+const NodeInfo& Partition::GetBrokerInfo() const { return this->broker_info_; }
 
-const uint32_t Partition::GetPartitionId() const {
-  return this->partition_id_;
-}
+const uint32_t Partition::GetPartitionId() const { return this->partition_id_; }
 
-const string& Partition::ToString() const {
-  return this->partition_info_;
-}
+const string& Partition::ToString() const { return this->partition_info_; }
 
 void Partition::buildPartitionKey() {
   stringstream ss1;
@@ -270,7 +243,6 @@ void Partition::buildPartitionKey() {
   this->partition_info_ = ss2.str();
 }
 
-
 // sub_info = consumerId@group#broker_info#topic:partitionId
 SubscribeInfo::SubscribeInfo(const string& sub_info) {
   string::size_type pos = 0;
@@ -295,59 +267,40 @@ SubscribeInfo::SubscribeInfo(const string& sub_info) {
   buildSubInfo();
 }
 
-SubscribeInfo::SubscribeInfo(const string& consumer_id,
-                 const string& group, const Partition& partition) {
+SubscribeInfo::SubscribeInfo(const string& consumer_id, const string& group,
+                             const Partition& partition) {
   this->consumer_id_ = consumer_id;
-  this->group_       = group;
-  this->partition_   = partition;
+  this->group_ = group;
+  this->partition_ = partition;
   buildSubInfo();
 }
 
-
 SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
   if (this != &target) {
     this->consumer_id_ = target.consumer_id_;
-    this->group_       = target.group_;
-    this->partition_   = target.partition_;
+    this->group_ = target.group_;
+    this->partition_ = target.partition_;
   }
   return *this;
 }
 
-const string& SubscribeInfo::GetConsumerId() const {
-  return this->consumer_id_;
-}
+const string& SubscribeInfo::GetConsumerId() const { return this->consumer_id_; }
 
-const string& SubscribeInfo::GetGroup() const {
-  return this->group_;
-}
+const string& SubscribeInfo::GetGroup() const { return this->group_; }
 
-const Partition& SubscribeInfo::GetPartition() const {
-  return this->partition_;
-}
+const Partition& SubscribeInfo::GetPartition() const { return this->partition_; }
 
-const uint32_t SubscribeInfo::GgetBrokerId() const {
-  return this->partition_.GetBrokerId();
-}
+const uint32_t SubscribeInfo::GgetBrokerId() const { return this->partition_.GetBrokerId(); }
 
-const string& SubscribeInfo::GetBrokerHost() const {
-  return this->partition_.GetBrokerHost();
-}
+const string& SubscribeInfo::GetBrokerHost() const { return this->partition_.GetBrokerHost(); }
 
-const uint32_t SubscribeInfo::GetBrokerPort() const {
-  return this->partition_.GetBrokerPort();
-}
+const uint32_t SubscribeInfo::GetBrokerPort() const { return this->partition_.GetBrokerPort(); }
 
-const string& SubscribeInfo::GetTopic() const {
-  return this->partition_.GetTopic();
-}
+const string& SubscribeInfo::GetTopic() const { return this->partition_.GetTopic(); }
 
-const uint32_t SubscribeInfo::GetPartitionId() const {
-  return this->partition_.GetPartitionId();
-}
+const uint32_t SubscribeInfo::GetPartitionId() const { return this->partition_.GetPartitionId(); }
 
-const string& SubscribeInfo::ToString() const {
-  return this->sub_info_;
-}
+const string& SubscribeInfo::ToString() const { return this->sub_info_; }
 
 void SubscribeInfo::buildSubInfo() {
   stringstream ss;
@@ -359,25 +312,24 @@ void SubscribeInfo::buildSubInfo() {
   this->sub_info_ = ss.str();
 }
 
-
 ConsumerEvent::ConsumerEvent() {
   this->rebalance_id_ = config::kInvalidValue;
-  this->event_type_   = config::kInvalidValue;
+  this->event_type_ = config::kInvalidValue;
   this->event_status_ = config::kInvalidValue;
 }
 
 ConsumerEvent::ConsumerEvent(const ConsumerEvent& target) {
   this->rebalance_id_ = target.rebalance_id_;
-  this->event_type_   = target.event_type_;
+  this->event_type_ = target.event_type_;
   this->event_status_ = target.event_status_;
   this->subscribe_list_ = target.subscribe_list_;
 }
 
 ConsumerEvent::ConsumerEvent(int64_t rebalance_id, int32_t event_type,
-                 const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
+                             const list<SubscribeInfo>& subscribeInfo_lst, int32_t event_status) {
   list<SubscribeInfo>::const_iterator it;
   this->rebalance_id_ = rebalance_id;
-  this->event_type_   = event_type;
+  this->event_type_ = event_type;
   this->event_status_ = event_status;
   for (it = subscribeInfo_lst.begin(); it != subscribeInfo_lst.end(); ++it) {
     this->subscribe_list_.push_back(*it);
@@ -394,25 +346,15 @@ ConsumerEvent& ConsumerEvent::operator=(const ConsumerEvent& target) {
   return *this;
 }
 
-const int64_t ConsumerEvent::GetRebalanceId() const {
-  return this->rebalance_id_;
-}
+const int64_t ConsumerEvent::GetRebalanceId() const { return this->rebalance_id_; }
 
-const int32_t ConsumerEvent::GetEventType() const {
-  return this->event_type_;
-}
+const int32_t ConsumerEvent::GetEventType() const { return this->event_type_; }
 
-const int32_t ConsumerEvent::GetEventStatus() const {
-  return this->event_status_;
-}
+const int32_t ConsumerEvent::GetEventStatus() const { return this->event_status_; }
 
-void ConsumerEvent::SetEventType(int32_t event_type) {
-  this->event_type_ = event_type;
-}
+void ConsumerEvent::SetEventType(int32_t event_type) { this->event_type_ = event_type; }
 
-void ConsumerEvent::SetEventStatus(int32_t event_status) {
-  this->event_status_ = event_status;
-}
+void ConsumerEvent::SetEventStatus(int32_t event_status) { this->event_status_ = event_status; }
 
 const list<SubscribeInfo>& ConsumerEvent::GetSubscribeInfoList() const {
   return this->subscribe_list_;
@@ -429,8 +371,7 @@ string ConsumerEvent::ToString() {
   ss << ", status=";
   ss << this->event_status_;
   ss << ", subscribeInfoList=[";
-  for (it = this->subscribe_list_.begin();
-          it != this->subscribe_list_.end(); ++it) {
+  for (it = this->subscribe_list_.begin(); it != this->subscribe_list_.end(); ++it) {
     if (count++ > 0) {
       ss << ",";
     }
@@ -441,4 +382,3 @@ string ConsumerEvent::ToString() {
 }
 
 };  // namespace tubemq
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index c699266..2fba193 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -18,14 +18,16 @@
  */
 
 #include "utils.h"
-#include <sstream>
-#include <vector>
-#include "const_config.h"
+
 #include <regex.h>
 #include <stdlib.h>
 #include <sys/time.h>
 #include <unistd.h>
 
+#include <sstream>
+#include <vector>
+
+#include "const_config.h"
 
 namespace tubemq {
 
@@ -71,9 +73,8 @@ void Utils::Split(const string& source, vector<string>& result, const string& de
   }
 }
 
-
-void Utils::Split(const string& source, map<string, int>& result,
-                const string& delimiter_step1, const string& delimiter_step2) {
+void Utils::Split(const string& source, map<string, int>& result, const string& delimiter_step1,
+                  const string& delimiter_step2) {
   string item_str;
   string key_str;
   string val_str;
@@ -84,7 +85,7 @@ void Utils::Split(const string& source, map<string, int>& result,
     pos1 = 0;
     pos2 = source.find(delimiter_step1);
     while (string::npos != pos2) {
-      item_str = source.substr(pos1, pos2-pos1);
+      item_str = source.substr(pos1, pos2 - pos1);
       item_str = Utils::Trim(item_str);
       pos1 = pos2 + delimiter_step1.length();
       pos2 = source.find(delimiter_step1, pos1);
@@ -96,7 +97,7 @@ void Utils::Split(const string& source, map<string, int>& result,
         continue;
       }
       key_str = item_str.substr(0, pos3);
-      val_str = item_str.substr(pos3+delimiter_step2.length());
+      val_str = item_str.substr(pos3 + delimiter_step2.length());
       key_str = Utils::Trim(key_str);
       val_str = Utils::Trim(val_str);
       if (key_str.empty()) {
@@ -110,7 +111,7 @@ void Utils::Split(const string& source, map<string, int>& result,
       pos3 = item_str.find(delimiter_step2);
       if (string::npos != pos3) {
         key_str = item_str.substr(0, pos3);
-        val_str = item_str.substr(pos3+delimiter_step2.length());
+        val_str = item_str.substr(pos3 + delimiter_step2.length());
         key_str = Utils::Trim(key_str);
         val_str = Utils::Trim(val_str);
         if (!key_str.empty()) {
@@ -132,9 +133,8 @@ void Utils::Join(const vector<string>& vec, const string& delimiter, string& tar
   }
 }
 
-bool Utils::ValidString(string& err_info, const string& source,
-                bool allow_empty, bool pat_match, bool check_max_length,
-                unsigned int maxlen) {
+bool Utils::ValidString(string& err_info, const string& source, bool allow_empty, bool pat_match,
+                        bool check_max_length, unsigned int maxlen) {
   if (source.empty()) {
     if (allow_empty) {
       err_info = "Ok";
@@ -174,8 +174,7 @@ bool Utils::ValidString(string& err_info, const string& source,
   return true;
 }
 
-bool Utils::ValidGroupName(string& err_info,
-                const string& group_name, string& tgt_group_name) {
+bool Utils::ValidGroupName(string& err_info, const string& group_name, string& tgt_group_name) {
   tgt_group_name = Utils::Trim(group_name);
   if (tgt_group_name.empty()) {
     err_info = "Illegal parameter: group_name is blank!";
@@ -210,8 +209,8 @@ bool Utils::ValidGroupName(string& err_info,
   return true;
 }
 
-bool Utils::ValidFilterItem(string& err_info,
-                const string& src_filteritem, string& tgt_filteritem) {
+bool Utils::ValidFilterItem(string& err_info, const string& src_filteritem,
+                            string& tgt_filteritem) {
   tgt_filteritem = Utils::Trim(src_filteritem);
   if (tgt_filteritem.empty()) {
     err_info = "value is blank!";
@@ -257,20 +256,18 @@ uint32_t Utils::IpToInt(const string& ipv4_addr) {
   vector<string> result_vec;
 
   Utils::Split(ipv4_addr, result_vec, delimiter::kDelimiterDot);
-  result = ((char) atoi(result_vec[3].c_str())) & 0xFF;
-  result |= ((char) atoi(result_vec[2].c_str()) << 8) & 0xFF00;
-  result |= ((char) atoi(result_vec[1].c_str()) << 16) & 0xFF0000;
-  result |= ((char) atoi(result_vec[0].c_str()) << 24) & 0xFF000000;
+  result = ((char)atoi(result_vec[3].c_str())) & 0xFF;
+  result |= ((char)atoi(result_vec[2].c_str()) << 8) & 0xFF00;
+  result |= ((char)atoi(result_vec[1].c_str()) << 16) & 0xFF0000;
+  result |= ((char)atoi(result_vec[0].c_str()) << 24) & 0xFF000000;
   return result;
 }
 
 int64_t Utils::GetCurrentTimeMillis() {
   struct timeval tv;
   gettimeofday(&tv, NULL);
-  return tv.tv_sec * 1000 + tv.tv_usec /1000;
+  return tv.tv_sec * 1000 + tv.tv_usec / 1000;
 }
 
-
-
 }  // namespace tubemq
 


[incubator-tubemq] 26/50: [TUBEMQ-274]Support CMake compilation (#197)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 82e02a8c4247a4da096999d85f389b3ef060fca0
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Thu Jul 9 21:33:01 2020 +0800

    [TUBEMQ-274]Support CMake compilation (#197)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .gitignore                                         |  1 +
 .../tubemq-client-cpp/CMakeLists.txt               | 50 ++++++++++++++++++++++
 .../tubemq-client-cpp/build_linux.sh               | 31 ++++++++++++++
 .../tubemq-client-cpp/example/CMakeLists.txt       | 32 ++++++++++++++
 .../tubemq-client-cpp/example/log/CMakeLists.txt   | 21 +++++++++
 .../tubemq-client-cpp/example/log/main.cc          |  2 +-
 .../tubemq-client-cpp/include/tubemq/logger.h      |  6 +--
 .../tubemq-client-cpp/src/CMakeLists.txt           | 32 ++++++++++++++
 .../tubemq-client-cpp/src/client_config.cc         |  9 ++--
 .../tubemq-client-cpp/src/flowctrl_def.cc          | 16 +++----
 .../tubemq-client-cpp/third_party/CMakeLists.txt   | 25 +++++++++++
 .../tubemq-client-cpp/third_party/log4cplus        |  2 +-
 12 files changed, 207 insertions(+), 20 deletions(-)

diff --git a/.gitignore b/.gitignore
index eeb8963..035ef97 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,3 +7,4 @@ target/
 !.gitignore
 !.gitkeep
 !.travis.yml
+tubemq-client-twins/tubemq-client-cpp/build/
diff --git a/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
new file mode 100644
index 0000000..db9014a
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
@@ -0,0 +1,50 @@
+#
+# 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 3.1)
+
+project (TubeMQ)
+
+
+INCLUDE_DIRECTORIES(include)
+
+INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/third_party/rapidjson/include)
+INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/third_party/log4cplus/include)
+INCLUDE_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/log4cplus/include)
+LINK_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/log4cplus/lib)
+
+ADD_SUBDIRECTORY(src)
+ADD_SUBDIRECTORY(third_party)
+ADD_SUBDIRECTORY(example)
+
+
+if (UNIX)
+    SET(CMAKE_CXX_FLAGS_DEBUG   "-O1 -g -ggdb -D_DEBUG")
+    SET(CMAKE_CXX_FLAGS_RELEASE "-O3 -g -ggdb -DNDEBUG")
+    SET(DEPENDENT_LIBRARIES log4cplus pthread)
+else (UNIX)
+    SET(DEPENDENT_LIBRARIES log4cplus)
+endif (UNIX)
+
+
+
+
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/build_linux.sh b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
new file mode 100755
index 0000000..be79e90
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
@@ -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.
+#
+
+
+#!/bin/bash
+
+cd ../../
+git submodule foreach --recursive git submodule init 
+git submodule foreach --recursive git submodule update 
+cd -
+
+mkdir build
+cd build
+cmake ../
+make
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
new file mode 100644
index 0000000..1d8fcdd
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
@@ -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.
+#
+
+
+# A function to set up a test, since it's the same for each one.  Note:
+# unit_tests test is not set up using this function because it does not like
+# the additional argument on commmand line and consequently does not run any
+# test.
+function(tubemq_add_example _name)
+  set(_srcs ${ARGN})
+  message (STATUS "${_name} sources: ${_srcs}")
+  add_executable (${_name} ${_srcs})
+  TARGET_LINK_LIBRARIES (${_name} tubemq log4cplus pthread)
+endfunction()
+
+add_subdirectory (log)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
new file mode 100644
index 0000000..b828bf8
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+
+tubemq_add_example(log main.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc b/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
index f21d022..9bb1685 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
@@ -23,7 +23,7 @@
 #include <string>
 #include <thread>
 
-#include "logger.h"
+#include "tubemq/logger.h"
 
 using namespace std;
 using namespace tubemq;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
index eb35075..74d083c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/logger.h
@@ -98,11 +98,7 @@ class Logger {
 
   static const char* Level2String(Level level) {
     static const char* level_names[] = {
-        "TRACE",
-        "DEBUG"
-        "INFO",
-        "WARN",
-        "ERROR",
+        "TRACE", "DEBUG", "INFO", "WARN", "ERROR",
     };
     return level_names[level];
   }
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
new file mode 100644
index 0000000..a6e76e5
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
@@ -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.
+#
+
+
+cmake_minimum_required (VERSION 3.1)
+
+
+set(CMAKE_C_FLAGS "-O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
+set(CMAKE_CXX_FLAGS "-std=c++11 -O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
+
+AUX_SOURCE_DIRECTORY(. CURRENT_DIR_SRCS)                                        
+ADD_LIBRARY(tubemq STATIC ${CURRENT_DIR_SRCS})   
+TARGET_LINK_LIBRARIES (tubemq)
+
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index b623497..dfd5b8c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -28,11 +28,10 @@
 
 namespace tubemq {
 
-using std::stringstream;
 using std::set;
+using std::stringstream;
 using std::vector;
 
-
 BaseConfig::BaseConfig() {
   this->master_addrinfo_ = "";
   this->auth_enable_ = false;
@@ -335,8 +334,8 @@ bool ConsumerConfig::setGroupConsumeTarget(
     string tmp_filteritem;
     set<string> tgt_filters;
     // check topic_name info
-    is_success =
-        Utils::ValidString(err_info, it_map->first, false, true, true, tb_config::kTopicNameMaxLength);
+    is_success = Utils::ValidString(err_info, it_map->first, false, true, true,
+                                    tb_config::kTopicNameMaxLength);
     if (!is_success) {
       stringstream ss;
       ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
@@ -503,7 +502,7 @@ void ConsumerConfig::SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_
   this->reb_confirm_wait_period_ms_ = reb_confirm_wait_period_ms;
 }
 
-const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const { this->max_confirm_wait_period_ms_; }
+const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const { return max_confirm_wait_period_ms_; }
 
 void ConsumerConfig::SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms) {
   this->max_confirm_wait_period_ms_ = max_confirm_wait_period_ms;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index f5c90a8..370e6e6 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -178,14 +178,13 @@ FlowCtrlRuleHandler::~FlowCtrlRuleHandler() { pthread_rwlock_destroy(&configrw_l
 
 void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id,
                                                 int64_t flowctrl_id, const string& flowctrl_info) {
-  bool result;
   map<int32_t, vector<FlowCtrlItem> > tmp_flowctrl_map;
   if (flowctrl_id == this->flowctrl_id_.Get()) {
     return;
   }
   int64_t curr_flowctrl_id = this->flowctrl_id_.Get();
   if (flowctrl_info.length() > 0) {
-    result = parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
+    parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
   }
   pthread_rwlock_wrlock(&this->configrw_lock_);
   this->flowctrl_id_.Set(flowctrl_id);
@@ -266,7 +265,8 @@ void FlowCtrlRuleHandler::clearStatisData() {
   this->filter_ctrl_item_.Clear();
 }
 
-bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result) const {
+bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt,
+                                          FlowCtrlResult& flowctrl_result) const {
   struct tm utc_tm;
   vector<FlowCtrlItem>::const_iterator it_vec;
   map<int, vector<FlowCtrlItem> >::const_iterator it_map;
@@ -291,7 +291,8 @@ bool FlowCtrlRuleHandler::GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult&
   return false;
 }
 
-int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit) const {
+int32_t FlowCtrlRuleHandler::GetCurFreqLimitTime(int32_t msg_zero_cnt,
+                                                 int32_t received_limit) const {
   int32_t rule_val = -2;
   vector<FlowCtrlItem>::const_iterator it_vec;
   map<int, vector<FlowCtrlItem> >::const_iterator it_map;
@@ -402,7 +403,6 @@ bool FlowCtrlRuleHandler::parseDataLimit(string& err_info, const rapidjson::Valu
   int32_t type_val;
   stringstream ss;
   string attr_sep = delimiter::kDelimiterColon;
-  string::size_type pos1;
   if (!parseIntMember(err_info, root, "type", type_val, true, 0)) {
     ss << "Decode Failure: ";
     ss << err_info;
@@ -573,13 +573,13 @@ bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info, const rapidjson::
     return false;
   }
   // parse rule info
-  const rapidjson::Value& node_item = root["rule"];
-  for (uint32_t i = 0; i < node_item.Size(); i++) {
+  const rapidjson::Value& nodes = root["rule"];
+  for (uint32_t i = 0; i < nodes.Size(); ++i) {
     int32_t norm_freq_ms = 0;
     int32_t filter_freq_ms = 0;
     int32_t min_filter_freq_ms = 0;
     FlowCtrlItem flowctrl_item;
-    const rapidjson::Value& node_item = node_item[i];
+    const rapidjson::Value& node_item = nodes[i];
     if (!node_item.IsObject()) {
       err_info = "Illegal rule'value item, must be dict type";
       return false;
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
new file mode 100644
index 0000000..7dc5e88
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/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.
+#
+
+
+CMAKE_MINIMUM_REQUIRED(VERSION 3.1)                                             
+PROJECT(TubeMQThirdParty)
+
+ADD_SUBDIRECTORY(log4cplus) 
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus b/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
index 76ff7e6..9d00f7d 160000
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
@@ -1 +1 @@
-Subproject commit 76ff7e68c35e277440d414ba782eceedad8db7b1
+Subproject commit 9d00f7d10f2507f68f9ab5fea8b842735d9c6cfe


[incubator-tubemq] 10/50: [TUBEMQ-262]Create C++ flow control handler

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit d1e8b3c4e4e26267e23c70268838818c50f97248
Author: gosonzhang <go...@tencent.com>
AuthorDate: Fri Jul 3 09:31:33 2020 +0800

    [TUBEMQ-262]Create C++ flow control handler
---
 tubemq-client-twins/tubemq-client-cpp/third_party/readme.md | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md b/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md
new file mode 100644
index 0000000..e1752ca
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md
@@ -0,0 +1,2 @@
+#Third-party libraries#
+tubemq-client-cpp depends on several third-party libraries, their source code is available (usually as a git submodule) in this directory.
\ No newline at end of file


[incubator-tubemq] 13/50: [TUBEMQ-267]Create C/C++ Message class (#182)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 86b88f3c622e09d11dea06e3fbf09a20002ce997
Author: gosonzhang <go...@tencent.com>
AuthorDate: Fri Jul 3 21:38:50 2020 +0800

    [TUBEMQ-267]Create C/C++ Message class (#182)
---
 .../tubemq-client-cpp/inc/message.h                |  78 +++++++
 .../tubemq-client-cpp/src/message.cc               | 249 +++++++++++++++++++++
 2 files changed, 327 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/message.h b/tubemq-client-twins/tubemq-client-cpp/inc/message.h
new file mode 100644
index 0000000..20d7fcf
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/message.h
@@ -0,0 +1,78 @@
+/**
+ * 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 _TUBEMQ_CLIENT_MESSAGE_H_
+#define _TUBEMQ_CLIENT_MESSAGE_H_
+
+
+#include <list>
+#include <map>
+#include <string>
+#include <stdio.h>
+
+
+namespace tubemq {
+
+using namespace std;
+
+class Message {
+ public:
+  Message();
+  Message(const Message& target);
+  Message(const string& topic, const char* data, int datalen);
+  virtual ~Message();
+  Message& operator=(const Message& target);
+  const long GetMessageId() const;
+  void SetMessageId(long message_id);
+  const string& GetTopic() const;
+  void SetTopic(const string& topic);
+  const char* GetData() const;
+  int GetDataLength() const;
+  void setData(const char* data, int datalen);
+  const int GetFlag() const;
+  void SetFlag(int flag);
+  const map<string, string>& GetProperties() const;
+  int GetProperties(string& attribute);
+  bool HasProperty(const string& key);
+  bool GetProperty(const string& key, string& value);
+  bool GetFilterItem(string& value);
+  bool AddProperty(string& err_info, const string& key, const string& value);  
+
+ private:
+  void clearData();
+  void copyData(const char* data, int datalen);  
+  void copyProperties(const map<string, string>& properties);
+
+  
+ private:
+  string topic_;
+  char* data_;
+  int   datalen_;
+  long  message_id_;
+  int   flag_;  
+  map<string, string> properties_;
+};
+
+}
+
+
+
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
new file mode 100644
index 0000000..c5162e3
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -0,0 +1,249 @@
+/**
+ * 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 <sstream> 
+#include <string.h> 
+
+#include "message.h"
+#include "utils.h"
+#include "const_config.h"
+
+
+namespace tubemq {
+
+
+// message flag's properties settings
+static const int kMsgFlagIncProperties = 0x01;
+// reserved property key Filter Item
+static const string kRsvPropKeyFilterItem = "$msgType$";
+// reserved property key message send time
+static const string kRsvPropKeyMsgTime = "$msgTime$";
+
+
+Message::Message() {
+  this->topic_      = "";
+  this->flag_       = 0;
+  this->message_id_ = config::kInvalidValue;
+  this->data_       = NULL;
+  this->datalen_    = 0;
+  this->properties_.clear();
+}
+
+Message::Message(const Message& target) {
+  this->topic_      = target.topic_;
+  this->message_id_ = target.message_id_;
+  copyData(target.data_, target.datalen_);
+  copyProperties(target.properties_);
+  this->flag_       = target.flag_;
+}
+
+Message::Message(const string& topic, const char* data, int datalen) {
+  this->topic_      = topic;
+  this->flag_       = 0;
+  this->message_id_ = config::kInvalidValue;
+  copyData(data, datalen);
+  this->properties_.clear();
+}
+
+Message::~Message() {
+  clearData();
+}
+
+Message& Message::operator=(const Message& target) {
+  if (this == &target) 
+    return *this;
+  this->topic_      = target.topic_;
+  this->message_id_ = target.message_id_;
+  clearData();
+  copyData(target.data_, target.datalen_);
+  copyProperties(target.properties_);
+  this->flag_       = target.flag_;
+  return *this;
+}
+
+const long Message::GetMessageId() const {
+  return this->message_id_;
+}
+
+void Message::SetMessageId(long message_id) {
+  this->message_id_ = message_id;
+}
+
+const string& Message::GetTopic() const {
+  return this->topic_;
+}
+
+void Message::SetTopic(const string& topic) {
+  this->topic_ = topic;
+}
+
+const char* Message::GetData() const {
+  return this->data_;
+}
+
+int Message::GetDataLength() const {
+  return this->datalen_;
+}
+
+void Message::setData(const char* data, int datalen) {
+  clearData();
+  copyData(data, datalen);
+}
+
+const int Message::GetFlag() const {
+  return this->flag_;
+}
+
+void Message::SetFlag(int flag) {
+  this->flag_ = flag;
+}
+
+const map<string, string>& Message::GetProperties() const {
+  return this->properties_;
+}
+
+int Message::GetProperties(string& attribute) {
+  attribute.clear();
+  map<string, string>::iterator it_map;
+  for (it_map = this->properties_.begin(); it_map != this->properties_.end(); ++it_map) {
+    if (!attribute.empty()) {
+      attribute += delimiter::kDelimiterComma;
+    }
+    attribute += it_map->first;
+    attribute += delimiter::kDelimiterEqual;
+    attribute += it_map->second;
+  }
+  return attribute.length();
+}
+
+bool Message::HasProperty(const string& key) {
+  map<string, string>::iterator it_map;
+  string trimed_key = Utils::Trim(key);
+  if (!trimed_key.empty()) {
+    it_map = this->properties_.find(trimed_key);
+    if (it_map != this->properties_.end()) {
+      return true;
+    }
+  }
+  return false;  
+}
+
+bool Message::GetProperty(const string& key, string& value) {
+  map<string, string>::iterator it_map;
+  string trimed_key = Utils::Trim(key);
+  if (!trimed_key.empty()) {
+    it_map = this->properties_.find(trimed_key);
+    if (it_map != this->properties_.end()) {
+      value = it_map->second;
+      return true;
+    }
+  }
+  return false;  
+}
+
+bool Message::GetFilterItem(string& value) {
+  return GetProperty(kRsvPropKeyFilterItem, value);
+}
+
+bool Message::AddProperty(string& err_info, const string& key, const string& value) {
+  string trimed_key = Utils::Trim(key);
+  string trimed_value = Utils::Trim(value);
+  if (trimed_key.empty() || trimed_value.empty()) {
+    err_info = "Not allowed null value of parmeter key or value";
+    return false;
+  }
+  if ((string::npos != trimed_key.find(delimiter::kDelimiterComma)) 
+    ||(string::npos != trimed_key.find(delimiter::kDelimiterEqual))) {
+    stringstream ss;
+    ss << "Reserved token '";
+    ss << delimiter::kDelimiterComma;
+    ss << "' or '";
+    ss << delimiter::kDelimiterEqual;
+    ss << "' in parmeter key!";
+    err_info = ss.str();
+    return false;
+  }
+  if ((string::npos != trimed_value.find(delimiter::kDelimiterComma)) 
+    ||(string::npos != trimed_value.find(delimiter::kDelimiterEqual))) {
+    stringstream ss;
+    ss << "Reserved token '";
+    ss << delimiter::kDelimiterComma;
+    ss << "' or '";
+    ss << delimiter::kDelimiterEqual;
+    ss << "' in parmeter value!";
+    err_info = ss.str();
+    return false;
+  }
+  if(trimed_key == kRsvPropKeyFilterItem 
+    || trimed_key == kRsvPropKeyMsgTime) {
+    stringstream ss;
+    ss << "Reserved token '";
+    ss << kRsvPropKeyFilterItem;
+    ss << "' or '";
+    ss << kRsvPropKeyMsgTime;
+    ss << "' must not be used in parmeter key!";
+    err_info = ss.str();
+    return false;
+  }
+  // add key and value
+  this->properties_[trimed_key] = trimed_value;
+  if (!this->properties_.empty()) {
+      this->flag_ |= kMsgFlagIncProperties;
+  }
+  err_info = "Ok";
+  return true;
+}
+
+void Message::clearData() {
+  if (this->data_ != NULL) {
+    delete[] this->data_;
+    this->data_ = NULL;
+    this->datalen_ = 0;
+  }
+}
+
+void Message::copyData(const char* data, int datalen) {
+  if (data == NULL) {
+    this->data_ = NULL;
+    this->datalen_ = 0;
+  } else {
+    this->datalen_ = datalen;
+    this->data_ = new char[datalen];
+    memset(this->data_, 0, datalen);
+    memcpy(this->data_, data, datalen);
+  }
+}
+
+void Message::copyProperties(const map<string, string>& properties) {
+  this->properties_.clear();
+  map<string, string>::const_iterator it_map;
+  for (it_map = properties.begin(); it_map != properties.end(); ++it_map) {
+    this->properties_[it_map->first] = it_map->second;
+  }
+  if (!this->properties_.empty()) {
+    this->flag_ |= kMsgFlagIncProperties;
+  }
+}
+
+
+}
+
+
+
+


[incubator-tubemq] 41/50: [TUBEMQ-290]C++ SDK TCP Connect (#252)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit ad04f00c4a2bf841399ba61d168d22772166c03a
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Sat Sep 12 09:35:35 2020 +0800

    [TUBEMQ-290]C++ SDK TCP Connect (#252)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/src/connection.cc            |  25 ++++
 .../tubemq-client-cpp/src/connection.h             | 114 ++++++++++++++++++
 .../tubemq-client-cpp/src/connection_pool.h        | 128 +++++++++++++++++++++
 3 files changed, 267 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/connection.cc b/tubemq-client-twins/tubemq-client-cpp/src/connection.cc
new file mode 100644
index 0000000..0a5949f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/connection.cc
@@ -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.
+ */
+
+#include "connection.h"
+
+using namespace tubemq;
+
+UniqueSeqId Connection::unique_id_;
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/connection.h b/tubemq-client-twins/tubemq-client-cpp/src/connection.h
new file mode 100644
index 0000000..30db70b
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/connection.h
@@ -0,0 +1,114 @@
+/**
+ * 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 _TUBEMQ_CONNECTION_
+#define _TUBEMQ_CONNECTION_
+
+#include <stdlib.h>
+
+#include <chrono>
+#include <ctime>
+#include <deque>
+#include <exception>
+#include <functional>
+#include <iostream>
+#include <sstream>
+#include <string>
+#include <unordered_map>
+
+#include "any.h"
+#include "buffer.h"
+#include "noncopyable.h"
+#include "transport.h"
+#include "unique_seq_id.h"
+
+namespace tubemq {
+
+// Check Package is done
+using ProtocalCheckerFunction = std::function<int(BufferPtr&, Any&, uint32_t&, bool&, size_t&)>;
+
+class Connection : noncopyable {
+ public:
+  enum Status { kConnecting, kConnected, kDisconnected };
+  Connection()
+      : connect_id_(unique_id_.Next()),
+        status_(kConnecting),
+        recv_time_(std::time(nullptr)),
+        package_length_(0),
+        create_time_(std::time(nullptr)) {
+    formatContextString();
+  }
+  Connection(const std::string& ip, uint16_t port)
+      : ip_(ip),
+        port_(port),
+        connect_id_(unique_id_.Next()),
+        status_(kConnecting),
+        recv_time_(std::time(nullptr)),
+        package_length_(0),
+        create_time_(std::time(nullptr)) {
+    formatContextString();
+  }
+  virtual ~Connection() {}
+
+  virtual void Close() = 0;
+
+  virtual void AsyncWrite(RequestContextPtr& req) = 0;
+
+  Status GetStatus() const { return status_; }
+  uint32_t GetConnectID() const { return connect_id_; }
+
+  inline bool IsStop() const { return status_ == kDisconnected; }
+  inline bool IsConnected() const { return status_ == kConnected; }
+
+  void SetCloseNotifier(CloseNotifier func) { notifier_ = func; }
+
+  void SetProtocalCheck(ProtocalCheckerFunction func) { check_ = func; }
+
+  inline std::time_t GetRecvTime() const { return recv_time_; }
+
+  inline const std::string& ToString() const { return context_string_; }
+
+ private:
+  void formatContextString() {
+    std::stringstream stream;
+    stream << "[id:" << connect_id_ << "]"
+           << "[time:" << create_time_ << "]";
+    context_string_ += stream.str();
+  }
+
+ protected:
+  CloseNotifier notifier_;
+  ProtocalCheckerFunction check_;
+  std::string ip_;
+  uint16_t port_;
+  uint32_t connect_id_;
+  std::atomic<Status> status_;
+  std::string context_string_;  // for log
+  std::time_t recv_time_;
+  size_t package_length_;
+
+ private:
+  std::time_t create_time_;
+  static UniqueSeqId unique_id_;
+};
+using ConnectionPtr = std::shared_ptr<Connection>;
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_CONNECTION_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
new file mode 100644
index 0000000..f3ec067
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/connection_pool.h
@@ -0,0 +1,128 @@
+/**
+ * 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 _TUBEMQ_CONNECTION_POLL_
+#define _TUBEMQ_CONNECTION_POLL_
+
+#include <stdlib.h>
+
+#include <chrono>
+#include <ctime>
+#include <deque>
+#include <exception>
+#include <functional>
+#include <iostream>
+#include <string>
+#include <unordered_map>
+
+#include "asio.hpp"
+#include "client_connection.h"
+#include "connection.h"
+#include "const_rpc.h"
+#include "logger.h"
+#include "noncopyable.h"
+#include "transport.h"
+
+namespace tubemq {
+class ConnectionPool : noncopyable {
+ public:
+  explicit ConnectionPool(ExecutorPoolPtr& executor_pool)
+      : executor_pool_(executor_pool), regular_timer_(executor_pool_->Get()->CreateSteadyTimer()) {
+    regular_timer_->expires_after(std::chrono::seconds(kRegularTimerSecond));
+    regular_timer_->async_wait([this](const std::error_code& ec) { ClearInvalidConnect(ec); });
+  }
+  ~ConnectionPool() { Clear(); }
+
+  void Clear() {
+    Lock lock(mutex_);
+    for (auto& connection : connection_pool_) {
+      connection.second->Close();
+    }
+    connection_pool_.clear();
+  }
+
+  void ClearInvalidConnect(const std::error_code& ec) {
+    if (ec) {
+      return;
+    }
+    Lock lock(mutex_);
+    for (auto it = connection_pool_.begin(); it != connection_pool_.end();) {
+      if (it->second->IsStop()) {
+        LOG_INFO("connection pool clear stop connect:%s", it->second->ToString().c_str());
+        it = connection_pool_.erase(it);
+        continue;
+      }
+      if (it->second->GetRecvTime() + rpc_config::kRpcInvalidConnectOverTime < std::time(nullptr)) {
+        it->second->Close();
+        it = connection_pool_.erase(it);
+        LOG_ERROR("connection pool clear overtime connect:%s", it->second->ToString().c_str());
+        continue;
+      }
+      ++it;
+    }
+    regular_timer_->expires_after(std::chrono::seconds(kRegularTimerSecond));
+    regular_timer_->async_wait([this](const std::error_code& ec) { ClearInvalidConnect(ec); });
+  }
+
+  ConnectionPtr GetConnection(RequestContextPtr& request) {
+    std::string key = generateConnectionKey(request);
+
+    Lock lock(mutex_);
+    auto it = connection_pool_.find(key);
+    if (it != connection_pool_.end() && !(it->second->IsStop())) {
+      return it->second;
+    }
+    auto executor = executor_pool_->Get();
+    auto connect = std::make_shared<ClientConnection>(executor, request->ip_, request->port_);
+    connection_pool_[key] = connect;
+    connect->SetCloseNotifier(request->close_notifier_);
+
+    auto codec = request->codec_;
+    connect->SetProtocalCheck([codec](BufferPtr& in, Any& out, uint32_t& request_id,
+                                      bool& has_request_id, size_t& package_length) -> int {
+      return codec->Check(in, out, request_id, has_request_id, package_length);
+    });
+    return connect;
+  }
+
+ private:
+  inline std::string generateConnectionKey(const RequestContextPtr& request) {
+    std::string key;
+    key += request->ip_;
+    key += "_";
+    key += std::to_string(request->port_);
+    key += "_";
+    key += std::to_string(request->connection_pool_id_);
+    return key;
+  }
+
+ private:
+  std::mutex mutex_;
+  std::unordered_map<std::string, ConnectionPtr> connection_pool_;
+  ExecutorPoolPtr executor_pool_;
+  SteadyTimerPtr regular_timer_;
+  static const uint32_t kRegularTimerSecond = 20;
+  typedef std::unique_lock<std::mutex> Lock;
+};
+
+using ConnectionPoolPtr = std::shared_ptr<ConnectionPool>;
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_CONNECTION_POLL_
+


[incubator-tubemq] 21/50: [TUBEMQ-262] Create C++ flow control handler (#192)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit e6df8da7312fbee8873a9cebe9748df1320fcc35
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 6 15:01:45 2020 +0000

    [TUBEMQ-262] Create C++ flow control handler (#192)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/inc/flowctrl_def.h           | 162 +++++
 tubemq-client-twins/tubemq-client-cpp/inc/logger.h |  13 +-
 .../tubemq-client-cpp/src/flowctrl_def.cc          | 787 +++++++++++++++++++++
 3 files changed, 956 insertions(+), 6 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
new file mode 100644
index 0000000..cff7cc6
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/flowctrl_def.h
@@ -0,0 +1,162 @@
+/**
+ * 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 _TUBEMQ_CLIENT_FLOW_CONTROL_H_
+#define _TUBEMQ_CLIENT_FLOW_CONTROL_H_
+
+#include <map>
+#include <list>
+#include <string>
+#include <vector>
+#include <algorithm>
+#include <rapidjson/document.h>
+#include "atomic_def.h"
+
+
+
+namespace tubemq {
+
+using namespace std;
+
+class FlowCtrlResult {
+ public:
+  FlowCtrlResult();
+  FlowCtrlResult(long datasize_limit, int freqms_limit);
+  FlowCtrlResult& operator=(const FlowCtrlResult& target);
+  void SetDataDltAndFreqLimit(long datasize_limit, int freqms_limit);
+  void SetDataSizeLimit(long datasize_limit);
+  void SetFreqMsLimit(int freqms_limit);
+  long GetDataSizeLimit();
+  int GetFreqMsLimit();
+
+ private:
+  long datasize_limit_;  
+  int  freqms_limit_;
+};
+
+
+class FlowCtrlItem {
+ public:
+  FlowCtrlItem();
+  FlowCtrlItem(int type,int zero_cnt,int freqms_limit);
+  FlowCtrlItem(int type, 
+    int datasize_limit,int freqms_limit,int min_data_filter_freqms);
+  FlowCtrlItem(int type, int start_time, 
+    int end_time, long datadlt_m, long datasize_limit, int freqms_limit);
+  FlowCtrlItem& operator=(const FlowCtrlItem& target);
+  void Clear();
+  void ResetFlowCtrlValue(int type, 
+    int datasize_limit,int freqms_limit,int min_data_filter_freqms);
+  int GetFreLimit(int msg_zero_cnt);
+  bool GetDataLimit(long datadlt_m, int curr_time, FlowCtrlResult& flowctrl_result);
+  const int GetType() const {
+    return type_;
+  }
+  const int GetZeroCnt() const {
+    return zero_cnt_;
+  }
+  const int GetStartTime() const {
+    return start_time_;
+  }
+  const int GetEndTime() const {
+    return end_time_;
+  }
+  const long GetDataSizeLimit() const { 
+    return datasize_limit_;
+  }
+  const int GetFreqMsLimit() const {
+    return freqms_limit_;
+  }
+  const long GetDltInM() const {
+    return datadlt_m_;
+  }
+
+ private:
+  int  type_;
+  int  start_time_;
+  int  end_time_;
+  long datadlt_m_;
+  long datasize_limit_;
+  int  freqms_limit_;
+  int  zero_cnt_;
+};
+
+class FlowCtrlRuleHandler {
+ public:
+  FlowCtrlRuleHandler();
+  ~FlowCtrlRuleHandler();
+  void UpdateDefFlowCtrlInfo(bool is_default, 
+    int qrypriority_id, long flowctrl_id, const string& flowctrl_info);
+  bool GetCurDataLimit(long last_datadlt,FlowCtrlResult& flowctrl_result);
+  int GetCurFreqLimitTime(int msg_zero_cnt, int received_limit);
+  int GetMinZeroCnt() { return this->min_zero_cnt_.Get();}
+  int GetQryPriorityId() { 
+    return this->qrypriority_id_.Get();
+  }
+  void SetQryPriorityId(int qrypriority_id) { 
+    this->qrypriority_id_.Set(qrypriority_id);
+  }
+  long GetFlowCtrlId() { 
+    return this->flowctrl_id_.Get();
+  }
+  const FlowCtrlItem& GetFilterCtrlItem() const {
+    return this->filter_ctrl_item_;
+  }
+  const string& GetFlowCtrlInfo() const { 
+    return this->flowctrl_info_;
+  }
+
+ private:
+  void initialStatisData();
+  void clearStatisData();
+  static bool compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2);
+  static bool compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2);
+  bool parseStringMember(string &err_info, const rapidjson::Value& root, 
+    const char* key, string& value, bool compare_value, string required_val);
+  bool parseLongMember(string &err_info, const rapidjson::Value& root, 
+    const char* key, long& value, bool compare_value, long required_val);
+  bool parseIntMember(string &err_info, const rapidjson::Value& root, 
+    const char* key, int& value, bool compare_value, int required_val);
+  bool parseFlowCtrlInfo(const string& flowctrl_info, map<int,vector<FlowCtrlItem> >& flowctrl_info_map);
+  bool parseDataLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowCtrlItems);
+  bool parseFreqLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
+  bool parseLowFetchLimit(string& err_info, const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items);
+  bool parseTimeMember(string& err_info, const rapidjson::Value& root, const char* key, int& value);
+
+ private:
+  AtomicLong    flowctrl_id_;
+  AtomicInteger qrypriority_id_;
+  string        flowctrl_info_;
+  AtomicInteger min_zero_cnt_;
+  AtomicLong    min_datadlt_limt_;
+  AtomicInteger datalimit_start_time_;
+  AtomicInteger datalimit_end_time_;
+  FlowCtrlItem  filter_ctrl_item_;
+  map<int, vector<FlowCtrlItem> > flowctrl_rules_;
+  pthread_rwlock_t configrw_lock_;
+  long last_update_time_;
+};
+
+  
+
+}
+
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
index 44c28b7..8c5f9b3 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/logger.h
@@ -22,6 +22,7 @@
 
 #include <string>
 #include <vector>
+#include <stdint.h>
 
 namespace tubemq {
 class Logger;
@@ -36,13 +37,13 @@ Logger& GetLogger();
     }                                                                                                       \
   }
 
-#define LOG_TRACE(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
-#define LOG_DEBUG(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
-#define LOG_INFO(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
-#define LOG_WARN(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
-#define LOG_ERROR(fmt, ...) LOG_TUBE(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
+#define LOG_TRACE(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kTrace, fmt, ##__VA_ARGS__)
+#define LOG_DEBUG(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kDebug, fmt, ##__VA_ARGS__)
+#define LOG_INFO(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kInfo, fmt, ##__VA_ARGS__)
+#define LOG_WARN(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kWarn, fmt, ##__VA_ARGS__)
+#define LOG_ERROR(fmt, ...) LOG_TUBEMQ(tubemq::GetLogger(), tubemq::Logger::kError, fmt, ##__VA_ARGS__)
 
-#define LOG_TUBE(logger, level, fmt, ...)                                                                      \
+#define LOG_TUBEMQ(logger, level, fmt, ...)                                                                      \
   {                                                                                                            \
     if (logger.IsEnable(level)) {                                                                              \
       logger.Write("[%s:%d][%s]" fmt, __func__, __LINE__, tubemq::Logger::Level2String(level), ##__VA_ARGS__); \
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
new file mode 100644
index 0000000..bdc8138
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -0,0 +1,787 @@
+/**
+ * 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 <time.h>
+#include <unistd.h>
+#include <sstream>
+#include "utils.h"
+#include "logger.h"
+#include "const_config.h"
+#include "flowctrl_def.h"
+
+
+
+namespace tubemq {
+
+
+FlowCtrlResult::FlowCtrlResult() {
+  this->datasize_limit_ = config::kMaxIntValue;  
+  this->freqms_limit_ = 0;
+}
+
+FlowCtrlResult::FlowCtrlResult(long datasize_limit, int freqms_limit) {
+  this->datasize_limit_ = datasize_limit;  
+  this->freqms_limit_ = freqms_limit;
+}
+
+FlowCtrlResult& FlowCtrlResult::operator=(const FlowCtrlResult& target) {
+  if (this == &target) 
+    return *this;
+  this->datasize_limit_ = target.datasize_limit_;  
+  this->freqms_limit_ = target.freqms_limit_;
+  return *this;
+}
+
+void FlowCtrlResult::SetDataDltAndFreqLimit(long datasize_limit, int freqms_limit) {
+  this->datasize_limit_ = datasize_limit;
+  this->freqms_limit_ = freqms_limit;
+}
+
+void FlowCtrlResult::SetDataSizeLimit(long datasize_limit) {
+  this->datasize_limit_ = datasize_limit;
+}
+
+void FlowCtrlResult::SetFreqMsLimit(int freqms_limit) {
+  this->freqms_limit_ = freqms_limit;
+}
+
+long FlowCtrlResult::GetDataSizeLimit() {
+  return this->datasize_limit_;
+}
+
+int FlowCtrlResult::GetFreqMsLimit() {
+  return this->freqms_limit_;
+}
+
+
+FlowCtrlItem::FlowCtrlItem() {
+  this->type_           = 0;
+  this->start_time_     = 2500;
+  this->end_time_       = config::kInvalidValue;
+  this->datadlt_m_      = config::kInvalidValue;
+  this->datasize_limit_ = config::kInvalidValue;
+  this->freqms_limit_   = config::kInvalidValue;
+  this->zero_cnt_       = config::kInvalidValue;
+}
+
+FlowCtrlItem::FlowCtrlItem(int type,int zero_cnt,int freqms_limit) {
+  this->type_           = type;
+  this->start_time_     = 2500;
+  this->end_time_       = config::kInvalidValue;
+  this->datadlt_m_      = config::kInvalidValue;
+  this->datasize_limit_ = config::kInvalidValue;
+  this->freqms_limit_   = freqms_limit;
+  this->zero_cnt_       = zero_cnt;
+}
+
+FlowCtrlItem::FlowCtrlItem(int type, 
+  int datasize_limit,int freqms_limit,int min_data_filter_freqms) {
+  this->type_           = type;
+  this->start_time_     = 2500;
+  this->end_time_       = config::kInvalidValue;
+  this->datadlt_m_      = config::kInvalidValue;
+  this->datasize_limit_ = datasize_limit;
+  this->freqms_limit_   = freqms_limit;
+  this->zero_cnt_       = min_data_filter_freqms;
+}
+
+FlowCtrlItem::FlowCtrlItem(int type, int start_time, int end_time, 
+  long datadlt_m, long datasize_limit, int freqms_limit) {
+  this->type_           = type;
+  this->start_time_     = start_time;
+  this->end_time_       = end_time;
+  this->datadlt_m_      = datadlt_m;
+  this->datasize_limit_ = datasize_limit;
+  this->freqms_limit_   = freqms_limit;
+  this->zero_cnt_       = config::kInvalidValue;
+}
+
+FlowCtrlItem& FlowCtrlItem::operator=(const FlowCtrlItem& target) {
+  if (this == &target) 
+    return *this;
+  this->type_           = target.type_;
+  this->start_time_     = target.start_time_;
+  this->end_time_       = target.end_time_;
+  this->datadlt_m_      = target.datadlt_m_;
+  this->datasize_limit_ = target.datasize_limit_;
+  this->freqms_limit_   = target.freqms_limit_;
+  this->zero_cnt_       = target.zero_cnt_;
+    return *this;
+}
+
+int FlowCtrlItem::GetFreLimit(int msg_zero_cnt) {
+  if (this->type_ != 1) {
+    return -1;
+  }
+  if (msg_zero_cnt >= this->zero_cnt_) {
+    return this->freqms_limit_;
+  }
+  return -1;
+}
+
+void FlowCtrlItem::ResetFlowCtrlValue(int type, 
+  int datasize_limit,int freqms_limit,int min_data_filter_freqms) {
+  this->type_           = type;
+  this->start_time_     = 2500;
+  this->end_time_       = config::kInvalidValue;
+  this->datadlt_m_      = config::kInvalidValue;
+  this->datasize_limit_ = datasize_limit;
+  this->freqms_limit_   = freqms_limit;
+  this->zero_cnt_       = min_data_filter_freqms;
+}
+
+void FlowCtrlItem::Clear() {
+  this->type_           = 0;
+  this->start_time_     = 2500;
+  this->end_time_       = config::kInvalidValue;
+  this->datadlt_m_      = config::kInvalidValue;
+  this->datasize_limit_ = config::kInvalidValue;
+  this->freqms_limit_   = config::kInvalidValue;
+  this->zero_cnt_       = config::kInvalidValue;
+}
+
+bool FlowCtrlItem::GetDataLimit(long datadlt_m, int curr_time, FlowCtrlResult& flowctrl_result) {
+  if (this->type_ != 0 || datadlt_m <= this->datadlt_m_ ) {
+    return false;
+  }
+  if (curr_time < this->start_time_ 
+    || curr_time > this->end_time_) {
+    return false;
+  }
+  flowctrl_result.SetDataDltAndFreqLimit(this->datasize_limit_,this->freqms_limit_);
+  return true;
+}
+
+
+FlowCtrlRuleHandler::FlowCtrlRuleHandler() {
+  this->flowctrl_id_.GetAndSet(config::kInvalidValue);
+  this->flowctrl_info_ = "";
+  this->min_zero_cnt_.Set(config::kMaxIntValue);
+  this->qrypriority_id_.Set(config::kInvalidValue);
+  this->min_datadlt_limt_.Set(config::kMaxLongValue);
+  this->datalimit_start_time_.Set(2500);
+  this->datalimit_end_time_.Set(config::kInvalidValue);
+  this->last_update_time_ = Utils::GetCurrentTimeMillis();
+  pthread_rwlock_init(&configrw_lock_, NULL);  
+}
+
+FlowCtrlRuleHandler::~FlowCtrlRuleHandler() {
+  pthread_rwlock_destroy(&configrw_lock_);
+}
+
+void FlowCtrlRuleHandler::UpdateDefFlowCtrlInfo(bool is_default, 
+  int qrypriority_id, long flowctrl_id, const string& flowctrl_info) {
+  bool result;
+  map<int, vector<FlowCtrlItem> > tmp_flowctrl_map;
+  if (flowctrl_id == this->flowctrl_id_.Get()) {
+      return;
+  }
+  long curr_flowctrl_id = this->flowctrl_id_.Get();
+  if (flowctrl_info.length() > 0) {
+    result = parseFlowCtrlInfo(flowctrl_info, tmp_flowctrl_map);
+  }
+  pthread_rwlock_wrlock(&this->configrw_lock_);
+  this->flowctrl_id_.Set(flowctrl_id);
+  this->qrypriority_id_.Set(qrypriority_id);
+  clearStatisData();
+  if (tmp_flowctrl_map.empty()) {
+    this->flowctrl_rules_.clear();
+    this->flowctrl_info_ = "";
+  } else {
+    this->flowctrl_rules_ = tmp_flowctrl_map;
+    this->flowctrl_info_ = flowctrl_info;
+    initialStatisData();
+  }
+  this->last_update_time_ = Utils::GetCurrentTimeMillis();
+  pthread_rwlock_unlock(&this->configrw_lock_);
+  if (is_default) {
+    LOG_INFO("[Flow Ctrl] Default FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id, flowctrl_id);
+  } else {
+    LOG_INFO("[Flow Ctrl] Group FlowCtrl's flowctrl_id from %ld to %ld\n", curr_flowctrl_id, flowctrl_id);
+  }
+  return;
+
+}
+
+
+void FlowCtrlRuleHandler::initialStatisData() {
+  vector<FlowCtrlItem>::iterator it_vec;
+  map<int, vector<FlowCtrlItem> >::iterator it_map;
+
+  it_map = this->flowctrl_rules_.find(0);
+  if (it_map != this->flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+      if (it_vec->GetType() != 0) {
+        continue;
+      }
+
+      if (it_vec->GetDltInM() < this->min_datadlt_limt_.Get()) {
+        this->min_datadlt_limt_.Set(it_vec->GetDltInM());
+      }
+      if (it_vec->GetStartTime() < this->datalimit_start_time_.Get()) {
+        this->datalimit_start_time_.Set(it_vec->GetStartTime());
+      }
+      if (it_vec->GetEndTime() > this->datalimit_end_time_.Get()) {
+        this->datalimit_end_time_.Set(it_vec->GetEndTime());
+      }
+    }
+  }
+  it_map = this->flowctrl_rules_.find(1);
+  if (it_map != this->flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+      if (it_vec->GetType() != 1) {
+          continue;
+      }
+      if (it_vec->GetZeroCnt() < this->min_zero_cnt_.Get()) {
+        this->min_zero_cnt_.Set(it_vec->GetZeroCnt());
+      }
+    }
+  }
+  it_map = this->flowctrl_rules_.find(3);
+  if (it_map != this->flowctrl_rules_.end()) {
+    for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+      if (it_vec->GetType() != 3) {
+        continue;
+      }
+      it_vec->GetDataSizeLimit();
+      this->filter_ctrl_item_.ResetFlowCtrlValue(3,
+          (int)(it_vec->GetDataSizeLimit()),it_vec->GetFreqMsLimit(),it_vec->GetZeroCnt());
+    }
+  }
+}
+
+void FlowCtrlRuleHandler::clearStatisData() {
+  this->min_zero_cnt_.GetAndSet(config::kMaxIntValue);
+  this->min_datadlt_limt_.GetAndSet(config::kMaxLongValue);
+  this->qrypriority_id_.Set(config::kInvalidValue);
+  this->datalimit_start_time_.Set(2500);
+  this->datalimit_end_time_.Set(config::kInvalidValue);
+  this->filter_ctrl_item_.Clear();
+}
+
+bool FlowCtrlRuleHandler::GetCurDataLimit(long last_datadlt, FlowCtrlResult& flowctrl_result) {
+  struct tm utc_tm;
+  vector<FlowCtrlItem>::iterator it_vec;
+  map<int, vector<FlowCtrlItem> >::iterator it_map;
+  time_t cur_time=time(NULL);
+
+  gmtime_r(&cur_time,&utc_tm);
+  int curr_time = (utc_tm.tm_hour+8)%24 * 100 + utc_tm.tm_min;
+  if ((last_datadlt < this->min_datadlt_limt_.Get()) 
+    || (curr_time < this->datalimit_start_time_.Get())
+    || (curr_time > this->datalimit_end_time_.Get())) {
+    return false;
+  }
+  it_map = this->flowctrl_rules_.find(0);
+  if (it_map == this->flowctrl_rules_.end()) {
+    return false;
+  }
+  for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+    if (it_vec->GetDataLimit(last_datadlt,curr_time,flowctrl_result)) {
+      return true;
+    }
+  }
+  return false;
+}
+
+int FlowCtrlRuleHandler::GetCurFreqLimitTime(int msg_zero_cnt, int received_limit)
+{
+  int rule_val = -2;
+  vector<FlowCtrlItem>::iterator it_vec;
+  map<int, vector<FlowCtrlItem> >::iterator it_map;
+
+  if (msg_zero_cnt < this->min_zero_cnt_.Get()) {
+    return received_limit;
+  }
+  it_map = this->flowctrl_rules_.find(1);
+  if (it_map == this->flowctrl_rules_.end()) {
+    return received_limit;
+  }
+  for (it_vec = it_map->second.begin(); it_vec != it_map->second.end(); ++it_vec) {
+    rule_val = it_vec->GetFreLimit(msg_zero_cnt);
+    if (rule_val >= 0) 
+    {
+      return rule_val;
+    }
+  }
+  return received_limit;
+}
+
+
+bool FlowCtrlRuleHandler::compareDataLimitQueue(const FlowCtrlItem& o1, const FlowCtrlItem& o2) {
+  if (o1.GetStartTime() >= o2.GetStartTime()) {
+    return true;
+  }
+  return false;
+}
+
+bool FlowCtrlRuleHandler::compareFeqQueue(const FlowCtrlItem& queue1, const FlowCtrlItem& queue2) {
+    return (queue1.GetZeroCnt() < queue2.GetZeroCnt());
+}
+
+bool FlowCtrlRuleHandler::parseFlowCtrlInfo(const string& flowctrl_info, 
+                                map<int,vector<FlowCtrlItem> >& flowctrl_info_map) {
+  int type;
+  string err_info;
+  stringstream ss;
+  rapidjson::Document doc;
+  // check flowctrl info length
+  if (flowctrl_info.length() == 0){
+    return false;
+  }
+  // parse flowctrl info
+  if (doc.Parse(flowctrl_info.c_str()).HasParseError()) {
+    LOG_ERROR("Parsing flowCtrlInfo failure! flowctrl_info=%s\n",flowctrl_info.c_str()); 
+    return false;
+  }
+  if (!doc.IsArray()) {
+    LOG_ERROR("flowCtrlInfo's value must be array! flowctrl_info=%s\n",flowctrl_info.c_str()); 
+    return false;
+  }
+  for (unsigned int i = 0; i < doc.Size(); i++) {
+    vector<FlowCtrlItem> flowctrl_item_vec;
+    const rapidjson::Value& node_item = doc[i];
+    if (!node_item.IsObject()) {
+      continue;
+    }
+    if (!parseIntMember(err_info, node_item, "type", type, false, -2)) {
+      ss << "Decode Failure: ";
+      ss << err_info;
+      ss << " of type field in parse flowctrl_info!";
+      err_info = ss.str();
+      LOG_ERROR("parse flowCtrlInfo failure %s", err_info.c_str()); 
+      return false;
+    }    
+    if (type < 0 || type > 3) {
+      ss << "type value must in [0,1,2,3] in index(";
+      ss << i;
+      ss << ") of flowctrl_info value!";
+      err_info = ss.str();
+      LOG_ERROR("parse flowCtrlInfo failure %s",err_info.c_str()); 
+      return false;
+    }
+    
+    switch (type) {
+      case 1: {
+        if (FlowCtrlRuleHandler::parseFreqLimit(err_info, node_item, flowctrl_item_vec)) {
+          flowctrl_info_map[1]=flowctrl_item_vec;
+        } else {
+          LOG_ERROR("parse flowCtrlInfo's freqLimit failure: %s",err_info.c_str()); 
+        }
+      }
+      break;
+       
+      case 3: {
+        if (FlowCtrlRuleHandler::parseLowFetchLimit(err_info, node_item, flowctrl_item_vec)) {
+            flowctrl_info_map[3]=flowctrl_item_vec;
+        } else {
+          LOG_ERROR("parse flowCtrlInfo's lowFetchLimit failure: %s",err_info.c_str()); 
+        }
+      }
+      break;
+
+      case 0: {
+        if (FlowCtrlRuleHandler::parseDataLimit(err_info, node_item, flowctrl_item_vec)) {
+          flowctrl_info_map[0]=flowctrl_item_vec;
+        } else {
+          LOG_ERROR("parse flowCtrlInfo's dataLimit failure: %s",err_info.c_str()); 
+        }
+      }
+      break;
+
+      default:
+        break;
+    }
+  }  
+  return true;
+}
+
+bool FlowCtrlRuleHandler::parseDataLimit(string& err_info, 
+                           const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+  int type_val;
+  stringstream ss;
+  string attr_sep = delimiter::kDelimiterColon;
+  string::size_type pos1;
+  if (!parseIntMember(err_info, root, "type", type_val, true, 0)) {
+    ss << "Decode Failure: ";
+    ss << err_info;
+    ss << " of type field in parse data limit!";
+    err_info = ss.str();
+    return false;
+  }
+  // check rule type
+  if (!root.HasMember("rule")) {
+    err_info = "rule field not existed";
+    return false;
+  }
+  if (!root["rule"].IsArray()) {
+    err_info = "Illegal value, rule must be list type";
+    return false;
+  }
+  // parse rule info
+  const rapidjson::Value& obj_set = root["rule"];
+  for (unsigned int index = 0 ; index < obj_set.Size() ; index++) {
+    int start_time = 0;
+    int end_time = 0;
+    long datadlt_m = 0;
+    long datasize_limit = 0;
+    int freqms_limit = 0;
+    const rapidjson::Value& node_item = obj_set[index];
+    if (!node_item.IsObject()) {
+        err_info = "Illegal rule'value item, must be dict type";
+        return false;
+    }
+    if (!parseTimeMember(err_info, node_item, "start", start_time)) {
+      return false;
+    }
+    if (!parseTimeMember(err_info, node_item, "end", end_time)) {
+      return false;
+    }
+    if (start_time > end_time) {
+      ss << "start value must lower than the End value in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }
+    if (!parseLongMember(err_info, node_item, "dltInM", datadlt_m, false, -1)) {
+      ss << "dltInM key is required in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }
+    if (datadlt_m <= 20) {
+      ss << "dltInM value must over than 20 in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }
+    datadlt_m = datadlt_m * 1024 * 1024;
+    if (!parseLongMember(err_info, node_item, "limitInM", datasize_limit, false, -1)) {
+      ss << "limitInM key is required in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }
+    if (datasize_limit < 0) {
+      ss << "limitInM value must over than equal or bigger than zero in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }    
+    datasize_limit = datasize_limit * 1024 * 1024;
+    if (!parseIntMember(err_info, node_item, "freqInMs", freqms_limit, false, -1)) {
+      ss << "freqInMs key is required in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }
+    if (freqms_limit < 200) {
+      ss << "freqInMs value must over than equal or bigger than 200 in index(";
+      ss << index;
+      ss << ") of data limit rule!";
+      err_info = ss.str();
+      return false;
+    }    
+    FlowCtrlItem flowctrl_item(0, start_time, end_time, datadlt_m, datasize_limit, freqms_limit);
+    flowctrl_items.push_back(flowctrl_item);
+  }
+  if (!flowctrl_items.empty()) {
+    std::sort(flowctrl_items.begin(), flowctrl_items.end(), compareDataLimitQueue);
+  }
+  err_info = "Ok";
+  return true;  
+}
+
+bool FlowCtrlRuleHandler::parseFreqLimit(string& err_info, 
+                           const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+  int type_val;
+  stringstream ss;
+
+  if (!parseIntMember(err_info, root, "type", type_val, true, 1)) {
+    ss << "Decode Failure: ";
+    ss << err_info;
+    ss << " of type field in parse freq limit!";
+    err_info = ss.str();
+    return false;
+  }
+  if (!root.HasMember("rule")) {
+    err_info = "rule field not existed";
+    return false;
+  }
+  if (!root["rule"].IsArray()) {
+    err_info = "Illegal value, rule must be list type";
+    return false;
+  }
+  // parse rule info
+  const rapidjson::Value& obj_set = root["rule"];
+  for (unsigned int i = 0 ; i < obj_set.Size() ; i++) {
+    int zeroCnt = -2;
+    int freqms_limit = -2;
+    const rapidjson::Value& node_item = obj_set[i];
+    if (!node_item.IsObject()) {
+      err_info = "Illegal rule'value item, must be dict type";
+      return false;
+    }
+    if (!parseIntMember(err_info, node_item, "zeroCnt", zeroCnt, false, -2)) {
+      ss << "Decode Failure: ";
+      ss << err_info;
+      ss << " of zeroCnt field in parse freq limit!";
+      err_info = ss.str();
+      return false;
+    }
+    if (!parseIntMember(err_info, node_item, "freqInMs", freqms_limit, false, -2)) {
+      ss << "Decode Failure: ";
+      ss << err_info;
+      ss << " of freqInMs field in parse freq limit!";
+      err_info = ss.str();
+      return false;
+    }
+    FlowCtrlItem flowctrl_item(1, zeroCnt, freqms_limit);
+    flowctrl_items.push_back(flowctrl_item);
+  }
+  if (!flowctrl_items.empty()) {
+    std::sort(flowctrl_items.begin(), flowctrl_items.end(), compareFeqQueue);
+  }
+  err_info = "Ok";
+  return true;  
+}
+
+bool FlowCtrlRuleHandler::parseLowFetchLimit(string& err_info, 
+                            const rapidjson::Value& root, vector<FlowCtrlItem>& flowctrl_items) {
+  int type_val;
+  stringstream ss;
+  if (!parseIntMember(err_info, root, "type", type_val, true, 3)) {
+    ss << "Decode Failure: ";
+    ss << err_info;
+    ss << " of type field in parse low fetch limit!";
+    err_info = ss.str();
+    return false;
+  }
+  if (!root.HasMember("rule")) {
+    err_info = "rule field not existed";
+    return false;
+  }
+  if (!root["rule"].IsArray()) {
+    err_info = "Illegal value, rule must be list type";
+    return false;
+  }
+  // parse rule info
+  const rapidjson::Value& node_item = root["rule"];
+  for (unsigned int i = 0 ; i < node_item.Size() ; i++) {
+    int norm_freq_ms = 0;
+    int filter_freq_ms = 0;
+    int min_filter_freq_ms = 0;
+    FlowCtrlItem flowctrl_item;
+    const rapidjson::Value& node_item = node_item[i];
+    if (!node_item.IsObject()) {
+      err_info = "Illegal rule'value item, must be dict type";
+      return false;
+    }
+    if (node_item.HasMember("filterFreqInMs") 
+      || node_item.HasMember("minDataFilterFreqInMs")) {
+      if (!parseIntMember(err_info, node_item, 
+        "filterFreqInMs", filter_freq_ms, false, -1)) {
+        ss << "Decode Failure: ";
+        ss << err_info;
+        ss << " of filterFreqInMs field in parse low fetch limit!";
+        err_info = ss.str();
+        return false;
+      }
+      if (!parseIntMember(err_info, node_item, 
+        "minDataFilterFreqInMs", min_filter_freq_ms, false, -1)) {
+        ss << "Decode Failure: ";
+        ss << err_info;
+        ss << " of minDataFilterFreqInMs field in parse low fetch limit!";
+        err_info = ss.str();
+        return false;
+      }
+      if (filter_freq_ms < 0 || filter_freq_ms > 300000) {
+        ss << "Decode Failure: ";
+        ss << "filterFreqInMs value must in [0, 300000] in index(";
+        ss << i;
+        ss << ") of low fetch limit rule!";
+        err_info = ss.str();
+        return false;
+      }
+      if (min_filter_freq_ms < 0 || min_filter_freq_ms > 300000) {
+        ss << "Decode Failure: ";
+        ss << "minDataFilterFreqInMs value must in [0, 300000] in index(";
+        ss << i;
+        ss << ") of low fetch limit rule!";
+        err_info = ss.str();
+        return false;
+      }
+      if (min_filter_freq_ms < filter_freq_ms) {
+        ss << "Decode Failure: ";
+        ss << "minDataFilterFreqInMs must lower than filterFreqInMs in index(";
+        ss << i;
+        ss << ") of low fetch limit rule!";
+        err_info = ss.str();
+        return false;
+      }
+    }
+    if (node_item.HasMember("normFreqInMs")) {
+      if (!parseIntMember(err_info, node_item, 
+        "normFreqInMs", norm_freq_ms, false, -1)) {
+        ss << "Decode Failure: ";
+        ss << err_info;
+        ss << " of normFreqInMs field in parse low fetch limit!";
+        err_info = ss.str();
+        return false;
+      }
+      if (norm_freq_ms < 0 || norm_freq_ms > 300000) {
+        ss << "Decode Failure: ";
+        ss << "normFreqInMs value must in [0, 300000] in index(";
+        ss << i;
+        ss << ") of low fetch limit rule!";
+        err_info = ss.str();
+        return false;
+      }        
+    }
+    flowctrl_item.ResetFlowCtrlValue(3,norm_freq_ms,filter_freq_ms,min_filter_freq_ms);
+    flowctrl_items.push_back(flowctrl_item);
+  }
+  err_info = "Ok";
+  return true;
+}
+
+bool FlowCtrlRuleHandler::parseStringMember(string& err_info, const rapidjson::Value& root, 
+                            const char* key, string& value, bool compare_value, string required_val) {
+  // check key if exist
+  if (!root.HasMember(key)) {
+    err_info = "Field not existed";
+    return false;
+  }
+  if (!root[key].IsString()) {
+    err_info = "Illegal value, must be string type";
+    return false;
+  }
+  
+  if (compare_value) {
+    if (root[key].GetString() != required_val) {
+      err_info = "Illegal value, not required value content";
+      return false;
+    }
+  }
+  value = root[key].GetString();
+  return true;
+}
+
+bool FlowCtrlRuleHandler::parseLongMember(string& err_info, const rapidjson::Value& root, 
+                            const char* key, long& value, bool compare_value, long required_val) {
+  if (!root.HasMember(key)) {
+    err_info = "Field not existed";
+    return false;
+  }
+  if (!root[key].IsNumber()) {
+    err_info = "Illegal value, must be number type";
+    return false;
+  }
+  if (compare_value) {
+    if ((long)root[key].GetInt64() != required_val) {
+      err_info = "Illegal value, not required value content";
+      return false;
+    }
+  }
+  value = (long)root[key].GetInt64();
+  return true;
+}
+
+bool FlowCtrlRuleHandler::parseIntMember(string& err_info, const rapidjson::Value& root, 
+                            const char* key, int& value, bool compare_value, int required_val) {
+  if (!root.HasMember(key)) {
+    err_info = "Field not existed";
+    return false;
+  }
+  if (!root[key].IsInt()) {
+    err_info = "Illegal value, must be int type";
+    return false;
+  }
+  if (compare_value) {
+    if (root[key].GetInt() != required_val) {
+      err_info = "Illegal value, not required value content";
+      return false;
+    }
+  }
+  value = root[key].GetInt();
+  return true;
+}
+
+bool FlowCtrlRuleHandler::parseTimeMember(string& err_info, 
+                           const rapidjson::Value& root, const char* key, int& value) {
+  // check key if exist
+  stringstream ss;
+  if (!root.HasMember(key)) {
+    ss << "field ";
+    ss << key;
+    ss << " not existed!";
+    err_info = ss.str();
+    return false;
+  }
+  if (!root[key].IsString()) {
+    ss << "field ";
+    ss << key;
+    ss << " must be string type!";
+    err_info = ss.str();
+    return false;
+  }
+  string::size_type pos1;
+  string str_value = root[key].GetString();
+  string attr_sep = delimiter::kDelimiterColon;
+  pos1 = str_value.find(attr_sep);
+  if (string::npos == pos1) {
+    ss << "field ";
+    ss << key;
+    ss << " must be 'aa:bb' and 'aa','bb' must be int value format!";
+    err_info = ss.str();
+    return false;
+  }
+  string sub_str_1 = str_value.substr(0,pos1);
+  string sub_str_2 = 
+    str_value.substr(pos1+attr_sep.size(),str_value.size());
+  int in_hour = atoi(sub_str_1.c_str());
+  int in_minute = atoi(sub_str_2.c_str());
+  if (in_hour < 0 || in_hour > 24) {
+    ss << "field ";
+    ss << key;
+    ss << " -hour value must in [0,23]!";
+    err_info = ss.str();
+    return false;      
+  }
+  if (in_minute < 0 || in_minute > 59) {
+    ss << "field ";
+    ss << key;
+    ss << " -minute value must in [0,59]!";
+    err_info = ss.str();
+    return false;      
+  }
+  value = in_hour * 100 + in_minute;
+  return true;
+}
+
+
+}
+


[incubator-tubemq] 05/50: [TUBEMQ-250] Create C/C++ configure files

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit cef2ff6db8da540444fc96a497fe707b365e6de2
Author: gosonzhang <go...@tencent.com>
AuthorDate: Wed Jul 1 17:18:13 2020 +0800

    [TUBEMQ-250] Create C/C++ configure files
---
 .../tubemq-client-cpp/inc/client_config.h          |  36 +-
 .../tubemq-client-cpp/inc/const_config.h           |  83 +++++
 tubemq-client-twins/tubemq-client-cpp/inc/utils.h  |  19 +-
 .../tubemq-client-cpp/src/client_config.cc         | 397 +++++++++++++++++++--
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc | 172 ++++++++-
 5 files changed, 627 insertions(+), 80 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
index 576a8e1..0dcf458 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
@@ -32,35 +32,6 @@ namespace tubemq {
 using namespace std;
 
 
-// configuration value setting
-namespace config {
-// rpc timeout define  
-static const int kRpcTimoutDef = 15;
-static const int kRpcTimoutMax = 300;
-static const int kRpcTimoutMin = 8;
-// heartbeat period define
-static const int kHeartBeatPeriodDef = 10;
-static const int kHeartBeatFailRetryTimesDef = 5;
-static const int kHeartBeatSleepPeriodDef = 60;
-// max masterAddrInfo length
-static const int kMasterAddrInfoMaxLength = 1024;
-// max TopicName length
-static const int kTopicNameMaxLength = 64;
-// max Consume GroupName length
-static const int kGroupNameMaxLength = 1024;
-// max subscribe info report times
-static const int kSubInfoReportMaxIntervalTimes = 6;
-// default message not found response wait period
-static const int kMsgNotfoundWaitPeriodMsDef = 200;
-// default confirm wait period if rebalance meeting
-static const int kRebConfirmWaitPeriodMsDef = 3000;
-// max confirm wait period anyway
-static const int kConfirmWaitPeriodMsMax = 60000;
-// default rebalance wait if shutdown meeting
-static const int kRebWaitPeriodWhenShutdownMs = 10000;
-}  // namespace config
-
-
 class BaseConfig {
  public:
   BaseConfig();
@@ -147,7 +118,12 @@ class ConsumerConfig : public BaseConfig {
   const int GetShutdownRebWaitPeriodMs() const;
   void SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms);
   string ToString();
-     
+
+ private:
+  bool setGroupConsumeTarget(string& err_info, bool is_bound_consume,
+    const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
+    const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map);
+    
   
  private: 
   string group_name_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.h
new file mode 100644
index 0000000..1f2e655
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/const_config.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 _TUBEMQ_CLIENT_CONST_CONFIG_H_
+#define _TUBEMQ_CLIENT_CONST_CONFIG_H_
+
+#include <map>
+#include <string>
+
+namespace tubemq {
+
+using namespace std;
+
+// configuration value setting
+namespace config {
+// rpc timeout define  
+static const int kRpcTimoutDef = 15;
+static const int kRpcTimoutMax = 300;
+static const int kRpcTimoutMin = 8;
+// heartbeat period define
+static const int kHeartBeatPeriodDef = 10;
+static const int kHeartBeatFailRetryTimesDef = 5;
+static const int kHeartBeatSleepPeriodDef = 60;
+// max masterAddrInfo length
+static const int kMasterAddrInfoMaxLength = 1024;
+
+// max TopicName length
+static const int kTopicNameMaxLength = 64;
+// max Consume GroupName length
+static const int kGroupNameMaxLength = 1024;
+// max filter item length
+static const int kFilterItemMaxLength = 256;
+// max allowed filter item count
+static const int kFilterItemMaxCount = 500;
+// max session key length
+static const int kSessionKeyMaxLength = 1024;
+
+
+// max subscribe info report times
+static const int kSubInfoReportMaxIntervalTimes = 6;
+// default message not found response wait period
+static const int kMsgNotfoundWaitPeriodMsDef = 200;
+// default confirm wait period if rebalance meeting
+static const int kRebConfirmWaitPeriodMsDef = 3000;
+// max confirm wait period anyway
+static const int kConfirmWaitPeriodMsMax = 60000;
+// default rebalance wait if shutdown meeting
+static const int kRebWaitPeriodWhenShutdownMs = 10000;
+
+}  // namespace config
+
+
+namespace delimiter {
+  static const string kDelimiterEqual = "=";
+  static const string kDelimiterAnd   = "&";
+  static const string kDelimiterComma = ",";
+  static const string kDelimiterColon = ":";
+  static const string kDelimiterAt    = "@";
+  static const string kDelimiterPound = "#";
+} // namespace delimiter
+
+
+ 
+}
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index e8df420..9326449 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -22,28 +22,31 @@
 
 #include <map>
 #include <string>
+#include <vector>
+
 
 namespace tubemq {
 
 using namespace std;
 
 
-namespace delimiter {
-  static const string kDelimiterEqual = "=";
-  static const string kDelimiterAnd   = "&";
-  static const string kDelimiterComma = ",";
-  static const string kDelimiterColon = ":";
-  static const string kDelimiterAt    = "@";
-  static const string kDelimiterPound = "#";
-}
 
 class Utils {
  public:
   // trim string info
   static string Trim(const string& source);
   // split string to vector
+  static void Split(const string& source, vector<string>& result, const string& delimiter);
+  // split string to map<string, int>
   static void Split(const string& source, map<string, int>& result, 
                    const string& delimiter_step1, const string& delimiter_step2);
+  static void Join(const vector<string>& vec, const string& delimiter, string& target);
+  static bool ValidString(string& err_info, const string& source,
+                   bool allow_empty, bool pat_match, bool check_max_length, unsigned int maxlen);
+  static bool ValidGroupName(string &err_info, 
+                   const string& group_name, string& tgt_group_name);
+  static bool ValidFilterItem(string& err_info, 
+                   const string& src_filteritem, string& tgt_filteritem);
 
 };
  
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index 822c0e5..b596d76 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -18,7 +18,9 @@
  */
 
 #include <sstream> 
+#include <vector>
 #include "client_config.h"
+#include "const_config.h"
 #include "utils.h"
 
 
@@ -44,7 +46,7 @@ BaseConfig::~BaseConfig() {
 }
 
 BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
-  if(this != &target) {
+  if (this != &target) {
     this->master_addrinfo_ = target.master_addrinfo_;
     this->auth_enable_    = target.auth_enable_;
     this->auth_usrname_   = target.auth_usrname_;
@@ -63,11 +65,11 @@ BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
 bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrinfo) {
   // check parameter masterAddrInfo
   string trimed_master_addr_info = Utils::Trim(master_addrinfo);
-  if(trimed_master_addr_info.empty()) {
+  if (trimed_master_addr_info.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
-  if(trimed_master_addr_info.length() > config::kMasterAddrInfoMaxLength) {
+  if (trimed_master_addr_info.length() > config::kMasterAddrInfoMaxLength) {
     stringstream ss;
     ss << "Illegal parameter: over max ";
     ss << config::kMasterAddrInfoMaxLength;
@@ -80,7 +82,7 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
   map<string, int> tgt_address_map;
   Utils::Split(master_addrinfo, tgt_address_map, 
     delimiter::kDelimiterComma, delimiter::kDelimiterColon);
-  if(tgt_address_map.empty()) {
+  if (tgt_address_map.empty()) {
     err_info = "Illegal parameter: master_addrinfo is blank!";
     return false;
   }
@@ -92,14 +94,14 @@ bool BaseConfig::SetMasterAddrInfo(string& err_info, const string& master_addrin
 bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
                 const string& trust_store_path, const string& trust_store_password) {
   this->tls_enabled_ = tls_enable;
-  if(tls_enable) {
+  if (tls_enable) {
     string trimed_trust_store_path = Utils::Trim(trust_store_path);  
-    if(trimed_trust_store_path.empty()) {
+    if (trimed_trust_store_path.empty()) {
       err_info = "Illegal parameter: trust_store_path is empty!";
       return false;
     }
     string trimed_trust_store_password = Utils::Trim(trust_store_password);  
-    if(trimed_trust_store_password.empty()) {
+    if (trimed_trust_store_password.empty()) {
       err_info = "Illegal parameter: trust_store_password is empty!";
       return false;
     }
@@ -114,16 +116,16 @@ bool BaseConfig::SetTlsInfo(string& err_info, bool tls_enable,
 }
 
 bool BaseConfig::SetAuthenticInfo(string& err_info, bool authentic_enable, 
-                                       const string& usr_name, const string& usr_password) {
+                const string& usr_name, const string& usr_password) {
   this->auth_enable_ = authentic_enable;
-  if(authentic_enable) {
+  if (authentic_enable) {
     string trimed_usr_name = Utils::Trim(usr_name);
-    if(trimed_usr_name.empty()) {
+    if (trimed_usr_name.empty()) {
       err_info = "Illegal parameter: usr_name is empty!";
       return false;
     }
-    string trimed_usr_password = Utils::trim(usr_password);  
-    if(trimed_usr_password.empty()) {
+    string trimed_usr_password = Utils::Trim(usr_password);  
+    if (trimed_usr_password.empty()) {
       err_info = "Illegal parameter: usr_password is empty!";
       return false;
     }
@@ -205,21 +207,21 @@ int BaseConfig::GetHeartbeatPeriodAftFailSec() {
 
 string BaseConfig::ToString() {
   stringstream ss;
-  ss << "BaseConfig={master_addrinfo_=";
+  ss << "BaseConfig={master_addrinfo_='";
   ss << this->master_addrinfo_;
-  ss << ", authEnable=";
+  ss << "', authEnable=";
   ss << this->auth_enable_;
   ss << ", auth_usrname_='";
   ss << this->auth_usrname_;
-  ss << "', auth_usrpassword_=";
+  ss << "', auth_usrpassword_='";
   ss << this->auth_usrpassword_;
-  ss << ", tls_enabled_=";
+  ss << "', tls_enabled_=";
   ss << this->tls_enabled_;
-  ss << ", tls_trust_store_path_=";
+  ss << ", tls_trust_store_path_='";
   ss << this->tls_trust_store_path_;
-  ss << ", tls_trust_store_password_=";
+  ss << "', tls_trust_store_password_='";
   ss << this->tls_trust_store_password_;
-  ss << ", rpc_read_timeout_sec_=";
+  ss << "', rpc_read_timeout_sec_=";
   ss << this->rpc_read_timeout_sec_;
   ss << ", heartbeat_period_sec_=";
   ss << this->heartbeat_period_sec_;
@@ -253,19 +255,9 @@ string BaseConfig::ToString() {
  }
 
 ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
-  if(this != &target) {
+  if (this != &target) {
     // parent class 
-    this->master_addrinfo_ = target.master_addrinfo_;
-    this->auth_enable_    = target.auth_enable_;
-    this->auth_usrname_   = target.auth_usrname_;
-    this->auth_usrpassword_ = target.auth_usrpassword_;
-    this->tls_enabled_      = target.tls_enabled_;
-    this->tls_trust_store_path_      = target.tls_trust_store_path_;
-    this->tls_trust_store_password_  = target.tls_trust_store_password_;
-    this->rpc_read_timeout_sec_      = target.rpc_read_timeout_sec_;
-    this->heartbeat_period_sec_     = target.heartbeat_period_sec_;
-    this->max_heartbeat_retry_times_ = target.max_heartbeat_retry_times_;
-    this->heartbeat_period_afterfail_sec_ = target.heartbeat_period_afterfail_sec_;    
+    BaseConfig::operator =(target);
     // child class
     this->group_name_ = target.group_name_;
     this->sub_topic_and_filter_map_ = target.sub_topic_and_filter_map_;
@@ -282,11 +274,352 @@ ConsumerConfig& ConsumerConfig::operator=(const ConsumerConfig& target) {
     this->reb_confirm_wait_period_ms_ = target.reb_confirm_wait_period_ms_;
     this->max_confirm_wait_period_ms_ = target.max_confirm_wait_period_ms_;
     this->shutdown_reb_wait_period_ms_ = target.shutdown_reb_wait_period_ms_;
-    
   }
   return *this;
 }
 
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info,
+                       const string& group_name, const set<string>& subscribed_topicset) {
+  string tgt_group_name;
+  bool is_success = Utils::ValidGroupName(err_info,group_name, tgt_group_name);
+  if (!is_success) {
+    return false;
+  }
+  if (subscribed_topicset.empty()) {
+    err_info = "Illegal parameter: subscribed_topicset is empty!";
+    return false;
+  }
+  string topic_name;
+  map<string, set<string> > tmp_sub_map;
+  for (set<string>::iterator it = subscribed_topicset.begin(); 
+          it != subscribed_topicset.end(); ++it) {
+    topic_name = Utils::Trim(*it);
+    is_success = Utils::ValidString(err_info, topic_name, 
+                         false, true, true, config::kTopicNameMaxLength);
+    if (!is_success) {
+      err_info = "Illegal parameter: subscribed_topicset's item error, " + err_info;
+      return false;
+    }
+    set<string> tmp_filters;
+    tmp_sub_map[topic_name] = tmp_filters; 
+  }
+  this->is_bound_consume_ = false;
+  this->group_name_ = tgt_group_name;
+  this->sub_topic_and_filter_map_ = tmp_sub_map;
+  err_info = "Ok";
+  return true;
+}
+
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, 
+  const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map) {
+  string session_key;
+  int source_count = 0;
+  bool is_select_big = false;
+  map<string, long> part_offset_map;
+  return setGroupConsumeTarget(err_info, false, 
+    group_name, subscribed_topic_and_filter_map, 
+    session_key, source_count, is_select_big, part_offset_map);
+}
+
+bool ConsumerConfig::SetGroupConsumeTarget(string& err_info, 
+      const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
+      const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map) {
+  return setGroupConsumeTarget(err_info, true, 
+    group_name, subscribed_topic_and_filter_map, 
+    session_key, source_count, is_select_big, part_offset_map);
+}
+
+bool ConsumerConfig::setGroupConsumeTarget(string& err_info, bool is_bound_consume,
+      const string& group_name, const map<string, set<string> >& subscribed_topic_and_filter_map,
+      const string& session_key, int source_count, bool is_select_big, const map<string, long>& part_offset_map) {
+  // check parameter group_name
+  string tgt_group_name;
+  bool is_success = Utils::ValidGroupName(err_info,group_name, tgt_group_name);
+  if (!is_success) {
+    return false;
+  }
+  // check parameter subscribed_topic_and_filter_map
+  if (subscribed_topic_and_filter_map.empty()) {
+    err_info = "Illegal parameter: subscribed_topic_and_filter_map is empty!";
+    return false;
+  }
+  map<string, set<string> > tmp_sub_map;
+  map<string, set<string> >::const_iterator itMap;
+  for (itMap = subscribed_topic_and_filter_map.begin(); itMap != subscribed_topic_and_filter_map.end(); ++itMap) {
+    int count=0;
+    string tmp_filteritem;
+    set<string> tgt_filters;
+    // check topic_name info
+    is_success = Utils::ValidString(err_info, itMap->first, 
+                         false, true, true, config::kTopicNameMaxLength);  
+    if (!is_success)
+    {
+      stringstream ss;
+      ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
+      ss << itMap->first;
+      ss << " ";
+      ss << err_info;
+      err_info = ss.str();
+      return false;
+    }
+    string topic_name = Utils::Trim(itMap->first);
+    // check filter info
+    set<string> subscribed_filters = itMap->second;
+    for (set<string>::iterator it = subscribed_filters.begin(); it != subscribed_filters.end(); ++it)
+    {
+      is_success = Utils::ValidFilterItem(err_info, *it, tmp_filteritem);
+      if (!is_success) {
+        stringstream ss;
+        ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
+        ss << topic_name;
+        ss << "'s filter item ";
+        ss << err_info;
+        err_info = ss.str();
+        return false;
+      }
+      tgt_filters.insert(tmp_filteritem);
+      count++;
+    }
+    if (count > config::kFilterItemMaxCount) {
+      stringstream ss;
+      ss << "Check parameter subscribed_topic_and_filter_map error: topic ";
+      ss << itMap->first;
+      ss << "'s filter item over max item count : ";
+      ss << config::kFilterItemMaxCount;
+      err_info = ss.str();
+      return false;
+    }
+    tmp_sub_map[topic_name] = tgt_filters;
+  }
+  // check if bound consume 
+  if (!is_bound_consume) {
+    this->is_bound_consume_ = false;
+    this->group_name_ = tgt_group_name;
+    this->sub_topic_and_filter_map_ = tmp_sub_map;
+    err_info = "Ok";
+    return true;
+  }
+  // check session_key
+  string tgt_session_key = Utils::Trim(session_key);
+  if (tgt_session_key.length() == 0 
+    || tgt_session_key.length() > config::kSessionKeyMaxLength) {
+    if (tgt_session_key.length() == 0) {
+      err_info = "Illegal parameter: session_key is empty!";
+    } else {
+      stringstream ss;
+      ss << "Illegal parameter: session_key's length over max length ";
+      ss << config::kSessionKeyMaxLength;
+      err_info = ss.str();
+    }
+    return false;
+  }
+  // check source_count
+  if (source_count <= 0) {
+    err_info = "Illegal parameter: source_count must over zero!";
+    return false;
+  }
+  // check part_offset_map
+  string part_key;
+  map<string, long> tmp_parts_map;
+  map<string, long>::const_iterator itPart;
+  for (itPart = part_offset_map.begin(); itPart != part_offset_map.end(); ++itPart) {
+    vector<string> result;
+    Utils::Split(itPart->first, result, delimiter::kDelimiterColon);
+    if (result.size() != 3) {
+      stringstream ss;
+      ss << "Illegal parameter: part_offset_map's key ";
+      ss << itPart->first;
+      ss << " format error, value must be aaaa:bbbb:cccc !";
+      err_info = ss.str();
+      return false;
+    }
+    if (tmp_sub_map.find(result[1]) != tmp_sub_map.end()) {
+      stringstream ss;
+      ss << "Illegal parameter: ";
+      ss << itPart->first;
+      ss << " subscribed topic ";
+      ss << result[1];
+      ss << " not included in subscribed_topic_and_filter_map's topic list!";
+      err_info = ss.str();
+      return false;
+    }
+    if (itPart->first.find_first_of(delimiter::kDelimiterComma) != string::npos) {
+      stringstream ss;
+      ss << "Illegal parameter: key ";
+      ss << itPart->first;
+      ss << " include ";
+      ss << delimiter::kDelimiterComma;
+      ss << " char!";
+      err_info = ss.str();
+      return false;
+    }
+    if (itPart->second < 0) {
+      stringstream ss;
+      ss << "Illegal parameter: ";
+      ss << itPart->first;
+      ss << "'s offset must over or equal zero, value is ";
+      ss << itPart->second;
+      err_info = ss.str();
+      return false;
+    }
+    Utils::Join(result, delimiter::kDelimiterColon, part_key);
+    tmp_parts_map[part_key] = itPart->second;
+  }
+  // set verified data
+  this->is_bound_consume_ = true;
+  this->group_name_ = tgt_group_name;
+  this->sub_topic_and_filter_map_ = tmp_sub_map;    
+  this->session_key_ = tgt_session_key;
+  this->source_count_ = source_count;
+  this->is_select_big_ = is_select_big;
+  this->part_offset_map_ = tmp_parts_map;
+  err_info = "Ok";
+  return true;
+}
+
+const string& ConsumerConfig::GetGroupName() const {
+  return this->group_name_;
+}
+
+const map<string, set<string> >& ConsumerConfig::GetSubTopicAndFilterMap() const {
+  return this->sub_topic_and_filter_map_;
+}
+  
+void ConsumerConfig::SetConsumePosition(ConsumePosition consume_from_where) {
+  this->consume_position_ = consume_from_where;
+}
+
+const ConsumePosition ConsumerConfig::GetConsumePosition() const {
+  return this->consume_position_;
+}
+
+const int ConsumerConfig::GetMsgNotFoundWaitPeriodMs() const {
+  return this->msg_notfound_wait_period_ms_;
+}
+
+void ConsumerConfig::SetMsgNotFoundWaitPeriodMs(int msg_notfound_wait_period_ms) {
+  this->msg_notfound_wait_period_ms_ = msg_notfound_wait_period_ms;
+}
+
+const int ConsumerConfig::GetMaxSubinfoReportIntvl() const {
+  return this->max_subinfo_report_intvl_;
+}
+
+void ConsumerConfig::SetMaxSubinfoReportIntvl(int max_subinfo_report_intvl) {
+  this->max_subinfo_report_intvl_ = max_subinfo_report_intvl;
+}
+
+bool ConsumerConfig::IsConfirmInLocal() {
+  return this->is_confirm_in_local_;
+}
+
+void ConsumerConfig::SetConfirmInLocal(bool confirm_in_local) {
+  this->is_confirm_in_local_ = confirm_in_local;
+}
+
+bool ConsumerConfig::IsRollbackIfConfirmTimeout() {
+  return this->is_rollback_if_confirm_timout_;
+}
+
+void ConsumerConfig::setRollbackIfConfirmTimeout(bool is_rollback_if_confirm_timeout) {
+  this->is_rollback_if_confirm_timout_ = is_rollback_if_confirm_timeout;
+}
+
+const int ConsumerConfig::GetWaitPeriodIfConfirmWaitRebalanceMs() const {
+  return this->reb_confirm_wait_period_ms_;
+}
+
+void ConsumerConfig::SetWaitPeriodIfConfirmWaitRebalanceMs(int reb_confirm_wait_period_ms) {
+  this->reb_confirm_wait_period_ms_ = reb_confirm_wait_period_ms;
+}
+
+const int ConsumerConfig::GetMaxConfirmWaitPeriodMs() const {
+  this->max_confirm_wait_period_ms_;
+}
+
+void ConsumerConfig::SetMaxConfirmWaitPeriodMs(int max_confirm_wait_period_ms) {
+  this->max_confirm_wait_period_ms_ = max_confirm_wait_period_ms;
+}
+
+const int ConsumerConfig::GetShutdownRebWaitPeriodMs() const {
+  return this->shutdown_reb_wait_period_ms_;
+}
+
+void ConsumerConfig::SetShutdownRebWaitPeriodMs(int wait_period_when_shutdown_ms) {
+  this->shutdown_reb_wait_period_ms_ = wait_period_when_shutdown_ms;
+}
+
+string ConsumerConfig::ToString() {
+  int i = 0;
+  stringstream ss;
+  map<string, long>::iterator it;
+  map<string, set<string> >::iterator itMap;
+
+  // print info
+  ss << "ConsumerConfig = {";
+  ss << BaseConfig::ToString();
+  ss << ", group_name_='";
+  ss << this->group_name_;
+  ss << "', sub_topic_and_filter_map_={";
+  for (itMap = this->sub_topic_and_filter_map_.begin(); 
+      itMap != this->sub_topic_and_filter_map_.end(); ++itMap) {
+    if (i++ > 0) {
+      ss << ",";
+    }
+    ss << "'";
+    ss << itMap->first;
+    ss << "'=[";
+    int j=0;
+    set<string> topicSet = itMap->second;
+    for (set<string>::iterator it = topicSet.begin(); it != topicSet.end(); ++it) {
+      if (j++ > 0) {
+        ss << ",";
+      }
+      ss << "'";
+      ss << *it;
+      ss << "'";
+    }
+    ss << "]";
+  }
+  ss << "}, is_bound_consume_=";
+  ss << this->is_bound_consume_;
+  ss << ", session_key_='";
+  ss << this->session_key_;  
+  ss << "', source_count_=";
+  ss << this->source_count_;  
+  ss << ", is_select_big_=";
+  ss << this->is_select_big_;  
+  ss << ", part_offset_map_={";  
+  i = 0;
+  for (it = this->part_offset_map_.begin(); 
+      it != this->part_offset_map_.end(); ++it) {
+    if (i++ > 0) {
+      ss << ",";
+    }
+    ss << "'";
+    ss << it->first;
+    ss << "'=";
+    ss << it->second;
+  }
+  ss << "}, consume_position_=";
+  ss << this->consume_position_;
+  ss << ", max_subinfo_report_intvl_=";
+  ss << this->max_subinfo_report_intvl_;
+  ss << ", msg_notfound_wait_period_ms_=";
+  ss << this->msg_notfound_wait_period_ms_;
+  ss << ", is_confirm_in_local_=";
+  ss << this->is_confirm_in_local_;
+  ss << ", is_rollback_if_confirm_timout_=";
+  ss << this->is_rollback_if_confirm_timout_;
+  ss << ", reb_confirm_wait_period_ms_=";
+  ss << this->reb_confirm_wait_period_ms_;
+  ss << ", max_confirm_wait_period_ms_=";
+  ss << this->max_confirm_wait_period_ms_;
+  ss << ", shutdown_reb_wait_period_ms_=";
+  ss << this->shutdown_reb_wait_period_ms_;
+  ss << "}";
+  return ss.str();
+}
 
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 8b2d08f..d3f36ff 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -18,8 +18,12 @@
  */
 
 #include <vector>
+#include <sstream> 
+#include <regex.h>
 #include <stdlib.h>
 #include "utils.h"
+#include "const_config.h"
+
 
 namespace tubemq {
 
@@ -27,26 +31,52 @@ static const string kWhitespaceCharSet = " \n\r\t\f\v";
 
 string Utils::Trim(const string& source) {
   string target = source;
-  if(!target.empty()) {
+  if (!target.empty()) {
     size_t foud_pos = target.find_first_not_of(kWhitespaceCharSet);
     if (foud_pos != string::npos) {
       target = target.substr(foud_pos);
     }
     foud_pos = target.find_last_not_of(kWhitespaceCharSet);
-    if(foud_pos != string::npos) {
+    if (foud_pos != string::npos) {
       target = target.substr(0, foud_pos + 1);
     }
   }
   return target;
 }
 
+void Utils::Split(const string& source, vector<string>& result, const string& delimiter) {
+  string item_str;
+  string::size_type pos1,pos2;
+  result.clear();
+  if (!source.empty()) {
+    pos1 = 0;
+    pos2 = source.find(delimiter);
+    while(string::npos != pos2) {
+      item_str = Utils::Trim(source.substr(pos1, pos2-pos1));
+      pos1 = pos2 + delimiter.size();
+      pos2 = source.find(delimiter, pos1);
+      if (!item_str.empty()) {
+        result.push_back(item_str);
+      }
+    }
+    if (pos1 != source.length())
+    {
+      item_str = Utils::Trim(source.substr(pos1));
+      if (!item_str.empty()) {
+        result.push_back(item_str);
+      }
+    }
+  }
+}
+
+
 void Utils::Split(const string& source, map<string, int>& result, 
-      const string& delimiter_step1, const string& delimiter_step2) {
+                const string& delimiter_step1, const string& delimiter_step2) {
   string item_str;
   string key_str;
   string val_str;
   string::size_type pos1,pos2,pos3;
-  if(!source.empty()) {
+  if (!source.empty()) {
     pos1 = 0;
     pos2 = source.find(delimiter_step1);
     while(string::npos != pos2) {
@@ -54,32 +84,32 @@ void Utils::Split(const string& source, map<string, int>& result,
       item_str = Utils::Trim(item_str);
       pos1 = pos2 + delimiter_step1.length();
       pos2 = source.find(delimiter_step1, pos1);
-      if(item_str.empty()) {
+      if (item_str.empty()) {
         continue;
       }
       pos3 = item_str.find(delimiter_step2);
-      if(string::npos == pos3) {
+      if (string::npos == pos3) {
         continue;
       }
       key_str = item_str.substr(0, pos3);
       val_str = item_str.substr(pos3+delimiter_step2.length());
       key_str = Utils::Trim(key_str);
       val_str = Utils::Trim(val_str);
-      if(key_str.empty()) {
+      if (key_str.empty()) {
         continue;
       }
       result[key_str] = atoi(val_str.c_str());
     }
-    if(pos1 != source.length()) {
+    if (pos1 != source.length()) {
       item_str = source.substr(pos1);
       item_str = Utils::Trim(item_str);
       pos3 = item_str.find(delimiter_step2);
-      if(string::npos != pos3) {
+      if (string::npos != pos3) {
         key_str = item_str.substr(0, pos3);
         val_str = item_str.substr(pos3+delimiter_step2.length());
         key_str = Utils::Trim(key_str);
         val_str = Utils::Trim(val_str);
-        if(!key_str.empty()){
+        if (!key_str.empty()){
           result[key_str] = atoi(val_str.c_str());
         }
       }
@@ -87,5 +117,127 @@ void Utils::Split(const string& source, map<string, int>& result,
   }
 }
 
+void Utils::Join(const vector<string>& vec, const string& delimiter, string& target) {
+  vector<string>::const_iterator it;
+  target.clear();
+  for (it = vec.begin(); it != vec.end(); ++it) {
+    target += *it;
+    if (it != vec.end() - 1) {
+      target += delimiter;
+    }
+  }
+}
+
+bool Utils::ValidString(string& err_info, const string& source, 
+                bool allow_empty, bool pat_match, bool check_max_length, 
+                unsigned int maxlen) {
+  if (source.empty()) {
+    if (allow_empty) {
+      err_info = "Ok";
+      return true;
+    }
+    err_info = "value is empty";
+    return false;
+  }
+  if (check_max_length) {
+    if (source.length() > maxlen) {
+      stringstream ss;
+      ss << source;
+      ss << " over max length, the max allowed length is ";
+      ss << maxlen;
+      err_info = ss.str();
+      return false;
+    }
+  }
+
+  if (pat_match) {
+    int cflags =REG_EXTENDED;     
+    regex_t reg;    
+    regmatch_t pmatch[1];
+    const char* patRule = "^[a-zA-Z]\\w+$";  
+    regcomp(&reg, patRule,cflags);
+    int status = regexec(&reg, source.c_str(), 1, pmatch, 0);
+    regfree(&reg);
+    if (status == REG_NOMATCH) {
+      stringstream ss;
+      ss << source;
+      ss << " must begin with a letter,can only contain characters,numbers,and underscores";
+      err_info = ss.str();
+      return false;
+    }
+  }
+  err_info = "Ok";
+  return true;        
+}
+
+bool Utils::ValidGroupName(string& err_info, 
+                const string& group_name, string& tgt_group_name) {
+  tgt_group_name = Utils::Trim(group_name);
+  if (tgt_group_name.empty()) {
+    err_info = "Illegal parameter: group_name is blank!";
+    return false;
+  }
+  if (tgt_group_name.length() > config::kGroupNameMaxLength) {
+    stringstream ss;
+    ss << "Illegal parameter: ";
+    ss << group_name;
+    ss << " over max length, the max allowed length is ";
+    ss << config::kGroupNameMaxLength;
+    err_info = ss.str();
+    return false;
+  }
+  int cflags =REG_EXTENDED;     
+  regex_t reg;    
+  regmatch_t pmatch[1];
+  const char* patRule = "^[a-zA-Z][\\w-]+$"; 
+  regcomp(&reg, patRule,cflags);
+  int status = regexec(&reg, tgt_group_name.c_str(), 1, pmatch, 0);
+  regfree(&reg);
+  if (status == REG_NOMATCH) {
+    stringstream ss;
+    ss << "Illegal parameter: ";
+    ss << group_name;
+    ss << " must begin with a letter,can only contain ";
+    ss << "characters,numbers,hyphen,and underscores";
+    err_info = ss.str();
+    return false;
+  }
+  err_info = "Ok";
+  return true;        
+}
+
+bool Utils::ValidFilterItem(string& err_info, 
+                const string& src_filteritem, string& tgt_filteritem) {
+  tgt_filteritem = Utils::Trim(src_filteritem);
+  if (tgt_filteritem.empty()) {
+    err_info = "value is blank!";
+    return false;
+  }
+
+  if (tgt_filteritem.length() > config::kFilterItemMaxLength) {
+    stringstream ss;
+    ss << "value over max length ";
+    ss << config::kFilterItemMaxLength;
+    err_info = ss.str();
+    return false;
+  }
+  int cflags =REG_EXTENDED;    
+  regex_t reg;    
+  regmatch_t pmatch[1];
+  const char* patRule = "^[_A-Za-z0-9]+$";  
+  regcomp(&reg, patRule,cflags);
+  int status = regexec(&reg, tgt_filteritem.c_str(), 1, pmatch, 0);
+  regfree(&reg);
+  if (status == REG_NOMATCH) {
+      err_info = "value only contain characters,numbers,and underscores";
+      return false;
+  }
+  err_info = "Ok";
+  return true;      
+}
+
+
+
+
 }
 


[incubator-tubemq] 37/50: [TUBEMQ-286]Create C/C++ SDK's manager class (#212)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 1fb720f1726023ccdcdbc1e365343682a64a991d
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 20 14:19:37 2020 +0000

    [TUBEMQ-286]Create C/C++ SDK's manager class (#212)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/client_service.h                | 20 ++++----
 .../tubemq-client-cpp/src/client_service.cc        | 60 +++++++++++++++++++++-
 2 files changed, 68 insertions(+), 12 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
index d78a690..5e0b113 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
@@ -48,6 +48,7 @@ class BaseClient {
  public:
   BaseClient(bool is_producer);
   virtual ~BaseClient();
+  virtual void ShutDown();
   void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
   bool IsProducer() { return is_producer_; }
   const int32_t GetClientIndex() { return client_index_; }
@@ -58,26 +59,25 @@ class BaseClient {
 };
 
 
-enum ServiceStatus {
-  kServiceReady = 0,
-  kServiceRunning = 1,
-  kServiceStop = 2,
-};  // enum ServiceStatus
-
-
 class TubeMQService : public Singleton<TubeMQService> {
  public:
-  TubeMQService();
+  // TubeMQService();
+  // ~TubeMQService();
   bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
   bool Stop(string& err_info);
   bool IsRunning();
-  const int32_t  getServiceStatus() const { return service_status_.Get(); }
+  const int32_t  GetServiceStatus() const { return service_status_.Get(); }
+  int32_t GetClientObjCnt();
   bool AddClientObj(string& err_info,
          BaseClient* client_obj, int32_t& client_index);
   BaseClient* GetClientObj(int32_t client_index) const;
+  BaseClient* RmvClientObj(int32_t client_index);
+  const ExecutorPoolPtr& GetTimerExecutor() const { return timer_executor_; }
+  const ExecutorPoolPtr& GetNetWorkExecutor() const { return network_executor_; }
 
  private:
-  bool iniLogger(const Fileini& fileini, const string& sector);
+  void iniLogger(const Fileini& fileini, const string& sector);
+  void shutDownClinets() const;
 
  private:
   AtomicInteger service_status_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
index bd1da8a..1357123 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
@@ -41,6 +41,18 @@ BaseClient::~BaseClient() {
   // no code
 }
 
+/*
+TubeMQService::TubeMQService() {
+  service_status_.Set(0);
+  client_index_base_.Set(0);
+}
+
+TubeMQService::~TubeMQService() {
+  string err_info;
+  Stop(err_info);
+}
+*/
+
 bool TubeMQService::Start(string& err_info, string conf_file) {
   // check configure file
   bool result = false;
@@ -55,11 +67,29 @@ bool TubeMQService::Start(string& err_info, string conf_file) {
   if (!result) {
     return result;
   }
+  if (!service_status_.CompareAndSet(0,1)) {
+    err_info = "TubeMQ Service has startted or Stopped!";
+    return false;
+  }
   iniLogger(fileini, sector);
+  service_status_.set(2);
+}
+
+bool TubeMQService::Stop(string& err_info) {
+  if (service_status_.CompareAndSet(2, -1)) {
+    shutDownClinets();
+    timer_executor_->Close();
+    network_executor_->Close();
+  }
+  err_info = "OK!";
+  return true;
 }
 
+bool TubeMQService::IsRunning() {
+  return (service_status_.Get() == 2);
+}
 
-bool TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
+void TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
   string err_info;
   int32_t log_num = 10;
   int32_t log_size = 10;
@@ -71,9 +101,15 @@ bool TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
   fileini.GetValue(err_info, sector, "log_level", log_level, 4);
   log_level = TUBEMQ_MID(log_level, 0, 4);
   GetLogger().Init(log_path, Logger::Level(log_level), log_size, log_num);
-  return true;
 }
 
+
+int32_t TubeMQService::GetClientObjCnt() {
+  lock_guard<mutex> lck(mutex_);
+  return clients_map_.size();
+}
+
+
 bool TubeMQService::AddClientObj(string& err_info,
            BaseClient* client_obj, int32_t& client_index) {
   if (service_status_.Get() != 0) {
@@ -99,6 +135,26 @@ BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
   return client_obj;
 }
 
+BaseClient* TubeMQService::RmvClientObj(int32_t client_index) {
+  BaseClient* client_obj = NULL;
+  map<int32_t, BaseClient*>::iterator it;
+  
+  lock_guard<mutex> lck(mutex_);
+  it = clients_map_.find(client_index);
+  if (it != clients_map_.end()) {
+    client_obj = it->second;
+    clients_map_.erase(client_index);
+  }
+  return client_obj;
+}
+
+void TubeMQService::shutDownClinets() const {
+  map<int32_t, BaseClient*>::const_iterator it;
+  lock_guard<mutex> lck(mutex_);
+  for (it = clients_map_.begin(); it != clients_map_.end(); it++) {
+    it->second->ShutDown();
+  }
+}
 
 
 }  // namespace tubemq


[incubator-tubemq] 32/50: [TUBEMQ-280]Create C/C++ subscribe info class (#206)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit f88d38a8bdc65f8fa7d6f38734557621bb573c78
Author: gosonzhang <46...@qq.com>
AuthorDate: Wed Jul 15 02:01:15 2020 +0000

    [TUBEMQ-280]Create C/C++ subscribe info class (#206)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/atomic_def.h  |   6 +-
 .../include/tubemq/client_subinfo.h                |  80 +++++++++++++
 .../include/tubemq/executor_pool.h                 |   2 +
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |   6 +-
 .../tubemq-client-cpp/src/client_subinfo.cc        | 130 +++++++++++++++++++++
 .../tubemq-client-cpp/src/rmt_data_cache.cc        |   7 +-
 6 files changed, 222 insertions(+), 9 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
index 30830d9..efd5332 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/atomic_def.h
@@ -29,7 +29,7 @@ class AtomicInteger {
  public:
   AtomicInteger() { this->counter_ = 0; }
 
-  AtomicInteger(int32_t initial_value) { this->counter_ = initial_value; }
+  explicit AtomicInteger(int32_t initial_value) { this->counter_ = initial_value; }
 
   int32_t Get() const { return this->counter_; }
 
@@ -118,7 +118,7 @@ class AtomicLong {
  public:
   AtomicLong() { this->counter_ = 0; }
 
-  AtomicLong(int64_t initial_value) { this->counter_ = initial_value; }
+  explicit AtomicLong(int64_t initial_value) { this->counter_ = initial_value; }
 
   int64_t Get() const { return this->counter_; }
 
@@ -207,7 +207,7 @@ class AtomicBoolean {
  public:
   AtomicBoolean() { this->counter_ = 0; }
 
-  AtomicBoolean(bool initial_value) { this->counter_ = initial_value ? 1 : 0; }
+  explicit AtomicBoolean(bool initial_value) { this->counter_ = initial_value ? 1 : 0; }
 
   bool Get() const { return this->counter_ != 0; }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
new file mode 100644
index 0000000..c609e71
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.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 TUBEMQ_CLIENT_SUBINFO_H_
+#define TUBEMQ_CLIENT_SUBINFO_H_
+
+#include <stdint.h>
+#include <list>
+#include <map>
+#include <set>
+#include <string>
+#include "tubemq/atomic_def.h"
+
+namespace tubemq {
+
+using std::list;
+using std::map;
+using std::set;
+using std::string;
+
+
+class ClientSubInfo {
+ public:
+  ClientSubInfo();
+  void SetConsumeTarget(bool bound_consume,
+                               const map<string, set<string> >& topic_and_filter_map,
+                               const string& session_key, uint32_t source_count,
+                               bool select_big, const map<string, int64_t>& part_offset_map);
+  bool CompAndSetNotAllocated(bool expect, bool update);
+  void BookFstRegistered() { first_registered_.Set(true); }
+  bool IsBoundConsume() { return bound_consume_; }
+  bool IsNotAllocated() { return not_allocated_.Get(); }
+  const int64_t GetSubscribedTime() const { return subscribed_time_; }
+  const string& GetSessionKey() const { return session_key_; }
+  const uint32_t GetSourceCnt() const { return source_count_; }
+  bool SelectBig() { return select_big_; }
+  bool IsFilterConsume(const string& topic);
+  void GetAssignedPartOffset(const string& partition_key, int64_t& offset);
+  const string& GetBoundPartInfo() const { return bound_partions_; }
+  const list<string>& GetSubTopics() const { return topics_; }
+  const list<string>& GetTopicConds() const { return topic_conds_; }
+  const map<string, set<string> >& GetTopicFilterMap() const;
+
+ private:
+  bool bound_consume_;
+  AtomicBoolean first_registered_;
+  AtomicBoolean not_allocated_;
+  int64_t  subscribed_time_;
+  map<string, set<string> > topic_and_filter_map_;
+  list<string> topics_;
+  list<string> topic_conds_;
+  map<string, bool> topic_filter_map_;
+  // bound info
+  string session_key_;
+  uint32_t source_count_;
+  bool select_big_;
+  map<string, int64_t> assigned_part_map_;
+  string bound_partions_;
+};
+
+}  // namespace tubemq
+
+
+#endif  // TUBEMQ_CLIENT_SUBINFO_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
index 0dd1f66..a5208a4 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
@@ -28,6 +28,8 @@
 #include <memory>
 #include <mutex>
 #include <thread>
+#include <vector>
+
 
 #include "tubemq/noncopyable.h"
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index ccd62e9..3091347 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -65,7 +65,7 @@ class NodeInfo {
 class Partition {
  public:
   Partition();
-  Partition(const string& partition_info);
+  explicit Partition(const string& partition_info);
   Partition(const NodeInfo& broker_info, const string& part_str);
   Partition(const NodeInfo& broker_info, const string& topic, uint32_t partition_id);
   ~Partition();
@@ -94,7 +94,7 @@ class Partition {
 class PartitionExt : public Partition {
  public:
   PartitionExt();
-  PartitionExt(const string& partition_info);
+  explicit PartitionExt(const string& partition_info);
   PartitionExt(const NodeInfo& broker_info, const string& part_str);
   ~PartitionExt();
   PartitionExt& operator=(const PartitionExt& target);
@@ -136,7 +136,7 @@ class PartitionExt : public Partition {
 class SubscribeInfo {
  public:
   SubscribeInfo();
-  SubscribeInfo(const string& sub_info);
+  explicit SubscribeInfo(const string& sub_info);
   SubscribeInfo(const string& consumer_id,
         const string& group_name, const PartitionExt& partition_ext);
   SubscribeInfo& operator=(const SubscribeInfo& target);
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
new file mode 100644
index 0000000..0c8064c
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
@@ -0,0 +1,130 @@
+/**
+ * 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 "tubemq/client_subinfo.h"
+#include "tubemq/const_config.h"
+#include "tubemq/utils.h"
+
+
+
+namespace tubemq {
+
+
+ClientSubInfo::ClientSubInfo() {
+  bound_consume_ = false;
+  select_big_ = false;
+  source_count_ = 0;
+  session_key_ = "";
+  not_allocated_.Set(true);
+  first_registered_.Set(false);
+  subscribed_time_ = tb_config::kInvalidValue;
+  bound_partions_ = "";
+}
+
+void ClientSubInfo::SetConsumeTarget(bool bound_consume,
+         const map<string, set<string> >& topic_and_filter_map,
+         const string& session_key, uint32_t source_count,
+         bool select_big, const map<string, int64_t>& part_offset_map) {
+  int32_t count = 0;
+  string tmpstr = "";
+  // book register time
+  subscribed_time_ = Utils::GetCurrentTimeMillis();
+  //
+  first_registered_.Set(false);
+  bound_consume_ = bound_consume;
+  topic_and_filter_map_ = topic_and_filter_map;
+  // build topic filter info
+  topics_.clear();
+  topic_conds_.clear();
+  set<string>::iterator it_set;
+  map<string, set<string> >::const_iterator it_topic;
+  for (it_topic = topic_and_filter_map.begin();
+      it_topic != topic_and_filter_map.end(); it_topic++) {
+    topics_.push_back(it_topic->first);
+    if (it_topic->second.empty()) {
+      topic_filter_map_[it_topic->first] = false;
+    } else {
+      topic_filter_map_[it_topic->first] = true;
+
+      //build topic conditions
+      count = 0;
+      tmpstr = it_topic->first;
+      tmpstr += delimiter::kDelimiterPound;
+      for (it_set = it_topic->second.begin();
+          it_set != it_topic->second.end(); it_set++) {
+        if (count++ > 0) {
+          tmpstr += delimiter::kDelimiterComma;
+        }
+        tmpstr += *it_set;
+      }
+      topic_conds_.push_back(tmpstr);
+    }
+  }
+
+  //build bound_partition info
+  if (bound_consume) {
+    session_key_ = session_key;
+    source_count_ = source_count;
+    select_big_ = select_big;
+    assigned_part_map_ = part_offset_map;
+    count = 0;
+    bound_partions_ = "";
+    map<string, int64_t>::const_iterator it;
+    for (it = part_offset_map.begin(); it != part_offset_map.end(); it++) {
+      if (count++ > 0) {
+        bound_partions_ += delimiter::kDelimiterComma;
+      }
+      bound_partions_ += it->first;
+      bound_partions_ += delimiter::kDelimiterEqual;
+      bound_partions_ += Utils::Long2str(it->second);
+    }
+  }
+}
+
+bool ClientSubInfo::CompAndSetNotAllocated(bool expect, bool update) {
+  return not_allocated_.CompareAndSet(expect, update);
+}
+
+bool ClientSubInfo::IsFilterConsume(const string& topic) {
+  map<string, bool>::iterator it;
+  it = topic_filter_map_.find(topic);
+  if (it == topic_filter_map_.end()) {
+    return false;
+  }
+  return it->second;
+}
+
+void ClientSubInfo::GetAssignedPartOffset(const string& partition_key, int64_t& offset) {
+  map<string, int64_t>::iterator it;
+  if (first_registered_.Get() && bound_consume_ && not_allocated_.Get()) {
+    it = assigned_part_map_.find(partition_key);
+    if (it != assigned_part_map_.end()) {
+      offset = it->second;
+    }
+  }
+  offset = tb_config::kInvalidValue;
+}
+
+const map<string, set<string> >& ClientSubInfo::GetTopicFilterMap() const {
+  return topic_and_filter_map_;
+}
+
+
+}  // namespace tubemq
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index caa4236..2164fa9 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -176,9 +176,10 @@ bool RmtDataCacheCsm::SelectPartition(string &err_info,
   return result;
 }
 
-void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t curr_offset,
-                                             int32_t err_code, bool esc_limit, int32_t msg_size,
-                                             int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow) {
+void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, 
+                     int64_t curr_offset, int32_t err_code, bool esc_limit,
+                     int32_t msg_size,int64_t limit_dlt, int64_t cur_data_dlt,
+                     bool require_slow) {
   map<string, PartitionExt>::iterator it_part;
   // book partition offset info
   if (curr_offset >= 0) {


[incubator-tubemq] 24/50: [TUBEMQ-269] Create C/C++ RmtDataCache class (#195)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit d7cebd501acbe284e41df827f7ae5bc99e7d2ecf
Author: gosonzhang <46...@qq.com>
AuthorDate: Thu Jul 9 03:29:21 2020 +0000

    [TUBEMQ-269] Create C/C++ RmtDataCache class (#195)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/client_config.h                 |   3 +-
 .../include/tubemq/const_errcode.h                 |  49 +++++++
 .../include/tubemq/flowctrl_def.h                  |   2 +-
 .../tubemq-client-cpp/include/tubemq/message.h     |   2 +-
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |  33 +++++
 .../tubemq-client-cpp/src/client_config.cc         |  10 +-
 .../tubemq-client-cpp/src/file_ini.cc              |   4 +
 .../tubemq-client-cpp/src/flowctrl_def.cc          |   2 +
 .../tubemq-client-cpp/src/message.cc               |   5 +-
 .../tubemq-client-cpp/src/meta_info.cc             | 148 ++++++++++++++++++++-
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |   3 +
 11 files changed, 250 insertions(+), 11 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
index 95bef67..fecbeb2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_config.h
@@ -32,7 +32,6 @@ namespace tubemq {
 using std::map;
 using std::set;
 using std::string;
-using std::vector;
 
 class BaseConfig {
  public:
@@ -122,7 +121,7 @@ class ConsumerConfig : public BaseConfig {
  private:
   bool setGroupConsumeTarget(string& err_info, bool is_bound_consume, const string& group_name,
                              const map<string, set<string> >& subscribed_topic_and_filter_map,
-                             const string& session_key, int32_t source_count, bool is_select_big,
+                             const string& session_key, uint32_t source_count, bool is_select_big,
                              const map<string, int64_t>& part_offset_map);
 
  private:
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.h
new file mode 100644
index 0000000..c4e6370
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_errcode.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 TUBEMQ_CLIENT_CONST_ERR_CODE_H_
+#define TUBEMQ_CLIENT_CONST_ERR_CODE_H_
+
+namespace tubemq {
+
+namespace err_code {
+  static const int32_t kErrSuccess = 200;
+  static const int32_t kErrNotReady = 201;
+  static const int32_t kErrMoved = 301;
+
+  static const int32_t kErrBadRequest = 400;
+  static const int32_t kErrUnAuthorized = 401;
+  static const int32_t kErrForbidden = 403;
+  static const int32_t kErrNotFound = 404;
+  static const int32_t kErrPartitionOccupied = 410;
+  static const int32_t kErrHbNoNode = 411;
+  static const int32_t kErrDuplicatePartition = 412;
+  static const int32_t kErrCertificateFailure = 415;
+  static const int32_t kErrServerOverflow = 419;
+  static const int32_t kErrConsumeGroupForbidden = 450;
+  static const int32_t kErrConsumeSpeedLimit = 452;
+  static const int32_t kErrConsumeContentForbidden = 455;
+  
+  static const int32_t kErrServerError = 500;
+  static const int32_t kErrServiceUnavilable = 503;
+  static const int32_t kErrServerMsgsetNullError = 510;
+  static const int32_t kErrWaitServerRspTimeout = 550;
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_CONST_ERR_CODE_H_
\ No newline at end of file
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
index 98d8be4..0f05bb3 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/flowctrl_def.h
@@ -91,7 +91,7 @@ class FlowCtrlRuleHandler {
   ~FlowCtrlRuleHandler();
   void UpdateDefFlowCtrlInfo(bool is_default, int32_t qrypriority_id, int64_t flowctrl_id,
                              const string& flowctrl_info);
-  bool GetCurDataLimit(int32_t last_datadlt, FlowCtrlResult& flowctrl_result);
+  bool GetCurDataLimit(int64_t last_datadlt, FlowCtrlResult& flowctrl_result);
   int32_t GetCurFreqLimitTime(int32_t msg_zero_cnt, int32_t received_limit);
   int32_t GetMinZeroCnt() { return this->min_zero_cnt_.Get(); }
   int32_t GetQryPriorityId() { return this->qrypriority_id_.Get(); }
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
index 945b59e..8d68201 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/message.h
@@ -45,7 +45,7 @@ class Message {
   void SetTopic(const string& topic);
   const char* GetData() const;
   uint32_t GetDataLength() const;
-  void setData(const char* data, int datalen);
+  void setData(const char* data, uint32_t datalen);
   const int32_t GetFlag() const;
   void SetFlag(int32_t flag);
   const map<string, string>& GetProperties() const;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index e99939d..6dcd324 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -141,6 +141,39 @@ class ConsumerEvent {
 class PartitionExt : public Partition {
  public:
   PartitionExt();
+  PartitionExt(const string& partition_info);
+  PartitionExt(const NodeInfo& broker_info, const string& part_str);
+  ~PartitionExt();
+  void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
+    int64_t rsp_dlt_limit, long last_datadlt, bool require_slow);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed);
+  int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+    const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
+    int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
+    int64_t last_datadlt, bool require_slow);
+  void SetLastConsumed(bool last_consumed);
+  bool IsLastConsumed();
+ private:
+  void resetParameters();
+
+ private:
+  bool is_last_consumed_;
+  FlowCtrlResult cur_flowctrl_;
+  FlowCtrlItem cur_freqctrl_;
+  int64_t next_stage_updtime_;
+  int64_t next_slice_updtime_;
+  int64_t limit_slice_msgsize_;
+  int64_t cur_stage_msgsize_;
+  int64_t cur_slice_msgsize_;
+  int32_t total_zero_cnt_;
+  int64_t booked_time_;
+  int32_t booked_errcode_;
+  bool    booked_esc_limit_;
+  int32_t booked_msgsize_;
+  int64_t booked_dlt_limit_;
+  int64_t booked_curdata_dlt_;
+  bool    booked_require_slow_;
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
index 6c1f757..c6da13d 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -28,6 +28,11 @@
 
 namespace tubemq {
 
+using std::stringstream;
+using std::set;
+using std::vector;
+
+
 BaseConfig::BaseConfig() {
   this->master_addrinfo_ = "";
   this->auth_enable_ = false;
@@ -391,11 +396,6 @@ bool ConsumerConfig::setGroupConsumeTarget(
     }
     return false;
   }
-  // check source_count
-  if (source_count <= 0) {
-    err_info = "Illegal parameter: source_count must over zero!";
-    return false;
-  }
   // check part_offset_map
   string part_key;
   map<string, int64_t> tmp_parts_map;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index fe8edcd..97cf806 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -29,6 +29,10 @@
 
 namespace tubemq {
 
+using std::ifstream;
+
+
+
 Fileini::Fileini() {
   this->init_flag_ = false;
   this->ini_map_.clear();
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
index 1975306..27caba8 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/flowctrl_def.cc
@@ -31,6 +31,8 @@
 
 namespace tubemq {
 
+using std::stringstream;
+
 FlowCtrlResult::FlowCtrlResult() {
   this->datasize_limit_ = config::kMaxIntValue;
   this->freqms_limit_ = 0;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/message.cc b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
index 7fb83f8..d0afacf 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/message.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/message.cc
@@ -28,6 +28,9 @@
 
 namespace tubemq {
 
+using std::stringstream;
+
+
 // message flag's properties settings
 static const int32_t kMsgFlagIncProperties = 0x01;
 // reserved property key Filter Item
@@ -73,7 +76,7 @@ Message& Message::operator=(const Message& target) {
   return *this;
 }
 
-const uint64_t Message::GetMessageId() const { return this->message_id_; }
+const int64_t Message::GetMessageId() const { return this->message_id_; }
 
 void Message::SetMessageId(int64_t message_id) { this->message_id_ = message_id; }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index c1f428c..81ecf79 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -25,11 +25,12 @@
 #include <vector>
 
 #include "const_config.h"
+#include "const_errcode.h"
 #include "utils.h"
 
 namespace tubemq {
 
-using std::sstream;
+using std::stringstream;
 using std::vector;
 
 NodeInfo::NodeInfo() {
@@ -381,4 +382,149 @@ string ConsumerEvent::ToString() {
   return ss.str();
 }
 
+
+PartitionExt::PartitionExt() : Partition() {
+  resetParameters();
+}
+
+PartitionExt::PartitionExt(const string& partition_info) : Partition(partition_info) {
+  resetParameters();
+}
+
+PartitionExt::PartitionExt(const NodeInfo& broker_info, const string& part_str)
+  : Partition(broker_info, part_str) {
+  resetParameters();
+}
+
+PartitionExt::~PartitionExt() {
+  //
+}
+
+void PartitionExt::BookConsumeData(int32_t errcode, int32_t msg_size,
+  bool req_esc_limit, int64_t rsp_dlt_limit, long last_datadlt, bool require_slow) {
+  this->booked_time_ =Utils::GetCurrentTimeMillis();
+  this->booked_errcode_ = errcode;
+  this->booked_esc_limit_= req_esc_limit;
+  this->booked_msgsize_ = msg_size;
+  this->booked_dlt_limit_ = rsp_dlt_limit;
+  this->booked_curdata_dlt_ = last_datadlt;
+  this->booked_require_slow_ = require_slow;
+}
+
+int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+  const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed) {
+  int64_t dlt_time = Utils::GetCurrentTimeMillis() - this->booked_time_;
+  return ProcConsumeResult(def_flowctrl_handler, group_flowctrl_handler, filter_consume,
+    last_consumed, this->booked_errcode_, this->booked_msgsize_, this->booked_esc_limit_,
+    this->booked_dlt_limit_, this->booked_curdata_dlt_, this->booked_require_slow_) - dlt_time;
+}
+
+int64_t PartitionExt::ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
+  const FlowCtrlRuleHandler& group_flowctrl_handler, bool filter_consume, bool last_consumed,
+  int32_t errcode, int32_t msg_size, bool req_esc_limit, int64_t rsp_dlt_limit,
+  int64_t last_datadlt, bool require_slow) {
+  bool result = false;
+  // Accumulated data received
+  this->_isLastConsumed = last_consumed;
+  this->cur_stage_msgsize_ += msg_size;
+  this->cur_slice_msgsize_ += msg_size;
+  // Update strategy data values
+  int64_t curr_time = Utils::GetCurrentTimeMillis();
+  if (curr_time - this->next_stage_updtime_) {
+    this->cur_stage_msgsize_ = 0;
+    this->cur_slice_msgsize_ = 0;
+    if (last_datadlt >= 0) {
+      result = group_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+      if (!result) {
+        result = def_flowctrl_handler.GetCurDataLimit(last_datadlt, this->cur_flowctrl_);
+        if (!result) {
+          this->cur_flowctrl_.SetDataDltAndFreqLimit(config::kMaxLongValue, 0);
+        }
+      }
+      this->cur_freqctrl_ = group_flowctrl_handler.GetFilterCtrlItem();
+      if (this->cur_freqctrl_.getFreqLtInMs() < 0) {
+        this->cur_freqctrl_ = def_flowctrl_handler.GetFilterCtrlItem();
+      }
+      curr_time = Utils::GetCurrentTimeMillis();
+    }
+    this->limit_slice_msgsize_ = this->cur_flowctrl_.GetDataSizeLimit() / 12;
+    this->next_stage_updtime_ = curr_time + 60000;
+    this->next_slice_updtime_ = curr_time + 5000;
+  } else if(curr_time > this->next_slice_updtime_) {
+    this->cur_slice_msgsize_ = 0;
+    this->next_slice_updtime_ = curr_time + 5000;
+  }
+  // Perform different strategies based on error codes
+  switch (errcode) {
+    case err_code::kErrNotFound:
+    case err_code::kErrSuccess:
+      if (msg_size == 0 && errcode != err_code::kErrSuccess) {
+        this->total_zero_cnt_ += 1;
+      } else {
+        this->total_zero_cnt_ = 0;
+      }
+      if (this->total_zero_cnt_ > 0) {
+        if (group_flowctrl_handler.GetMinZeroCnt() != config::kMaxIntValue) {
+          return (int64_t)group_flowctrl_handler.GetCurFreqLimitTime(
+            this->total_zero_cnt_, (int32_t)rsp_dlt_limit);
+        } else {
+          return (int64_t)def_flowctrl_handler.GetCurFreqLimitTime(
+            this->_totalRcvZeroCount, (int32_t)rsp_dlt_limit);
+        }
+      }
+      if (req_esc_limit) {
+        return 0;
+      } else {
+        if (this->cur_stage_msgsize_ >= this->cur_flowctrl_.GetDataSizeLimit()
+          || this->cur_slice_msgsize_ >= this->limit_slice_msgsize_) {
+          return this->cur_flowctrl_.GetFreqMsLimit() > rsp_dlt_limit
+            ? this->cur_flowctrl_.GetFreqMsLimit() : rsp_dlt_limit;
+        }
+        if (errcode == err_code::kErrSuccess) {
+          if (filter_consume && this->cur_freqctrl_.GetFreqMsLimit() >= 0) {
+            if (require_slow) {
+              return this->cur_freqctrl_.GetZeroCnt();
+            } else {
+              return this->cur_freqctrl_.GetFreqMsLimit();
+            }
+          } else if (!filter_consume && this->cur_freqctrl_.GetDataSizeLimit() >=0) {
+            return this->cur_freqctrl_.GetDataSizeLimit();
+          }
+        }
+        return rsp_dlt_limit;
+      }
+      break;
+
+    default:
+      return rsp_dlt_limit;
+  }
+}
+
+void PartitionExt::SetLastConsumed(bool last_consumed) {
+  this->is_last_consumed_ = last_consumed;
+}
+
+bool PartitionExt::IsLastConsumed() {
+  return this->is_last_consumed_;
+}
+
+void PartitionExt::resetParameters() {
+  this->is_last_consumed_ = false;
+  this->cur_flowctrl_.SetDataDltAndFreqLimit(config::kMaxLongValue, 20);
+  this->next_stage_updtime_ = 0;
+  this->next_slice_updtime_ = 0;
+  this->limit_slice_msgsize_ = 0;
+  this->cur_stage_msgsize_ = 0;
+  this->cur_slice_msgsize_ = 0;
+  this->total_zero_cnt_ = 0;
+  this->booked_time_ = 0;
+  this->booked_errcode_ = 0;
+  this->booked_esc_limit_= false;
+  this->booked_msgsize_ = 0;
+  this->booked_dlt_limit_ = 0;
+  this->booked_curdata_dlt_ = 0;
+  this->booked_require_slow_ = false;
+}
+
+
 };  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 2fba193..bbfba96 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -31,6 +31,9 @@
 
 namespace tubemq {
 
+using std::stringstream;
+
+
 static const string kWhitespaceCharSet = " \n\r\t\f\v";
 
 string Utils::Trim(const string& source) {


[incubator-tubemq] 29/50: [TUBEMQ-269]Create C/C++ RmtDataCache class (#202)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 37055d1bd9ca78638b8613827f04cb927465f5ed
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 13 02:22:37 2020 +0000

    [TUBEMQ-269]Create C/C++ RmtDataCache class (#202)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../tubemq-client-cpp/include/tubemq/meta_info.h   |   3 +
 .../include/tubemq/rmt_data_cache.h                |  24 +-
 .../tubemq-client-cpp/src/meta_info.cc             |  36 +++
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 320 +++++++++++++++------
 4 files changed, 297 insertions(+), 86 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
index 813a9c5..2d2796e 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/meta_info.h
@@ -97,6 +97,7 @@ class PartitionExt : public Partition {
   PartitionExt(const string& partition_info);
   PartitionExt(const NodeInfo& broker_info, const string& part_str);
   ~PartitionExt();
+  PartitionExt& operator=(const PartitionExt& target);
   void BookConsumeData(int32_t errcode, int32_t msg_size, bool req_esc_limit,
     int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow);
   int64_t ProcConsumeResult(const FlowCtrlRuleHandler& def_flowctrl_handler,
@@ -135,6 +136,8 @@ class PartitionExt : public Partition {
 class SubscribeInfo {
  public:
   SubscribeInfo(const string& sub_info);
+  SubscribeInfo(const string& consumer_id,
+        const string& group_name, const PartitionExt& partition_ext);
   SubscribeInfo& operator=(const SubscribeInfo& target);
   const string& GetConsumerId() const;
   const string& GetGroup() const;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index a4daf91..7c25757 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -43,7 +43,7 @@ using std::list;
 // consumer remote data cache
 class RmtDataCacheCsm {
  public:
-  RmtDataCacheCsm();
+  RmtDataCacheCsm(const string& client_id, const string& group_name);
   ~RmtDataCacheCsm();
   void AddNewPartition(const PartitionExt& partition_ext);
   bool SelectPartition(string &err_info,
@@ -51,10 +51,24 @@ class RmtDataCacheCsm {
   void BookedPartionInfo(const string& partition_key, int64_t curr_offset,
                             int32_t err_code, bool esc_limit, int32_t msg_size,
                             int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow);
-  bool RelPartition(string &err_info, bool is_filterconsume,
+  bool RelPartition(string &err_info, bool filter_consume,
                          const string& confirm_context, bool is_consumed);
+  bool RelPartition(string &err_info, const string& confirm_context, bool is_consumed);
+  bool RelPartition(string &err_info, bool filter_consume, 
+                         const string& confirm_context, bool is_consumed,
+                         int64_t curr_offset, int32_t err_code, bool esc_limit,
+                         int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt);
+  void FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
+                    list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions);
+  void GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst);
+  bool GetPartitionExt(const string& part_key, PartitionExt& partition_ext);
+  void GetRegBrokers(list<NodeInfo>& brokers);
+  void GetCurPartitionOffsets(map<string, int64_t> part_offset_map);
+  void GetAllBrokerPartitions(map<NodeInfo, list<PartitionExt> >& broker_parts);
   void RemovePartition(const list<PartitionExt>& partition_list);
+  void RemovePartition(const set<string>& partition_keys);
   bool RemovePartition(string &err_info, const string& confirm_context);
+  bool BookPartition(const string& partition_key);
   void OfferEvent(const ConsumerEvent& event);
   void TakeEvent(ConsumerEvent& event);
   void ClearEvent();
@@ -62,15 +76,20 @@ class RmtDataCacheCsm {
   bool PollEventResult(ConsumerEvent& event);
 
  private:
+  void rmvMetaInfo(const string& partition_key);
   void buildConfirmContext(const string& partition_key,
                     int64_t booked_time, string& confirm_context);
   bool parseConfirmContext(string &err_info,
     const string& confirm_context, string& partition_key, int64_t& booked_time);
+  bool inRelPartition(string &err_info, bool need_delay_check,
+    bool filter_consume, const string& confirm_context, bool is_consumed);
 
  private:
   // timer begin
 
   // timer end
+  string consumer_id_;
+  string group_name_;  
   // flow ctrl
   FlowCtrlRuleHandler group_flowctrl_handler_;
   FlowCtrlRuleHandler def_flowctrl_handler_;
@@ -81,6 +100,7 @@ class RmtDataCacheCsm {
   map<string, set<string> > topic_partition_;
   // broker parition map
   map<NodeInfo, set<string> > broker_partition_;
+  map<string, SubscribeInfo>  part_subinfo_;
   // for idle partitions occupy
   pthread_mutex_t  part_mutex_;
   // for partiton idle map
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index c01c260..aea6239 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -261,6 +261,34 @@ PartitionExt::~PartitionExt() {
   //
 }
 
+PartitionExt& PartitionExt::operator=(const PartitionExt& target) {
+  if (this != &target) {
+    // parent class
+    Partition::operator=(target);
+    // child class    
+    this->is_last_consumed_ = target.is_last_consumed_;
+    this->cur_flowctrl_ = target.cur_flowctrl_;
+    this->cur_freqctrl_ = target.cur_freqctrl_;
+    this->next_stage_updtime_ = target.next_stage_updtime_;
+    this->next_slice_updtime_ = target.next_slice_updtime_;
+    this->limit_slice_msgsize_ = target.limit_slice_msgsize_;
+    this->cur_stage_msgsize_ = target.cur_stage_msgsize_;
+    this->cur_slice_msgsize_ = target.cur_slice_msgsize_;
+    this->total_zero_cnt_ = target.total_zero_cnt_;
+    this->booked_time_ = target.booked_time_;
+    this->booked_errcode_ = target.booked_errcode_;
+    this->booked_esc_limit_ = target.booked_esc_limit_;
+    this->booked_msgsize_ = target.booked_msgsize_;
+    this->booked_dlt_limit_ = target.booked_dlt_limit_;
+    this->booked_curdata_dlt_ = target.booked_curdata_dlt_;
+    this->booked_require_slow_ = target.booked_require_slow_;
+    this->booked_errcode_ = target.booked_errcode_;
+    this->booked_errcode_ = target.booked_errcode_;
+  }
+  return *this;
+}
+
+
 void PartitionExt::BookConsumeData(int32_t errcode, int32_t msg_size,
   bool req_esc_limit, int64_t rsp_dlt_limit, int64_t last_datadlt, bool require_slow) {
   this->booked_time_ = Utils::GetCurrentTimeMillis();
@@ -420,6 +448,14 @@ SubscribeInfo::SubscribeInfo(const string& sub_info) {
   buildSubInfo();
 }
 
+SubscribeInfo::SubscribeInfo(const string& consumer_id,
+        const string& group_name, const PartitionExt& partition_ext) {
+  this->consumer_id_ = consumer_id;
+  this->group_ = group_name;
+  this->partitionext_ = partition_ext;
+  buildSubInfo();
+}
+
 SubscribeInfo& SubscribeInfo::operator=(const SubscribeInfo& target) {
   if (this != &target) {
     this->consumer_id_ = target.consumer_id_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index cb596ec..3b86b43 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -34,7 +34,10 @@ namespace tubemq {
 
 
 
-RmtDataCacheCsm::RmtDataCacheCsm() {
+RmtDataCacheCsm::RmtDataCacheCsm(const string& client_id,
+                                      const string& group_name) {
+  consumer_id_ = client_id;
+  group_name_ = group_name;
   pthread_rwlock_init(&meta_rw_lock_, NULL);
   pthread_mutex_init(&part_mutex_, NULL);
   pthread_mutex_init(&data_book_mutex_, NULL);
@@ -58,6 +61,7 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   map<string, set<string> >::iterator it_topic;
   map<NodeInfo, set<string> >::iterator it_broker;
   //
+  SubscribeInfo sub_info(consumer_id_, group_name_, partition_ext);
   string partition_key = partition_ext.GetPartitionKey();
   pthread_rwlock_wrlock(&meta_rw_lock_);
   it_map = partitions_.find(partition_key);
@@ -83,13 +87,16 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
         it_broker->second.insert(partition_key);
       }
     }
+    part_subinfo_[partition_key] = sub_info;
   }
   // check partition_key status
+  pthread_mutex_lock(&part_mutex_);
   if (partition_useds_.find(partition_key) == partition_useds_.end()
     && partition_timeouts_.find(partition_key) == partition_timeouts_.end()) {
     index_partitions_.remove(partition_key);
     index_partitions_.push_back(partition_key);
   }
+  pthread_mutex_unlock(&part_mutex_);
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
@@ -150,64 +157,137 @@ void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t cur
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
-bool RmtDataCacheCsm::RelPartition(string &err_info, bool is_filterconsume,
+// success process release partition
+bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
                                  const string& confirm_context, bool is_consumed) {
-  int64_t wait_time;
+  return inRelPartition(err_info, true, filter_consume, confirm_context, is_consumed);
+}
+
+// release partiton without response return
+bool RmtDataCacheCsm::RelPartition(string &err_info,
+                                 const string& confirm_context, bool is_consumed) {
+  return inRelPartition(err_info, true, false, confirm_context, is_consumed);
+}
+
+// release partiton with error response return
+bool RmtDataCacheCsm::RelPartition(string &err_info, bool filter_consume,
+                              const string& confirm_context, bool is_consumed,
+                              int64_t curr_offset, int32_t err_code, bool esc_limit, 
+                              int32_t msg_size, int64_t limit_dlt, int64_t cur_data_dlt) {
   int64_t booked_time;
   string  partition_key;
-  map<string, PartitionExt>::iterator it_Part;
-  map<string, int64_t>::iterator it_used;
   // parse confirm context  
   bool result = parseConfirmContext(err_info,
                       confirm_context, partition_key, booked_time);
   if (!result) {
     return false;
   }
+  BookedPartionInfo(partition_key, curr_offset, err_code,
+            esc_limit, msg_size, limit_dlt, cur_data_dlt, false);
+  return inRelPartition(err_info, true, 
+    filter_consume, confirm_context, is_consumed);
+}
+
+void RmtDataCacheCsm::FilterPartitions(const list<SubscribeInfo>& subscribe_info_lst,
+                    list<PartitionExt>& subscribed_partitions, list<PartitionExt>& unsub_partitions) {
+  //
+  map<string, PartitionExt>::iterator it_part;
+  list<SubscribeInfo>::const_iterator it_lst;
+  // initial return;
+  subscribed_partitions.clear();
+  unsub_partitions.clear();
   pthread_rwlock_rdlock(&meta_rw_lock_);
-  it_Part = partitions_.find(partition_key);
-  if (it_Part == partitions_.end()) {
-    pthread_mutex_lock(&part_mutex_);
-    partition_useds_.erase(partition_key);
-    index_partitions_.remove(partition_key);
-    pthread_mutex_unlock(&part_mutex_);
-    err_info = "Not found the partition in Consume Partition set!";
-    result = false;
+  if (partitions_.empty()) {
+    for (it_lst = subscribe_info_lst.begin(); it_lst != subscribe_info_lst.end(); it_lst++) {
+      unsub_partitions.push_back(it_lst->GetPartitionExt());
+    }
   } else {
-    pthread_mutex_lock(&part_mutex_);
-    it_used = partition_useds_.find(partition_key);
-    if (it_used == partition_useds_.end()) {
-      index_partitions_.remove(partition_key);
-      index_partitions_.push_back(partition_key);       
-    } else {
-      if (it_used->second == booked_time) {
-        partition_useds_.erase(partition_key);
-        wait_time = it_Part->second.ProcConsumeResult(def_flowctrl_handler_, 
-                      group_flowctrl_handler_, is_filterconsume, is_consumed);
-        if (wait_time >= 10) {
-          // todo add timer 
-          // end todo
-        } else {
-          partition_useds_.erase(partition_key);
-          index_partitions_.remove(partition_key);
-        }
-        err_info = "Ok";
-        result = true;    
-      } else {
-        err_info = "Illegel confirmContext content: context not equal!";
-        result = false;
+    for (it_lst = subscribe_info_lst.begin(); it_lst != subscribe_info_lst.end(); it_lst++) {
+      it_part = partitions_.find(it_lst->GetPartitionExt().GetPartitionKey());
+    	if (it_part == partitions_.end()) {
+        unsub_partitions.push_back(it_lst->GetPartitionExt());
+    	} else {
+    		subscribed_partitions.push_back(it_lst->GetPartitionExt());
       }
     }
-    pthread_mutex_unlock(&part_mutex_);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+void RmtDataCacheCsm::GetSubscribedInfo(list<SubscribeInfo>& subscribe_info_lst) {
+  map<string, SubscribeInfo>::iterator it_sub;
+  subscribe_info_lst.clear();                                             
+	pthread_rwlock_rdlock(&meta_rw_lock_);
+	for (it_sub = part_subinfo_.begin(); it_sub != part_subinfo_.end(); ++it_sub) {
+    subscribe_info_lst.push_back(it_sub->second);
+	}
+	pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+void RmtDataCacheCsm::GetAllBrokerPartitions(
+                    map<NodeInfo, list<PartitionExt> >& broker_parts) {
+  map<string, PartitionExt>::iterator it_part;
+  map<NodeInfo, list<PartitionExt> >::iterator it_broker;
+
+  broker_parts.clear();
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  for (it_part = partitions_.begin(); it_part != partitions_.end(); ++it_part) {
+    it_broker = broker_parts.find(it_part->second.GetBrokerInfo());
+    if (it_broker == broker_parts.end()) {
+      list<PartitionExt> tmp_part_lst;
+      tmp_part_lst.push_back(it_part->second);
+      broker_parts[it_part->second.GetBrokerInfo()] = tmp_part_lst;
+    } else {
+      it_broker->second.push_back(it_part->second);
+    }
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+
+bool RmtDataCacheCsm::GetPartitionExt(const string& part_key, PartitionExt& partition_ext) {
+  bool result = false;
+  map<string, PartitionExt>::iterator it_map;
+  
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  it_map = partitions_.find(part_key);
+  if (it_map != partitions_.end()) {
+    result = true;
+    partition_ext = it_map->second;  
   }
   pthread_rwlock_unlock(&meta_rw_lock_);
   return result;
 }
 
+void RmtDataCacheCsm::GetRegBrokers(list<NodeInfo>& brokers) {
+  map<NodeInfo, set<string> >::iterator it;  
+
+  brokers.clear();
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  for (it = broker_partition_.begin(); it != broker_partition_.end(); ++it) {
+    brokers.push_back(it->first);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+void RmtDataCacheCsm::GetCurPartitionOffsets(map<string, int64_t> part_offset_map) {
+  map<string, int64_t>::iterator it;
+
+  part_offset_map.clear();
+  pthread_mutex_lock(&data_book_mutex_);
+  for (it = partition_offset_.begin(); it != partition_offset_.end(); ++it) {
+    part_offset_map[it->first] = it->second;
+  }
+  pthread_mutex_unlock(&data_book_mutex_);
+}
+
+
+//
 bool RmtDataCacheCsm::RemovePartition(string &err_info,
                                   const string& confirm_context) {
   int64_t booked_time;
   string  partition_key;
-  map<string, PartitionExt>::iterator it_Part;
+  map<string, PartitionExt>::iterator it_part;
   map<string, set<string> >::iterator it_topic;
   map<NodeInfo, set<string> >::iterator it_broker;
   // parse confirm context  
@@ -217,68 +297,53 @@ bool RmtDataCacheCsm::RemovePartition(string &err_info,
     return false;
   }
   // remove partiton
-  pthread_rwlock_wrlock(&meta_rw_lock_);
-  partition_useds_.erase(partition_key);
-  index_partitions_.remove(partition_key);
-  // todo need modify if timer build finished
-  partition_timeouts_.erase(partition_key);
-  // end todo
-  it_Part = partitions_.find(partition_key);
-  if (it_Part != partitions_.end()) {
-    it_topic = topic_partition_.find(it_Part->second.GetTopic());
-    if (it_topic != topic_partition_.end()) {
-      it_topic->second.erase(it_Part->second.GetPartitionKey());
-      if (it_topic->second.empty()) {
-        topic_partition_.erase(it_Part->second.GetTopic());
-      }
-    }
-    it_broker = broker_partition_.find(it_Part->second.GetBrokerInfo());
-    if (it_broker != broker_partition_.end()) {
-      it_broker->second.erase(it_Part->second.GetPartitionKey());
-      if (it_broker->second.empty()) {
-        broker_partition_.erase(it_Part->second.GetBrokerInfo());
-      }
-    }
-    partitions_.erase(partition_key);
-  }
-  pthread_rwlock_unlock(&meta_rw_lock_);
+  set<string> partition_keys;
+  partition_keys.insert(partition_key);
+  RemovePartition(partition_keys);
   err_info = "Ok";
   return true;
 }
 
 void RmtDataCacheCsm::RemovePartition(const list<PartitionExt>& partition_list) {
+  set<string> partition_keys;
   list<PartitionExt>::const_iterator it_lst;
-  map<string, PartitionExt>::iterator it_Part;
-  map<string, set<string> >::iterator it_topic;
-  map<NodeInfo, set<string> >::iterator it_broker;
+  for (it_lst = partition_list.begin(); it_lst != partition_list.end(); it_lst++) {
+    partition_keys.insert(it_lst->GetPartitionKey());
+  }
+  RemovePartition(partition_keys);
+}
+
+void RmtDataCacheCsm::RemovePartition(const set<string>& partition_keys) {
+  set<string>::const_iterator it_lst;
 
   pthread_rwlock_wrlock(&meta_rw_lock_);
-  for (it_lst = partition_list.begin(); it_lst != partition_list.end(); it_lst++) {
-    partition_useds_.erase(it_lst->GetPartitionKey());
-    index_partitions_.remove(it_lst->GetPartitionKey());
-    partitions_.erase(it_lst->GetPartitionKey());
+  for (it_lst = partition_keys.begin(); it_lst != partition_keys.end(); it_lst++) {
+    pthread_mutex_lock(&part_mutex_);
+    partition_useds_.erase(*it_lst);
+    index_partitions_.remove(*it_lst);
     // todo need modify if timer build finished
-    partition_timeouts_.erase(it_lst->GetPartitionKey());
+    partition_timeouts_.erase(*it_lst);
     // end todo
-		it_topic = topic_partition_.find(it_lst->GetTopic());
-		if (it_topic != topic_partition_.end()) {
-      it_topic->second.erase(it_lst->GetPartitionKey());
-      if (it_topic->second.empty()) {
-        topic_partition_.erase(it_lst->GetTopic());
-      }
-    }
-    it_broker = broker_partition_.find(it_lst->GetBrokerInfo());
-		if (it_broker != broker_partition_.end()) {
-      it_broker->second.erase(it_lst->GetPartitionKey());
-      if (it_broker->second.empty()) {
-        broker_partition_.erase(it_lst->GetBrokerInfo());
-      }
-    }
+    pthread_mutex_unlock(&part_mutex_);
+    // remove meta info set info
+    rmvMetaInfo(*it_lst);
   }
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
 
+bool RmtDataCacheCsm::BookPartition(const string& partition_key) {
+  bool result = false;
+  map<string, bool>::iterator it;
+  pthread_mutex_lock(&data_book_mutex_);
+  it = part_reg_booked_.find(partition_key);
+  if (it == part_reg_booked_.end()) {
+    part_reg_booked_[partition_key] = true;
+  }
+  pthread_mutex_unlock(&data_book_mutex_);
+  return result;
+}
+
 void RmtDataCacheCsm::OfferEvent(const ConsumerEvent& event) {
   pthread_mutex_lock(&event_read_mutex_);
   this->rebalance_events_.push_back(event);
@@ -343,4 +408,91 @@ bool RmtDataCacheCsm::parseConfirmContext(string &err_info,
   return true;
 }
 
+void RmtDataCacheCsm::rmvMetaInfo(const string& partition_key) {
+  map<string, PartitionExt>::iterator it_part;
+  map<string, set<string> >::iterator it_topic;
+  map<NodeInfo, set<string> >::iterator it_broker;  
+  it_part = partitions_.find(partition_key);
+  if (it_part != partitions_.end()) {
+    it_topic = topic_partition_.find(it_part->second.GetTopic());
+    if (it_topic != topic_partition_.end()) {
+      it_topic->second.erase(it_part->second.GetPartitionKey());
+      if (it_topic->second.empty()) {
+        topic_partition_.erase(it_part->second.GetTopic());
+      }
+    }
+    it_broker = broker_partition_.find(it_part->second.GetBrokerInfo());
+    if (it_broker != broker_partition_.end()) {
+      it_broker->second.erase(it_part->second.GetPartitionKey());
+      if (it_broker->second.empty()) {
+        broker_partition_.erase(it_part->second.GetBrokerInfo());
+      }
+    }
+    partitions_.erase(partition_key);
+    part_subinfo_.erase(partition_key);
+  }
+}
+
+bool RmtDataCacheCsm::inRelPartition(string &err_info, bool need_delay_check,
+                     bool filter_consume, const string& confirm_context, bool is_consumed) {
+  int64_t wait_time;
+  int64_t booked_time;
+  string  partition_key;
+  map<string, PartitionExt>::iterator it_part;
+  map<string, int64_t>::iterator it_used;
+  // parse confirm context  
+  bool result = parseConfirmContext(err_info,
+                      confirm_context, partition_key, booked_time);
+  if (!result) {
+    return false;
+  }
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  it_part = partitions_.find(partition_key);
+  if (it_part == partitions_.end()) {
+    // partition is unregister, release partition
+    pthread_mutex_lock(&part_mutex_);
+    partition_useds_.erase(partition_key);
+    index_partitions_.remove(partition_key);
+    pthread_mutex_unlock(&part_mutex_);
+    err_info = "Not found the partition in Consume Partition set!";
+    result = false;
+  } else {
+    pthread_mutex_lock(&part_mutex_);
+    it_used = partition_useds_.find(partition_key);
+    if (it_used == partition_useds_.end()) {
+      // partition is release but registered
+      index_partitions_.remove(partition_key);
+      index_partitions_.push_back(partition_key);
+    } else {
+      if (it_used->second == booked_time) {
+        // wait release
+        partition_useds_.erase(partition_key);
+        index_partitions_.remove(partition_key);
+        wait_time = 0;
+        if (need_delay_check) {
+          wait_time = it_part->second.ProcConsumeResult(def_flowctrl_handler_,
+                        group_flowctrl_handler_, filter_consume, is_consumed);
+        } 
+        if (wait_time >= 10) {
+          // todo add timer 
+          // end todo
+        } else {
+          index_partitions_.push_back(partition_key);
+        }
+        err_info = "Ok";
+        result = true;    
+      } else {
+        // partiton is used by other thread
+        err_info = "Illegel confirmContext content: context not equal!";
+        result = false;
+      }
+    }
+    pthread_mutex_unlock(&part_mutex_);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+  return result;
+}
+
+
+
 }  // namespace tubemq


[incubator-tubemq] 36/50: [TUBEMQ-286]Create C/C++ SDK's manager class (#211)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit a9a6e073d1cf7931371004f098303353aba9ba0b
Author: gosonzhang <46...@qq.com>
AuthorDate: Mon Jul 20 01:39:05 2020 +0000

    [TUBEMQ-286]Create C/C++ SDK's manager class (#211)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/client_service.h                |  94 +++++++++++++++++++
 .../include/tubemq/client_subinfo.h                |  13 +++
 .../include/tubemq/const_config.h                  |   5 +
 .../tubemq-client-cpp/include/tubemq/file_ini.h    |   8 +-
 .../tubemq-client-cpp/include/tubemq/utils.h       |   1 +
 .../tubemq-client-cpp/src/client_service.cc        | 104 +++++++++++++++++++++
 .../tubemq-client-cpp/src/client_subinfo.cc        |  43 +++++++++
 .../tubemq-client-cpp/src/file_ini.cc              |  10 +-
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  19 ++++
 9 files changed, 288 insertions(+), 9 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
new file mode 100644
index 0000000..d78a690
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_service.h
@@ -0,0 +1,94 @@
+/**
+ * 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 TUBEMQ_CLIENT_BASE_CLIENT_H_
+#define TUBEMQ_CLIENT_BASE_CLIENT_H_
+
+#include <mutex>
+#include <stdint.h>
+#include <string>
+#include <thread>
+
+#include "tubemq/atomic_def.h"
+#include "tubemq/file_ini.h"
+#include "tubemq/rmt_data_cache.h"
+#include "tubemq/singleton.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_return.h"
+
+
+
+
+namespace tubemq {
+
+using std::map;
+using std::mutex;
+using std::string;
+using std::thread;
+
+
+class BaseClient {
+ public:
+  BaseClient(bool is_producer);
+  virtual ~BaseClient();
+  void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
+  bool IsProducer() { return is_producer_; }
+  const int32_t GetClientIndex() { return client_index_; }
+
+ private:
+  bool  is_producer_;
+  int32_t client_index_;
+};
+
+
+enum ServiceStatus {
+  kServiceReady = 0,
+  kServiceRunning = 1,
+  kServiceStop = 2,
+};  // enum ServiceStatus
+
+
+class TubeMQService : public Singleton<TubeMQService> {
+ public:
+  TubeMQService();
+  bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
+  bool Stop(string& err_info);
+  bool IsRunning();
+  const int32_t  getServiceStatus() const { return service_status_.Get(); }
+  bool AddClientObj(string& err_info,
+         BaseClient* client_obj, int32_t& client_index);
+  BaseClient* GetClientObj(int32_t client_index) const;
+
+ private:
+  bool iniLogger(const Fileini& fileini, const string& sector);
+
+ private:
+  AtomicInteger service_status_;
+  AtomicInteger client_index_base_;
+  mutable mutex mutex_;
+  map<int32_t, BaseClient*> clients_map_;
+  ExecutorPoolPtr timer_executor_;
+  ExecutorPoolPtr network_executor_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_BASE_CLIENT_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
index c609e71..613f97c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/client_subinfo.h
@@ -35,6 +35,19 @@ using std::set;
 using std::string;
 
 
+class MasterAddrInfo {
+ public:
+  MasterAddrInfo();
+  bool InitMasterAddress(string& err_info, const string& master_info);
+  void GetNextMasterAddr(string& ipaddr, int32_t& port);
+  void GetCurrentMasterAddr(string& ipaddr, int32_t& port);
+  int32_t GetTotalMasterAddrCnt() { return master_addr_.size(); } 
+
+ private:
+   string curr_master_addr_;
+   map<string, int32_t> master_addr_;
+};
+
 class ClientSubInfo {
  public:
   ClientSubInfo();
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
index d843929..1cfb962 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/const_config.h
@@ -29,6 +29,11 @@ namespace tubemq {
 
 using std::string;
 
+#define TUBEMQ_MAX(a, b) ( ((a)>(b))?(a):(b) )
+#define TUBEMQ_MIN(a, b) ( ((a)>(b))?(b):(a) )
+#define TUBEMQ_MID(data, max, min)   TUBEMQ_MAX(min,TUBEMQ_MIN((max),(data)))
+
+
 // configuration value setting
 namespace tb_config {
 // heartbeat period define
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
index 2fb81e8..1bede63 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/file_ini.h
@@ -35,10 +35,10 @@ class Fileini {
   Fileini();
   ~Fileini();
   bool Loadini(string& err_info, const string& file_name);
-  bool GetValue(string& err_info, const string& sector, const string& key, string& value,
-                const string& def);
-  bool GetValue(string& err_info, const string& sector, const string& key, int32_t& value,
-                const int32_t def);
+  bool GetValue(string& err_info, const string& sector, const string& key,
+                    string& value, const string& def) const;
+  bool GetValue(string& err_info, const string& sector, const string& key,
+                   int32_t& value, int32_t def) const;
 
  private:
   bool init_flag_;
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
index a25832f..3fa9d48 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/utils.h
@@ -51,6 +51,7 @@ class Utils {
   static string Long2str(int64_t data);
   static uint32_t IpToInt(const string& ipv4_addr);
   static int64_t GetCurrentTimeMillis();
+  static bool ValidConfigFile(string& err_info, const string& conf_file);
 };
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
new file mode 100644
index 0000000..bd1da8a
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.cc
@@ -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 "tubemq/client_service.h"
+
+#include <sstream>
+
+#include "tubemq/const_config.h"
+#include "tubemq/logger.h"
+#include "tubemq/utils.h"
+
+
+
+namespace tubemq {
+
+using std::lock_guard;
+using std::stringstream;
+
+
+BaseClient::BaseClient(bool is_producer) {
+  this->is_producer_ = is_producer;
+}
+
+BaseClient::~BaseClient() {
+  // no code
+}
+
+bool TubeMQService::Start(string& err_info, string conf_file) {
+  // check configure file
+  bool result = false;
+  Fileini fileini;
+  string sector = "TubeMQ";  
+
+  result = Utils::ValidConfigFile(err_info, conf_file);
+  if (!result) {
+    return result;
+  }
+  result = fileini.Loadini(err_info, conf_file);
+  if (!result) {
+    return result;
+  }
+  iniLogger(fileini, sector);
+}
+
+
+bool TubeMQService::iniLogger(const Fileini& fileini, const string& sector) {
+  string err_info;
+  int32_t log_num = 10;
+  int32_t log_size = 10;
+  int32_t log_level = 4;
+  string log_path = "../log/";
+  fileini.GetValue(err_info, sector, "log_num", log_num, 10);
+  fileini.GetValue(err_info, sector, "log_size", log_size, 100);
+  fileini.GetValue(err_info, sector, "log_path", log_path, "../log/");
+  fileini.GetValue(err_info, sector, "log_level", log_level, 4);
+  log_level = TUBEMQ_MID(log_level, 0, 4);
+  GetLogger().Init(log_path, Logger::Level(log_level), log_size, log_num);
+  return true;
+}
+
+bool TubeMQService::AddClientObj(string& err_info,
+           BaseClient* client_obj, int32_t& client_index) {
+  if (service_status_.Get() != 0) {
+    err_info = "Service not startted!";
+    return false;
+  }
+  client_index = client_index_base_.IncrementAndGet();
+  lock_guard<mutex> lck(mutex_);
+  this->clients_map_[client_index] = client_obj;
+  err_info = "Ok";
+  return true;
+}
+
+BaseClient* TubeMQService::GetClientObj(int32_t client_index) const {
+  BaseClient* client_obj = NULL;
+  map<int32_t, BaseClient*>::const_iterator it;
+  
+  lock_guard<mutex> lck(mutex_);
+  it = clients_map_.find(client_index);
+  if (it != clients_map_.end()) {
+    client_obj = it->second;
+  }
+  return client_obj;
+}
+
+
+
+}  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
index 9e959e6..e57950c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_subinfo.cc
@@ -26,6 +26,49 @@
 namespace tubemq {
 
 
+MasterAddrInfo::MasterAddrInfo() {
+  curr_master_addr_ = "";
+  master_addr_.clear();
+}
+
+bool MasterAddrInfo::InitMasterAddress(string& err_info, 
+                                       const string& master_info) {
+  master_addr_.clear();
+  Utils::Split(master_info, master_addr_, delimiter::kDelimiterComma,
+               delimiter::kDelimiterColon);
+  if (master_addr_.empty()) {
+    err_info = "Illegal parameter: master_info is blank!";
+    return false;
+  }
+
+  map<string, int32_t>::iterator it = master_addr_.begin();
+  curr_master_addr_ = it->first;
+  err_info = "Ok";
+  return true;
+}
+
+void MasterAddrInfo::GetNextMasterAddr(string& ipaddr, int32_t& port) {
+  map<string, int32_t>::iterator it;
+  it = master_addr_.find(curr_master_addr_);
+  if(it != master_addr_.end()) {
+    it++;
+    if (it == master_addr_.end()) {
+      it = master_addr_.begin();
+    }
+  } else {
+    it = master_addr_.begin();
+  }
+  port   = it->second;
+  ipaddr = it->first;
+  curr_master_addr_ = it->first;
+}
+
+void MasterAddrInfo::GetCurrentMasterAddr(string& ipaddr, int32_t& port) {
+   ipaddr = curr_master_addr_;
+   port = master_addr_[curr_master_addr_];
+}
+
+
 ClientSubInfo::ClientSubInfo() {
   bound_consume_ = false;
   select_big_ = false;
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
index b073cde..9af90fc 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/file_ini.cc
@@ -115,7 +115,7 @@ bool Fileini::Loadini(string& err_info, const string& file_name) {
 }
 
 bool Fileini::GetValue(string& err_info, const string& sector, const string& key, string& value,
-                       const string& def) {
+                       const string& def) const {
   if (!this->init_flag_) {
     err_info = "Please load configure file first!";
     return false;
@@ -123,8 +123,8 @@ bool Fileini::GetValue(string& err_info, const string& sector, const string& key
   err_info = "Ok";
   value.clear();
   // search key's value in sector
-  map<string, map<string, string> >::iterator it_sec;
-  map<string, string>::iterator it_keyval;
+  map<string, map<string, string> >::const_iterator it_sec;
+  map<string, string>::const_iterator it_keyval;
   it_sec = this->ini_map_.find(sector);
   if (it_sec == this->ini_map_.end()) {
     value = def;
@@ -139,8 +139,8 @@ bool Fileini::GetValue(string& err_info, const string& sector, const string& key
   return true;
 }
 
-bool Fileini::GetValue(string& err_info, const string& sector, const string& key, int32_t& value,
-                       const int32_t def) {
+bool Fileini::GetValue(string& err_info, const string& sector, const string& key,
+                          int32_t& value, int32_t def) const {
   string val_str;
   string def_str = Utils::Int2str(def);
   bool result = GetValue(err_info, sector, key, val_str, def_str);
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
index 2fbaecd..ba6ac3a 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -21,6 +21,7 @@
 
 #include <regex.h>
 #include <stdlib.h>
+#include <stdio.h>
 #include <sys/time.h>
 #include <unistd.h>
 
@@ -272,5 +273,23 @@ int64_t Utils::GetCurrentTimeMillis() {
   return tv.tv_sec * 1000 + tv.tv_usec / 1000;
 }
 
+bool Utils::ValidConfigFile(string& err_info, const string& conf_file) {
+  FILE *fp = NULL;
+  
+  if (conf_file.length() == 0) {
+    err_info = "Configure file is blank";
+    return false;
+  }  
+  fp = fopen(configFile.c_str(),"r");
+  if(fp == NULL) {
+    err_info = "Open configure file Failed!";
+    return false;
+  }
+  fclose(fp);
+  err_info = "Ok";
+  return true;
+}
+
+
 }  // namespace tubemq
 


[incubator-tubemq] 02/50: [TUBEMQ-250] Create C/C++ configure files

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 6b112f6c11b6c441dfc92f55aefea2e43d4a5b5c
Author: gosonzhang <go...@tencent.com>
AuthorDate: Tue Jun 30 19:47:51 2020 +0800

    [TUBEMQ-250] Create C/C++ configure files
---
 .../tubemq-client-cpp/inc/client_config.h          | 121 +++++++++++
 .../inc/{TubeClientConfig.h => utils.h}            | 113 +++++-----
 .../{src/TubeClientConfig.cpp => inc/version.h}    |  19 +-
 .../tubemq-client-cpp/src/client_config.cc         | 237 +++++++++++++++++++++
 tubemq-client-twins/tubemq-client-cpp/src/utils.cc |  91 ++++++++
 5 files changed, 510 insertions(+), 71 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
new file mode 100644
index 0000000..9d3afac
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/client_config.h
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ 
+#ifndef _TUBEMQ_CLIENT_CONFIGURE_H_
+#define _TUBEMQ_CLIENT_CONFIGURE_H_
+
+#include <string>
+#include <stdio.h>
+#include <map>
+
+
+
+namespace TubeMQ {
+
+  using namespace std;
+
+  // configuration value setting
+  namespace config {
+  
+    // rpc timeout define  
+    static const int kRpcTimoutDef = 15;
+    static const int kRpcTimoutMax = 300;
+    static const int kRpcTimoutMin = 8;
+    // heartbeat period define
+    static const int kHeartBeatPeriodDef = 10;
+    static const int kHeartBeatFailRetryTimesDef = 5;
+    static const int kHeartBeatSleepPeriodDef = 60;
+    // max masterAddrInfo length
+    static const int kMasterAddrInfoMaxLength = 1024;
+    // max TopicName length
+    static const int kTopicNameMaxLength = 64;
+    // max Consume GroupName length
+    static const int kGroupNameMaxLength = 1024;
+  }  // namespace config
+
+  class BaseConfig {
+   public:
+    BaseConfig();
+    ~BaseConfig();
+    BaseConfig& operator=(const BaseConfig& target);
+    bool SetMasterAddrInfo(string& errInfo, const string& masterAddrInfo);
+    bool SetTlsInfo(string& errInfo, bool tlsEnable, 
+                        const string& trustStorePath, const string& trustStorePassword);
+    bool SetAuthenticInfo(string& errInfo, bool needAuthentic, 
+                                const string& usrName, const string& usrPassWord);
+    const string& GetMasterAddrInfo() const;
+    bool IsTlsEnabled();
+    const string& GetTrustStorePath() const;
+    const string& GetTrustStorePassword() const;
+    bool IsAuthenticEnabled();
+    const string& GetUsrName() const;
+    const string& GetUsrPassWord() const;            
+    // set the rpc timout, unit second, duration [8, 300], default 15 seconds;
+    void SetRpcReadTimeoutSec(int rpcReadTimeoutSec);
+    int GetRpcReadTimeoutSec();
+    // Set the duration of the client's heartbeat cycle, in seconds, the default is 10 seconds
+    void SetHeartbeatPeriodSec(int heartbeatPeriodInSec);
+    int GetHeartbeatPeriodSec();
+    void SetMaxHeartBeatRetryTimes(int maxHeartBeatRetryTimes);
+    int GetMaxHeartBeatRetryTimes();
+    void SetHeartbeatPeriodAftFailSec(int heartbeatPeriodSecAfterFailSec);
+    int GetHeartbeatPeriodAftFailSec();
+    string ToString();
+  
+ private:
+  string masterAddrStr_;
+  // user authenticate
+  bool   authEnable_;
+  string authUsrName_;
+  string authUsrPassWord_;
+  // TLS configuration
+  bool   tlsEnabled_;
+  string tlsTrustStorePath_;
+  string tlsTrustStorePassword_;
+  // other setting
+  int   rpcReadTimeoutSec_;
+  int   heartbeatPeriodSec_;
+  int    maxHeartBeatRetryTimes_;
+  int   heartbeatPeriodAfterFailSec_;
+};
+
+
+  class ConsumerConfig {
+   public:
+     ConsumerConfig();
+  };
+
+}
+
+#endif
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
similarity index 51%
rename from tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h
rename to tubemq-client-twins/tubemq-client-cpp/inc/utils.h
index 97f5978..d15df37 100644
--- a/tubemq-client-twins/tubemq-client-cpp/inc/TubeClientConfig.h
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/utils.h
@@ -1,59 +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.
- */
- 
-#ifndef _TUBEMQ_CLIENT_CONFIGURE_H_
-#define _TUBEMQ_CLIENT_CONFIGURE_H_
-
-#include <string>
-#include <stdio.h>
-
-
-
-
-namespace TubeMQ {
-
-    using namespace std;
-
-    class TubeConsumerConfig
-       public:
-            TubeConsumerConfig();
-
-    };
-{
-		
- 
-}
-
-
-#endif
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
+/**
+ * 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 _TUBEMQ_CLIENT_UTILS_H_
+#define _TUBEMQ_CLIENT_UTILS_H_
+
+#include <map>
+#include <string>
+
+namespace TubeMQ {
+
+  using namespace std;
+
+  static const string tWhitespaceCharSet = " \n\r\t\f\v";
+
+  namespace delimiter {
+    static const string tDelimiterEqual = "=";
+    static const string tDelimiterAnd   = "&";
+    static const string tDelimiterComma = ",";
+    static const string tDelimiterColon = ":";
+    static const string tDelimiterAt    = "@";
+    static const string tDelimiterPound = "#";
+  }
+
+  class Utils {
+   public:
+    // trim string info
+    static string trim(const string& source);
+    // split string to vector
+    static void split(const string& source, map<string, int>& result, 
+                     const string& delimiterStep1, const string& delimiterStep2);
+
+  };
+ 
+}
+
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
similarity index 79%
rename from tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp
rename to tubemq-client-twins/tubemq-client-cpp/inc/version.h
index 91107fc..1fc1bda 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/TubeClientConfig.cpp
+++ b/tubemq-client-twins/tubemq-client-cpp/inc/version.h
@@ -16,24 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
- 
-#include "TubeClientConfig.h"
+      
+#ifndef _TUBEMQ_CLIENT_VERSION_H_
+#define _TUBEMQ_CLIENT_VERSION_H_
 
+#include <string>
 
 namespace TubeMQ {
 
+  using namespace std;
 
-TubeConsumerConfig::TubeConsumerConfig()
-{
+  static const String tTubeMQClientVersion = "0.5.0";
 
 }
 
-TubeConsumerConfig::~TubeConsumerConfig()
-{
-
-}
-
-
-
-}
+#endif
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
new file mode 100644
index 0000000..bc11183
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_config.cc
@@ -0,0 +1,237 @@
+/**
+ * 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 <sstream> 
+#include "client_config.h"
+#include "utils.h"
+
+
+namespace TubeMQ {
+
+
+BaseConfig::BaseConfig() {
+  this->masterAddrStr_ = "";
+  this->authEnable_ = false;
+  this->authUsrName_ = "";
+  this->authUsrPassWord_ = "";
+  this->tlsEnabled_ = false;
+  this->tlsTrustStorePath_ = "";
+  this->tlsTrustStorePassword_ = "";
+  this->rpcReadTimeoutSec_ = config::kRpcTimoutDef;
+  this->heartbeatPeriodSec_ = config::kHeartBeatPeriodDef;
+  this->maxHeartBeatRetryTimes_ = config::kHeartBeatFailRetryTimesDef;
+  this->heartbeatPeriodAfterFailSec_ = config::kHeartBeatSleepPeriodDef;
+}
+
+BaseConfig::~BaseConfig() {
+
+}
+
+BaseConfig& BaseConfig::operator=(const BaseConfig& target) {
+  if(this != &target) {
+    this->masterAddrStr_ = target.masterAddrStr_;
+    this->authEnable_    = target.authEnable_;
+    this->authUsrName_   = target.authUsrName_;
+    this->authUsrPassWord_ = target.authUsrPassWord_;
+    this->tlsEnabled_      = target.tlsEnabled_;
+    this->tlsTrustStorePath_      = target.tlsTrustStorePath_;
+    this->tlsTrustStorePassword_  = target.tlsTrustStorePassword_;
+    this->rpcReadTimeoutSec_      = target.rpcReadTimeoutSec_;
+    this->heartbeatPeriodSec_     = target.heartbeatPeriodSec_;
+    this->maxHeartBeatRetryTimes_ = target.maxHeartBeatRetryTimes_;
+    this->heartbeatPeriodAfterFailSec_ = target.heartbeatPeriodAfterFailSec_;
+  }
+  return *this;
+}
+    
+bool BaseConfig::SetMasterAddrInfo(string& errInfo, const string& masterAddrInfo) {
+  // check parameter masterAddrInfo
+  string trimMasterAddrInfo = Utils::trim(masterAddrInfo);
+  if(trimMasterAddrInfo.empty()) {
+    errInfo = "Illegal parameter: masterAddrInfo is blank!";
+    return false;
+  }
+  if(trimMasterAddrInfo.length() > config::kMasterAddrInfoMaxLength) {
+    stringstream ss;
+    ss << "Illegal parameter: over max ";
+    ss << config::kMasterAddrInfoMaxLength;
+    ss << " length of masterAddrInfo parameter!";   
+    errInfo = ss.str();
+    return false;
+  }
+  // parse and verify master address info
+  // masterAddrInfo's format like ip1:port1,ip2:port2,ip3:port3
+  map<string, int> tgtAddressMap;
+  Utils::split(masterAddrInfo, tgtAddressMap, 
+    delimiter::tDelimiterComma, delimiter::tDelimiterColon);
+  if(tgtAddressMap.empty()) {
+    errInfo = "Illegal parameter: masterAddrInfo is blank!";
+    return false;
+  }
+  this->masterAddrStr_ = trimMasterAddrInfo;
+  errInfo = "Ok";
+  return true;
+}
+
+bool BaseConfig::SetTlsInfo(string& errInfo, bool tlsEnable,
+                              const string& trustStorePath, const string& trustStorePassword) {
+  this->tlsEnabled_ = tlsEnable;
+  if(tlsEnable) {
+    string trimTrustStorePath = Utils::trim(trustStorePath);  
+    if(trimTrustStorePath.empty()) {
+      errInfo = "Illegal parameter: trustStorePath is empty!";
+      return false;
+    }
+    string trimTrustStorePassword = Utils::trim(trustStorePassword);  
+    if(trimTrustStorePassword.empty()) {
+      errInfo = "Illegal parameter: trustStorePassword is empty!";
+      return false;
+    }
+      this->tlsTrustStorePath_= trimTrustStorePath;
+      this->tlsTrustStorePassword_= trimTrustStorePassword;    
+  } else {
+    this->tlsTrustStorePath_ = "";
+    this->tlsTrustStorePassword_ = "";
+  }
+  errInfo = "Ok";
+  return true;  
+}
+
+bool BaseConfig::SetAuthenticInfo(string& errInfo, bool needAuthentic, 
+                                       const string& usrName, const string& usrPassWord) {
+  this->authEnable_ = needAuthentic;
+  if(needAuthentic) {
+    string trimUsrName = Utils::trim(usrName);
+    if(trimUsrName.empty()) {
+      errInfo = "Illegal parameter: usrName is empty!";
+      return false;
+    }
+    string trimUsrPassWord = Utils::trim(usrPassWord);  
+    if(trimUsrPassWord.empty()) {
+      errInfo = "Illegal parameter: usrPassWord is empty!";
+      return false;
+    }
+    this->authUsrName_ = trimUsrName;
+    this->authUsrPassWord_ = trimUsrPassWord;
+  } else {
+    this->authUsrName_ = "";
+    this->authUsrPassWord_ = "";
+  }
+  errInfo = "Ok";
+  return true;
+}
+
+const string& BaseConfig::GetMasterAddrInfo() const {
+    return this->masterAddrStr_;
+}
+
+bool BaseConfig::IsTlsEnabled() {
+  return this->tlsEnabled_;
+}
+
+const string& BaseConfig::GetTrustStorePath() const {
+  return this->tlsTrustStorePath_;
+}
+
+const string& BaseConfig::GetTrustStorePassword() const {
+  return this->tlsTrustStorePassword_;
+}
+
+bool BaseConfig::IsAuthenticEnabled() {
+  return this->authEnable_;
+}
+
+const string& BaseConfig::GetUsrName() const {
+  return this->authUsrName_;
+}
+
+const string& BaseConfig::GetUsrPassWord() const {
+  return this->authUsrPassWord_;
+}
+
+void BaseConfig::SetRpcReadTimeoutSec(int rpcReadTimeoutSec) {
+  if (rpcReadTimeoutSec >= config::kRpcTimoutMax) {
+    this->rpcReadTimeoutSec_ = config::kRpcTimoutMax;
+  } else if (rpcReadTimeoutSec <= config::kRpcTimoutMin) {
+    this->rpcReadTimeoutSec_ = config::kRpcTimoutMin;
+  } else {
+    this->rpcReadTimeoutSec_ = rpcReadTimeoutSec;
+  }
+}
+
+int BaseConfig::GetRpcReadTimeoutSec() {
+  return this->rpcReadTimeoutSec_;
+}
+
+void BaseConfig::SetHeartbeatPeriodSec(int heartbeatPeriodSec) {
+  this->heartbeatPeriodSec_ = heartbeatPeriodSec;
+}
+
+int BaseConfig::GetHeartbeatPeriodSec() {
+  return this->heartbeatPeriodSec_;
+}
+
+void BaseConfig::SetMaxHeartBeatRetryTimes(int maxHeartBeatRetryTimes) {
+  this->maxHeartBeatRetryTimes_ = maxHeartBeatRetryTimes;
+}
+
+int BaseConfig::GetMaxHeartBeatRetryTimes() {
+  return this->maxHeartBeatRetryTimes_;
+}
+
+void BaseConfig::SetHeartbeatPeriodAftFailSec(int heartbeatPeriodSecAfterFailSec) {
+  this->heartbeatPeriodAfterFailSec_ = heartbeatPeriodSecAfterFailSec;
+}
+
+int BaseConfig::GetHeartbeatPeriodAftFailSec() {
+  return this->heartbeatPeriodAfterFailSec_;
+}
+
+string BaseConfig::ToString() {
+  stringstream ss;
+  ss << "BaseConfig={masterAddrStr=";
+  ss << this->masterAddrStr_;
+  ss << ", authEnable=";
+  ss << this->authEnable_;
+  ss << ", authUsrName='";
+  ss << this->authUsrName_;
+  ss << "', authUsrPassWord=";
+  ss << this->authUsrPassWord_;
+  ss << ", tlsEnable=";
+  ss << this->tlsEnabled_;
+  ss << ", tlsTrustStorePath=";
+  ss << this->tlsTrustStorePath_;
+  ss << ", tlsTrustStorePassword=";
+  ss << this->tlsTrustStorePassword_;
+  ss << ", rpcReadTimeoutSec=";
+  ss << this->rpcReadTimeoutSec_;
+  ss << ", heartbeatPeriodSec=";
+  ss << this->heartbeatPeriodSec_;
+  ss << ", maxHeartBeatRetryTimes=";
+  ss << this->maxHeartBeatRetryTimes_;
+  ss << ", heartbeatPeriodAfterFail=";
+  ss << this->heartbeatPeriodAfterFailSec_;
+  ss << "}";
+  return ss.str();
+}
+
+
+
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/utils.cc b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
new file mode 100644
index 0000000..8edbe09
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/utils.cc
@@ -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.
+ */
+
+#include <vector>
+#include <stdlib.h>
+#include "utils.h"
+
+namespace TubeMQ {
+
+
+string Utils::trim(const string& source) {
+  string target = source;
+  if(!target.empty()) {
+    size_t foudPos = target.find_first_not_of(tWhitespaceCharSet);
+    if (foudPos != string::npos) {
+      target = target.substr(foudPos);
+    }
+    foudPos = target.find_last_not_of(tWhitespaceCharSet);
+    if(foudPos != string::npos) {
+      target = target.substr(0, foudPos + 1);
+    }
+  }
+  return target;
+}
+
+void Utils::split(const string& source, map<string, int>& result, 
+      const string& delimiterStep1, const string& delimiterStep2) {
+  int tmpValue;
+  string subStr;
+  string keyStr;
+  string valStr;
+  string::size_type pos1,pos2,pos3;
+  if(!source.empty()) {
+    pos1 = 0;
+    pos2 = source.find(delimiterStep1);
+    while(string::npos != pos2) {
+      subStr = source.substr(pos1, pos2-pos1);
+      subStr = Utils::trim(subStr);
+      pos1 = pos2 + delimiterStep1.length();
+      pos2 = source.find(delimiterStep1, pos1);
+      if(subStr.empty()) {
+        continue;
+      }
+      pos3 = subStr.find(delimiterStep2);
+      if(string::npos == pos3) {
+        continue;
+      }
+      keyStr = subStr.substr(0, pos3);
+      valStr = subStr.substr(pos3+delimiterStep2.length());
+      keyStr = Utils::trim(keyStr);
+      valStr = Utils::trim(valStr);
+      if(keyStr.empty()) {
+        continue;
+      }
+      result[keyStr] = atoi(valStr.c_str());
+    }
+    if(pos1 != source.length()) {
+      subStr = source.substr(pos1);
+      subStr = Utils::trim(subStr);
+      pos3 = subStr.find(delimiterStep2);
+      if(string::npos != pos3) {
+        keyStr = subStr.substr(0, pos3);
+        valStr = subStr.substr(pos3+delimiterStep2.length());
+        keyStr = Utils::trim(keyStr);
+        valStr = Utils::trim(valStr);
+        if(!keyStr.empty()){
+          result[keyStr] = atoi(valStr.c_str());
+        }
+      }
+    }
+  }
+}
+
+}
+


[incubator-tubemq] 28/50: [TUBEMQ-269]Create C/C++ RmtDataCache class (#199)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 3be5ddc7c315b19776b98996a4096973417ee81a
Author: gosonzhang <46...@qq.com>
AuthorDate: Sat Jul 11 09:00:59 2020 +0000

    [TUBEMQ-269]Create C/C++ RmtDataCache class (#199)
    
    Co-authored-by: gosonzhang <go...@tencent.com>
---
 .../include/tubemq/rmt_data_cache.h                |  19 +-
 .../tubemq-client-cpp/src/meta_info.cc             |   4 +-
 .../tubemq-client-cpp/src/rmt_data_cache.cc        | 226 ++++++++++++++++++++-
 3 files changed, 240 insertions(+), 9 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
index 11f9018..a4daf91 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/rmt_data_cache.h
@@ -20,10 +20,9 @@
 #ifndef TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
 #define TUBEMQ_CLIENT_RMT_DATA_CACHE_H_
 
-#include <stdint.h>
 #include <pthread.h>
+#include <stdint.h>
 
-#include <atomic>
 #include <list>
 #include <map>
 #include <set>
@@ -47,12 +46,26 @@ class RmtDataCacheCsm {
   RmtDataCacheCsm();
   ~RmtDataCacheCsm();
   void AddNewPartition(const PartitionExt& partition_ext);
+  bool SelectPartition(string &err_info,
+           PartitionExt& partition_ext, string& confirm_context);
+  void BookedPartionInfo(const string& partition_key, int64_t curr_offset,
+                            int32_t err_code, bool esc_limit, int32_t msg_size,
+                            int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow);
+  bool RelPartition(string &err_info, bool is_filterconsume,
+                         const string& confirm_context, bool is_consumed);
+  void RemovePartition(const list<PartitionExt>& partition_list);
+  bool RemovePartition(string &err_info, const string& confirm_context);
   void OfferEvent(const ConsumerEvent& event);
   void TakeEvent(ConsumerEvent& event);
   void ClearEvent();
   void OfferEventResult(const ConsumerEvent& event);
   bool PollEventResult(ConsumerEvent& event);
 
+ private:
+  void buildConfirmContext(const string& partition_key,
+                    int64_t booked_time, string& confirm_context);
+  bool parseConfirmContext(string &err_info,
+    const string& confirm_context, string& partition_key, int64_t& booked_time);
 
  private:
   // timer begin
@@ -68,6 +81,8 @@ class RmtDataCacheCsm {
   map<string, set<string> > topic_partition_;
   // broker parition map
   map<NodeInfo, set<string> > broker_partition_;
+  // for idle partitions occupy
+  pthread_mutex_t  part_mutex_;
   // for partiton idle map
   list<string> index_partitions_;
   // for partition used map
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
index 4f0f860..c01c260 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/meta_info.cc
@@ -443,7 +443,9 @@ const uint32_t SubscribeInfo::GetBrokerPort() const { return this->partitionext_
 
 const string& SubscribeInfo::GetTopic() const { return this->partitionext_.GetTopic(); }
 
-const uint32_t SubscribeInfo::GetPartitionId() const { return this->partitionext_.GetPartitionId(); }
+const uint32_t SubscribeInfo::GetPartitionId() const {
+  return this->partitionext_.GetPartitionId();
+}
 
 const string& SubscribeInfo::ToString() const { return this->sub_info_; }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
index d239d26..cb596ec 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/rmt_data_cache.cc
@@ -18,15 +18,25 @@
  */
 
 #include "tubemq/rmt_data_cache.h"
+
+#include <string>
+
+#include <stdlib.h>
+
+#include "tubemq/const_config.h"
 #include "tubemq/meta_info.h"
+#include "tubemq/utils.h"
 
 
 
 namespace tubemq {
- 
+
+
+
 
 RmtDataCacheCsm::RmtDataCacheCsm() {
   pthread_rwlock_init(&meta_rw_lock_, NULL);
+  pthread_mutex_init(&part_mutex_, NULL);
   pthread_mutex_init(&data_book_mutex_, NULL);
   pthread_mutex_init(&event_read_mutex_, NULL);
   pthread_cond_init(&event_read_cond_, NULL);
@@ -38,6 +48,7 @@ RmtDataCacheCsm::~RmtDataCacheCsm() {
   pthread_mutex_destroy(&event_read_mutex_);
   pthread_mutex_destroy(&data_book_mutex_);
   pthread_cond_destroy(&event_read_cond_);
+  pthread_mutex_destroy(&part_mutex_);
   pthread_rwlock_destroy(&meta_rw_lock_);
 }
 
@@ -74,7 +85,7 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
     }
   }
   // check partition_key status
-  if (partition_useds_.find(partition_key) == partition_useds_.end() 
+  if (partition_useds_.find(partition_key) == partition_useds_.end()
     && partition_timeouts_.find(partition_key) == partition_timeouts_.end()) {
     index_partitions_.remove(partition_key);
     index_partitions_.push_back(partition_key);
@@ -82,6 +93,192 @@ void RmtDataCacheCsm::AddNewPartition(const PartitionExt& partition_ext) {
   pthread_rwlock_unlock(&meta_rw_lock_);
 }
 
+bool RmtDataCacheCsm::SelectPartition(string &err_info,
+                        PartitionExt& partition_ext, string& confirm_context) {
+  bool result = false;
+  int64_t booked_time = 0;
+  string partition_key;
+  map<string, PartitionExt>::iterator it_map;
+
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  if (partitions_.empty()) {
+    err_info = "No partition info in local cache, please retry later!";
+    result = false;
+  } else {
+    pthread_mutex_lock(&part_mutex_);
+    if (index_partitions_.empty()) {
+      err_info = "No idle partition to consume, please retry later!";
+      result = false;
+    } else {
+      result = false;
+      err_info = "No idle partition to consume data 2, please retry later!";
+      booked_time =Utils::GetCurrentTimeMillis();
+      partition_key = index_partitions_.front();
+      index_partitions_.pop_front();
+      buildConfirmContext(partition_key, booked_time, confirm_context);
+      it_map = partitions_.find(partition_key);
+      if (it_map != partitions_.end()) {
+        partition_ext = it_map->second;
+        partition_useds_[partition_key] = booked_time;
+        result = true;
+        err_info = "Ok";
+      }
+    }
+    pthread_mutex_unlock(&part_mutex_);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+  return result;
+}
+
+void RmtDataCacheCsm::BookedPartionInfo(const string& partition_key, int64_t curr_offset,
+                                             int32_t err_code, bool esc_limit, int32_t msg_size, 
+                                             int64_t limit_dlt, int64_t cur_data_dlt, bool require_slow) {
+  map<string, PartitionExt>::iterator it_part;
+  // book partition offset info
+  if (curr_offset >= 0) {
+    pthread_mutex_lock(&data_book_mutex_);
+    partition_offset_[partition_key] = curr_offset;
+    pthread_mutex_unlock(&data_book_mutex_);
+  }
+  // book partition temp info
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  it_part = partitions_.find(partition_key);
+  if(it_part != partitions_.end()) {
+    it_part->second.BookConsumeData(err_code, msg_size,
+              esc_limit, limit_dlt, cur_data_dlt, require_slow);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+bool RmtDataCacheCsm::RelPartition(string &err_info, bool is_filterconsume,
+                                 const string& confirm_context, bool is_consumed) {
+  int64_t wait_time;
+  int64_t booked_time;
+  string  partition_key;
+  map<string, PartitionExt>::iterator it_Part;
+  map<string, int64_t>::iterator it_used;
+  // parse confirm context  
+  bool result = parseConfirmContext(err_info,
+                      confirm_context, partition_key, booked_time);
+  if (!result) {
+    return false;
+  }
+  pthread_rwlock_rdlock(&meta_rw_lock_);
+  it_Part = partitions_.find(partition_key);
+  if (it_Part == partitions_.end()) {
+    pthread_mutex_lock(&part_mutex_);
+    partition_useds_.erase(partition_key);
+    index_partitions_.remove(partition_key);
+    pthread_mutex_unlock(&part_mutex_);
+    err_info = "Not found the partition in Consume Partition set!";
+    result = false;
+  } else {
+    pthread_mutex_lock(&part_mutex_);
+    it_used = partition_useds_.find(partition_key);
+    if (it_used == partition_useds_.end()) {
+      index_partitions_.remove(partition_key);
+      index_partitions_.push_back(partition_key);       
+    } else {
+      if (it_used->second == booked_time) {
+        partition_useds_.erase(partition_key);
+        wait_time = it_Part->second.ProcConsumeResult(def_flowctrl_handler_, 
+                      group_flowctrl_handler_, is_filterconsume, is_consumed);
+        if (wait_time >= 10) {
+          // todo add timer 
+          // end todo
+        } else {
+          partition_useds_.erase(partition_key);
+          index_partitions_.remove(partition_key);
+        }
+        err_info = "Ok";
+        result = true;    
+      } else {
+        err_info = "Illegel confirmContext content: context not equal!";
+        result = false;
+      }
+    }
+    pthread_mutex_unlock(&part_mutex_);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+  return result;
+}
+
+bool RmtDataCacheCsm::RemovePartition(string &err_info,
+                                  const string& confirm_context) {
+  int64_t booked_time;
+  string  partition_key;
+  map<string, PartitionExt>::iterator it_Part;
+  map<string, set<string> >::iterator it_topic;
+  map<NodeInfo, set<string> >::iterator it_broker;
+  // parse confirm context  
+  bool result = parseConfirmContext(err_info,
+                      confirm_context, partition_key, booked_time);
+  if (!result) {
+    return false;
+  }
+  // remove partiton
+  pthread_rwlock_wrlock(&meta_rw_lock_);
+  partition_useds_.erase(partition_key);
+  index_partitions_.remove(partition_key);
+  // todo need modify if timer build finished
+  partition_timeouts_.erase(partition_key);
+  // end todo
+  it_Part = partitions_.find(partition_key);
+  if (it_Part != partitions_.end()) {
+    it_topic = topic_partition_.find(it_Part->second.GetTopic());
+    if (it_topic != topic_partition_.end()) {
+      it_topic->second.erase(it_Part->second.GetPartitionKey());
+      if (it_topic->second.empty()) {
+        topic_partition_.erase(it_Part->second.GetTopic());
+      }
+    }
+    it_broker = broker_partition_.find(it_Part->second.GetBrokerInfo());
+    if (it_broker != broker_partition_.end()) {
+      it_broker->second.erase(it_Part->second.GetPartitionKey());
+      if (it_broker->second.empty()) {
+        broker_partition_.erase(it_Part->second.GetBrokerInfo());
+      }
+    }
+    partitions_.erase(partition_key);
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+  err_info = "Ok";
+  return true;
+}
+
+void RmtDataCacheCsm::RemovePartition(const list<PartitionExt>& partition_list) {
+  list<PartitionExt>::const_iterator it_lst;
+  map<string, PartitionExt>::iterator it_Part;
+  map<string, set<string> >::iterator it_topic;
+  map<NodeInfo, set<string> >::iterator it_broker;
+
+  pthread_rwlock_wrlock(&meta_rw_lock_);
+  for (it_lst = partition_list.begin(); it_lst != partition_list.end(); it_lst++) {
+    partition_useds_.erase(it_lst->GetPartitionKey());
+    index_partitions_.remove(it_lst->GetPartitionKey());
+    partitions_.erase(it_lst->GetPartitionKey());
+    // todo need modify if timer build finished
+    partition_timeouts_.erase(it_lst->GetPartitionKey());
+    // end todo
+		it_topic = topic_partition_.find(it_lst->GetTopic());
+		if (it_topic != topic_partition_.end()) {
+      it_topic->second.erase(it_lst->GetPartitionKey());
+      if (it_topic->second.empty()) {
+        topic_partition_.erase(it_lst->GetTopic());
+      }
+    }
+    it_broker = broker_partition_.find(it_lst->GetBrokerInfo());
+		if (it_broker != broker_partition_.end()) {
+      it_broker->second.erase(it_lst->GetPartitionKey());
+      if (it_broker->second.empty()) {
+        broker_partition_.erase(it_lst->GetBrokerInfo());
+      }
+    }
+  }
+  pthread_rwlock_unlock(&meta_rw_lock_);
+}
+
+
 void RmtDataCacheCsm::OfferEvent(const ConsumerEvent& event) {
   pthread_mutex_lock(&event_read_mutex_);
   this->rebalance_events_.push_back(event);
@@ -123,10 +320,27 @@ bool RmtDataCacheCsm::PollEventResult(ConsumerEvent& event) {
   return result;
 }
 
+void RmtDataCacheCsm::buildConfirmContext(const string& partition_key,
+                                   int64_t booked_time, string& confirm_context) {
+  confirm_context.clear();
+  confirm_context += partition_key;
+  confirm_context += delimiter::kDelimiterAt;
+  confirm_context += Utils::Long2str(booked_time);
+}
 
-
-
-
-
+bool RmtDataCacheCsm::parseConfirmContext(string &err_info,
+     const string& confirm_context, string& partition_key, int64_t& booked_time) {
+  //
+  vector<string> result;
+  Utils::Split(confirm_context, result, delimiter::kDelimiterAt); 
+  if(result.empty()) {
+    err_info = "Illegel confirmContext content: unregular value format!";
+    return false;
+  }
+  partition_key = result[0];
+  booked_time = (int64_t)atol(result[1].c_str());
+  err_info = "Ok";
+  return true;
+}
 
 }  // namespace tubemq


[incubator-tubemq] 42/50: TUBEMQ-287 add io buffer (#253)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 9929b10555e405e41c92d76026fe2f55a5cbe318
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Sat Sep 12 09:36:11 2020 +0800

    TUBEMQ-287 add io buffer (#253)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 tubemq-client-twins/tubemq-client-cpp/src/buffer.h | 356 +++++++++++++++++++++
 1 file changed, 356 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/buffer.h b/tubemq-client-twins/tubemq-client-cpp/src/buffer.h
new file mode 100644
index 0000000..0851375
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/buffer.h
@@ -0,0 +1,356 @@
+/**
+ * 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.
+ */
+
+// Modified from evpp
+// @see https://github.com/Qihoo360/evpp/blob/master/evpp/buffer.h
+
+#ifndef _TUBEMQ_BUFFER_H_
+#define _TUBEMQ_BUFFER_H_
+
+#include <arpa/inet.h>
+#include <assert.h>
+#include <stdint.h>
+#include <string.h>
+
+#include <algorithm>
+#include <memory>
+#include <string>
+
+namespace tubemq {
+
+class Buffer;
+using BufferPtr = std::shared_ptr<Buffer>;
+
+class Buffer {
+ public:
+  static const size_t kCheapPrependSize = 0;
+  static const size_t kInitialSize = 8192;
+
+  explicit Buffer(size_t initial_size = kInitialSize,
+                  size_t reserved_prepend_size = kCheapPrependSize)
+      : capacity_(reserved_prepend_size + initial_size),
+        read_index_(reserved_prepend_size),
+        write_index_(reserved_prepend_size),
+        reserved_prepend_size_(reserved_prepend_size) {
+    buffer_ = new char[capacity_];
+    has_mem_ = true;
+    assert(length() == 0);
+    assert(WritableBytes() == initial_size);
+    assert(PrependableBytes() == reserved_prepend_size);
+  }
+
+  ~Buffer() {
+    if (has_mem_) {
+      delete[] buffer_;
+    }
+    buffer_ = nullptr;
+    capacity_ = 0;
+  }
+
+  std::string String() {
+    char buf[1024];
+    snprintf(buf, sizeof(buf),
+             "buffer:%p,capacity:%ld,readindex:%ld,writeindex:%ld,prependsize:%ld,hasmem:%d",
+             buffer_, capacity_, read_index_, write_index_, reserved_prepend_size_, has_mem_);
+    return buf;
+  }
+
+  BufferPtr Slice() {
+    auto buff = std::make_shared<Buffer>(*this);
+    buff->has_mem_ = false;
+    return buff;
+  }
+
+  void Swap(Buffer& rhs) {
+    std::swap(buffer_, rhs.buffer_);
+    std::swap(capacity_, rhs.capacity_);
+    std::swap(read_index_, rhs.read_index_);
+    std::swap(write_index_, rhs.write_index_);
+    std::swap(reserved_prepend_size_, rhs.reserved_prepend_size_);
+  }
+
+  // Skip advances the reading index of the buffer
+  void Skip(size_t len) {
+    if (len < length()) {
+      read_index_ += len;
+    } else {
+      Reset();
+    }
+  }
+
+  // Retrieve advances the reading index of the buffer
+  // Retrieve it the same as Skip.
+  void Retrieve(size_t len) { Skip(len); }
+
+  // Truncate discards all but the first n unread bytes from the buffer
+  // but continues to use the same allocated storage.
+  // It does nothing if n is greater than the length of the buffer.
+  void Truncate(size_t n) {
+    if (n == 0) {
+      read_index_ = reserved_prepend_size_;
+      write_index_ = reserved_prepend_size_;
+    } else if (write_index_ > read_index_ + n) {
+      write_index_ = read_index_ + n;
+    }
+  }
+
+  // Reset resets the buffer to be empty,
+  // but it retains the underlying storage for use by future writes.
+  // Reset is the same as Truncate(0).
+  void Reset() { Truncate(0); }
+
+  // Increase the capacity of the container to a value that's greater
+  // or equal to len. If len is greater than the current capacity(),
+  // new storage is allocated, otherwise the method does nothing.
+  void Reserve(size_t len) {
+    if (capacity_ >= len + reserved_prepend_size_) {
+      return;
+    }
+
+    // TODO add the implementation logic here
+    grow(len + reserved_prepend_size_);
+  }
+
+  // Make sure there is enough memory space to append more data with length len
+  void EnsureWritableBytes(size_t len) {
+    if (WritableBytes() < len) {
+      grow(len);
+    }
+
+    assert(WritableBytes() >= len);
+  }
+
+  // ToText appends char '\0' to buffer to convert the underlying data to a c-style string text.
+  // It will not change the length of buffer.
+  void ToText() {
+    AppendInt8('\0');
+    UnwriteBytes(1);
+  }
+
+  // Write
+ public:
+  void Write(const void* /*restrict*/ d, size_t len) {
+    EnsureWritableBytes(len);
+    memcpy(WriteBegin(), d, len);
+    assert(write_index_ + len <= capacity_);
+    write_index_ += len;
+  }
+
+  void Append(const char* /*restrict*/ d, size_t len) { Write(d, len); }
+
+  void Append(const void* /*restrict*/ d, size_t len) { Write(d, len); }
+
+  void AppendInt32(int32_t x) {
+    int32_t be32 = htonl(x);
+    Write(&be32, sizeof be32);
+  }
+
+  void AppendInt16(int16_t x) {
+    int16_t be16 = htons(x);
+    Write(&be16, sizeof be16);
+  }
+
+  void AppendInt8(int8_t x) { Write(&x, sizeof x); }
+
+  void PrependInt32(int32_t x) {
+    int32_t be32 = htonl(x);
+    Prepend(&be32, sizeof be32);
+  }
+
+  void PrependInt16(int16_t x) {
+    int16_t be16 = htons(x);
+    Prepend(&be16, sizeof be16);
+  }
+
+  void PrependInt8(int8_t x) { Prepend(&x, sizeof x); }
+
+  // Insert content, specified by the parameter, into the front of reading index
+  void Prepend(const void* /*restrict*/ d, size_t len) {
+    assert(len <= PrependableBytes());
+    read_index_ -= len;
+    const char* p = static_cast<const char*>(d);
+    memcpy(begin() + read_index_, p, len);
+  }
+
+  void UnwriteBytes(size_t n) {
+    assert(n <= length());
+    write_index_ -= n;
+  }
+
+  void WriteBytes(size_t n) {
+    assert(n <= WritableBytes());
+    write_index_ += n;
+  }
+
+  // Read
+ public:
+  // Peek int32_t/int16_t/int8_t with network endian
+
+  uint32_t ReadUint32() {
+    uint32_t result = PeekUint32();
+    Skip(sizeof result);
+    return result;
+  }
+
+  int32_t ReadInt32() {
+    int32_t result = PeekInt32();
+    Skip(sizeof result);
+    return result;
+  }
+
+  int16_t ReadInt16() {
+    int16_t result = PeekInt16();
+    Skip(sizeof result);
+    return result;
+  }
+
+  int8_t ReadInt8() {
+    int8_t result = PeekInt8();
+    Skip(sizeof result);
+    return result;
+  }
+
+  std::string ToString() const { return std::string(data(), length()); }
+
+  // ReadByte reads and returns the next byte from the buffer.
+  // If no byte is available, it returns '\0'.
+  char ReadByte() {
+    assert(length() >= 1);
+
+    if (length() == 0) {
+      return '\0';
+    }
+
+    return buffer_[read_index_++];
+  }
+
+  // UnreadBytes unreads the last n bytes returned
+  // by the most recent read operation.
+  void UnreadBytes(size_t n) {
+    assert(n < read_index_);
+    read_index_ -= n;
+  }
+
+  // Peek
+ public:
+  // Peek int64_t/int32_t/int16_t/int8_t with network endian
+
+  uint32_t PeekUint32() const {
+    assert(length() >= sizeof(uint32_t));
+    uint32_t be32 = 0;
+    ::memcpy(&be32, data(), sizeof be32);
+    return ntohl(be32);
+  }
+
+  int32_t PeekInt32() const {
+    assert(length() >= sizeof(int32_t));
+    int32_t be32 = 0;
+    ::memcpy(&be32, data(), sizeof be32);
+    return ntohl(be32);
+  }
+
+  int16_t PeekInt16() const {
+    assert(length() >= sizeof(int16_t));
+    int16_t be16 = 0;
+    ::memcpy(&be16, data(), sizeof be16);
+    return ntohs(be16);
+  }
+
+  int8_t PeekInt8() const {
+    assert(length() >= sizeof(int8_t));
+    int8_t x = *data();
+    return x;
+  }
+
+ public:
+  // data returns a pointer of length Buffer.length() holding the unread portion of the buffer.
+  // The data is valid for use only until the next buffer modification (that is,
+  // only until the next call to a method like Read, Write, Reset, or Truncate).
+  // The data aliases the buffer content at least until the next buffer modification,
+  // so immediate changes to the slice will affect the result of future reads.
+  const char* data() const { return buffer_ + read_index_; }
+
+  char* WriteBegin() { return begin() + write_index_; }
+
+  const char* WriteBegin() const { return begin() + write_index_; }
+
+  // length returns the number of bytes of the unread portion of the buffer
+  size_t length() const {
+    assert(write_index_ >= read_index_);
+    return write_index_ - read_index_;
+  }
+
+  // size returns the number of bytes of the unread portion of the buffer.
+  // It is the same as length().
+  size_t size() const { return length(); }
+
+  // capacity returns the capacity of the buffer's underlying byte slice, that is, the
+  // total space allocated for the buffer's data.
+  size_t capacity() const { return capacity_; }
+
+  size_t WritableBytes() const {
+    assert(capacity_ >= write_index_);
+    return capacity_ - write_index_;
+  }
+
+  size_t PrependableBytes() const { return read_index_; }
+
+ public:
+  char* begin() { return buffer_; }
+
+  const char* begin() const { return buffer_; }
+
+  void grow(size_t len) {
+    if (!has_mem_) {
+      return;
+    }
+    if (WritableBytes() + PrependableBytes() < len + reserved_prepend_size_) {
+      // grow the capacity
+      size_t n = (capacity_ << 1) + len;
+      size_t m = length();
+      char* d = new char[n];
+      memcpy(d + reserved_prepend_size_, begin() + read_index_, m);
+      write_index_ = m + reserved_prepend_size_;
+      read_index_ = reserved_prepend_size_;
+      capacity_ = n;
+      delete[] buffer_;
+      buffer_ = d;
+    } else {
+      // move readable data to the front, make space inside buffer
+      assert(reserved_prepend_size_ < read_index_);
+      size_t readable = length();
+      memmove(begin() + reserved_prepend_size_, begin() + read_index_, length());
+      read_index_ = reserved_prepend_size_;
+      write_index_ = read_index_ + readable;
+      assert(readable == length());
+      assert(WritableBytes() >= len);
+    }
+  }
+
+ private:
+  char* buffer_;
+  size_t capacity_;
+  size_t read_index_;
+  size_t write_index_;
+  size_t reserved_prepend_size_;
+  bool has_mem_{true};
+};
+
+}  // namespace tubemq
+#endif /* _TUBEMQ_BUFFER_H_ */


[incubator-tubemq] 31/50: [TUBEMQ-275]Thread Pool & Timer (#205)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 30e6fcf4c653c3a67fc264cc5adb7dbe3c7513a4
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Tue Jul 14 15:33:09 2020 +0800

    [TUBEMQ-275]Thread Pool & Timer (#205)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/CMakeLists.txt               | 33 ++++----
 tubemq-client-twins/tubemq-client-cpp/README.md    | 33 ++++++++
 .../tubemq-client-cpp/example/CMakeLists.txt       |  3 +-
 .../{src => example/executor_pool}/CMakeLists.txt  | 13 +---
 .../singleton.h => example/executor_pool/main.cc}  | 63 ++++++++--------
 .../include/tubemq/executor_pool.h                 | 88 ++++++++++++++++++++++
 .../tubemq-client-cpp/include/tubemq/singleton.h   |  5 +-
 .../tubemq-client-cpp/src/CMakeLists.txt           |  3 -
 .../tubemq-client-cpp/src/executor_pool.cc         | 87 +++++++++++++++++++++
 .../tubemq-client-cpp/third_party/README.md        | 23 ++++++
 .../tubemq-client-cpp/third_party/readme.md        |  2 -
 11 files changed, 288 insertions(+), 65 deletions(-)

diff --git a/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
index db9014a..984a28a 100644
--- a/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
@@ -22,6 +22,26 @@ cmake_minimum_required (VERSION 3.1)
 
 project (TubeMQ)
 
+set (CXX_FLAGS
+    -g
+    -fPIC
+    -Wall
+    -D__STDC_FORMAT_MACROS
+    -Wno-unused-parameter
+    -Wno-unused-function
+    -Wunused-variable
+    -Wunused-value
+    -Wshadow
+    -Wcast-qual
+    -Wcast-align
+    -Wwrite-strings
+    -Wsign-compare
+    -Winvalid-pch
+    -fms-extensions
+    -Wfloat-equal
+    -Wextra
+    -std=c++11
+    )
 
 INCLUDE_DIRECTORIES(include)
 
@@ -35,16 +55,3 @@ ADD_SUBDIRECTORY(third_party)
 ADD_SUBDIRECTORY(example)
 
 
-if (UNIX)
-    SET(CMAKE_CXX_FLAGS_DEBUG   "-O1 -g -ggdb -D_DEBUG")
-    SET(CMAKE_CXX_FLAGS_RELEASE "-O3 -g -ggdb -DNDEBUG")
-    SET(DEPENDENT_LIBRARIES log4cplus pthread)
-else (UNIX)
-    SET(DEPENDENT_LIBRARIES log4cplus)
-endif (UNIX)
-
-
-
-
-
-
diff --git a/tubemq-client-twins/tubemq-client-cpp/README.md b/tubemq-client-twins/tubemq-client-cpp/README.md
new file mode 100644
index 0000000..ecf0f6f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/README.md
@@ -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.
+
+-->
+
+
+# TubeMQ C++ client library
+## Requirements
+
+ * CMake
+ * [ASIO](https://github.com/chriskohlhoff/asio.git)
+ * [OpenSSL](https://github.com/openssl/openssl.git)
+ * [Protocol Buffer](https://developers.google.com/protocol-buffers/)
+ * [Log4cplus](https://github.com/log4cplus/log4cplus.git)
+ * [Rapidjson](https://github.com/Tencent/rapidjson.git)
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
index 1d8fcdd..9d729c2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
@@ -26,7 +26,8 @@ function(tubemq_add_example _name)
   set(_srcs ${ARGN})
   message (STATUS "${_name} sources: ${_srcs}")
   add_executable (${_name} ${_srcs})
-  TARGET_LINK_LIBRARIES (${_name} tubemq log4cplus pthread)
+  TARGET_LINK_LIBRARIES (${_name} tubemq ssl crypto log4cplus pthread)
 endfunction()
 
 add_subdirectory (log)
+add_subdirectory (executor_pool)
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt
similarity index 67%
copy from tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
copy to tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt
index a6e76e5..bef5bbb 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt
@@ -18,15 +18,4 @@
 #
 
 
-cmake_minimum_required (VERSION 3.1)
-
-
-set(CMAKE_C_FLAGS "-O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
-set(CMAKE_CXX_FLAGS "-std=c++11 -O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
-
-AUX_SOURCE_DIRECTORY(. CURRENT_DIR_SRCS)                                        
-ADD_LIBRARY(tubemq STATIC ${CURRENT_DIR_SRCS})   
-TARGET_LINK_LIBRARIES (tubemq)
-
-
-
+tubemq_add_example(executor_pool main.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h b/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc
similarity index 50%
copy from tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
copy to tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc
index fb22587..8c5ca25 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc
@@ -17,42 +17,39 @@
  * under the License.
  */
 
-#ifndef _TUBEMQ_SINGLETON_H
-#define _TUBEMQ_SINGLETON_H
-
-#include <assert.h>
-#include <stdlib.h>
-
-#include <mutex>
+#include <chrono>
+#include <exception>
+#include <functional>
+#include <iostream>
+#include <string>
 #include <thread>
 
-#include "tubemq/noncopyable.h"
+#include "tubemq/executor_pool.h"
 
-namespace tubemq {
+using namespace std;
+using namespace tubemq;
 
-template <typename T>
-class Singleton : noncopyable {
- public:
-  Singleton() = delete;
-  ~Singleton() = delete;
-
-  static T& instance() {
-    std::call_once(once_, Singleton::init);
-    assert(value_ != nullptr);
-    return *value_;
+void handler(int a, const asio::error_code& error) {
+  if (!error) {
+    // Timer expired.
+    std::cout << "handlertimeout:" << a << endl;
   }
+}
+
+int main() {
+  using namespace std::placeholders;  // for _1, _2, _3...
+  ExecutorPool pool(4);
+  auto timer = pool.Get()->CreateSteadyTimer();
+  timer->expires_after(std::chrono::seconds(5));
+  std::cout << "startwait" << endl;
+  timer->wait();
+  std::cout << "endwait" << endl;
+
+  timer->expires_after(std::chrono::milliseconds(100));
+  std::cout << "startsyncwait" << endl;
+  timer->async_wait(std::bind(handler, 5, _1));
+  std::cout << "endsyncwait" << endl;
+  std::this_thread::sleep_for(5s);
+  return 0;
+}
 
- private:
-  static void init() { value_ = new T(); }
-
- private:
-  static std::once_flag once_;
-  static T* value_;
-};
-
-template <typename T>
-T* Singleton<T>::value_ = nullptr;
-
-}  // namespace tubemq
-
-#endif  // _TUBEMQ_SINGLETON_H
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
new file mode 100644
index 0000000..0dd1f66
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/executor_pool.h
@@ -0,0 +1,88 @@
+/**
+ * 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 _TUBEMQ_EXECUTOR_POOL_
+#define _TUBEMQ_EXECUTOR_POOL_
+
+#include <stdlib.h>
+
+#include <asio.hpp>
+#include <asio/ssl.hpp>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <thread>
+
+#include "tubemq/noncopyable.h"
+
+namespace tubemq {
+
+typedef std::shared_ptr<asio::ip::tcp::socket> SocketPtr;
+typedef std::shared_ptr<asio::ssl::stream<asio::ip::tcp::socket &> > TlsSocketPtr;
+typedef std::shared_ptr<asio::ip::tcp::resolver> TcpResolverPtr;
+typedef std::shared_ptr<asio::steady_timer> SteadyTimerPtr;
+
+class Executor : noncopyable {
+ public:
+  Executor();
+  ~Executor();
+
+  SocketPtr CreateSocket();
+  TlsSocketPtr CreateTlsSocket(SocketPtr &socket, asio::ssl::context &ctx);
+  TcpResolverPtr CreateTcpResolver();
+  SteadyTimerPtr CreateSteadyTimer();
+  using func = std::function<void(void)>;
+
+  void Post(func task);
+
+  std::shared_ptr<asio::io_context> GetIoContext() { return io_context_; }
+
+  // Close executor and exit thread
+  void Close();
+
+ private:
+  void StartWorker(std::shared_ptr<asio::io_context> io_context);
+  std::shared_ptr<asio::io_context> io_context_;
+  using io_context_work = asio::executor_work_guard<asio::io_context::executor_type>;
+  io_context_work work_;
+  std::thread worker_;
+};
+
+typedef std::shared_ptr<Executor> ExecutorPtr;
+
+class ExecutorPool : noncopyable {
+ public:
+  explicit ExecutorPool(int nthreads = 2);
+
+  ExecutorPtr Get();
+
+  void Close();
+
+ private:
+  typedef std::vector<ExecutorPtr> ExecutorList;
+  ExecutorList executors_;
+  uint32_t executorIdx_;
+  std::mutex mutex_;
+  typedef std::unique_lock<std::mutex> Lock;
+};
+
+typedef std::shared_ptr<ExecutorPool> ExecutorPoolPtr;
+}  // namespace tubemq
+
+#endif  //_TUBEMQ_EXECUTOR_POOL_
diff --git a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
index fb22587..2761bda 100644
--- a/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
+++ b/tubemq-client-twins/tubemq-client-cpp/include/tubemq/singleton.h
@@ -36,7 +36,7 @@ class Singleton : noncopyable {
   Singleton() = delete;
   ~Singleton() = delete;
 
-  static T& instance() {
+  static T& Instance() {
     std::call_once(once_, Singleton::init);
     assert(value_ != nullptr);
     return *value_;
@@ -51,6 +51,9 @@ class Singleton : noncopyable {
 };
 
 template <typename T>
+std::once_flag Singleton<T>::once_;
+
+template <typename T>
 T* Singleton<T>::value_ = nullptr;
 
 }  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
index a6e76e5..5995d5c 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/src/CMakeLists.txt
@@ -21,9 +21,6 @@
 cmake_minimum_required (VERSION 3.1)
 
 
-set(CMAKE_C_FLAGS "-O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
-set(CMAKE_CXX_FLAGS "-std=c++11 -O2 -g -Wall -Werror -Wsign-compare -fno-strict-aliasing -fPIC")
-
 AUX_SOURCE_DIRECTORY(. CURRENT_DIR_SRCS)                                        
 ADD_LIBRARY(tubemq STATIC ${CURRENT_DIR_SRCS})   
 TARGET_LINK_LIBRARIES (tubemq)
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
new file mode 100644
index 0000000..a2536ae
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/executor_pool.cc
@@ -0,0 +1,87 @@
+/**
+ * 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 "tubemq/executor_pool.h"
+
+#include <asio.hpp>
+#include <functional>
+#include <memory>
+
+namespace tubemq {
+
+Executor::Executor()
+    : io_context_(new asio::io_context()),
+      work_(asio::make_work_guard(*io_context_)),
+      worker_(std::bind(&Executor::StartWorker, this, io_context_)) {}
+
+Executor::~Executor() {
+  Close();
+  if (worker_.joinable()) {
+    worker_.detach();
+  }
+}
+
+void Executor::StartWorker(std::shared_ptr<asio::io_context> io_context) { io_context_->run(); }
+
+SocketPtr Executor::CreateSocket() { return SocketPtr(new asio::ip::tcp::socket(*io_context_)); }
+
+TlsSocketPtr Executor::CreateTlsSocket(SocketPtr &socket, asio::ssl::context &ctx) {
+  return std::shared_ptr<asio::ssl::stream<asio::ip::tcp::socket &> >(
+      new asio::ssl::stream<asio::ip::tcp::socket &>(*socket, ctx));
+}
+
+TcpResolverPtr Executor::CreateTcpResolver() {
+  return TcpResolverPtr(new asio::ip::tcp::resolver(*io_context_));
+}
+
+SteadyTimerPtr Executor::CreateSteadyTimer() {
+  return SteadyTimerPtr(new asio::steady_timer(*io_context_));
+}
+
+void Executor::Close() {
+  io_context_->stop();
+  if (std::this_thread::get_id() != worker_.get_id() && worker_.joinable()) {
+    worker_.join();
+  }
+}
+
+void Executor::Post(Executor::func task) { io_context_->post(task); }
+
+ExecutorPool::ExecutorPool(int nthreads) : executors_(nthreads), executorIdx_(0), mutex_() {}
+
+ExecutorPtr ExecutorPool::Get() {
+  Lock lock(mutex_);
+
+  int idx = executorIdx_++ % executors_.size();
+  if (!executors_[idx]) {
+    executors_[idx] = std::make_shared<Executor>();
+  }
+
+  return executors_[idx];
+}
+
+void ExecutorPool::Close() {
+  for (auto it = executors_.begin(); it != executors_.end(); ++it) {
+    if (*it != nullptr) {
+      (*it)->Close();
+    }
+    it->reset();
+  }
+}
+}  // namespace tubemq
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/README.md b/tubemq-client-twins/tubemq-client-cpp/third_party/README.md
new file mode 100644
index 0000000..6497505
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/README.md
@@ -0,0 +1,23 @@
+<!--
+
+    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.
+
+-->
+
+#Third-party libraries#
+tubemq-client-cpp depends on several third-party libraries, their source code is available (usually as a git submodule) in this directory.
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md b/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md
deleted file mode 100644
index e1752ca..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/readme.md
+++ /dev/null
@@ -1,2 +0,0 @@
-#Third-party libraries#
-tubemq-client-cpp depends on several third-party libraries, their source code is available (usually as a git submodule) in this directory.
\ No newline at end of file


[incubator-tubemq] 44/50: [TUBEMQ-289]C++ SDK Codec TubeMQ proto support (#255)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit de6fb619fe1fcbd85f57ee1bc6aeda3e9f57a4e5
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Sat Sep 12 09:37:57 2020 +0800

    [TUBEMQ-289]C++ SDK Codec TubeMQ proto support (#255)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 tubemq-client-twins/tubemq-client-cpp/src/any.h    | 160 +++++++++++
 .../tubemq-client-cpp/src/transport.h              |  60 ++++
 .../tubemq-client-cpp/src/tubemq_codec.h           | 317 +++++++++++++++++++++
 .../tubemq-client-cpp/src/tubemq_transport.h       |  58 ++++
 4 files changed, 595 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/any.h b/tubemq-client-twins/tubemq-client-cpp/src/any.h
new file mode 100644
index 0000000..722d944
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/any.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.
+ */
+
+// Copy from evpp project
+// @see https://github.com/Qihoo360/evpp/blob/master/evpp/any.h
+
+#ifndef _TUBEMQ_ANY_H_
+#define _TUBEMQ_ANY_H_
+
+#include <assert.h>
+
+#include <algorithm>
+#include <typeinfo>
+
+namespace tubemq {
+
+// A variant type that can hold any other type.
+//
+// Usage 1 :
+//
+//    Buffer* buf(new Buffer());
+//    Any any(buf);
+//    Buffer* b = any_cast<Buffer*>(any);
+//    assert(buf == b);
+//    delete buf;
+//
+//
+// Usage 2 :
+//
+//    std::shared_ptr<Buffer> buf(new Buffer());
+//    Any any(buf);
+//    std::shared_ptr<Buffer> b = any_cast<std::shared_ptr<Buffer>>(any);
+//    assert(buf.get() == b.get());
+//
+//
+// Usage 3 :
+//
+//    std::shared_ptr<Buffer> buf(new Buffer());
+//    Any any(buf);
+//    std::shared_ptr<Buffer> b = any.Get<std::shared_ptr<Buffer>>();
+//    assert(buf.get() == b.get());
+//
+
+class Any {
+ public:
+  Any() : content_(nullptr) {}
+  ~Any() { delete content_; }
+
+  template <typename ValueType>
+  explicit Any(const ValueType& value) : content_(new Holder<ValueType>(value)) {}
+
+  Any(const Any& other) : content_(other.content_ ? other.content_->clone() : nullptr) {}
+
+ public:
+  Any& swap(Any& rhs) {
+    std::swap(content_, rhs.content_);
+    return *this;
+  }
+
+  template <typename ValueType>
+  Any& operator=(const ValueType& rhs) {
+    Any(rhs).swap(*this);
+    return *this;
+  }
+
+  Any& operator=(const Any& rhs) {
+    Any(rhs).swap(*this);
+    return *this;
+  }
+
+  bool IsEmpty() const { return !content_; }
+
+  const std::type_info& GetType() const { return content_ ? content_->GetType() : typeid(void); }
+
+  template <typename ValueType>
+  ValueType operator()() const {
+    return Get<ValueType>();
+  }
+
+  template <typename ValueType>
+  ValueType Get() const {
+    if (GetType() == typeid(ValueType)) {
+      return static_cast<Any::Holder<ValueType>*>(content_)->held_;
+    } else {
+      return ValueType();
+    }
+  }
+
+ protected:
+  class PlaceHolder {
+   public:
+    virtual ~PlaceHolder() {}
+
+   public:
+    virtual const std::type_info& GetType() const = 0;
+    virtual PlaceHolder* clone() const = 0;
+  };
+
+  template <typename ValueType>
+  class Holder : public PlaceHolder {
+   public:
+    Holder(const ValueType& value) : held_(value) {}
+
+    virtual const std::type_info& GetType() const { return typeid(ValueType); }
+
+    virtual PlaceHolder* clone() const { return new Holder(held_); }
+
+    ValueType held_;
+  };
+
+ protected:
+  PlaceHolder* content_;
+  template <typename ValueType>
+  friend ValueType* any_cast(Any*);
+};
+
+template <typename ValueType>
+ValueType* any_cast(Any* any) {
+  if (any && any->GetType() == typeid(ValueType)) {
+    return &(static_cast<Any::Holder<ValueType>*>(any->content_)->held_);
+  }
+
+  return nullptr;
+}
+
+template <typename ValueType>
+const ValueType* any_cast(const Any* any) {
+  return any_cast<ValueType>(const_cast<Any*>(any));
+}
+
+template <typename ValueType>
+ValueType any_cast(const Any& any) {
+  const ValueType* result = any_cast<ValueType>(&any);
+  assert(result);
+
+  if (!result) {
+    return ValueType();
+  }
+
+  return *result;
+}
+}  // namespace tubemq
+
+#endif /* _TUBEMQ_ANY_H_ */
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/transport.h b/tubemq-client-twins/tubemq-client-cpp/src/transport.h
new file mode 100644
index 0000000..c8b6c27
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/transport.h
@@ -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.
+ */
+
+#ifndef _TUBEMQ_TRANSPORT_H_
+#define _TUBEMQ_TRANSPORT_H_
+
+#include <string>
+
+#include "any.h"
+#include "buffer.h"
+#include "codec_protocol.h"
+#include "future.h"
+
+namespace tubemq {
+
+// On Close Callback
+using CloseNotifier = std::function<void(const std::error_code*)>;
+
+struct ResponseContext;
+using ResponseContextPtr = std::shared_ptr<ResponseContext>;
+
+struct RequestContext {
+  uint32_t request_id_{0};
+  std::string ip_;
+  uint32_t port_;
+  uint32_t timeout_{0};  // millisecond
+  uint32_t connection_pool_id_{0};
+  uint64_t create_time_ms_{0};  // create time millisecond
+
+  CodecProtocolPtr codec_;
+  CloseNotifier close_notifier_;
+
+  BufferPtr buf_;
+  Promise<ResponseContext> promise_;
+};
+using RequestContextPtr = std::shared_ptr<RequestContext>;
+
+struct ResponseContext {
+  Any rsp_;
+};
+
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_TRANSPORT_H_
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
new file mode 100644
index 0000000..89759df
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_codec.h
@@ -0,0 +1,317 @@
+/**
+ * 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 _TUBEMQ_TUBEMQ_CODEC_H_
+#define _TUBEMQ_TUBEMQ_CODEC_H_
+
+#include <assert.h>
+#include <google/protobuf/io/coded_stream.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+#include <google/protobuf/io/zero_copy_stream_impl.h>
+#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
+#include <google/protobuf/message.h>
+
+#include <string>
+
+#include "BrokerService.pb.h"
+#include "MasterService.pb.h"
+#include "RPC.pb.h"
+#include "any.h"
+#include "buffer.h"
+#include "codec_protocol.h"
+#include "const_config.h"
+#include "const_rpc.h"
+#include "logger.h"
+#include "tubemq/tubemq_errcode.h"
+#include "tubemq/tubemq_return.h"
+#include "utils.h"
+
+namespace tubemq {
+
+class TubeMQCodec final : public CodecProtocol {
+ public:
+  struct ReqProtocol {
+    int32_t rpc_read_timeout_;
+    uint32_t request_id_;
+    int32_t method_id_;
+    string prot_msg_;
+  };
+
+  struct RspProtocol {
+    int32_t serial_no_;
+    bool success_;
+    int32_t code_;
+    string error_msg_;
+    int64_t message_id_;
+    int32_t method_;
+    RspResponseBody rsp_body_;
+  };
+  using ReqProtocolPtr = std::shared_ptr<ReqProtocol>;
+  using RspProtocolPtr = std::shared_ptr<RspProtocol>;
+
+ public:
+  TubeMQCodec() {}
+
+  virtual ~TubeMQCodec() {}
+
+  virtual std::string Name() const { return "tubemq_v1"; }
+
+  virtual bool Decode(const BufferPtr &buff, Any &out) {
+    // check total length
+    LOG_TRACE("Decode: full message Decode come, begin decode");
+    int32_t total_len = buff->length();
+    if (total_len <= 0) {
+      LOG_TRACE("Decode: total_len <= 0, out, total_len = %d", total_len);
+      return false;
+    }
+    // check package is valid
+    RpcConnHeader rpc_header;
+    ResponseHeader rsp_header;
+    RspProtocolPtr rsp_protocol = GetRspProtocol();
+    // parse pb data
+    google::protobuf::io::ArrayInputStream rawOutput(buff->data(), total_len);
+    bool result = readDelimitedFrom(&rawOutput, &rpc_header);
+    if (!result) {
+      LOG_TRACE("Decode: parse RpcConnHeader failure, out");
+      return result;
+    }
+    result = readDelimitedFrom(&rawOutput, &rsp_header);
+    if (!result) {
+      LOG_TRACE("Decode: parse ResponseHeader failure, out");
+      return result;
+    }
+    ResponseHeader_Status rspStatus = rsp_header.status();
+    if (rspStatus == ResponseHeader_Status_SUCCESS) {
+      RspResponseBody response_body;
+      rsp_protocol->success_ = true;
+      rsp_protocol->code_ = err_code::kErrSuccess;
+      rsp_protocol->error_msg_ = "OK";
+      result = readDelimitedFrom(&rawOutput, &response_body);
+      if (!result) {
+        LOG_TRACE("Decode: parse RspResponseBody failure, out");
+        return false;
+      }
+      rsp_protocol->method_ = response_body.method();
+      rsp_protocol->rsp_body_ = response_body;
+    } else {
+      RspExceptionBody rpc_exception;
+      rsp_protocol->success_ = false;
+      result = readDelimitedFrom(&rawOutput, &rpc_exception);
+      if (!result) {
+        LOG_TRACE("Decode: parse RspExceptionBody failure, out");
+        return false;
+      }
+      string errInfo = rpc_exception.exceptionname();
+      errInfo += delimiter::kDelimiterPound;
+      errInfo += rpc_exception.stacktrace();
+      rsp_protocol->code_ = err_code::kErrRcvThrowError;
+      rsp_protocol->error_msg_ = errInfo;
+    }
+    out = Any(rsp_protocol);
+    LOG_TRACE("Decode: decode message success, finished");
+    return true;
+  }
+
+  virtual bool Encode(const Any &in, BufferPtr &buff) {
+    RequestBody req_body;
+    ReqProtocolPtr req_protocol = any_cast<ReqProtocolPtr>(in);
+
+    LOG_TRACE("Encode: begin encode message, request_id=%d, method_id=%d",
+              req_protocol->request_id_, req_protocol->method_id_);
+
+    req_body.set_method(req_protocol->method_id_);
+    req_body.set_timeout(req_protocol->rpc_read_timeout_);
+    req_body.set_request(req_protocol->prot_msg_);
+    RequestHeader req_header;
+    req_header.set_servicetype(Utils::GetServiceTypeByMethodId(req_protocol->method_id_));
+    req_header.set_protocolver(2);
+    RpcConnHeader rpc_header;
+    rpc_header.set_flag(rpc_config::kRpcFlagMsgRequest);
+    // calc total list size
+    uint32_t serial_len =
+        4 + rpc_header.ByteSizeLong() + 4 + req_header.ByteSizeLong() + 4 + req_body.ByteSizeLong();
+    std::string step_buff;
+    step_buff.resize(serial_len);
+    google::protobuf::io::ArrayOutputStream rawOutput((void *)step_buff.data(), serial_len);
+    bool result = writeDelimitedTo(rpc_header, &rawOutput);
+    if (!result) {
+      return result;
+    }
+    result = writeDelimitedTo(req_header, &rawOutput);
+    if (!result) {
+      return result;
+    }
+    result = writeDelimitedTo(req_body, &rawOutput);
+    if (!result) {
+      return result;
+    }
+    // append data to buffer
+    uint32_t list_size = calcBlockCount(serial_len);
+    buff->AppendInt32((int32_t)rpc_config::kRpcPrtBeginToken);
+    buff->AppendInt32((int32_t)req_protocol->request_id_);
+    buff->AppendInt32(list_size);
+    uint32_t write_pos = 0;
+    for (uint32_t i = 0; i < list_size; i++) {
+      uint32_t slice_len = serial_len - i * rpc_config::kRpcMaxBufferSize;
+      if (slice_len > rpc_config::kRpcMaxBufferSize) {
+        slice_len = rpc_config::kRpcMaxBufferSize;
+      }
+      LOG_TRACE("Encode: encode slice [%d] slice_len = %d, serial_len = %d", i, slice_len,
+                serial_len);
+      buff->AppendInt32(slice_len);
+      buff->Write(step_buff.data() + write_pos, slice_len);
+      write_pos += slice_len;
+    }
+    LOG_TRACE("Encode: encode message success, finished!");
+    return true;
+  }
+
+  // return code: -1 failed; 0-Unfinished; > 0 package buffer size
+  virtual int32_t Check(BufferPtr &in, Any &out, uint32_t &request_id, bool &has_request_id,
+                        size_t &package_length) {
+    // check package is valid
+    if (in->length() < 12) {
+      package_length = 12;
+      LOG_TRACE("Check: data's length < 12, is %ld, out", in->length());
+      return 0;
+    }
+    // check frameToken
+    uint32_t token = in->ReadUint32();
+    if (token != rpc_config::kRpcPrtBeginToken) {
+      LOG_TRACE("Check: first token is illegal, is %d, out", token);
+      return -1;
+    }
+    // get request_id
+    request_id = in->ReadUint32();
+    uint32_t list_size = in->ReadUint32();
+    if (list_size > rpc_config::kRpcMaxFrameListCnt) {
+      LOG_TRACE("Check: list_size over max, is %d, out", list_size);
+      return -1;
+    }
+    // check data list
+    uint32_t item_len = 0;
+    auto check_buf = in->Slice();
+    for (uint32_t i = 0; i < list_size; i++) {
+      if (check_buf->length() < 4) {
+        package_length += 4;
+        if (i > 0) {
+          package_length += i * rpc_config::kRpcMaxBufferSize;
+        }
+        LOG_TRACE("Check: buffer Remaining length < 4, is %ld, out", check_buf->length());
+        return 0;
+      }
+      item_len = check_buf->ReadUint32();
+      if (item_len == 0) {
+        LOG_TRACE("Check: slice length == 0, is %d, out", item_len);
+        return -1;
+      }
+      if (item_len > rpc_config::kRpcMaxBufferSize) {
+        LOG_TRACE("Check: item_len(%d) > max item length(%d), out", item_len,
+                  rpc_config::kRpcMaxBufferSize);
+        return -1;
+      }
+      if (item_len > check_buf->length()) {
+        package_length += 4 + item_len;
+        if (i > 0) {
+          package_length += i * rpc_config::kRpcMaxBufferSize;
+        }
+        LOG_TRACE("Check: item_len(%d) > remaining length(%ld), out", item_len,
+                  check_buf->length());
+        return 0;
+      }
+      check_buf->Skip(item_len);
+    }
+    has_request_id = true;
+    uint32_t readed_len = 12;
+    auto buf = std::make_shared<Buffer>();
+    for (uint32_t i = 0; i < list_size; i++) {
+      item_len = in->ReadUint32();
+      readed_len += 4;
+      buf->Write(in->data(), item_len);
+      readed_len += item_len;
+      in->Skip(item_len);
+    }
+    out = buf;
+    LOG_TRACE("Check: received message check finished, request_id=%d, readed_len:%d",
+      request_id, readed_len);
+    return readed_len;
+  }
+
+  static ReqProtocolPtr GetReqProtocol() { return std::make_shared<ReqProtocol>(); }
+  static RspProtocolPtr GetRspProtocol() { return std::make_shared<RspProtocol>(); }
+
+  static bool readDelimitedFrom(google::protobuf::io::ZeroCopyInputStream *rawInput,
+                                google::protobuf::MessageLite *message) {
+    // We create a new coded stream for each message.  Don't worry, this is fast,
+    // and it makes sure the 64MB total size limit is imposed per-message rather
+    // than on the whole stream.  (See the CodedInputStream interface for more
+    // info on this limit.)
+    google::protobuf::io::CodedInputStream input(rawInput);
+
+    // Read the size.
+    uint32_t size;
+    if (!input.ReadVarint32(&size)) return false;
+    // Tell the stream not to read beyond that size.
+    google::protobuf::io::CodedInputStream::Limit limit = input.PushLimit(size);
+    // Parse the message.
+    if (!message->MergeFromCodedStream(&input)) return false;
+    if (!input.ConsumedEntireMessage()) return false;
+
+    // Release the limit.
+    input.PopLimit(limit);
+
+    return true;
+  }
+
+  static bool writeDelimitedTo(const google::protobuf::MessageLite &message,
+                               google::protobuf::io::ZeroCopyOutputStream *rawOutput) {
+    // We create a new coded stream for each message.  Don't worry, this is fast.
+    google::protobuf::io::CodedOutputStream output(rawOutput);
+
+    // Write the size.
+    const int32_t size = message.ByteSizeLong();
+    output.WriteVarint32(size);
+
+    uint8_t *buffer = output.GetDirectBufferForNBytesAndAdvance(size);
+    if (buffer != NULL) {
+      // Optimization:  The message fits in one buffer, so use the faster
+      // direct-to-array serialization path.
+      message.SerializeWithCachedSizesToArray(buffer);
+    } else {
+      // Slightly-slower path when the message is multiple buffers.
+      message.SerializeWithCachedSizes(&output);
+      if (output.HadError()) return false;
+    }
+
+    return true;
+  }
+
+  uint32_t calcBlockCount(uint32_t content_len) {
+    uint32_t block_cnt = content_len / rpc_config::kRpcMaxBufferSize;
+    uint32_t remain_size = content_len % rpc_config::kRpcMaxBufferSize;
+    if (remain_size > 0) {
+      block_cnt++;
+    }
+    return block_cnt;
+  }
+
+ };
+}  // namespace tubemq
+#endif
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/tubemq_transport.h b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_transport.h
new file mode 100644
index 0000000..ffee0c1
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/tubemq_transport.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 _TUBEMQ_TUBEMQ_TRANSPORT_H_
+#define _TUBEMQ_TUBEMQ_TRANSPORT_H_
+
+#include "buffer.h"
+#include "codec_protocol.h"
+#include "connection_pool.h"
+#include "executor_pool.h"
+#include "future.h"
+#include "logger.h"
+#include "transport.h"
+#include "tubemq_codec.h"
+
+namespace tubemq {
+
+template <typename RequestProtocol>
+Future<ResponseContext> AsyncRequest(RequestContextPtr& request, RequestProtocol& protocol) {
+  request->buf_ = std::make_shared<Buffer>();
+  Any in(protocol);
+  request->codec_->Encode(in, request->buf_);
+  auto future = request->promise_.GetFuture();
+  auto pool = TubeMQService::Instance()->GetConnectionPool();
+  if (pool != nullptr) {
+    pool->GetConnection(request)->AsyncWrite(request);
+  } else {
+    request->promise_.SetFailed(ErrorCode(err_code::kErrServerStop, "server is stop"));
+  }
+  return future;
+}
+
+template <typename RequestProtocol>
+ErrorCode SyncRequest(ResponseContext& response_context, RequestContextPtr& request,
+                      RequestProtocol& protocol) {
+  auto future = AsyncRequest(request, protocol);
+  return future.Get(response_context);
+}
+
+}  // namespace tubemq
+
+#endif  // _TUBEMQ_TUBEMQ_TRANSPORT_H_


[incubator-tubemq] 47/50: [TUBEMQ-349]C++ SDK Create Thread Pool (#259)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit 8ff63ddd182c2382b8da7dd5a43b33e452d629dc
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Mon Sep 14 20:24:56 2020 +0800

    [TUBEMQ-349]C++ SDK Create Thread Pool (#259)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .../tubemq-client-cpp/src/thread_pool.h            | 68 ++++++++++++++++++++++
 1 file changed, 68 insertions(+)

diff --git a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
new file mode 100644
index 0000000..1adcdf8
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
@@ -0,0 +1,68 @@
+/**
+ * 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 _TUBEMQ_THREAD_POOL_
+#define _TUBEMQ_THREAD_POOL_
+
+#include <stdlib.h>
+
+#include <asio.hpp>
+#include <asio/ssl.hpp>
+#include <chrono>
+#include <functional>
+#include <memory>
+#include <mutex>
+#include <thread>
+#include <vector>
+
+#include "noncopyable.h"
+
+namespace tubemq {
+// ThreadPool use one io_context for thread pool
+class ThreadPool : noncopyable {
+ public:
+  explicit ThreadPool(std::size_t size)
+      : io_context_(size), work_(asio::make_work_guard(io_context_)) {
+    for (size_t i = 0; i < size; ++i) {
+      workers_.emplace_back([this] { io_context_.run(); });
+    }
+  }
+
+  ~ThreadPool() {
+    work_.reset();
+    io_context_.stop();
+    for (std::thread &worker : workers_) {
+      worker.join();
+    }
+    workers_.clear();
+  }
+
+  template <class function>
+  void Post(function f) {
+    io_context_.post(f);
+  }
+
+ private:
+  asio::io_context io_context_;
+  using io_context_work = asio::executor_work_guard<asio::io_context::executor_type>;
+  io_context_work work_;
+  std::vector<std::thread> workers_;
+};  // namespace tubemq
+}  // namespace tubemq
+#endif  // _TUBEMQ_THREAD_POOL_


[incubator-tubemq] 50/50: [TUBEMQ-351]C++ SDK example&tests (#263)

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit cd5ba19db9370205c3fbba1845741cc66efec9a2
Author: charlely <41...@users.noreply.github.com>
AuthorDate: Tue Sep 15 16:51:44 2020 +0800

    [TUBEMQ-351]C++ SDK example&tests (#263)
    
    Co-authored-by: charleli <ch...@tencent.com>
---
 .gitmodules                                        |   4 -
 .../tubemq-client-cpp/CMakeLists.txt               |  37 +---
 tubemq-client-twins/tubemq-client-cpp/README.md    |   7 +-
 .../tubemq-client-cpp/build_linux.sh               |   7 -
 .../tubemq-client-cpp/example/CMakeLists.txt       |   6 +-
 .../example/{log => consumer}/CMakeLists.txt       |   3 +-
 .../example/consumer/test_consumer.cc              | 137 ++++++++++++
 .../example/consumer/test_multithread_pull.cc      | 166 ++++++++++++++
 .../{third_party => proto}/CMakeLists.txt          |   9 +-
 .../tubemq-client-cpp/src/baseconsumer.cc          |  22 +-
 .../tubemq-client-cpp/src/client_connection.cc     |  38 +++-
 .../tubemq-client-cpp/src/client_connection.h      |   2 +-
 .../tubemq-client-cpp/src/client_service.h         | 242 ++++++++++-----------
 .../tubemq-client-cpp/src/thread_pool.h            |   2 +-
 .../{example => tests}/CMakeLists.txt              |   5 +-
 .../{example/log/main.cc => tests/README.md}       |  68 ++----
 .../executor_pool/CMakeLists.txt                   |   2 +-
 .../{example => tests}/executor_pool/main.cc       |   9 +-
 .../{example => tests}/log/CMakeLists.txt          |   2 +-
 .../{example => tests}/log/main.cc                 |  32 ++-
 .../log => tests/thread_pool}/CMakeLists.txt       |   2 +-
 .../{example/log => tests/thread_pool}/main.cc     |  16 +-
 .../tubemq-client-cpp/third_party/CMakeLists.txt   |  62 ++++--
 .../tubemq-client-cpp/third_party/log4cplus        |   1 -
 .../tubemq-client-cpp/third_party/rapidjson        |   1 -
 25 files changed, 581 insertions(+), 301 deletions(-)

diff --git a/.gitmodules b/.gitmodules
index c80f44d..417049c 100644
--- a/.gitmodules
+++ b/.gitmodules
@@ -2,7 +2,3 @@
 	path = tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
 	url = https://github.com/Tencent/rapidjson.git
 	tag = v1.1.0
-[submodule "tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus"]
-	path = tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
-	url = https://github.com/log4cplus/log4cplus
-	tag = REL_2_0_5
diff --git a/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
index 984a28a..94ac9f1 100644
--- a/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/CMakeLists.txt
@@ -22,36 +22,23 @@ cmake_minimum_required (VERSION 3.1)
 
 project (TubeMQ)
 
-set (CXX_FLAGS
-    -g
-    -fPIC
-    -Wall
-    -D__STDC_FORMAT_MACROS
-    -Wno-unused-parameter
-    -Wno-unused-function
-    -Wunused-variable
-    -Wunused-value
-    -Wshadow
-    -Wcast-qual
-    -Wcast-align
-    -Wwrite-strings
-    -Wsign-compare
-    -Winvalid-pch
-    -fms-extensions
-    -Wfloat-equal
-    -Wextra
-    -std=c++11
-    )
+find_package(Protobuf REQUIRED)
+
+SET(CMAKE_CXX_FLAGS "-std=c++11 -O2 -g -Wall -Werror -Wsign-compare -Wfloat-equal -fno-strict-aliasing -fPIC -DASIO_STANDALONE")
+
+INCLUDE_DIRECTORIES(${CMAKE_BINARY_DIR}/proto)
 
 INCLUDE_DIRECTORIES(include)
+INCLUDE_DIRECTORIES(src)
 
-INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/third_party/rapidjson/include)
-INCLUDE_DIRECTORIES(${CMAKE_SOURCE_DIR}/third_party/log4cplus/include)
-INCLUDE_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/log4cplus/include)
-LINK_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/log4cplus/lib)
+INCLUDE_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/rapidjson/src/rapidjson/include)
+INCLUDE_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/include)
+LINK_DIRECTORIES(${CMAKE_BINARY_DIR}/third_party/lib)
 
-ADD_SUBDIRECTORY(src)
 ADD_SUBDIRECTORY(third_party)
+ADD_SUBDIRECTORY(proto)
+ADD_SUBDIRECTORY(src)
 ADD_SUBDIRECTORY(example)
+ADD_SUBDIRECTORY(tests)
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/README.md b/tubemq-client-twins/tubemq-client-cpp/README.md
index 0f566d5..c2c1960 100644
--- a/tubemq-client-twins/tubemq-client-cpp/README.md
+++ b/tubemq-client-twins/tubemq-client-cpp/README.md
@@ -30,9 +30,8 @@
  * [Log4cplus](https://github.com/log4cplus/log4cplus.git)
  * [Rapidjson](https://github.com/Tencent/rapidjson.git)
 
-## Install ASIO
-  * ./autogen.sh
-  * ./configure CFLAGS=-std=c++11 CPPFLAGS=-std=c++11 CXXFLAGS=-std=c++11
-  * make && make install
+## Step to build
+  * install protobuf
+  * ./build_linux.sh
 
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/build_linux.sh b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
index 7664d5b..fd2dfdb 100755
--- a/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
+++ b/tubemq-client-twins/tubemq-client-cpp/build_linux.sh
@@ -20,13 +20,6 @@
 
 #!/bin/bash
 
-cd ../../
-git submodule init
-git submodule update
-git submodule foreach --recursive git submodule init 
-git submodule foreach --recursive git submodule update 
-cd -
-
 mkdir build
 cd build
 cmake ../
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
index 9d729c2..14c0ec2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
@@ -26,8 +26,8 @@ function(tubemq_add_example _name)
   set(_srcs ${ARGN})
   message (STATUS "${_name} sources: ${_srcs}")
   add_executable (${_name} ${_srcs})
-  TARGET_LINK_LIBRARIES (${_name} tubemq ssl crypto log4cplus pthread)
+  TARGET_LINK_LIBRARIES (${_name} tubemq ssl crypto liblog4cplus.a libprotobuf.a tubemq_proto pthread rt)
 endfunction()
 
-add_subdirectory (log)
-add_subdirectory (executor_pool)
+add_subdirectory (consumer)
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/example/consumer/CMakeLists.txt
similarity index 88%
copy from tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
copy to tubemq-client-twins/tubemq-client-cpp/example/consumer/CMakeLists.txt
index b828bf8..8f8b7f2 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/example/consumer/CMakeLists.txt
@@ -18,4 +18,5 @@
 #
 
 
-tubemq_add_example(log main.cc)
+tubemq_add_example(consumer test_consumer.cc)
+tubemq_add_example(multi_pull test_multithread_pull.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_consumer.cc b/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_consumer.cc
new file mode 100644
index 0000000..bd1b487
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_consumer.cc
@@ -0,0 +1,137 @@
+/**
+ * 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 <string.h>
+#include <unistd.h>
+#include <signal.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <time.h>
+#include <fcntl.h>
+#include <errno.h>
+#include <libgen.h>
+#include <sys/time.h>
+#include <chrono>
+#include <set>
+#include <string>
+#include <thread>
+#include "tubemq/tubemq_client.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_errcode.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
+
+using namespace std;
+using namespace tubemq;
+
+using std::set;
+using std::string;
+using tubemq::ConsumerConfig;
+using tubemq::ConsumerResult;
+using tubemq::TubeMQConsumer;
+
+
+
+
+
+
+int main(int argc, char* argv[]) {
+  bool result;
+  string err_info;
+  string conf_file = "../conf/client.conf";
+  string group_name = "test_c_v8";
+  string master_addr = "10.215.128.83:8000,10.215.128.83:8000";
+  TubeMQConsumer consumer_1;
+
+  set<string> topic_list;
+  topic_list.insert("test_1");
+  ConsumerConfig consumer_config;
+
+  consumer_config.SetRpcReadTimeoutMs(20000);
+  result = consumer_config.SetMasterAddrInfo(err_info, master_addr);
+  if (!result) {
+    printf("\n Set Master AddrInfo failure: %s", err_info.c_str());
+    return -1;
+  }
+  result = consumer_config.SetGroupConsumeTarget(err_info, group_name, topic_list);
+  if (!result) {
+    printf("\n Set GroupConsume Target failure: %s", err_info.c_str());
+    return -1;
+  }
+  result = StartTubeMQService(err_info, conf_file);
+  if (!result) {
+    printf("\n StartTubeMQService failure: %s", err_info.c_str());
+    return -1;
+  }
+
+  result = consumer_1.Start(err_info, consumer_config);
+  if (!result) {
+    printf("\n Initial consumer failure, error is: %s ", err_info.c_str());
+    return -2;
+  }
+
+  ConsumerResult gentRet;
+  ConsumerResult confirm_result;
+  int64_t start_time = time(NULL);
+  do {
+    // 1. get Message;
+    result = consumer_1.GetMessage(gentRet);
+    if (result) {
+      // 2.1.1  if success, process message
+      list<Message> msgs = gentRet.GetMessageList();
+      printf("\n GetMessage success, msssage count =%ld ", msgs.size());
+      // 2.1.2 confirm message result
+      consumer_1.Confirm(gentRet.GetConfirmContext(), true, confirm_result);
+    } else {
+      // 2.2.1 if failure, check error code
+      // print error message if errcode not in
+      // [no partitions assigned, all partitions in use,
+      //    or all partitons idle, reach max position]
+      if (!(gentRet.GetErrCode() == err_code::kErrNotFound
+        || gentRet.GetErrCode() == err_code::kErrNoPartAssigned
+        || gentRet.GetErrCode() == err_code::kErrAllPartInUse
+        || gentRet.GetErrCode() == err_code::kErrAllPartWaiting)) {
+        printf("\n GetMessage failure, err_code=%d, err_msg is: %s ",
+          gentRet.GetErrCode(), gentRet.GetErrMessage().c_str());
+      }
+    }
+    // used for test, consume 10 minutes only
+    if (time(NULL) - start_time > 10 * 60) {
+      break;
+    }
+  } while (true);
+
+  getchar();  // for test hold the test thread
+  consumer_1.ShutDown();
+
+  getchar();  // for test hold the test thread
+  result = StopTubeMQService(err_info);
+  if (!result) {
+    printf("\n *** StopTubeMQService failure, reason is %s ", err_info.c_str());
+  }
+
+  printf("\n finishe test exist ");
+  return 0;
+}
+
+
+
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_multithread_pull.cc b/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_multithread_pull.cc
new file mode 100644
index 0000000..fc0bf8f
--- /dev/null
+++ b/tubemq-client-twins/tubemq-client-cpp/example/consumer/test_multithread_pull.cc
@@ -0,0 +1,166 @@
+/**
+ * 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 <errno.h>
+#include <fcntl.h>
+#include <libgen.h>
+#include <signal.h>
+#include <stdio.h>
+#include <string.h>
+#include <sys/time.h>
+#include <sys/types.h>
+#include <sys/stat.h>
+#include <unistd.h>
+#include <time.h>
+#include <string>
+#include <chrono>
+#include <set>
+#include <thread>
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_client.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_errcode.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
+
+using namespace std;
+using namespace tubemq;
+
+TubeMQConsumer consumer_1;
+
+
+
+AtomicLong last_print_time(0);
+AtomicLong last_msg_count(0);
+AtomicLong last_print_count(0);
+
+
+
+void calc_message_count(int64_t msg_count) {
+  int64_t last_time = last_print_time.Get();
+  int64_t cur_count = last_msg_count.AddAndGet(msg_count);
+  int64_t cur_time = time(NULL);
+  if (cur_count - last_print_count.Get() >= 50000
+    || cur_time - last_time > 90) {
+    if (last_print_time.CompareAndSet(last_time, cur_time)) {
+      printf("\n %ld Current message count=%ld", cur_time, last_msg_count.Get());
+      last_print_count.Set(cur_count);
+    }
+  }
+}
+
+
+void thread_task_pull(int32_t thread_no) {
+  bool result;
+  int64_t msg_count = 0;
+  ConsumerResult gentRet;
+  ConsumerResult confirm_result;
+  printf("\n thread_task_pull start: %d", thread_no);
+  do {
+    msg_count = 0;
+    // 1. get Message;
+    result = consumer_1.GetMessage(gentRet);
+    if (result) {
+      // 2.1.1  if success, process message
+      list<Message> msgs = gentRet.GetMessageList();
+      msg_count = msgs.size();
+      // 2.1.2 confirm message result
+      consumer_1.Confirm(gentRet.GetConfirmContext(), true, confirm_result);
+    } else {
+      // 2.2.1 if failure, check error code
+      // print error message if errcode not in
+      // [no partitions assigned, all partitions in use,
+      //    or all partitons idle, reach max position]
+      if (!(gentRet.GetErrCode() == err_code::kErrNotFound
+        || gentRet.GetErrCode() == err_code::kErrNoPartAssigned
+        || gentRet.GetErrCode() == err_code::kErrAllPartInUse
+        || gentRet.GetErrCode() == err_code::kErrAllPartWaiting)) {
+        if (gentRet.GetErrCode() == err_code::kErrMQServiceStop
+          || gentRet.GetErrCode() == err_code::kErrClientStop) {
+          break;
+        }
+        printf("\n GetMessage failure, err_code=%d, err_msg is: %s ",
+          gentRet.GetErrCode(), gentRet.GetErrMessage().c_str());
+      }
+    }
+    calc_message_count(msg_count);
+  } while (true);
+  printf("\n thread_task_pull finished: %d", thread_no);
+}
+
+
+int main(int argc, char* argv[]) {
+  bool result;
+  string err_info;
+  string conf_file = "../conf/client.conf";
+  string group_name = "test_c_v8";
+  string master_addr = "10.215.128.83:8000,10.215.128.83:8000";
+  int32_t thread_num = 15;
+  set<string> topic_list;
+  topic_list.insert("test_1");
+  ConsumerConfig consumer_config;
+
+  consumer_config.SetRpcReadTimeoutMs(20000);
+  result = consumer_config.SetMasterAddrInfo(err_info, master_addr);
+  if (!result) {
+    printf("\n Set Master AddrInfo failure: %s ", err_info.c_str());
+    return -1;
+  }
+  result = consumer_config.SetGroupConsumeTarget(err_info, group_name, topic_list);
+  if (!result) {
+    printf("\n Set GroupConsume Target failure: %s", err_info.c_str());
+    return -1;
+  }
+  result = StartTubeMQService(err_info, conf_file);
+  if (!result) {
+    printf("\n StartTubeMQService failure: %s", err_info.c_str());
+    return -1;
+  }
+
+  result = consumer_1.Start(err_info, consumer_config);
+  if (!result) {
+    printf("\n Initial consumer failure, error is: %s ", err_info.c_str());
+    return -2;
+  }
+
+  std::thread pull_threads[thread_num];
+  for (int32_t i = 0; i < thread_num; i++) {
+    pull_threads[i] = std::thread(thread_task_pull, i);
+  }
+
+  getchar();  // for test hold the test thread
+  consumer_1.ShutDown();
+  //
+  for (int32_t i = 0; i < thread_num; i++) {
+    if (pull_threads[i].joinable()) {
+      pull_threads[i].join();
+    }
+  }
+
+  getchar();  // for test hold the test thread
+  result = StopTubeMQService(err_info);
+  if (!result) {
+    printf("\n *** StopTubeMQService failure, reason is %s", err_info.c_str());
+  }
+
+  printf("\n finishe test exist");
+  return 0;
+}
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/proto/CMakeLists.txt
similarity index 74%
copy from tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
copy to tubemq-client-twins/tubemq-client-cpp/proto/CMakeLists.txt
index 7dc5e88..c10ad25 100644
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/proto/CMakeLists.txt
@@ -17,9 +17,10 @@
 # under the License.
 #
 
+cmake_minimum_required (VERSION 3.1)
 
-CMAKE_MINIMUM_REQUIRED(VERSION 3.1)                                             
-PROJECT(TubeMQThirdParty)
-
-ADD_SUBDIRECTORY(log4cplus) 
 
+file(GLOB ProtoFiles "${CMAKE_CURRENT_SOURCE_DIR}/*.proto")
+PROTOBUF_GENERATE_CPP(ProtoSources ProtoHeaders ${ProtoFiles})
+add_library(tubemq_proto STATIC ${ProtoSources} ${ProtoHeaders})
+target_link_libraries(tubemq_proto libprotobuf.a)
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
index 714b353..46d103b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/baseconsumer.cc
@@ -146,9 +146,7 @@ bool BaseConsumer::GetMessage(ConsumerResult& result) {
     result.SetFailureResult(error_code, err_info);
     return false;
   }
-
   int64_t curr_offset = tb_config::kInvalidValue;
-
   bool filter_consume = sub_info_.IsFilterConsume(partition_ext.GetTopic());
   PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
     partition_ext.GetPartitionKey(), curr_offset);
@@ -208,11 +206,9 @@ bool BaseConsumer::IsConsumeReady(ConsumerResult& result) {
     if (err_code::kErrSuccess == ret_code) {
       return true;
     }
-
     if ((config_.GetMaxPartCheckPeriodMs() >= 0)
       && (Utils::GetCurrentTimeMillis() - start_time
       >= config_.GetMaxPartCheckPeriodMs())) {
-
       switch (ret_code) {
         case err_code::kErrNoPartAssigned: {
           result.SetFailureResult(ret_code,
@@ -281,9 +277,7 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
   string part_key = Utils::Trim(confirm_context.substr(0, pos1));
   string booked_time_str =
       Utils::Trim(confirm_context.substr(pos1 + token1.size(), confirm_context.size()));
-
   int64_t booked_time = atol(booked_time_str.c_str());
-
   pos1 = part_key.find(token2);
   if (string::npos == pos1) {
     result.SetFailureResult(err_code::kErrBadRequest,
@@ -311,9 +305,7 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
                             "Not found the partition by confirm_context!");
     return false;
   }
-
   int64_t curr_offset = tb_config::kInvalidValue;
-
   PeerInfo peer_info(partition_ext.GetBrokerHost(), partition_ext.GetPartitionId(),
     partition_ext.GetPartitionKey(), curr_offset);
   auto request = std::make_shared<RequestContext>();
@@ -352,9 +344,7 @@ bool BaseConsumer::Confirm(const string& confirm_context, bool is_consumed,
     if (rsp->success_) {
       CommitOffsetResponseB2C rsp_b2c;
       ret_result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
-
                                           (int32_t)(rsp->rsp_body_.data().length()));
-
       if (ret_result) {
         if (rsp_b2c.success()) {
           curr_offset = rsp_b2c.curroffset();
@@ -449,7 +439,6 @@ bool BaseConsumer::register2Master(int32_t& error_code, string& err_info, bool n
       error_code = error.Value();
       err_info = error.Message();
     }
-
     if (error_code == err_code::kErrConsumeGroupForbidden
       || error_code == err_code::kErrConsumeContentForbidden) {
       // set regist process status to existed
@@ -816,8 +805,7 @@ void BaseConsumer::processHeartBeat2Broker(NodeInfo broker_info) {
           if (rsp->success_) {
             HeartBeatResponseB2C rsp_b2c;
             bool result = rsp_b2c.ParseFromArray(rsp->rsp_body_.data().c_str(),
-                                                (int32_t)(rsp->rsp_body_.data().length()));
-
+                                                 (int32_t)(rsp->rsp_body_.data().length()));
             if (result) {
               set<string> partition_keys;
               if (rsp_b2c.success()) {
@@ -1079,7 +1067,6 @@ bool BaseConsumer::processRegisterResponseM2C(int32_t& error_code, string& err_i
   RegisterResponseM2C rsp_m2c;
   bool result = rsp_m2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
                                        (int32_t)(rsp_protocol->rsp_body_.data().length()));
-
   if (!result) {
     error_code = err_code::kErrParseFailure;
     err_info = "Parse RegisterResponseM2C response failure!";
@@ -1237,7 +1224,6 @@ bool BaseConsumer::processRegResponseB2C(int32_t& error_code, string& err_info,
   RegisterResponseB2C rsp_b2c;
   bool result = rsp_b2c.ParseFromArray(rsp_protocol->rsp_body_.data().c_str(),
                                        (int32_t)(rsp_protocol->rsp_body_.data().length()));
-
   if (!result) {
     error_code = err_code::kErrParseFailure;
     err_info = "Parse RegisterResponseB2C response failure!";
@@ -1332,7 +1318,6 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
                                              bool filter_consume, const PartitionExt& partition_ext,
                                              const string& confirm_context,
                                              const TubeMQCodec::RspProtocolPtr& rsp) {
-
   // #lizard forgives
   string err_info;
 
@@ -1348,13 +1333,11 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
   GetMessageResponseB2C rsp_b2c;
   bool ret_result = rsp_b2c.ParseFromArray(
     rsp->rsp_body_.data().c_str(), (int32_t)(rsp->rsp_body_.data().length()));
-
   if (!ret_result) {
     rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false);
     result.SetFailureResult(err_code::kErrServerError,
                             "Parse GetMessageResponseB2C response failure!",
                             partition_ext.GetTopic(), peer_info);
-
     LOG_TRACE("[CONSUMER] processGetMessageRspB2C parse failure, client=%s", client_uuid_.c_str());
     return false;
   }
@@ -1362,12 +1345,10 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
   switch (rsp_b2c.errcode()) {
     case err_code::kErrSuccess: {
       bool esc_limit = (rsp_b2c.has_escflowctrl() && rsp_b2c.escflowctrl());
-
       int64_t data_dltval =
           rsp_b2c.has_currdatadlt() ? rsp_b2c.currdatadlt() : tb_config::kInvalidValue;
       int64_t curr_offset = rsp_b2c.has_curroffset() ?
         rsp_b2c.curroffset() : tb_config::kInvalidValue;
-
       bool req_slow = rsp_b2c.has_requireslow() ? rsp_b2c.requireslow() : false;
       int msg_size = 0;
       list<Message> message_list;
@@ -1392,7 +1373,6 @@ bool BaseConsumer::processGetMessageRspB2C(ConsumerResult& result, PeerInfo& pee
 
     case err_code::kErrConsumeSpeedLimit: {
       // Process with server side speed limit
-
       int64_t def_dlttime = rsp_b2c.has_minlimittime() ? rsp_b2c.minlimittime()
                                                     : config_.GetMsgNotFoundWaitPeriodMs();
       rmtdata_cache_.RelPartition(err_info, filter_consume, confirm_context, false,
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
index c795d35..e6ae503 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.cc
@@ -150,7 +150,7 @@ void ClientConnection::asyncRead() {
   }
   recv_buffer_->EnsureWritableBytes(rpc_config::kRpcEnsureWriteableBytes);
   auto self = shared_from_this();
-  socket_->async_read_some(
+  socket_->async_receive(
       asio::buffer(recv_buffer_->WriteBegin(), recv_buffer_->WritableBytes()),
       [self, this](std::error_code ec, std::size_t len) {
         if (ec) {
@@ -166,21 +166,36 @@ void ClientConnection::asyncRead() {
         }
         recv_time_ = std::time(nullptr);
         recv_buffer_->WriteBytes(len);
-        checkPackageDone();
-        LOG_TRACE("[%s]async read done, len:%ld, package_length_:%ld, recvbuffer:%s",
-                  ToString().c_str(), len, package_length_, recv_buffer_->String().c_str());
+        std::error_code error;
+        size_t availsize = socket_->available(error);
+        LOG_TRACE("[%s]async read done, len:%ld, package_length_:%ld, availsize:%ld, recvbuffer:%s",
+                  ToString().c_str(), len, package_length_, availsize,
+                  recv_buffer_->String().c_str());
+        if (availsize > 0 && !error) {
+          recv_buffer_->EnsureWritableBytes(availsize);
+          size_t rlen = socket_->receive(asio::buffer(recv_buffer_->WriteBegin(), availsize));
+          if (rlen > 0) {
+            recv_buffer_->WriteBytes(rlen);
+          }
+          LOG_TRACE("[%s]syncread done, receivelen:%ld, recvbuffer:%s", ToString().c_str(), rlen,
+                    recv_buffer_->String().c_str());
+        }
+        while (checkPackageDone() > 0 && recv_buffer_->length() > 0) {
+          LOG_TRACE("[%s]recheck packagedone package_length_:%ld, recvbuffer:%s",
+                    ToString().c_str(), package_length_, recv_buffer_->String().c_str());
+        }
         asyncRead();
       });
 }
 
-void ClientConnection::checkPackageDone() {
+int ClientConnection::checkPackageDone() {
   if (check_ == nullptr) {
     recv_buffer_->Reset();
     LOG_ERROR("check codec func not set");
-    return;
+    return -1;
   }
   if (package_length_ > recv_buffer_->length()) {
-    return;
+    return 0;
   }
   uint32_t request_id = 0;
   bool has_request_id = false;
@@ -192,11 +207,11 @@ void ClientConnection::checkPackageDone() {
     package_length_ = 0;
     LOG_ERROR("%s, check codec package result:%d", ToString().c_str(), result);
     close();
-    return;
+    return -1;
   }
   if (result == 0) {
     package_length_ = package_length;
-    return;
+    return 0;
   }
   ++read_package_number_;
   package_length_ = 0;
@@ -205,12 +220,13 @@ void ClientConnection::checkPackageDone() {
     auto it = request_list_.begin();
     if (it == request_list_.end()) {
       LOG_ERROR("%s, not find request", ToString().c_str());
-      return;
+      return 1;
     }
     requestCallback(it->first, nullptr, &check_out);
-    return;
+    return 1;
   }
   requestCallback(request_id, nullptr, &check_out);
+  return 1;
 }
 
 void ClientConnection::requestCallback(uint32_t request_id, ErrorCode* err, Any* check_out) {
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
index b6c7a36..2000e21 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_connection.h
@@ -78,7 +78,7 @@ class ClientConnection : public Connection, public std::enable_shared_from_this<
   void checkDeadline(const std::error_code& ec);
   void contextString();
   void asyncRead();
-  void checkPackageDone();
+  int checkPackageDone();
   void requestCallback(uint32_t request_id, ErrorCode* err = nullptr, Any* check_out = nullptr);
   TransportRequest* nextTransportRequest();
   void asyncWrite();
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/client_service.h b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
index 11f5b42..7271ca0 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/client_service.h
@@ -1,121 +1,121 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-
-#ifndef TUBEMQ_CLIENT_BASE_CLIENT_H_
-#define TUBEMQ_CLIENT_BASE_CLIENT_H_
-
-#include <stdint.h>
-
-#include <map>
-#include <mutex>
-#include <string>
-#include <thread>
-
-#include "connection_pool.h"
-#include "file_ini.h"
-#include "noncopyable.h"
-#include "rmt_data_cache.h"
-#include "thread_pool.h"
-#include "tubemq/tubemq_atomic.h"
-#include "tubemq/tubemq_config.h"
-#include "tubemq/tubemq_message.h"
-#include "tubemq/tubemq_return.h"
-
-namespace tubemq {
-
-using std::map;
-using std::mutex;
-using std::string;
-using std::thread;
-
-class BaseClient {
- public:
-  explicit BaseClient(bool is_producer);
-  virtual ~BaseClient();
-  virtual void ShutDown() {}
-  void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
-  bool IsProducer() { return is_producer_; }
-  const int32_t GetClientIndex() { return client_index_; }
-
- protected:
-  bool is_producer_;
-  int32_t client_index_;
-};
-
-class TubeMQService : public noncopyable {
- public:
-  static TubeMQService* Instance();
-  bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
-  bool Stop(string& err_info);
-  bool IsRunning();
-  const int32_t GetServiceStatus() const { return service_status_.Get(); }
-  int32_t GetClientObjCnt();
-  bool AddClientObj(string& err_info, BaseClient* client_obj);
-  BaseClient* GetClientObj(int32_t client_index) const;
-  void RmvClientObj(BaseClient* client_obj);
-  const string& GetLocalHost() const { return local_host_; }
-  ExecutorPoolPtr GetTimerExecutorPool() { return timer_executor_; }
-  SteadyTimerPtr CreateTimer() { return timer_executor_->Get()->CreateSteadyTimer(); }
-  ExecutorPoolPtr GetNetWorkExecutorPool() { return network_executor_; }
-  ConnectionPoolPtr GetConnectionPool() { return connection_pool_; }
-  template <class function>
-  void Post(function f) {
-    if (thread_pool_ != nullptr) {
-      thread_pool_->Post(f);
-    }
-  }
-  bool AddMasterAddress(string& err_info, const string& master_info);
-  void GetXfsMasterAddress(const string& source, string& target);
-
- protected:
-  void updMasterAddrByDns();
-
- private:
-  TubeMQService();
-  ~TubeMQService();
-  void iniLogger(const Fileini& fileini, const string& sector);
-  void iniPoolThreads(const Fileini& fileini, const string& sector);
-  void iniXfsThread(const Fileini& fileini, const string& sector);
-  void thread_task_dnsxfs(int dns_xfs_period_ms);
-  void shutDownClinets() const;
-  bool hasXfsTask(map<string, int32_t>& src_addr_map);
-  bool addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map);
-
- private:
-  static TubeMQService* _instance;
-  string local_host_;
-  AtomicInteger service_status_;
-  AtomicInteger client_index_base_;
-  mutable mutex mutex_;
-  map<int32_t, BaseClient*> clients_map_;
-  ExecutorPoolPtr timer_executor_;
-  ExecutorPoolPtr network_executor_;
-  ConnectionPoolPtr connection_pool_;
-  std::shared_ptr<ThreadPool> thread_pool_;
-  thread dns_xfs_thread_;
-  mutable mutex dns_mutex_;
-  int64_t last_check_time_;
-  map<string, int32_t> master_source_;
-  map<string, string> master_target_;
-};
-
-}  // namespace tubemq
-
-#endif  // TUBEMQ_CLIENT_BASE_CLIENT_H_
\ No newline at end of file
+/**
+ * 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 TUBEMQ_CLIENT_BASE_CLIENT_H_
+#define TUBEMQ_CLIENT_BASE_CLIENT_H_
+
+#include <stdint.h>
+
+#include <map>
+#include <mutex>
+#include <string>
+#include <thread>
+
+#include "connection_pool.h"
+#include "file_ini.h"
+#include "noncopyable.h"
+#include "rmt_data_cache.h"
+#include "thread_pool.h"
+#include "tubemq/tubemq_atomic.h"
+#include "tubemq/tubemq_config.h"
+#include "tubemq/tubemq_message.h"
+#include "tubemq/tubemq_return.h"
+
+namespace tubemq {
+
+using std::map;
+using std::mutex;
+using std::string;
+using std::thread;
+
+class BaseClient {
+ public:
+  explicit BaseClient(bool is_producer);
+  virtual ~BaseClient();
+  virtual void ShutDown() {}
+  void SetClientIndex(int32_t client_index) { client_index_ = client_index; }
+  bool IsProducer() { return is_producer_; }
+  const int32_t GetClientIndex() { return client_index_; }
+
+ protected:
+  bool is_producer_;
+  int32_t client_index_;
+};
+
+class TubeMQService : public noncopyable {
+ public:
+  static TubeMQService* Instance();
+  bool Start(string& err_info, string conf_file = "../conf/tubemqclient.conf");
+  bool Stop(string& err_info);
+  bool IsRunning();
+  const int32_t GetServiceStatus() const { return service_status_.Get(); }
+  int32_t GetClientObjCnt();
+  bool AddClientObj(string& err_info, BaseClient* client_obj);
+  BaseClient* GetClientObj(int32_t client_index) const;
+  void RmvClientObj(BaseClient* client_obj);
+  const string& GetLocalHost() const { return local_host_; }
+  ExecutorPoolPtr GetTimerExecutorPool() { return timer_executor_; }
+  SteadyTimerPtr CreateTimer() { return timer_executor_->Get()->CreateSteadyTimer(); }
+  ExecutorPoolPtr GetNetWorkExecutorPool() { return network_executor_; }
+  ConnectionPoolPtr GetConnectionPool() { return connection_pool_; }
+  template <class function>
+  void Post(function f) {
+    if (thread_pool_ != nullptr) {
+      thread_pool_->Post(f);
+    }
+  }
+  bool AddMasterAddress(string& err_info, const string& master_info);
+  void GetXfsMasterAddress(const string& source, string& target);
+
+ protected:
+  void updMasterAddrByDns();
+
+ private:
+  TubeMQService();
+  ~TubeMQService();
+  void iniLogger(const Fileini& fileini, const string& sector);
+  void iniPoolThreads(const Fileini& fileini, const string& sector);
+  void iniXfsThread(const Fileini& fileini, const string& sector);
+  void thread_task_dnsxfs(int dns_xfs_period_ms);
+  void shutDownClinets() const;
+  bool hasXfsTask(map<string, int32_t>& src_addr_map);
+  bool addNeedDnsXfsAddr(map<string, int32_t>& src_addr_map);
+
+ private:
+  static TubeMQService* _instance;
+  string local_host_;
+  AtomicInteger service_status_;
+  AtomicInteger client_index_base_;
+  mutable mutex mutex_;
+  map<int32_t, BaseClient*> clients_map_;
+  ExecutorPoolPtr timer_executor_;
+  ExecutorPoolPtr network_executor_;
+  ConnectionPoolPtr connection_pool_;
+  std::shared_ptr<ThreadPool> thread_pool_;
+  thread dns_xfs_thread_;
+  mutable mutex dns_mutex_;
+  int64_t last_check_time_;
+  map<string, int32_t> master_source_;
+  map<string, string> master_target_;
+};
+
+}  // namespace tubemq
+
+#endif  // TUBEMQ_CLIENT_BASE_CLIENT_H_
+
diff --git a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
index 803eb15..6406d97 100644
--- a/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
+++ b/tubemq-client-twins/tubemq-client-cpp/src/thread_pool.h
@@ -22,7 +22,6 @@
 
 #include <stdlib.h>
 
-
 #include <chrono>
 #include <functional>
 #include <memory>
@@ -34,6 +33,7 @@
 #include <asio/ssl.hpp>
 #include "noncopyable.h"
 
+
 namespace tubemq {
 // ThreadPool use one io_context for thread pool
 class ThreadPool : noncopyable {
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/tests/CMakeLists.txt
similarity index 87%
copy from tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
copy to tubemq-client-twins/tubemq-client-cpp/tests/CMakeLists.txt
index 9d729c2..c1866ea 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/CMakeLists.txt
@@ -22,12 +22,13 @@
 # unit_tests test is not set up using this function because it does not like
 # the additional argument on commmand line and consequently does not run any
 # test.
-function(tubemq_add_example _name)
+function(tubemq_add_tests _name)
   set(_srcs ${ARGN})
   message (STATUS "${_name} sources: ${_srcs}")
   add_executable (${_name} ${_srcs})
-  TARGET_LINK_LIBRARIES (${_name} tubemq ssl crypto log4cplus pthread)
+  TARGET_LINK_LIBRARIES (${_name} tubemq ssl crypto liblog4cplus.a libprotobuf.a tubemq_proto pthread rt)
 endfunction()
 
 add_subdirectory (log)
 add_subdirectory (executor_pool)
+add_subdirectory (thread_pool)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc b/tubemq-client-twins/tubemq-client-cpp/tests/README.md
similarity index 65%
copy from tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
copy to tubemq-client-twins/tubemq-client-cpp/tests/README.md
index 9bb1685..4998b5f 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/README.md
@@ -1,46 +1,22 @@
-/**
- * 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 <chrono>
-#include <exception>
-#include <iostream>
-#include <string>
-#include <thread>
-
-#include "tubemq/logger.h"
-
-using namespace std;
-using namespace tubemq;
-
-void log() {
-  int i = 0;
-  while (1) {
-    LOG_ERROR("threadid:%ld, i:%d", std::this_thread::get_id(), i++);
-  }
-}
-
-int main() {
-  std::thread t1(log);
-  std::thread t2(log);
-  std::thread t3(log);
-  std::thread t4(log);
-  t1.join();
-  return 0;
-}
-
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#Example#
+
+tubemq-client-cpp example. 
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/CMakeLists.txt
similarity index 94%
rename from tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt
rename to tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/CMakeLists.txt
index bef5bbb..dac0999 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/CMakeLists.txt
@@ -18,4 +18,4 @@
 #
 
 
-tubemq_add_example(executor_pool main.cc)
+tubemq_add_tests(executor_pool main.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc b/tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/main.cc
similarity index 84%
rename from tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc
rename to tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/main.cc
index 8c5ca25..04c6f48 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/executor_pool/main.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/executor_pool/main.cc
@@ -24,7 +24,7 @@
 #include <string>
 #include <thread>
 
-#include "tubemq/executor_pool.h"
+#include "executor_pool.h"
 
 using namespace std;
 using namespace tubemq;
@@ -40,7 +40,7 @@ int main() {
   using namespace std::placeholders;  // for _1, _2, _3...
   ExecutorPool pool(4);
   auto timer = pool.Get()->CreateSteadyTimer();
-  timer->expires_after(std::chrono::seconds(5));
+  timer->expires_after(std::chrono::seconds(1));
   std::cout << "startwait" << endl;
   timer->wait();
   std::cout << "endwait" << endl;
@@ -49,7 +49,10 @@ int main() {
   std::cout << "startsyncwait" << endl;
   timer->async_wait(std::bind(handler, 5, _1));
   std::cout << "endsyncwait" << endl;
-  std::this_thread::sleep_for(5s);
+  std::this_thread::sleep_for(std::chrono::seconds(1));
+  timer->expires_after(std::chrono::milliseconds(100));
+  timer->async_wait(std::bind(handler, 6, _1));
+  std::this_thread::sleep_for(std::chrono::seconds(1));
   return 0;
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/tests/log/CMakeLists.txt
similarity index 96%
copy from tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
copy to tubemq-client-twins/tubemq-client-cpp/tests/log/CMakeLists.txt
index b828bf8..a7a7fe9 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/log/CMakeLists.txt
@@ -18,4 +18,4 @@
 #
 
 
-tubemq_add_example(log main.cc)
+tubemq_add_tests(log main.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc b/tubemq-client-twins/tubemq-client-cpp/tests/log/main.cc
similarity index 60%
copy from tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
copy to tubemq-client-twins/tubemq-client-cpp/tests/log/main.cc
index 9bb1685..d012c32 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/log/main.cc
@@ -23,23 +23,39 @@
 #include <string>
 #include <thread>
 
-#include "tubemq/logger.h"
+#include "buffer.h"
+#include "logger.h"
+#include "tubemq/tubemq_atomic.h"
 
 using namespace std;
 using namespace tubemq;
 
-void log() {
-  int i = 0;
+AtomicInteger ati;
+
+void logfunc() {
   while (1) {
-    LOG_ERROR("threadid:%ld, i:%d", std::this_thread::get_id(), i++);
+    LOG_TRACE("atomic:%d", ati.IncrementAndGet());
   }
 }
 
 int main() {
-  std::thread t1(log);
-  std::thread t2(log);
-  std::thread t3(log);
-  std::thread t4(log);
+  {
+    auto buf = std::make_shared<Buffer>();
+    std::string data = "abcdef";
+    buf->Write(data.data(), data.size());
+    auto buf2 = buf->Slice();
+    buf2->ReadUint32();
+    buf->Write(data.data(), data.size());
+    printf("%s\n", buf->String().c_str());
+    printf("%s\n", buf2->String().c_str());
+  }
+
+  ati.GetAndSet(1);
+  GetLogger().Init("./tubemq", tubemq::Logger::Level(0));
+  std::thread t1(logfunc);
+  std::thread t2(logfunc);
+  std::thread t3(logfunc);
+  std::thread t4(logfunc);
   t1.join();
   return 0;
 }
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/CMakeLists.txt
similarity index 95%
rename from tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
rename to tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/CMakeLists.txt
index b828bf8..4fe33bb 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/CMakeLists.txt
@@ -18,4 +18,4 @@
 #
 
 
-tubemq_add_example(log main.cc)
+tubemq_add_tests(thread_pool main.cc)
diff --git a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc b/tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/main.cc
similarity index 79%
rename from tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
rename to tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/main.cc
index 9bb1685..afeef04 100644
--- a/tubemq-client-twins/tubemq-client-cpp/example/log/main.cc
+++ b/tubemq-client-twins/tubemq-client-cpp/tests/thread_pool/main.cc
@@ -23,24 +23,14 @@
 #include <string>
 #include <thread>
 
-#include "tubemq/logger.h"
+#include "thread_pool.h"
 
 using namespace std;
 using namespace tubemq;
 
-void log() {
-  int i = 0;
-  while (1) {
-    LOG_ERROR("threadid:%ld, i:%d", std::this_thread::get_id(), i++);
-  }
-}
-
 int main() {
-  std::thread t1(log);
-  std::thread t2(log);
-  std::thread t3(log);
-  std::thread t4(log);
-  t1.join();
+  ThreadPool pool(std::thread::hardware_concurrency());
+  pool.Post([] { std::cout << "abcd" << endl; });
   return 0;
 }
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt b/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
index 7dc5e88..c223f9b 100644
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
+++ b/tubemq-client-twins/tubemq-client-cpp/third_party/CMakeLists.txt
@@ -1,25 +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.
-#
-
-
 CMAKE_MINIMUM_REQUIRED(VERSION 3.1)                                             
+
 PROJECT(TubeMQThirdParty)
 
-ADD_SUBDIRECTORY(log4cplus) 
+SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-error")
+
+include(ExternalProject)
+
+ExternalProject_Add(
+    rapidjson
+    PREFIX "rapidjson"
+    GIT_REPOSITORY "https://github.com/Tencent/rapidjson.git"
+    GIT_TAG f54b0e47a08782a6131cc3d60f94d038fa6e0a51
+    TIMEOUT 20
+    CMAKE_ARGS
+    -DRAPIDJSON_BUILD_TESTS=OFF
+    -DRAPIDJSON_BUILD_DOC=OFF
+    -DRAPIDJSON_BUILD_EXAMPLES=OFF
+    CONFIGURE_COMMAND ""
+    BUILD_COMMAND ""
+    INSTALL_COMMAND ""
+    UPDATE_COMMAND ""
+    )
+
+ExternalProject_Add(
+    log4cplus_proj
+    URL https://github.com/log4cplus/log4cplus/releases/download/REL_2_0_5/log4cplus-2.0.5.tar.gz
+    CONFIGURE_COMMAND ./configure --prefix=<INSTALL_DIR> --disable-shared --with-pic CFLAGS=-O2\  CXXFLAGS=-O2\ -fPIC
+    TEST_BEFORE_INSTALL 0
+    BUILD_IN_SOURCE 1
+    INSTALL_DIR ${CMAKE_BINARY_DIR}/third_party/
+    )
+
+ExternalProject_Add(
+    asio_proj
+    URL https://github.com/chriskohlhoff/asio/archive/asio-1-18-0.tar.gz 
+    INSTALL_DIR ${CMAKE_BINARY_DIR}/third_party/
+    CONFIGURE_COMMAND cd ../asio_proj/asio && ./autogen.sh && ./configure --prefix=<INSTALL_DIR> CFLAGS=-std=c++11 CPPFLAGS=-std=c++11 CXXFLAGS=-std=c++11 && make install
+    TEST_BEFORE_INSTALL 0
+    BUILD_IN_SOURCE 0
+    BUILD_COMMAND "" 
+    INSTALL_COMMAND ""
+    )
+
 
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus b/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
deleted file mode 160000
index 9d00f7d..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/log4cplus
+++ /dev/null
@@ -1 +0,0 @@
-Subproject commit 9d00f7d10f2507f68f9ab5fea8b842735d9c6cfe
diff --git a/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson b/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
deleted file mode 160000
index f54b0e4..0000000
--- a/tubemq-client-twins/tubemq-client-cpp/third_party/rapidjson
+++ /dev/null
@@ -1 +0,0 @@
-Subproject commit f54b0e47a08782a6131cc3d60f94d038fa6e0a51


[incubator-tubemq] 11/50: [TUBEMQ-262]Create C++ flow control handler

Posted by gx...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gxcheng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-tubemq.git

commit d52d146214003262f0cc341beee0be3f5fc4c5e2
Author: gosonzhang <go...@tencent.com>
AuthorDate: Fri Jul 3 09:48:19 2020 +0800

    [TUBEMQ-262]Create C++ flow control handler
---
 .gitmodules | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/.gitmodules b/.gitmodules
new file mode 100644
index 0000000..b9a6c24
--- /dev/null
+++ b/.gitmodules
@@ -0,0 +1,3 @@
+[submodule "third_party/jsoncpp"]
+	path = third_party/jsoncpp
+	url = https://github.com/open-source-parsers/jsoncpp.git