You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by aa...@apache.org on 2023/02/23 15:14:40 UTC

[rocketmq-clients] branch master updated (57c5935b -> 520bd4f2)

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

aaronai pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git


    from 57c5935b Bump rocketmq-proto to 2.0.2 (#364)
     new b3f7ba6d WIP: refactor producer
     new 1cc2e59c Format log output
     new 57531fab Modify log level temporarily
     new 8417c0a9 Add more logs
     new 9c5d8858 Polish code
     new 2e2b23a8 Add PublishingLoadBalancer#TakeMessageQueueByMessageGroup
     new 26bb67c5 Implement simple consumer
     new 8f7418ed Bugfix: ICollection is read-only
     new 44943d58 Polish logs
     new 70282e85 Add Client#GetTopics
     new 85ae3d54 Add more logs
     new 7ce29d7d Notify remote endpoints that current client is terminated
     new 6cde562e Add error log for scheduled task
     new 30c1832b Implement Client#OnVerrifyMessageCommand and Client#OnPrintThreadStackTraceCommand
     new 7de5b395 Implement transaction message
     new c9e3d21f Implement Producer#send with transaction
     new 7be8d6da Add more stats info
     new d7a0a5c6 Add more method for IClientConfig
     new d7072119 Polish code
     new 2f734d63 Polish code
     new ebd236ad Make simpleconsumer works
     new b2ee3b31 Polish code
     new e272ee3d Add nonce for TelemetryCommand
     new f5c2878a Add state machine for rocketmq producer/simpleConsumer
     new 15007046 Apply state machine in transactional message
     new f4e4647c Add more response code cases
     new e3fbe1db Implement ExponentialBackoffRetryPolicy
     new 520bd4f2 Add license header

The 28 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:
 csharp/examples/ProducerBenchmark.cs               | 103 +++
 csharp/examples/ProducerDelayMessageExample.cs     |  30 +-
 csharp/examples/ProducerFifoMessageExample.cs      |  40 +-
 csharp/examples/ProducerNormalMessageExample.cs    |  35 +-
 ...ple.cs => ProducerTransactionMessageExample.cs} |  45 +-
 csharp/examples/QuickStart.cs                      |  15 +-
 csharp/examples/SimpleConsumerExample.cs           |  47 +-
 csharp/rocketmq-client-csharp/AccessPoint.cs       |  74 ---
 .../{Topic.cs => Address.cs}                       |  55 +-
 .../AddressListEqualityComparer.cs}                |  40 +-
 ...aticCredentialsProvider.cs => AddressScheme.cs} |  33 +-
 .../{ProcessQueue.cs => Broker.cs}                 |  34 +-
 csharp/rocketmq-client-csharp/Client.cs            | 700 ++++++++++-----------
 csharp/rocketmq-client-csharp/ClientConfig.cs      | 151 +----
 .../ClientLoggerInterceptor.cs                     |  17 +-
 csharp/rocketmq-client-csharp/ClientManager.cs     | 285 ++-------
 ...{StaticCredentialsProvider.cs => ClientType.cs} |  30 +-
 .../ConfigFileCredentialsProvider.cs               |   5 +-
 csharp/rocketmq-client-csharp/Consumer.cs          | 104 +++
 csharp/rocketmq-client-csharp/Endpoints.cs         | 185 ++++++
 .../BadRequestException.cs}                        |  20 +-
 .../{SendReceipt.cs => Error/ClientException.cs}   |  29 +-
 .../ForbiddenException.cs}                         |  20 +-
 .../InternalErrorException.cs}                     |  30 +-
 .../NotFoundException.cs}                          |  25 +-
 .../PayloadTooLargeException.cs}                   |  20 +-
 .../PaymentRequiredException.cs}                   |  14 +-
 .../ProxyTimeoutException.cs}                      |  14 +-
 .../RequestHeaderFieldsTooLargeException.cs}       |  22 +-
 .../TooManyRequestsException.cs}                   |  20 +-
 .../UnauthorizedException.cs}                      |  21 +-
 .../UnsupportedException.cs}                       |  20 +-
 .../ExponentialBackoffRetryPolicy.cs               |  92 +++
 csharp/rocketmq-client-csharp/ExpressionType.cs    |   7 +-
 csharp/rocketmq-client-csharp/FilterExpression.cs  |   6 +
 csharp/rocketmq-client-csharp/IClient.cs           |  47 +-
 csharp/rocketmq-client-csharp/IClientConfig.cs     |  15 +-
 csharp/rocketmq-client-csharp/IClientManager.cs    |  64 +-
 .../rocketmq-client-csharp/ICredentialsProvider.cs |   6 +-
 csharp/rocketmq-client-csharp/IProducer.cs         |  30 -
 .../{IClientConfig.cs => IRetryPolicy.cs}          |  35 +-
 .../{ExpressionType.cs => ITransaction.cs}         |  10 +-
 .../{IClientConfig.cs => ITransactionChecker.cs}   |  23 +-
 csharp/rocketmq-client-csharp/Message.cs           |  85 +--
 .../rocketmq-client-csharp/MessageIdGenerator.cs   |  26 +-
 csharp/rocketmq-client-csharp/MessageQueue.cs      |  72 +++
 csharp/rocketmq-client-csharp/MessageType.cs       |  38 +-
 csharp/rocketmq-client-csharp/MessageView.cs       | 194 ++++++
 csharp/rocketmq-client-csharp/MetadataConstants.cs |  66 +-
 .../MqEncoding.cs}                                 |  27 +-
 csharp/rocketmq-client-csharp/MqLogManager.cs      |  14 +-
 csharp/rocketmq-client-csharp/Permission.cs        |  93 +++
 csharp/rocketmq-client-csharp/Producer.cs          | 382 ++++++-----
 .../rocketmq-client-csharp/PublishLoadBalancer.cs  | 127 ----
 csharp/rocketmq-client-csharp/Publishing.cs        |   4 +-
 .../PublishingLoadBalancer.cs                      |  97 +++
 csharp/rocketmq-client-csharp/PublishingMessage.cs | 116 ++++
 .../rocketmq-client-csharp/PublishingSettings.cs   |  85 +++
 ...IMessageListener.cs => ReceiveMessageResult.cs} |  14 +-
 .../{SendReceipt.cs => Resource.cs}                |  35 +-
 csharp/rocketmq-client-csharp/RpcClient.cs         |  54 +-
 csharp/rocketmq-client-csharp/SendReceipt.cs       |  46 +-
 csharp/rocketmq-client-csharp/SendStatus.cs        |  27 -
 csharp/rocketmq-client-csharp/SequenceGenerator.cs | 131 ----
 csharp/rocketmq-client-csharp/Session.cs           | 167 ++---
 csharp/rocketmq-client-csharp/Settings.cs          |  59 ++
 csharp/rocketmq-client-csharp/Signature.cs         |  62 +-
 csharp/rocketmq-client-csharp/SimpleConsumer.cs    | 384 +++++------
 .../SimpleSubscriptionSettings.cs                  |  99 +++
 .../{MessageType.cs => State.cs}                   |  12 +-
 .../StaticCredentialsProvider.cs                   |  13 +-
 csharp/rocketmq-client-csharp/StatusChecker.cs     | 110 ++++
 .../SubscriptionLoadBalancer.cs                    |  37 +-
 csharp/rocketmq-client-csharp/TopicRouteData.cs    |  45 +-
 .../rocketmq-client-csharp/TopicRouteException.cs  |   1 +
 csharp/rocketmq-client-csharp/Transaction.cs       | 131 ++++
 ...{ExpressionType.cs => TransactionResolution.cs} |  10 +-
 .../UserAgent.cs}                                  |  37 +-
 csharp/rocketmq-client-csharp/Utilities.cs         |  77 ++-
 .../rocketmq-client-csharp.nlog                    |   4 +-
 csharp/tests/ClientManagerTest.cs                  |  58 --
 csharp/tests/ConfigFileCredentialsProviderTest.cs  |   2 +-
 csharp/tests/DateTimeTest.cs                       |   2 +-
 .../{ClientConfigTest.cs => EndpointsTest.cs}      |  17 +-
 csharp/tests/MessageIdGeneratorTest.cs             |   4 +-
 csharp/tests/MessageTest.cs                        |   2 -
 csharp/tests/ProducerTest.cs                       | 189 ------
 csharp/tests/RpcClientTest.cs                      | 146 -----
 csharp/tests/SendResultTest.cs                     |  21 +-
 csharp/tests/SignatureTest.cs                      |  23 +-
 csharp/tests/SimpleConsumerTest.cs                 |  97 ---
 csharp/tests/StaticCredentialsProviderTest.cs      |  40 --
 csharp/tests/UnitTest1.cs                          |  57 +-
 93 files changed, 3346 insertions(+), 2911 deletions(-)
 create mode 100644 csharp/examples/ProducerBenchmark.cs
 copy csharp/examples/{ProducerNormalMessageExample.cs => ProducerTransactionMessageExample.cs} (54%)
 delete mode 100644 csharp/rocketmq-client-csharp/AccessPoint.cs
 copy csharp/rocketmq-client-csharp/{Topic.cs => Address.cs} (59%)
 copy csharp/{tests/SequenceGeneratorTest.cs => rocketmq-client-csharp/AddressListEqualityComparer.cs} (58%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => AddressScheme.cs} (58%)
 copy csharp/rocketmq-client-csharp/{ProcessQueue.cs => Broker.cs} (62%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => ClientType.cs} (59%)
 create mode 100644 csharp/rocketmq-client-csharp/Consumer.cs
 create mode 100644 csharp/rocketmq-client-csharp/Endpoints.cs
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/BadRequestException.cs} (64%)
 copy csharp/rocketmq-client-csharp/{SendReceipt.cs => Error/ClientException.cs} (57%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/ForbiddenException.cs} (64%)
 copy csharp/rocketmq-client-csharp/{SendReceipt.cs => Error/InternalErrorException.cs} (59%)
 rename csharp/rocketmq-client-csharp/{ProcessQueue.cs => Error/NotFoundException.cs} (63%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/PayloadTooLargeException.cs} (63%)
 copy csharp/rocketmq-client-csharp/{MessageException.cs => Error/PaymentRequiredException.cs} (69%)
 rename csharp/rocketmq-client-csharp/{MessageException.cs => Error/ProxyTimeoutException.cs} (70%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/RequestHeaderFieldsTooLargeException.cs} (56%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/TooManyRequestsException.cs} (64%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/UnauthorizedException.cs} (62%)
 copy csharp/rocketmq-client-csharp/{StaticCredentialsProvider.cs => Error/UnsupportedException.cs} (66%)
 create mode 100644 csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
 delete mode 100644 csharp/rocketmq-client-csharp/IProducer.cs
 copy csharp/rocketmq-client-csharp/{IClientConfig.cs => IRetryPolicy.cs} (50%)
 copy csharp/rocketmq-client-csharp/{ExpressionType.cs => ITransaction.cs} (91%)
 copy csharp/rocketmq-client-csharp/{IClientConfig.cs => ITransactionChecker.cs} (67%)
 create mode 100644 csharp/rocketmq-client-csharp/MessageQueue.cs
 create mode 100644 csharp/rocketmq-client-csharp/MessageView.cs
 copy csharp/{tests/DateTimeTest.cs => rocketmq-client-csharp/MqEncoding.cs} (65%)
 create mode 100644 csharp/rocketmq-client-csharp/Permission.cs
 delete mode 100644 csharp/rocketmq-client-csharp/PublishLoadBalancer.cs
 create mode 100644 csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
 create mode 100644 csharp/rocketmq-client-csharp/PublishingMessage.cs
 create mode 100644 csharp/rocketmq-client-csharp/PublishingSettings.cs
 rename csharp/rocketmq-client-csharp/{IMessageListener.cs => ReceiveMessageResult.cs} (74%)
 copy csharp/rocketmq-client-csharp/{SendReceipt.cs => Resource.cs} (57%)
 delete mode 100644 csharp/rocketmq-client-csharp/SendStatus.cs
 delete mode 100644 csharp/rocketmq-client-csharp/SequenceGenerator.cs
 create mode 100644 csharp/rocketmq-client-csharp/Settings.cs
 create mode 100644 csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
 copy csharp/rocketmq-client-csharp/{MessageType.cs => State.cs} (87%)
 create mode 100644 csharp/rocketmq-client-csharp/StatusChecker.cs
 create mode 100644 csharp/rocketmq-client-csharp/Transaction.cs
 copy csharp/rocketmq-client-csharp/{ExpressionType.cs => TransactionResolution.cs} (90%)
 rename csharp/{tests/SequenceGeneratorTest.cs => rocketmq-client-csharp/UserAgent.cs} (59%)
 delete mode 100644 csharp/tests/ClientManagerTest.cs
 rename csharp/tests/{ClientConfigTest.cs => EndpointsTest.cs} (73%)
 delete mode 100644 csharp/tests/ProducerTest.cs
 delete mode 100644 csharp/tests/RpcClientTest.cs
 delete mode 100644 csharp/tests/SimpleConsumerTest.cs
 delete mode 100644 csharp/tests/StaticCredentialsProviderTest.cs


[rocketmq-clients] 09/28: Polish logs

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 44943d58fd2d6715b4532b7844ffa61e6baa28a8
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 19:19:49 2023 +0800

    Polish logs
---
 csharp/examples/ProducerBenchmark.cs      | 74 ++++++++++++++++++++++++++++++-
 csharp/examples/QuickStart.cs             |  3 +-
 csharp/rocketmq-client-csharp/Producer.cs | 15 +++++--
 3 files changed, 87 insertions(+), 5 deletions(-)

diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerBenchmark.cs
index 361aa95d..8ad03847 100644
--- a/csharp/examples/ProducerBenchmark.cs
+++ b/csharp/examples/ProducerBenchmark.cs
@@ -15,6 +15,10 @@
  * limitations under the License.
  */
 
+using System;
+using System.Collections.Generic;
+using System.Text;
+using System.Threading;
 using System.Threading.Tasks;
 using NLog;
 using Org.Apache.Rocketmq;
@@ -25,8 +29,76 @@ namespace examples
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        internal static async Task QuickStart()
+        private static readonly SemaphoreSlim _semaphore = new SemaphoreSlim(0);
+        private static long _counter = 0;
+
+        internal static void QuickStart()
         {
+            const string accessKey = "5jFk0wK7OU6Uq395";
+            const string secretKey = "V1u8z19URHs4o6RQ";
+
+            // Credential provider is optional for client configuration.
+            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            var clientConfig = new ClientConfig(endpoints)
+            {
+                CredentialsProvider = credentialsProvider
+            };
+            // In most case, you don't need to create too many producers, single pattern is recommended.
+            var producer = new Producer(clientConfig);
+
+            const string topic = "lingchu_normal_topic";
+            producer.SetTopics(topic);
+            // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // the topic route before message publishing.
+            producer.Start().Wait();
+            // Define your message body.
+            var bytes = Encoding.UTF8.GetBytes("foobar");
+            const string tag = "yourMessageTagA";
+            // You could set multiple keys for the single message.
+            var keys = new List<string>
+            {
+                "yourMessageKey-7044358f98fc",
+                "yourMessageKey-f72539fbc246"
+            };
+            // Set topic for current message.
+            var message = new Message(topic, bytes)
+            {
+                Tag = tag,
+                Keys = keys
+            };
+
+            const int tpsLimit = 1000;
+
+            Task.Run(async () =>
+            {
+                while (true)
+                {
+                    _semaphore.Release(tpsLimit/1000);
+                    await Task.Delay(TimeSpan.FromMilliseconds(1));
+                }
+            });
+
+            Task.Run(async () =>
+            {
+                while (true)
+                {
+                    Logger.Info($"Send {_counter} messages successfully.");
+                    Interlocked.Exchange(ref _counter, 0);
+                    await Task.Delay(TimeSpan.FromSeconds(1));
+                }
+            });
+
+            var tasks = new List<Task>();
+            while (true)
+            {
+                _semaphore.Wait();
+                Interlocked.Increment(ref _counter);
+                var task = producer.Send(message);
+                tasks.Add(task);
+            }
+
+            Task.WhenAll(tasks).Wait();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/QuickStart.cs b/csharp/examples/QuickStart.cs
index 4d6b423d..8323218f 100644
--- a/csharp/examples/QuickStart.cs
+++ b/csharp/examples/QuickStart.cs
@@ -27,10 +27,11 @@ namespace examples
         {
             // Console.WriteLine(MetadataConstants.Instance.ClientVersion);
 
-            ProducerNormalMessageExample.QuickStart().Wait();
+            // ProducerNormalMessageExample.QuickStart().Wait();
             // await ProducerFifoMessageExample.QuickStart();
             // await ProducerDelayMessageExample.QuickStart();
             // await SimpleConsumerExample.QuickStart();
+            ProducerBenchmark.QuickStart();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 6a9040ec..cc7794f6 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -42,7 +42,8 @@ namespace Org.Apache.Rocketmq
         {
         }
 
-        private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> publishingTopics, int maxAttempts) :
+        private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> publishingTopics,
+            int maxAttempts) :
             base(clientConfig, publishingTopics.Keys)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
@@ -120,7 +121,7 @@ namespace Org.Apache.Rocketmq
                 : new List<MessageQueue>
                     { publishingLoadBalancer.TakeMessageQueueByMessageGroup(publishingMessage.MessageGroup) };
             Exception exception = null;
-            for (var attempt = 0; attempt < maxAttempts; attempt++)
+            for (var attempt = 1; attempt <= maxAttempts; attempt++)
             {
                 try
                 {
@@ -179,8 +180,16 @@ namespace Org.Apache.Rocketmq
             {
                 // Isolate current endpoints.
                 Isolated[endpoints] = true;
+                if (attempt >= maxAttempts)
+                {
+                    Logger.Error(
+                        $"Failed to send message finally, run out of attempt times, topic={message.Topic}, " +
+                        $"maxAttempt={maxAttempts}, attempt={attempt}, endpoints={endpoints}, clientId={ClientId}");
+                    throw;
+                }
+
                 Logger.Warn(e, $"Failed to send message, topic={message.Topic}, maxAttempts={maxAttempts}, " +
-                               $"endpoints={endpoints}, clientId={ClientId}");
+                               $"attempt={attempt}, endpoints={endpoints}, clientId={ClientId}");
                 throw;
             }
         }


[rocketmq-clients] 01/28: WIP: refactor producer

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit b3f7ba6d2e719b89ec0360a3f9f3d200a93a3401
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 8 17:58:03 2023 +0800

    WIP: refactor producer
---
 csharp/examples/ProducerDelayMessageExample.cs     |  75 +--
 csharp/examples/ProducerFifoMessageExample.cs      |  73 +--
 csharp/examples/ProducerNormalMessageExample.cs    |  31 +-
 csharp/examples/QuickStart.cs                      |  14 +-
 csharp/examples/SimpleConsumerExample.cs           |  68 +--
 csharp/rocketmq-client-csharp/Address.cs           |  69 +++
 .../AddressListEqualityComparer.cs                 |  34 ++
 csharp/rocketmq-client-csharp/AddressScheme.cs     |  28 ++
 csharp/rocketmq-client-csharp/Broker.cs            |  18 +
 csharp/rocketmq-client-csharp/Client.cs            | 557 ++++++---------------
 csharp/rocketmq-client-csharp/ClientConfig.cs      | 149 +-----
 csharp/rocketmq-client-csharp/ClientManager.cs     | 277 ++--------
 .../{IClient.cs => ClientType.cs}                  |  33 +-
 .../ConfigFileCredentialsProvider.cs               |   4 +-
 csharp/rocketmq-client-csharp/Endpoints.cs         | 133 +++++
 .../BadRequestException.cs}                        |  20 +-
 .../{SendReceipt.cs => Error/ClientException.cs}   |  29 +-
 .../ForbiddenException.cs}                         |  20 +-
 .../InternalErrorException.cs}                     |  26 +-
 .../NotFoundException.cs}                          |  25 +-
 .../PayloadTooLargeException.cs}                   |  20 +-
 .../Error/PaymentRequiredException.cs}             |  61 ++-
 .../ProxyTimeoutException.cs}                      |  17 +-
 .../RequestHeaderFieldsTooLargeException.cs}       |  29 +-
 .../rocketmq-client-csharp/Error/StatusChecker.cs  |  41 ++
 .../TooManyRequestsException.cs}                   |  20 +-
 .../UnauthorizedException.cs}                      |  21 +-
 .../Error/UnsupportedException.cs}                 |  22 +-
 .../ExponentialBackoffRetryPolicy.cs               |  56 +++
 csharp/rocketmq-client-csharp/ExpressionType.cs    |   7 +-
 csharp/rocketmq-client-csharp/IClient.cs           |  27 +-
 csharp/rocketmq-client-csharp/IClientConfig.cs     |  20 +-
 csharp/rocketmq-client-csharp/IClientManager.cs    |  28 +-
 .../rocketmq-client-csharp/ICredentialsProvider.cs |   5 +-
 csharp/rocketmq-client-csharp/Message.cs           |  12 +-
 csharp/rocketmq-client-csharp/MessageQueue.cs      |  41 ++
 csharp/rocketmq-client-csharp/MessageType.cs       |  42 +-
 csharp/rocketmq-client-csharp/MessageView.cs       | 187 +++++++
 csharp/rocketmq-client-csharp/MetadataConstants.cs |  66 ++-
 csharp/rocketmq-client-csharp/MqEncoding.cs        |  26 +
 csharp/rocketmq-client-csharp/Permission.cs        |  81 +++
 csharp/rocketmq-client-csharp/Producer.cs          | 273 ++++------
 ...shLoadBalancer.cs => PublishingLoadBalancer.cs} |  55 +-
 csharp/rocketmq-client-csharp/PublishingMessage.cs | 119 +++++
 .../rocketmq-client-csharp/PublishingSettings.cs   |  63 +++
 csharp/rocketmq-client-csharp/Resource.cs          |  25 +
 csharp/rocketmq-client-csharp/RetryPolicy.cs       |  13 +
 csharp/rocketmq-client-csharp/RpcClient.cs         |  34 +-
 csharp/rocketmq-client-csharp/SendReceipt.cs       |  40 +-
 csharp/rocketmq-client-csharp/SendStatus.cs        |  27 -
 csharp/rocketmq-client-csharp/Session.cs           | 171 ++++---
 csharp/rocketmq-client-csharp/Settings.cs          |  42 ++
 csharp/rocketmq-client-csharp/Signature.cs         |  39 +-
 csharp/rocketmq-client-csharp/SimpleConsumer.cs    | 319 ------------
 .../StaticCredentialsProvider.cs                   |  13 +-
 csharp/rocketmq-client-csharp/StatusChecker.cs     | 103 ++++
 csharp/rocketmq-client-csharp/TopicRouteData.cs    |  23 +-
 csharp/rocketmq-client-csharp/UserAgent.cs         |  31 ++
 csharp/rocketmq-client-csharp/Utilities.cs         |  46 +-
 .../rocketmq-client-csharp.nlog                    |   4 +-
 csharp/tests/ClientManagerTest.cs                  |  58 ---
 csharp/tests/ConfigFileCredentialsProviderTest.cs  |   2 +-
 csharp/tests/DateTimeTest.cs                       |   2 +-
 csharp/tests/MessageTest.cs                        |   2 -
 csharp/tests/ProducerTest.cs                       | 189 -------
 csharp/tests/RpcClientTest.cs                      | 146 ------
 csharp/tests/SendResultTest.cs                     |  15 -
 csharp/tests/SignatureTest.cs                      |   7 +-
 csharp/tests/SimpleConsumerTest.cs                 |  97 ----
 csharp/tests/StaticCredentialsProviderTest.cs      |  40 --
 csharp/tests/UnitTest1.cs                          |   6 -
 71 files changed, 2122 insertions(+), 2394 deletions(-)

diff --git a/csharp/examples/ProducerDelayMessageExample.cs b/csharp/examples/ProducerDelayMessageExample.cs
index 58c8bcd0..27e32e76 100644
--- a/csharp/examples/ProducerDelayMessageExample.cs
+++ b/csharp/examples/ProducerDelayMessageExample.cs
@@ -19,49 +19,54 @@ using System;
 using System.Collections.Generic;
 using System.Text;
 using System.Threading.Tasks;
+using NLog;
 using Org.Apache.Rocketmq;
 
 namespace examples
 {
     static class ProducerDelayMessageExample
     {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
         internal static async Task QuickStart()
         {
-            string accessKey = "yourAccessKey";
-            string secretKey = "yourSecretKey";
-            // Credential provider is optional for client configuration.
-            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            string endpoints = "foobar.com:8080";
-
-            var producer = new Producer(endpoints)
-            {
-                CredentialsProvider = credentialsProvider
-            };
-            string topic = "yourDelayTopic";
-            // Set the topic name(s), which is optional. It makes producer could prefetch the topic route before 
-            // message publishing.
-            producer.AddTopicOfInterest(topic);
-
-            await producer.Start();
-            // Define your message body.
-            var bytes = Encoding.UTF8.GetBytes("foobar");
-            string tag = "yourMessageTagA";
-            // You could set multiple keys for the single message.
-            var keys = new List<string>
-            {
-                "yourMessageKey-2f00df144e48",
-                "yourMessageKey-49df1dd332b7"
-            };
-            // Set topic for current message.
-            var message = new Message(topic, bytes)
-            {
-                Tag = tag,
-                Keys = keys,
-                // Essential for DELAY message.
-                DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(30)
-            };
-            await producer.Send(message);
-            await producer.Shutdown();
+            // string accessKey = "yourAccessKey";
+            // string secretKey = "yourSecretKey";
+            // // Credential provider is optional for client configuration.
+            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            // string endpoints = "foobar.com:8080";
+            // // In most case, you don't need to create too many producers, single pattern is recommended.
+            // var producer = new Producer(endpoints)
+            // {
+            //     CredentialsProvider = credentialsProvider
+            // };
+            // string topic = "yourDelayTopic";
+            // // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // // the topic route before message publishing.
+            // producer.AddTopicOfInterest(topic);
+            //
+            // await producer.Start();
+            // // Define your message body.
+            // var bytes = Encoding.UTF8.GetBytes("foobar");
+            // string tag = "yourMessageTagA";
+            // // You could set multiple keys for the single message.
+            // var keys = new List<string>
+            // {
+            //     "yourMessageKey-2f00df144e48",
+            //     "yourMessageKey-49df1dd332b7"
+            // };
+            // // Set topic for current message.
+            // var message = new Message(topic, bytes)
+            // {
+            //     Tag = tag,
+            //     Keys = keys,
+            //     // Essential for DELAY message.
+            //     DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(30)
+            // };
+            // var sendReceipt = await producer.Send(message);
+            // Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
+            // // Close the producer if you don't need it anymore.
+            // await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/ProducerFifoMessageExample.cs b/csharp/examples/ProducerFifoMessageExample.cs
index 4938cb1c..d9a72a59 100644
--- a/csharp/examples/ProducerFifoMessageExample.cs
+++ b/csharp/examples/ProducerFifoMessageExample.cs
@@ -29,42 +29,43 @@ namespace examples
 
         internal static async Task QuickStart()
         {
-            string accessKey = "yourAccessKey";
-            string secretKey = "yourSecretKey";
-            // Credential provider is optional for client configuration.
-            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            string endpoints = "foobar.com:8080";
-
-            var producer = new Producer(endpoints)
-            {
-                CredentialsProvider = credentialsProvider
-            };
-            string topic = "yourFifoTopic";
-            // Set the topic name(s), which is optional. It makes producer could prefetch the topic route before 
-            // message publishing.
-            producer.AddTopicOfInterest(topic);
-
-            await producer.Start();
-            // Define your message body.
-            byte[] bytes = Encoding.UTF8.GetBytes("foobar");
-            string tag = "yourMessageTagA";
-            // You could set multiple keys for the single message.
-            var keys = new List<string>
-            {
-                "yourMessageKey-6cc8b65ed1c8",
-                "yourMessageKey-43783375d9a5"
-            };
-            // Set topic for current message.
-            var message = new Message(topic, bytes)
-            {
-                Tag = tag,
-                Keys = keys,
-                // Essential for FIFO message, messages that belongs to the same message group follow the FIFO semantics.
-                MessageGroup = "yourMessageGroup0"
-            };
-            var sendReceipt = await producer.Send(message);
-            Logger.Info($"Send message successfully, sendReceipt={sendReceipt}.");
-            await producer.Shutdown();
+            // string accessKey = "yourAccessKey";
+            // string secretKey = "yourSecretKey";
+            // // Credential provider is optional for client configuration.
+            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            // string endpoints = "foobar.com:8080";
+            // // In most case, you don't need to create too many producers, single pattern is recommended.
+            // var producer = new Producer(endpoints)
+            // {
+            //     CredentialsProvider = credentialsProvider
+            // };
+            // string topic = "yourFifoTopic";
+            // // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // // the topic route before message publishing.
+            // producer.AddTopicOfInterest(topic);
+            //
+            // await producer.Start();
+            // // Define your message body.
+            // byte[] bytes = Encoding.UTF8.GetBytes("foobar");
+            // string tag = "yourMessageTagA";
+            // // You could set multiple keys for the single message.
+            // var keys = new List<string>
+            // {
+            //     "yourMessageKey-6cc8b65ed1c8",
+            //     "yourMessageKey-43783375d9a5"
+            // };
+            // // Set topic for current message.
+            // var message = new Message(topic, bytes)
+            // {
+            //     Tag = tag,
+            //     Keys = keys,
+            //     // Essential for FIFO message, messages that belongs to the same message group follow the FIFO semantics.
+            //     MessageGroup = "yourMessageGroup0"
+            // };
+            // var sendReceipt = await producer.Send(message);
+            // Logger.Info($"Send FIFO message successfully, sendReceipt={sendReceipt}.");
+            // // Close the producer if you don't need it anymore.
+            // await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index 7acb4571..7aade855 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -18,28 +18,31 @@
 using System.Collections.Generic;
 using System.Text;
 using System.Threading.Tasks;
+using NLog;
 using Org.Apache.Rocketmq;
 
 namespace examples
 {
     static class ProducerNormalMessageExample
     {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
         internal static async Task QuickStart()
         {
-            string accessKey = "yourAccessKey";
-            string secretKey = "yourSecretKey";
+            string accessKey = "5jFk0wK7OU6Uq395";
+            string secretKey = "V1u8z19URHs4o6RQ";
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            string endpoints = "foobar.com:8080";
-
-            var producer = new Producer(endpoints)
-            {
-                CredentialsProvider = credentialsProvider
-            };
-            string topic = "yourNormalTopic";
-            // Set the topic name(s), which is optional. It makes producer could prefetch the topic route before 
-            // message publishing.
-            producer.AddTopicOfInterest(topic);
+            string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            var clientConfig = new ClientConfig(endpoints);
+            clientConfig.CredentialsProvider = credentialsProvider;
+            // In most case, you don't need to create too many producers, single pattern is recommended.
+            var producer = new Producer(clientConfig);
+            
+            string topic = "lingchu_normal_topic";
+            producer.SetTopics(topic);
+            // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // the topic route before message publishing.
 
             await producer.Start();
             // Define your message body.
@@ -57,7 +60,9 @@ namespace examples
                 Tag = tag,
                 Keys = keys
             };
-            await producer.Send(message);
+            var sendReceipt = await producer.Send(message);
+            Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
+            // Close the producer if you don't need it anymore.
             await producer.Shutdown();
         }
     }
diff --git a/csharp/examples/QuickStart.cs b/csharp/examples/QuickStart.cs
index ff28d3e3..4d6b423d 100644
--- a/csharp/examples/QuickStart.cs
+++ b/csharp/examples/QuickStart.cs
@@ -15,18 +15,22 @@
  * limitations under the License.
  */
 
+using System;
 using System.Threading.Tasks;
+using Org.Apache.Rocketmq;
 
 namespace examples
 {
     static class QuickStart
     {
-        public static async Task Main()
+        public static void Main()
         {
-            await ProducerNormalMessageExample.QuickStart();
-            await ProducerFifoMessageExample.QuickStart();
-            await ProducerDelayMessageExample.QuickStart();
-            await SimpleConsumerExample.QuickStart();
+            // Console.WriteLine(MetadataConstants.Instance.ClientVersion);
+
+            ProducerNormalMessageExample.QuickStart().Wait();
+            // await ProducerFifoMessageExample.QuickStart();
+            // await ProducerDelayMessageExample.QuickStart();
+            // await SimpleConsumerExample.QuickStart();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/SimpleConsumerExample.cs b/csharp/examples/SimpleConsumerExample.cs
index e5fdf418..80299fbb 100644
--- a/csharp/examples/SimpleConsumerExample.cs
+++ b/csharp/examples/SimpleConsumerExample.cs
@@ -29,39 +29,41 @@ namespace examples
 
         internal static async Task QuickStart()
         {
-            string accessKey = "yourAccessKey";
-            string secretKey = "yourSecretKey";
-            // Credential provider is optional for client configuration.
-            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            string endpoints = "foobar.com:8080";
-
-            string consumerGroup = "yourConsumerGroup";
-            SimpleConsumer simpleConsumer = new SimpleConsumer(endpoints, consumerGroup)
-            {
-                CredentialsProvider = credentialsProvider
-            };
-
-            string topic = "yourTopic";
-            string tag = "tagA";
-            // Set topic subscription for consumer.
-            simpleConsumer.Subscribe(topic, new FilterExpression(tag, ExpressionType.TAG));
-            await simpleConsumer.Start();
-
-            int maxMessageNum = 16;
-            TimeSpan invisibleDuration = TimeSpan.FromSeconds(15);
-            var messages = await simpleConsumer.Receive(maxMessageNum, invisibleDuration);
-            Logger.Info($"{messages.Count} messages has been received.");
-
-            var tasks = new List<Task>();
-            foreach (var message in messages)
-            {
-                Logger.Info($"Received a message, topic={message.Topic}, message-id={message.MessageId}.");
-                var task = simpleConsumer.Ack(message);
-                tasks.Add(task);
-            }
-
-            await Task.WhenAll(tasks);
-            Logger.Info($"{tasks.Count} messages have been acknowledged.");
+            // string accessKey = "yourAccessKey";
+            // string secretKey = "yourSecretKey";
+            // // Credential provider is optional for client configuration.
+            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            // string endpoints = "foobar.com:8080";
+            //
+            // string consumerGroup = "yourConsumerGroup";
+            // SimpleConsumer simpleConsumer = new SimpleConsumer(endpoints, consumerGroup)
+            // {
+            //     CredentialsProvider = credentialsProvider
+            // };
+            //
+            // string topic = "yourTopic";
+            // string tag = "tagA";
+            // // Set topic subscription for consumer.
+            // simpleConsumer.Subscribe(topic, new FilterExpression(tag, ExpressionType.Tag));
+            // await simpleConsumer.Start();
+            //
+            // int maxMessageNum = 16;
+            // TimeSpan invisibleDuration = TimeSpan.FromSeconds(15);
+            // var messages = await simpleConsumer.Receive(maxMessageNum, invisibleDuration);
+            // Logger.Info($"{messages.Count} messages has been received.");
+            //
+            // var tasks = new List<Task>();
+            // foreach (var message in messages)
+            // {
+            //     Logger.Info($"Received a message, topic={message.Topic}, message-id={message.MessageId}.");
+            //     var task = simpleConsumer.Ack(message);
+            //     tasks.Add(task);
+            // }
+            //
+            // await Task.WhenAll(tasks);
+            // Logger.Info($"{tasks.Count} messages have been acknowledged.");
+            // // Close the consumer if you don't need it anymore.
+            // await simpleConsumer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Address.cs b/csharp/rocketmq-client-csharp/Address.cs
new file mode 100644
index 00000000..316323c9
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Address.cs
@@ -0,0 +1,69 @@
+using System;
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class Address : IEquatable<Address>
+    {
+        public Address(rmq.Address address)
+        {
+            Host = address.Host;
+            Port = address.Port;
+        }
+
+        public Address(string host, int port)
+        {
+            Host = host;
+            Port = port;
+        }
+
+        public string Host { get; }
+
+        public int Port { get; }
+
+        public rmq.Address ToProtobuf()
+        {
+            return new rmq.Address
+            {
+                Host = Host,
+                Port = Port
+            };
+        }
+
+        public bool Equals(Address other)
+        {
+            if (ReferenceEquals(null, other))
+            {
+                return false;
+            }
+
+            if (ReferenceEquals(this, other))
+            {
+                return true;
+            }
+
+            return Host == other.Host && Port == other.Port;
+        }
+
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj))
+            {
+                return false;
+            }
+
+            if (ReferenceEquals(this, obj))
+            {
+                return true;
+            }
+
+            if (obj.GetType() != this.GetType()) return false;
+            return Equals((Address)obj);
+        }
+
+        public override int GetHashCode()
+        {
+            return HashCode.Combine(Host, Port);
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs b/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs
new file mode 100644
index 00000000..5b793f37
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs
@@ -0,0 +1,34 @@
+using System.Collections.Generic;
+using System.Linq;
+
+namespace Org.Apache.Rocketmq
+{
+    internal class AddressListEqualityComparer : IEqualityComparer<List<Address>>
+    {
+        public bool Equals(List<Address> a1, List<Address> a2)
+        {
+            if (ReferenceEquals(a1, a2))
+            {
+                return true;
+            }
+
+            if (ReferenceEquals(a1, null) || ReferenceEquals(a2, null))
+            {
+                return false;
+            }
+
+            return a1.Count == a2.Count && a1.SequenceEqual(a2);
+        }
+
+        public int GetHashCode(List<Address> s1)
+        {
+            var hash = 17;
+            unchecked
+            {
+                hash = s1.Aggregate(hash, (current, s) => (current * 31) + s.GetHashCode());
+            }
+
+            return hash;
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/AddressScheme.cs b/csharp/rocketmq-client-csharp/AddressScheme.cs
new file mode 100644
index 00000000..f9c1c290
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/AddressScheme.cs
@@ -0,0 +1,28 @@
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public enum AddressScheme
+    {
+        DomainName,
+        Ipv4,
+        Ipv6,
+    }
+
+    public static class AddressSchemeHelper
+    {
+        public static rmq.AddressScheme ToProtobuf(AddressScheme scheme)
+        {
+            switch (scheme)
+            {
+                case AddressScheme.Ipv4:
+                    return rmq.AddressScheme.Ipv4;
+                case AddressScheme.Ipv6:
+                    return rmq.AddressScheme.Ipv6;
+                case AddressScheme.DomainName:
+                default:
+                    return rmq.AddressScheme.DomainName;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Broker.cs b/csharp/rocketmq-client-csharp/Broker.cs
new file mode 100644
index 00000000..370ac96a
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Broker.cs
@@ -0,0 +1,18 @@
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class Broker
+    {
+        public Broker(rmq.Broker broker)
+        {
+            Name = broker.Name;
+            Id = broker.Id;
+            Endpoints = new Endpoints(broker.Endpoints);
+        }
+
+        public string Name { get; }
+        public int Id { get; }
+        public Endpoints Endpoints { get; }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index dc1418e6..a63dc038 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -19,528 +19,293 @@ using System.Collections.Generic;
 using System.Collections.Concurrent;
 using System.Threading.Tasks;
 using System.Threading;
-using System.Diagnostics;
 using System;
-using rmq = Apache.Rocketmq.V2;
+using System.Linq;
+using Proto = Apache.Rocketmq.V2;
 using grpc = Grpc.Core;
 using NLog;
-using System.Diagnostics.Metrics;
 
 namespace Org.Apache.Rocketmq
 {
-    public abstract class Client : ClientConfig, IClient
+    public abstract class Client : IClient
     {
-        protected static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        protected Client(string accessUrl)
-        {
-            AccessPoint = new AccessPoint(accessUrl);
+        private static readonly TimeSpan HeartbeatScheduleDelay = TimeSpan.FromSeconds(10);
+        private readonly CancellationTokenSource _heartbeatCts;
 
-            AccessPointScheme = AccessPoint.HostScheme();
-            var serviceEndpoint = new rmq::Address
-            {
-                Host = AccessPoint.Host,
-                Port = AccessPoint.Port
-            };
-            AccessPointEndpoints = new List<rmq::Address> { serviceEndpoint };
+        private static readonly TimeSpan TopicRouteUpdateScheduleDelay = TimeSpan.FromSeconds(30);
+        private readonly CancellationTokenSource _topicRouteUpdateCtx;
 
-            _resourceNamespace = "";
+        private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromMinutes(5);
+        private readonly CancellationTokenSource _settingsSyncCtx;
 
-            ClientSettings = new rmq::Settings
-            {
-                AccessPoint = new rmq::Endpoints
-                {
-                    Scheme = AccessPoint.HostScheme()
-                }
-            };
+        protected readonly ClientConfig ClientConfig;
+        protected readonly IClientManager Manager;
+        protected readonly string ClientId;
 
-            ClientSettings.AccessPoint.Addresses.Add(serviceEndpoint);
+        protected readonly ConcurrentDictionary<string, bool> Topics;
 
-            ClientSettings.RequestTimeout = Google.Protobuf.WellKnownTypes.Duration.FromTimeSpan(TimeSpan.FromSeconds(3));
+        private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteCache;
+        private readonly CancellationTokenSource _telemetryCts;
 
-            ClientSettings.UserAgent = new rmq.UA
-            {
-                Language = rmq::Language.DotNet,
-                Version = MetadataConstants.CLIENT_VERSION,
-                Platform = Environment.OSVersion.ToString(),
-                Hostname = System.Net.Dns.GetHostName()
-            };
+        private readonly Dictionary<Endpoints, Session> _sessionsTable;
+        private readonly ReaderWriterLockSlim _sessionLock;
 
-            Manager = new ClientManager();
+        protected Client(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics)
+        {
+            ClientConfig = clientConfig;
+            Topics = topics;
+            ClientId = Utilities.GetClientId();
 
-            _topicRouteTable = new ConcurrentDictionary<string, TopicRouteData>();
-            _updateTopicRouteCts = new CancellationTokenSource();
-            _HeartbeatCts = new CancellationTokenSource();
+            Manager = new ClientManager(this);
+
+            _topicRouteCache = new ConcurrentDictionary<string, TopicRouteData>();
+
+            _topicRouteUpdateCtx = new CancellationTokenSource();
+            _settingsSyncCtx = new CancellationTokenSource();
+            _heartbeatCts = new CancellationTokenSource();
             _telemetryCts = new CancellationTokenSource();
+
+            _sessionsTable = new Dictionary<Endpoints, Session>();
+            _sessionLock = new ReaderWriterLockSlim();
         }
 
         public virtual async Task Start()
         {
-            Schedule(async () =>
+            Logger.Debug($"Begin to start the rocketmq client, clientId={ClientId}");
+            ScheduleWithFixedDelay(UpdateTopicRouteCache, TopicRouteUpdateScheduleDelay, _topicRouteUpdateCtx.Token);
+            ScheduleWithFixedDelay(Heartbeat, HeartbeatScheduleDelay, _heartbeatCts.Token);
+            ScheduleWithFixedDelay(SyncSettings, SettingsSyncScheduleDelay, _settingsSyncCtx.Token);
+            foreach (var topic in Topics.Keys)
             {
-                Logger.Debug("Update topic route by schedule");
-                await UpdateTopicRoute();
-
-            }, 30, _updateTopicRouteCts.Token);
-
-            // Get routes for topics of interest.
-            Logger.Debug("Step of #Start: get route for topics of interest");
-            await UpdateTopicRoute();
-
-            string accessPointUrl = AccessPoint.TargetUrl();
-            CreateSession(accessPointUrl);
-            await _sessions[accessPointUrl].AwaitSettingNegotiationCompletion();
-            Logger.Debug($"Session has been created for {accessPointUrl}");
-
-            Schedule(async () =>
-            {
-                Logger.Debug("Sending heartbeat by schedule");
-                await Heartbeat();
+                await FetchTopicRoute(topic);
+            }
 
-            }, 10, _HeartbeatCts.Token);
-            await Heartbeat();
+            Logger.Debug($"Start the rocketmq client successfully, clientId={ClientId}");
         }
 
         public virtual async Task Shutdown()
         {
-            Logger.Info($"Shutdown client");
-            _updateTopicRouteCts.Cancel();
-            _HeartbeatCts.Cancel();
+            Logger.Debug($"Begin to shutdown rocketmq client, clientId={ClientId}");
+            _topicRouteUpdateCtx.Cancel();
+            _heartbeatCts.Cancel();
             _telemetryCts.Cancel();
             await Manager.Shutdown();
+            Logger.Debug($"Shutdown the rocketmq client successfully, clientId={ClientId}");
         }
 
-        private string FilterBroker(Func<string, bool> acceptor)
+        private (bool, Session) GetSession(Endpoints endpoints)
         {
-            foreach (var item in _topicRouteTable)
+            _sessionLock.EnterReadLock();
+            try
             {
-                foreach (var partition in item.Value.MessageQueues)
+                // Session exists, return in advance.
+                if (_sessionsTable.TryGetValue(endpoints, out var session))
                 {
-                    var target = Utilities.TargetUrl(partition);
-                    if (acceptor(target))
-                    {
-                        return target;
-                    }
+                    return (false, session);
                 }
             }
-            return null;
-        }
+            finally
+            {
+                _sessionLock.ExitReadLock();
+            }
 
-        /**
-         * Return all endpoints of brokers in route table.
-         */
-        private List<string> AvailableBrokerEndpoints()
-        {
-            var endpoints = new List<string>();
-            foreach (var item in _topicRouteTable)
+            _sessionLock.EnterWriteLock();
+            try
             {
-                foreach (var partition in item.Value.MessageQueues)
+                // Session exists, return in advance.
+                if (_sessionsTable.TryGetValue(endpoints, out var session))
                 {
-                    string endpoint = Utilities.TargetUrl(partition);
-                    if (!endpoints.Contains(endpoint))
-                    {
-                        endpoints.Add(endpoint);
-                    }
+                    return (false, session);
                 }
-            }
-            return endpoints;
-        }
-
-        private async Task UpdateTopicRoute()
-        {
-            HashSet<string> topics = new HashSet<string>(_topicsOfInterest.Keys);
 
-            foreach (var item in _topicRouteTable)
-            {
-                topics.Add(item.Key);
+                var stream = Manager.Telemetry(endpoints);
+                var created = new Session(endpoints, stream, this);
+                _sessionsTable.Add(endpoints, created);
+                return (true, created);
             }
-            Logger.Debug($"Fetch topic route for {topics.Count} topics");
-
-            // Wrap topics into list such that we can map async result to topic 
-            List<string> topicList = new List<string>();
-            topicList.AddRange(topics);
-
-            var tasks = new List<Task<TopicRouteData>>();
-            foreach (var item in topicList)
+            finally
             {
-                tasks.Add(GetRouteFor(item, true));
+                _sessionLock.ExitWriteLock();
             }
+        }
 
-            // Update topic route data
-            TopicRouteData[] result = await Task.WhenAll(tasks);
-            var i = 0;
-            foreach (var item in result)
-            {
-                if (null == item)
-                {
-                    Logger.Warn($"Failed to fetch route for {topicList[i]}, null response");
-                    ++i;
-                    continue;
-                }
+        protected abstract Proto::HeartbeatRequest WrapHeartbeatRequest();
 
-                if (0 == item.MessageQueues.Count)
-                {
-                    Logger.Warn($"Failed to fetch route for {topicList[i]}, empty message queue");
-                    ++i;
-                    continue;
-                }
 
-                var topicName = item.MessageQueues[0].Topic.Name;
+        protected abstract void OnTopicDataFetched0(string topic, TopicRouteData topicRouteData);
 
-                // Make assertion
-                Debug.Assert(topicName.Equals(topicList[i]));
 
-                var existing = _topicRouteTable[topicName];
-                if (!existing.Equals(item))
-                {
-                    _topicRouteTable[topicName] = item;
-                }
-                ++i;
-            }
-        }
-
-        protected void Schedule(Action action, int seconds, CancellationToken token)
+        private async Task OnTopicRouteDataFetched(string topic, TopicRouteData topicRouteData)
         {
-            if (null == action)
+            var routeEndpoints = new HashSet<Endpoints>();
+            foreach (var mq in topicRouteData.MessageQueues)
             {
-                // TODO: log warning
-                return;
+                routeEndpoints.Add(mq.Broker.Endpoints);
             }
 
-            Task.Run(async () =>
+            var existedRouteEndpoints = GetTotalRouteEndpoints();
+            var newEndpoints = routeEndpoints.Except(existedRouteEndpoints);
+
+            foreach (var endpoints in newEndpoints)
             {
-                while (!token.IsCancellationRequested)
+                var (created, session) = GetSession(endpoints);
+                if (created)
                 {
-                    action();
-                    await Task.Delay(TimeSpan.FromSeconds(seconds), token);
+                    await session.SyncSettings(true);
                 }
-            });
+            }
+
+            _topicRouteCache[topic] = topicRouteData;
+            OnTopicDataFetched0(topic, topicRouteData);
         }
 
+
         /**
-         * Parameters:
-         * topic
-         *    Topic to query
-         * direct
-         *    Indicate if we should by-pass cache and fetch route entries from name server.
+         * Return all endpoints of brokers in route table.
          */
-        protected async Task<TopicRouteData> GetRouteFor(string topic, bool direct)
+        private HashSet<Endpoints> GetTotalRouteEndpoints()
         {
-            Logger.Debug($"Get route for topic={topic}, direct={direct}");
-            if (!direct && _topicRouteTable.TryGetValue(topic, out var routeData))
-            {
-                Logger.Debug($"Return cached route for {topic}");
-                return routeData;
-            }
-
-            // We got one or more name servers available.
-            var request = new rmq::QueryRouteRequest
-            {
-                Topic = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = topic
-                },
-                Endpoints = new rmq::Endpoints
-                {
-                    Scheme = AccessPointScheme,
-                    Addresses = { AccessPointEndpoints },
-                }
-            };
-
-            var metadata = new grpc.Metadata();
-            Signature.Sign(this, metadata);
-            int index = _random.Next(0, AccessPointEndpoints.Count);
-            var serviceEndpoint = AccessPointEndpoints[index];
-            // AccessPointAddresses.Count
-            string target = $"https://{serviceEndpoint.Host}:{serviceEndpoint.Port}";
-            try
+            var endpoints = new HashSet<Endpoints>();
+            foreach (var item in _topicRouteCache)
             {
-                Logger.Debug($"Resolving route for topic={topic}");
-                var topicRouteData = await Manager.ResolveRoute(target, metadata, request, RequestTimeout);
-                if (null != topicRouteData)
+                foreach (var endpoint in item.Value.MessageQueues.Select(mq => mq.Broker.Endpoints))
                 {
-                    Logger.Debug($"Got route entries for {topic} from name server");
-                    _topicRouteTable.TryAdd(topic, topicRouteData);
-                    Logger.Debug($"Got route for {topic} from {target}");
-                    return topicRouteData;
+                    endpoints.Add(endpoint);
                 }
-                Logger.Warn($"Failed to query route of {topic} from {target}");
-            }
-            catch (Exception e)
-            {
-                Logger.Warn(e, "Failed when querying route");
             }
 
-            return null;
+            return endpoints;
         }
 
-        protected abstract void PrepareHeartbeatData(rmq::HeartbeatRequest request);
-
-        public async Task Heartbeat()
+        private async void UpdateTopicRouteCache()
         {
-            List<string> endpoints = AvailableBrokerEndpoints();
-            if (0 == endpoints.Count)
-            {
-                Logger.Debug("No broker endpoints available in topic route");
-                return;
-            }
-
-            var request = new rmq::HeartbeatRequest
+            foreach (var topic in Topics.Keys)
             {
-                Group = null,
-                ClientType = rmq.ClientType.Unspecified
-            };
-            PrepareHeartbeatData(request);
-
-            var metadata = new grpc::Metadata();
-            Signature.Sign(this, metadata);
-
-            List<Task> tasks = new List<Task>();
-            foreach (var endpoint in endpoints)
-            {
-                tasks.Add(Manager.Heartbeat(endpoint, metadata, request, RequestTimeout));
+                var topicRouteData = await FetchTopicRoute(topic);
+                _topicRouteCache[topic] = topicRouteData;
             }
-
-            await Task.WhenAll(tasks);
         }
 
-        private List<string> BlockedBrokerEndpoints()
+        private async void SyncSettings()
         {
-            List<string> endpoints = new List<string>();
-            return endpoints;
-        }
-
-        private void RemoveFromBlockList(string endpoint)
-        {
-
+            var totalRouteEndpoints = GetTotalRouteEndpoints();
+            foreach (var (_, session) in totalRouteEndpoints.Select(GetSession))
+            {
+                await session.SyncSettings(false);
+            }
         }
 
-        protected async Task<List<rmq::Assignment>> ScanLoadAssignment(string topic, string group)
+        private static void ScheduleWithFixedDelay(Action action, TimeSpan period, CancellationToken token)
         {
-            // Pick a broker randomly
-            string target = FilterBroker((s) => true);
-            var request = new rmq::QueryAssignmentRequest
+            Task.Run(async () =>
             {
-                Topic = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = topic
-                },
-                Group = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = group
-                },
-                Endpoints = new rmq::Endpoints
+                while (!token.IsCancellationRequested)
                 {
-                    Scheme = AccessPointScheme,
-                    Addresses = { AccessPointEndpoints },
+                    action();
+                    await Task.Delay(period, token);
                 }
-            };
-            
-            try
-            {
-                var metadata = new grpc::Metadata();
-                Signature.Sign(this, metadata);
-                return await Manager.QueryLoadAssignment(target, metadata, request, RequestTimeout);
-            }
-            catch (System.Exception e)
-            {
-                Logger.Warn(e, $"Failed to acquire load assignments from {target}");
-            }
-            // Just return an empty list.
-            return new List<rmq.Assignment>();
-        }
-
-        private string TargetUrl(rmq::Assignment assignment)
-        {
-            var broker = assignment.MessageQueue.Broker;
-            var addresses = broker.Endpoints.Addresses;
-            // TODO: use the first address for now. 
-            var address = addresses[0];
-            return $"https://{address.Host}:{address.Port}";
+            });
         }
 
-        public virtual void BuildClientSetting(rmq::Settings settings)
+        protected async Task<TopicRouteData> FetchTopicRoute(string topic)
         {
-            settings.MergeFrom(ClientSettings);
+            var topicRouteData = await FetchTopicRoute0(topic);
+            await OnTopicRouteDataFetched(topic, topicRouteData);
+            return topicRouteData;
         }
 
-        private async Task CreateSession(string url)
-        {
-            Logger.Debug($"Create session for url={url}");
-            var metadata = new grpc::Metadata();
-            Signature.Sign(this, metadata);
-            var stream = Manager.Telemetry(url, metadata);
-            var session = new Session(url, stream, this);
-            _sessions.TryAdd(url, session);
-            await session.Loop();
-        }
 
-        internal async Task<List<Message>> ReceiveMessage(rmq::Assignment assignment, string group)
+        private async Task<TopicRouteData> FetchTopicRoute0(string topic)
         {
-            var targetUrl = TargetUrl(assignment);
-            var metadata = new grpc::Metadata();
-            Signature.Sign(this, metadata);
-            var request = new rmq::ReceiveMessageRequest
+            var request = new Proto::QueryRouteRequest
             {
-                Group = new rmq::Resource
+                Topic = new Proto::Resource
                 {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = group
+                    Name = topic
                 },
-                MessageQueue = assignment.MessageQueue
+                Endpoints = ClientConfig.Endpoints.ToProtobuf()
             };
-            var messages = await Manager.ReceiveMessage(targetUrl, metadata, request, 
-                ClientSettings.Subscription.LongPollingTimeout.ToTimeSpan());
-            return messages;
+
+            var queryRouteResponse =
+                await Manager.QueryRoute(ClientConfig.Endpoints, request, ClientConfig.RequestTimeout);
+            var messageQueues = queryRouteResponse.MessageQueues.ToList();
+            return new TopicRouteData(messageQueues);
         }
 
-        public async Task<Boolean> Ack(string target, string group, string topic, string receiptHandle, String messageId)
+        public async void Heartbeat()
         {
-            var request = new rmq::AckMessageRequest
+            var endpoints = GetTotalRouteEndpoints();
+            if (0 == endpoints.Count)
             {
-                Group = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = group
-                },
-                Topic = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = topic
-                }
-            };
+                Logger.Debug("No broker endpoints available in topic route");
+                return;
+            }
 
-            var entry = new rmq::AckMessageEntry
-            {
-                ReceiptHandle = receiptHandle,
-                MessageId = messageId
-            };
-            request.Entries.Add(entry);
+            var request = WrapHeartbeatRequest();
 
-            var metadata = new grpc::Metadata();
-            Signature.Sign(this, metadata);
-            return await Manager.Ack(target, metadata, request, RequestTimeout);
+            var tasks = endpoints.Select(endpoint => Manager.Heartbeat(endpoint, request, ClientConfig.RequestTimeout))
+                .Cast<Task>().ToList();
+
+            await Task.WhenAll(tasks);
         }
 
-        public async Task<Boolean> ChangeInvisibleDuration(string target, string group, string topic, string receiptHandle, String messageId)
-        {
-            var request = new rmq::ChangeInvisibleDurationRequest
-            {
-                ReceiptHandle = receiptHandle,
-                Group = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = group
-                },
-                Topic = new rmq::Resource
-                {
-                    ResourceNamespace = _resourceNamespace,
-                    Name = topic
-                },
-                MessageId = messageId
-            };
 
+        public grpc.Metadata Sign()
+        {
             var metadata = new grpc::Metadata();
-            Signature.Sign(this, metadata);
-            return await Manager.ChangeInvisibleDuration(target, metadata, request, RequestTimeout);
+            Signature.Sign(ClientConfig, metadata);
+            return metadata;
         }
 
-        public async Task<bool> NotifyClientTermination(rmq.Resource group)
+        public async void NotifyClientTermination(Proto.Resource group)
         {
-            List<string> endpoints = AvailableBrokerEndpoints();
-            var request = new rmq::NotifyClientTerminationRequest
+            var endpoints = GetTotalRouteEndpoints();
+            var request = new Proto::NotifyClientTerminationRequest
             {
                 Group = group
             };
-            var metadata = new grpc.Metadata();
-            Signature.Sign(this, metadata);
-
-            List<Task<Boolean>> tasks = new List<Task<Boolean>>();
-
-            foreach (var endpoint in endpoints)
-            {
-                tasks.Add(Manager.NotifyClientTermination(endpoint, metadata, request, RequestTimeout));
-            }
-
-            bool[] results = await Task.WhenAll(tasks);
-            foreach (bool b in results)
-            {
-                if (!b)
-                {
-                    return false;
-                }
-            }
-            return true;
-        }
-
-        internal virtual void OnSettingsReceived(rmq::Settings settings)
-        {
-            if (null != settings.Metric)
+            foreach (var item in endpoints)
             {
-                ClientSettings.Metric = new rmq::Metric();
-                ClientSettings.Metric.MergeFrom(settings.Metric);
-            }
-
-            if (null != settings.BackoffPolicy)
-            {
-                ClientSettings.BackoffPolicy = new rmq::RetryPolicy();
-                ClientSettings.BackoffPolicy.MergeFrom(settings.BackoffPolicy);
-            }
-
-            switch (settings.PubSubCase)
-            {
-                case rmq.Settings.PubSubOneofCase.Publishing:
+                var response = await Manager.NotifyClientTermination(item, request, ClientConfig.RequestTimeout);
+                try
                 {
-                    ClientSettings.Publishing = settings.Publishing;
-                    break;
+                    StatusChecker.Check(response.Status, request);
                 }
-
-                case rmq.Settings.PubSubOneofCase.Subscription:
+                catch (Exception e)
                 {
-                    ClientSettings.Subscription = settings.Subscription;
-                    break;
+                    Logger.Error(e, $"Failed to notify client's termination, clientId=${ClientId}, " +
+                                    $"endpoints=${item}");
                 }
             }
         }
 
-        protected readonly IClientManager Manager;
+        public CancellationTokenSource TelemetryCts()
+        {
+            return _telemetryCts;
+        }
 
-        protected readonly ConcurrentDictionary<string, bool> _topicsOfInterest = new ();
+        public abstract Proto.Settings GetSettings();
 
-        public void AddTopicOfInterest(string topic)
+        public string GetClientId()
         {
-            _topicsOfInterest.TryAdd(topic, true);
+            return ClientId;
         }
 
-        public void RemoveTopicOfInterest(string topic)
+        public void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
+            Proto.RecoverOrphanedTransactionCommand command)
         {
-            _topicsOfInterest.TryRemove(topic, out var _);
         }
 
-        private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteTable;
-        private readonly CancellationTokenSource _updateTopicRouteCts;
-        private readonly CancellationTokenSource _HeartbeatCts;
-        private readonly CancellationTokenSource _telemetryCts;
-
-        public CancellationTokenSource TelemetryCts()
+        public void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
         {
-            return _telemetryCts;
         }
 
-        protected readonly AccessPoint AccessPoint;
-
-        // This field is subject changes from servers.
-        protected readonly rmq::Settings ClientSettings;
-
-        private readonly Random _random = new Random();
-
-        private readonly ConcurrentDictionary<string, Session> _sessions = new ConcurrentDictionary<string, Session>();
-
-        protected const string MeterName = "Apache.RocketMQ.Client";
+        public void OnPrintThreadStackTraceCommand(Endpoints endpoints, Proto.PrintThreadStackTraceCommand command)
+        {
+        }
 
-        protected static readonly Meter MetricMeter = new(MeterName, "1.0");
+        public abstract void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings);
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ClientConfig.cs b/csharp/rocketmq-client-csharp/ClientConfig.cs
index beca1067..e5fd8643 100644
--- a/csharp/rocketmq-client-csharp/ClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/ClientConfig.cs
@@ -14,159 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
-using System.Collections.Generic;
 using System.Threading;
-using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
-
     public class ClientConfig : IClientConfig
     {
-        private static long instanceSequence = 0;
+        private static long _instanceSequence = 0;
 
-        public ClientConfig()
+        public ClientConfig(string endpoints)
         {
             var hostName = System.Net.Dns.GetHostName();
             var pid = System.Diagnostics.Process.GetCurrentProcess().Id;
-            this.clientId_ = string.Format("{0}@{1}#{2}", hostName, pid, Interlocked.Increment(ref instanceSequence));
-            this._requestTimeout = TimeSpan.FromSeconds(3);
-            this.longPollingIoTimeout_ = TimeSpan.FromSeconds(30);
-            this.client_type_ = rmq::ClientType.Unspecified;
-            this.access_point_ = new rmq::Endpoints();
-            this.back_off_policy_ = new rmq::RetryPolicy();
-            this._publishing = new Publishing();
-        }
-
-        public string region()
-        {
-            return _region;
-        }
-        public string Region
-        {
-            set { _region = value; }
-        }
-
-        public string serviceName()
-        {
-            return _serviceName;
-        }
-        public string ServiceName
-        {
-            set { _serviceName = value; }
-        }
-
-        public string resourceNamespace()
-        {
-            return _resourceNamespace;
-        }
-        public string ResourceNamespace
-        {
-            get { return _resourceNamespace; }
-            set { _resourceNamespace = value; }
+            ClientId = $"{hostName}@{pid}@{Interlocked.Increment(ref _instanceSequence)}";
+            RequestTimeout = TimeSpan.FromSeconds(3);
+            Endpoints = new Endpoints(endpoints);
         }
 
-        public ICredentialsProvider credentialsProvider()
-        {
-            return credentialsProvider_;
-        }
+        public ICredentialsProvider CredentialsProvider { get; set; }
 
-        public ICredentialsProvider CredentialsProvider
-        {
-            set { credentialsProvider_ = value; }
-        }
+        public TimeSpan RequestTimeout { get; set; }
 
-        public TimeSpan RequestTimeout
-        {
-            get
-            {
-                return _requestTimeout;
-            }
-            set
-            {
-                _requestTimeout = value;
-            }
-        }
+        public string ClientId { get; }
 
-        public string getGroupName()
-        {
-            return groupName_;
-        }
-        public string GroupName
-        {
-            set { groupName_ = value; }
-        }
-
-        public string clientId()
-        {
-            return clientId_;
-        }
-
-        public bool isTracingEnabled()
-        {
-            return tracingEnabled_;
-        }
-        public bool TracingEnabled
-        {
-            set { tracingEnabled_ = value; }
-        }
-
-        private string _region = "cn-hangzhou";
-        private string _serviceName = "ONS";
-
-        protected string _resourceNamespace;
-
-        private ICredentialsProvider credentialsProvider_;
-        
-        private TimeSpan _requestTimeout;
-
-        private TimeSpan longPollingIoTimeout_;
-
-        private string groupName_;
-
-        private string clientId_;
-
-        private bool tracingEnabled_;
-
-        private rmq::ClientType client_type_;
-        public rmq::ClientType ClientType
-        {
-            get { return client_type_; }
-            set { client_type_ = value; }
-        }
-
-
-        private rmq::Endpoints access_point_;
-
-        public rmq::AddressScheme AccessPointScheme
-        {
-            get { return access_point_.Scheme; }
-            set { access_point_.Scheme = value; }
-        }
-
-        public List<rmq::Address> AccessPointEndpoints
-        {
-            get
-            {
-                return new List<rmq::Address>(access_point_.Addresses);
-            }
-
-            set
-            {
-                access_point_.Addresses.Clear();
-                access_point_.Addresses.AddRange(value);
-            }
-        }
-
-        private rmq::RetryPolicy back_off_policy_;
-
-        private Publishing _publishing;
-        public Publishing Publishing
-        {
-            get { return _publishing; }
-        }
 
+        public Endpoints Endpoints { get; }
     }
-
-}
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ClientManager.cs b/csharp/rocketmq-client-csharp/ClientManager.cs
index 64955b76..bd18ebc4 100644
--- a/csharp/rocketmq-client-csharp/ClientManager.cs
+++ b/csharp/rocketmq-client-csharp/ClientManager.cs
@@ -17,34 +17,34 @@
 
 using rmq = Apache.Rocketmq.V2;
 using System;
-using System.IO;
-using System.IO.Compression;
 using System.Threading;
 using System.Threading.Tasks;
 using grpc = Grpc.Core;
 using System.Collections.Generic;
-using System.Security.Cryptography;
 using NLog;
 
 namespace Org.Apache.Rocketmq
 {
     public class ClientManager : IClientManager
     {
-        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+        private readonly IClient _client;
+        private readonly Dictionary<Endpoints, RpcClient> _rpcClients;
+        private readonly ReaderWriterLockSlim _clientLock;
 
-        public ClientManager()
+        public ClientManager(IClient client)
         {
-            _rpcClients = new Dictionary<string, RpcClient>();
+            _client = client;
+            _rpcClients = new Dictionary<Endpoints, RpcClient>();
             _clientLock = new ReaderWriterLockSlim();
         }
 
-        public IRpcClient GetRpcClient(string target)
+        private IRpcClient GetRpcClient(Endpoints endpoints)
         {
             _clientLock.EnterReadLock();
             try
             {
                 // client exists, return in advance.
-                if (_rpcClients.TryGetValue(target, out var cachedClient))
+                if (_rpcClients.TryGetValue(endpoints, out var cachedClient))
                 {
                     return cachedClient;
                 }
@@ -58,14 +58,14 @@ namespace Org.Apache.Rocketmq
             try
             {
                 // client exists, return in advance.
-                if (_rpcClients.TryGetValue(target, out var cachedClient))
+                if (_rpcClients.TryGetValue(endpoints, out var cachedClient))
                 {
                     return cachedClient;
                 }
 
                 // client does not exist, generate a new one
-                var client = new RpcClient(target);
-                _rpcClients.Add(target, client);
+                var client = new RpcClient(endpoints);
+                _rpcClients.Add(endpoints, client);
                 return client;
             }
             finally
@@ -74,254 +74,77 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> Telemetry(string target, grpc::Metadata metadata)
-        {
-            var rpcClient = GetRpcClient(target);
-            return rpcClient.Telemetry(metadata);
-        }
-
-        public async Task<TopicRouteData> ResolveRoute(string target, grpc::Metadata metadata,
-            rmq::QueryRouteRequest request, TimeSpan timeout)
+        public async Task Shutdown()
         {
-            var rpcClient = GetRpcClient(target);
-            Logger.Debug($"QueryRouteRequest: {request}");
-            var queryRouteResponse = await rpcClient.QueryRoute(metadata, request, timeout);
-
-            if (queryRouteResponse.Status.Code != rmq::Code.Ok)
+            _clientLock.EnterReadLock();
+            try
             {
-                Logger.Warn($"Failed to query route entries for topic={request.Topic.Name} from {target}: {queryRouteResponse.Status}");
-                // Raise an application layer exception
-            }
-            Logger.Debug($"QueryRouteResponse: {queryRouteResponse}");
+                List<Task> tasks = new List<Task>();
+                foreach (var item in _rpcClients)
+                {
+                    tasks.Add(item.Value.Shutdown());
+                }
 
-            var messageQueues = new List<rmq::MessageQueue>();
-            foreach (var messageQueue in queryRouteResponse.MessageQueues)
+                await Task.WhenAll(tasks);
+            }
+            finally
             {
-                messageQueues.Add(messageQueue);
+                _clientLock.ExitReadLock();
             }
-            var topicRouteData = new TopicRouteData(messageQueues);
-            return topicRouteData;
         }
 
-        public async Task<Boolean> Heartbeat(string target, grpc::Metadata metadata, rmq::HeartbeatRequest request,
-            TimeSpan timeout)
+        public grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> Telemetry(
+            Endpoints endpoints)
         {
-            var rpcClient = GetRpcClient(target);
-            Logger.Debug($"Heartbeat to {target}, Request: {request}");
-            var response = await rpcClient.Heartbeat(metadata, request, timeout);
-            Logger.Debug($"Heartbeat to {target} response status: {response.Status}");
-            return response.Status.Code == rmq::Code.Ok;
+            return GetRpcClient(endpoints).Telemetry(_client.Sign());
         }
 
-        public async Task<rmq::SendMessageResponse> SendMessage(string target, grpc::Metadata metadata,
-            rmq::SendMessageRequest request, TimeSpan timeout)
+        public async Task<rmq.QueryRouteResponse> QueryRoute(Endpoints endpoints, rmq.QueryRouteRequest request,
+            TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            var response = await rpcClient.SendMessage(metadata, request, timeout);
-            return response;
+            return await GetRpcClient(endpoints).QueryRoute(_client.Sign(), request, timeout);
         }
 
-        public async Task<Boolean> NotifyClientTermination(string target, grpc::Metadata metadata,
-            rmq::NotifyClientTerminationRequest request, TimeSpan timeout)
+        public async Task<rmq.HeartbeatResponse> Heartbeat(Endpoints endpoints, rmq.HeartbeatRequest request,
+            TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            rmq::NotifyClientTerminationResponse response =
-                await rpcClient.NotifyClientTermination(metadata, request, timeout);
-            return response.Status.Code == rmq::Code.Ok;
+            return await GetRpcClient(endpoints).Heartbeat(_client.Sign(), request, timeout);
         }
 
-        public async Task<List<rmq::Assignment>> QueryLoadAssignment(string target, grpc::Metadata metadata, rmq::QueryAssignmentRequest request, TimeSpan timeout)
+        public async Task<rmq.NotifyClientTerminationResponse> NotifyClientTermination(Endpoints endpoints,
+            rmq.NotifyClientTerminationRequest request, TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            rmq::QueryAssignmentResponse response = await rpcClient.QueryAssignment(metadata, request, timeout);
-            if (response.Status.Code != rmq::Code.Ok)
-            {
-                // TODO: Build exception hierarchy
-                throw new Exception($"Failed to query load assignment from server. Cause: {response.Status.Message}");
-            }
-
-            List<rmq::Assignment> assignments = new List<rmq.Assignment>();
-            foreach (var item in response.Assignments)
-            {
-                assignments.Add(item);
-            }
-            return assignments;
+            return await GetRpcClient(endpoints).NotifyClientTermination(_client.Sign(), request, timeout);
         }
 
-        public async Task<List<Message>> ReceiveMessage(string target, grpc::Metadata metadata, 
-            rmq::ReceiveMessageRequest request, TimeSpan timeout)
+        public async Task<rmq::SendMessageResponse> SendMessage(Endpoints endpoints, rmq::SendMessageRequest request,
+            TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            List<rmq::ReceiveMessageResponse> response = await rpcClient.ReceiveMessage(metadata, request, timeout);
-
-            if (null == response || 0 == response.Count)
-            {
-                // TODO: throw an exception to propagate this error?
-                return new List<Message>();
-            }
-
-            List<Message> messages = new List<Message>();
-            
-            foreach (var entry in response)
-            {
-                switch (entry.ContentCase)
-                {
-                    case rmq.ReceiveMessageResponse.ContentOneofCase.None:
-                    {
-                        Logger.Warn("Unexpected ReceiveMessageResponse content type");
-                        break;
-                    }
-                    
-                    case rmq.ReceiveMessageResponse.ContentOneofCase.Status:
-                    {
-                        switch (entry.Status.Code)
-                        {
-                            case rmq.Code.Ok:
-                            {
-                                break;
-                            }
-
-                            case rmq.Code.Forbidden:
-                            {
-                                Logger.Warn("Receive message denied");
-                                break;
-                            }
-                            case rmq.Code.TooManyRequests:
-                            {
-                                Logger.Warn("TooManyRequest: servers throttled");
-                                break;
-                            }
-                            case rmq.Code.MessageNotFound:
-                            {
-                                Logger.Info("No message is found in the server");
-                                break;
-                            }
-                            default:
-                            {
-                                Logger.Warn($"Unknown error status({entry.Status.Code}): {entry.Status.Message}");
-                                break;
-                            }
-                        }
-                        break;
-                    }
-
-                    case rmq.ReceiveMessageResponse.ContentOneofCase.Message:
-                    {
-                        var message = Convert(target, entry.Message);
-                        messages.Add(message);
-                        break;
-                    }
-
-                    case rmq.ReceiveMessageResponse.ContentOneofCase.DeliveryTimestamp:
-                    {
-                        var begin = entry.DeliveryTimestamp;
-                        var costs = DateTime.UtcNow - begin.ToDateTime();
-                        // TODO: Collect metrics
-                        break;
-                    }
-                }
-            }
-            return messages;
+            return await GetRpcClient(endpoints).SendMessage(_client.Sign(), request, timeout);
         }
 
-        private Message Convert(string sourceHost, rmq::Message message)
+        public async Task<rmq::QueryAssignmentResponse> QueryAssignment(Endpoints endpoints,
+            rmq.QueryAssignmentRequest request, TimeSpan timeout)
         {
-            var msg = new Message
-            {
-                Topic = message.Topic.Name,
-                MessageId = message.SystemProperties.MessageId,
-                Tag = message.SystemProperties.Tag
-            };
-
-            // Validate message body checksum
-            byte[] raw = message.Body.ToByteArray();
-            if (rmq::DigestType.Crc32 == message.SystemProperties.BodyDigest.Type)
-            {
-                uint checksum = Force.Crc32.Crc32Algorithm.Compute(raw, 0, raw.Length);
-                if (!message.SystemProperties.BodyDigest.Checksum.Equals(checksum.ToString("X")))
-                {
-                    msg._checksumVerifiedOk = false;
-                }
-            }
-            else if (rmq::DigestType.Md5 == message.SystemProperties.BodyDigest.Type)
-            {
-                var checksum = MD5.HashData(raw);
-                if (!message.SystemProperties.BodyDigest.Checksum.Equals(System.Convert.ToHexString(checksum)))
-                {
-                    msg._checksumVerifiedOk = false;
-                }
-            }
-            else if (rmq::DigestType.Sha1 == message.SystemProperties.BodyDigest.Type)
-            {
-                var checksum = SHA1.HashData(raw);
-                if (!message.SystemProperties.BodyDigest.Checksum.Equals(System.Convert.ToHexString(checksum)))
-                {
-                    msg._checksumVerifiedOk = false;
-                }
-            }
-
-            foreach (var entry in message.UserProperties)
-            {
-                msg.UserProperties.Add(entry.Key, entry.Value);
-            }
-
-            msg._receiptHandle = message.SystemProperties.ReceiptHandle;
-            msg._sourceHost = sourceHost;
-
-            msg.Keys.AddRange(message.SystemProperties.Keys);
-            msg.DeliveryAttempt = message.SystemProperties.DeliveryAttempt;
-
-            if (message.SystemProperties.BodyEncoding == rmq::Encoding.Gzip)
-            {
-                // Decompress/Inflate message body
-                using var inputStream = new MemoryStream(raw);
-                using var gzipStream = new GZipStream(inputStream, CompressionMode.Decompress);
-                using var outputStream = new MemoryStream();
-                gzipStream.CopyTo(outputStream);
-                msg.Body = outputStream.ToArray();
-            }
-            else
-            {
-                msg.Body = raw;
-            }
-
-            return msg;
+            return await GetRpcClient(endpoints).QueryAssignment(_client.Sign(), request, timeout);
         }
 
-        public async Task<Boolean> Ack(string target, grpc::Metadata metadata, rmq::AckMessageRequest request, TimeSpan timeout)
+        public async Task<List<rmq::ReceiveMessageResponse>> ReceiveMessage(Endpoints endpoints,
+            rmq.ReceiveMessageRequest request, TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            var response = await rpcClient.AckMessage(metadata, request, timeout);
-            return response.Status.Code == rmq::Code.Ok;
+            return await GetRpcClient(endpoints).ReceiveMessage(_client.Sign(), request, timeout);
         }
 
-        public async Task<Boolean> ChangeInvisibleDuration(string target, grpc::Metadata metadata, rmq::ChangeInvisibleDurationRequest request, TimeSpan timeout)
+        public async Task<rmq::AckMessageResponse> AckMessage(Endpoints endpoints,
+            rmq.AckMessageRequest request, TimeSpan timeout)
         {
-            var rpcClient = GetRpcClient(target);
-            var response = await rpcClient.ChangeInvisibleDuration(metadata, request, timeout);
-            return response.Status.Code == rmq::Code.Ok;
+            return await GetRpcClient(endpoints).AckMessage(_client.Sign(), request, timeout);
         }
 
-        public async Task Shutdown()
+        public async Task<rmq::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Endpoints endpoints,
+            rmq.ChangeInvisibleDurationRequest request, TimeSpan timeout)
         {
-            _clientLock.EnterReadLock();
-            try
-            {
-                List<Task> tasks = new List<Task>();
-                foreach (var item in _rpcClients)
-                {
-                    tasks.Add(item.Value.Shutdown());
-                }
-
-                await Task.WhenAll(tasks);
-            }
-            finally
-            {
-                _clientLock.ExitReadLock();
-            }
+            return await GetRpcClient(endpoints).ChangeInvisibleDuration(_client.Sign(), request, timeout);
         }
-
-        private readonly Dictionary<string, RpcClient> _rpcClients;
-        private readonly ReaderWriterLockSlim _clientLock;
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClient.cs b/csharp/rocketmq-client-csharp/ClientType.cs
similarity index 56%
copy from csharp/rocketmq-client-csharp/IClient.cs
copy to csharp/rocketmq-client-csharp/ClientType.cs
index b1e992a8..15481c98 100644
--- a/csharp/rocketmq-client-csharp/IClient.cs
+++ b/csharp/rocketmq-client-csharp/ClientType.cs
@@ -15,23 +15,32 @@
  * limitations under the License.
  */
 
-using System.Threading.Tasks;
-using System.Threading;
-using System;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
-    public interface IClient : IClientConfig
+    public enum ClientType
     {
+        Producer,
+        SimpleConsumer,
+        PushConsumer
+    }
 
-        Task Heartbeat();
-
-        Task<bool> NotifyClientTermination(rmq.Resource group);
-
-        void BuildClientSetting(rmq::Settings settings);
-        
-
-        CancellationTokenSource TelemetryCts();
+    public static class ClientTypeHelper
+    {
+        public static rmq.ClientType ToProtobuf(ClientType clientType)
+        {
+            switch (clientType)
+            {
+                case ClientType.Producer:
+                    return rmq.ClientType.Producer;
+                case ClientType.SimpleConsumer:
+                    return rmq.ClientType.SimpleConsumer;
+                case ClientType.PushConsumer:
+                    return rmq.ClientType.PushConsumer;
+                default:
+                    return rmq.ClientType.Unspecified;
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs b/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
index 0b80fcea..73d05f63 100644
--- a/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
@@ -28,7 +28,7 @@ namespace Org.Apache.Rocketmq
      * A sample config content is as follows:
      * {"AccessKey": "key", "AccessSecret": "secret"}
      */
-    public class ConfigFileCredentialsProvider : ICredentialsProvider
+    public class ConfigFileCredentialsProvider 
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
@@ -63,7 +63,7 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public Credentials getCredentials()
+        public Credentials GetCredentials()
         {
             if (!_valid)
             {
diff --git a/csharp/rocketmq-client-csharp/Endpoints.cs b/csharp/rocketmq-client-csharp/Endpoints.cs
new file mode 100644
index 00000000..3228881c
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Endpoints.cs
@@ -0,0 +1,133 @@
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class Endpoints : IEquatable<Endpoints>
+    {
+        private static readonly AddressListEqualityComparer AddressListComparer = new();
+        private static readonly string EndpointSeparator = ":";
+        private List<Address> Addresses { get; }
+        private AddressScheme Scheme { get; }
+        private readonly int _hashCode;
+
+        public Endpoints(global::Apache.Rocketmq.V2.Endpoints endpoints)
+        {
+            Addresses = new List<Address>();
+            foreach (var address in endpoints.Addresses)
+            {
+                Addresses.Add(new Address(address));
+            }
+
+            if (!Addresses.Any())
+            {
+                throw new NotSupportedException("No available address");
+            }
+
+            switch (endpoints.Scheme)
+            {
+                case rmq.AddressScheme.Ipv4:
+                    Scheme = AddressScheme.Ipv4;
+                    break;
+                case rmq.AddressScheme.Ipv6:
+                    Scheme = AddressScheme.Ipv6;
+                    break;
+                case rmq.AddressScheme.DomainName:
+                default:
+                    Scheme = AddressScheme.DomainName;
+                    if (Addresses.Count > 1)
+                    {
+                        throw new NotSupportedException("Multiple addresses are not allowed in domain scheme");
+                    }
+
+                    break;
+            }
+
+            unchecked
+            {
+                var hash = 17;
+                hash = (hash * 31) + AddressListComparer.GetHashCode(Addresses);
+                hash = (hash * 31) + (int)Scheme;
+                _hashCode = hash;
+            }
+        }
+
+        public Endpoints(string endpoints)
+        {
+            // TODO
+            var strs = endpoints.Split(EndpointSeparator);
+            Scheme = AddressScheme.DomainName;
+            string host = strs[0];
+            int port = int.Parse(strs[1]);
+            Address address = new Address(host, port);
+            var addresses = new List<Address>();
+            addresses.Add(address);
+            Addresses = addresses;
+        }
+
+        public string GrpcTarget
+        {
+            // TODO
+            get
+            {
+                foreach (var address in Addresses)
+                {
+                    var target = "https://" + address.Host + ":" + address.Port;
+                    // Console.WriteLine(target);
+                    return "https://" + address.Host + ":" + address.Port;
+                }
+
+                return "";
+            }
+        }
+
+        public bool Equals(Endpoints other)
+        {
+            if (ReferenceEquals(null, other))
+            {
+                return false;
+            }
+
+            if (ReferenceEquals(this, other))
+            {
+                return true;
+            }
+
+            return Addresses.SequenceEqual(other.Addresses) && Scheme == other.Scheme;
+        }
+
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj))
+            {
+                return false;
+            }
+
+            if (ReferenceEquals(this, obj))
+            {
+                return true;
+            }
+
+            return obj.GetType() == GetType() && Equals((Endpoints)obj);
+        }
+
+        public override int GetHashCode()
+        {
+            return _hashCode;
+        }
+
+        public rmq.Endpoints ToProtobuf()
+        {
+            var endpoints = new rmq.Endpoints();
+            foreach (var address in Addresses)
+            {
+                endpoints.Addresses.Add(address.ToProtobuf());
+            }
+
+            endpoints.Scheme = AddressSchemeHelper.ToProtobuf(Scheme);
+            return endpoints;
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/Error/BadRequestException.cs
similarity index 64%
copy from csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
copy to csharp/rocketmq-client-csharp/Error/BadRequestException.cs
index edd810dd..04fcbd3a 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/Error/BadRequestException.cs
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class StaticCredentialsProvider : ICredentialsProvider
+    /// <summary>
+    /// Generic exception for bad request, indicating that the required fields of headers are missing.
+    /// </summary>
+    public class BadRequestException : ClientException
     {
-
-        public StaticCredentialsProvider(string accessKey, string accessSecret)
+        public BadRequestException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
         }
 
-        public Credentials getCredentials()
+        public BadRequestException(int responseCode, string message) : base(responseCode, message)
         {
-            return new Credentials(accessKey, accessSecret);
         }
-
-        private string accessKey;
-        private string accessSecret;
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SendReceipt.cs b/csharp/rocketmq-client-csharp/Error/ClientException.cs
similarity index 57%
copy from csharp/rocketmq-client-csharp/SendReceipt.cs
copy to csharp/rocketmq-client-csharp/Error/ClientException.cs
index 0f29991b..f5fc4a92 100644
--- a/csharp/rocketmq-client-csharp/SendReceipt.cs
+++ b/csharp/rocketmq-client-csharp/Error/ClientException.cs
@@ -15,35 +15,30 @@
  * limitations under the License.
  */
 
-namespace Org.Apache.Rocketmq
+using System;
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public sealed class SendReceipt
+    public class ClientException : Exception
     {
-        public SendReceipt(string messageId)
+        private static readonly string RequestIdKey = "request-id";
+        private static readonly string ResponseCodeKey = "response-code";
+
+        protected ClientException(string message, Exception exception) : base(message, exception)
         {
-            status_ = SendStatus.SEND_OK;
-            messageId_ = messageId;
         }
 
-        public SendReceipt(string messageId, SendStatus status)
+        protected ClientException(string message) : base(message)
         {
-            status_ = status;
-            messageId_ = messageId;
         }
 
-        private string messageId_;
-
-        public string MessageId
+        protected ClientException(int responseCode, string requestId, string message) : base(
+            $"[{RequestIdKey}={requestId}, {ResponseCodeKey}={responseCode}] {message}")
         {
-            get { return messageId_; }
         }
 
-
-        private SendStatus status_;
-
-        public SendStatus Status
+        protected ClientException(int responseCode, string message) : base()
         {
-            get { return status_; }
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/Error/ForbiddenException.cs
similarity index 64%
copy from csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
copy to csharp/rocketmq-client-csharp/Error/ForbiddenException.cs
index edd810dd..443a58e2 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/Error/ForbiddenException.cs
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class StaticCredentialsProvider : ICredentialsProvider
+    /// <summary>
+    /// Generic exception for the case that user does not have permission to access/operation the resource.
+    /// </summary>
+    public class ForbiddenException : ClientException
     {
-
-        public StaticCredentialsProvider(string accessKey, string accessSecret)
+        public ForbiddenException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
         }
 
-        public Credentials getCredentials()
+        public ForbiddenException(int responseCode, string message) : base(responseCode, message)
         {
-            return new Credentials(accessKey, accessSecret);
         }
-
-        private string accessKey;
-        private string accessSecret;
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ProcessQueue.cs b/csharp/rocketmq-client-csharp/Error/InternalErrorException.cs
similarity index 59%
copy from csharp/rocketmq-client-csharp/ProcessQueue.cs
copy to csharp/rocketmq-client-csharp/Error/InternalErrorException.cs
index 1022978f..306d6af5 100644
--- a/csharp/rocketmq-client-csharp/ProcessQueue.cs
+++ b/csharp/rocketmq-client-csharp/Error/InternalErrorException.cs
@@ -14,30 +14,26 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class ProcessQueue
+    /// <summary>
+    /// Generic exception indicates that the server/client encountered an unexpected condition
+    /// that prevented it from fulfilling the request.
+    /// </summary>
+    public class InternalErrorException : ClientException
     {
-
-        public ProcessQueue()
+        public InternalErrorException(string message) : base(message)
         {
-            _lastReceivedTime = DateTime.UtcNow;
         }
-        public bool Dropped { get; set; }
 
-        private DateTime _lastReceivedTime;
-
-        public DateTime LastReceiveTime
+        public InternalErrorException(int responseCode, string requestId, string message) : base(responseCode,
+            requestId, message)
         {
-            get { return _lastReceivedTime; }
-            set { _lastReceivedTime = value; }
         }
 
-        internal bool Expired()
+        public InternalErrorException(int responseCode, string message) : base(responseCode, message)
         {
-            return DateTime.UtcNow.Subtract(_lastReceivedTime).TotalMilliseconds > 30 * 1000;
         }
-
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ProcessQueue.cs b/csharp/rocketmq-client-csharp/Error/NotFoundException.cs
similarity index 63%
copy from csharp/rocketmq-client-csharp/ProcessQueue.cs
copy to csharp/rocketmq-client-csharp/Error/NotFoundException.cs
index 1022978f..1a1badb4 100644
--- a/csharp/rocketmq-client-csharp/ProcessQueue.cs
+++ b/csharp/rocketmq-client-csharp/Error/NotFoundException.cs
@@ -14,30 +14,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class ProcessQueue
+    /// <summary>
+    /// Generic exception for resource not found.
+    /// </summary>
+    public class NotFoundException : ClientException
     {
-
-        public ProcessQueue()
+        public NotFoundException(string message) : base(message)
         {
-            _lastReceivedTime = DateTime.UtcNow;
         }
-        public bool Dropped { get; set; }
 
-        private DateTime _lastReceivedTime;
-
-        public DateTime LastReceiveTime
+        public NotFoundException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
         {
-            get { return _lastReceivedTime; }
-            set { _lastReceivedTime = value; }
         }
 
-        internal bool Expired()
+        public NotFoundException(int responseCode, string message) : base(responseCode, message)
         {
-            return DateTime.UtcNow.Subtract(_lastReceivedTime).TotalMilliseconds > 30 * 1000;
         }
-
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/Error/PayloadTooLargeException.cs
similarity index 63%
copy from csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
copy to csharp/rocketmq-client-csharp/Error/PayloadTooLargeException.cs
index edd810dd..0aa653cc 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/Error/PayloadTooLargeException.cs
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class StaticCredentialsProvider : ICredentialsProvider
+    /// <summary>
+    /// Generic exception represents that the request entity is larger than the limits defined by the server.
+    /// </summary>
+    public class PayloadTooLargeException : ClientException
     {
-
-        public StaticCredentialsProvider(string accessKey, string accessSecret)
+        public PayloadTooLargeException(int responseCode, string requestId, string message) : base(responseCode,
+            requestId, message)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
         }
 
-        public Credentials getCredentials()
+        public PayloadTooLargeException(int responseCode, string message) : base(responseCode, message)
         {
-            return new Credentials(accessKey, accessSecret);
         }
-
-        private string accessKey;
-        private string accessSecret;
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/QuickStart.cs b/csharp/rocketmq-client-csharp/Error/PaymentRequiredException.cs
similarity index 68%
copy from csharp/examples/QuickStart.cs
copy to csharp/rocketmq-client-csharp/Error/PaymentRequiredException.cs
index ff28d3e3..763f1141 100644
--- a/csharp/examples/QuickStart.cs
+++ b/csharp/rocketmq-client-csharp/Error/PaymentRequiredException.cs
@@ -1,32 +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.
- */
-
-using System.Threading.Tasks;
-
-namespace examples
-{
-    static class QuickStart
-    {
-        public static async Task Main()
-        {
-            await ProducerNormalMessageExample.QuickStart();
-            await ProducerFifoMessageExample.QuickStart();
-            await ProducerDelayMessageExample.QuickStart();
-            await SimpleConsumerExample.QuickStart();
-        }
-    }
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Org.Apache.Rocketmq.Error
+{
+    public class PaymentRequiredException : ClientException
+    {
+        public PaymentRequiredException(int responseCode, string requestId, string message) : base(responseCode,
+            requestId, message)
+        {
+        }
+
+        public PaymentRequiredException(int responseCode, string message) : base(responseCode, message)
+        {
+        }
+    }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IMessageListener.cs b/csharp/rocketmq-client-csharp/Error/ProxyTimeoutException.cs
similarity index 68%
rename from csharp/rocketmq-client-csharp/IMessageListener.cs
rename to csharp/rocketmq-client-csharp/Error/ProxyTimeoutException.cs
index f46efd51..9121cae1 100644
--- a/csharp/rocketmq-client-csharp/IMessageListener.cs
+++ b/csharp/rocketmq-client-csharp/Error/ProxyTimeoutException.cs
@@ -15,16 +15,17 @@
  * limitations under the License.
  */
 
-using System.Collections.Generic;
-using System.Threading.Tasks;
-
-namespace Org.Apache.Rocketmq
+namespace Org.Apache.Rocketmq.Error
 {
-
-    public interface IMessageListener
+    public class ProxyTimeoutException : ClientException
     {
-        Task Consume(List<Message> messages, List<Message> failed);
+        public ProxyTimeoutException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
+        {
+        }
 
+        public ProxyTimeoutException(int responseCode, string message) : base(responseCode, message)
+        {
+        }
     }
-
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ProcessQueue.cs b/csharp/rocketmq-client-csharp/Error/RequestHeaderFieldsTooLargeException.cs
similarity index 56%
rename from csharp/rocketmq-client-csharp/ProcessQueue.cs
rename to csharp/rocketmq-client-csharp/Error/RequestHeaderFieldsTooLargeException.cs
index 1022978f..f7303136 100644
--- a/csharp/rocketmq-client-csharp/ProcessQueue.cs
+++ b/csharp/rocketmq-client-csharp/Error/RequestHeaderFieldsTooLargeException.cs
@@ -14,30 +14,23 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using System;
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class ProcessQueue
+    /// <summary>
+    /// Generic exception for the case that the server is unwilling to process the request
+    /// because its header fields are too large. The request may be resubmitted after reducing
+    /// the size of the request header fields.
+    /// </summary>
+    public class RequestHeaderFieldsTooLargeException : ClientException
     {
-
-        public ProcessQueue()
+        public RequestHeaderFieldsTooLargeException(int responseCode, string requestId, string message) : base(
+            responseCode, requestId, message)
         {
-            _lastReceivedTime = DateTime.UtcNow;
         }
-        public bool Dropped { get; set; }
 
-        private DateTime _lastReceivedTime;
-
-        public DateTime LastReceiveTime
+        public RequestHeaderFieldsTooLargeException(int responseCode, string message) : base(responseCode, message)
         {
-            get { return _lastReceivedTime; }
-            set { _lastReceivedTime = value; }
         }
-
-        internal bool Expired()
-        {
-            return DateTime.UtcNow.Subtract(_lastReceivedTime).TotalMilliseconds > 30 * 1000;
-        }
-
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Error/StatusChecker.cs b/csharp/rocketmq-client-csharp/Error/StatusChecker.cs
new file mode 100644
index 00000000..55f56d57
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Error/StatusChecker.cs
@@ -0,0 +1,41 @@
+using Apache.Rocketmq.V2;
+using Google.Protobuf;
+
+namespace Org.Apache.Rocketmq.Error
+{
+    public class StatusChecker
+    {
+        public static void check(Status status, IMessage message)
+        {
+            // var code = status.Code;
+            // switch (code)
+            // {
+            //     case Code.Ok:
+            //     case Code.MultipleResults:
+            //         return;
+            //     case Code.BadRequest:
+            //     case Code.IllegalAccessPoint:
+            //     case Code.IllegalTopic:
+            //     case Code.IllegalConsumerGroup:
+            //     case Code.IllegalMessageTag:
+            //     case Code.IllegalMessageKey:
+            //     case Code.IllegalMessageGroup:
+            //     case Code.IllegalMessagePropertyKey:
+            //     case Code.InvalidTransactionId:
+            //     case Code.IllegalMessageId:
+            //     case Code.IllegalFilterExpression:
+            //     case Code.IllegalInvisibleTime:
+            //     case Code.IllegalDeliveryTime:
+            //     case Code.InvalidReceiptHandle:
+            //     case Code.MessagePropertyConflictWithType:
+            //     case Code.UnrecognizedClientType:
+            //     case Code.MessageCorrupted:
+            //     case Code.ClientIdRequired:
+            //     case Code.IllegalPollingTime:
+            //         throw new BadRequestException(code)
+            //
+            //     case ILLEGAL_POLLING_TIME:
+            // }
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/Error/TooManyRequestsException.cs
similarity index 64%
copy from csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
copy to csharp/rocketmq-client-csharp/Error/TooManyRequestsException.cs
index edd810dd..c8f0655b 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/Error/TooManyRequestsException.cs
@@ -14,23 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class StaticCredentialsProvider : ICredentialsProvider
+    /// <summary>
+    /// Generic exception indicates that too many requests are made in short period of duration.
+    /// </summary>
+    public class TooManyRequestsException : ClientException
     {
-
-        public StaticCredentialsProvider(string accessKey, string accessSecret)
+        public TooManyRequestsException(int responseCode, string requestId, string message) : base(responseCode,
+            requestId, message)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
         }
 
-        public Credentials getCredentials()
+        public TooManyRequestsException(int responseCode, string message) : base(responseCode, message)
         {
-            return new Credentials(accessKey, accessSecret);
         }
-
-        private string accessKey;
-        private string accessSecret;
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/Error/UnauthorizedException.cs
similarity index 62%
copy from csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
copy to csharp/rocketmq-client-csharp/Error/UnauthorizedException.cs
index edd810dd..d12ad6d5 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/Error/UnauthorizedException.cs
@@ -14,23 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace Org.Apache.Rocketmq
+
+namespace Org.Apache.Rocketmq.Error
 {
-    public class StaticCredentialsProvider : ICredentialsProvider
+    /// <summary>
+    /// Generic exception indicates that the client request lacks valid authentication credentials
+    /// for the requested resource.
+    /// </summary>
+    public class UnauthorizedException : ClientException
     {
-
-        public StaticCredentialsProvider(string accessKey, string accessSecret)
+        public UnauthorizedException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
         }
 
-        public Credentials getCredentials()
+        public UnauthorizedException(int responseCode, string message) : base(responseCode, message)
         {
-            return new Credentials(accessKey, accessSecret);
         }
-
-        private string accessKey;
-        private string accessSecret;
     }
 }
\ No newline at end of file
diff --git a/csharp/tests/ClientConfigTest.cs b/csharp/rocketmq-client-csharp/Error/UnsupportedException.cs
similarity index 65%
rename from csharp/tests/ClientConfigTest.cs
rename to csharp/rocketmq-client-csharp/Error/UnsupportedException.cs
index 4d8dec18..e39b40da 100644
--- a/csharp/tests/ClientConfigTest.cs
+++ b/csharp/rocketmq-client-csharp/Error/UnsupportedException.cs
@@ -14,21 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using System;
 
-namespace Org.Apache.Rocketmq
+namespace Org.Apache.Rocketmq.Error
 {
-    [TestClass]
-    public class ClientConfigTest
+    /// <summary>
+    /// Generic exception for unsupported operation.
+    /// </summary>
+    public class UnsupportedException : ClientException
     {
-        [TestMethod]
-        public void testClientId()
+        public UnsupportedException(int responseCode, string requestId, string message) : base(responseCode, requestId,
+            message)
+        {
+        }
+
+        public UnsupportedException(int responseCode, string message) : base(responseCode, message)
         {
-            var clientConfig = new ClientConfig();
-            string clientId = clientConfig.clientId();
-            Assert.IsTrue(clientId.Contains("@"));
-            Assert.IsTrue(clientId.Contains("#default"));
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
new file mode 100644
index 00000000..e987d979
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
@@ -0,0 +1,56 @@
+using System;
+using Apache.Rocketmq.V2;
+using Google.Protobuf.WellKnownTypes;
+
+namespace Org.Apache.Rocketmq
+{
+    public class ExponentialBackoffRetryPolicy : RetryPolicy
+    {
+        private int _maxAttempts;
+
+        public ExponentialBackoffRetryPolicy(int maxAttempts, TimeSpan initialBackoff, TimeSpan maxBackoff,
+            double backoffMultiplier)
+        {
+            _maxAttempts = maxAttempts;
+            InitialBackoff = initialBackoff;
+            MaxBackoff = maxBackoff;
+            BackoffMultiplier = backoffMultiplier;
+        }
+
+        public int getMaxAttempts()
+        {
+            return _maxAttempts;
+        }
+
+        public TimeSpan InitialBackoff { get; }
+
+        public TimeSpan MaxBackoff { get; }
+
+        public double BackoffMultiplier { get; }
+
+        public TimeSpan getNextAttemptDelay(int attempt)
+        {
+            return TimeSpan.Zero;
+        }
+
+        public static ExponentialBackoffRetryPolicy immediatelyRetryPolicy(int maxAttempts)
+        {
+            return new ExponentialBackoffRetryPolicy(maxAttempts, TimeSpan.Zero, TimeSpan.Zero, 1);
+        }
+
+        public global::Apache.Rocketmq.V2.RetryPolicy toProtobuf()
+        {
+            var exponentialBackoff = new ExponentialBackoff
+            {
+                Multiplier = (float)BackoffMultiplier,
+                Max = Duration.FromTimeSpan(MaxBackoff),
+                Initial = Duration.FromTimeSpan(InitialBackoff)
+            };
+            return new global::Apache.Rocketmq.V2.RetryPolicy
+            {
+                MaxAttempts = _maxAttempts,
+                ExponentialBackoff = exponentialBackoff
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ExpressionType.cs b/csharp/rocketmq-client-csharp/ExpressionType.cs
index 0caaf8e5..138a1a51 100644
--- a/csharp/rocketmq-client-csharp/ExpressionType.cs
+++ b/csharp/rocketmq-client-csharp/ExpressionType.cs
@@ -14,13 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 namespace Org.Apache.Rocketmq
 {
-
     public enum ExpressionType
     {
-        TAG,
-        SQL92,
+        Tag,
+        Sql92,
     }
-
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClient.cs b/csharp/rocketmq-client-csharp/IClient.cs
index b1e992a8..db219af9 100644
--- a/csharp/rocketmq-client-csharp/IClient.cs
+++ b/csharp/rocketmq-client-csharp/IClient.cs
@@ -15,23 +15,32 @@
  * limitations under the License.
  */
 
-using System.Threading.Tasks;
 using System.Threading;
-using System;
-using rmq = Apache.Rocketmq.V2;
+using Grpc.Core;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
-    public interface IClient : IClientConfig
+    public interface IClient
     {
+        void Heartbeat();
 
-        Task Heartbeat();
+        void NotifyClientTermination(Proto.Resource group);
 
-        Task<bool> NotifyClientTermination(rmq.Resource group);
+        CancellationTokenSource TelemetryCts();
 
-        void BuildClientSetting(rmq::Settings settings);
-        
+        Proto.Settings GetSettings();
 
-        CancellationTokenSource TelemetryCts();
+        string GetClientId();
+
+        void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings);
+
+        void OnRecoverOrphanedTransactionCommand(Endpoints endpoints, Proto.RecoverOrphanedTransactionCommand command);
+
+        void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command);
+
+        void OnPrintThreadStackTraceCommand(Endpoints endpoints, Proto.PrintThreadStackTraceCommand command);
+
+        Metadata Sign();
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClientConfig.cs b/csharp/rocketmq-client-csharp/IClientConfig.cs
index 57325b40..5603a616 100644
--- a/csharp/rocketmq-client-csharp/IClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/IClientConfig.cs
@@ -14,24 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 
 namespace Org.Apache.Rocketmq
 {
     public interface IClientConfig
     {
-        string region();
-
-        string serviceName();
-
-        string resourceNamespace();
-
-        ICredentialsProvider credentialsProvider();
-
-        string getGroupName();
 
-        string clientId();
+        ICredentialsProvider CredentialsProvider
+        {
+            get;
+        }
 
-        bool isTracingEnabled();
+        string ClientId
+        {
+            get;
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClientManager.cs b/csharp/rocketmq-client-csharp/IClientManager.cs
index afccfde0..beb8880b 100644
--- a/csharp/rocketmq-client-csharp/IClientManager.cs
+++ b/csharp/rocketmq-client-csharp/IClientManager.cs
@@ -18,7 +18,8 @@
 using System.Threading.Tasks;
 using System;
 using System.Collections.Generic;
-using grpc = global::Grpc.Core;
+using Apache.Rocketmq.V2;
+using grpc = Grpc.Core;
 using rmq = Apache.Rocketmq.V2;
 
 
@@ -26,25 +27,28 @@ namespace Org.Apache.Rocketmq
 {
     public interface IClientManager
     {
-        IRpcClient GetRpcClient(string target);
+        grpc::AsyncDuplexStreamingCall<TelemetryCommand, TelemetryCommand> Telemetry(Endpoints endpoints);
 
-        grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> Telemetry(string target, grpc::Metadata metadata);
+        Task<QueryRouteResponse> QueryRoute(Endpoints endpoints, QueryRouteRequest request, TimeSpan timeout);
 
-        Task<TopicRouteData> ResolveRoute(string target, grpc::Metadata metadata, rmq::QueryRouteRequest request, TimeSpan timeout);
+        Task<HeartbeatResponse> Heartbeat(Endpoints endpoints, HeartbeatRequest request, TimeSpan timeout);
 
-        Task<Boolean> Heartbeat(string target, grpc::Metadata metadata, rmq::HeartbeatRequest request, TimeSpan timeout);
+        Task<NotifyClientTerminationResponse> NotifyClientTermination(Endpoints endpoints,
+            NotifyClientTerminationRequest request, TimeSpan timeout);
 
-        Task<Boolean> NotifyClientTermination(string target, grpc::Metadata metadata, rmq::NotifyClientTerminationRequest request, TimeSpan timeout);
+        Task<SendMessageResponse> SendMessage(Endpoints endpoints, SendMessageRequest request,
+            TimeSpan timeout);
 
-        Task<rmq::SendMessageResponse> SendMessage(string target, grpc::Metadata metadata, rmq::SendMessageRequest request, TimeSpan timeout);
+        Task<QueryAssignmentResponse> QueryAssignment(Endpoints endpoints, QueryAssignmentRequest request,
+            TimeSpan timeout);
 
-        Task<List<rmq::Assignment>> QueryLoadAssignment(string target, grpc::Metadata metadata, rmq::QueryAssignmentRequest request, TimeSpan timeout);
+        Task<List<ReceiveMessageResponse>> ReceiveMessage(Endpoints endpoints, ReceiveMessageRequest request,
+            TimeSpan timeout);
 
-        Task<List<Message>> ReceiveMessage(string target, grpc::Metadata metadata, rmq::ReceiveMessageRequest request, TimeSpan timeout);
+        Task<AckMessageResponse> AckMessage(Endpoints endpoints, AckMessageRequest request, TimeSpan timeout);
 
-        Task<Boolean> Ack(string target, grpc::Metadata metadata, rmq::AckMessageRequest request, TimeSpan timeout);
-
-        Task<Boolean> ChangeInvisibleDuration(string target, grpc::Metadata metadata, rmq::ChangeInvisibleDurationRequest request, TimeSpan timeout);
+        Task<ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Endpoints endpoints,
+            ChangeInvisibleDurationRequest request, TimeSpan timeout);
 
         Task Shutdown();
     }
diff --git a/csharp/rocketmq-client-csharp/ICredentialsProvider.cs b/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
index 1fb892bd..2f6e71eb 100644
--- a/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
@@ -18,6 +18,9 @@ namespace Org.Apache.Rocketmq
 {
     public interface ICredentialsProvider
     {
-        Credentials getCredentials();
+        Credentials Credentials
+        {
+            get;
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Message.cs b/csharp/rocketmq-client-csharp/Message.cs
index b527311a..fb004da3 100644
--- a/csharp/rocketmq-client-csharp/Message.cs
+++ b/csharp/rocketmq-client-csharp/Message.cs
@@ -35,22 +35,15 @@ namespace Org.Apache.Rocketmq
 
         public Message(string topic, string tag, List<string> keys, byte[] body)
         {
-            MessageId = SequenceGenerator.Instance.Next();
             MaxAttemptTimes = 3;
             Topic = topic;
             Tag = tag;
             Keys = keys;
             Body = body;
             UserProperties = new Dictionary<string, string>();
-            DeliveryTimestamp = DateTime.MinValue;
+            DeliveryTimestamp = null;
         }
 
-        public string MessageId
-        {
-            get;
-            internal set;
-        }
-        
         public string Topic
         {
             get;
@@ -74,7 +67,6 @@ namespace Org.Apache.Rocketmq
             get;
             set;
         }
-
         public Dictionary<string, string> UserProperties
         {
             get;
@@ -88,7 +80,7 @@ namespace Org.Apache.Rocketmq
         }
 
 
-        public DateTime DeliveryTimestamp
+        public DateTime? DeliveryTimestamp
         {
             get;
             set;
diff --git a/csharp/rocketmq-client-csharp/MessageQueue.cs b/csharp/rocketmq-client-csharp/MessageQueue.cs
new file mode 100644
index 00000000..b7a6f922
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/MessageQueue.cs
@@ -0,0 +1,41 @@
+using System.Collections.Generic;
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class MessageQueue
+    {
+        public MessageQueue(rmq::MessageQueue messageQueue)
+        {
+            TopicResource = new Resource(messageQueue.Topic);
+            QueueId = messageQueue.Id;
+            Permission = PermissionHelper.FromProtobuf(messageQueue.Permission);
+            var messageTypes = new List<MessageType>();
+            foreach (var acceptMessageType in messageQueue.AcceptMessageTypes)
+            {
+                var messageType = MessageTypeHelper.FromProtobuf(acceptMessageType);
+                messageTypes.Add(messageType);
+            }
+
+            AcceptMessageTypes = messageTypes;
+            Broker = new Broker(messageQueue.Broker);
+        }
+
+        public Broker Broker { get; }
+
+        public Resource TopicResource { get; }
+
+        public Permission Permission { get; }
+
+        public int QueueId { get; }
+
+        public List<MessageType> AcceptMessageTypes { get; }
+
+        public string Topic
+        {
+            get { return TopicResource.Name; }
+        }
+        
+        
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageType.cs b/csharp/rocketmq-client-csharp/MessageType.cs
index a459e932..6338e365 100644
--- a/csharp/rocketmq-client-csharp/MessageType.cs
+++ b/csharp/rocketmq-client-csharp/MessageType.cs
@@ -15,6 +15,9 @@
  * limitations under the License.
  */
 
+using Org.Apache.Rocketmq.Error;
+using rmq = Apache.Rocketmq.V2;
+
 namespace Org.Apache.Rocketmq
 {
     public enum MessageType
@@ -22,6 +25,43 @@ namespace Org.Apache.Rocketmq
         Normal,
         Fifo,
         Delay,
-        Transaction,
+        Transaction
+    }
+
+    public static class MessageTypeHelper
+    {
+        public static MessageType FromProtobuf(rmq.MessageType messageType)
+        {
+            switch (messageType)
+            {
+                case rmq.MessageType.Normal:
+                    return MessageType.Normal;
+                case rmq.MessageType.Fifo:
+                    return MessageType.Fifo;
+                case rmq.MessageType.Delay:
+                    return MessageType.Delay;
+                case rmq.MessageType.Transaction:
+                    return MessageType.Transaction;
+                default:
+                    throw new InternalErrorException("MessageType is not specified");
+            }
+        }
+
+        public static rmq.MessageType ToProtobuf(MessageType messageType)
+        {
+            switch (messageType)
+            {
+                case MessageType.Normal:
+                    return rmq.MessageType.Normal;
+                case MessageType.Fifo:
+                    return rmq.MessageType.Fifo;
+                case MessageType.Delay:
+                    return rmq.MessageType.Delay;
+                case MessageType.Transaction:
+                    return rmq.MessageType.Transaction;
+                default:
+                    return rmq.MessageType.Unspecified;
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
new file mode 100644
index 00000000..57b573ac
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using rmq = Apache.Rocketmq.V2;
+using System;
+using System.Collections.Generic;
+using System.Security.Cryptography;
+using NLog;
+
+namespace Org.Apache.Rocketmq
+{
+    /// <summary>
+    /// Provides a read-only view for message.
+    /// </summary>
+    public class MessageView
+    {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
+        private readonly rmq.MessageQueue _messageQueue;
+        private readonly string _receiptHandle;
+        private readonly long _offset;
+        private readonly bool _corrupted;
+
+        internal MessageView(string messageId, string topic, byte[] body, string tag, string messageGroup,
+            DateTime deliveryTime, List<string> keys, Dictionary<string, string> properties, string bornHost,
+            DateTime bornTime, int deliveryAttempt, rmq.MessageQueue messageQueue, string receiptHandle, long offset,
+            bool corrupted)
+        {
+            MessageId = messageId;
+            Topic = topic;
+            Body = body;
+            Tag = tag;
+            MessageGroup = messageGroup;
+            DeliveryTime = deliveryTime;
+            Keys = keys;
+            Properties = properties;
+            BornHost = bornHost;
+            BornTime = bornTime;
+            DeliveryAttempt = deliveryAttempt;
+            _messageQueue = messageQueue;
+            _receiptHandle = receiptHandle;
+            _offset = offset;
+            _corrupted = corrupted;
+        }
+
+        public string MessageId { get; }
+
+        public string Topic { get; }
+
+        public byte[] Body { get; }
+
+        public string Tag { get; }
+
+        public string MessageGroup { get; }
+
+        public DateTime DeliveryTime { get; }
+
+        public List<string> Keys { get; }
+
+        public Dictionary<string, string> Properties { get; }
+
+        public string BornHost { get; }
+
+        public DateTime BornTime { get; }
+
+        public int DeliveryAttempt { get; }
+
+        public static MessageView fromProtobuf(rmq.Message message, rmq.MessageQueue messageQueue)
+        {
+            var topic = message.Topic.Name;
+            var systemProperties = message.SystemProperties;
+            var messageId = systemProperties.MessageId;
+            var bodyDigest = systemProperties.BodyDigest;
+            var checkSum = bodyDigest.Checksum;
+            var raw = message.Body.ToByteArray();
+            bool corrupted = false;
+            var type = bodyDigest.Type;
+            switch (type)
+            {
+                case rmq.DigestType.Crc32:
+                {
+                    var expectedCheckSum = Force.Crc32.Crc32Algorithm.Compute(raw, 0, raw.Length).ToString("X");
+                    if (!expectedCheckSum.Equals(checkSum))
+                    {
+                        corrupted = true;
+                    }
+
+                    break;
+                }
+                case rmq.DigestType.Md5:
+                {
+                    var expectedCheckSum = Convert.ToHexString(MD5.HashData(raw));
+                    if (!expectedCheckSum.Equals(checkSum))
+                    {
+                        corrupted = true;
+                    }
+
+                    break;
+                }
+                case rmq.DigestType.Sha1:
+                {
+                    var expectedCheckSum = Convert.ToHexString(SHA1.HashData(raw));
+                    if (!expectedCheckSum.Equals(checkSum))
+                    {
+                        corrupted = true;
+                    }
+
+                    break;
+                }
+                default:
+                {
+                    Logger.Error(
+                        $"Unsupported message body digest algorithm," +
+                        $"digestType={type}, topic={topic}, messageId={messageId}");
+                    break;
+                }
+            }
+
+            var bodyEncoding = systemProperties.BodyEncoding;
+            byte[] body = raw;
+            switch (bodyEncoding)
+            {
+                case rmq.Encoding.Gzip:
+                {
+                    body = Utilities.uncompressBytesGzip(message.Body.ToByteArray());
+                    break;
+                }
+                case rmq.Encoding.Identity:
+                {
+                    break;
+                }
+                default:
+                {
+                    Logger.Error($"Unsupported message encoding algorithm," +
+                                 $" topic={topic}, messageId={messageId}, bodyEncoding={bodyEncoding}");
+                    break;
+                }
+            }
+
+            string tag = systemProperties.HasTag ? systemProperties.Tag : null;
+            string messageGroup = systemProperties.HasMessageGroup ? systemProperties.MessageGroup : null;
+            var deliveryTime = systemProperties.DeliveryTimestamp.ToDateTime();
+            List<string> keys = new List<string>();
+            foreach (var key in systemProperties.Keys)
+            {
+                keys.Add(key);
+            }
+
+            var bornHost = systemProperties.BornHost;
+            var bornTime = systemProperties.BornTimestamp.ToDateTime();
+            var deliveryAttempt = systemProperties.DeliveryAttempt;
+            var queueOffset = systemProperties.QueueOffset;
+            Dictionary<string, string> properties = new Dictionary<string, string>();
+            foreach (var (key, value) in message.UserProperties)
+            {
+                properties.Add(key, value);
+            }
+
+            var receiptHandle = systemProperties.ReceiptHandle;
+            return new MessageView(messageId, topic, body, tag, messageGroup, deliveryTime, keys, properties, bornHost,
+                bornTime, deliveryAttempt, messageQueue, receiptHandle, queueOffset, corrupted);
+        }
+
+        public override string ToString()
+        {
+            return
+                $"{nameof(MessageId)}: {MessageId}, {nameof(Topic)}: {Topic}, {nameof(Tag)}: {Tag}," +
+                $" {nameof(MessageGroup)}: {MessageGroup}, {nameof(DeliveryTime)}: {DeliveryTime}," +
+                $" {nameof(Keys)}: {Keys}, {nameof(Properties)}: {Properties}, {nameof(BornHost)}: {BornHost}, " +
+                $"{nameof(BornTime)}: {BornTime}, {nameof(DeliveryAttempt)}: {DeliveryAttempt}";
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MetadataConstants.cs b/csharp/rocketmq-client-csharp/MetadataConstants.cs
index d61f0294..07907758 100644
--- a/csharp/rocketmq-client-csharp/MetadataConstants.cs
+++ b/csharp/rocketmq-client-csharp/MetadataConstants.cs
@@ -15,33 +15,55 @@
  * limitations under the License.
  */
 
-using System;
 using System.Reflection;
 
 namespace Org.Apache.Rocketmq
 {
     public class MetadataConstants
     {
-        public const string TENANT_ID_KEY = "x-mq-tenant-id";
-        public const string NAMESPACE_KEY = "x-mq-namespace";
-        public const string AUTHORIZATION = "authorization";
-        public const string STS_SESSION_TOKEN = "x-mq-session-token";
-        public const string DATE_TIME_KEY = "x-mq-date-time";
-        public const string ALGORITHM_KEY = "MQv2-HMAC-SHA1";
-        public const string CREDENTIAL_KEY = "Credential";
-        public const string SIGNED_HEADERS_KEY = "SignedHeaders";
-        public const string SIGNATURE_KEY = "Signature";
-        public const string DATE_TIME_FORMAT = "yyyyMMddTHHmmssZ";
-        public const string LANGUAGE_KEY = "x-mq-language";
-        public const string CLIENT_VERSION_KEY = "x-mq-client-version";
-        public const string PROTOCOL_VERSION_KEY = "x-mq-protocol-version";
-        public const string REQUEST_ID_KEY = "x-mq-request-id";
-
-        public const string CLIENT_ID_KEY = "x-mq-client-id";
-
-        public static readonly string CLIENT_VERSION = Assembly
-            .GetAssembly(typeof(MetadataConstants))
-            .GetCustomAttribute<AssemblyInformationalVersionAttribute>()
-            .InformationalVersion;
+        public const string NamespaceKey = "x-mq-namespace";
+        public const string SessionTokenKey = "x-mq-session-token";
+        public const string DateTimeKey = "x-mq-date-time";
+        public const string LanguageKey = "x-mq-language";
+        public const string ClientVersionKey = "x-mq-client-version";
+        public const string ClientIdKey = "x-mq-client-id";
+        public const string RequestIdKey = "x-mq-request-id";
+        public const string ProtocolVersionKey = "x-mq-protocol-version";
+
+        public const string Authorization = "authorization";
+        public const string AlgorithmKey = "MQv2-HMAC-SHA1";
+        public const string CredentialKey = "Credential";
+        public const string SignedHeadersKey = "SignedHeaders";
+        public const string SignatureKey = "Signature";
+        public const string DateTimeFormat = "yyyyMMddTHHmmssZ";
+        public const string LanguageValue = "DOTNET";
+
+        private const string UnknownVersion = "unknown";
+
+        public string ClientVersion { get; }
+
+
+        public static readonly MetadataConstants Instance = new();
+
+        private MetadataConstants()
+        {
+            var assembly = Assembly.GetAssembly(typeof(MetadataConstants));
+            if (null == assembly)
+            {
+                ClientVersion = UnknownVersion;
+                return;
+            }
+
+            var assemblyInformationalVersionAttribute =
+                assembly.GetCustomAttribute<AssemblyInformationalVersionAttribute>();
+            if (null == assemblyInformationalVersionAttribute)
+            {
+                ClientVersion = UnknownVersion;
+                return;
+            }
+
+            ClientVersion = assemblyInformationalVersionAttribute
+                .InformationalVersion;
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MqEncoding.cs b/csharp/rocketmq-client-csharp/MqEncoding.cs
new file mode 100644
index 00000000..ba2a489d
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/MqEncoding.cs
@@ -0,0 +1,26 @@
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public enum MqEncoding
+    {
+        Identity,
+        Gzip
+    }
+
+    public static class EncodingHelper
+    {
+        public static rmq.Encoding ToProtobuf(MqEncoding mqEncoding)
+        {
+            switch (mqEncoding)
+            {
+                case MqEncoding.Gzip:
+                    return rmq.Encoding.Gzip;
+                case MqEncoding.Identity:
+                    return rmq.Encoding.Identity;
+                default:
+                    return rmq.Encoding.Unspecified;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Permission.cs b/csharp/rocketmq-client-csharp/Permission.cs
new file mode 100644
index 00000000..d5fe6348
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Permission.cs
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using Org.Apache.Rocketmq.Error;
+using Proto = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public enum Permission
+    {
+        None,
+        Read,
+        Write,
+        ReadWrite
+    }
+
+    public static class PermissionHelper
+    {
+        public static Permission FromProtobuf(Proto.Permission permission)
+        {
+            switch (permission)
+            {
+                case Proto.Permission.Read:
+                    return Permission.Read;
+                case Proto.Permission.Write:
+                    return Permission.Write;
+                case Proto.Permission.ReadWrite:
+                    return Permission.ReadWrite;
+                case Proto.Permission.None:
+                    return Permission.None;
+                default:
+                    throw new InternalErrorException("Permission is not specified");
+            }
+        }
+        
+        public static bool IsWritable(Permission permission) {
+            if (Permission.Write.Equals(permission))
+            {
+                return true;
+            }
+
+            if (Permission.ReadWrite.Equals(permission))
+            {
+                return true;
+            }
+
+            return false;
+        }
+
+        public static bool IsReadable(Permission permission)
+        {
+            if (Permission.Read.Equals(permission))
+            {
+                return true;
+            }
+
+            if (Permission.ReadWrite.Equals(permission))
+            {
+                return true;
+            }
+
+            return false;
+        }
+    }
+    
+
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 69a4b118..9fa10ad3 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -1,224 +1,163 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
 using System;
-using System.Threading.Tasks;
-using rmq = Apache.Rocketmq.V2;
-using System.Collections.Generic;
 using System.Collections.Concurrent;
-using System.Diagnostics;
-using System.Diagnostics.Metrics;
-using Google.Protobuf;
-using Google.Protobuf.WellKnownTypes;
-using Grpc.Core;
-using OpenTelemetry;
-using OpenTelemetry.Exporter;
-using OpenTelemetry.Metrics;
+using System.Collections.Generic;
+using System.Linq;
+using System.Threading.Tasks;
+using Proto = Apache.Rocketmq.V2;
+using NLog;
 
 namespace Org.Apache.Rocketmq
 {
     public class Producer : Client, IProducer
     {
-        public Producer(string accessUrl) : base(accessUrl)
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+        private readonly ConcurrentDictionary<string /* topic */, PublishingLoadBalancer> _publishingRouteDataCache;
+        private readonly PublishingSettings _publishingSettings;
+
+
+        public Producer(ClientConfig clientConfig) : this(clientConfig, new ConcurrentDictionary<string, bool>(), 3)
         {
-            _loadBalancer = new ConcurrentDictionary<string, PublishLoadBalancer>();
-            _sendFailureTotal = MetricMeter.CreateCounter<long>("rocketmq_send_failure_total");
-            _sendLatency = MetricMeter.CreateHistogram<double>(SendLatencyName, 
-                description: "Measure the duration of publishing messages to brokers",
-                unit: "milliseconds");
         }
 
-        public override async Task Start()
+        public Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
+            base(clientConfig, topics)
         {
-            await base.Start();
+            var retryPolicy = ExponentialBackoffRetryPolicy.immediatelyRetryPolicy(maxAttempts);
+            _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
+                clientConfig.RequestTimeout, topics);
+            _publishingRouteDataCache = new ConcurrentDictionary<string, PublishingLoadBalancer>();
+        }
 
-            _meterProvider = Sdk.CreateMeterProviderBuilder()
-                .AddMeter("Apache.RocketMQ.Client")
-                .AddOtlpExporter(delegate(OtlpExporterOptions options, MetricReaderOptions readerOptions)
-                {
-                    options.Protocol = OtlpExportProtocol.Grpc;
-                    options.Endpoint = new Uri(AccessPoint.TargetUrl());
-                    options.TimeoutMilliseconds = (int) ClientSettings.RequestTimeout.ToTimeSpan().TotalMilliseconds;
+        public void SetTopics(params string[] topics)
+        {
+            foreach (var topic in topics)
+            {
+                Topics[topic] = false;
+            }
+        }
 
-                    readerOptions.PeriodicExportingMetricReaderOptions.ExportIntervalMilliseconds = 60 * 1000;
-                })
-                .AddView((instrument) =>
-                {
-                    if (instrument.Meter.Name == MeterName && instrument.Name == SendLatencyName)
-                    {
-                        return new ExplicitBucketHistogramConfiguration()
-                        {
-                            Boundaries = new double[] {1, 5, 10, 20, 50, 200, 500},
-                        };
-                    }
-                    return null;
-                })
-                .Build();
+        public override async Task Start()
+        {
+            Logger.Info($"Begin to start the rocketmq producer, clientId={ClientId}");
+            await base.Start();
+            Logger.Info($"The rocketmq producer starts successfully, clientId={ClientId}");
         }
 
         public override async Task Shutdown()
         {
-            // Release local resources
+            Logger.Info($"Begin to shutdown the rocketmq producer, clientId={ClientId}");
             await base.Shutdown();
+            Logger.Info($"Shutdown the rocketmq producer successfully, clientId={ClientId}");
         }
 
-        protected override void PrepareHeartbeatData(rmq::HeartbeatRequest request)
+        protected override Proto::HeartbeatRequest WrapHeartbeatRequest()
         {
-            request.ClientType = rmq::ClientType.Producer;
-
-            // Concept of ProducerGroup has been removed.
+            return new Proto::HeartbeatRequest
+            {
+                ClientType = Proto.ClientType.Producer
+            };
         }
 
-        public override void BuildClientSetting(rmq::Settings settings)
+        protected override void OnTopicDataFetched0(string topic, TopicRouteData topicRouteData)
         {
-            base.BuildClientSetting(settings);
-
-            settings.ClientType = rmq.ClientType.Producer;
-            var publishing = new rmq.Publishing();
-            
-            foreach (var topic in _topicsOfInterest)
-            {
-                var resource = new rmq.Resource()
-                {
-                    Name = topic.Key,
-                    ResourceNamespace = ResourceNamespace
-                };
-                publishing.Topics.Add(resource);
-            }
+        }
 
-            settings.Publishing = publishing;
+        private RetryPolicy GetRetryPolicy()
+        {
+            return _publishingSettings.GetRetryPolicy();
         }
 
         public async Task<SendReceipt> Send(Message message)
         {
-            _topicsOfInterest.TryAdd(message.Topic, true);
-
-            if (!_loadBalancer.TryGetValue(message.Topic, out var publishLb))
+            if (!_publishingRouteDataCache.TryGetValue(message.Topic, out var publishingLoadBalancer))
             {
-                var topicRouteData = await GetRouteFor(message.Topic, false);
-                if (null == topicRouteData || null == topicRouteData.MessageQueues || 0 == topicRouteData.MessageQueues.Count)
+                var topicRouteData = await FetchTopicRoute(message.Topic);
+                if (null == topicRouteData || null == topicRouteData.MessageQueues ||
+                    0 == topicRouteData.MessageQueues.Count)
                 {
-                    Logger.Error($"Failed to resolve route info for {message.Topic}");
-                    throw new TopicRouteException(string.Format("No topic route for {0}", message.Topic));
+                    throw new TopicRouteException($"No topic route for {message.Topic}");
                 }
 
-                publishLb = new PublishLoadBalancer(topicRouteData);
-                _loadBalancer.TryAdd(message.Topic, publishLb);
+                publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
+                _publishingRouteDataCache.TryAdd(message.Topic, publishingLoadBalancer);
             }
 
-            var request = new rmq::SendMessageRequest();
-            var entry = new rmq::Message
+            var publishingMessage = new PublishingMessage(message, _publishingSettings, false);
+            var retryPolicy = GetRetryPolicy();
+            var maxAttempts = retryPolicy.getMaxAttempts();
+            var candidates = publishingLoadBalancer.TakeMessageQueues(publishingMessage.MessageGroup, maxAttempts);
+            Exception exception = null;
+            for (int attempt = 0; attempt < maxAttempts; attempt++)
             {
-                Body = ByteString.CopyFrom(message.Body),
-                Topic = new rmq::Resource
-                {
-                    ResourceNamespace = resourceNamespace(),
-                    Name = message.Topic
-                },
-                UserProperties = { message.UserProperties },
-                SystemProperties = new rmq::SystemProperties
+                try
                 {
-                    MessageId = message.MessageId,
-                    MessageType = rmq::MessageType.Normal,
-                    Keys = { message.Keys },
-                },
-            };
-            request.Messages.Add(entry);
-
-            if (DateTime.MinValue != message.DeliveryTimestamp)
-            {
-                entry.SystemProperties.MessageType = rmq::MessageType.Delay;
-                entry.SystemProperties.DeliveryTimestamp = Timestamp.FromDateTime(message.DeliveryTimestamp);
-
-                if (message.Fifo())
+                    var sendReceipt = await Send0(publishingMessage, candidates, attempt, maxAttempts);
+                    return sendReceipt;
+                }
+                catch (Exception e)
                 {
-                    Logger.Warn("A message may not be FIFO and delayed at the same time");
-                    throw new MessageException("A message may not be both FIFO and Timed");
+                    exception = e;
                 }
             }
-            else if (!String.IsNullOrEmpty(message.MessageGroup))
+
+            throw exception!;
+        }
+
+        private Proto.SendMessageRequest WrapSendMessageRequest(PublishingMessage message, MessageQueue mq)
+        {
+            return new Proto.SendMessageRequest
             {
-                entry.SystemProperties.MessageType = rmq::MessageType.Fifo;
-                entry.SystemProperties.MessageGroup = message.MessageGroup;
-            }
+                Messages = { message.ToProtobuf(mq.QueueId) }
+            };
+        }
 
-            if (!string.IsNullOrEmpty(message.Tag))
+        private async Task<SendReceipt> Send0(PublishingMessage message, List<MessageQueue> candidates, int attempt,
+            int maxAttempts)
+        {
+            var candidateIndex = (attempt - 1) % candidates.Count;
+            var mq = candidates[candidateIndex];
+            if (_publishingSettings.IsValidateMessageType() &&
+                !mq.AcceptMessageTypes.Contains(message.MessageType))
             {
-                entry.SystemProperties.Tag = message.Tag;
+                throw new ArgumentException($"Current message type does not match with the accept message types," +
+                                            $" topic={message.Topic}, actualMessageType={message.MessageType}" +
+                                            $" acceptMessageType={string.Join(",", mq.AcceptMessageTypes)}");
             }
 
-            var metadata = new Metadata();
-            Signature.Sign(this, metadata);
-
-            Exception ex = null;
-
-            var candidates = publishLb.Select(message.MessageGroup, message.MaxAttemptTimes);
-            foreach (var messageQueue in candidates)
+            var sendMessageRequest = WrapSendMessageRequest(message, mq);
+            var endpoints = mq.Broker.Endpoints;
+            Proto.SendMessageResponse response =
+                await Manager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
+            try
             {
-                var target = Utilities.TargetUrl(messageQueue);
-                entry.SystemProperties.BornTimestamp = Timestamp.FromDateTimeOffset(DateTimeOffset.UtcNow);
-                entry.SystemProperties.QueueId = messageQueue.Id;
+                var sendReceipts = SendReceipt.processSendMessageResponse(response);
 
-                try
+                var sendReceipt = sendReceipts.First();
+                if (attempt > 1)
                 {
-                    var stopWatch = new Stopwatch();
-                    stopWatch.Start();
-                    rmq::SendMessageResponse response = await Manager.SendMessage(target, metadata, request, RequestTimeout);
-                    if (null != response && rmq::Code.Ok == response.Status.Code)
-                    {
-                        var messageId = response.Entries[0].MessageId;
-                        
-                        // Account latency histogram
-                        stopWatch.Stop();
-                        var latency = stopWatch.ElapsedMilliseconds;
-                        _sendLatency.Record(latency, new("topic", message.Topic), new("client_id", clientId()));
-                        
-                        return new SendReceipt(messageId);
-                    }
-                    else if (response?.Status is not null)
-                    {
-                        Logger.Warn($"Send failed with code: {response.Status.Code}, error: {response.Status.Message}");
-                    }
+                    Logger.Info(
+                        $"Re-send message successfully, topic={message.Topic}, messageId={sendReceipt.MessageId}," +
+                        $" maxAttempts={maxAttempts}, endpoints={endpoints}, clientId={ClientId}");
                 }
-                catch (Exception e)
-                {
-                    // Account failure count
-                    _sendFailureTotal.Add(1, new("topic", message.Topic), new("client_id", clientId()));                    
-                    Logger.Info(e, $"Failed to send message to {target}");
-                    ex = e;
-                }
-            }
 
-            if (null != ex)
+                return sendReceipt;
+            }
+            catch (Exception e)
             {
-                Logger.Error(ex, $"Failed to send message after {message.MaxAttemptTimes} attempts");
-                throw ex;
+                Logger.Warn(e, $"Failed to send message, topic={message.Topic}, maxAttempts={maxAttempts}, " +
+                               $"endpoints={endpoints}, clientId={ClientId}");
+                throw;
             }
-
-            Logger.Error($"Failed to send message after {message.MaxAttemptTimes} attempts with unspecified reasons");
-            throw new Exception("Send message failed");
         }
 
-        private readonly ConcurrentDictionary<string, PublishLoadBalancer> _loadBalancer;
-
-        private readonly Counter<long> _sendFailureTotal;
-        private readonly Histogram<double> _sendLatency;
+        public override Proto.Settings GetSettings()
+        {
+            return _publishingSettings.ToProtobuf();
+        }
 
-        private static readonly string SendLatencyName = "rocketmq_send_success_cost_time";
-        private MeterProvider _meterProvider;
+        public override void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
+        {
+            _publishingSettings.Sync(settings);
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/PublishLoadBalancer.cs b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
similarity index 60%
rename from csharp/rocketmq-client-csharp/PublishLoadBalancer.cs
rename to csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
index 6670beb6..c083e23f 100644
--- a/csharp/rocketmq-client-csharp/PublishLoadBalancer.cs
+++ b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
@@ -14,61 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 using System.Collections.Generic;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
-    public class PublishLoadBalancer
+    public class PublishingLoadBalancer
     {
-        public PublishLoadBalancer(TopicRouteData route)
+        private readonly List<MessageQueue> _messageQueues;
+        private int _roundRobinIndex;
+
+        public PublishingLoadBalancer(TopicRouteData route)
         {
-            this._messageQueues = new List<rmq::MessageQueue>();
+            _messageQueues = new List<MessageQueue>();
             foreach (var messageQueue in route.MessageQueues)
             {
-                if (rmq::Permission.Unspecified == messageQueue.Permission)
-                {
-                    continue;
-                }
-
-                if (rmq::Permission.Read == messageQueue.Permission)
+                if (!PermissionHelper.IsWritable(messageQueue.Permission))
                 {
                     continue;
                 }
 
-                this._messageQueues.Add(messageQueue);
+                _messageQueues.Add(messageQueue);
             }
 
-            this._messageQueues.Sort(Utilities.CompareMessageQueue);
             Random random = new Random();
-            this._roundRobinIndex = random.Next(0, this._messageQueues.Count);
-        }
-
-        public void Update(TopicRouteData route)
-        {
-            List<rmq::MessageQueue> partitions = new List<rmq::MessageQueue>();
-            foreach (var partition in route.MessageQueues)
-            {
-                if (rmq::Permission.Unspecified == partition.Permission)
-                {
-                    continue;
-                }
-
-                if (rmq::Permission.Read == partition.Permission)
-                {
-                    continue;
-                }
-                partitions.Add(partition);
-            }
-            partitions.Sort();
-            this._messageQueues = partitions;
+            _roundRobinIndex = random.Next(0, _messageQueues.Count);
         }
 
         /**
          * Accept a partition iff its broker is different.
          */
-        private bool Accept(List<rmq::MessageQueue> existing, rmq::MessageQueue messageQueue)
+        private bool Accept(List<MessageQueue> existing, MessageQueue messageQueue)
         {
             if (0 == existing.Count)
             {
@@ -82,14 +60,15 @@ namespace Org.Apache.Rocketmq
                     return false;
                 }
             }
+
             return true;
         }
 
-        public List<rmq::MessageQueue> Select(string messageGroup, int maxAttemptTimes)
+        public List<MessageQueue> TakeMessageQueues(string messageGroup, int maxAttemptTimes)
         {
-            List<rmq::MessageQueue> result = new List<rmq::MessageQueue>();
+            List<MessageQueue> result = new List<MessageQueue>();
 
-            List<rmq::MessageQueue> all = this._messageQueues;
+            List<MessageQueue> all = _messageQueues;
             if (0 == all.Count)
             {
                 return result;
@@ -119,9 +98,5 @@ namespace Org.Apache.Rocketmq
 
             return result;
         }
-
-        private List<rmq::MessageQueue> _messageQueues;
-
-        private int _roundRobinIndex;
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/PublishingMessage.cs b/csharp/rocketmq-client-csharp/PublishingMessage.cs
new file mode 100644
index 00000000..93eb2de6
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/PublishingMessage.cs
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+using System.IO;
+using Google.Protobuf;
+using Google.Protobuf.WellKnownTypes;
+using rmq = Apache.Rocketmq.V2;
+using Org.Apache.Rocketmq.Error;
+
+namespace Org.Apache.Rocketmq
+{
+    /// <summary>
+    /// Provides a view for message to publish.
+    /// </summary>
+    public class PublishingMessage : Message
+    {
+        public MessageType MessageType { set; get; }
+
+        public String MessageId { get; }
+
+        public PublishingMessage(Message message, PublishingSettings publishingSettings, bool txEnabled) : base(
+            message.Topic, message.Body)
+        {
+            var maxBodySizeBytes = publishingSettings.GetMaxBodySizeBytes();
+            if (message.Body.Length > maxBodySizeBytes)
+            {
+                throw new IOException($"Message body size exceed the threshold, max size={maxBodySizeBytes} bytes");
+            }
+
+            // Generate message id.
+            MessageId = MessageIdGenerator.GetInstance().Next();
+            // For NORMAL message.
+            if (String.IsNullOrEmpty(message.MessageGroup) && !message.DeliveryTimestamp.HasValue &&
+                !txEnabled)
+            {
+                MessageType = MessageType.Normal;
+                return;
+            }
+
+            // For FIFO message.
+            if (!String.IsNullOrEmpty(message.MessageGroup) && !txEnabled)
+            {
+                MessageType = MessageType.Fifo;
+                return;
+            }
+
+            // For DELAY message.
+            if (message.DeliveryTimestamp.HasValue && !txEnabled)
+            {
+                MessageType = MessageType.Delay;
+                return;
+            }
+
+            // For TRANSACTION message.
+            if (!String.IsNullOrEmpty(message.MessageGroup) && !message.DeliveryTimestamp.HasValue && txEnabled)
+            {
+                MessageType = MessageType.Transaction;
+                return;
+            }
+
+            throw new InternalErrorException("Transactional message should not set messageGroup or deliveryTimestamp");
+        }
+
+        public rmq::Message ToProtobuf(int queueId)
+        {
+            rmq.SystemProperties systemProperties = new rmq.SystemProperties
+            {
+                Keys = { Keys },
+                MessageId = MessageId,
+                BornTimestamp = Timestamp.FromDateTime(DateTime.UtcNow),
+                BornHost = Utilities.GetHostName(),
+                BodyEncoding = EncodingHelper.ToProtobuf(MqEncoding.Identity),
+                QueueId = queueId,
+                MessageType = MessageTypeHelper.ToProtobuf(MessageType)
+            };
+            if (null != Tag)
+            {
+                systemProperties.Tag = Tag;
+            }
+
+            if (DeliveryTimestamp.HasValue)
+            {
+                systemProperties.DeliveryTimestamp = Timestamp.FromDateTime(DeliveryTimestamp.Value);
+            }
+
+            if (null != MessageGroup)
+            {
+                systemProperties.MessageGroup = MessageGroup;
+            }
+
+            rmq.Resource topicResource = new rmq.Resource
+            {
+                Name = Topic
+            };
+            return new rmq.Message
+            {
+                Topic = topicResource,
+                Body = ByteString.CopyFrom(Body),
+                SystemProperties = systemProperties,
+                UserProperties = { UserProperties }
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/PublishingSettings.cs b/csharp/rocketmq-client-csharp/PublishingSettings.cs
new file mode 100644
index 00000000..b543cb71
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/PublishingSettings.cs
@@ -0,0 +1,63 @@
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using Google.Protobuf.WellKnownTypes;
+using Proto = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class PublishingSettings : Settings
+    {
+        private volatile int _maxBodySizeBytes = 4 * 1024 * 1024;
+        private volatile bool _validateMessageType = true;
+
+        public PublishingSettings(string clientId, Endpoints accessPoint, ExponentialBackoffRetryPolicy retryPolicy,
+            TimeSpan requestTimeout, ConcurrentDictionary<string, bool> topics) : base(clientId, ClientType.Producer, accessPoint,
+            retryPolicy, requestTimeout)
+        {
+            Topics = topics;
+        }
+
+        public ConcurrentDictionary<string, bool> Topics { get; }
+
+        public int GetMaxBodySizeBytes()
+        {
+            return _maxBodySizeBytes;
+        }
+
+        public bool IsValidateMessageType()
+        {
+            return _validateMessageType;
+        }
+
+        public override void Sync(Proto::Settings settings)
+        {
+            // TODO
+        }
+
+        public override Proto.Settings ToProtobuf()
+        {
+            List<Proto.Resource> topics = new List<Proto.Resource>();
+            foreach (var topic in Topics)
+            {
+                topics.Add(new Proto.Resource
+                {
+                    Name = topic.Key
+                });
+            }
+
+            var publishing = new Proto.Publishing();
+            publishing.Topics.Add(topics);
+            publishing.ValidateMessageType = _validateMessageType;
+            return new Proto.Settings
+            {
+                Publishing = publishing,
+                AccessPoint = AccessPoint.ToProtobuf(),
+                ClientType = ClientTypeHelper.ToProtobuf(ClientType),
+                RequestTimeout = Duration.FromTimeSpan(RequestTimeout),
+                BackoffPolicy = RetryPolicy.toProtobuf(),
+                UserAgent = UserAgent.Instance.ToProtobuf()
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Resource.cs b/csharp/rocketmq-client-csharp/Resource.cs
new file mode 100644
index 00000000..aeca3e4e
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Resource.cs
@@ -0,0 +1,25 @@
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class Resource
+    {
+        public Resource(rmq.Resource resource)
+        {
+            Namespace = resource.ResourceNamespace;
+            Name = resource.Name;
+        }
+
+        public string Namespace { get; }
+        public string Name { get; }
+
+        public rmq.Resource ToProtobuf()
+        {
+            return new rmq.Resource
+            {
+                ResourceNamespace = Namespace,
+                Name = Name
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/RetryPolicy.cs b/csharp/rocketmq-client-csharp/RetryPolicy.cs
new file mode 100644
index 00000000..9169b5d1
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/RetryPolicy.cs
@@ -0,0 +1,13 @@
+using System;
+
+namespace Org.Apache.Rocketmq
+{
+    public interface RetryPolicy
+    {
+        int getMaxAttempts();
+
+        TimeSpan getNextAttemptDelay(int attempt);
+
+        global::Apache.Rocketmq.V2.RetryPolicy toProtobuf();
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/RpcClient.cs b/csharp/rocketmq-client-csharp/RpcClient.cs
index 50cd8e90..de28c184 100644
--- a/csharp/rocketmq-client-csharp/RpcClient.cs
+++ b/csharp/rocketmq-client-csharp/RpcClient.cs
@@ -31,15 +31,15 @@ namespace Org.Apache.Rocketmq
 {
     public class RpcClient : IRpcClient
     {
-        protected static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
         private readonly rmq::MessagingService.MessagingServiceClient _stub;
         private readonly GrpcChannel _channel;
         private readonly string _target;
 
-        public RpcClient(string target)
+        public RpcClient(Endpoints endpoints)
         {
-            _target = target;
-            _channel = GrpcChannel.ForAddress(target, new GrpcChannelOptions
+            _target = endpoints.GrpcTarget;
+            _channel = GrpcChannel.ForAddress(_target, new GrpcChannelOptions
             {
                 HttpHandler = CreateHttpHandler()
             });
@@ -68,8 +68,6 @@ namespace Org.Apache.Rocketmq
             var handler = new SocketsHttpHandler
             {
                 PooledConnectionIdleTimeout = Timeout.InfiniteTimeSpan,
-                KeepAlivePingDelay = TimeSpan.FromSeconds(60),
-                KeepAlivePingTimeout = TimeSpan.FromSeconds(30),
                 EnableMultipleHttp2Connections = true,
                 SslOptions = sslOptions,
             };
@@ -83,7 +81,8 @@ namespace Org.Apache.Rocketmq
             return _stub.Telemetry(callOptions);
         }
 
-        public async Task<rmq::QueryRouteResponse> QueryRoute(Metadata metadata, rmq::QueryRouteRequest request, TimeSpan timeout)
+        public async Task<rmq::QueryRouteResponse> QueryRoute(Metadata metadata, rmq::QueryRouteRequest request,
+            TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
@@ -93,7 +92,8 @@ namespace Org.Apache.Rocketmq
         }
 
 
-        public async Task<rmq::HeartbeatResponse> Heartbeat(Metadata metadata, rmq::HeartbeatRequest request, TimeSpan timeout)
+        public async Task<rmq::HeartbeatResponse> Heartbeat(Metadata metadata, rmq::HeartbeatRequest request,
+            TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
@@ -112,7 +112,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::QueryAssignmentResponse> QueryAssignment(Metadata metadata, rmq::QueryAssignmentRequest request,
+        public async Task<rmq::QueryAssignmentResponse> QueryAssignment(Metadata metadata,
+            rmq::QueryAssignmentRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -122,8 +123,9 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<List<rmq::ReceiveMessageResponse>> ReceiveMessage(Metadata metadata, 
-            rmq::ReceiveMessageRequest request, TimeSpan timeout) {
+        public async Task<List<rmq::ReceiveMessageResponse>> ReceiveMessage(Metadata metadata,
+            rmq::ReceiveMessageRequest request, TimeSpan timeout)
+        {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
             var call = _stub.ReceiveMessage(request, callOptions);
@@ -136,6 +138,7 @@ namespace Org.Apache.Rocketmq
                 Logger.Debug($"Got ReceiveMessageResponse {entry} from {_target}");
                 result.Add(entry);
             }
+
             Logger.Debug($"Receiving messages from {_target} completed");
             return result;
         }
@@ -150,7 +153,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Metadata metadata, rmq::ChangeInvisibleDurationRequest request,
+        public async Task<rmq::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Metadata metadata,
+            rmq::ChangeInvisibleDurationRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -160,7 +164,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::ForwardMessageToDeadLetterQueueResponse> ForwardMessageToDeadLetterQueue(Metadata metadata,
+        public async Task<rmq::ForwardMessageToDeadLetterQueueResponse> ForwardMessageToDeadLetterQueue(
+            Metadata metadata,
             rmq::ForwardMessageToDeadLetterQueueRequest request, TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -170,7 +175,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::EndTransactionResponse> EndTransaction(Metadata metadata, rmq::EndTransactionRequest request,
+        public async Task<rmq::EndTransactionResponse> EndTransaction(Metadata metadata,
+            rmq::EndTransactionRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
diff --git a/csharp/rocketmq-client-csharp/SendReceipt.cs b/csharp/rocketmq-client-csharp/SendReceipt.cs
index 0f29991b..a4ff1e3e 100644
--- a/csharp/rocketmq-client-csharp/SendReceipt.cs
+++ b/csharp/rocketmq-client-csharp/SendReceipt.cs
@@ -15,35 +15,45 @@
  * limitations under the License.
  */
 
+using System.Collections.Generic;
+using rmq = Apache.Rocketmq.V2;
+
 namespace Org.Apache.Rocketmq
 {
     public sealed class SendReceipt
     {
         public SendReceipt(string messageId)
         {
-            status_ = SendStatus.SEND_OK;
-            messageId_ = messageId;
-        }
-
-        public SendReceipt(string messageId, SendStatus status)
-        {
-            status_ = status;
-            messageId_ = messageId;
+            MessageId = messageId;
         }
 
-        private string messageId_;
+        public string MessageId { get; }
 
-        public string MessageId
+        public override string ToString()
         {
-            get { return messageId_; }
+            return $"{nameof(MessageId)}: {MessageId}";
         }
 
+        public static List<SendReceipt> processSendMessageResponse(rmq.SendMessageResponse response)
+        {
+            rmq.Status status = response.Status;
+            foreach (var entry in response.Entries)
+            {
+                if (rmq.Code.Ok.Equals(entry.Status.Code))
+                {
+                    status = entry.Status;
+                }
+            }
 
-        private SendStatus status_;
+            // May throw exception.
+            StatusChecker.Check(status, response);
+            List<SendReceipt> sendReceipts = new List<SendReceipt>();
+            foreach (var entry in response.Entries)
+            {
+                sendReceipts.Add(new SendReceipt(entry.MessageId));
+            }
 
-        public SendStatus Status
-        {
-            get { return status_; }
+            return sendReceipts;
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SendStatus.cs b/csharp/rocketmq-client-csharp/SendStatus.cs
deleted file mode 100644
index 7586d22c..00000000
--- a/csharp/rocketmq-client-csharp/SendStatus.cs
+++ /dev/null
@@ -1,27 +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.
- */
-
-namespace Org.Apache.Rocketmq
-{
-    public enum SendStatus
-    {
-        SEND_OK,
-        FLUSH_DISK_TIMEOUT,
-        FLUSH_SLAVE_TIMEOUT,
-        SLAVE_NOT_AVAILABLE,
-    }
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index 4d098949..82f4f1ef 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -15,104 +15,133 @@
  * limitations under the License.
  */
 
+using System;
 using System.Threading;
 using System.Threading.Channels;
 using System.Threading.Tasks;
+using Grpc.Core;
 using grpc = Grpc.Core;
 using NLog;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
+    // refer to  https://learn.microsoft.com/en-us/aspnet/core/grpc/client?view=aspnetcore-7.0#bi-directional-streaming-call.
     public class Session
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        public Session(string target, 
-            grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> stream,
+        private readonly grpc::AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand>
+            _streamingCall;
+
+        private readonly Client _client;
+        private readonly Channel<bool> _channel;
+        private readonly Endpoints _endpoints;
+        private readonly SemaphoreSlim _semaphore;
+
+        public Session(Endpoints endpoints,
+            AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand> streamingCall,
             Client client)
         {
-            _target = target;
-            _stream = stream;
+            _endpoints = endpoints;
+            _semaphore = new SemaphoreSlim(1);
+            _streamingCall = streamingCall;
             _client = client;
-            _channel = Channel.CreateUnbounded<bool>();
+            _channel = Channel.CreateBounded<bool>(new BoundedChannelOptions(1)
+            {
+                FullMode = BoundedChannelFullMode.DropOldest
+            });
+            Loop();
         }
 
-        public async Task Loop()
+
+        public async Task SyncSettings(bool awaitResp)
         {
-            var reader = _stream.ResponseStream;
-            var writer = _stream.RequestStream;
-            var request = new rmq::TelemetryCommand
+            await _semaphore.WaitAsync();
+            try
             {
-                Settings = new rmq::Settings()
-            };
-            _client.BuildClientSetting(request.Settings);
-            await writer.WriteAsync(request);
-            Logger.Debug($"Writing Client Settings to {_target} Done: {request.Settings}");
-            while (!_client.TelemetryCts().IsCancellationRequested)
-            {
-                if (await reader.MoveNext(_client.TelemetryCts().Token))
+                var writer = _streamingCall.RequestStream;
+                // await readTask;
+                var settings = _client.GetSettings();
+                Proto.TelemetryCommand telemetryCommand = new Proto.TelemetryCommand
                 {
-                    var cmd = reader.Current;
-                    Logger.Debug($"Received a TelemetryCommand from {_target}: {cmd}");
-                    switch (cmd.CommandCase)
-                    {
-                        case rmq::TelemetryCommand.CommandOneofCase.None:
-                            {
-                                Logger.Warn($"Telemetry failed: {cmd.Status}");
-                                if (0 == Interlocked.CompareExchange(ref _established, 0, 2))
-                                {
-                                    await _channel.Writer.WriteAsync(false);
-                                }
-                                break;
-                            }
-                        case rmq::TelemetryCommand.CommandOneofCase.Settings:
-                            {
-                                if (0 == Interlocked.CompareExchange(ref _established, 0, 1))
-                                {
-                                    await _channel.Writer.WriteAsync(true);
-                                }
-
-                                Logger.Info($"Received settings from {_target}: {cmd.Settings}");
-                                _client.OnSettingsReceived(cmd.Settings);
-                                break;
-                            }
-                        case rmq::TelemetryCommand.CommandOneofCase.PrintThreadStackTraceCommand:
-                            {
-                                break;
-                            }
-                        case rmq::TelemetryCommand.CommandOneofCase.RecoverOrphanedTransactionCommand:
-                            {
-                                break;
-                            }
-                        case rmq::TelemetryCommand.CommandOneofCase.VerifyMessageCommand:
-                            {
-                                break;
-                            }
-                    }
+                    Settings = settings
+                };
+                await writer.WriteAsync(telemetryCommand);
+                // await writer.CompleteAsync();
+                if (awaitResp)
+                {
+                    await _channel.Reader.ReadAsync();
                 }
             }
-            Logger.Info($"Telemetry stream for {_target} is cancelled");
-            await writer.CompleteAsync();
-        }
-
-        public async Task AwaitSettingNegotiationCompletion()
-        {
-            if (0 != Interlocked.Read(ref _established))
+            finally
             {
-                return;
+                _semaphore.Release();
             }
-
-            Logger.Debug("Await setting negotiation");
-            await _channel.Reader.ReadAsync();
         }
 
-        private readonly grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> _stream;
-        private readonly Client _client;
+        // public async void xx()
+        // {
+        //     while (true)
+        //     {
+        //         var reader = _streamingCall.ResponseStream;
+        //         if (await reader.MoveNext(_client.TelemetryCts().Token))
+        //         {
+        //             var command = reader.Current;
+        //             Console.WriteLine("xxxxxxxx");
+        //             Console.WriteLine(command);
+        //         }
+        //     }
+        // }
 
-        private long _established;
 
-        private readonly Channel<bool> _channel;
-        private readonly string _target;
+        private void Loop()
+        {
+            Task.Run(async () =>
+            {
+                await foreach (var response in _streamingCall.ResponseStream.ReadAllAsync())
+                {
+                    switch (response.CommandCase)
+                    {
+                        case Proto.TelemetryCommand.CommandOneofCase.Settings:
+                        {
+                            Logger.Info(
+                                $"Receive setting from remote, endpoints={_endpoints}, clientId={_client.GetClientId()}");
+                            await _channel.Writer.WriteAsync(true);
+                            _client.OnSettingsCommand(_endpoints, response.Settings);
+                            break;
+                        }
+                        case Proto.TelemetryCommand.CommandOneofCase.RecoverOrphanedTransactionCommand:
+                        {
+                            Logger.Info(
+                                $"Receive orphaned transaction recovery command from remote, endpoints={_endpoints}, clientId={_client.GetClientId()}");
+                            _client.OnRecoverOrphanedTransactionCommand(_endpoints,
+                                response.RecoverOrphanedTransactionCommand);
+                            break;
+                        }
+                        case Proto.TelemetryCommand.CommandOneofCase.VerifyMessageCommand:
+                        {
+                            Logger.Info(
+                                $"Receive message verification command from remote, endpoints={_endpoints}, clientId={_client.GetClientId()}");
+                            _client.OnVerifyMessageCommand(_endpoints, response.VerifyMessageCommand);
+                            break;
+                        }
+                        case Proto.TelemetryCommand.CommandOneofCase.PrintThreadStackTraceCommand:
+                        {
+                            Logger.Info(
+                                $"Receive thread stack print command from remote, endpoints={_endpoints}, clientId={_client.GetClientId()}");
+                            _client.OnPrintThreadStackTraceCommand(_endpoints, response.PrintThreadStackTraceCommand);
+                            break;
+                        }
+                        default:
+                        {
+                            Logger.Warn(
+                                $"Receive unrecognized command from remote, endpoints={_endpoints}, command={response}, clientId={_client.GetClientId()}");
+                            break;
+                        }
+                    }
+                }
+            });
+        }
     };
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Settings.cs b/csharp/rocketmq-client-csharp/Settings.cs
new file mode 100644
index 00000000..e7ea4e92
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Settings.cs
@@ -0,0 +1,42 @@
+using System;
+using rmq = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public abstract class Settings
+    {
+        protected readonly string ClientId;
+        protected readonly ClientType ClientType;
+        protected readonly Endpoints AccessPoint;
+        protected volatile RetryPolicy RetryPolicy;
+        protected readonly TimeSpan RequestTimeout;
+
+        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, RetryPolicy retryPolicy,
+            TimeSpan requestTimeout)
+        {
+            ClientId = clientId;
+            ClientType = clientType;
+            AccessPoint = accessPoint;
+            RetryPolicy = retryPolicy;
+            RequestTimeout = requestTimeout;
+        }
+
+        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, TimeSpan requestTimeout)
+        {
+            ClientId = clientId;
+            ClientType = clientType;
+            AccessPoint = accessPoint;
+            RetryPolicy = null;
+            RequestTimeout = requestTimeout;
+        }
+
+        public abstract rmq::Settings ToProtobuf();
+
+        public abstract void Sync(rmq::Settings settings);
+
+        public RetryPolicy GetRetryPolicy()
+        {
+            return RetryPolicy;
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Signature.cs b/csharp/rocketmq-client-csharp/Signature.cs
index 729640b8..e65125b6 100644
--- a/csharp/rocketmq-client-csharp/Signature.cs
+++ b/csharp/rocketmq-client-csharp/Signature.cs
@@ -25,21 +25,16 @@ namespace Org.Apache.Rocketmq
     {
         public static void Sign(IClientConfig clientConfig, grpc::Metadata metadata)
         {
-            metadata.Add(MetadataConstants.LANGUAGE_KEY, "DOTNET");
-            metadata.Add(MetadataConstants.CLIENT_VERSION_KEY, MetadataConstants.CLIENT_VERSION);
-            metadata.Add(MetadataConstants.CLIENT_ID_KEY, clientConfig.clientId());
+            metadata.Add(MetadataConstants.LanguageKey, MetadataConstants.LanguageValue);
+            metadata.Add(MetadataConstants.ClientVersionKey, MetadataConstants.Instance.ClientVersion);
+            metadata.Add(MetadataConstants.ClientIdKey, clientConfig.ClientId);
+            
+            string time = DateTime.Now.ToString(MetadataConstants.DateTimeFormat);
+            metadata.Add(MetadataConstants.DateTimeKey, time);
 
-            if (!String.IsNullOrEmpty(clientConfig.resourceNamespace()))
+            if (null != clientConfig.CredentialsProvider)
             {
-                metadata.Add(MetadataConstants.NAMESPACE_KEY, clientConfig.resourceNamespace());
-            }
-
-            string time = DateTime.Now.ToString(MetadataConstants.DATE_TIME_FORMAT);
-            metadata.Add(MetadataConstants.DATE_TIME_KEY, time);
-
-            if (null != clientConfig.credentialsProvider())
-            {
-                var credentials = clientConfig.credentialsProvider().getCredentials();
+                var credentials = clientConfig.CredentialsProvider.Credentials;
                 if (null == credentials || credentials.expired())
                 {
                     return;
@@ -47,7 +42,7 @@ namespace Org.Apache.Rocketmq
 
                 if (!String.IsNullOrEmpty(credentials.SessionToken))
                 {
-                    metadata.Add(MetadataConstants.STS_SESSION_TOKEN, credentials.SessionToken);
+                    metadata.Add(MetadataConstants.SessionTokenKey, credentials.SessionToken);
                 }
 
                 byte[] secretData = Encoding.ASCII.GetBytes(credentials.AccessSecret);
@@ -55,17 +50,15 @@ namespace Org.Apache.Rocketmq
                 HMACSHA1 signer = new HMACSHA1(secretData);
                 byte[] digest = signer.ComputeHash(data);
                 string hmac = BitConverter.ToString(digest).Replace("-", "");
-                string authorization = string.Format("{0} {1}={2}/{3}/{4}, {5}={6}, {7}={8}",
-                    MetadataConstants.ALGORITHM_KEY,
-                    MetadataConstants.CREDENTIAL_KEY,
+                string authorization = string.Format("{0} {1}={2}, {3}={4}, {5}={6}",
+                    MetadataConstants.AlgorithmKey,
+                    MetadataConstants.CredentialKey,
                     credentials.AccessKey,
-                    clientConfig.region(),
-                    clientConfig.serviceName(),
-                    MetadataConstants.SIGNED_HEADERS_KEY,
-                    MetadataConstants.DATE_TIME_KEY,
-                    MetadataConstants.SIGNATURE_KEY,
+                    MetadataConstants.SignedHeadersKey,
+                    MetadataConstants.DateTimeKey,
+                    MetadataConstants.SignatureKey,
                     hmac);
-                metadata.Add(MetadataConstants.AUTHORIZATION, authorization);
+                metadata.Add(MetadataConstants.Authorization, authorization);
             }
         }
     }
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
deleted file mode 100644
index ff92044f..00000000
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ /dev/null
@@ -1,319 +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.
- */
-
-using System;
-using rmq = Apache.Rocketmq.V2;
-using System.Threading.Tasks;
-using System.Collections.Concurrent;
-using System.Threading;
-using Grpc.Core;
-using System.Collections.Generic;
-using System.Linq;
-using Google.Protobuf.WellKnownTypes;
-
-namespace Org.Apache.Rocketmq
-{
-    public class SimpleConsumer : Client
-    {
-
-        public SimpleConsumer(string accessUrl, string group)
-        : base(accessUrl)
-        {
-            _subscriptions = new();
-            _topicAssignments = new();
-            _group = group;
-        }
-
-        public override void BuildClientSetting(rmq::Settings settings)
-        {
-            base.BuildClientSetting(settings);
-
-            settings.ClientType = rmq::ClientType.SimpleConsumer;
-            settings.Subscription = new rmq::Subscription
-            {
-                Group = new rmq::Resource
-                {
-                    Name = _group,
-                    ResourceNamespace = ResourceNamespace
-                }
-            };
-
-            foreach (var kv in _subscriptions)
-            {
-                settings.Subscription.Subscriptions.Add(kv.Value);
-            }
-            Logger.Info($"ClientSettings built OK. {settings}");
-        }
-
-        public override async Task Start()
-        {
-            await base.Start();
-            
-            // Scan load assignment periodically
-            Schedule(async () =>
-            {
-                Logger.Debug("Scan load assignments by schedule");
-                await ScanLoadAssignments();
-            }, 30, _scanAssignmentCts.Token);
-
-            await ScanLoadAssignments();
-            Logger.Debug("Step of #Start: ScanLoadAssignments completed");
-        }
-
-        public override async Task Shutdown()
-        {
-            _scanAssignmentCts.Cancel();
-            await base.Shutdown();
-            var group = new rmq.Resource()
-            {
-                Name = _group,
-                ResourceNamespace = "",
-            };
-            if (!await NotifyClientTermination(group))
-            {
-                Logger.Warn("Failed to NotifyClientTermination");
-            }
-        }
-        
-        /**
-         * For 5.x, we can assume there is a load-balancer before gateway nodes.
-         */
-        private async Task ScanLoadAssignments()
-        {
-            var tasks = new List<Task<List<rmq.Assignment>>>();
-            var topics = new List<string>();
-            foreach (var sub in _subscriptions)
-            {
-                var request = new rmq::QueryAssignmentRequest
-                {
-                    Topic = new rmq::Resource
-                    {
-                        ResourceNamespace = ResourceNamespace,
-                        Name = sub.Key
-                    }
-                };
-                topics.Add(sub.Key);
-                request.Group = new rmq::Resource
-                {
-                    Name = _group,
-                    ResourceNamespace = ResourceNamespace
-                };
-
-                request.Endpoints = new rmq::Endpoints
-                {
-                    Scheme = AccessPoint.HostScheme()
-                };
-                var address = new rmq::Address
-                {
-                    Host = AccessPoint.Host,
-                    Port = AccessPoint.Port
-                };
-                request.Endpoints.Addresses.Add(address);
-
-                var metadata = new Metadata();
-                Signature.Sign(this, metadata);
-                tasks.Add(Manager.QueryLoadAssignment(AccessPoint.TargetUrl(), metadata, request, TimeSpan.FromSeconds(3)));
-            }
-
-            var list = await Task.WhenAll(tasks);
-            var i = 0;
-            foreach (var assignments in list)
-            {
-                string topic = topics[i++];
-                if (null == assignments || 0 == assignments.Count)
-                {
-                    Logger.Warn($"Faild to acquire assignments. Topic={topic}, Group={_group}");
-                    continue;
-                }
-
-                Logger.Debug($"Assignments received. Topic={topic}, Group={_group}");
-                var newSubscriptionLB = new SubscriptionLoadBalancer(assignments);
-                _topicAssignments.AddOrUpdate(topic, newSubscriptionLB, (t, prev) => prev.Update(assignments));
-            }
-        }
-
-        protected override void PrepareHeartbeatData(rmq::HeartbeatRequest request)
-        {
-            request.ClientType = rmq::ClientType.SimpleConsumer;
-            request.Group = new rmq::Resource
-            {
-                Name = _group,
-                ResourceNamespace = ResourceNamespace
-            };
-        }
-
-        public void Subscribe(string topic, FilterExpression filterExpression)
-        {
-            var entry = new rmq::SubscriptionEntry
-            {
-                Topic = new rmq::Resource
-                {
-                    Name = topic,
-                    ResourceNamespace = ResourceNamespace
-                },
-                Expression = new rmq::FilterExpression
-                {
-                    Type = filterExpression.Type switch {
-                        ExpressionType.TAG => rmq::FilterType.Tag,
-                        ExpressionType.SQL92 => rmq::FilterType.Sql,
-                        _ => rmq.FilterType.Tag
-                    },
-                    Expression = filterExpression.Expression
-                }
-            };
-
-            _subscriptions.AddOrUpdate(topic, entry, (k, prev) => entry);
-            AddTopicOfInterest(topic);
-        }
-
-        public void Unsubscribe(string topic)
-        {
-            _subscriptions.TryRemove(topic, out var _);
-            RemoveTopicOfInterest(topic);
-        }
-
-        internal override void OnSettingsReceived(rmq.Settings settings)
-        {
-            base.OnSettingsReceived(settings);
-
-            if (settings.Subscription.Fifo)
-            {
-                Logger.Info($"#OnSettingsReceived: Group {_group} is FIFO");
-            }
-        }
-
-        public async Task<List<Message>> Receive(int batchSize, TimeSpan invisibleDuration, TimeSpan? awaitDuration = null)
-        {
-            var messageQueue = NextQueue();
-            if (null == messageQueue)
-            {
-                throw new TopicRouteException("No topic to receive message from");
-            }
-
-            var request = new rmq.ReceiveMessageRequest
-            {
-                Group = new rmq.Resource
-                {
-                    ResourceNamespace = ResourceNamespace,
-                    Name = _group
-                },
-                MessageQueue = new rmq.MessageQueue()
-            };
-
-            request.MessageQueue.MergeFrom(messageQueue);
-            request.BatchSize = batchSize;
-            request.InvisibleDuration = Duration.FromTimeSpan(invisibleDuration);
-            
-            // Client is responsible of extending message invisibility duration
-            request.AutoRenew = false;
-            
-            var targetUrl = Utilities.TargetUrl(messageQueue);
-            var metadata = new Metadata();
-            Signature.Sign(this, metadata);
-
-            var timeout = (awaitDuration ?? ClientSettings.Subscription.LongPollingTimeout.ToTimeSpan())
-                .Add(this.RequestTimeout);
-
-            return await Manager.ReceiveMessage(targetUrl, metadata, request, timeout);
-        }
-
-
-        public async Task Ack(Message message)
-        {
-            var request = new rmq.AckMessageRequest
-            {
-                Group = new rmq.Resource
-                {
-                    ResourceNamespace = ResourceNamespace,
-                    Name = _group
-                },
-                Topic = new rmq.Resource
-                {
-                    ResourceNamespace = ResourceNamespace,
-                    Name = message.Topic
-                }
-            };
-
-            var entry = new rmq.AckMessageEntry();
-            request.Entries.Add(entry);
-            entry.MessageId = message.MessageId;
-            entry.ReceiptHandle = message._receiptHandle;
-
-            var targetUrl = message._sourceHost;
-            var metadata = new Metadata();
-            Signature.Sign(this, metadata);
-            await Manager.Ack(targetUrl, metadata, request, RequestTimeout);
-        }
-
-        public async Task ChangeInvisibleDuration(Message message, TimeSpan invisibleDuration)
-        {
-            var request = new rmq.ChangeInvisibleDurationRequest
-            {
-                Group = new rmq.Resource
-                {
-                    ResourceNamespace = ResourceNamespace,
-                    Name = _group
-                },
-                Topic = new rmq.Resource
-                {
-                    ResourceNamespace = ResourceNamespace,
-                    Name = message.Topic
-                },
-                ReceiptHandle = message._receiptHandle,
-                MessageId = message.MessageId,
-                InvisibleDuration = Duration.FromTimeSpan(invisibleDuration)
-            };
-
-            var targetUrl = message._sourceHost;
-            var metadata = new Metadata();
-            Signature.Sign(this, metadata);
-            await Manager.ChangeInvisibleDuration(targetUrl, metadata, request, RequestTimeout);
-        }
-        
-        private rmq.MessageQueue NextQueue()
-        {
-            if (_topicAssignments.IsEmpty)
-            {
-                return null;
-            }
-            
-            var topicSeq = _currentTopicSequence.Value;
-            _currentTopicSequence.Value = topicSeq + 1;
-
-            var total = _topicAssignments.Count;
-            var topicIndex = topicSeq % total;
-            var topic = _topicAssignments.Keys.Skip((int)topicIndex).First();
-
-            if (!_topicAssignments.TryGetValue(topic, out var subscriptionLB))
-            {
-                return null;
-            }
-
-            return subscriptionLB.TakeMessageQueue();
-        }
-
-        private readonly ThreadLocal<UInt32> _currentTopicSequence = new ThreadLocal<UInt32>(true)
-        {
-            Value = 0
-        };
-
-        private readonly string _group;
-        private readonly ConcurrentDictionary<string, rmq::SubscriptionEntry> _subscriptions;
-        private readonly ConcurrentDictionary<string, SubscriptionLoadBalancer> _topicAssignments;
-        private readonly CancellationTokenSource _scanAssignmentCts = new CancellationTokenSource();
-    }
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs b/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
index edd810dd..b76cd496 100644
--- a/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/StaticCredentialsProvider.cs
@@ -14,23 +14,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 namespace Org.Apache.Rocketmq
 {
     public class StaticCredentialsProvider : ICredentialsProvider
     {
-
         public StaticCredentialsProvider(string accessKey, string accessSecret)
         {
-            this.accessKey = accessKey;
-            this.accessSecret = accessSecret;
-        }
-
-        public Credentials getCredentials()
-        {
-            return new Credentials(accessKey, accessSecret);
+            Credentials = new Credentials(accessKey, accessSecret);
         }
 
-        private string accessKey;
-        private string accessSecret;
+        public Credentials Credentials { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StatusChecker.cs b/csharp/rocketmq-client-csharp/StatusChecker.cs
new file mode 100644
index 00000000..cf15c204
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/StatusChecker.cs
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using Google.Protobuf;
+using NLog;
+using Org.Apache.Rocketmq.Error;
+using Proto = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public static class StatusChecker
+    {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
+        public static void Check(Proto.Status status, IMessage message)
+        {
+            Proto.Code statusCode = status.Code;
+
+            var statusMessage = status.Message;
+            switch (statusCode)
+            {
+                case Proto.Code.Ok:
+                case Proto.Code.MultipleResults:
+                    return;
+                case Proto.Code.BadRequest:
+                case Proto.Code.IllegalAccessPoint:
+                case Proto.Code.IllegalTopic:
+                case Proto.Code.IllegalConsumerGroup:
+                case Proto.Code.IllegalMessageTag:
+                case Proto.Code.IllegalMessageKey:
+                case Proto.Code.IllegalMessageGroup:
+                case Proto.Code.IllegalMessagePropertyKey:
+                case Proto.Code.InvalidTransactionId:
+                case Proto.Code.IllegalMessageId:
+                case Proto.Code.IllegalFilterExpression:
+                case Proto.Code.IllegalInvisibleTime:
+                case Proto.Code.IllegalDeliveryTime:
+                case Proto.Code.InvalidReceiptHandle:
+                case Proto.Code.MessagePropertyConflictWithType:
+                case Proto.Code.UnrecognizedClientType:
+                case Proto.Code.MessageCorrupted:
+                case Proto.Code.ClientIdRequired:
+                case Proto.Code.IllegalPollingTime:
+                    throw new BadRequestException((int)statusCode, statusMessage);
+                case Proto.Code.Unauthorized:
+                    throw new UnauthorizedException((int)statusCode, statusMessage);
+                case Proto.Code.PaymentRequired:
+                    throw new PaymentRequiredException((int)statusCode, statusMessage);
+                case Proto.Code.Forbidden:
+                    throw new ForbiddenException((int)statusCode, statusMessage);
+                case Proto.Code.MessageNotFound:
+                    if (message is Proto.ReceiveMessageRequest)
+                    {
+                        return;
+                    }
+
+                    // Fall through on purpose.
+                    goto case Proto.Code.NotFound;
+                case Proto.Code.NotFound:
+                case Proto.Code.TopicNotFound:
+                case Proto.Code.ConsumerGroupNotFound:
+                    throw new NotFoundException((int)statusCode, statusMessage);
+                case Proto.Code.PayloadTooLarge:
+                case Proto.Code.MessageBodyTooLarge:
+                    throw new PayloadTooLargeException((int)statusCode, statusMessage);
+                case Proto.Code.TooManyRequests:
+                    throw new TooManyRequestsException((int)statusCode, statusMessage);
+                case Proto.Code.RequestHeaderFieldsTooLarge:
+                case Proto.Code.MessagePropertiesTooLarge:
+                    throw new RequestHeaderFieldsTooLargeException((int)statusCode, statusMessage);
+                case Proto.Code.InternalError:
+                case Proto.Code.InternalServerError:
+                case Proto.Code.HaNotAvailable:
+                    throw new InternalErrorException((int)statusCode, statusMessage);
+                case Proto.Code.ProxyTimeout:
+                case Proto.Code.MasterPersistenceTimeout:
+                case Proto.Code.SlavePersistenceTimeout:
+                    throw new ProxyTimeoutException((int)statusCode, statusMessage);
+                case Proto.Code.Unsupported:
+                case Proto.Code.VersionUnsupported:
+                case Proto.Code.VerifyFifoMessageUnsupported:
+                    throw new UnsupportedException((int)statusCode, statusMessage);
+                default:
+                    Logger.Warn($"Unrecognized status code={statusCode}, statusMessage={statusMessage}");
+                    throw new UnsupportedException((int)statusCode, statusMessage);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/TopicRouteData.cs b/csharp/rocketmq-client-csharp/TopicRouteData.cs
index e4aa04c6..3c5689a4 100644
--- a/csharp/rocketmq-client-csharp/TopicRouteData.cs
+++ b/csharp/rocketmq-client-csharp/TopicRouteData.cs
@@ -23,21 +23,25 @@ namespace Org.Apache.Rocketmq
 {
     public class TopicRouteData : IEquatable<TopicRouteData>
     {
-        public TopicRouteData(List<rmq::MessageQueue> partitions)
+        public TopicRouteData(List<rmq::MessageQueue> messageQueues)
         {
-            _messageQueues = partitions;
+            List<MessageQueue> messageQueuesList = new List<MessageQueue>();
+            foreach (var mq in messageQueues)
+            {
+                messageQueuesList.Add(new MessageQueue(mq));
+            }
 
-            _messageQueues.Sort(Utilities.CompareMessageQueue);
+            MessageQueues = messageQueuesList;
         }
 
-        private List<rmq::MessageQueue> _messageQueues;
-        public List<rmq::MessageQueue> MessageQueues { get { return _messageQueues; } }
+        public List<MessageQueue> MessageQueues { get; }
+
 
         public bool Equals(TopicRouteData other)
         {
             if (ReferenceEquals(null, other)) return false;
             if (ReferenceEquals(this, other)) return true;
-            return Equals(_messageQueues, other._messageQueues);
+            return Equals(MessageQueues, other.MessageQueues);
         }
 
         public override bool Equals(object obj)
@@ -50,7 +54,12 @@ namespace Org.Apache.Rocketmq
 
         public override int GetHashCode()
         {
-            return (_messageQueues != null ? _messageQueues.GetHashCode() : 0);
+            return (MessageQueues != null ? MessageQueues.GetHashCode() : 0);
+        }
+
+        public override string ToString()
+        {
+            return $"{nameof(MessageQueues)}: {MessageQueues}";
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/UserAgent.cs b/csharp/rocketmq-client-csharp/UserAgent.cs
new file mode 100644
index 00000000..7b589212
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/UserAgent.cs
@@ -0,0 +1,31 @@
+using System;
+using Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class UserAgent
+    {
+        private readonly string _version;
+        private readonly string _platform;
+        private readonly string _hostName;
+
+        public static readonly UserAgent Instance = new();
+
+        private UserAgent()
+        {
+            _version = MetadataConstants.Instance.ClientVersion;
+            _platform = Environment.OSVersion.ToString();
+            _hostName = System.Net.Dns.GetHostName();
+        }
+
+        public UA ToProtobuf()
+        {
+            return new UA
+            {
+                Version = _version,
+                Hostname = _hostName,
+                Platform = _platform
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Utilities.cs b/csharp/rocketmq-client-csharp/Utilities.cs
index 23ed8db2..993f8ab2 100644
--- a/csharp/rocketmq-client-csharp/Utilities.cs
+++ b/csharp/rocketmq-client-csharp/Utilities.cs
@@ -20,12 +20,16 @@ using System.Linq;
 using System.Net.NetworkInformation;
 using System.Text;
 using System;
+using System.IO;
+using System.IO.Compression;
+using System.Threading;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public static class Utilities
     {
+        private static long _instanceSequence = 0;
         public static byte[] GetMacAddress()
         {
             return NetworkInterface.GetAllNetworkInterfaces().FirstOrDefault(nic =>
@@ -38,6 +42,36 @@ namespace Org.Apache.Rocketmq
             return Process.GetCurrentProcess().Id;
         }
 
+        public static String GetHostName()
+        {
+            return System.Net.Dns.GetHostName();
+        }
+
+        public static String GetClientId()
+        {
+            var hostName = System.Net.Dns.GetHostName();
+            var pid = Process.GetCurrentProcess().Id;
+            var index = Interlocked.Increment(ref _instanceSequence);
+            var nowMillisecond = (long)(DateTime.UtcNow - new DateTime(1970, 1, 1)).TotalMilliseconds;
+            var no = DecimalToBase36(nowMillisecond);
+            return $"{hostName}@{pid}@{index}@{no}";
+        }
+        
+        
+        static string DecimalToBase36(long decimalNumber)
+        {
+            const string chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";
+            string result = string.Empty;
+
+            while (decimalNumber > 0)
+            {
+                result = chars[(int)(decimalNumber % 36)] + result;
+                decimalNumber /= 36;
+            }
+
+            return result;
+        }
+
         public static string ByteArrayToHexString(byte[] bytes)
         {
             StringBuilder result = new StringBuilder(bytes.Length * 2);
@@ -52,6 +86,15 @@ namespace Org.Apache.Rocketmq
             return result.ToString();
         }
 
+        public static byte[] uncompressBytesGzip(byte[] src)
+        {
+            var inputStream = new MemoryStream(src);
+            var gzipStream = new GZipStream(inputStream, CompressionMode.Decompress);
+            var outputStream = new MemoryStream();
+            gzipStream.CopyTo(outputStream);
+            return outputStream.ToArray();
+        }
+
         public static string TargetUrl(rmq::MessageQueue messageQueue)
         {
             // TODO: Assert associated broker has as least one service endpoint.
@@ -61,7 +104,8 @@ namespace Org.Apache.Rocketmq
 
         public static int CompareMessageQueue(rmq::MessageQueue lhs, rmq::MessageQueue rhs)
         {
-            int topic_comparison = String.Compare(lhs.Topic.ResourceNamespace + lhs.Topic.Name, rhs.Topic.ResourceNamespace + rhs.Topic.Name);
+            int topic_comparison = String.Compare(lhs.Topic.ResourceNamespace + lhs.Topic.Name,
+                rhs.Topic.ResourceNamespace + rhs.Topic.Name);
             if (topic_comparison != 0)
             {
                 return topic_comparison;
diff --git a/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog b/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
index bef8d30f..a03537bc 100644
--- a/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
+++ b/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
@@ -35,7 +35,7 @@
         </target>
     </targets>
     <rules>
-        <logger name="*" writeTo="asyncFile" />
-        <logger name="*" writeTo="colorConsole" />
+        <logger name="*" minlevel="info" writeTo="asyncFile" />
+        <logger name="*" minlevel="Info" writeTo="colorConsole" />
     </rules>
 </nlog>
\ No newline at end of file
diff --git a/csharp/tests/ClientManagerTest.cs b/csharp/tests/ClientManagerTest.cs
deleted file mode 100644
index e12c0276..00000000
--- a/csharp/tests/ClientManagerTest.cs
+++ /dev/null
@@ -1,58 +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.
- */
-using System;
-using Grpc.Core;
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using rmq = Apache.Rocketmq.V2;
-
-namespace Org.Apache.Rocketmq
-{
-
-    [TestClass]
-    public class ClientManagerTest
-    {
-
-        [TestMethod]
-        public void TestResolveRoute()
-        {
-            string topic = "cpp_sdk_standard";
-            string resourceNamespace = "MQ_INST_1080056302921134_BXuIbML7";
-            var request = new rmq::QueryRouteRequest();
-            request.Topic = new rmq::Resource();
-            request.Topic.ResourceNamespace = resourceNamespace;
-            request.Topic.Name = topic;
-            request.Endpoints = new rmq::Endpoints();
-            request.Endpoints.Scheme = rmq::AddressScheme.Ipv4;
-            var address = new rmq::Address();
-            address.Host = "116.62.231.199";
-            address.Port = 80;
-            request.Endpoints.Addresses.Add(address);
-
-            var metadata = new Metadata();
-            var clientConfig = new ClientConfig();
-            var credentialsProvider = new ConfigFileCredentialsProvider();
-            clientConfig.CredentialsProvider = credentialsProvider;
-            clientConfig.ResourceNamespace = resourceNamespace;
-            clientConfig.Region = "cn-hangzhou-pre";
-            Signature.Sign(clientConfig, metadata);
-            var clientManager = new ClientManager();
-            string target = "https://116.62.231.199:80";
-            var topicRouteData = clientManager.ResolveRoute(target, metadata, request, TimeSpan.FromSeconds(3)).GetAwaiter().GetResult();
-            Console.WriteLine(topicRouteData);
-        }
-    }
-}
\ No newline at end of file
diff --git a/csharp/tests/ConfigFileCredentialsProviderTest.cs b/csharp/tests/ConfigFileCredentialsProviderTest.cs
index d5333592..0ebea9e3 100644
--- a/csharp/tests/ConfigFileCredentialsProviderTest.cs
+++ b/csharp/tests/ConfigFileCredentialsProviderTest.cs
@@ -42,7 +42,7 @@ namespace Org.Apache.Rocketmq
         public void TestGetCredentials()
         {
             var provider = new ConfigFileCredentialsProvider();
-            var credentials = provider.getCredentials();
+            var credentials = provider.GetCredentials();
             Assert.IsNotNull(credentials);
             Assert.AreEqual(credentials.AccessKey, "key");
             Assert.AreEqual(credentials.AccessSecret, "secret");
diff --git a/csharp/tests/DateTimeTest.cs b/csharp/tests/DateTimeTest.cs
index fdf7d532..17bbd889 100644
--- a/csharp/tests/DateTimeTest.cs
+++ b/csharp/tests/DateTimeTest.cs
@@ -28,7 +28,7 @@ namespace Org.Apache.Rocketmq
         public void testFormat()
         {
             DateTime instant = new DateTime(2022, 02, 15, 08, 31, 56);
-            string time = instant.ToString(MetadataConstants.DATE_TIME_FORMAT);
+            string time = instant.ToString(MetadataConstants.DateTimeFormat);
             string expected = "20220215T083156Z";
             Assert.AreEqual(time, expected);
         }
diff --git a/csharp/tests/MessageTest.cs b/csharp/tests/MessageTest.cs
index f1c71f85..9bb0f66c 100644
--- a/csharp/tests/MessageTest.cs
+++ b/csharp/tests/MessageTest.cs
@@ -29,8 +29,6 @@ namespace Org.Apache.Rocketmq
         public void testCtor()
         {
             var msg1 = new Message();
-            Assert.IsNotNull(msg1.MessageId);
-            Assert.IsTrue(msg1.MessageId.StartsWith("01"));
             Assert.IsNull(msg1.Topic);
             Assert.IsNull(msg1.Body);
             Assert.IsNull(msg1.Tag);
diff --git a/csharp/tests/ProducerTest.cs b/csharp/tests/ProducerTest.cs
deleted file mode 100644
index bcf36a5b..00000000
--- a/csharp/tests/ProducerTest.cs
+++ /dev/null
@@ -1,189 +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.
- */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using System;
-using System.Collections.Generic;
-using System.Threading.Tasks;
-using Org.Apache.Rocketmq;
-
-
-namespace tests
-{
-    [TestClass]
-    public class ProducerTest
-    {
-        [TestMethod]
-        public async Task TestLifecycle()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            await producer.Shutdown();
-        }
-
-        [TestMethod]
-        public async Task TestSendStandardMessage()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            byte[] body = new byte[1024];
-            Array.Fill(body, (byte)'x');
-            var msg = new Message(TOPIC, body)
-            {
-                // Tag the massage. A message has at most one tag.
-                Tag = "Tag-0"
-            };
-
-            // Associate the message with one or multiple keys
-            var keys = new List<string>
-            {
-                "k1",
-                "k2"
-            };
-            msg.Keys = keys;
-            
-            var sendResult = await producer.Send(msg);
-            Assert.IsNotNull(sendResult);
-            await producer.Shutdown();
-        }
-        
-        [TestMethod]
-        public async Task TestSendMultipleMessages()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            byte[] body = new byte[1024];
-            Array.Fill(body, (byte)'x');
-            for (var i = 0; i < 128; i++)
-            {
-                var msg = new Message(TOPIC, body)
-                {
-                    // Tag the massage. A message has at most one tag.
-                    Tag = "Tag-0"
-                };
-
-                // Associate the message with one or multiple keys
-                var keys = new List<string>
-                {
-                    "k1",
-                    "k2"
-                };
-                msg.Keys = keys;
-                var sendResult = await producer.Send(msg);
-                Assert.IsNotNull(sendResult);                
-            }
-            await producer.Shutdown();
-        }
-        
-        [TestMethod]
-        public async Task TestSendFifoMessage()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            byte[] body = new byte[1024];
-            Array.Fill(body, (byte)'x');
-            var msg = new Message(TOPIC, body)
-            {
-                // Messages of the same group will get delivered one after another. 
-                MessageGroup = "message-group-0"
-            };
-
-            // Verify messages are FIFO iff their message group is not null or empty.
-            Assert.IsTrue(msg.Fifo());
-
-            var sendResult = await producer.Send(msg);
-            Assert.IsNotNull(sendResult);
-            await producer.Shutdown();
-        }
-        
-        [TestMethod]
-        public async Task TestSendScheduledMessage()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            byte[] body = new byte[1024];
-            Array.Fill(body, (byte)'x');
-            var msg = new Message(TOPIC, body)
-            {
-                DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(10)
-            };
-
-            Assert.IsTrue(msg.Scheduled());
-            
-            var sendResult = await producer.Send(msg);
-            Assert.IsNotNull(sendResult);
-            await producer.Shutdown();
-        }
-        
-        
-        /**
-         * Trying send a message that is both FIFO and Scheduled should fail.
-         */
-        [TestMethod]
-        public async Task TestSendMessage_Failure()
-        {
-            var producer = new Producer($"{HOST}:{PORT}")
-            {
-                CredentialsProvider = new ConfigFileCredentialsProvider(),
-                Region = "cn-hangzhou-pre"
-            };
-            await producer.Start();
-            byte[] body = new byte[1024];
-            Array.Fill(body, (byte)'x');
-            var msg = new Message(TOPIC, body)
-            {
-                MessageGroup = "Group-0",
-                DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(10)
-            };
-            Assert.IsTrue(msg.Scheduled());
-
-            try
-            {
-                await producer.Send(msg);
-                Assert.Fail("Should have raised an exception");
-            }
-            catch (MessageException)
-            {
-            }
-            await producer.Shutdown();
-        }
-        
-        private const string TOPIC = "cpp_sdk_standard";
-        private const string HOST = "127.0.0.1";
-        private const int PORT = 8081;
-    }
-
-}
\ No newline at end of file
diff --git a/csharp/tests/RpcClientTest.cs b/csharp/tests/RpcClientTest.cs
deleted file mode 100644
index b4380473..00000000
--- a/csharp/tests/RpcClientTest.cs
+++ /dev/null
@@ -1,146 +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.
- */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using System;
-using System.Text;
-using System.Threading;
-using System.Threading.Tasks;
-using System.Collections.Generic;
-using grpc = Grpc.Core;
-using rmq = Apache.Rocketmq.V2;
-
-
-namespace Org.Apache.Rocketmq
-{
-
-    [TestClass]
-    public class RpcClientTest
-    {
-
-        [TestMethod]
-        public async Task testTelemetry()
-        {
-            Console.WriteLine("Test Telemetry streaming");
-            string target = "https://11.166.42.94:8081";
-            var rpc_client = new RpcClient(target);
-            var client_config = new ClientConfig();
-            var metadata = new grpc::Metadata();
-            Signature.Sign(client_config, metadata);
-
-            var cmd = new rmq::TelemetryCommand();
-            cmd.Settings = new rmq::Settings();
-            cmd.Settings.ClientType = rmq::ClientType.Producer;
-            cmd.Settings.AccessPoint = new rmq::Endpoints();
-            cmd.Settings.AccessPoint.Scheme = rmq::AddressScheme.Ipv4;
-            var address = new rmq::Address();
-            address.Port = 8081;
-            address.Host = "11.166.42.94";
-            cmd.Settings.AccessPoint.Addresses.Add(address);
-            cmd.Settings.RequestTimeout = new Google.Protobuf.WellKnownTypes.Duration();
-            cmd.Settings.RequestTimeout.Seconds = 3;
-            cmd.Settings.RequestTimeout.Nanos = 0;
-            cmd.Settings.Publishing = new rmq::Publishing();
-            var topic = new rmq::Resource();
-            topic.Name = "cpp_sdk_standard";
-            cmd.Settings.Publishing.Topics.Add(topic);
-            cmd.Settings.UserAgent = new rmq::UA();
-            cmd.Settings.UserAgent.Language = rmq::Language.DotNet;
-            cmd.Settings.UserAgent.Version = "1.0";
-            cmd.Settings.UserAgent.Hostname = System.Net.Dns.GetHostName();
-            cmd.Settings.UserAgent.Platform = System.Environment.OSVersion.ToString();
-
-            var duplexStreaming = rpc_client.Telemetry(metadata);
-            var reader = duplexStreaming.ResponseStream;
-            var writer = duplexStreaming.RequestStream;
-
-            var cts = new CancellationTokenSource();
-            await writer.WriteAsync(cmd);
-            Console.WriteLine("Command written");
-            if (await reader.MoveNext(cts.Token))
-            {
-                var response = reader.Current;
-                switch (response.CommandCase)
-                {
-                    case rmq::TelemetryCommand.CommandOneofCase.Settings:
-                        {
-                            var responded_settings = response.Settings;
-                            Console.WriteLine($"{responded_settings.ToString()}");
-                            break;
-                        }
-                    case rmq::TelemetryCommand.CommandOneofCase.None:
-                        {
-                            Console.WriteLine($"Unknown response command type: {response.Status.ToString()}");
-                            break;
-                        }
-                }
-                Console.WriteLine("Server responded ");
-            }
-            else
-            {
-                Console.WriteLine("Server is not responding");
-                var status = duplexStreaming.GetStatus();
-                Console.WriteLine($"status={status.ToString()}");
-
-                var trailers = duplexStreaming.GetTrailers();
-                Console.WriteLine($"trailers={trailers.ToString()}");
-            }
-        }
-
-        [TestMethod]
-        public void testQueryRoute()
-        {
-            string target = "https://11.166.42.94:8081";
-            var rpc_client = new RpcClient(target);
-            var client_config = new ClientConfig();
-            var metadata = new grpc::Metadata();
-            Signature.Sign(client_config, metadata);
-            var request = new rmq::QueryRouteRequest();
-            request.Topic = new rmq::Resource();
-            request.Topic.Name = "cpp_sdk_standard";
-            request.Endpoints = new rmq::Endpoints();
-            request.Endpoints.Scheme = rmq::AddressScheme.Ipv4;
-            var address = new rmq::Address();
-            address.Port = 8081;
-            address.Host = "11.166.42.94";
-            request.Endpoints.Addresses.Add(address);
-            var response = rpc_client.QueryRoute(metadata, request, client_config.RequestTimeout);
-            var result = response.GetAwaiter().GetResult();
-        }
-
-        [TestMethod]
-        public async Task TestSend()
-        {
-            string target = "https://11.166.42.94:8081";
-            var rpc_client = new RpcClient(target);
-            var client_config = new ClientConfig();
-            var metadata = new grpc::Metadata();
-            Signature.Sign(client_config, metadata);
-
-            var request = new rmq::SendMessageRequest();
-            var message = new rmq::Message();
-            message.Topic = new rmq::Resource();
-            message.Topic.Name = "cpp_sdk_standard";
-            message.Body = Google.Protobuf.ByteString.CopyFromUtf8("Test Body");
-            message.SystemProperties = new rmq::SystemProperties();
-            message.SystemProperties.Tag = "TagA";
-            message.SystemProperties.MessageId = "abc";
-            request.Messages.Add(message);
-            var response = await rpc_client.SendMessage(metadata, request, TimeSpan.FromSeconds(3));
-            Assert.AreEqual(rmq::Code.Ok, response.Status.Code);
-        }
-    }
-}
\ No newline at end of file
diff --git a/csharp/tests/SendResultTest.cs b/csharp/tests/SendResultTest.cs
index 4e3d9a0c..fae7a7bb 100644
--- a/csharp/tests/SendResultTest.cs
+++ b/csharp/tests/SendResultTest.cs
@@ -19,30 +19,15 @@ using Microsoft.VisualStudio.TestTools.UnitTesting;
 
 namespace Org.Apache.Rocketmq
 {
-
     [TestClass]
     public class SendResultTest
     {
-
         [TestMethod]
         public void testCtor()
         {
             string messageId = new string("abc");
             var sendResult = new SendReceipt(messageId);
             Assert.AreEqual(messageId, sendResult.MessageId);
-            Assert.AreEqual(SendStatus.SEND_OK, sendResult.Status);
         }
-
-
-        [TestMethod]
-        public void testCtor2()
-        {
-            string messageId = new string("abc");
-            var sendResult = new SendReceipt(messageId, SendStatus.FLUSH_DISK_TIMEOUT);
-            Assert.AreEqual(messageId, sendResult.MessageId);
-            Assert.AreEqual(SendStatus.FLUSH_DISK_TIMEOUT, sendResult.Status);
-        }
-
     }
-
 }
\ No newline at end of file
diff --git a/csharp/tests/SignatureTest.cs b/csharp/tests/SignatureTest.cs
index 12d1c10d..24055e8b 100644
--- a/csharp/tests/SignatureTest.cs
+++ b/csharp/tests/SignatureTest.cs
@@ -30,19 +30,14 @@ namespace tests
         public void TestSign()
         {
             var mock = new Mock<IClientConfig>();
-            mock.Setup(x => x.getGroupName()).Returns("G1");
-            mock.Setup(x => x.resourceNamespace()).Returns("mq:arn:test:");
-            mock.Setup(x => x.serviceName()).Returns("mq");
-            mock.Setup(x => x.region()).Returns("cn-hangzhou");
 
             string accessKey = "key";
             string accessSecret = "secret";
             var credentialsProvider = new StaticCredentialsProvider(accessKey, accessSecret);
-            mock.Setup(x => x.credentialsProvider()).Returns(credentialsProvider);
 
             var metadata = new grpc::Metadata();
             Signature.Sign(mock.Object, metadata);
-            Assert.IsNotNull(metadata.Get(MetadataConstants.AUTHORIZATION));
+            Assert.IsNotNull(metadata.Get(MetadataConstants.Authorization));
         }
     }
 
diff --git a/csharp/tests/SimpleConsumerTest.cs b/csharp/tests/SimpleConsumerTest.cs
deleted file mode 100644
index 6f49eb1e..00000000
--- a/csharp/tests/SimpleConsumerTest.cs
+++ /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.
- */
-
-using System;
-using System.Threading;
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using rmq = Apache.Rocketmq.V2;
-using System.Threading.Tasks;
-using Org.Apache.Rocketmq;
-
-namespace tests
-{
-
-    [TestClass]
-    public class SimpleConsumerTest
-    {
-
-        private static string _group = "GID_cpp_sdk_standard";
-        private static string _topic = "cpp_sdk_standard";
-        private const string HOST = "127.0.0.1";
-        private const int PORT = 8081;
-        
-
-        [TestMethod]
-        public async Task TestLifecycle()
-        {
-            var simpleConsumer = new SimpleConsumer($"{HOST}:{PORT}", _group);
-            simpleConsumer.Subscribe(_topic, new FilterExpression("*", ExpressionType.TAG));
-            await simpleConsumer.Start();
-            Thread.Sleep(1_000);
-            await simpleConsumer.Shutdown();
-        }
-
-        [TestMethod]
-        public async Task TestReceive()
-        {
-            var simpleConsumer = new SimpleConsumer($"{HOST}:{PORT}", _group);
-            simpleConsumer.Subscribe(_topic, new FilterExpression("*", ExpressionType.TAG));
-            await simpleConsumer.Start();
-            var batchSize = 32;
-            var invisibleDuration = TimeSpan.FromSeconds(10);
-            var messages  = await simpleConsumer.Receive(batchSize, invisibleDuration);
-            Assert.IsTrue(messages.Count > 0);
-            Assert.IsTrue(messages.Count <= batchSize);
-            await simpleConsumer.Shutdown();
-        }
-        
-        
-        [TestMethod]
-        public async Task TestAck()
-        {
-            var simpleConsumer = new SimpleConsumer($"{HOST}:{PORT}", _group);
-            simpleConsumer.Subscribe(_topic, new FilterExpression("*", ExpressionType.TAG));
-            await simpleConsumer.Start();
-            var batchSize = 32;
-            var invisibleDuration = TimeSpan.FromSeconds(10);
-            var messages  = await simpleConsumer.Receive(batchSize, invisibleDuration);
-            foreach (var message in messages)
-            {
-                await simpleConsumer.Ack(message);
-                Console.WriteLine($"Ack {message.MessageId} OK");
-            }
-            await simpleConsumer.Shutdown();
-        }
-        
-        [TestMethod]
-        public async Task TestChangeInvisibleDuration()
-        {
-            var simpleConsumer = new SimpleConsumer($"{HOST}:{PORT}", _group);
-            simpleConsumer.Subscribe(_topic, new FilterExpression("*", ExpressionType.TAG));
-            await simpleConsumer.Start();
-            var batchSize = 32;
-            var invisibleDuration = TimeSpan.FromSeconds(10);
-            var messages  = await simpleConsumer.Receive(batchSize, invisibleDuration);
-            foreach (var message in messages)
-            {
-                await simpleConsumer.ChangeInvisibleDuration(message, TimeSpan.FromSeconds(10));
-                Console.WriteLine($"ChangeInvisibleDuration for message[MsgId={message.MessageId}] OK");
-            }
-            await simpleConsumer.Shutdown();
-        }
-    }
-}
\ No newline at end of file
diff --git a/csharp/tests/StaticCredentialsProviderTest.cs b/csharp/tests/StaticCredentialsProviderTest.cs
deleted file mode 100644
index 8b5f0126..00000000
--- a/csharp/tests/StaticCredentialsProviderTest.cs
+++ /dev/null
@@ -1,40 +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.
- */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-
-
-namespace Org.Apache.Rocketmq
-{
-    [TestClass]
-    public class StaticCredentialsProviderTest
-    {
-
-        [TestMethod]
-        public void testGetCredentials()
-        {
-            var accessKey = "key";
-            var accessSecret = "secret";
-            var provider = new StaticCredentialsProvider(accessKey, accessSecret);
-            var credentials = provider.getCredentials();
-            Assert.IsNotNull(credentials);
-            Assert.IsFalse(credentials.expired(), "Credentials from StaticCredentialsProvider should never expire");
-            Assert.AreEqual(credentials.AccessKey, accessKey);
-            Assert.AreEqual(credentials.AccessSecret, accessSecret);
-        }
-
-    }
-}
\ No newline at end of file
diff --git a/csharp/tests/UnitTest1.cs b/csharp/tests/UnitTest1.cs
index 2706c545..af13d3dc 100644
--- a/csharp/tests/UnitTest1.cs
+++ b/csharp/tests/UnitTest1.cs
@@ -61,12 +61,6 @@ namespace tests
             }
         }
 
-        [TestMethod]
-        public void TestRpcClientImplCtor()
-        {
-            RpcClient impl = new RpcClient("https://localhost:5001");
-        }
-
         [TestMethod]
         public void TestConcurrentDictionary()
         {


[rocketmq-clients] 26/28: Add more response code cases

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit f4e4647c78b427acb3215eb2a6ff7a23ea4eeb07
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 22 19:07:02 2023 +0800

    Add more response code cases
---
 csharp/rocketmq-client-csharp/MessageView.cs   | 2 +-
 csharp/rocketmq-client-csharp/StatusChecker.cs | 8 +++++++-
 2 files changed, 8 insertions(+), 2 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index b88c40b4..21cd3677 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -36,7 +36,7 @@ namespace Org.Apache.Rocketmq
         private readonly long _offset;
         private readonly bool _corrupted;
 
-        internal MessageView(string messageId, string topic, byte[] body, string tag, string messageGroup,
+        private MessageView(string messageId, string topic, byte[] body, string tag, string messageGroup,
             DateTime? deliveryTimestamp, List<string> keys, Dictionary<string, string> properties, string bornHost,
             DateTime bornTime, int deliveryAttempt, MessageQueue messageQueue, string receiptHandle, long offset,
             bool corrupted)
diff --git a/csharp/rocketmq-client-csharp/StatusChecker.cs b/csharp/rocketmq-client-csharp/StatusChecker.cs
index 2802fd78..abc0b160 100644
--- a/csharp/rocketmq-client-csharp/StatusChecker.cs
+++ b/csharp/rocketmq-client-csharp/StatusChecker.cs
@@ -31,7 +31,7 @@ namespace Org.Apache.Rocketmq
             var statusCode = status.Code;
 
             var statusMessage = status.Message;
-            // TODO
+            // TODO: add request-id.
             switch (statusCode)
             {
                 case Proto.Code.Ok:
@@ -95,6 +95,12 @@ namespace Org.Apache.Rocketmq
                 case Proto.Code.VersionUnsupported:
                 case Proto.Code.VerifyFifoMessageUnsupported:
                     throw new UnsupportedException((int)statusCode, statusMessage);
+                // Not used code.
+                case Proto.Code.RequestTimeout:
+                case Proto.Code.PreconditionFailed:
+                case Proto.Code.NotImplemented:
+                case Proto.Code.FailedToConsumeMessage:
+                case Proto.Code.Unspecified:
                 default:
                     Logger.Warn($"Unrecognized status code={statusCode}, statusMessage={statusMessage}");
                     throw new UnsupportedException((int)statusCode, statusMessage);


[rocketmq-clients] 08/28: Bugfix: ICollection is read-only

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 8f7418ed80b62ac163b20ea33aaa0058fa9279fe
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 17:27:23 2023 +0800

    Bugfix: ICollection is read-only
---
 csharp/examples/ProducerBenchmark.cs            | 32 +++++++++++++++++++++++++
 csharp/rocketmq-client-csharp/Client.cs         |  7 ++++--
 csharp/rocketmq-client-csharp/IClient.cs        |  2 +-
 csharp/rocketmq-client-csharp/Producer.cs       | 19 +++++++--------
 csharp/rocketmq-client-csharp/Session.cs        |  5 ++--
 csharp/rocketmq-client-csharp/SimpleConsumer.cs | 27 ++++++++++++++-------
 6 files changed, 67 insertions(+), 25 deletions(-)

diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerBenchmark.cs
new file mode 100644
index 00000000..361aa95d
--- /dev/null
+++ b/csharp/examples/ProducerBenchmark.cs
@@ -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.
+ */
+
+using System.Threading.Tasks;
+using NLog;
+using Org.Apache.Rocketmq;
+
+namespace examples
+{
+    public class ProducerBenchmark
+    {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
+        internal static async Task QuickStart()
+        {
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 3b260022..133fed00 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -333,7 +333,7 @@ namespace Org.Apache.Rocketmq
             return _telemetryCts;
         }
 
-        public abstract Proto.Settings GetSettings();
+        public abstract Settings GetSettings();
 
         public string GetClientId()
         {
@@ -358,6 +358,9 @@ namespace Org.Apache.Rocketmq
         {
         }
 
-        public abstract void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings);
+        public void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
+        {
+            GetSettings().Sync(settings);
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClient.cs b/csharp/rocketmq-client-csharp/IClient.cs
index 5ba4c6f1..fc4c0127 100644
--- a/csharp/rocketmq-client-csharp/IClient.cs
+++ b/csharp/rocketmq-client-csharp/IClient.cs
@@ -27,7 +27,7 @@ namespace Org.Apache.Rocketmq
 
         ClientConfig GetClientConfig();
 
-        Proto.Settings GetSettings();
+        Settings GetSettings();
 
         /// <summary>
         /// Get the identifier of current client. 
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 62387a98..6a9040ec 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -30,6 +30,7 @@ namespace Org.Apache.Rocketmq
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
         private readonly ConcurrentDictionary<string /* topic */, PublishingLoadBalancer> _publishingRouteDataCache;
         private readonly PublishingSettings _publishingSettings;
+        private readonly ConcurrentDictionary<string, bool> _publishingTopics;
 
 
         public Producer(ClientConfig clientConfig) : this(clientConfig, new ConcurrentDictionary<string, bool>(), 3)
@@ -41,20 +42,21 @@ namespace Org.Apache.Rocketmq
         {
         }
 
-        private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
-            base(clientConfig, topics.Keys)
+        private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> publishingTopics, int maxAttempts) :
+            base(clientConfig, publishingTopics.Keys)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
             _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
-                clientConfig.RequestTimeout, topics);
+                clientConfig.RequestTimeout, publishingTopics);
             _publishingRouteDataCache = new ConcurrentDictionary<string, PublishingLoadBalancer>();
+            _publishingTopics = publishingTopics;
         }
 
         public void SetTopics(params string[] topics)
         {
             foreach (var topic in topics)
             {
-                Topics.Add(topic);
+                _publishingTopics.TryAdd(topic, true);
             }
         }
 
@@ -183,14 +185,9 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public override Proto.Settings GetSettings()
+        public override Settings GetSettings()
         {
-            return _publishingSettings.ToProtobuf();
-        }
-
-        public override void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
-        {
-            _publishingSettings.Sync(settings);
+            return _publishingSettings;
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index 99d61268..dd3da7bd 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using System;
 using System.Threading;
 using System.Threading.Channels;
 using System.Threading.Tasks;
@@ -63,9 +62,9 @@ namespace Org.Apache.Rocketmq
                 var writer = _streamingCall.RequestStream;
                 // await readTask;
                 var settings = _client.GetSettings();
-                Proto.TelemetryCommand telemetryCommand = new Proto.TelemetryCommand
+                var telemetryCommand = new Proto.TelemetryCommand
                 {
-                    Settings = settings
+                    Settings = settings.ToProtobuf()
                 };
                 await writer.WriteAsync(telemetryCommand);
                 // await writer.CompleteAsync();
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index 1a0f0ec2..cb380d89 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using System.Collections.Concurrent;
 using System.Collections.Generic;
@@ -77,17 +94,11 @@ namespace Org.Apache.Rocketmq
             _subscriptionRouteDataCache.TryAdd(topic, subscriptionLoadBalancer);
         }
 
-        public override Proto.Settings GetSettings()
+        public override Settings GetSettings()
         {
-            return _simpleSubscriptionSettings.ToProtobuf();
+            return _simpleSubscriptionSettings;
         }
 
-        public override void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
-        {
-            _simpleSubscriptionSettings.Sync(settings);
-        }
-
-
         private async Task<SubscriptionLoadBalancer> GetSubscriptionLoadBalancer(string topic)
         {
             if (_subscriptionRouteDataCache.TryGetValue(topic, out var subscriptionLoadBalancer))


[rocketmq-clients] 15/28: Implement transaction message

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 7de5b3957cf7d32dcff0edee61d6ed0710febb56
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Thu Feb 16 13:51:02 2023 +0800

    Implement transaction message
---
 csharp/examples/ProducerBenchmark.cs               |   2 +-
 csharp/examples/ProducerFifoMessageExample.cs      |  81 +++++++-------
 ...ark.cs => ProducerTransactionMessageExample.cs} |  71 +++++-------
 csharp/examples/QuickStart.cs                      |   4 +-
 csharp/rocketmq-client-csharp/Client.cs            |  18 ++-
 csharp/rocketmq-client-csharp/ClientManager.cs     |   6 +
 csharp/rocketmq-client-csharp/IClientManager.cs    |   3 +
 .../ITransaction.cs}                               |  15 +--
 .../ITransactionChecker.cs}                        |  19 ++--
 csharp/rocketmq-client-csharp/MessageView.cs       |   5 +
 csharp/rocketmq-client-csharp/Producer.cs          |  78 +++++++++++--
 csharp/rocketmq-client-csharp/SendReceipt.cs       |  15 ++-
 csharp/rocketmq-client-csharp/Transaction.cs       | 121 +++++++++++++++++++++
 .../TransactionResolution.cs}                      |  15 +--
 csharp/tests/SendResultTest.cs                     |   6 +-
 15 files changed, 322 insertions(+), 137 deletions(-)

diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerBenchmark.cs
index 4e334104..3918666d 100644
--- a/csharp/examples/ProducerBenchmark.cs
+++ b/csharp/examples/ProducerBenchmark.cs
@@ -68,7 +68,7 @@ namespace examples
                 Keys = keys
             };
 
-            const int tpsLimit = 800;
+            const int tpsLimit = 500;
 
             Task.Run(async () =>
             {
diff --git a/csharp/examples/ProducerFifoMessageExample.cs b/csharp/examples/ProducerFifoMessageExample.cs
index d9a72a59..87f953c3 100644
--- a/csharp/examples/ProducerFifoMessageExample.cs
+++ b/csharp/examples/ProducerFifoMessageExample.cs
@@ -17,55 +17,60 @@
 
 using System.Collections.Generic;
 using System.Text;
+using System.Threading;
 using System.Threading.Tasks;
 using NLog;
 using Org.Apache.Rocketmq;
 
 namespace examples
 {
-    static class ProducerFifoMessageExample
+    internal static class ProducerFifoMessageExample
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
         internal static async Task QuickStart()
         {
-            // string accessKey = "yourAccessKey";
-            // string secretKey = "yourSecretKey";
-            // // Credential provider is optional for client configuration.
-            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            // string endpoints = "foobar.com:8080";
-            // // In most case, you don't need to create too many producers, single pattern is recommended.
-            // var producer = new Producer(endpoints)
-            // {
-            //     CredentialsProvider = credentialsProvider
-            // };
-            // string topic = "yourFifoTopic";
-            // // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
-            // // the topic route before message publishing.
-            // producer.AddTopicOfInterest(topic);
-            //
-            // await producer.Start();
-            // // Define your message body.
-            // byte[] bytes = Encoding.UTF8.GetBytes("foobar");
-            // string tag = "yourMessageTagA";
-            // // You could set multiple keys for the single message.
-            // var keys = new List<string>
-            // {
-            //     "yourMessageKey-6cc8b65ed1c8",
-            //     "yourMessageKey-43783375d9a5"
-            // };
-            // // Set topic for current message.
-            // var message = new Message(topic, bytes)
-            // {
-            //     Tag = tag,
-            //     Keys = keys,
-            //     // Essential for FIFO message, messages that belongs to the same message group follow the FIFO semantics.
-            //     MessageGroup = "yourMessageGroup0"
-            // };
-            // var sendReceipt = await producer.Send(message);
-            // Logger.Info($"Send FIFO message successfully, sendReceipt={sendReceipt}.");
-            // // Close the producer if you don't need it anymore.
-            // await producer.Shutdown();
+            const string accessKey = "5jFk0wK7OU6Uq395";
+            const string secretKey = "V1u8z19URHs4o6RQ";
+
+            // Credential provider is optional for client configuration.
+            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            var clientConfig = new ClientConfig(endpoints)
+            {
+                CredentialsProvider = credentialsProvider
+            };
+            // In most case, you don't need to create too many producers, single pattern is recommended.
+            var producer = new Producer(clientConfig);
+
+            const string topic = "lingchu_fifo_topic";
+            producer.SetTopics(topic);
+            // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // the topic route before message publishing.
+            await producer.Start();
+            // Define your message body.
+            var bytes = Encoding.UTF8.GetBytes("foobar");
+            const string tag = "yourMessageTagA";
+            // You could set multiple keys for the single message.
+            var keys = new List<string>
+            {
+                "yourMessageKey-7044358f98fc",
+                "yourMessageKey-f72539fbc246"
+            };
+            const string messageGroup = "yourMessageGroup";
+            // Set topic for current message.
+            var message = new Message(topic, bytes)
+            {
+                Tag = tag,
+                Keys = keys,
+                // Set message group for FIFO message.
+                MessageGroup = messageGroup
+            };
+            var sendReceipt = await producer.Send(message);
+            Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
+            Thread.Sleep(9999999);
+            // Close the producer if you don't need it anymore.
+            await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerTransactionMessageExample.cs
similarity index 58%
copy from csharp/examples/ProducerBenchmark.cs
copy to csharp/examples/ProducerTransactionMessageExample.cs
index 4e334104..edc4d41f 100644
--- a/csharp/examples/ProducerBenchmark.cs
+++ b/csharp/examples/ProducerTransactionMessageExample.cs
@@ -15,31 +15,33 @@
  * limitations under the License.
  */
 
-using System;
 using System.Collections.Generic;
 using System.Text;
-using System.Threading;
 using System.Threading.Tasks;
 using NLog;
 using Org.Apache.Rocketmq;
 
 namespace examples
 {
-    public class ProducerBenchmark
+    internal static class ProducerTransactionMessageExample
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        private static readonly SemaphoreSlim _semaphore = new SemaphoreSlim(0);
-        private static long _counter = 0;
-
-        internal static void QuickStart()
+        private class TransactionChecker : ITransactionChecker
         {
-            const string accessKey = "amKhwEM40L61znSz";
-            const string secretKey = "bT6c3gpF3EFB10F3";
+            public TransactionResolution Check(MessageView messageView)
+            {
+                return TransactionResolution.COMMIT;
+            }
+        }
 
+        internal static async Task QuickStart()
+        {
+            const string accessKey = "yourAccessKey";
+            const string secretKey = "yourSecretKey";
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-nwy337bf81g.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
@@ -47,11 +49,12 @@ namespace examples
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
 
-            const string topic = "lingchu_normal_topic";
+            const string topic = "lingchu_transactional_topic";
             producer.SetTopics(topic);
-            // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
-            // the topic route before message publishing.
-            producer.Start().Wait();
+            producer.SetTransactionChecker(new TransactionChecker());
+
+            await producer.Start();
+            var transaction = producer.BeginTransaction();
             // Define your message body.
             var bytes = Encoding.UTF8.GetBytes("foobar");
             const string tag = "yourMessageTagA";
@@ -67,38 +70,14 @@ namespace examples
                 Tag = tag,
                 Keys = keys
             };
-
-            const int tpsLimit = 800;
-
-            Task.Run(async () =>
-            {
-                while (true)
-                {
-                    _semaphore.Release(tpsLimit);
-                    await Task.Delay(TimeSpan.FromMilliseconds(1000));
-                }
-            });
-
-            Task.Run(async () =>
-            {
-                while (true)
-                {
-                    Logger.Info($"Send {_counter} messages successfully.");
-                    Interlocked.Exchange(ref _counter, 0);
-                    await Task.Delay(TimeSpan.FromSeconds(1));
-                }
-            });
-
-            var tasks = new List<Task>();
-            while (true)
-            {
-                _semaphore.Wait();
-                Interlocked.Increment(ref _counter);
-                var task = producer.Send(message);
-                tasks.Add(task);
-            }
-
-            Task.WhenAll(tasks).Wait();
+            var sendReceipt = await producer.Send(message, transaction);
+            Logger.Info("Send transaction message successfully, messageId={}", sendReceipt.MessageId);
+            // Commit the transaction.
+            transaction.commit();
+            // Or rollback the transaction.
+            // transaction.rollback();
+            // Close the producer if you don't need it anymore.
+            await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/QuickStart.cs b/csharp/examples/QuickStart.cs
index 474d6063..8323218f 100644
--- a/csharp/examples/QuickStart.cs
+++ b/csharp/examples/QuickStart.cs
@@ -27,11 +27,11 @@ namespace examples
         {
             // Console.WriteLine(MetadataConstants.Instance.ClientVersion);
 
-            ProducerNormalMessageExample.QuickStart().Wait();
+            // ProducerNormalMessageExample.QuickStart().Wait();
             // await ProducerFifoMessageExample.QuickStart();
             // await ProducerDelayMessageExample.QuickStart();
             // await SimpleConsumerExample.QuickStart();
-            // ProducerBenchmark.QuickStart();
+            ProducerBenchmark.QuickStart();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index fc6871b9..63c24c91 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -389,10 +389,22 @@ namespace Org.Apache.Rocketmq
             return ClientConfig;
         }
 
-        public void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
+        public virtual async void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
             Proto.RecoverOrphanedTransactionCommand command)
         {
-            // TODO
+            Logger.Warn($"Ignore orphaned transaction recovery command from remote, which is not expected, " +
+                        $"clientId={ClientId}, endpoints={endpoints}");
+            var status = new Proto.Status
+            {
+                Code = Proto.Code.InternalError,
+                Message = "Current client don't support transaction message recovery"
+            };
+            var telemetryCommand = new Proto.TelemetryCommand
+            {
+                Status = status
+            };
+            var (_, session) = GetSession(endpoints);
+            await session.write(telemetryCommand);
         }
 
         public async void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
@@ -405,7 +417,7 @@ namespace Org.Apache.Rocketmq
                 Code = Proto.Code.Unsupported,
                 Message = "Message consumption verification is not supported"
             };
-            var telemetryCommand = new Proto.TelemetryCommand()
+            var telemetryCommand = new Proto.TelemetryCommand
             {
                 Status = status
             };
diff --git a/csharp/rocketmq-client-csharp/ClientManager.cs b/csharp/rocketmq-client-csharp/ClientManager.cs
index 967a2ac1..f464d461 100644
--- a/csharp/rocketmq-client-csharp/ClientManager.cs
+++ b/csharp/rocketmq-client-csharp/ClientManager.cs
@@ -146,5 +146,11 @@ namespace Org.Apache.Rocketmq
         {
             return await GetRpcClient(endpoints).ChangeInvisibleDuration(_client.Sign(), request, timeout);
         }
+
+        public async Task<Proto.EndTransactionResponse> EndTransaction(Endpoints endpoints,
+            Proto.EndTransactionRequest request, TimeSpan timeout)
+        {
+            return await GetRpcClient(endpoints).EndTransaction(_client.Sign(), request, timeout);
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClientManager.cs b/csharp/rocketmq-client-csharp/IClientManager.cs
index df2035ab..2082584c 100644
--- a/csharp/rocketmq-client-csharp/IClientManager.cs
+++ b/csharp/rocketmq-client-csharp/IClientManager.cs
@@ -81,6 +81,9 @@ namespace Org.Apache.Rocketmq
         Task<ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Endpoints endpoints,
             ChangeInvisibleDurationRequest request, TimeSpan timeout);
 
+        Task<EndTransactionResponse> EndTransaction(Endpoints endpoints, EndTransactionRequest request,
+            TimeSpan timeout);
+
         Task Shutdown();
     }
 }
\ No newline at end of file
diff --git a/csharp/tests/SendResultTest.cs b/csharp/rocketmq-client-csharp/ITransaction.cs
similarity index 71%
copy from csharp/tests/SendResultTest.cs
copy to csharp/rocketmq-client-csharp/ITransaction.cs
index fae7a7bb..b9898de0 100644
--- a/csharp/tests/SendResultTest.cs
+++ b/csharp/rocketmq-client-csharp/ITransaction.cs
@@ -15,19 +15,12 @@
  * limitations under the License.
  */
 
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-
 namespace Org.Apache.Rocketmq
 {
-    [TestClass]
-    public class SendResultTest
+    public interface ITransaction
     {
-        [TestMethod]
-        public void testCtor()
-        {
-            string messageId = new string("abc");
-            var sendResult = new SendReceipt(messageId);
-            Assert.AreEqual(messageId, sendResult.MessageId);
-        }
+        void commit();
+
+        void rollback();
     }
 }
\ No newline at end of file
diff --git a/csharp/tests/SendResultTest.cs b/csharp/rocketmq-client-csharp/ITransactionChecker.cs
similarity index 67%
copy from csharp/tests/SendResultTest.cs
copy to csharp/rocketmq-client-csharp/ITransactionChecker.cs
index fae7a7bb..f03350b1 100644
--- a/csharp/tests/SendResultTest.cs
+++ b/csharp/rocketmq-client-csharp/ITransactionChecker.cs
@@ -15,19 +15,16 @@
  * limitations under the License.
  */
 
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-
 namespace Org.Apache.Rocketmq
 {
-    [TestClass]
-    public class SendResultTest
+    public interface ITransactionChecker
     {
-        [TestMethod]
-        public void testCtor()
-        {
-            string messageId = new string("abc");
-            var sendResult = new SendReceipt(messageId);
-            Assert.AreEqual(messageId, sendResult.MessageId);
-        }
+        /// <summary>
+        /// Interface that implement this interface will be able to check transactions and
+        /// return a TransactionResolution object representing the result of the check.
+        /// </summary>
+        /// <param name="messageView"></param>
+        /// <returns></returns>
+        TransactionResolution Check(MessageView messageView);
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index dfb45e45..fd095819 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -80,6 +80,11 @@ namespace Org.Apache.Rocketmq
 
         public int DeliveryAttempt { get; }
 
+        public static MessageView FromProtobuf(Proto.Message message)
+        {
+            return FromProtobuf(message, null);
+        }
+
         public static MessageView FromProtobuf(Proto.Message message, MessageQueue messageQueue)
         {
             var topic = message.Topic.Name;
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 4bbc4bfa..23041e4c 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -29,9 +29,9 @@ namespace Org.Apache.Rocketmq
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
         private readonly ConcurrentDictionary<string /* topic */, PublishingLoadBalancer> _publishingRouteDataCache;
-        private readonly PublishingSettings _publishingSettings;
+        internal readonly PublishingSettings PublishingSettings;
         private readonly ConcurrentDictionary<string, bool> _publishingTopics;
-
+        private ITransactionChecker _checker = null;
 
         public Producer(ClientConfig clientConfig) : this(clientConfig, new ConcurrentDictionary<string, bool>(), 3)
         {
@@ -47,7 +47,7 @@ namespace Org.Apache.Rocketmq
             base(clientConfig)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
-            _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
+            PublishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
                 clientConfig.RequestTimeout, publishingTopics);
             _publishingRouteDataCache = new ConcurrentDictionary<string, PublishingLoadBalancer>();
             _publishingTopics = publishingTopics;
@@ -61,6 +61,11 @@ namespace Org.Apache.Rocketmq
             }
         }
 
+        public void SetTransactionChecker(ITransactionChecker checker)
+        {
+            _checker = checker;
+        }
+
         protected override IEnumerable<string> GetTopics()
         {
             return _publishingTopics.Keys;
@@ -115,13 +120,13 @@ namespace Org.Apache.Rocketmq
 
         private IRetryPolicy GetRetryPolicy()
         {
-            return _publishingSettings.GetRetryPolicy();
+            return PublishingSettings.GetRetryPolicy();
         }
 
         public async Task<SendReceipt> Send(Message message)
         {
             var publishingLoadBalancer = await GetPublishingLoadBalancer(message.Topic);
-            var publishingMessage = new PublishingMessage(message, _publishingSettings, false);
+            var publishingMessage = new PublishingMessage(message, PublishingSettings, false);
             var retryPolicy = GetRetryPolicy();
             var maxAttempts = retryPolicy.GetMaxAttempts();
 
@@ -147,6 +152,12 @@ namespace Org.Apache.Rocketmq
             throw exception!;
         }
 
+        public async Task<SendReceipt> Send(Message message, Transaction transaction)
+        {
+            // TODO
+            return null;
+        }
+
         private static Proto.SendMessageRequest WrapSendMessageRequest(PublishingMessage message, MessageQueue mq)
         {
             return new Proto.SendMessageRequest
@@ -160,7 +171,7 @@ namespace Org.Apache.Rocketmq
         {
             var candidateIndex = (attempt - 1) % candidates.Count;
             var mq = candidates[candidateIndex];
-            if (_publishingSettings.IsValidateMessageType() &&
+            if (PublishingSettings.IsValidateMessageType() &&
                 !mq.AcceptMessageTypes.Contains(message.MessageType))
             {
                 throw new ArgumentException("Current message type does not match with the accept message types," +
@@ -174,7 +185,7 @@ namespace Org.Apache.Rocketmq
                 await ClientManager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
             try
             {
-                var sendReceipts = SendReceipt.ProcessSendMessageResponse(response);
+                var sendReceipts = SendReceipt.ProcessSendMessageResponse(mq, response);
 
                 var sendReceipt = sendReceipts.First();
                 if (attempt > 1)
@@ -206,7 +217,58 @@ namespace Org.Apache.Rocketmq
 
         public override Settings GetSettings()
         {
-            return _publishingSettings;
+            return PublishingSettings;
+        }
+
+        public override async void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
+            Proto.RecoverOrphanedTransactionCommand command)
+        {
+            var messageId = command.Message.SystemProperties.MessageId;
+            if (null == _checker)
+            {
+                Logger.Error($"No transaction checker registered, ignore it, messageId={messageId}, " +
+                             $"transactionId={command.TransactionId}, endpoints={endpoints}, clientId={ClientId}");
+                return;
+            }
+
+            var message = MessageView.FromProtobuf(command.Message);
+            var transactionResolution = _checker.Check(message);
+            switch (transactionResolution)
+            {
+                case TransactionResolution.COMMIT:
+                case TransactionResolution.ROLLBACK:
+                    await EndTransaction(endpoints, message.Topic, message.MessageId, command.TransactionId,
+                        transactionResolution);
+                    break;
+                case TransactionResolution.UNKNOWN:
+                default:
+                    break;
+            }
+        }
+
+        public Transaction BeginTransaction()
+        {
+            return new Transaction(this);
+        }
+
+        internal async Task EndTransaction(Endpoints endpoints, string topic, string messageId, string transactionId,
+            TransactionResolution resolution)
+        {
+            var topicResource = new Proto.Resource
+            {
+                Name = topic
+            };
+            var request = new Proto.EndTransactionRequest
+            {
+                TransactionId = transactionId,
+                MessageId = messageId,
+                Topic = topicResource,
+                Resolution = TransactionResolution.COMMIT == resolution
+                    ? Proto.TransactionResolution.Commit
+                    : Proto.TransactionResolution.Rollback
+            };
+            var response = await ClientManager.EndTransaction(endpoints, request, ClientConfig.RequestTimeout);
+            StatusChecker.Check(response.Status, request);
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SendReceipt.cs b/csharp/rocketmq-client-csharp/SendReceipt.cs
index fa5c75c7..1e7c61bd 100644
--- a/csharp/rocketmq-client-csharp/SendReceipt.cs
+++ b/csharp/rocketmq-client-csharp/SendReceipt.cs
@@ -23,19 +23,28 @@ namespace Org.Apache.Rocketmq
 {
     public sealed class SendReceipt
     {
-        public SendReceipt(string messageId)
+        public SendReceipt(string messageId, string transactionId, MessageQueue messageQueue)
         {
             MessageId = messageId;
+            TransactionId = transactionId;
+            MessageQueue = messageQueue;
         }
 
         public string MessageId { get; }
 
+        public string TransactionId { get; }
+
+        public MessageQueue MessageQueue { get; }
+
+        public Endpoints Endpoints => MessageQueue.Broker.Endpoints;
+
         public override string ToString()
         {
             return $"{nameof(MessageId)}: {MessageId}";
         }
 
-        public static List<SendReceipt> ProcessSendMessageResponse(Proto.SendMessageResponse response)
+        public static IEnumerable<SendReceipt> ProcessSendMessageResponse(MessageQueue mq,
+            Proto.SendMessageResponse response)
         {
             var status = response.Status;
             foreach (var entry in response.Entries)
@@ -48,7 +57,7 @@ namespace Org.Apache.Rocketmq
 
             // May throw exception.
             StatusChecker.Check(status, response);
-            return response.Entries.Select(entry => new SendReceipt(entry.MessageId)).ToList();
+            return response.Entries.Select(entry => new SendReceipt(entry.MessageId, entry.TransactionId, mq)).ToList();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Transaction.cs b/csharp/rocketmq-client-csharp/Transaction.cs
new file mode 100644
index 00000000..e44c0675
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Transaction.cs
@@ -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.
+ */
+
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using System.Threading;
+
+namespace Org.Apache.Rocketmq
+{
+    public class Transaction : ITransaction
+    {
+        private const int MaxMessageNum = 1;
+
+        private readonly Producer _producer;
+        private readonly HashSet<PublishingMessage> _messages;
+        private readonly ReaderWriterLockSlim _messagesLock;
+        private readonly ConcurrentDictionary<PublishingMessage, SendReceipt> _messageSendReceiptDict;
+
+        public Transaction(Producer producer)
+        {
+            _producer = producer;
+            _messages = new HashSet<PublishingMessage>();
+            _messagesLock = new ReaderWriterLockSlim();
+            _messageSendReceiptDict = new ConcurrentDictionary<PublishingMessage, SendReceipt>();
+        }
+
+        public PublishingMessage TryAddMessage(Message message)
+        {
+            _messagesLock.EnterReadLock();
+            try
+            {
+                if (_messages.Count > MaxMessageNum)
+                {
+                    throw new ArgumentException($"Message in transaction has exceed the threshold: {MaxMessageNum}");
+                }
+            }
+            finally
+            {
+                _messagesLock.ExitReadLock();
+            }
+
+            _messagesLock.EnterWriteLock();
+            try
+            {
+                if (_messages.Count > MaxMessageNum)
+                {
+                    throw new ArgumentException($"Message in transaction has exceed the threshold: {MaxMessageNum}");
+                }
+
+                var publishingMessage = new PublishingMessage(message, _producer.PublishingSettings, true);
+                _messages.Add(publishingMessage);
+                return publishingMessage;
+            }
+            finally
+            {
+                _messagesLock.ExitWriteLock();
+            }
+        }
+
+        public void TryAddReceipt(PublishingMessage publishingMessage, SendReceipt sendReceipt)
+        {
+            _messagesLock.EnterReadLock();
+            try
+            {
+                if (!_messages.Contains(publishingMessage))
+                {
+                    throw new ArgumentException("Message is not in the transaction");
+                }
+
+                _messageSendReceiptDict[publishingMessage] = sendReceipt;
+            }
+            finally
+            {
+                _messagesLock.ExitReadLock();
+            }
+        }
+
+        public async void commit()
+        {
+            if (_messageSendReceiptDict.IsEmpty)
+            {
+                throw new ArgumentException("Transactional message has not been sent yet");
+            }
+
+            foreach (var (publishingMessage, sendReceipt) in _messageSendReceiptDict)
+            {
+                await _producer.EndTransaction(sendReceipt.Endpoints, publishingMessage.Topic, sendReceipt.MessageId,
+                    sendReceipt.TransactionId, TransactionResolution.COMMIT);
+            }
+        }
+
+        public async void rollback()
+        {
+            if (_messageSendReceiptDict.IsEmpty)
+            {
+                throw new ArgumentException("Transaction message has not been sent yet");
+            }
+
+            foreach (var (publishingMessage, sendReceipt) in _messageSendReceiptDict)
+            {
+                await _producer.EndTransaction(sendReceipt.Endpoints, publishingMessage.Topic, sendReceipt.MessageId,
+                    sendReceipt.TransactionId, TransactionResolution.ROLLBACK);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/tests/SendResultTest.cs b/csharp/rocketmq-client-csharp/TransactionResolution.cs
similarity index 71%
copy from csharp/tests/SendResultTest.cs
copy to csharp/rocketmq-client-csharp/TransactionResolution.cs
index fae7a7bb..5bb4d5e1 100644
--- a/csharp/tests/SendResultTest.cs
+++ b/csharp/rocketmq-client-csharp/TransactionResolution.cs
@@ -15,19 +15,12 @@
  * limitations under the License.
  */
 
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-
 namespace Org.Apache.Rocketmq
 {
-    [TestClass]
-    public class SendResultTest
+    public enum TransactionResolution
     {
-        [TestMethod]
-        public void testCtor()
-        {
-            string messageId = new string("abc");
-            var sendResult = new SendReceipt(messageId);
-            Assert.AreEqual(messageId, sendResult.MessageId);
-        }
+        COMMIT,
+        ROLLBACK,
+        UNKNOWN
     }
 }
\ No newline at end of file
diff --git a/csharp/tests/SendResultTest.cs b/csharp/tests/SendResultTest.cs
index fae7a7bb..262410da 100644
--- a/csharp/tests/SendResultTest.cs
+++ b/csharp/tests/SendResultTest.cs
@@ -25,9 +25,9 @@ namespace Org.Apache.Rocketmq
         [TestMethod]
         public void testCtor()
         {
-            string messageId = new string("abc");
-            var sendResult = new SendReceipt(messageId);
-            Assert.AreEqual(messageId, sendResult.MessageId);
+            // string messageId = new string("abc");
+            // var sendResult = new SendReceipt(messageId);
+            // Assert.AreEqual(messageId, sendResult.MessageId);
         }
     }
 }
\ No newline at end of file


[rocketmq-clients] 06/28: Add PublishingLoadBalancer#TakeMessageQueueByMessageGroup

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 2e2b23a87835b551e7cda8b361d3b6d6a7b3cf7f
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Fri Feb 10 14:30:37 2023 +0800

    Add PublishingLoadBalancer#TakeMessageQueueByMessageGroup
---
 csharp/rocketmq-client-csharp/Client.cs            | 14 +++-
 .../IRetryPolicy.cs}                               | 45 ++++++------
 csharp/rocketmq-client-csharp/MessageView.cs       |  2 +-
 csharp/rocketmq-client-csharp/Producer.cs          | 11 ++-
 .../PublishingLoadBalancer.cs                      | 80 ++++++++++------------
 csharp/rocketmq-client-csharp/RetryPolicy.cs       | 13 ----
 csharp/rocketmq-client-csharp/Utilities.cs         | 50 +++++---------
 csharp/tests/SignatureTest.cs                      | 18 ++---
 8 files changed, 106 insertions(+), 127 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 26ff9fcc..51ecb936 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -45,7 +45,8 @@ namespace Org.Apache.Rocketmq
         protected readonly string ClientId;
 
         protected readonly ConcurrentDictionary<string, bool> Topics;
-
+        
+        protected readonly ConcurrentDictionary<Endpoints, bool> Isolated;
         private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteCache;
         private readonly CancellationTokenSource _telemetryCts;
 
@@ -59,7 +60,7 @@ namespace Org.Apache.Rocketmq
             ClientId = Utilities.GetClientId();
 
             ClientManager = new ClientManager(this);
-
+            Isolated = new ConcurrentDictionary<Endpoints, bool>();
             _topicRouteCache = new ConcurrentDictionary<string, TopicRouteData>();
 
             _topicRouteUpdateCtx = new CancellationTokenSource();
@@ -276,12 +277,19 @@ namespace Org.Apache.Rocketmq
             {
                 var response = await responses[item];
                 var code = response.Status.Code;
-                
+
                 if (code.Equals(Proto.Code.Ok))
                 {
                     Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
+                    if (Isolated.TryRemove(item, out _))
+                    {
+                        Logger.Info(
+                            $"Rejoin endpoints which was isolate before, endpoints={item}, clientId={ClientId}");
+                    }
+
                     return;
                 }
+
                 var statusMessage = response.Status.Message;
                 Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
             }
diff --git a/csharp/tests/SignatureTest.cs b/csharp/rocketmq-client-csharp/IRetryPolicy.cs
similarity index 50%
copy from csharp/tests/SignatureTest.cs
copy to csharp/rocketmq-client-csharp/IRetryPolicy.cs
index 24055e8b..c006b1bd 100644
--- a/csharp/tests/SignatureTest.cs
+++ b/csharp/rocketmq-client-csharp/IRetryPolicy.cs
@@ -14,31 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using grpc = Grpc.Core;
-using Moq;
-using Org.Apache.Rocketmq;
 
-namespace tests
-{
+using System;
+using Apache.Rocketmq.V2;
 
-    [TestClass]
-    public class SignatureTest
+namespace Org.Apache.Rocketmq
+{
+    /// <summary>
+    /// Internal interface for retry policy.
+    /// </summary>
+    public interface IRetryPolicy
     {
+        /// <summary>
+        /// Get the max attempt times for retry.
+        /// </summary>
+        /// <returns>The max attempt times.</returns>
+        int GetMaxAttempts();
 
-        [TestMethod]
-        public void TestSign()
-        {
-            var mock = new Mock<IClientConfig>();
+        /// <summary>
+        /// Get await time after current attempts, the attempt index starts at 1.
+        /// </summary>
+        /// <param name="attempt">Current attempt.</param>
+        /// <returns>Await time.</returns>
+        TimeSpan GetNextAttemptDelay(int attempt);
 
-            string accessKey = "key";
-            string accessSecret = "secret";
-            var credentialsProvider = new StaticCredentialsProvider(accessKey, accessSecret);
-
-            var metadata = new grpc::Metadata();
-            Signature.Sign(mock.Object, metadata);
-            Assert.IsNotNull(metadata.Get(MetadataConstants.Authorization));
-        }
+        /// <summary>
+        /// Convert to protobuf.
+        /// </summary>
+        /// <returns></returns>
+        RetryPolicy ToProtobuf();
     }
-
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index 26d7fcc6..b790fcb9 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -136,7 +136,7 @@ namespace Org.Apache.Rocketmq
             {
                 case rmq.Encoding.Gzip:
                 {
-                    body = Utilities.uncompressBytesGzip(message.Body.ToByteArray());
+                    body = Utilities.DecompressBytesGzip(message.Body.ToByteArray());
                     break;
                 }
                 case rmq.Encoding.Identity:
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index c2782fc0..fc41926b 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -111,7 +111,12 @@ namespace Org.Apache.Rocketmq
             var publishingMessage = new PublishingMessage(message, _publishingSettings, false);
             var retryPolicy = GetRetryPolicy();
             var maxAttempts = retryPolicy.GetMaxAttempts();
-            var candidates = publishingLoadBalancer.TakeMessageQueues(publishingMessage.MessageGroup, maxAttempts);
+
+            // Prepare the candidate message queue(s) for retry-sending in advance.
+            var candidates = null == publishingMessage.MessageGroup
+                ? publishingLoadBalancer.TakeMessageQueues(new HashSet<Endpoints>(Isolated.Keys), maxAttempts)
+                : new List<MessageQueue>
+                    { publishingLoadBalancer.TakeMessageQueueByMessageGroup(publishingMessage.MessageGroup) };
             Exception exception = null;
             for (var attempt = 0; attempt < maxAttempts; attempt++)
             {
@@ -145,7 +150,7 @@ namespace Org.Apache.Rocketmq
             if (_publishingSettings.IsValidateMessageType() &&
                 !mq.AcceptMessageTypes.Contains(message.MessageType))
             {
-                throw new ArgumentException($"Current message type does not match with the accept message types," +
+                throw new ArgumentException("Current message type does not match with the accept message types," +
                                             $" topic={message.Topic}, actualMessageType={message.MessageType}" +
                                             $" acceptMessageType={string.Join(",", mq.AcceptMessageTypes)}");
             }
@@ -170,6 +175,8 @@ namespace Org.Apache.Rocketmq
             }
             catch (Exception e)
             {
+                // Isolate current endpoints.
+                Isolated[endpoints] = true;
                 Logger.Warn(e, $"Failed to send message, topic={message.Topic}, maxAttempts={maxAttempts}, " +
                                $"endpoints={endpoints}, clientId={ClientId}");
                 throw;
diff --git a/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
index c083e23f..c33bc7dc 100644
--- a/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
+++ b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
@@ -17,6 +17,7 @@
 
 using System;
 using System.Collections.Generic;
+using System.Linq;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
@@ -29,74 +30,65 @@ namespace Org.Apache.Rocketmq
         public PublishingLoadBalancer(TopicRouteData route)
         {
             _messageQueues = new List<MessageQueue>();
-            foreach (var messageQueue in route.MessageQueues)
+            foreach (var messageQueue in route.MessageQueues.Where(messageQueue =>
+                         PermissionHelper.IsWritable(messageQueue.Permission) &&
+                         Utilities.MasterBrokerId == messageQueue.Broker.Id))
             {
-                if (!PermissionHelper.IsWritable(messageQueue.Permission))
-                {
-                    continue;
-                }
-
                 _messageQueues.Add(messageQueue);
             }
 
-            Random random = new Random();
+            var random = new Random();
             _roundRobinIndex = random.Next(0, _messageQueues.Count);
         }
 
-        /**
-         * Accept a partition iff its broker is different.
-         */
-        private bool Accept(List<MessageQueue> existing, MessageQueue messageQueue)
-        {
-            if (0 == existing.Count)
-            {
-                return true;
-            }
-
-            foreach (var item in existing)
-            {
-                if (item.Broker.Equals(messageQueue.Broker))
-                {
-                    return false;
-                }
-            }
 
-            return true;
+        public MessageQueue TakeMessageQueueByMessageGroup(string messageGroup)
+        {
+            // TODO: use SipHash24 algorithm
+            var index = Utilities.GetPositiveMod(messageGroup.GetHashCode(), _messageQueues.Count);
+            return _messageQueues[index];
         }
 
-        public List<MessageQueue> TakeMessageQueues(string messageGroup, int maxAttemptTimes)
+        public List<MessageQueue> TakeMessageQueues(HashSet<Endpoints> excluded, int count)
         {
-            List<MessageQueue> result = new List<MessageQueue>();
+            var next = ++_roundRobinIndex;
+            var candidates = new List<MessageQueue>();
+            var candidateBrokerNames = new HashSet<string>();
 
-            List<MessageQueue> all = _messageQueues;
-            if (0 == all.Count)
+            foreach (var mq in _messageQueues.Select(_ => Utilities.GetPositiveMod(next++, _messageQueues.Count))
+                         .Select(index => _messageQueues[index]))
             {
-                return result;
-            }
+                if (!excluded.Contains(mq.Broker.Endpoints) && !candidateBrokerNames.Contains(mq.Broker.Name))
+                {
+                    candidateBrokerNames.Add(mq.Broker.Name);
+                    candidates.Add(mq);
+                }
 
-            if (!string.IsNullOrEmpty(messageGroup))
-            {
-                result.Add(all[messageGroup.GetHashCode() % all.Count]);
-                return result;
+                if (candidates.Count >= count)
+                {
+                    return candidates;
+                }
             }
 
-            int start = ++_roundRobinIndex;
-            int found = 0;
-
-            for (int i = 0; i < all.Count; i++)
+            if (candidates.Count != 0) return candidates;
             {
-                int idx = ((start + i) & int.MaxValue) % all.Count;
-                if (Accept(result, all[idx]))
+                foreach (var mq in _messageQueues.Select(_ => Utilities.GetPositiveMod(next++, _messageQueues.Count))
+                             .Select(positiveMod => _messageQueues[positiveMod]))
                 {
-                    result.Add(all[idx]);
-                    if (++found >= maxAttemptTimes)
+                    if (!candidateBrokerNames.Contains(mq.Broker.Name))
+                    {
+                        candidateBrokerNames.Add(mq.Broker.Name);
+                        candidates.Add(mq);
+                    }
+
+                    if (candidates.Count >= count)
                     {
                         break;
                     }
                 }
             }
 
-            return result;
+            return candidates;
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/RetryPolicy.cs b/csharp/rocketmq-client-csharp/RetryPolicy.cs
deleted file mode 100644
index 92b82013..00000000
--- a/csharp/rocketmq-client-csharp/RetryPolicy.cs
+++ /dev/null
@@ -1,13 +0,0 @@
-using System;
-
-namespace Org.Apache.Rocketmq
-{
-    public interface IRetryPolicy
-    {
-        int GetMaxAttempts();
-
-        TimeSpan GetNextAttemptDelay(int attempt);
-
-        global::Apache.Rocketmq.V2.RetryPolicy ToProtobuf();
-    }
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Utilities.cs b/csharp/rocketmq-client-csharp/Utilities.cs
index 993f8ab2..592f364e 100644
--- a/csharp/rocketmq-client-csharp/Utilities.cs
+++ b/csharp/rocketmq-client-csharp/Utilities.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using System.Diagnostics;
 using System.Linq;
 using System.Net.NetworkInformation;
 using System.Text;
@@ -30,6 +29,14 @@ namespace Org.Apache.Rocketmq
     public static class Utilities
     {
         private static long _instanceSequence = 0;
+        public const int MasterBrokerId = 0;
+
+        public static int GetPositiveMod(int k, int n)
+        {
+            var result = k % n;
+            return result < 0 ? result + n : result;
+        }
+
         public static byte[] GetMacAddress()
         {
             return NetworkInterface.GetAllNetworkInterfaces().FirstOrDefault(nic =>
@@ -39,26 +46,26 @@ namespace Org.Apache.Rocketmq
 
         public static int GetProcessId()
         {
-            return Process.GetCurrentProcess().Id;
+            return Environment.ProcessId;
         }
 
-        public static String GetHostName()
+        public static string GetHostName()
         {
             return System.Net.Dns.GetHostName();
         }
 
-        public static String GetClientId()
+        public static string GetClientId()
         {
             var hostName = System.Net.Dns.GetHostName();
-            var pid = Process.GetCurrentProcess().Id;
+            var pid = Environment.ProcessId;
             var index = Interlocked.Increment(ref _instanceSequence);
             var nowMillisecond = (long)(DateTime.UtcNow - new DateTime(1970, 1, 1)).TotalMilliseconds;
             var no = DecimalToBase36(nowMillisecond);
             return $"{hostName}@{pid}@{index}@{no}";
         }
-        
-        
-        static string DecimalToBase36(long decimalNumber)
+
+
+        private static string DecimalToBase36(long decimalNumber)
         {
             const string chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ";
             string result = string.Empty;
@@ -86,7 +93,7 @@ namespace Org.Apache.Rocketmq
             return result.ToString();
         }
 
-        public static byte[] uncompressBytesGzip(byte[] src)
+        public static byte[] DecompressBytesGzip(byte[] src)
         {
             var inputStream = new MemoryStream(src);
             var gzipStream = new GZipStream(inputStream, CompressionMode.Decompress);
@@ -94,30 +101,5 @@ namespace Org.Apache.Rocketmq
             gzipStream.CopyTo(outputStream);
             return outputStream.ToArray();
         }
-
-        public static string TargetUrl(rmq::MessageQueue messageQueue)
-        {
-            // TODO: Assert associated broker has as least one service endpoint.
-            var serviceEndpoint = messageQueue.Broker.Endpoints.Addresses[0];
-            return $"https://{serviceEndpoint.Host}:{serviceEndpoint.Port}";
-        }
-
-        public static int CompareMessageQueue(rmq::MessageQueue lhs, rmq::MessageQueue rhs)
-        {
-            int topic_comparison = String.Compare(lhs.Topic.ResourceNamespace + lhs.Topic.Name,
-                rhs.Topic.ResourceNamespace + rhs.Topic.Name);
-            if (topic_comparison != 0)
-            {
-                return topic_comparison;
-            }
-
-            int broker_name_comparison = String.Compare(lhs.Broker.Name, rhs.Broker.Name);
-            if (0 != broker_name_comparison)
-            {
-                return broker_name_comparison;
-            }
-
-            return lhs.Id < rhs.Id ? -1 : (lhs.Id == rhs.Id ? 0 : 1);
-        }
     }
 }
\ No newline at end of file
diff --git a/csharp/tests/SignatureTest.cs b/csharp/tests/SignatureTest.cs
index 24055e8b..63b7cdf8 100644
--- a/csharp/tests/SignatureTest.cs
+++ b/csharp/tests/SignatureTest.cs
@@ -29,15 +29,15 @@ namespace tests
         [TestMethod]
         public void TestSign()
         {
-            var mock = new Mock<IClientConfig>();
-
-            string accessKey = "key";
-            string accessSecret = "secret";
-            var credentialsProvider = new StaticCredentialsProvider(accessKey, accessSecret);
-
-            var metadata = new grpc::Metadata();
-            Signature.Sign(mock.Object, metadata);
-            Assert.IsNotNull(metadata.Get(MetadataConstants.Authorization));
+            // var mock = new Mock<IClientConfig>();
+            //
+            // string accessKey = "key";
+            // string accessSecret = "secret";
+            // var credentialsProvider = new StaticCredentialsProvider(accessKey, accessSecret);
+            //
+            // var metadata = new grpc::Metadata();
+            // Signature.Sign(mock.Object, metadata);
+            // Assert.IsNotNull(metadata.Get(MetadataConstants.Authorization));
         }
     }
 


[rocketmq-clients] 27/28: Implement ExponentialBackoffRetryPolicy

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit e3fbe1dbdfee0fe81b7f0e41cd8784ed42fd4a0e
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Thu Feb 23 18:42:48 2023 +0800

    Implement ExponentialBackoffRetryPolicy
---
 csharp/examples/ProducerDelayMessageExample.cs     |  2 +-
 csharp/examples/ProducerFifoMessageExample.cs      |  8 ++---
 csharp/examples/ProducerNormalMessageExample.cs    |  8 ++---
 .../examples/ProducerTransactionMessageExample.cs  |  4 +--
 .../ExponentialBackoffRetryPolicy.cs               | 42 ++++++++++++++++++++--
 csharp/rocketmq-client-csharp/IRetryPolicy.cs      |  3 ++
 .../rocketmq-client-csharp/PublishingSettings.cs   | 18 ++++++++--
 .../SimpleSubscriptionSettings.cs                  |  6 +++-
 8 files changed, 73 insertions(+), 18 deletions(-)

diff --git a/csharp/examples/ProducerDelayMessageExample.cs b/csharp/examples/ProducerDelayMessageExample.cs
index 9ad5fbb9..edb356a5 100644
--- a/csharp/examples/ProducerDelayMessageExample.cs
+++ b/csharp/examples/ProducerDelayMessageExample.cs
@@ -34,7 +34,7 @@ namespace examples
             const string secretKey = "yourSecretKey";
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "foobar.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
diff --git a/csharp/examples/ProducerFifoMessageExample.cs b/csharp/examples/ProducerFifoMessageExample.cs
index 87f953c3..bfca32f5 100644
--- a/csharp/examples/ProducerFifoMessageExample.cs
+++ b/csharp/examples/ProducerFifoMessageExample.cs
@@ -30,12 +30,12 @@ namespace examples
 
         internal static async Task QuickStart()
         {
-            const string accessKey = "5jFk0wK7OU6Uq395";
-            const string secretKey = "V1u8z19URHs4o6RQ";
+            const string accessKey = "yourAccessKey";
+            const string secretKey = "yourSecretKey";
 
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "foobar.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
@@ -43,7 +43,7 @@ namespace examples
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
 
-            const string topic = "lingchu_fifo_topic";
+            const string topic = "yourFifoTopic";
             producer.SetTopics(topic);
             // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
             // the topic route before message publishing.
diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index 3274ade2..09e4dff1 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -30,12 +30,12 @@ namespace examples
 
         internal static async Task QuickStart()
         {
-            const string accessKey = "5jFk0wK7OU6Uq395";
-            const string secretKey = "V1u8z19URHs4o6RQ";
+            const string accessKey = "yourAccessKey";
+            const string secretKey = "yourSecretKey";
 
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "foobar.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
@@ -43,7 +43,7 @@ namespace examples
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
 
-            const string topic = "lingchu_normal_topic";
+            const string topic = "yourNormalTopic";
             producer.SetTopics(topic);
             // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
             // the topic route before message publishing.
diff --git a/csharp/examples/ProducerTransactionMessageExample.cs b/csharp/examples/ProducerTransactionMessageExample.cs
index 8b331722..10b61142 100644
--- a/csharp/examples/ProducerTransactionMessageExample.cs
+++ b/csharp/examples/ProducerTransactionMessageExample.cs
@@ -41,7 +41,7 @@ namespace examples
             const string secretKey = "yourSecretKey";
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "foobar.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
@@ -49,7 +49,7 @@ namespace examples
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
 
-            const string topic = "lingchu_transactional_topic";
+            const string topic = "yourTransactionTopic";
             producer.SetTopics(topic);
             producer.SetTransactionChecker(new TransactionChecker());
 
diff --git a/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
index 094c2607..ddc4d281 100644
--- a/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
+++ b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
@@ -1,5 +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.
+ */
+
 using System;
-using Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using Google.Protobuf.WellKnownTypes;
 
 namespace Org.Apache.Rocketmq
@@ -28,9 +45,28 @@ namespace Org.Apache.Rocketmq
 
         public double BackoffMultiplier { get; }
 
+        public IRetryPolicy InheritBackoff(Proto.RetryPolicy retryPolicy)
+        {
+            if (retryPolicy.StrategyCase != Proto.RetryPolicy.StrategyOneofCase.ExponentialBackoff)
+            {
+                throw new InvalidOperationException("Strategy must be exponential backoff");
+            }
+
+            return InheritBackoff(retryPolicy.ExponentialBackoff);
+        }
+
+        private IRetryPolicy InheritBackoff(Proto.ExponentialBackoff retryPolicy)
+        {
+            return new ExponentialBackoffRetryPolicy(_maxAttempts, retryPolicy.Initial.ToTimeSpan(),
+                retryPolicy.Max.ToTimeSpan(), retryPolicy.Multiplier);
+        }
+
         public TimeSpan GetNextAttemptDelay(int attempt)
         {
-            return TimeSpan.Zero;
+            var delayMillis = Math.Min(
+                InitialBackoff.TotalMilliseconds * Math.Pow(BackoffMultiplier, 1.0 * (attempt - 1)),
+                MaxBackoff.TotalMilliseconds);
+            return delayMillis < 0 ? TimeSpan.Zero : TimeSpan.FromMilliseconds(delayMillis);
         }
 
         public static ExponentialBackoffRetryPolicy ImmediatelyRetryPolicy(int maxAttempts)
@@ -40,7 +76,7 @@ namespace Org.Apache.Rocketmq
 
         public global::Apache.Rocketmq.V2.RetryPolicy ToProtobuf()
         {
-            var exponentialBackoff = new ExponentialBackoff
+            var exponentialBackoff = new Proto.ExponentialBackoff
             {
                 Multiplier = (float)BackoffMultiplier,
                 Max = Duration.FromTimeSpan(MaxBackoff),
diff --git a/csharp/rocketmq-client-csharp/IRetryPolicy.cs b/csharp/rocketmq-client-csharp/IRetryPolicy.cs
index c006b1bd..86d280eb 100644
--- a/csharp/rocketmq-client-csharp/IRetryPolicy.cs
+++ b/csharp/rocketmq-client-csharp/IRetryPolicy.cs
@@ -17,6 +17,7 @@
 
 using System;
 using Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -43,5 +44,7 @@ namespace Org.Apache.Rocketmq
         /// </summary>
         /// <returns></returns>
         RetryPolicy ToProtobuf();
+
+        IRetryPolicy InheritBackoff(Proto.RetryPolicy retryPolicy);
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/PublishingSettings.cs b/csharp/rocketmq-client-csharp/PublishingSettings.cs
index b9f8f454..17830a00 100644
--- a/csharp/rocketmq-client-csharp/PublishingSettings.cs
+++ b/csharp/rocketmq-client-csharp/PublishingSettings.cs
@@ -19,18 +19,21 @@ using System;
 using System.Collections.Concurrent;
 using System.Linq;
 using Google.Protobuf.WellKnownTypes;
+using NLog;
 using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class PublishingSettings : Settings
     {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
         private volatile int _maxBodySizeBytes = 4 * 1024 * 1024;
         private volatile bool _validateMessageType = true;
 
         public PublishingSettings(string clientId, Endpoints endpoints, ExponentialBackoffRetryPolicy retryPolicy,
-            TimeSpan requestTimeout, ConcurrentDictionary<string, bool> topics) : base(clientId, ClientType.Producer, endpoints,
-            retryPolicy, requestTimeout)
+            TimeSpan requestTimeout, ConcurrentDictionary<string, bool> topics) : base(clientId, ClientType.Producer,
+            endpoints, retryPolicy, requestTimeout)
         {
             Topics = topics;
         }
@@ -49,7 +52,16 @@ namespace Org.Apache.Rocketmq
 
         public override void Sync(Proto::Settings settings)
         {
-            // TODO
+            if (Proto.Settings.PubSubOneofCase.Publishing != settings.PubSubCase)
+            {
+                Logger.Error($"[Bug] Issued settings does not match with the client type, clientId={ClientId}, " +
+                             $"pubSubCase={settings.PubSubCase}, clientType={ClientType}");
+                return;
+            }
+
+            RetryPolicy = RetryPolicy.InheritBackoff(settings.BackoffPolicy);
+            _validateMessageType = settings.Publishing.ValidateMessageType;
+            _maxBodySizeBytes = settings.Publishing.MaxBodySize;
         }
 
         public override Proto.Settings ToProtobuf()
diff --git a/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs b/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
index a6a409d7..c83cca7d 100644
--- a/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
+++ b/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
@@ -44,7 +44,11 @@ namespace Org.Apache.Rocketmq
 
         public override void Sync(Proto::Settings settings)
         {
-            // TODO
+            if (Proto.Settings.PubSubOneofCase.Subscription != settings.PubSubCase)
+            {
+                Logger.Error($"[Bug] Issued settings doesn't match with the client type, clientId={ClientId}, " +
+                             $"pubSubCase={settings.PubSubCase}, clientType={ClientType}");
+            }
         }
 
         public override Proto.Settings ToProtobuf()


[rocketmq-clients] 13/28: Add error log for scheduled task

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 6cde562e368dbe947a009fc4cba3bec252a2b649
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 15 15:54:03 2023 +0800

    Add error log for scheduled task
---
 csharp/examples/ProducerBenchmark.cs               | 12 ++--
 csharp/examples/SimpleConsumerExample.cs           |  3 +-
 csharp/rocketmq-client-csharp/Client.cs            | 84 ++++++++++++++--------
 .../ClientLoggerInterceptor.cs                     | 14 +---
 csharp/rocketmq-client-csharp/Session.cs           |  1 +
 5 files changed, 68 insertions(+), 46 deletions(-)

diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerBenchmark.cs
index 8ad03847..4e334104 100644
--- a/csharp/examples/ProducerBenchmark.cs
+++ b/csharp/examples/ProducerBenchmark.cs
@@ -34,12 +34,12 @@ namespace examples
 
         internal static void QuickStart()
         {
-            const string accessKey = "5jFk0wK7OU6Uq395";
-            const string secretKey = "V1u8z19URHs4o6RQ";
+            const string accessKey = "amKhwEM40L61znSz";
+            const string secretKey = "bT6c3gpF3EFB10F3";
 
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            const string endpoints = "rmq-cn-nwy337bf81g.cn-hangzhou.rmq.aliyuncs.com:8080";
             var clientConfig = new ClientConfig(endpoints)
             {
                 CredentialsProvider = credentialsProvider
@@ -68,14 +68,14 @@ namespace examples
                 Keys = keys
             };
 
-            const int tpsLimit = 1000;
+            const int tpsLimit = 800;
 
             Task.Run(async () =>
             {
                 while (true)
                 {
-                    _semaphore.Release(tpsLimit/1000);
-                    await Task.Delay(TimeSpan.FromMilliseconds(1));
+                    _semaphore.Release(tpsLimit);
+                    await Task.Delay(TimeSpan.FromMilliseconds(1000));
                 }
             });
 
diff --git a/csharp/examples/SimpleConsumerExample.cs b/csharp/examples/SimpleConsumerExample.cs
index b41125c8..fa78e845 100644
--- a/csharp/examples/SimpleConsumerExample.cs
+++ b/csharp/examples/SimpleConsumerExample.cs
@@ -41,8 +41,9 @@ namespace examples
             };
             // Add your subscriptions.
             const string consumerGroup = "yourConsumerGroup";
+            const string topic = "yourTopic";
             var subscription = new Dictionary<string, FilterExpression>
-                { { consumerGroup, new FilterExpression("*") } };
+                { { topic, new FilterExpression("*") } };
             // In most case, you don't need to create too many consumers, single pattern is recommended.
             var simpleConsumer =
                 new SimpleConsumer(clientConfig, consumerGroup, TimeSpan.FromSeconds(15), subscription);
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 4849857b..89208db5 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -191,19 +191,33 @@ namespace Org.Apache.Rocketmq
 
         private async void UpdateTopicRouteCache()
         {
-            Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
-            foreach (var topic in GetTopics())
+            try
             {
-                await FetchTopicRoute(topic);
+                Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
+                foreach (var topic in GetTopics())
+                {
+                    await FetchTopicRoute(topic);
+                }
+            }
+            catch (Exception e)
+            {
+                Logger.Error(e, $"[Bug] unexpected exception raised during topic route cache update, clientId={ClientId}");
             }
         }
 
         private async void SyncSettings()
         {
-            var totalRouteEndpoints = GetTotalRouteEndpoints();
-            foreach (var (_, session) in totalRouteEndpoints.Select(GetSession))
+            try
             {
-                await session.SyncSettings(false);
+                var totalRouteEndpoints = GetTotalRouteEndpoints();
+                foreach (var (_, session) in totalRouteEndpoints.Select(GetSession))
+                {
+                    await session.SyncSettings(false);
+                }
+            }
+            catch (Exception e)
+            {
+                Logger.Error(e, $"[Bug] unexpected exception raised during setting sync, clientId={ClientId}");
             }
         }
 
@@ -279,36 +293,50 @@ namespace Org.Apache.Rocketmq
 
         private async void Heartbeat()
         {
-            var endpoints = GetTotalRouteEndpoints();
-            var request = WrapHeartbeatRequest();
-            Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new();
-            // Collect task into a map.
-            foreach (var item in endpoints)
+            try
             {
-                var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
-                responses[item] = task;
-            }
+                var endpoints = GetTotalRouteEndpoints();
+                var request = WrapHeartbeatRequest();
+                Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new();
+
+                // Collect task into a map.
+                foreach (var item in endpoints)
+                {
+                    try
+                    {
+                        var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
+                        responses[item] = task;
+                    }
+                    catch (Exception e)
+                    {
+                        Logger.Error(e, $"Failed to send heartbeat, endpoints={item}");
+                    }
+                }
 
-            foreach (var item in responses.Keys)
-            {
-                var response = await responses[item];
-                var code = response.Status.Code;
 
-                if (code.Equals(Proto.Code.Ok))
+                foreach (var item in responses.Keys)
                 {
-                    Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
-                    if (Isolated.TryRemove(item, out _))
+                    var response = await responses[item];
+                    var code = response.Status.Code;
+
+                    if (code.Equals(Proto.Code.Ok))
                     {
-                        Logger.Info(
-                            $"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
+                        Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
+                        if (Isolated.TryRemove(item, out _))
+                        {
+                            Logger.Info($"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
+                        }
+
+                        return;
                     }
 
-                    return;
+                    var statusMessage = response.Status.Message;
+                    Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
                 }
-
-                var statusMessage = response.Status.Message;
-                Logger.Info(
-                    $"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
+            }
+            catch (Exception e)
+            {
+                Logger.Error(e, $"[Bug] unexpected exception raised during heartbeat, clientId={ClientId}");
             }
         }
 
diff --git a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
index 890ce877..6f990de7 100644
--- a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
+++ b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
@@ -54,17 +54,9 @@ namespace Org.Apache.Rocketmq
 
         private async Task<TResponse> HandleResponse<TResponse>(Task<TResponse> t)
         {
-            try
-            {
-                var response = await t;
-                Logger.Trace($"Response received: {response}");
-                return response;
-            }
-            catch (Exception ex)
-            {
-                Logger.Error($"Call error: {ex.Message}");
-                throw;
-            }
+            var response = await t;
+            Logger.Trace($"Response received: {response}");
+            return response;
         }
 
         public override AsyncClientStreamingCall<TRequest, TResponse> AsyncClientStreamingCall<TRequest, TResponse>(
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index dd3da7bd..c948ef86 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -56,6 +56,7 @@ namespace Org.Apache.Rocketmq
 
         public async Task SyncSettings(bool awaitResp)
         {
+            // TODO
             await _semaphore.WaitAsync();
             try
             {


[rocketmq-clients] 25/28: Apply state machine in transactional message

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 15007046eae023bf6320ec1605fbb3f02845d589
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 22 18:44:31 2023 +0800

    Apply state machine in transactional message
---
 csharp/examples/ProducerTransactionMessageExample.cs |  2 +-
 csharp/rocketmq-client-csharp/Client.cs              |  2 +-
 csharp/rocketmq-client-csharp/ITransaction.cs        |  4 ++--
 csharp/rocketmq-client-csharp/Producer.cs            | 13 +++++++++++--
 csharp/rocketmq-client-csharp/SimpleConsumer.cs      | 15 +++++++++++++++
 csharp/rocketmq-client-csharp/Transaction.cs         | 14 ++++++++++++--
 6 files changed, 42 insertions(+), 8 deletions(-)

diff --git a/csharp/examples/ProducerTransactionMessageExample.cs b/csharp/examples/ProducerTransactionMessageExample.cs
index edc4d41f..8b331722 100644
--- a/csharp/examples/ProducerTransactionMessageExample.cs
+++ b/csharp/examples/ProducerTransactionMessageExample.cs
@@ -73,7 +73,7 @@ namespace examples
             var sendReceipt = await producer.Send(message, transaction);
             Logger.Info("Send transaction message successfully, messageId={}", sendReceipt.MessageId);
             // Commit the transaction.
-            transaction.commit();
+            transaction.Commit();
             // Or rollback the transaction.
             // transaction.rollback();
             // Close the producer if you don't need it anymore.
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index a9cd093a..305383e2 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -58,7 +58,7 @@ namespace Org.Apache.Rocketmq
         private readonly Dictionary<Endpoints, Session> _sessionsTable;
         private readonly ReaderWriterLockSlim _sessionLock;
 
-        protected volatile State State;
+        internal volatile State State;
 
         protected Client(ClientConfig clientConfig)
         {
diff --git a/csharp/rocketmq-client-csharp/ITransaction.cs b/csharp/rocketmq-client-csharp/ITransaction.cs
index b9898de0..27c770b1 100644
--- a/csharp/rocketmq-client-csharp/ITransaction.cs
+++ b/csharp/rocketmq-client-csharp/ITransaction.cs
@@ -19,8 +19,8 @@ namespace Org.Apache.Rocketmq
 {
     public interface ITransaction
     {
-        void commit();
+        void Commit();
 
-        void rollback();
+        void Rollback();
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index d376d14c..838263a2 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -179,12 +179,20 @@ namespace Org.Apache.Rocketmq
 
         public async Task<SendReceipt> Send(Message message)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Producer is not running");
+            }
             var sendReceipt = await Send(message, false);
             return sendReceipt;
         }
 
         public async Task<SendReceipt> Send(Message message, ITransaction transaction)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Producer is not running");
+            }
             var tx = (Transaction)transaction;
             var publishingMessage = tx.TryAddMessage(message);
             var sendReceipt = await Send(message, true);
@@ -223,8 +231,9 @@ namespace Org.Apache.Rocketmq
                 var sendReceipt = sendReceipts.First();
                 if (attempt > 1)
                 {
-                    Logger.Info($"Re-send message successfully, topic={message.Topic}, messageId={sendReceipt.MessageId}," +
-                                $" maxAttempts={maxAttempts}, endpoints={endpoints}, clientId={ClientId}");
+                    Logger.Info(
+                        $"Re-send message successfully, topic={message.Topic}, messageId={sendReceipt.MessageId}," +
+                        $" maxAttempts={maxAttempts}, endpoints={endpoints}, clientId={ClientId}");
                 }
 
                 return sendReceipt;
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index bf9614e1..d25dceab 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -157,6 +157,11 @@ namespace Org.Apache.Rocketmq
 
         public async Task<List<MessageView>> Receive(int maxMessageNum, TimeSpan invisibleDuration)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Simple consumer is not running");
+            }
+
             if (maxMessageNum <= 0)
             {
                 throw new InternalErrorException("maxMessageNum must be greater than 0");
@@ -182,6 +187,11 @@ namespace Org.Apache.Rocketmq
 
         public async void ChangeInvisibleDuration(MessageView messageView, TimeSpan invisibleDuration)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Simple consumer is not running");
+            }
+
             var request = WrapChangeInvisibleDuration(messageView, invisibleDuration);
             var response = await ClientManager.ChangeInvisibleDuration(messageView.MessageQueue.Broker.Endpoints,
                 request, ClientConfig.RequestTimeout);
@@ -191,6 +201,11 @@ namespace Org.Apache.Rocketmq
 
         public async Task Ack(MessageView messageView)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Simple consumer is not running");
+            }
+
             var request = WrapAckMessageRequest(messageView);
             var response = await ClientManager.AckMessage(messageView.MessageQueue.Broker.Endpoints, request,
                 ClientConfig.RequestTimeout);
diff --git a/csharp/rocketmq-client-csharp/Transaction.cs b/csharp/rocketmq-client-csharp/Transaction.cs
index e44c0675..5084ae4e 100644
--- a/csharp/rocketmq-client-csharp/Transaction.cs
+++ b/csharp/rocketmq-client-csharp/Transaction.cs
@@ -90,8 +90,13 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public async void commit()
+        public async void Commit()
         {
+            if (State.Running != _producer.State)
+            {
+                throw new InvalidOperationException("Producer is not running");
+            }
+            
             if (_messageSendReceiptDict.IsEmpty)
             {
                 throw new ArgumentException("Transactional message has not been sent yet");
@@ -104,8 +109,13 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public async void rollback()
+        public async void Rollback()
         {
+            if (State.Running != _producer.State)
+            {
+                throw new InvalidOperationException("Producer is not running");
+            }
+            
             if (_messageSendReceiptDict.IsEmpty)
             {
                 throw new ArgumentException("Transaction message has not been sent yet");


[rocketmq-clients] 22/28: Polish code

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit b2ee3b311306d08a5e11c72642bb43db92463beb
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Tue Feb 21 14:27:01 2023 +0800

    Polish code
---
 csharp/rocketmq-client-csharp/Endpoints.cs | 53 +++++++++++++++++++++++-------
 csharp/tests/EndpointsTest.cs              | 18 ++++++++++
 2 files changed, 59 insertions(+), 12 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Endpoints.cs b/csharp/rocketmq-client-csharp/Endpoints.cs
index 54d8f0d2..27130a33 100644
--- a/csharp/rocketmq-client-csharp/Endpoints.cs
+++ b/csharp/rocketmq-client-csharp/Endpoints.cs
@@ -24,6 +24,10 @@ namespace Org.Apache.Rocketmq
 {
     public class Endpoints : IEquatable<Endpoints>
     {
+        private const string HttpPrefix = "http://";
+        private const string HttpsPrefix = "https://";
+        private const int DefaultPort = 80;
+
         private static readonly AddressListEqualityComparer AddressListComparer = new();
         private const string EndpointSeparator = ":";
         private List<Address> Addresses { get; }
@@ -72,16 +76,43 @@ namespace Org.Apache.Rocketmq
             }
         }
 
+        // TODO: Multiple addresses has not been supported yet.
         public Endpoints(string endpoints)
         {
-            // TODO
-            var strs = endpoints.Split(EndpointSeparator);
+            if (endpoints.StartsWith(HttpPrefix))
+            {
+                endpoints = endpoints[HttpPrefix.Length..];
+            }
+
+            if (endpoints.StartsWith(HttpsPrefix))
+            {
+                endpoints = endpoints[HttpsPrefix.Length..];
+            }
+
+            var index = endpoints.IndexOf(EndpointSeparator, StringComparison.Ordinal);
+            var port = index > 0 ? int.Parse(endpoints[(1 + index)..]) : DefaultPort;
+            var host = index > 0 ? endpoints.Substring(0, index) : endpoints;
+
+            var uriHostNameType = Uri.CheckHostName(host);
             Scheme = AddressScheme.DomainName;
-            string host = strs[0];
-            int port = int.Parse(strs[1]);
-            Address address = new Address(host, port);
-            var addresses = new List<Address>();
-            addresses.Add(address);
+            switch (uriHostNameType)
+            {
+                case UriHostNameType.IPv4:
+                    Scheme = AddressScheme.Ipv4;
+                    break;
+                case UriHostNameType.IPv6:
+                    Scheme = AddressScheme.Ipv6;
+                    break;
+                case UriHostNameType.Dns:
+                case UriHostNameType.Basic:
+                case UriHostNameType.Unknown:
+                default:
+                    Scheme = AddressScheme.DomainName;
+                    break;
+            }
+
+            var address = new Address(host, port);
+            var addresses = new List<Address> { address };
             Addresses = addresses;
         }
 
@@ -90,22 +121,20 @@ namespace Org.Apache.Rocketmq
             return GrpcTarget;
         }
 
+        // TODO: Support non-TLS and multiple addresses.
         public string GrpcTarget
         {
-            // TODO
             get
             {
                 foreach (var address in Addresses)
                 {
-                    var target = "https://" + address.Host + ":" + address.Port;
-                    // Console.WriteLine(target);
-                    return "https://" + address.Host + ":" + address.Port;
+                    return HttpsPrefix + address.Host + EndpointSeparator + address.Port;
                 }
 
                 return "";
             }
         }
-        
+
         public bool Equals(Endpoints other)
         {
             if (ReferenceEquals(null, other))
diff --git a/csharp/tests/EndpointsTest.cs b/csharp/tests/EndpointsTest.cs
new file mode 100644
index 00000000..e68274b8
--- /dev/null
+++ b/csharp/tests/EndpointsTest.cs
@@ -0,0 +1,18 @@
+using System;
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Org.Apache.Rocketmq;
+
+namespace tests
+{
+    [TestClass]
+    public class EndpointsTest
+    {
+        [TestMethod]
+        public void testConstructor()
+        {
+            Console.WriteLine(Uri.CheckHostName("127.0.0.1"));
+            Console.WriteLine(Uri.CheckHostName("1050:0000:0000:0000:0005:0600:300c:326b"));
+            Console.WriteLine(Uri.CheckHostName("baidu.com"));
+        }
+    }
+}
\ No newline at end of file


[rocketmq-clients] 28/28: Add license header

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 520bd4f2c12d20d7385a581925e87070f6fb32d5
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Thu Feb 23 23:11:13 2023 +0800

    Add license header
---
 csharp/rocketmq-client-csharp/Resource.cs  | 17 +++++++++++++++++
 csharp/rocketmq-client-csharp/UserAgent.cs | 17 +++++++++++++++++
 csharp/tests/EndpointsTest.cs              | 17 +++++++++++++++++
 3 files changed, 51 insertions(+)

diff --git a/csharp/rocketmq-client-csharp/Resource.cs b/csharp/rocketmq-client-csharp/Resource.cs
index 76c0ba9e..3395a16f 100644
--- a/csharp/rocketmq-client-csharp/Resource.cs
+++ b/csharp/rocketmq-client-csharp/Resource.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using Proto = Apache.Rocketmq.V2;
 
diff --git a/csharp/rocketmq-client-csharp/UserAgent.cs b/csharp/rocketmq-client-csharp/UserAgent.cs
index 7b589212..b457b608 100644
--- a/csharp/rocketmq-client-csharp/UserAgent.cs
+++ b/csharp/rocketmq-client-csharp/UserAgent.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using Apache.Rocketmq.V2;
 
diff --git a/csharp/tests/EndpointsTest.cs b/csharp/tests/EndpointsTest.cs
index e68274b8..6c4a6c71 100644
--- a/csharp/tests/EndpointsTest.cs
+++ b/csharp/tests/EndpointsTest.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using Microsoft.VisualStudio.TestTools.UnitTesting;
 using Org.Apache.Rocketmq;


[rocketmq-clients] 16/28: Implement Producer#send with transaction

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit c9e3d21fe927c87358c15226534e2a6102723c8f
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Thu Feb 16 16:42:50 2023 +0800

    Implement Producer#send with transaction
---
 csharp/rocketmq-client-csharp/Producer.cs          | 43 ++++++++++++++--------
 csharp/rocketmq-client-csharp/PublishingMessage.cs |  2 +-
 2 files changed, 29 insertions(+), 16 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 23041e4c..f9825d32 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -123,12 +123,17 @@ namespace Org.Apache.Rocketmq
             return PublishingSettings.GetRetryPolicy();
         }
 
-        public async Task<SendReceipt> Send(Message message)
+        private async Task<SendReceipt> Send(Message message, bool txEnabled)
         {
             var publishingLoadBalancer = await GetPublishingLoadBalancer(message.Topic);
-            var publishingMessage = new PublishingMessage(message, PublishingSettings, false);
+            var publishingMessage = new PublishingMessage(message, PublishingSettings, txEnabled);
             var retryPolicy = GetRetryPolicy();
             var maxAttempts = retryPolicy.GetMaxAttempts();
+            if (MessageType.Transaction == publishingMessage.MessageType)
+            {
+                // No more retries for transactional message.
+                maxAttempts = 1;
+            }
 
             // Prepare the candidate message queue(s) for retry-sending in advance.
             var candidates = null == publishingMessage.MessageGroup
@@ -152,10 +157,19 @@ namespace Org.Apache.Rocketmq
             throw exception!;
         }
 
-        public async Task<SendReceipt> Send(Message message, Transaction transaction)
+        public async Task<SendReceipt> Send(Message message)
+        {
+            var sendReceipt = await Send(message, false);
+            return sendReceipt;
+        }
+
+        public async Task<SendReceipt> Send(Message message, ITransaction transaction)
         {
-            // TODO
-            return null;
+            var tx = (Transaction) transaction;
+            var publishingMessage = tx.TryAddMessage(message);
+            var sendReceipt = await Send(message, true);
+            tx.TryAddReceipt(publishingMessage, sendReceipt);
+            return sendReceipt;
         }
 
         private static Proto.SendMessageRequest WrapSendMessageRequest(PublishingMessage message, MessageQueue mq)
@@ -181,8 +195,7 @@ namespace Org.Apache.Rocketmq
 
             var sendMessageRequest = WrapSendMessageRequest(message, mq);
             var endpoints = mq.Broker.Endpoints;
-            var response =
-                await ClientManager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
+            var response = await ClientManager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
             try
             {
                 var sendReceipts = SendReceipt.ProcessSendMessageResponse(mq, response);
@@ -190,9 +203,8 @@ namespace Org.Apache.Rocketmq
                 var sendReceipt = sendReceipts.First();
                 if (attempt > 1)
                 {
-                    Logger.Info(
-                        $"Re-send message successfully, topic={message.Topic}, messageId={sendReceipt.MessageId}," +
-                        $" maxAttempts={maxAttempts}, endpoints={endpoints}, clientId={ClientId}");
+                    Logger.Info($"Re-send message successfully, topic={message.Topic}, messageId={sendReceipt.MessageId}," +
+                                $" maxAttempts={maxAttempts}, endpoints={endpoints}, clientId={ClientId}");
                 }
 
                 return sendReceipt;
@@ -203,14 +215,15 @@ namespace Org.Apache.Rocketmq
                 Isolated[endpoints] = true;
                 if (attempt >= maxAttempts)
                 {
-                    Logger.Error(
-                        $"Failed to send message finally, run out of attempt times, topic={message.Topic}, " +
-                        $"maxAttempt={maxAttempts}, attempt={attempt}, endpoints={endpoints}, clientId={ClientId}");
+                    Logger.Error("Failed to send message finally, run out of attempt times, " +
+                                 $"topic={message.Topic}, maxAttempt={maxAttempts}, attempt={attempt}, " +
+                                 $"endpoints={endpoints}, messageId={message.MessageId}, clientId={ClientId}");
                     throw;
                 }
 
                 Logger.Warn(e, $"Failed to send message, topic={message.Topic}, maxAttempts={maxAttempts}, " +
-                               $"attempt={attempt}, endpoints={endpoints}, clientId={ClientId}");
+                               $"attempt={attempt}, endpoints={endpoints}, messageId={message.MessageId}," +
+                               $" clientId={ClientId}");
                 throw;
             }
         }
@@ -246,7 +259,7 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public Transaction BeginTransaction()
+        public ITransaction BeginTransaction()
         {
             return new Transaction(this);
         }
diff --git a/csharp/rocketmq-client-csharp/PublishingMessage.cs b/csharp/rocketmq-client-csharp/PublishingMessage.cs
index 7839edaa..c5b4b22c 100644
--- a/csharp/rocketmq-client-csharp/PublishingMessage.cs
+++ b/csharp/rocketmq-client-csharp/PublishingMessage.cs
@@ -31,7 +31,7 @@ namespace Org.Apache.Rocketmq
     {
         public MessageType MessageType { set; get; }
 
-        private string MessageId { get; }
+        internal string MessageId { get; }
 
         public PublishingMessage(Message message, PublishingSettings publishingSettings, bool txEnabled) : base(
             message.Topic, message.Body)


[rocketmq-clients] 12/28: Notify remote endpoints that current client is terminated

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 7ce29d7d70d2b9e63dc94c35d53ca3dcf902df26
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 20:38:04 2023 +0800

    Notify remote endpoints that current client is terminated
---
 csharp/rocketmq-client-csharp/Client.cs         | 13 +++++++------
 csharp/rocketmq-client-csharp/Producer.cs       |  5 +++++
 csharp/rocketmq-client-csharp/SimpleConsumer.cs |  8 ++++++++
 3 files changed, 20 insertions(+), 6 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 9d4d7e69..4849857b 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -94,6 +94,7 @@ namespace Org.Apache.Rocketmq
             _topicRouteUpdateCtx.Cancel();
             _heartbeatCts.Cancel();
             _telemetryCts.Cancel();
+            NotifyClientTermination();
             await ClientManager.Shutdown();
             Logger.Debug($"Shutdown the rocketmq client successfully, clientId={ClientId}");
         }
@@ -299,7 +300,7 @@ namespace Org.Apache.Rocketmq
                     if (Isolated.TryRemove(item, out _))
                     {
                         Logger.Info(
-                            $"Rejoin endpoints which was isolate before, endpoints={item}, clientId={ClientId}");
+                            $"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
                     }
 
                     return;
@@ -319,13 +320,13 @@ namespace Org.Apache.Rocketmq
             return metadata;
         }
 
-        public async void NotifyClientTermination(Proto.Resource group)
+        protected abstract Proto::NotifyClientTerminationRequest WrapNotifyClientTerminationRequest();
+
+        private async void NotifyClientTermination()
         {
+            Logger.Info($"Notify remote endpoints that current client is terminated, clientId={ClientId}");
             var endpoints = GetTotalRouteEndpoints();
-            var request = new Proto::NotifyClientTerminationRequest
-            {
-                Group = group
-            };
+            var request = WrapNotifyClientTerminationRequest();
             foreach (var item in endpoints)
             {
                 var response = await ClientManager.NotifyClientTermination(item, request, ClientConfig.RequestTimeout);
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 170dbeab..4bbc4bfa 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -88,6 +88,11 @@ namespace Org.Apache.Rocketmq
             };
         }
 
+        protected override Proto::NotifyClientTerminationRequest WrapNotifyClientTerminationRequest()
+        {
+            return new Proto::NotifyClientTerminationRequest();
+        }
+
         private async Task<PublishingLoadBalancer> GetPublishingLoadBalancer(string topic)
         {
             if (_publishingRouteDataCache.TryGetValue(topic, out var publishingLoadBalancer))
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index 8a8922fd..cdd5aa64 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -84,6 +84,14 @@ namespace Org.Apache.Rocketmq
             return _subscriptionExpressions.Keys;
         }
 
+        protected override Proto.NotifyClientTerminationRequest WrapNotifyClientTerminationRequest()
+        {
+            return new Proto.NotifyClientTerminationRequest()
+            {
+                Group = GetProtobufGroup()
+            };
+        }
+
         protected override Proto.HeartbeatRequest WrapHeartbeatRequest()
         {
             return new Proto::HeartbeatRequest


[rocketmq-clients] 03/28: Modify log level temporarily

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 57531fab6e68dc73764af2c62a8553a178dbede9
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Tue Feb 7 17:43:07 2023 +0800

    Modify log level temporarily
---
 csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog b/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
index a03537bc..5da3a600 100644
--- a/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
+++ b/csharp/rocketmq-client-csharp/rocketmq-client-csharp.nlog
@@ -35,7 +35,7 @@
         </target>
     </targets>
     <rules>
-        <logger name="*" minlevel="info" writeTo="asyncFile" />
-        <logger name="*" minlevel="Info" writeTo="colorConsole" />
+        <logger name="*" minlevel="Debug" writeTo="asyncFile" />
+        <logger name="*" minlevel="Debug" writeTo="colorConsole" />
     </rules>
 </nlog>
\ No newline at end of file


[rocketmq-clients] 19/28: Polish code

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit d70721197113c66f063acd40efa64395569236d4
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Fri Feb 17 11:57:51 2023 +0800

    Polish code
---
 csharp/examples/ProducerBenchmark.cs          |  7 +--
 csharp/rocketmq-client-csharp/Client.cs       | 86 +++++++++++++++++----------
 csharp/rocketmq-client-csharp/MqLogManager.cs | 13 ++--
 csharp/rocketmq-client-csharp/Session.cs      |  9 ++-
 csharp/rocketmq-client-csharp/State.cs        | 24 ++++++++
 5 files changed, 90 insertions(+), 49 deletions(-)

diff --git a/csharp/examples/ProducerBenchmark.cs b/csharp/examples/ProducerBenchmark.cs
index 3918666d..6f94028e 100644
--- a/csharp/examples/ProducerBenchmark.cs
+++ b/csharp/examples/ProducerBenchmark.cs
@@ -68,14 +68,14 @@ namespace examples
                 Keys = keys
             };
 
-            const int tpsLimit = 500;
+            const int tpsLimit = 1;
 
             Task.Run(async () =>
             {
                 while (true)
                 {
                     _semaphore.Release(tpsLimit);
-                    await Task.Delay(TimeSpan.FromMilliseconds(1000));
+                    await Task.Delay(TimeSpan.FromSeconds(1));
                 }
             });
 
@@ -83,8 +83,7 @@ namespace examples
             {
                 while (true)
                 {
-                    Logger.Info($"Send {_counter} messages successfully.");
-                    Interlocked.Exchange(ref _counter, 0);
+                    Logger.Info($"Send {Interlocked.Exchange(ref _counter, 0)} messages successfully.");
                     await Task.Delay(TimeSpan.FromSeconds(1));
                 }
             });
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 9311b48a..5a2cd69f 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -40,11 +40,11 @@ namespace Org.Apache.Rocketmq
         private readonly CancellationTokenSource _topicRouteUpdateCts;
 
         private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromSeconds(1);
-        private static readonly TimeSpan SettingsSyncSchedulePeriod = TimeSpan.FromMinutes(5);
+        private static readonly TimeSpan SettingsSyncSchedulePeriod = TimeSpan.FromSeconds(1);
         private readonly CancellationTokenSource _settingsSyncCts;
         
-        private static readonly TimeSpan StatsScheduleDelay = TimeSpan.FromSeconds(1);
-        private static readonly TimeSpan StatsSchedulePeriod = TimeSpan.FromSeconds(1);
+        private static readonly TimeSpan StatsScheduleDelay = TimeSpan.FromSeconds(60);
+        private static readonly TimeSpan StatsSchedulePeriod = TimeSpan.FromSeconds(60);
         private readonly CancellationTokenSource _statsCts;
 
         protected readonly ClientConfig ClientConfig;
@@ -201,6 +201,26 @@ namespace Org.Apache.Rocketmq
             try
             {
                 Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
+                Dictionary<string, Task<TopicRouteData>> responses = new();
+                
+                foreach (var topic in GetTopics())
+                {
+                    var task = FetchTopicRoute(topic);
+                    responses[topic] = task;
+                }
+                
+                foreach (var item in responses.Keys)
+                {
+                    try
+                    {
+                        await responses[item];
+                    }
+                    catch (Exception e)
+                    {
+                        Logger.Error(e, $"Failed to update topic route cache, topic={item}");
+                    }
+                }
+                
                 foreach (var topic in GetTopics())
                 {
                     await FetchTopicRoute(topic);
@@ -208,8 +228,8 @@ namespace Org.Apache.Rocketmq
             }
             catch (Exception e)
             {
-                Logger.Error(e,
-                    $"[Bug] unexpected exception raised during topic route cache update, clientId={ClientId}");
+                Logger.Error(e, $"[Bug] unexpected exception raised during topic route cache update, " +
+                                $"clientId={ClientId}");
             }
         }
 
@@ -218,9 +238,12 @@ namespace Org.Apache.Rocketmq
             try
             {
                 var totalRouteEndpoints = GetTotalRouteEndpoints();
-                foreach (var (_, session) in totalRouteEndpoints.Select(GetSession))
+                foreach (var endpoints in totalRouteEndpoints)
                 {
+                    var (_, session) = GetSession(endpoints);
                     await session.SyncSettings(false);
+                    Logger.Info($"Sync settings to remote, endpoints={endpoints}");
+
                 }
             }
             catch (Exception e)
@@ -319,38 +342,37 @@ namespace Org.Apache.Rocketmq
                 // Collect task into a map.
                 foreach (var item in endpoints)
                 {
-                    try
-                    {
-                        var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
-                        responses[item] = task;
-                    }
-                    catch (Exception e)
-                    {
-                        Logger.Error(e, $"Failed to send heartbeat, endpoints={item}");
-                    }
+                    var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
+                    responses[item] = task;
                 }
 
-
                 foreach (var item in responses.Keys)
                 {
-                    var response = await responses[item];
-                    var code = response.Status.Code;
-
-                    if (code.Equals(Proto.Code.Ok))
+                    try
                     {
-                        Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
-                        if (Isolated.TryRemove(item, out _))
+                        var response = await responses[item];
+                        var code = response.Status.Code;
+
+                        if (code.Equals(Proto.Code.Ok))
                         {
-                            Logger.Info(
-                                $"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
+                            Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
+                            if (Isolated.TryRemove(item, out _))
+                            {
+                                Logger.Info($"Rejoin endpoints which was isolated before, endpoints={item}, " +
+                                            $"clientId={ClientId}");
+                            }
+
+                            return;
                         }
 
-                        return;
+                        var statusMessage = response.Status.Message;
+                        Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, " +
+                                    $"statusMessage={statusMessage}, clientId={ClientId}");
+                    }
+                    catch (Exception e)
+                    {
+                        Logger.Error(e, $"Failed to send heartbeat, endpoints={item}");
                     }
-
-                    var statusMessage = response.Status.Message;
-                    Logger.Info(
-                        $"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
                 }
             }
             catch (Exception e)
@@ -421,7 +443,7 @@ namespace Org.Apache.Rocketmq
                 Status = status
             };
             var (_, session) = GetSession(endpoints);
-            await session.write(telemetryCommand);
+            await session.WriteAsync(telemetryCommand);
         }
 
         public async void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
@@ -439,7 +461,7 @@ namespace Org.Apache.Rocketmq
                 Status = status
             };
             var (_, session) = GetSession(endpoints);
-            await session.write(telemetryCommand);
+            await session.WriteAsync(telemetryCommand);
         }
 
         public async void OnPrintThreadStackTraceCommand(Endpoints endpoints,
@@ -457,7 +479,7 @@ namespace Org.Apache.Rocketmq
                 Status = status
             };
             var (_, session) = GetSession(endpoints);
-            await session.write(telemetryCommand);
+            await session.WriteAsync(telemetryCommand);
         }
 
         public void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
diff --git a/csharp/rocketmq-client-csharp/MqLogManager.cs b/csharp/rocketmq-client-csharp/MqLogManager.cs
index 7fa2b7bf..6b117ea5 100644
--- a/csharp/rocketmq-client-csharp/MqLogManager.cs
+++ b/csharp/rocketmq-client-csharp/MqLogManager.cs
@@ -28,22 +28,19 @@ namespace Org.Apache.Rocketmq
      *
      * Configure component logging, please refer to https://github.com/NLog/NLog/wiki/Configure-component-logging
      */
-    public class MqLogManager
+    public static class MqLogManager
     {
-        public static LogFactory Instance
-        {
-            get { return LazyInstance.Value; }
-        }
+        public static LogFactory Instance => LazyInstance.Value;
 
         private static readonly Lazy<LogFactory> LazyInstance = new(BuildLogFactory);
 
         private static LogFactory BuildLogFactory()
         {
             // Use name of current assembly to construct NLog config filename 
-            Assembly thisAssembly = Assembly.GetExecutingAssembly();
-            string configFilePath = Path.ChangeExtension(thisAssembly.Location, ".nlog");
+            var thisAssembly = Assembly.GetExecutingAssembly();
+            var configFilePath = Path.ChangeExtension(thisAssembly.Location, ".nlog");
 
-            LogFactory logFactory = new LogFactory();
+            var logFactory = new LogFactory();
             logFactory.Configuration = new XmlLoggingConfiguration(configFilePath, logFactory);
             return logFactory;
         }
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index 0d35be0a..92465291 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -33,7 +33,7 @@ namespace Org.Apache.Rocketmq
 
         private static readonly TimeSpan SettingsInitializationTimeout = TimeSpan.FromSeconds(3);
 
-        private readonly grpc::AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand>
+        private readonly AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand>
             _streamingCall;
 
         private readonly IClient _client;
@@ -56,26 +56,25 @@ namespace Org.Apache.Rocketmq
             Loop();
         }
 
-        public async Task write(Proto.TelemetryCommand telemetryCommand)
+        public async Task WriteAsync(Proto.TelemetryCommand telemetryCommand)
         {
             var writer = _streamingCall.RequestStream;
             await writer.WriteAsync(telemetryCommand);
         }
 
+        // TODO: Test concurrency.
         public async Task SyncSettings(bool awaitResp)
         {
             // Add more buffer time.
             await _semaphore.WaitAsync(_client.GetClientConfig().RequestTimeout.Add(SettingsInitializationTimeout));
             try
             {
-                var writer = _streamingCall.RequestStream;
-                // await readTask;
                 var settings = _client.GetSettings();
                 var telemetryCommand = new Proto.TelemetryCommand
                 {
                     Settings = settings.ToProtobuf()
                 };
-                await writer.WriteAsync(telemetryCommand);
+                await WriteAsync(telemetryCommand);
                 // await writer.CompleteAsync();
                 if (awaitResp)
                 {
diff --git a/csharp/rocketmq-client-csharp/State.cs b/csharp/rocketmq-client-csharp/State.cs
new file mode 100644
index 00000000..1dbd6b30
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/State.cs
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Org.Apache.Rocketmq
+{
+    public enum State
+    {
+        
+    }
+}
\ No newline at end of file


[rocketmq-clients] 10/28: Add Client#GetTopics

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 70282e854e28643057cabe7fb32e30aa34922d96
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 20:23:19 2023 +0800

    Add Client#GetTopics
---
 csharp/rocketmq-client-csharp/Client.cs         | 53 +++++++++++++++----------
 csharp/rocketmq-client-csharp/Consumer.cs       |  4 +-
 csharp/rocketmq-client-csharp/Producer.cs       |  9 ++++-
 csharp/rocketmq-client-csharp/SimpleConsumer.cs | 10 +++--
 4 files changed, 47 insertions(+), 29 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 133fed00..afbfbe5b 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -31,21 +31,22 @@ namespace Org.Apache.Rocketmq
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        private static readonly TimeSpan HeartbeatScheduleDelay = TimeSpan.FromSeconds(10);
+        private static readonly TimeSpan HeartbeatScheduleDelay = TimeSpan.FromSeconds(1);
+        private static readonly TimeSpan HeartbeatSchedulePeriod = TimeSpan.FromSeconds(10);
         private readonly CancellationTokenSource _heartbeatCts;
 
-        private static readonly TimeSpan TopicRouteUpdateScheduleDelay = TimeSpan.FromSeconds(30);
+        private static readonly TimeSpan TopicRouteUpdateScheduleDelay = TimeSpan.FromSeconds(10);
+        private static readonly TimeSpan TopicRouteUpdateSchedulePeriod = TimeSpan.FromSeconds(30);
         private readonly CancellationTokenSource _topicRouteUpdateCtx;
 
-        private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromMinutes(5);
+        private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromSeconds(1);
+        private static readonly TimeSpan SettingsSyncSchedulePeriod = TimeSpan.FromMinutes(5);
         private readonly CancellationTokenSource _settingsSyncCtx;
 
         protected readonly ClientConfig ClientConfig;
         protected readonly IClientManager ClientManager;
         protected readonly string ClientId;
 
-        protected readonly ICollection<string> Topics;
-
         protected readonly ConcurrentDictionary<Endpoints, bool> Isolated;
         private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteCache;
         private readonly CancellationTokenSource _telemetryCts;
@@ -53,10 +54,9 @@ namespace Org.Apache.Rocketmq
         private readonly Dictionary<Endpoints, Session> _sessionsTable;
         private readonly ReaderWriterLockSlim _sessionLock;
 
-        protected Client(ClientConfig clientConfig, ICollection<string> topics)
+        protected Client(ClientConfig clientConfig)
         {
             ClientConfig = clientConfig;
-            Topics = topics;
             ClientId = Utilities.GetClientId();
 
             ClientManager = new ClientManager(this);
@@ -75,10 +75,12 @@ namespace Org.Apache.Rocketmq
         public virtual async Task Start()
         {
             Logger.Debug($"Begin to start the rocketmq client, clientId={ClientId}");
-            ScheduleWithFixedDelay(UpdateTopicRouteCache, TopicRouteUpdateScheduleDelay, _topicRouteUpdateCtx.Token);
-            ScheduleWithFixedDelay(Heartbeat, HeartbeatScheduleDelay, _heartbeatCts.Token);
-            ScheduleWithFixedDelay(SyncSettings, SettingsSyncScheduleDelay, _settingsSyncCtx.Token);
-            foreach (var topic in Topics)
+            ScheduleWithFixedDelay(UpdateTopicRouteCache, TopicRouteUpdateScheduleDelay, TopicRouteUpdateSchedulePeriod,
+                _topicRouteUpdateCtx.Token);
+            ScheduleWithFixedDelay(Heartbeat, HeartbeatScheduleDelay, HeartbeatSchedulePeriod, _heartbeatCts.Token);
+            ScheduleWithFixedDelay(SyncSettings, SettingsSyncScheduleDelay, SettingsSyncSchedulePeriod,
+                _settingsSyncCtx.Token);
+            foreach (var topic in GetTopics())
             {
                 await FetchTopicRoute(topic);
             }
@@ -132,6 +134,8 @@ namespace Org.Apache.Rocketmq
             }
         }
 
+        protected abstract ICollection<string> GetTopics();
+
         protected abstract Proto::HeartbeatRequest WrapHeartbeatRequest();
 
 
@@ -183,10 +187,9 @@ namespace Org.Apache.Rocketmq
         private async void UpdateTopicRouteCache()
         {
             Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
-            foreach (var topic in Topics)
+            foreach (var topic in GetTopics())
             {
-                var topicRouteData = await FetchTopicRoute(topic);
-                _topicRouteCache[topic] = topicRouteData;
+                await FetchTopicRoute(topic);
             }
         }
 
@@ -199,10 +202,11 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        private void ScheduleWithFixedDelay(Action action, TimeSpan period, CancellationToken token)
+        private void ScheduleWithFixedDelay(Action action, TimeSpan delay, TimeSpan period, CancellationToken token)
         {
             Task.Run(async () =>
             {
+                await Task.Delay(delay, token);
                 while (!token.IsCancellationRequested)
                 {
                     try
@@ -221,7 +225,18 @@ namespace Org.Apache.Rocketmq
             }, token);
         }
 
-        protected async Task<TopicRouteData> FetchTopicRoute(string topic)
+        protected async Task<TopicRouteData> GetRouteData(string topic)
+        {
+            if (_topicRouteCache.TryGetValue(topic, out var topicRouteData))
+            {
+                return topicRouteData;
+            }
+
+            topicRouteData = await FetchTopicRoute(topic);
+            return topicRouteData;
+        }
+
+        private async Task<TopicRouteData> FetchTopicRoute(string topic)
         {
             var topicRouteData = await FetchTopicRoute0(topic);
             await OnTopicRouteDataFetched(topic, topicRouteData);
@@ -260,12 +275,6 @@ namespace Org.Apache.Rocketmq
         private async void Heartbeat()
         {
             var endpoints = GetTotalRouteEndpoints();
-            if (0 == endpoints.Count)
-            {
-                Logger.Debug("No broker endpoints available in topic route");
-                return;
-            }
-
             var request = WrapHeartbeatRequest();
             Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new();
             // Collect task into a map.
diff --git a/csharp/rocketmq-client-csharp/Consumer.cs b/csharp/rocketmq-client-csharp/Consumer.cs
index 25df84d3..9a58104f 100644
--- a/csharp/rocketmq-client-csharp/Consumer.cs
+++ b/csharp/rocketmq-client-csharp/Consumer.cs
@@ -28,8 +28,8 @@ namespace Org.Apache.Rocketmq
     {
         protected readonly string ConsumerGroup;
 
-        protected Consumer(ClientConfig clientConfig, string consumerGroup, ICollection<string> topics) : base(
-            clientConfig, topics)
+        protected Consumer(ClientConfig clientConfig, string consumerGroup) : base(
+            clientConfig)
         {
             ConsumerGroup = consumerGroup;
         }
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index cc7794f6..7803e2aa 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -44,7 +44,7 @@ namespace Org.Apache.Rocketmq
 
         private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> publishingTopics,
             int maxAttempts) :
-            base(clientConfig, publishingTopics.Keys)
+            base(clientConfig)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
             _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
@@ -61,6 +61,11 @@ namespace Org.Apache.Rocketmq
             }
         }
 
+        protected override ICollection<string> GetTopics()
+        {
+            return _publishingTopics.Keys;
+        }
+
         public override async Task Start()
         {
             Logger.Info($"Begin to start the rocketmq producer, clientId={ClientId}");
@@ -90,7 +95,7 @@ namespace Org.Apache.Rocketmq
                 return publishingLoadBalancer;
             }
 
-            var topicRouteData = await FetchTopicRoute(topic);
+            var topicRouteData = await GetRouteData(topic);
             publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
             _publishingRouteDataCache.TryAdd(topic, publishingLoadBalancer);
 
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index cb380d89..0481f7e8 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -43,8 +43,7 @@ namespace Org.Apache.Rocketmq
         }
 
         private SimpleConsumer(ClientConfig clientConfig, string consumerGroup, TimeSpan awaitDuration,
-            ConcurrentDictionary<string, FilterExpression> subscriptionExpressions) : base(clientConfig, consumerGroup,
-            subscriptionExpressions.Keys)
+            ConcurrentDictionary<string, FilterExpression> subscriptionExpressions) : base(clientConfig, consumerGroup)
         {
             _awaitDuration = awaitDuration;
             _subscriptionRouteDataCache = new ConcurrentDictionary<string, SubscriptionLoadBalancer>();
@@ -79,6 +78,11 @@ namespace Org.Apache.Rocketmq
             await base.Shutdown();
             Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
         }
+        
+        protected override ICollection<string> GetTopics()
+        {
+            return _subscriptionExpressions.Keys;
+        }
 
         protected override Proto.HeartbeatRequest WrapHeartbeatRequest()
         {
@@ -106,7 +110,7 @@ namespace Org.Apache.Rocketmq
                 return subscriptionLoadBalancer;
             }
 
-            var topicRouteData = await FetchTopicRoute(topic);
+            var topicRouteData = await GetRouteData(topic);
             subscriptionLoadBalancer = new SubscriptionLoadBalancer(topicRouteData);
             _subscriptionRouteDataCache.TryAdd(topic, subscriptionLoadBalancer);
 


[rocketmq-clients] 05/28: Polish code

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 9c5d8858460e0676e302abcc3cc64733f737852b
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Tue Feb 7 20:00:08 2023 +0800

    Polish code
---
 csharp/examples/ProducerNormalMessageExample.cs    |   2 +-
 csharp/rocketmq-client-csharp/AccessPoint.cs       |  74 ------------
 csharp/rocketmq-client-csharp/Address.cs           |   3 +-
 .../AddressListEqualityComparer.cs                 |  17 +++
 csharp/rocketmq-client-csharp/AddressScheme.cs     |  17 +++
 csharp/rocketmq-client-csharp/Broker.cs            |  21 +++-
 csharp/rocketmq-client-csharp/Client.cs            |  47 +++++---
 csharp/rocketmq-client-csharp/ClientConfig.cs      |   8 --
 .../ClientLoggerInterceptor.cs                     |   1 +
 csharp/rocketmq-client-csharp/ClientManager.cs     |   1 -
 csharp/rocketmq-client-csharp/ClientType.cs        |  20 ++--
 .../ConfigFileCredentialsProvider.cs               |   1 +
 csharp/rocketmq-client-csharp/Endpoints.cs         |  34 ++++--
 .../ExponentialBackoffRetryPolicy.cs               |  14 +--
 csharp/rocketmq-client-csharp/IClient.cs           |  30 ++++-
 csharp/rocketmq-client-csharp/IClientConfig.cs     |  13 +-
 csharp/rocketmq-client-csharp/IClientManager.cs    |   6 +-
 .../rocketmq-client-csharp/ICredentialsProvider.cs |   1 +
 csharp/rocketmq-client-csharp/Message.cs           |  75 ++----------
 csharp/rocketmq-client-csharp/MessageException.cs  |  29 -----
 .../rocketmq-client-csharp/MessageIdGenerator.cs   |  26 ++--
 csharp/rocketmq-client-csharp/MessageQueue.cs      |  17 +++
 csharp/rocketmq-client-csharp/MessageType.cs       |  34 +++---
 csharp/rocketmq-client-csharp/MessageView.cs       |   2 +-
 csharp/rocketmq-client-csharp/MqEncoding.cs        |  30 +++--
 csharp/rocketmq-client-csharp/MqLogManager.cs      |   1 +
 csharp/rocketmq-client-csharp/Permission.cs        |  34 +++---
 csharp/rocketmq-client-csharp/Producer.cs          |  57 ++++++---
 csharp/rocketmq-client-csharp/PublishingMessage.cs |  27 ++---
 .../rocketmq-client-csharp/PublishingSettings.cs   |  19 ++-
 csharp/rocketmq-client-csharp/Resource.cs          |   8 +-
 csharp/rocketmq-client-csharp/RetryPolicy.cs       |   8 +-
 csharp/rocketmq-client-csharp/SendReceipt.cs       |  17 +--
 csharp/rocketmq-client-csharp/SequenceGenerator.cs | 131 ---------------------
 csharp/rocketmq-client-csharp/Session.cs           |  19 +--
 csharp/rocketmq-client-csharp/Settings.cs          |  29 ++++-
 csharp/rocketmq-client-csharp/Signature.cs         |  53 ++++-----
 csharp/rocketmq-client-csharp/StatusChecker.cs     |   2 +-
 .../SubscriptionLoadBalancer.cs                    |   1 -
 .../rocketmq-client-csharp/TopicRouteException.cs  |   1 +
 csharp/tests/MessageIdGeneratorTest.cs             |   4 +-
 csharp/tests/SequenceGeneratorTest.cs              |  49 --------
 42 files changed, 395 insertions(+), 588 deletions(-)

diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index 1f80671c..16791a13 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -24,7 +24,7 @@ using Org.Apache.Rocketmq;
 
 namespace examples
 {
-    static class ProducerNormalMessageExample
+    internal static class ProducerNormalMessageExample
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
diff --git a/csharp/rocketmq-client-csharp/AccessPoint.cs b/csharp/rocketmq-client-csharp/AccessPoint.cs
deleted file mode 100644
index f05fa293..00000000
--- a/csharp/rocketmq-client-csharp/AccessPoint.cs
+++ /dev/null
@@ -1,74 +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.
- */
-
-using System;
-using rmq = Apache.Rocketmq.V2;
-using System.Net;
-using System.Net.Sockets;
-
-namespace Org.Apache.Rocketmq
-{
-    public class AccessPoint
-    {
-        public AccessPoint()
-        {
-            
-        }
-
-        public AccessPoint(string accessUrl)
-        {
-            string[] segments = accessUrl.Split(":");
-            if (segments.Length != 2)
-            {
-                throw new ArgumentException("Access url should be of format host:port");
-            }
-
-            Host = segments[0];
-            Port = Int32.Parse(segments[1]);
-        }
-
-        public string Host { get; }
-
-        public int Port { get; set; }
-
-        public string TargetUrl()
-        {
-            return $"https://{Host}:{Port}";
-        }
-
-        public rmq::AddressScheme HostScheme()
-        {
-            return SchemeOf(Host);
-        }
-
-        private static rmq::AddressScheme SchemeOf(string host)
-        {
-            var result = IPAddress.TryParse(host, out var ip);
-            if (!result)
-            {
-                return rmq::AddressScheme.DomainName;
-            }
-
-            return ip.AddressFamily switch
-            {
-                AddressFamily.InterNetwork => rmq::AddressScheme.Ipv4,
-                AddressFamily.InterNetworkV6 => rmq::AddressScheme.Ipv6,
-                _ => rmq::AddressScheme.Unspecified
-            };
-        }
-    }
-}
diff --git a/csharp/rocketmq-client-csharp/Address.cs b/csharp/rocketmq-client-csharp/Address.cs
index 316323c9..fca83530 100644
--- a/csharp/rocketmq-client-csharp/Address.cs
+++ b/csharp/rocketmq-client-csharp/Address.cs
@@ -57,8 +57,7 @@ namespace Org.Apache.Rocketmq
                 return true;
             }
 
-            if (obj.GetType() != this.GetType()) return false;
-            return Equals((Address)obj);
+            return obj.GetType() == GetType() && Equals((Address)obj);
         }
 
         public override int GetHashCode()
diff --git a/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs b/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs
index 5b793f37..b8aff27a 100644
--- a/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs
+++ b/csharp/rocketmq-client-csharp/AddressListEqualityComparer.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System.Collections.Generic;
 using System.Linq;
 
diff --git a/csharp/rocketmq-client-csharp/AddressScheme.cs b/csharp/rocketmq-client-csharp/AddressScheme.cs
index f9c1c290..6f36f546 100644
--- a/csharp/rocketmq-client-csharp/AddressScheme.cs
+++ b/csharp/rocketmq-client-csharp/AddressScheme.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
diff --git a/csharp/rocketmq-client-csharp/Broker.cs b/csharp/rocketmq-client-csharp/Broker.cs
index 370ac96a..6b426a5a 100644
--- a/csharp/rocketmq-client-csharp/Broker.cs
+++ b/csharp/rocketmq-client-csharp/Broker.cs
@@ -1,10 +1,27 @@
-using rmq = Apache.Rocketmq.V2;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class Broker
     {
-        public Broker(rmq.Broker broker)
+        public Broker(Proto.Broker broker)
         {
             Name = broker.Name;
             Id = broker.Id;
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index a1c4b821..26ff9fcc 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -41,7 +41,7 @@ namespace Org.Apache.Rocketmq
         private readonly CancellationTokenSource _settingsSyncCtx;
 
         protected readonly ClientConfig ClientConfig;
-        protected readonly IClientManager Manager;
+        protected readonly IClientManager ClientManager;
         protected readonly string ClientId;
 
         protected readonly ConcurrentDictionary<string, bool> Topics;
@@ -58,7 +58,7 @@ namespace Org.Apache.Rocketmq
             Topics = topics;
             ClientId = Utilities.GetClientId();
 
-            Manager = new ClientManager(this);
+            ClientManager = new ClientManager(this);
 
             _topicRouteCache = new ConcurrentDictionary<string, TopicRouteData>();
 
@@ -91,7 +91,7 @@ namespace Org.Apache.Rocketmq
             _topicRouteUpdateCtx.Cancel();
             _heartbeatCts.Cancel();
             _telemetryCts.Cancel();
-            await Manager.Shutdown();
+            await ClientManager.Shutdown();
             Logger.Debug($"Shutdown the rocketmq client successfully, clientId={ClientId}");
         }
 
@@ -120,7 +120,7 @@ namespace Org.Apache.Rocketmq
                     return (false, session);
                 }
 
-                var stream = Manager.Telemetry(endpoints);
+                var stream = ClientManager.Telemetry(endpoints);
                 var created = new Session(endpoints, stream, this);
                 _sessionsTable.Add(endpoints, created);
                 return (true, created);
@@ -134,7 +134,7 @@ namespace Org.Apache.Rocketmq
         protected abstract Proto::HeartbeatRequest WrapHeartbeatRequest();
 
 
-        protected abstract void OnTopicDataFetched0(string topic, TopicRouteData topicRouteData);
+        protected abstract void OnTopicRouteDataFetched0(string topic, TopicRouteData topicRouteData);
 
 
         private async Task OnTopicRouteDataFetched(string topic, TopicRouteData topicRouteData)
@@ -158,7 +158,7 @@ namespace Org.Apache.Rocketmq
             }
 
             _topicRouteCache[topic] = topicRouteData;
-            OnTopicDataFetched0(topic, topicRouteData);
+            OnTopicRouteDataFetched0(topic, topicRouteData);
         }
 
 
@@ -198,16 +198,26 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        private static void ScheduleWithFixedDelay(Action action, TimeSpan period, CancellationToken token)
+        private void ScheduleWithFixedDelay(Action action, TimeSpan period, CancellationToken token)
         {
             Task.Run(async () =>
             {
                 while (!token.IsCancellationRequested)
                 {
-                    action();
-                    await Task.Delay(period, token);
+                    try
+                    {
+                        action();
+                    }
+                    catch (Exception e)
+                    {
+                        Logger.Error(e, $"Failed to execute scheduled task, ClientId={ClientId}");
+                    }
+                    finally
+                    {
+                        await Task.Delay(period, token);
+                    }
                 }
-            });
+            }, token);
         }
 
         protected async Task<TopicRouteData> FetchTopicRoute(string topic)
@@ -232,7 +242,7 @@ namespace Org.Apache.Rocketmq
             };
 
             var response =
-                await Manager.QueryRoute(ClientConfig.Endpoints, request, ClientConfig.RequestTimeout);
+                await ClientManager.QueryRoute(ClientConfig.Endpoints, request, ClientConfig.RequestTimeout);
             var code = response.Status.Code;
             if (!Proto.Code.Ok.Equals(code))
             {
@@ -245,7 +255,7 @@ namespace Org.Apache.Rocketmq
             return new TopicRouteData(messageQueues);
         }
 
-        public async void Heartbeat()
+        private async void Heartbeat()
         {
             var endpoints = GetTotalRouteEndpoints();
             if (0 == endpoints.Count)
@@ -259,7 +269,7 @@ namespace Org.Apache.Rocketmq
             // Collect task into a map.
             foreach (var item in endpoints)
             {
-                var task = Manager.Heartbeat(item, request, ClientConfig.RequestTimeout);
+                var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
                 responses[item]= task;
             }
             foreach (var item in responses.Keys)
@@ -276,12 +286,14 @@ namespace Org.Apache.Rocketmq
                 Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
             }
         }
+        
+        
 
 
         public grpc.Metadata Sign()
         {
             var metadata = new grpc::Metadata();
-            Signature.Sign(ClientConfig, metadata);
+            Signature.Sign(this, metadata);
             return metadata;
         }
 
@@ -294,7 +306,7 @@ namespace Org.Apache.Rocketmq
             };
             foreach (var item in endpoints)
             {
-                var response = await Manager.NotifyClientTermination(item, request, ClientConfig.RequestTimeout);
+                var response = await ClientManager.NotifyClientTermination(item, request, ClientConfig.RequestTimeout);
                 try
                 {
                     StatusChecker.Check(response.Status, request);
@@ -319,6 +331,11 @@ namespace Org.Apache.Rocketmq
             return ClientId;
         }
 
+        public ClientConfig GetClientConfig()
+        {
+            return ClientConfig;
+        }
+
         public void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
             Proto.RecoverOrphanedTransactionCommand command)
         {
diff --git a/csharp/rocketmq-client-csharp/ClientConfig.cs b/csharp/rocketmq-client-csharp/ClientConfig.cs
index e5fd8643..7e434eae 100644
--- a/csharp/rocketmq-client-csharp/ClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/ClientConfig.cs
@@ -22,13 +22,8 @@ namespace Org.Apache.Rocketmq
 {
     public class ClientConfig : IClientConfig
     {
-        private static long _instanceSequence = 0;
-
         public ClientConfig(string endpoints)
         {
-            var hostName = System.Net.Dns.GetHostName();
-            var pid = System.Diagnostics.Process.GetCurrentProcess().Id;
-            ClientId = $"{hostName}@{pid}@{Interlocked.Increment(ref _instanceSequence)}";
             RequestTimeout = TimeSpan.FromSeconds(3);
             Endpoints = new Endpoints(endpoints);
         }
@@ -37,9 +32,6 @@ namespace Org.Apache.Rocketmq
 
         public TimeSpan RequestTimeout { get; set; }
 
-        public string ClientId { get; }
-
-
         public Endpoints Endpoints { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
index d9622291..890ce877 100644
--- a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
+++ b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 using System.Threading.Tasks;
 using Grpc.Core;
diff --git a/csharp/rocketmq-client-csharp/ClientManager.cs b/csharp/rocketmq-client-csharp/ClientManager.cs
index bd18ebc4..3eef2fe6 100644
--- a/csharp/rocketmq-client-csharp/ClientManager.cs
+++ b/csharp/rocketmq-client-csharp/ClientManager.cs
@@ -21,7 +21,6 @@ using System.Threading;
 using System.Threading.Tasks;
 using grpc = Grpc.Core;
 using System.Collections.Generic;
-using NLog;
 
 namespace Org.Apache.Rocketmq
 {
diff --git a/csharp/rocketmq-client-csharp/ClientType.cs b/csharp/rocketmq-client-csharp/ClientType.cs
index 15481c98..487b3bec 100644
--- a/csharp/rocketmq-client-csharp/ClientType.cs
+++ b/csharp/rocketmq-client-csharp/ClientType.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -28,19 +28,15 @@ namespace Org.Apache.Rocketmq
 
     public static class ClientTypeHelper
     {
-        public static rmq.ClientType ToProtobuf(ClientType clientType)
+        public static Proto.ClientType ToProtobuf(ClientType clientType)
         {
-            switch (clientType)
+            return clientType switch
             {
-                case ClientType.Producer:
-                    return rmq.ClientType.Producer;
-                case ClientType.SimpleConsumer:
-                    return rmq.ClientType.SimpleConsumer;
-                case ClientType.PushConsumer:
-                    return rmq.ClientType.PushConsumer;
-                default:
-                    return rmq.ClientType.Unspecified;
-            }
+                ClientType.Producer => Proto.ClientType.Producer,
+                ClientType.SimpleConsumer => Proto.ClientType.SimpleConsumer,
+                ClientType.PushConsumer => Proto.ClientType.PushConsumer,
+                _ => Proto.ClientType.Unspecified
+            };
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs b/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
index 73d05f63..7764dc34 100644
--- a/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/ConfigFileCredentialsProvider.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System.IO;
 using System;
 using System.Text.Json;
diff --git a/csharp/rocketmq-client-csharp/Endpoints.cs b/csharp/rocketmq-client-csharp/Endpoints.cs
index e7cf5f9c..54d8f0d2 100644
--- a/csharp/rocketmq-client-csharp/Endpoints.cs
+++ b/csharp/rocketmq-client-csharp/Endpoints.cs
@@ -1,19 +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.
+ */
+
 using System;
 using System.Collections.Generic;
 using System.Linq;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class Endpoints : IEquatable<Endpoints>
     {
         private static readonly AddressListEqualityComparer AddressListComparer = new();
-        private static readonly string EndpointSeparator = ":";
+        private const string EndpointSeparator = ":";
         private List<Address> Addresses { get; }
         private AddressScheme Scheme { get; }
         private readonly int _hashCode;
 
-        public Endpoints(global::Apache.Rocketmq.V2.Endpoints endpoints)
+        public Endpoints(Proto.Endpoints endpoints)
         {
             Addresses = new List<Address>();
             foreach (var address in endpoints.Addresses)
@@ -28,13 +45,14 @@ namespace Org.Apache.Rocketmq
 
             switch (endpoints.Scheme)
             {
-                case rmq.AddressScheme.Ipv4:
+                case Proto.AddressScheme.Ipv4:
                     Scheme = AddressScheme.Ipv4;
                     break;
-                case rmq.AddressScheme.Ipv6:
+                case Proto.AddressScheme.Ipv6:
                     Scheme = AddressScheme.Ipv6;
                     break;
-                case rmq.AddressScheme.DomainName:
+                case Proto.AddressScheme.DomainName:
+                case Proto.AddressScheme.Unspecified:
                 default:
                     Scheme = AddressScheme.DomainName;
                     if (Addresses.Count > 1)
@@ -123,9 +141,9 @@ namespace Org.Apache.Rocketmq
             return _hashCode;
         }
 
-        public rmq.Endpoints ToProtobuf()
+        public Proto.Endpoints ToProtobuf()
         {
-            var endpoints = new rmq.Endpoints();
+            var endpoints = new Proto.Endpoints();
             foreach (var address in Addresses)
             {
                 endpoints.Addresses.Add(address.ToProtobuf());
diff --git a/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
index e987d979..094c2607 100644
--- a/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
+++ b/csharp/rocketmq-client-csharp/ExponentialBackoffRetryPolicy.cs
@@ -4,11 +4,11 @@ using Google.Protobuf.WellKnownTypes;
 
 namespace Org.Apache.Rocketmq
 {
-    public class ExponentialBackoffRetryPolicy : RetryPolicy
+    public class ExponentialBackoffRetryPolicy : IRetryPolicy
     {
-        private int _maxAttempts;
+        private readonly int _maxAttempts;
 
-        public ExponentialBackoffRetryPolicy(int maxAttempts, TimeSpan initialBackoff, TimeSpan maxBackoff,
+        private ExponentialBackoffRetryPolicy(int maxAttempts, TimeSpan initialBackoff, TimeSpan maxBackoff,
             double backoffMultiplier)
         {
             _maxAttempts = maxAttempts;
@@ -17,7 +17,7 @@ namespace Org.Apache.Rocketmq
             BackoffMultiplier = backoffMultiplier;
         }
 
-        public int getMaxAttempts()
+        public int GetMaxAttempts()
         {
             return _maxAttempts;
         }
@@ -28,17 +28,17 @@ namespace Org.Apache.Rocketmq
 
         public double BackoffMultiplier { get; }
 
-        public TimeSpan getNextAttemptDelay(int attempt)
+        public TimeSpan GetNextAttemptDelay(int attempt)
         {
             return TimeSpan.Zero;
         }
 
-        public static ExponentialBackoffRetryPolicy immediatelyRetryPolicy(int maxAttempts)
+        public static ExponentialBackoffRetryPolicy ImmediatelyRetryPolicy(int maxAttempts)
         {
             return new ExponentialBackoffRetryPolicy(maxAttempts, TimeSpan.Zero, TimeSpan.Zero, 1);
         }
 
-        public global::Apache.Rocketmq.V2.RetryPolicy toProtobuf()
+        public global::Apache.Rocketmq.V2.RetryPolicy ToProtobuf()
         {
             var exponentialBackoff = new ExponentialBackoff
             {
diff --git a/csharp/rocketmq-client-csharp/IClient.cs b/csharp/rocketmq-client-csharp/IClient.cs
index db219af9..5ba4c6f1 100644
--- a/csharp/rocketmq-client-csharp/IClient.cs
+++ b/csharp/rocketmq-client-csharp/IClient.cs
@@ -23,22 +23,44 @@ namespace Org.Apache.Rocketmq
 {
     public interface IClient
     {
-        void Heartbeat();
-
-        void NotifyClientTermination(Proto.Resource group);
-
         CancellationTokenSource TelemetryCts();
 
+        ClientConfig GetClientConfig();
+
         Proto.Settings GetSettings();
 
+        /// <summary>
+        /// Get the identifier of current client. 
+        /// </summary>
+        /// <returns>Client identifier.</returns>
         string GetClientId();
 
+        /// <summary>
+        /// This method will be triggered when client settings is received from remote endpoints.
+        /// </summary>
+        /// <param name="endpoints"></param>
+        /// <param name="settings"></param>
         void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings);
 
+        /// <summary>
+        /// This method will be triggered when orphaned transaction need to be recovered.
+        /// </summary>
+        /// <param name="endpoints">Remote endpoints.</param>
+        /// <param name="command">Command of orphaned transaction recovery.</param>
         void OnRecoverOrphanedTransactionCommand(Endpoints endpoints, Proto.RecoverOrphanedTransactionCommand command);
 
+        /// <summary>
+        /// This method will be triggered when message verification command is received.
+        /// </summary>
+        /// <param name="endpoints">Remote endpoints.</param>
+        /// <param name="command">Command of message verification.</param>
         void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command);
 
+        /// <summary>
+        /// This method will be triggered when thread stack trace command is received.
+        /// </summary>
+        /// <param name="endpoints">Remote endpoints.</param>
+        /// <param name="command">Command of printing thread stack trace.</param>
         void OnPrintThreadStackTraceCommand(Endpoints endpoints, Proto.PrintThreadStackTraceCommand command);
 
         Metadata Sign();
diff --git a/csharp/rocketmq-client-csharp/IClientConfig.cs b/csharp/rocketmq-client-csharp/IClientConfig.cs
index 5603a616..a50bdf93 100644
--- a/csharp/rocketmq-client-csharp/IClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/IClientConfig.cs
@@ -15,21 +15,10 @@
  * limitations under the License.
  */
 
-using System;
-
 namespace Org.Apache.Rocketmq
 {
     public interface IClientConfig
     {
-
-        ICredentialsProvider CredentialsProvider
-        {
-            get;
-        }
-
-        string ClientId
-        {
-            get;
-        }
+        ICredentialsProvider CredentialsProvider { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClientManager.cs b/csharp/rocketmq-client-csharp/IClientManager.cs
index beb8880b..f2e48e36 100644
--- a/csharp/rocketmq-client-csharp/IClientManager.cs
+++ b/csharp/rocketmq-client-csharp/IClientManager.cs
@@ -19,15 +19,13 @@ using System.Threading.Tasks;
 using System;
 using System.Collections.Generic;
 using Apache.Rocketmq.V2;
-using grpc = Grpc.Core;
-using rmq = Apache.Rocketmq.V2;
-
+using Grpc.Core;
 
 namespace Org.Apache.Rocketmq
 {
     public interface IClientManager
     {
-        grpc::AsyncDuplexStreamingCall<TelemetryCommand, TelemetryCommand> Telemetry(Endpoints endpoints);
+        AsyncDuplexStreamingCall<TelemetryCommand, TelemetryCommand> Telemetry(Endpoints endpoints);
 
         Task<QueryRouteResponse> QueryRoute(Endpoints endpoints, QueryRouteRequest request, TimeSpan timeout);
 
diff --git a/csharp/rocketmq-client-csharp/ICredentialsProvider.cs b/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
index 2f6e71eb..e98df14a 100644
--- a/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
+++ b/csharp/rocketmq-client-csharp/ICredentialsProvider.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 namespace Org.Apache.Rocketmq
 {
     public interface ICredentialsProvider
diff --git a/csharp/rocketmq-client-csharp/Message.cs b/csharp/rocketmq-client-csharp/Message.cs
index fb004da3..9993b52f 100644
--- a/csharp/rocketmq-client-csharp/Message.cs
+++ b/csharp/rocketmq-client-csharp/Message.cs
@@ -20,14 +20,15 @@ using System.Collections.Generic;
 
 namespace Org.Apache.Rocketmq
 {
-
     public class Message
     {
         public Message() : this(null, null)
         {
         }
 
-        public Message(string topic, byte[] body) : this(topic, null, new List<string>(), body) { }
+        public Message(string topic, byte[] body) : this(topic, null, new List<string>(), body)
+        {
+        }
 
         public Message(string topic, string tag, byte[] body) : this(topic, tag, new List<string>(), body)
         {
@@ -35,7 +36,6 @@ namespace Org.Apache.Rocketmq
 
         public Message(string topic, string tag, List<string> keys, byte[] body)
         {
-            MaxAttemptTimes = 3;
             Topic = topic;
             Tag = tag;
             Keys = keys;
@@ -44,73 +44,20 @@ namespace Org.Apache.Rocketmq
             DeliveryTimestamp = null;
         }
 
-        public string Topic
-        {
-            get;
-            set;
-        }
+        public string Topic { get; set; }
 
-        public byte[] Body
-        {
-            get;
-            set;
-        }
+        public byte[] Body { get; set; }
 
-        public string Tag
-        {
-            get;
-            set;
-        }
+        public string Tag { get; set; }
 
-        public List<string> Keys
-        {
-            get;
-            set;
-        }
-        public Dictionary<string, string> UserProperties
-        {
-            get;
-            set;
-        }
+        public List<string> Keys { get; set; }
+        public Dictionary<string, string> UserProperties { get; set; }
 
-        public int MaxAttemptTimes
-        {
-            get;
-            set;
-        }
 
+        public DateTime? DeliveryTimestamp { get; set; }
 
-        public DateTime? DeliveryTimestamp
-        {
-            get;
-            set;
-        }
-        
-        public int DeliveryAttempt
-        {
-            get;
-            internal set;
-        }
-        
-        public string MessageGroup
-        {
-            get;
-            set;
-        }
-        
-        public bool Fifo()
-        {
-            return !String.IsNullOrEmpty(MessageGroup);
-        }
-
-        public bool Scheduled()
-        {
-            return DeliveryTimestamp > DateTime.UtcNow;
-        }
+        public int DeliveryAttempt { get; internal set; }
 
-        internal bool _checksumVerifiedOk = true;
-        internal string _receiptHandle;
-        internal string _sourceHost;
+        public string MessageGroup { get; set; }
     }
-
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageException.cs b/csharp/rocketmq-client-csharp/MessageException.cs
deleted file mode 100644
index 7ef10df3..00000000
--- a/csharp/rocketmq-client-csharp/MessageException.cs
+++ /dev/null
@@ -1,29 +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.
- */
-
-using System;
-
-namespace Org.Apache.Rocketmq
-{
-    [Serializable]
-    public class MessageException : Exception
-    {
-        public MessageException(string message) : base(message)
-        {
-        }
-    }
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageIdGenerator.cs b/csharp/rocketmq-client-csharp/MessageIdGenerator.cs
index 8dc370d1..60620ef0 100644
--- a/csharp/rocketmq-client-csharp/MessageIdGenerator.cs
+++ b/csharp/rocketmq-client-csharp/MessageIdGenerator.cs
@@ -27,7 +27,7 @@ namespace Org.Apache.Rocketmq
      */
     public class MessageIdGenerator
     {
-        public static readonly string version = "01";
+        public const string Version = "01";
         private static readonly MessageIdGenerator Instance = new();
 
         private readonly string _prefix;
@@ -39,15 +39,15 @@ namespace Org.Apache.Rocketmq
 
         private MessageIdGenerator()
         {
-            MemoryStream stream = new MemoryStream();
-            BinaryWriter writer = new BinaryWriter(stream);
+            var stream = new MemoryStream();
+            var writer = new BinaryWriter(stream);
 
             var macAddress = Utilities.GetMacAddress();
             writer.Write(macAddress, 0, 6);
 
-            int processId = Utilities.GetProcessId();
+            var processId = Utilities.GetProcessId();
 
-            byte[] processIdBytes = BitConverter.GetBytes(processId);
+            var processIdBytes = BitConverter.GetBytes(processId);
             if (BitConverter.IsLittleEndian)
             {
                 Array.Reverse(processIdBytes);
@@ -55,9 +55,9 @@ namespace Org.Apache.Rocketmq
 
             writer.Write(processIdBytes, 2, 2);
             var array = stream.ToArray();
-            _prefix = version + Utilities.ByteArrayToHexString(array);
+            _prefix = Version + Utilities.ByteArrayToHexString(array);
 
-            DateTime epoch = new DateTime(2021, 1, 1,
+            var epoch = new DateTime(2021, 1, 1,
                 0, 0, 0, 0, DateTimeKind.Utc);
 
             var now = DateTime.Now;
@@ -67,12 +67,12 @@ namespace Org.Apache.Rocketmq
             _sequence = 0;
         }
 
-        public String Next()
+        public string Next()
         {
-            long deltaSeconds = _secondsSinceCustomEpoch + _stopwatch.ElapsedMilliseconds / 1_000;
+            var deltaSeconds = _secondsSinceCustomEpoch + _stopwatch.ElapsedMilliseconds / 1_000;
 
-            MemoryStream stream = new MemoryStream();
-            BinaryWriter writer = new BinaryWriter(stream);
+            var stream = new MemoryStream();
+            var writer = new BinaryWriter(stream);
 
             byte[] deltaSecondsBytes = BitConverter.GetBytes(deltaSeconds);
             if (BitConverter.IsLittleEndian)
@@ -82,8 +82,8 @@ namespace Org.Apache.Rocketmq
 
             writer.Write(deltaSecondsBytes, 4, 4);
 
-            int no = Interlocked.Increment(ref _sequence);
-            byte[] noBytes = BitConverter.GetBytes(no);
+            var no = Interlocked.Increment(ref _sequence);
+            var noBytes = BitConverter.GetBytes(no);
             if (BitConverter.IsLittleEndian)
             {
                 Array.Reverse(noBytes);
diff --git a/csharp/rocketmq-client-csharp/MessageQueue.cs b/csharp/rocketmq-client-csharp/MessageQueue.cs
index 385e392c..cd6f0ce3 100644
--- a/csharp/rocketmq-client-csharp/MessageQueue.cs
+++ b/csharp/rocketmq-client-csharp/MessageQueue.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System.Collections.Generic;
 using rmq = Apache.Rocketmq.V2;
 
diff --git a/csharp/rocketmq-client-csharp/MessageType.cs b/csharp/rocketmq-client-csharp/MessageType.cs
index 6338e365..09896a0b 100644
--- a/csharp/rocketmq-client-csharp/MessageType.cs
+++ b/csharp/rocketmq-client-csharp/MessageType.cs
@@ -16,7 +16,7 @@
  */
 
 using Org.Apache.Rocketmq.Error;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -30,38 +30,34 @@ namespace Org.Apache.Rocketmq
 
     public static class MessageTypeHelper
     {
-        public static MessageType FromProtobuf(rmq.MessageType messageType)
+        public static MessageType FromProtobuf(Proto.MessageType messageType)
         {
             switch (messageType)
             {
-                case rmq.MessageType.Normal:
+                case Proto.MessageType.Normal:
                     return MessageType.Normal;
-                case rmq.MessageType.Fifo:
+                case Proto.MessageType.Fifo:
                     return MessageType.Fifo;
-                case rmq.MessageType.Delay:
+                case Proto.MessageType.Delay:
                     return MessageType.Delay;
-                case rmq.MessageType.Transaction:
+                case Proto.MessageType.Transaction:
                     return MessageType.Transaction;
+                case Proto.MessageType.Unspecified:
                 default:
                     throw new InternalErrorException("MessageType is not specified");
             }
         }
 
-        public static rmq.MessageType ToProtobuf(MessageType messageType)
+        public static Proto.MessageType ToProtobuf(MessageType messageType)
         {
-            switch (messageType)
+            return messageType switch
             {
-                case MessageType.Normal:
-                    return rmq.MessageType.Normal;
-                case MessageType.Fifo:
-                    return rmq.MessageType.Fifo;
-                case MessageType.Delay:
-                    return rmq.MessageType.Delay;
-                case MessageType.Transaction:
-                    return rmq.MessageType.Transaction;
-                default:
-                    return rmq.MessageType.Unspecified;
-            }
+                MessageType.Normal => Proto.MessageType.Normal,
+                MessageType.Fifo => Proto.MessageType.Fifo,
+                MessageType.Delay => Proto.MessageType.Delay,
+                MessageType.Transaction => Proto.MessageType.Transaction,
+                _ => Proto.MessageType.Unspecified
+            };
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index 57b573ac..26d7fcc6 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -79,7 +79,7 @@ namespace Org.Apache.Rocketmq
 
         public int DeliveryAttempt { get; }
 
-        public static MessageView fromProtobuf(rmq.Message message, rmq.MessageQueue messageQueue)
+        public static MessageView FromProtobuf(rmq.Message message, rmq.MessageQueue messageQueue)
         {
             var topic = message.Topic.Name;
             var systemProperties = message.SystemProperties;
diff --git a/csharp/rocketmq-client-csharp/MqEncoding.cs b/csharp/rocketmq-client-csharp/MqEncoding.cs
index ba2a489d..27dfb052 100644
--- a/csharp/rocketmq-client-csharp/MqEncoding.cs
+++ b/csharp/rocketmq-client-csharp/MqEncoding.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
@@ -12,15 +29,12 @@ namespace Org.Apache.Rocketmq
     {
         public static rmq.Encoding ToProtobuf(MqEncoding mqEncoding)
         {
-            switch (mqEncoding)
+            return mqEncoding switch
             {
-                case MqEncoding.Gzip:
-                    return rmq.Encoding.Gzip;
-                case MqEncoding.Identity:
-                    return rmq.Encoding.Identity;
-                default:
-                    return rmq.Encoding.Unspecified;
-            }
+                MqEncoding.Gzip => rmq.Encoding.Gzip,
+                MqEncoding.Identity => rmq.Encoding.Identity,
+                _ => rmq.Encoding.Unspecified
+            };
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MqLogManager.cs b/csharp/rocketmq-client-csharp/MqLogManager.cs
index dcbdce57..7fa2b7bf 100644
--- a/csharp/rocketmq-client-csharp/MqLogManager.cs
+++ b/csharp/rocketmq-client-csharp/MqLogManager.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 using System.IO;
 using System.Reflection;
diff --git a/csharp/rocketmq-client-csharp/Permission.cs b/csharp/rocketmq-client-csharp/Permission.cs
index d5fe6348..eedd6247 100644
--- a/csharp/rocketmq-client-csharp/Permission.cs
+++ b/csharp/rocketmq-client-csharp/Permission.cs
@@ -48,32 +48,30 @@ namespace Org.Apache.Rocketmq
         }
         
         public static bool IsWritable(Permission permission) {
-            if (Permission.Write.Equals(permission))
-            {
-                return true;
-            }
-
-            if (Permission.ReadWrite.Equals(permission))
+            switch (permission)
             {
-                return true;
+                case Permission.Write:
+                case Permission.ReadWrite:
+                    return true;
+                case Permission.None:
+                case Permission.Read:
+                default:
+                    return false;
             }
-
-            return false;
         }
 
         public static bool IsReadable(Permission permission)
         {
-            if (Permission.Read.Equals(permission))
-            {
-                return true;
-            }
-
-            if (Permission.ReadWrite.Equals(permission))
+            switch (permission)
             {
-                return true;
+                case Permission.Read:
+                case Permission.ReadWrite:
+                    return true;
+                case Permission.None:
+                case Permission.Write:
+                default:
+                    return false;
             }
-
-            return false;
         }
     }
     
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 56ec5b40..c2782fc0 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using System.Collections.Concurrent;
 using System.Collections.Generic;
@@ -27,7 +44,7 @@ namespace Org.Apache.Rocketmq
         private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
             base(clientConfig, topics)
         {
-            var retryPolicy = ExponentialBackoffRetryPolicy.immediatelyRetryPolicy(maxAttempts);
+            var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
             _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
                 clientConfig.RequestTimeout, topics);
             _publishingRouteDataCache = new ConcurrentDictionary<string, PublishingLoadBalancer>();
@@ -63,27 +80,37 @@ namespace Org.Apache.Rocketmq
             };
         }
 
-        protected override void OnTopicDataFetched0(string topic, TopicRouteData topicRouteData)
+        private async Task<PublishingLoadBalancer> GetPublishingLoadBalancer(string topic)
         {
+            if (_publishingRouteDataCache.TryGetValue(topic, out var publishingLoadBalancer))
+            {
+                return publishingLoadBalancer;
+            }
+
+            var topicRouteData = await FetchTopicRoute(topic);
+            publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
+            _publishingRouteDataCache.TryAdd(topic, publishingLoadBalancer);
+
+            return publishingLoadBalancer;
         }
 
-        private RetryPolicy GetRetryPolicy()
+        protected override void OnTopicRouteDataFetched0(string topic, TopicRouteData topicRouteData)
+        {
+            var publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
+            _publishingRouteDataCache.TryAdd(topic, publishingLoadBalancer);
+        }
+
+        private IRetryPolicy GetRetryPolicy()
         {
             return _publishingSettings.GetRetryPolicy();
         }
 
         public async Task<SendReceipt> Send(Message message)
         {
-            if (!_publishingRouteDataCache.TryGetValue(message.Topic, out var publishingLoadBalancer))
-            {
-                var topicRouteData = await FetchTopicRoute(message.Topic);
-                publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
-                _publishingRouteDataCache.TryAdd(message.Topic, publishingLoadBalancer);
-            }
-
+            var publishingLoadBalancer = await GetPublishingLoadBalancer(message.Topic);
             var publishingMessage = new PublishingMessage(message, _publishingSettings, false);
             var retryPolicy = GetRetryPolicy();
-            var maxAttempts = retryPolicy.getMaxAttempts();
+            var maxAttempts = retryPolicy.GetMaxAttempts();
             var candidates = publishingLoadBalancer.TakeMessageQueues(publishingMessage.MessageGroup, maxAttempts);
             Exception exception = null;
             for (var attempt = 0; attempt < maxAttempts; attempt++)
@@ -102,7 +129,7 @@ namespace Org.Apache.Rocketmq
             throw exception!;
         }
 
-        private Proto.SendMessageRequest WrapSendMessageRequest(PublishingMessage message, MessageQueue mq)
+        private static Proto.SendMessageRequest WrapSendMessageRequest(PublishingMessage message, MessageQueue mq)
         {
             return new Proto.SendMessageRequest
             {
@@ -125,11 +152,11 @@ namespace Org.Apache.Rocketmq
 
             var sendMessageRequest = WrapSendMessageRequest(message, mq);
             var endpoints = mq.Broker.Endpoints;
-            Proto.SendMessageResponse response =
-                await Manager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
+            var response =
+                await ClientManager.SendMessage(endpoints, sendMessageRequest, ClientConfig.RequestTimeout);
             try
             {
-                var sendReceipts = SendReceipt.processSendMessageResponse(response);
+                var sendReceipts = SendReceipt.ProcessSendMessageResponse(response);
 
                 var sendReceipt = sendReceipts.First();
                 if (attempt > 1)
diff --git a/csharp/rocketmq-client-csharp/PublishingMessage.cs b/csharp/rocketmq-client-csharp/PublishingMessage.cs
index 93eb2de6..7839edaa 100644
--- a/csharp/rocketmq-client-csharp/PublishingMessage.cs
+++ b/csharp/rocketmq-client-csharp/PublishingMessage.cs
@@ -19,7 +19,7 @@ using System;
 using System.IO;
 using Google.Protobuf;
 using Google.Protobuf.WellKnownTypes;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using Org.Apache.Rocketmq.Error;
 
 namespace Org.Apache.Rocketmq
@@ -31,7 +31,7 @@ namespace Org.Apache.Rocketmq
     {
         public MessageType MessageType { set; get; }
 
-        public String MessageId { get; }
+        private string MessageId { get; }
 
         public PublishingMessage(Message message, PublishingSettings publishingSettings, bool txEnabled) : base(
             message.Topic, message.Body)
@@ -45,7 +45,7 @@ namespace Org.Apache.Rocketmq
             // Generate message id.
             MessageId = MessageIdGenerator.GetInstance().Next();
             // For NORMAL message.
-            if (String.IsNullOrEmpty(message.MessageGroup) && !message.DeliveryTimestamp.HasValue &&
+            if (string.IsNullOrEmpty(message.MessageGroup) && !message.DeliveryTimestamp.HasValue &&
                 !txEnabled)
             {
                 MessageType = MessageType.Normal;
@@ -53,7 +53,7 @@ namespace Org.Apache.Rocketmq
             }
 
             // For FIFO message.
-            if (!String.IsNullOrEmpty(message.MessageGroup) && !txEnabled)
+            if (!string.IsNullOrEmpty(message.MessageGroup) && !txEnabled)
             {
                 MessageType = MessageType.Fifo;
                 return;
@@ -67,18 +67,15 @@ namespace Org.Apache.Rocketmq
             }
 
             // For TRANSACTION message.
-            if (!String.IsNullOrEmpty(message.MessageGroup) && !message.DeliveryTimestamp.HasValue && txEnabled)
-            {
-                MessageType = MessageType.Transaction;
-                return;
-            }
-
-            throw new InternalErrorException("Transactional message should not set messageGroup or deliveryTimestamp");
+            if (string.IsNullOrEmpty(message.MessageGroup) || message.DeliveryTimestamp.HasValue || !txEnabled)
+                throw new InternalErrorException(
+                    "Transactional message should not set messageGroup or deliveryTimestamp");
+            MessageType = MessageType.Transaction;
         }
 
-        public rmq::Message ToProtobuf(int queueId)
+        public Proto::Message ToProtobuf(int queueId)
         {
-            rmq.SystemProperties systemProperties = new rmq.SystemProperties
+            var systemProperties = new Proto.SystemProperties
             {
                 Keys = { Keys },
                 MessageId = MessageId,
@@ -103,11 +100,11 @@ namespace Org.Apache.Rocketmq
                 systemProperties.MessageGroup = MessageGroup;
             }
 
-            rmq.Resource topicResource = new rmq.Resource
+            var topicResource = new Proto.Resource
             {
                 Name = Topic
             };
-            return new rmq.Message
+            return new Proto.Message
             {
                 Topic = topicResource,
                 Body = ByteString.CopyFrom(Body),
diff --git a/csharp/rocketmq-client-csharp/PublishingSettings.cs b/csharp/rocketmq-client-csharp/PublishingSettings.cs
index b543cb71..023b0be3 100644
--- a/csharp/rocketmq-client-csharp/PublishingSettings.cs
+++ b/csharp/rocketmq-client-csharp/PublishingSettings.cs
@@ -1,3 +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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
 using System.Collections.Concurrent;
 using System.Collections.Generic;
@@ -55,7 +72,7 @@ namespace Org.Apache.Rocketmq
                 AccessPoint = AccessPoint.ToProtobuf(),
                 ClientType = ClientTypeHelper.ToProtobuf(ClientType),
                 RequestTimeout = Duration.FromTimeSpan(RequestTimeout),
-                BackoffPolicy = RetryPolicy.toProtobuf(),
+                BackoffPolicy = RetryPolicy.ToProtobuf(),
                 UserAgent = UserAgent.Instance.ToProtobuf()
             };
         }
diff --git a/csharp/rocketmq-client-csharp/Resource.cs b/csharp/rocketmq-client-csharp/Resource.cs
index 5af67d1e..a1825f15 100644
--- a/csharp/rocketmq-client-csharp/Resource.cs
+++ b/csharp/rocketmq-client-csharp/Resource.cs
@@ -1,11 +1,11 @@
 using System;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class Resource
     {
-        public Resource(rmq.Resource resource)
+        public Resource(Proto.Resource resource)
         {
             Namespace = resource.ResourceNamespace;
             Name = resource.Name;
@@ -14,9 +14,9 @@ namespace Org.Apache.Rocketmq
         public string Namespace { get; }
         public string Name { get; }
 
-        public rmq.Resource ToProtobuf()
+        public Proto.Resource ToProtobuf()
         {
-            return new rmq.Resource
+            return new Proto.Resource
             {
                 ResourceNamespace = Namespace,
                 Name = Name
diff --git a/csharp/rocketmq-client-csharp/RetryPolicy.cs b/csharp/rocketmq-client-csharp/RetryPolicy.cs
index 9169b5d1..92b82013 100644
--- a/csharp/rocketmq-client-csharp/RetryPolicy.cs
+++ b/csharp/rocketmq-client-csharp/RetryPolicy.cs
@@ -2,12 +2,12 @@ using System;
 
 namespace Org.Apache.Rocketmq
 {
-    public interface RetryPolicy
+    public interface IRetryPolicy
     {
-        int getMaxAttempts();
+        int GetMaxAttempts();
 
-        TimeSpan getNextAttemptDelay(int attempt);
+        TimeSpan GetNextAttemptDelay(int attempt);
 
-        global::Apache.Rocketmq.V2.RetryPolicy toProtobuf();
+        global::Apache.Rocketmq.V2.RetryPolicy ToProtobuf();
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SendReceipt.cs b/csharp/rocketmq-client-csharp/SendReceipt.cs
index a4ff1e3e..fa5c75c7 100644
--- a/csharp/rocketmq-client-csharp/SendReceipt.cs
+++ b/csharp/rocketmq-client-csharp/SendReceipt.cs
@@ -16,7 +16,8 @@
  */
 
 using System.Collections.Generic;
-using rmq = Apache.Rocketmq.V2;
+using System.Linq;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -34,12 +35,12 @@ namespace Org.Apache.Rocketmq
             return $"{nameof(MessageId)}: {MessageId}";
         }
 
-        public static List<SendReceipt> processSendMessageResponse(rmq.SendMessageResponse response)
+        public static List<SendReceipt> ProcessSendMessageResponse(Proto.SendMessageResponse response)
         {
-            rmq.Status status = response.Status;
+            var status = response.Status;
             foreach (var entry in response.Entries)
             {
-                if (rmq.Code.Ok.Equals(entry.Status.Code))
+                if (Proto.Code.Ok.Equals(entry.Status.Code))
                 {
                     status = entry.Status;
                 }
@@ -47,13 +48,7 @@ namespace Org.Apache.Rocketmq
 
             // May throw exception.
             StatusChecker.Check(status, response);
-            List<SendReceipt> sendReceipts = new List<SendReceipt>();
-            foreach (var entry in response.Entries)
-            {
-                sendReceipts.Add(new SendReceipt(entry.MessageId));
-            }
-
-            return sendReceipts;
+            return response.Entries.Select(entry => new SendReceipt(entry.MessageId)).ToList();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SequenceGenerator.cs b/csharp/rocketmq-client-csharp/SequenceGenerator.cs
deleted file mode 100644
index 97a1eb91..00000000
--- a/csharp/rocketmq-client-csharp/SequenceGenerator.cs
+++ /dev/null
@@ -1,131 +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.
- */
-using System;
-using System.Threading;
-using System.Net.NetworkInformation;
-using NLog;
-
-namespace Org.Apache.Rocketmq
-{
-    /**
-     * See https://yuque.antfin-inc.com/aone709911/ca1edg/af2t6o for Sequence ID spec.
-     * 
-     * In the implementation layer, this class follows Singleton pattern.
-     */
-    public sealed class SequenceGenerator
-    {
-        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
-
-        public static SequenceGenerator Instance
-        {
-            get
-            {
-                return Nested.instance;
-            }
-        }
-
-        private class Nested
-        {
-            static Nested()
-            {
-
-            }
-
-            internal static readonly SequenceGenerator instance = new SequenceGenerator();
-        }
-
-        private SequenceGenerator()
-        {
-            currentSecond = SecondsSinceCustomEpoch();
-            macAddress = MacAddress();
-            pidBytes = ToArray(pid);
-            if (BitConverter.IsLittleEndian)
-            {
-                Array.Reverse(version);
-            }
-        }
-
-        /**
-         * Sequence version, 2 bytes.
-         */
-        private static byte[] version = new byte[2] { 0x00, 0x01 };
-
-        /**
-         * MAC address, 6 bytes.
-         */
-        private byte[] macAddress;
-
-        private int sequenceInSecond = 0;
-        private int currentSecond;
-
-        private static int pid = System.Diagnostics.Process.GetCurrentProcess().Id;
-        private static byte[] pidBytes;
-
-        private static byte[] ToArray(int number)
-        {
-            byte[] bytes = BitConverter.GetBytes(number);
-            if (BitConverter.IsLittleEndian)
-                Array.Reverse(bytes);
-            return bytes;
-        }
-
-        private static int SecondsSinceCustomEpoch()
-        {
-            var customEpoch = new DateTime(2021, 01, 01, 00, 00, 00, DateTimeKind.Utc);
-            var diff = DateTime.UtcNow.Subtract(customEpoch);
-            return (int)diff.TotalSeconds;
-        }
-
-        private static byte[] MacAddress()
-        {
-            foreach (var nic in NetworkInterface.GetAllNetworkInterfaces())
-            {
-                if (nic.OperationalStatus == OperationalStatus.Up)
-                {
-                    if (nic.Name.StartsWith("lo"))
-                    {
-                        continue;
-                    }
-                    Logger.Debug($"NIC={nic.Name}");
-                    return nic.GetPhysicalAddress().GetAddressBytes();
-                }
-            }
-            return null;
-        }
-
-        public string Next()
-        {
-            byte[] data = new byte[18];
-            Array.Copy(version, 0, data, 0, 2);
-            Array.Copy(macAddress, 0, data, 2, 6);
-            Array.Copy(pidBytes, 2, data, 8, 2);
-            int second = SecondsSinceCustomEpoch();
-            if (second != currentSecond)
-            {
-                currentSecond = second;
-                Interlocked.Exchange(ref sequenceInSecond, 0);
-            }
-            byte[] secondBytes = ToArray(second);
-            Array.Copy(secondBytes, 0, data, 10, 4);
-            int sequence = Interlocked.Increment(ref sequenceInSecond);
-            byte[] sequenceBytes = ToArray(sequence);
-            Array.Copy(sequenceBytes, 0, data, 14, 4);
-            return BitConverter.ToString(data).Replace("-", ""); ;
-        }
-    }
-
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index 82f4f1ef..99d61268 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -34,14 +34,14 @@ namespace Org.Apache.Rocketmq
         private readonly grpc::AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand>
             _streamingCall;
 
-        private readonly Client _client;
+        private readonly IClient _client;
         private readonly Channel<bool> _channel;
         private readonly Endpoints _endpoints;
         private readonly SemaphoreSlim _semaphore;
 
         public Session(Endpoints endpoints,
             AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand> streamingCall,
-            Client client)
+            IClient client)
         {
             _endpoints = endpoints;
             _semaphore = new SemaphoreSlim(1);
@@ -80,21 +80,6 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        // public async void xx()
-        // {
-        //     while (true)
-        //     {
-        //         var reader = _streamingCall.ResponseStream;
-        //         if (await reader.MoveNext(_client.TelemetryCts().Token))
-        //         {
-        //             var command = reader.Current;
-        //             Console.WriteLine("xxxxxxxx");
-        //             Console.WriteLine(command);
-        //         }
-        //     }
-        // }
-
-
         private void Loop()
         {
             Task.Run(async () =>
diff --git a/csharp/rocketmq-client-csharp/Settings.cs b/csharp/rocketmq-client-csharp/Settings.cs
index e7ea4e92..7716fc2d 100644
--- a/csharp/rocketmq-client-csharp/Settings.cs
+++ b/csharp/rocketmq-client-csharp/Settings.cs
@@ -1,5 +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.
+ */
+
 using System;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -8,10 +25,10 @@ namespace Org.Apache.Rocketmq
         protected readonly string ClientId;
         protected readonly ClientType ClientType;
         protected readonly Endpoints AccessPoint;
-        protected volatile RetryPolicy RetryPolicy;
+        protected volatile IRetryPolicy RetryPolicy;
         protected readonly TimeSpan RequestTimeout;
 
-        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, RetryPolicy retryPolicy,
+        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, IRetryPolicy retryPolicy,
             TimeSpan requestTimeout)
         {
             ClientId = clientId;
@@ -30,11 +47,11 @@ namespace Org.Apache.Rocketmq
             RequestTimeout = requestTimeout;
         }
 
-        public abstract rmq::Settings ToProtobuf();
+        public abstract Proto::Settings ToProtobuf();
 
-        public abstract void Sync(rmq::Settings settings);
+        public abstract void Sync(Proto::Settings settings);
 
-        public RetryPolicy GetRetryPolicy()
+        public IRetryPolicy GetRetryPolicy()
         {
             return RetryPolicy;
         }
diff --git a/csharp/rocketmq-client-csharp/Signature.cs b/csharp/rocketmq-client-csharp/Signature.cs
index e65125b6..8588c25a 100644
--- a/csharp/rocketmq-client-csharp/Signature.cs
+++ b/csharp/rocketmq-client-csharp/Signature.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 using System.Text;
 using grpc = Grpc.Core;
@@ -23,43 +24,37 @@ namespace Org.Apache.Rocketmq
 {
     public static class Signature
     {
-        public static void Sign(IClientConfig clientConfig, grpc::Metadata metadata)
+        public static void Sign(IClient client, grpc::Metadata metadata)
         {
+            var clientConfig = client.GetClientConfig();
             metadata.Add(MetadataConstants.LanguageKey, MetadataConstants.LanguageValue);
             metadata.Add(MetadataConstants.ClientVersionKey, MetadataConstants.Instance.ClientVersion);
-            metadata.Add(MetadataConstants.ClientIdKey, clientConfig.ClientId);
-            
-            string time = DateTime.Now.ToString(MetadataConstants.DateTimeFormat);
+            metadata.Add(MetadataConstants.ClientIdKey, client.GetClientId());
+
+            var time = DateTime.Now.ToString(MetadataConstants.DateTimeFormat);
             metadata.Add(MetadataConstants.DateTimeKey, time);
 
-            if (null != clientConfig.CredentialsProvider)
+            var credentials = clientConfig.CredentialsProvider?.Credentials;
+            if (credentials == null || credentials.expired())
             {
-                var credentials = clientConfig.CredentialsProvider.Credentials;
-                if (null == credentials || credentials.expired())
-                {
-                    return;
-                }
-
-                if (!String.IsNullOrEmpty(credentials.SessionToken))
-                {
-                    metadata.Add(MetadataConstants.SessionTokenKey, credentials.SessionToken);
-                }
+                return;
+            }
 
-                byte[] secretData = Encoding.ASCII.GetBytes(credentials.AccessSecret);
-                byte[] data = Encoding.ASCII.GetBytes(time);
-                HMACSHA1 signer = new HMACSHA1(secretData);
-                byte[] digest = signer.ComputeHash(data);
-                string hmac = BitConverter.ToString(digest).Replace("-", "");
-                string authorization = string.Format("{0} {1}={2}, {3}={4}, {5}={6}",
-                    MetadataConstants.AlgorithmKey,
-                    MetadataConstants.CredentialKey,
-                    credentials.AccessKey,
-                    MetadataConstants.SignedHeadersKey,
-                    MetadataConstants.DateTimeKey,
-                    MetadataConstants.SignatureKey,
-                    hmac);
-                metadata.Add(MetadataConstants.Authorization, authorization);
+            if (!string.IsNullOrEmpty(credentials.SessionToken))
+            {
+                metadata.Add(MetadataConstants.SessionTokenKey, credentials.SessionToken);
             }
+
+            var secretData = Encoding.ASCII.GetBytes(credentials.AccessSecret);
+            var data = Encoding.ASCII.GetBytes(time);
+            var signer = new HMACSHA1(secretData);
+            var digest = signer.ComputeHash(data);
+            var hmac = BitConverter.ToString(digest).Replace("-", "");
+            var authorization = $"{MetadataConstants.AlgorithmKey} " +
+                                $"{MetadataConstants.CredentialKey}={credentials.AccessKey}, " +
+                                $"{MetadataConstants.SignedHeadersKey}={MetadataConstants.DateTimeKey}, " +
+                                $"{MetadataConstants.SignatureKey}={hmac}";
+            metadata.Add(MetadataConstants.Authorization, authorization);
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StatusChecker.cs b/csharp/rocketmq-client-csharp/StatusChecker.cs
index cf15c204..641fd097 100644
--- a/csharp/rocketmq-client-csharp/StatusChecker.cs
+++ b/csharp/rocketmq-client-csharp/StatusChecker.cs
@@ -28,7 +28,7 @@ namespace Org.Apache.Rocketmq
 
         public static void Check(Proto.Status status, IMessage message)
         {
-            Proto.Code statusCode = status.Code;
+            var statusCode = status.Code;
 
             var statusMessage = status.Message;
             switch (statusCode)
diff --git a/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs b/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
index cf803377..b77da833 100644
--- a/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
+++ b/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-using System;
 using System.Collections.Generic;
 using System.Threading;
 using rmq = Apache.Rocketmq.V2;
diff --git a/csharp/rocketmq-client-csharp/TopicRouteException.cs b/csharp/rocketmq-client-csharp/TopicRouteException.cs
index 75462fd3..c80e8699 100644
--- a/csharp/rocketmq-client-csharp/TopicRouteException.cs
+++ b/csharp/rocketmq-client-csharp/TopicRouteException.cs
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 using System;
 namespace Org.Apache.Rocketmq
 {
diff --git a/csharp/tests/MessageIdGeneratorTest.cs b/csharp/tests/MessageIdGeneratorTest.cs
index c98e1131..19d0cebd 100644
--- a/csharp/tests/MessageIdGeneratorTest.cs
+++ b/csharp/tests/MessageIdGeneratorTest.cs
@@ -29,11 +29,11 @@ namespace tests
             MessageIdGenerator instance = MessageIdGenerator.GetInstance();
             var firstMessageId = instance.Next();
             Assert.AreEqual(34, firstMessageId.Length);
-            Assert.AreEqual(MessageIdGenerator.version, firstMessageId.Substring(0, 2));
+            Assert.AreEqual(MessageIdGenerator.Version, firstMessageId.Substring(0, 2));
 
             var secondMessageId = instance.Next();
             Assert.AreEqual(34, secondMessageId.Length);
-            Assert.AreEqual(MessageIdGenerator.version, secondMessageId.Substring(0, 2));
+            Assert.AreEqual(MessageIdGenerator.Version, secondMessageId.Substring(0, 2));
 
             Assert.AreNotEqual(firstMessageId, secondMessageId);
             Assert.AreEqual(firstMessageId.Substring(0, 24), secondMessageId.Substring(0, 24));
diff --git a/csharp/tests/SequenceGeneratorTest.cs b/csharp/tests/SequenceGeneratorTest.cs
deleted file mode 100644
index 9b553346..00000000
--- a/csharp/tests/SequenceGeneratorTest.cs
+++ /dev/null
@@ -1,49 +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.
- */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-
-using System;
-using System.Collections.Generic;
-
-namespace Org.Apache.Rocketmq
-{
-    [TestClass]
-    public class SequenceGeneratorTest
-    {
-
-        [ClassInitialize]
-        public static void SetUp(TestContext context)
-        {
-        }
-
-        [TestMethod]
-        public void testNext()
-        {
-            var set = new HashSet<string>();
-            for (int i = 0; i < 500000; i++)
-            {
-                var nextId = SequenceGenerator.Instance.Next();
-                if (set.Contains(nextId))
-                {
-                    Assert.Fail("SequenceGenerator violates uniqueness");
-                }
-                set.Add(nextId);
-            }
-        }
-    }
-}
-


[rocketmq-clients] 21/28: Make simpleconsumer works

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit ebd236ad94270407f90b105393e6ed58ce354ec8
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 20 19:55:29 2023 +0800

    Make simpleconsumer works
---
 csharp/rocketmq-client-csharp/Consumer.cs    | 11 ++++++++---
 csharp/rocketmq-client-csharp/MessageView.cs | 12 +++++++-----
 2 files changed, 15 insertions(+), 8 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Consumer.cs b/csharp/rocketmq-client-csharp/Consumer.cs
index 9a58104f..d5a1d9a5 100644
--- a/csharp/rocketmq-client-csharp/Consumer.cs
+++ b/csharp/rocketmq-client-csharp/Consumer.cs
@@ -33,7 +33,7 @@ namespace Org.Apache.Rocketmq
         {
             ConsumerGroup = consumerGroup;
         }
-        
+
         protected async Task<ReceiveMessageResult> ReceiveMessage(Proto.ReceiveMessageRequest request, MessageQueue mq,
             TimeSpan awaitDuration)
         {
@@ -83,11 +83,16 @@ namespace Org.Apache.Rocketmq
             };
         }
 
-        protected static Proto.ReceiveMessageRequest WrapReceiveMessageRequest(int batchSize, MessageQueue mq,
+        protected Proto.ReceiveMessageRequest WrapReceiveMessageRequest(int batchSize, MessageQueue mq,
             FilterExpression filterExpression, TimeSpan invisibleDuration)
         {
-            return new Proto.ReceiveMessageRequest()
+            var group = new Proto.Resource
+            {
+                Name = ConsumerGroup
+            };
+            return new Proto.ReceiveMessageRequest
             {
+                Group = group,
                 MessageQueue = mq.ToProtobuf(),
                 FilterExpression = WrapFilterExpression(filterExpression),
                 BatchSize = batchSize,
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index fd095819..b88c40b4 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -37,7 +37,7 @@ namespace Org.Apache.Rocketmq
         private readonly bool _corrupted;
 
         internal MessageView(string messageId, string topic, byte[] body, string tag, string messageGroup,
-            DateTime deliveryTime, List<string> keys, Dictionary<string, string> properties, string bornHost,
+            DateTime? deliveryTimestamp, List<string> keys, Dictionary<string, string> properties, string bornHost,
             DateTime bornTime, int deliveryAttempt, MessageQueue messageQueue, string receiptHandle, long offset,
             bool corrupted)
         {
@@ -46,7 +46,7 @@ namespace Org.Apache.Rocketmq
             Body = body;
             Tag = tag;
             MessageGroup = messageGroup;
-            DeliveryTime = deliveryTime;
+            DeliveryTimestamp = deliveryTimestamp;
             Keys = keys;
             Properties = properties;
             BornHost = bornHost;
@@ -68,7 +68,7 @@ namespace Org.Apache.Rocketmq
 
         public string MessageGroup { get; }
 
-        public DateTime DeliveryTime { get; }
+        public DateTime? DeliveryTimestamp { get; }
 
         public List<string> Keys { get; }
 
@@ -161,7 +161,9 @@ namespace Org.Apache.Rocketmq
 
             var tag = systemProperties.HasTag ? systemProperties.Tag : null;
             var messageGroup = systemProperties.HasMessageGroup ? systemProperties.MessageGroup : null;
-            var deliveryTime = systemProperties.DeliveryTimestamp.ToDateTime();
+            DateTime? deliveryTime = null == systemProperties.DeliveryTimestamp
+                ? null
+                : systemProperties.DeliveryTimestamp.ToDateTime();
             var keys = systemProperties.Keys.ToList();
 
             var bornHost = systemProperties.BornHost;
@@ -184,7 +186,7 @@ namespace Org.Apache.Rocketmq
         {
             return
                 $"{nameof(MessageId)}: {MessageId}, {nameof(Topic)}: {Topic}, {nameof(Tag)}: {Tag}," +
-                $" {nameof(MessageGroup)}: {MessageGroup}, {nameof(DeliveryTime)}: {DeliveryTime}," +
+                $" {nameof(MessageGroup)}: {MessageGroup}, {nameof(DeliveryTimestamp)}: {DeliveryTimestamp}," +
                 $" {nameof(Keys)}: {Keys}, {nameof(Properties)}: {Properties}, {nameof(BornHost)}: {BornHost}, " +
                 $"{nameof(BornTime)}: {BornTime}, {nameof(DeliveryAttempt)}: {DeliveryAttempt}";
         }


[rocketmq-clients] 23/28: Add nonce for TelemetryCommand

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit e272ee3da21560ae86ab560f6bd0bd9243ee2047
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 22 17:43:22 2023 +0800

    Add nonce for TelemetryCommand
---
 csharp/rocketmq-client-csharp/Client.cs | 38 ++++++++++++++++-----------------
 1 file changed, 19 insertions(+), 19 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 24e1e4ac..e0a4c553 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -42,7 +42,7 @@ namespace Org.Apache.Rocketmq
         private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromSeconds(1);
         private static readonly TimeSpan SettingsSyncSchedulePeriod = TimeSpan.FromSeconds(1);
         private readonly CancellationTokenSource _settingsSyncCts;
-        
+
         private static readonly TimeSpan StatsScheduleDelay = TimeSpan.FromSeconds(60);
         private static readonly TimeSpan StatsSchedulePeriod = TimeSpan.FromSeconds(60);
         private readonly CancellationTokenSource _statsCts;
@@ -201,13 +201,13 @@ namespace Org.Apache.Rocketmq
             {
                 Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
                 Dictionary<string, Task<TopicRouteData>> responses = new();
-                
+
                 foreach (var topic in GetTopics())
                 {
                     var task = FetchTopicRoute(topic);
                     responses[topic] = task;
                 }
-                
+
                 foreach (var item in responses.Keys)
                 {
                     try
@@ -219,7 +219,7 @@ namespace Org.Apache.Rocketmq
                         Logger.Error(e, $"Failed to update topic route cache, topic={item}");
                     }
                 }
-                
+
                 foreach (var topic in GetTopics())
                 {
                     await FetchTopicRoute(topic);
@@ -242,7 +242,6 @@ namespace Org.Apache.Rocketmq
                     var (_, session) = GetSession(endpoints);
                     await session.SyncSettings(false);
                     Logger.Info($"Sync settings to remote, endpoints={endpoints}");
-
                 }
             }
             catch (Exception e)
@@ -422,22 +421,11 @@ namespace Org.Apache.Rocketmq
             return ClientConfig;
         }
 
-        public virtual async void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
+        public virtual void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
             Proto.RecoverOrphanedTransactionCommand command)
         {
             Logger.Warn($"Ignore orphaned transaction recovery command from remote, which is not expected, " +
                         $"clientId={ClientId}, endpoints={endpoints}");
-            var status = new Proto.Status
-            {
-                Code = Proto.Code.InternalError,
-                Message = "Current client don't support transaction message recovery"
-            };
-            var telemetryCommand = new Proto.TelemetryCommand
-            {
-                Status = status
-            };
-            var (_, session) = GetSession(endpoints);
-            await session.WriteAsync(telemetryCommand);
         }
 
         public async void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
@@ -450,8 +438,14 @@ namespace Org.Apache.Rocketmq
                 Code = Proto.Code.Unsupported,
                 Message = "Message consumption verification is not supported"
             };
+            var verifyMessageResult = new Proto.VerifyMessageResult
+            {
+                Nonce = command.Nonce
+            };
+
             var telemetryCommand = new Proto.TelemetryCommand
             {
+                VerifyMessageResult = verifyMessageResult,
                 Status = status
             };
             var (_, session) = GetSession(endpoints);
@@ -468,9 +462,15 @@ namespace Org.Apache.Rocketmq
                 Code = Proto.Code.Unsupported,
                 Message = "C# don't support thread stack trace printing"
             };
-            var telemetryCommand = new Proto.TelemetryCommand()
+            var threadStackTrace = new Proto.ThreadStackTrace
             {
-                Status = status
+                Nonce = command.Nonce
+            };
+
+            var telemetryCommand = new Proto.TelemetryCommand
+            {
+                ThreadStackTrace = threadStackTrace,
+                Status = status,
             };
             var (_, session) = GetSession(endpoints);
             await session.WriteAsync(telemetryCommand);


[rocketmq-clients] 04/28: Add more logs

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 8417c0a9beaf055b17073c79e7db7008a9c87b23
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Tue Feb 7 19:03:30 2023 +0800

    Add more logs
---
 csharp/examples/ProducerNormalMessageExample.cs    |  4 ++-
 csharp/rocketmq-client-csharp/Client.cs            | 38 ++++++++++++++++++----
 .../ClientLoggerInterceptor.cs                     |  4 +--
 csharp/rocketmq-client-csharp/Endpoints.cs         |  7 +++-
 csharp/rocketmq-client-csharp/Producer.cs          | 15 ++++-----
 csharp/rocketmq-client-csharp/TopicRouteData.cs    | 26 ++++++++++-----
 6 files changed, 67 insertions(+), 27 deletions(-)

diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index c22bc695..1f80671c 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -17,6 +17,7 @@
 
 using System.Collections.Generic;
 using System.Text;
+using System.Threading;
 using System.Threading.Tasks;
 using NLog;
 using Org.Apache.Rocketmq;
@@ -64,8 +65,9 @@ namespace examples
             };
             var sendReceipt = await producer.Send(message);
             Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
+            Thread.Sleep(9999999);
             // Close the producer if you don't need it anymore.
-            await producer.Shutdown();
+            // await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index c4392e2d..a1c4b821 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -181,6 +181,7 @@ namespace Org.Apache.Rocketmq
 
         private async void UpdateTopicRouteCache()
         {
+            Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
             foreach (var topic in Topics.Keys)
             {
                 var topicRouteData = await FetchTopicRoute(topic);
@@ -230,9 +231,17 @@ namespace Org.Apache.Rocketmq
                 Endpoints = ClientConfig.Endpoints.ToProtobuf()
             };
 
-            var queryRouteResponse =
+            var response =
                 await Manager.QueryRoute(ClientConfig.Endpoints, request, ClientConfig.RequestTimeout);
-            var messageQueues = queryRouteResponse.MessageQueues.ToList();
+            var code = response.Status.Code;
+            if (!Proto.Code.Ok.Equals(code))
+            {
+                Logger.Error($"Failed to fetch topic route, clientId={ClientId}, topic={topic}, code={code}, " +
+                             $"statusMessage={response.Status.Message}");
+            }
+            StatusChecker.Check(response.Status, request);
+
+            var messageQueues = response.MessageQueues.ToList();
             return new TopicRouteData(messageQueues);
         }
 
@@ -246,11 +255,26 @@ namespace Org.Apache.Rocketmq
             }
 
             var request = WrapHeartbeatRequest();
-
-            var tasks = endpoints.Select(endpoint => Manager.Heartbeat(endpoint, request, ClientConfig.RequestTimeout))
-                .Cast<Task>().ToList();
-
-            await Task.WhenAll(tasks);
+            Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new ();
+            // Collect task into a map.
+            foreach (var item in endpoints)
+            {
+                var task = Manager.Heartbeat(item, request, ClientConfig.RequestTimeout);
+                responses[item]= task;
+            }
+            foreach (var item in responses.Keys)
+            {
+                var response = await responses[item];
+                var code = response.Status.Code;
+                
+                if (code.Equals(Proto.Code.Ok))
+                {
+                    Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
+                    return;
+                }
+                var statusMessage = response.Status.Message;
+                Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
+            }
         }
 
 
diff --git a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
index 01adddc8..d9622291 100644
--- a/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
+++ b/csharp/rocketmq-client-csharp/ClientLoggerInterceptor.cs
@@ -56,7 +56,7 @@ namespace Org.Apache.Rocketmq
             try
             {
                 var response = await t;
-                Logger.Debug($"Response received: {response}");
+                Logger.Trace($"Response received: {response}");
                 return response;
             }
             catch (Exception ex)
@@ -101,7 +101,7 @@ namespace Org.Apache.Rocketmq
             where TRequest : class
             where TResponse : class
         {
-            Logger.Debug($"Starting call. Type: {method.Type}. Request: {typeof(TRequest)}. Response: {typeof(TResponse)}");
+            Logger.Trace($"Starting call. Type: {method.Type}. Request: {typeof(TRequest)}. Response: {typeof(TResponse)}");
         }
 
         private void AddCallerMetadata<TRequest, TResponse>(ref ClientInterceptorContext<TRequest, TResponse> context)
diff --git a/csharp/rocketmq-client-csharp/Endpoints.cs b/csharp/rocketmq-client-csharp/Endpoints.cs
index 3228881c..e7cf5f9c 100644
--- a/csharp/rocketmq-client-csharp/Endpoints.cs
+++ b/csharp/rocketmq-client-csharp/Endpoints.cs
@@ -67,6 +67,11 @@ namespace Org.Apache.Rocketmq
             Addresses = addresses;
         }
 
+        public override string ToString()
+        {
+            return GrpcTarget;
+        }
+
         public string GrpcTarget
         {
             // TODO
@@ -82,7 +87,7 @@ namespace Org.Apache.Rocketmq
                 return "";
             }
         }
-
+        
         public bool Equals(Endpoints other)
         {
             if (ReferenceEquals(null, other))
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 9fa10ad3..56ec5b40 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -19,7 +19,12 @@ namespace Org.Apache.Rocketmq
         {
         }
 
-        public Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
+        public Producer(ClientConfig clientConfig, int maxAttempts) : this(clientConfig,
+            new ConcurrentDictionary<string, bool>(), maxAttempts)
+        {
+        }
+
+        private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
             base(clientConfig, topics)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.immediatelyRetryPolicy(maxAttempts);
@@ -72,12 +77,6 @@ namespace Org.Apache.Rocketmq
             if (!_publishingRouteDataCache.TryGetValue(message.Topic, out var publishingLoadBalancer))
             {
                 var topicRouteData = await FetchTopicRoute(message.Topic);
-                if (null == topicRouteData || null == topicRouteData.MessageQueues ||
-                    0 == topicRouteData.MessageQueues.Count)
-                {
-                    throw new TopicRouteException($"No topic route for {message.Topic}");
-                }
-
                 publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
                 _publishingRouteDataCache.TryAdd(message.Topic, publishingLoadBalancer);
             }
@@ -87,7 +86,7 @@ namespace Org.Apache.Rocketmq
             var maxAttempts = retryPolicy.getMaxAttempts();
             var candidates = publishingLoadBalancer.TakeMessageQueues(publishingMessage.MessageGroup, maxAttempts);
             Exception exception = null;
-            for (int attempt = 0; attempt < maxAttempts; attempt++)
+            for (var attempt = 0; attempt < maxAttempts; attempt++)
             {
                 try
                 {
diff --git a/csharp/rocketmq-client-csharp/TopicRouteData.cs b/csharp/rocketmq-client-csharp/TopicRouteData.cs
index 05418eaa..2be2b9a5 100644
--- a/csharp/rocketmq-client-csharp/TopicRouteData.cs
+++ b/csharp/rocketmq-client-csharp/TopicRouteData.cs
@@ -26,7 +26,7 @@ namespace Org.Apache.Rocketmq
     {
         public TopicRouteData(List<rmq::MessageQueue> messageQueues)
         {
-            List<MessageQueue> messageQueuesList = new List<MessageQueue>();
+            var messageQueuesList = new List<MessageQueue>();
             foreach (var mq in messageQueues)
             {
                 messageQueuesList.Add(new MessageQueue(mq));
@@ -40,17 +40,27 @@ namespace Org.Apache.Rocketmq
 
         public bool Equals(TopicRouteData other)
         {
-            if (ReferenceEquals(null, other)) return false;
-            if (ReferenceEquals(this, other)) return true;
-            return Equals(MessageQueues, other.MessageQueues);
+            if (ReferenceEquals(null, other))
+            {
+                return false;
+            }
+
+            return ReferenceEquals(this, other) || Equals(MessageQueues, other.MessageQueues);
         }
 
         public override bool Equals(object obj)
         {
-            if (ReferenceEquals(null, obj)) return false;
-            if (ReferenceEquals(this, obj)) return true;
-            if (obj.GetType() != this.GetType()) return false;
-            return Equals((TopicRouteData)obj);
+            if (ReferenceEquals(null, obj))
+            {
+                return false;
+            }
+
+            if (ReferenceEquals(this, obj))
+            {
+                return true;
+            }
+
+            return obj.GetType() == GetType() && Equals((TopicRouteData)obj);
         }
 
         public override int GetHashCode()


[rocketmq-clients] 14/28: Implement Client#OnVerrifyMessageCommand and Client#OnPrintThreadStackTraceCommand

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 30c1832bb83c00c946786a2b44cb073e5050150e
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 15 17:22:11 2023 +0800

    Implement Client#OnVerrifyMessageCommand and Client#OnPrintThreadStackTraceCommand
---
 csharp/rocketmq-client-csharp/Client.cs  | 43 +++++++++++++++++++++++++++-----
 csharp/rocketmq-client-csharp/Session.cs |  5 ++++
 2 files changed, 42 insertions(+), 6 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 89208db5..fc6871b9 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -171,7 +171,6 @@ namespace Org.Apache.Rocketmq
             OnTopicRouteDataUpdated0(topic, topicRouteData);
         }
 
-
         /**
          * Return all endpoints of brokers in route table.
          */
@@ -201,7 +200,8 @@ namespace Org.Apache.Rocketmq
             }
             catch (Exception e)
             {
-                Logger.Error(e, $"[Bug] unexpected exception raised during topic route cache update, clientId={ClientId}");
+                Logger.Error(e,
+                    $"[Bug] unexpected exception raised during topic route cache update, clientId={ClientId}");
             }
         }
 
@@ -324,14 +324,16 @@ namespace Org.Apache.Rocketmq
                         Logger.Info($"Send heartbeat successfully, endpoints={item}, clientId={ClientId}");
                         if (Isolated.TryRemove(item, out _))
                         {
-                            Logger.Info($"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
+                            Logger.Info(
+                                $"Rejoin endpoints which was isolated before, endpoints={item}, clientId={ClientId}");
                         }
 
                         return;
                     }
 
                     var statusMessage = response.Status.Message;
-                    Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
+                    Logger.Info(
+                        $"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
                 }
             }
             catch (Exception e)
@@ -390,14 +392,43 @@ namespace Org.Apache.Rocketmq
         public void OnRecoverOrphanedTransactionCommand(Endpoints endpoints,
             Proto.RecoverOrphanedTransactionCommand command)
         {
+            // TODO
         }
 
-        public void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
+        public async void OnVerifyMessageCommand(Endpoints endpoints, Proto.VerifyMessageCommand command)
         {
+            // Only push consumer support message consumption verification.
+            Logger.Warn($"Ignore verify message command from remote, which is not expected, clientId={ClientId}, " +
+                        $"endpoints={endpoints}, command={command}");
+            var status = new Proto.Status
+            {
+                Code = Proto.Code.Unsupported,
+                Message = "Message consumption verification is not supported"
+            };
+            var telemetryCommand = new Proto.TelemetryCommand()
+            {
+                Status = status
+            };
+            var (_, session) = GetSession(endpoints);
+            await session.write(telemetryCommand);
         }
 
-        public void OnPrintThreadStackTraceCommand(Endpoints endpoints, Proto.PrintThreadStackTraceCommand command)
+        public async void OnPrintThreadStackTraceCommand(Endpoints endpoints,
+            Proto.PrintThreadStackTraceCommand command)
         {
+            Logger.Warn("Ignore thread stack trace printing command from remote because it is still not supported, " +
+                        $"clientId={ClientId}, endpoints={endpoints}");
+            var status = new Proto.Status
+            {
+                Code = Proto.Code.Unsupported,
+                Message = "C# don't support thread stack trace printing"
+            };
+            var telemetryCommand = new Proto.TelemetryCommand()
+            {
+                Status = status
+            };
+            var (_, session) = GetSession(endpoints);
+            await session.write(telemetryCommand);
         }
 
         public void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index c948ef86..24ff9c7c 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -53,6 +53,11 @@ namespace Org.Apache.Rocketmq
             Loop();
         }
 
+        public async Task write(Proto.TelemetryCommand telemetryCommand)
+        {
+            var writer = _streamingCall.RequestStream;
+            await writer.WriteAsync(telemetryCommand);
+        } 
 
         public async Task SyncSettings(bool awaitResp)
         {


[rocketmq-clients] 17/28: Add more stats info

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 7be8d6da9fae153f5781fea13022849e9288d4d6
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Fri Feb 17 10:31:42 2023 +0800

    Add more stats info
---
 csharp/rocketmq-client-csharp/Client.cs | 29 ++++++++++++++++++++++-------
 1 file changed, 22 insertions(+), 7 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 63c24c91..8e49851a 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -37,11 +37,15 @@ namespace Org.Apache.Rocketmq
 
         private static readonly TimeSpan TopicRouteUpdateScheduleDelay = TimeSpan.FromSeconds(10);
         private static readonly TimeSpan TopicRouteUpdateSchedulePeriod = TimeSpan.FromSeconds(30);
-        private readonly CancellationTokenSource _topicRouteUpdateCtx;
+        private readonly CancellationTokenSource _topicRouteUpdateCts;
 
         private static readonly TimeSpan SettingsSyncScheduleDelay = TimeSpan.FromSeconds(1);
         private static readonly TimeSpan SettingsSyncSchedulePeriod = TimeSpan.FromMinutes(5);
-        private readonly CancellationTokenSource _settingsSyncCtx;
+        private readonly CancellationTokenSource _settingsSyncCts;
+        
+        private static readonly TimeSpan StatsScheduleDelay = TimeSpan.FromSeconds(1);
+        private static readonly TimeSpan StatsSchedulePeriod = TimeSpan.FromSeconds(1);
+        private readonly CancellationTokenSource _statsCts;
 
         protected readonly ClientConfig ClientConfig;
         protected readonly IClientManager ClientManager;
@@ -63,10 +67,11 @@ namespace Org.Apache.Rocketmq
             Isolated = new ConcurrentDictionary<Endpoints, bool>();
             _topicRouteCache = new ConcurrentDictionary<string, TopicRouteData>();
 
-            _topicRouteUpdateCtx = new CancellationTokenSource();
-            _settingsSyncCtx = new CancellationTokenSource();
+            _topicRouteUpdateCts = new CancellationTokenSource();
+            _settingsSyncCts = new CancellationTokenSource();
             _heartbeatCts = new CancellationTokenSource();
             _telemetryCts = new CancellationTokenSource();
+            _statsCts = new CancellationTokenSource();
 
             _sessionsTable = new Dictionary<Endpoints, Session>();
             _sessionLock = new ReaderWriterLockSlim();
@@ -76,10 +81,11 @@ namespace Org.Apache.Rocketmq
         {
             Logger.Debug($"Begin to start the rocketmq client, clientId={ClientId}");
             ScheduleWithFixedDelay(UpdateTopicRouteCache, TopicRouteUpdateScheduleDelay, TopicRouteUpdateSchedulePeriod,
-                _topicRouteUpdateCtx.Token);
+                _topicRouteUpdateCts.Token);
             ScheduleWithFixedDelay(Heartbeat, HeartbeatScheduleDelay, HeartbeatSchedulePeriod, _heartbeatCts.Token);
             ScheduleWithFixedDelay(SyncSettings, SettingsSyncScheduleDelay, SettingsSyncSchedulePeriod,
-                _settingsSyncCtx.Token);
+                _settingsSyncCts.Token);
+            ScheduleWithFixedDelay(Stats, StatsScheduleDelay, StatsSchedulePeriod, _statsCts.Token);
             foreach (var topic in GetTopics())
             {
                 await FetchTopicRoute(topic);
@@ -91,7 +97,7 @@ namespace Org.Apache.Rocketmq
         public virtual async Task Shutdown()
         {
             Logger.Debug($"Begin to shutdown rocketmq client, clientId={ClientId}");
-            _topicRouteUpdateCtx.Cancel();
+            _topicRouteUpdateCts.Cancel();
             _heartbeatCts.Cancel();
             _telemetryCts.Cancel();
             NotifyClientTermination();
@@ -221,6 +227,15 @@ namespace Org.Apache.Rocketmq
             }
         }
 
+        private void Stats()
+        {
+            ThreadPool.GetAvailableThreads(out var availableWorker, out var availableIo);
+            Logger.Info($"ThreadCount={ThreadPool.ThreadCount}, " +
+                        $"CompletedWorkItemCount={ThreadPool.CompletedWorkItemCount}, " +
+                        $"PendingWorkItemCount={ThreadPool.PendingWorkItemCount}, AvailableWorkerThreads={availableWorker}, " +
+                        $"AvailableCompletionPortThreads={availableIo}, ClientId={ClientId}");
+        }
+
         private void ScheduleWithFixedDelay(Action action, TimeSpan delay, TimeSpan period, CancellationToken token)
         {
             Task.Run(async () =>


[rocketmq-clients] 11/28: Add more logs

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 85ae3d541f06034e2e5dbd5130b4d706c4fbd79c
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 20:28:59 2023 +0800

    Add more logs
---
 csharp/examples/QuickStart.cs                   |  4 ++--
 csharp/rocketmq-client-csharp/Client.cs         | 10 +++++++---
 csharp/rocketmq-client-csharp/Producer.cs       |  2 +-
 csharp/rocketmq-client-csharp/SimpleConsumer.cs |  2 +-
 4 files changed, 11 insertions(+), 7 deletions(-)

diff --git a/csharp/examples/QuickStart.cs b/csharp/examples/QuickStart.cs
index 8323218f..474d6063 100644
--- a/csharp/examples/QuickStart.cs
+++ b/csharp/examples/QuickStart.cs
@@ -27,11 +27,11 @@ namespace examples
         {
             // Console.WriteLine(MetadataConstants.Instance.ClientVersion);
 
-            // ProducerNormalMessageExample.QuickStart().Wait();
+            ProducerNormalMessageExample.QuickStart().Wait();
             // await ProducerFifoMessageExample.QuickStart();
             // await ProducerDelayMessageExample.QuickStart();
             // await SimpleConsumerExample.QuickStart();
-            ProducerBenchmark.QuickStart();
+            // ProducerBenchmark.QuickStart();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index afbfbe5b..9d4d7e69 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -134,7 +134,7 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        protected abstract ICollection<string> GetTopics();
+        protected abstract IEnumerable<string> GetTopics();
 
         protected abstract Proto::HeartbeatRequest WrapHeartbeatRequest();
 
@@ -156,10 +156,14 @@ namespace Org.Apache.Rocketmq
             foreach (var endpoints in newEndpoints)
             {
                 var (created, session) = GetSession(endpoints);
-                if (created)
+                if (!created)
                 {
-                    await session.SyncSettings(true);
+                    continue;
                 }
+
+                Logger.Info($"Begin to establish session for endpoints={endpoints}, clientId={ClientId}");
+                await session.SyncSettings(true);
+                Logger.Info($"Establish session for endpoints={endpoints} successfully, clientId={ClientId}");
             }
 
             _topicRouteCache[topic] = topicRouteData;
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index 7803e2aa..170dbeab 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -61,7 +61,7 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        protected override ICollection<string> GetTopics()
+        protected override IEnumerable<string> GetTopics()
         {
             return _publishingTopics.Keys;
         }
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index 0481f7e8..8a8922fd 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -79,7 +79,7 @@ namespace Org.Apache.Rocketmq
             Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
         }
         
-        protected override ICollection<string> GetTopics()
+        protected override IEnumerable<string> GetTopics()
         {
             return _subscriptionExpressions.Keys;
         }


[rocketmq-clients] 07/28: Implement simple consumer

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 26bb67c51c279013fabd32e4eeb7809aaf9e4f56
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 13 17:13:57 2023 +0800

    Implement simple consumer
---
 csharp/examples/ProducerDelayMessageExample.cs     |  77 +++++----
 csharp/examples/ProducerNormalMessageExample.cs    |   6 +-
 csharp/examples/SimpleConsumerExample.cs           |  68 ++++----
 csharp/rocketmq-client-csharp/Address.cs           |  25 ++-
 csharp/rocketmq-client-csharp/AddressScheme.cs     |  10 +-
 csharp/rocketmq-client-csharp/Broker.cs            |  10 ++
 csharp/rocketmq-client-csharp/Client.cs            |  25 +--
 csharp/rocketmq-client-csharp/ClientManager.cs     |  33 ++--
 csharp/rocketmq-client-csharp/Consumer.cs          |  99 +++++++++++
 .../rocketmq-client-csharp/Error/StatusChecker.cs  |  41 -----
 csharp/rocketmq-client-csharp/FilterExpression.cs  |   6 +
 csharp/rocketmq-client-csharp/IClientManager.cs    |  33 ++++
 csharp/rocketmq-client-csharp/MessageQueue.cs      |  27 ++-
 csharp/rocketmq-client-csharp/MessageView.cs       |  44 ++---
 csharp/rocketmq-client-csharp/MqEncoding.cs        |  10 +-
 csharp/rocketmq-client-csharp/Permission.cs        |  22 ++-
 csharp/rocketmq-client-csharp/Producer.cs          |   8 +-
 csharp/rocketmq-client-csharp/Publishing.cs        |   4 +-
 .../PublishingLoadBalancer.cs                      |  33 ++--
 .../rocketmq-client-csharp/PublishingSettings.cs   |  17 +-
 .../{IProducer.cs => ReceiveMessageResult.cs}      |  14 +-
 csharp/rocketmq-client-csharp/Resource.cs          |   8 +-
 csharp/rocketmq-client-csharp/RpcClient.cs         |  42 ++---
 csharp/rocketmq-client-csharp/Settings.cs          |  10 +-
 csharp/rocketmq-client-csharp/SimpleConsumer.cs    | 192 +++++++++++++++++++++
 .../SimpleSubscriptionSettings.cs                  |  95 ++++++++++
 csharp/rocketmq-client-csharp/StatusChecker.cs     |   1 +
 .../SubscriptionLoadBalancer.cs                    |  38 ++--
 csharp/rocketmq-client-csharp/TopicRouteData.cs    |  10 +-
 csharp/rocketmq-client-csharp/Utilities.cs         |   5 +-
 csharp/tests/UnitTest1.cs                          |  51 +++---
 31 files changed, 748 insertions(+), 316 deletions(-)

diff --git a/csharp/examples/ProducerDelayMessageExample.cs b/csharp/examples/ProducerDelayMessageExample.cs
index 27e32e76..9ad5fbb9 100644
--- a/csharp/examples/ProducerDelayMessageExample.cs
+++ b/csharp/examples/ProducerDelayMessageExample.cs
@@ -24,49 +24,50 @@ using Org.Apache.Rocketmq;
 
 namespace examples
 {
-    static class ProducerDelayMessageExample
+    internal static class ProducerDelayMessageExample
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
         internal static async Task QuickStart()
         {
-            // string accessKey = "yourAccessKey";
-            // string secretKey = "yourSecretKey";
-            // // Credential provider is optional for client configuration.
-            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            // string endpoints = "foobar.com:8080";
-            // // In most case, you don't need to create too many producers, single pattern is recommended.
-            // var producer = new Producer(endpoints)
-            // {
-            //     CredentialsProvider = credentialsProvider
-            // };
-            // string topic = "yourDelayTopic";
-            // // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
-            // // the topic route before message publishing.
-            // producer.AddTopicOfInterest(topic);
-            //
-            // await producer.Start();
-            // // Define your message body.
-            // var bytes = Encoding.UTF8.GetBytes("foobar");
-            // string tag = "yourMessageTagA";
-            // // You could set multiple keys for the single message.
-            // var keys = new List<string>
-            // {
-            //     "yourMessageKey-2f00df144e48",
-            //     "yourMessageKey-49df1dd332b7"
-            // };
-            // // Set topic for current message.
-            // var message = new Message(topic, bytes)
-            // {
-            //     Tag = tag,
-            //     Keys = keys,
-            //     // Essential for DELAY message.
-            //     DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(30)
-            // };
-            // var sendReceipt = await producer.Send(message);
-            // Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
-            // // Close the producer if you don't need it anymore.
-            // await producer.Shutdown();
+            const string accessKey = "yourAccessKey";
+            const string secretKey = "yourSecretKey";
+            // Credential provider is optional for client configuration.
+            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            var clientConfig = new ClientConfig(endpoints)
+            {
+                CredentialsProvider = credentialsProvider
+            };
+            // In most case, you don't need to create too many producers, single pattern is recommended.
+            var producer = new Producer(clientConfig);
+            const string topic = "yourDelayTopic";
+            // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
+            // the topic route before message publishing.
+            producer.SetTopics(topic);
+
+            await producer.Start();
+            // Define your message body.
+            var bytes = Encoding.UTF8.GetBytes("foobar");
+            const string tag = "yourMessageTagA";
+            // You could set multiple keys for the single message.
+            var keys = new List<string>
+            {
+                "yourMessageKey-2f00df144e48",
+                "yourMessageKey-49df1dd332b7"
+            };
+            // Set topic for current message.
+            var message = new Message(topic, bytes)
+            {
+                Tag = tag,
+                Keys = keys,
+                // Essential for DELAY message.
+                DeliveryTimestamp = DateTime.UtcNow + TimeSpan.FromSeconds(30)
+            };
+            var sendReceipt = await producer.Send(message);
+            Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
+            // Close the producer if you don't need it anymore.
+            await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index 16791a13..3274ade2 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -32,6 +32,7 @@ namespace examples
         {
             const string accessKey = "5jFk0wK7OU6Uq395";
             const string secretKey = "V1u8z19URHs4o6RQ";
+
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
             const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
@@ -41,12 +42,11 @@ namespace examples
             };
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
-            
+
             const string topic = "lingchu_normal_topic";
             producer.SetTopics(topic);
             // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
             // the topic route before message publishing.
-
             await producer.Start();
             // Define your message body.
             var bytes = Encoding.UTF8.GetBytes("foobar");
@@ -67,7 +67,7 @@ namespace examples
             Logger.Info($"Send message successfully, sendReceipt={sendReceipt}");
             Thread.Sleep(9999999);
             // Close the producer if you don't need it anymore.
-            // await producer.Shutdown();
+            await producer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/examples/SimpleConsumerExample.cs b/csharp/examples/SimpleConsumerExample.cs
index 80299fbb..b41125c8 100644
--- a/csharp/examples/SimpleConsumerExample.cs
+++ b/csharp/examples/SimpleConsumerExample.cs
@@ -23,47 +23,43 @@ using Org.Apache.Rocketmq;
 
 namespace examples
 {
-    static class SimpleConsumerExample
+    internal static class SimpleConsumerExample
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
         internal static async Task QuickStart()
         {
-            // string accessKey = "yourAccessKey";
-            // string secretKey = "yourSecretKey";
-            // // Credential provider is optional for client configuration.
-            // var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            // string endpoints = "foobar.com:8080";
-            //
-            // string consumerGroup = "yourConsumerGroup";
-            // SimpleConsumer simpleConsumer = new SimpleConsumer(endpoints, consumerGroup)
-            // {
-            //     CredentialsProvider = credentialsProvider
-            // };
-            //
-            // string topic = "yourTopic";
-            // string tag = "tagA";
-            // // Set topic subscription for consumer.
-            // simpleConsumer.Subscribe(topic, new FilterExpression(tag, ExpressionType.Tag));
-            // await simpleConsumer.Start();
-            //
-            // int maxMessageNum = 16;
-            // TimeSpan invisibleDuration = TimeSpan.FromSeconds(15);
-            // var messages = await simpleConsumer.Receive(maxMessageNum, invisibleDuration);
-            // Logger.Info($"{messages.Count} messages has been received.");
-            //
-            // var tasks = new List<Task>();
-            // foreach (var message in messages)
-            // {
-            //     Logger.Info($"Received a message, topic={message.Topic}, message-id={message.MessageId}.");
-            //     var task = simpleConsumer.Ack(message);
-            //     tasks.Add(task);
-            // }
-            //
-            // await Task.WhenAll(tasks);
-            // Logger.Info($"{tasks.Count} messages have been acknowledged.");
-            // // Close the consumer if you don't need it anymore.
-            // await simpleConsumer.Shutdown();
+            const string accessKey = "yourAccessKey";
+            const string secretKey = "yourSecretKey";
+
+            // Credential provider is optional for client configuration.
+            var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
+            const string endpoints = "foobar.com:8080";
+            var clientConfig = new ClientConfig(endpoints)
+            {
+                CredentialsProvider = credentialsProvider
+            };
+            // Add your subscriptions.
+            const string consumerGroup = "yourConsumerGroup";
+            var subscription = new Dictionary<string, FilterExpression>
+                { { consumerGroup, new FilterExpression("*") } };
+            // In most case, you don't need to create too many consumers, single pattern is recommended.
+            var simpleConsumer =
+                new SimpleConsumer(clientConfig, consumerGroup, TimeSpan.FromSeconds(15), subscription);
+
+            await simpleConsumer.Start();
+            var messageViews = await simpleConsumer.Receive(16, TimeSpan.FromSeconds(15));
+            foreach (var message in messageViews)
+            {
+                Logger.Info($"Received a message, topic={message.Topic}, message-id={message.MessageId}");
+                await simpleConsumer.Ack(message);
+                Logger.Info($"Message is acknowledged successfully, message-id={message.MessageId}");
+                // await simpleConsumer.ChangeInvisibleDuration(message, TimeSpan.FromSeconds(15));
+                // Logger.Info($"Changing message invisible duration successfully, message=id={message.MessageId}");
+            }
+
+            // Close the consumer if you don't need it anymore.
+            await simpleConsumer.Shutdown();
         }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Address.cs b/csharp/rocketmq-client-csharp/Address.cs
index fca83530..a66a6369 100644
--- a/csharp/rocketmq-client-csharp/Address.cs
+++ b/csharp/rocketmq-client-csharp/Address.cs
@@ -1,11 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 using System;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class Address : IEquatable<Address>
     {
-        public Address(rmq.Address address)
+        public Address(Proto.Address address)
         {
             Host = address.Host;
             Port = address.Port;
@@ -21,9 +38,9 @@ namespace Org.Apache.Rocketmq
 
         public int Port { get; }
 
-        public rmq.Address ToProtobuf()
+        public Proto.Address ToProtobuf()
         {
-            return new rmq.Address
+            return new Proto.Address
             {
                 Host = Host,
                 Port = Port
diff --git a/csharp/rocketmq-client-csharp/AddressScheme.cs b/csharp/rocketmq-client-csharp/AddressScheme.cs
index 6f36f546..06a5ea32 100644
--- a/csharp/rocketmq-client-csharp/AddressScheme.cs
+++ b/csharp/rocketmq-client-csharp/AddressScheme.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -28,17 +28,17 @@ namespace Org.Apache.Rocketmq
 
     public static class AddressSchemeHelper
     {
-        public static rmq.AddressScheme ToProtobuf(AddressScheme scheme)
+        public static Proto.AddressScheme ToProtobuf(AddressScheme scheme)
         {
             switch (scheme)
             {
                 case AddressScheme.Ipv4:
-                    return rmq.AddressScheme.Ipv4;
+                    return Proto.AddressScheme.Ipv4;
                 case AddressScheme.Ipv6:
-                    return rmq.AddressScheme.Ipv6;
+                    return Proto.AddressScheme.Ipv6;
                 case AddressScheme.DomainName:
                 default:
-                    return rmq.AddressScheme.DomainName;
+                    return Proto.AddressScheme.DomainName;
             }
         }
     }
diff --git a/csharp/rocketmq-client-csharp/Broker.cs b/csharp/rocketmq-client-csharp/Broker.cs
index 6b426a5a..6a2f957a 100644
--- a/csharp/rocketmq-client-csharp/Broker.cs
+++ b/csharp/rocketmq-client-csharp/Broker.cs
@@ -31,5 +31,15 @@ namespace Org.Apache.Rocketmq
         public string Name { get; }
         public int Id { get; }
         public Endpoints Endpoints { get; }
+
+        public Proto.Broker ToProtobuf()
+        {
+            return new Proto.Broker()
+            {
+                Name = Name,
+                Id = Id,
+                Endpoints = Endpoints.ToProtobuf()
+            };
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 51ecb936..3b260022 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -44,8 +44,8 @@ namespace Org.Apache.Rocketmq
         protected readonly IClientManager ClientManager;
         protected readonly string ClientId;
 
-        protected readonly ConcurrentDictionary<string, bool> Topics;
-        
+        protected readonly ICollection<string> Topics;
+
         protected readonly ConcurrentDictionary<Endpoints, bool> Isolated;
         private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteCache;
         private readonly CancellationTokenSource _telemetryCts;
@@ -53,7 +53,7 @@ namespace Org.Apache.Rocketmq
         private readonly Dictionary<Endpoints, Session> _sessionsTable;
         private readonly ReaderWriterLockSlim _sessionLock;
 
-        protected Client(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics)
+        protected Client(ClientConfig clientConfig, ICollection<string> topics)
         {
             ClientConfig = clientConfig;
             Topics = topics;
@@ -78,7 +78,7 @@ namespace Org.Apache.Rocketmq
             ScheduleWithFixedDelay(UpdateTopicRouteCache, TopicRouteUpdateScheduleDelay, _topicRouteUpdateCtx.Token);
             ScheduleWithFixedDelay(Heartbeat, HeartbeatScheduleDelay, _heartbeatCts.Token);
             ScheduleWithFixedDelay(SyncSettings, SettingsSyncScheduleDelay, _settingsSyncCtx.Token);
-            foreach (var topic in Topics.Keys)
+            foreach (var topic in Topics)
             {
                 await FetchTopicRoute(topic);
             }
@@ -135,7 +135,7 @@ namespace Org.Apache.Rocketmq
         protected abstract Proto::HeartbeatRequest WrapHeartbeatRequest();
 
 
-        protected abstract void OnTopicRouteDataFetched0(string topic, TopicRouteData topicRouteData);
+        protected abstract void OnTopicRouteDataUpdated0(string topic, TopicRouteData topicRouteData);
 
 
         private async Task OnTopicRouteDataFetched(string topic, TopicRouteData topicRouteData)
@@ -159,7 +159,7 @@ namespace Org.Apache.Rocketmq
             }
 
             _topicRouteCache[topic] = topicRouteData;
-            OnTopicRouteDataFetched0(topic, topicRouteData);
+            OnTopicRouteDataUpdated0(topic, topicRouteData);
         }
 
 
@@ -183,7 +183,7 @@ namespace Org.Apache.Rocketmq
         private async void UpdateTopicRouteCache()
         {
             Logger.Info($"Start to update topic route cache for a new round, clientId={ClientId}");
-            foreach (var topic in Topics.Keys)
+            foreach (var topic in Topics)
             {
                 var topicRouteData = await FetchTopicRoute(topic);
                 _topicRouteCache[topic] = topicRouteData;
@@ -250,6 +250,7 @@ namespace Org.Apache.Rocketmq
                 Logger.Error($"Failed to fetch topic route, clientId={ClientId}, topic={topic}, code={code}, " +
                              $"statusMessage={response.Status.Message}");
             }
+
             StatusChecker.Check(response.Status, request);
 
             var messageQueues = response.MessageQueues.ToList();
@@ -266,13 +267,14 @@ namespace Org.Apache.Rocketmq
             }
 
             var request = WrapHeartbeatRequest();
-            Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new ();
+            Dictionary<Endpoints, Task<Proto.HeartbeatResponse>> responses = new();
             // Collect task into a map.
             foreach (var item in endpoints)
             {
                 var task = ClientManager.Heartbeat(item, request, ClientConfig.RequestTimeout);
-                responses[item]= task;
+                responses[item] = task;
             }
+
             foreach (var item in responses.Keys)
             {
                 var response = await responses[item];
@@ -291,11 +293,10 @@ namespace Org.Apache.Rocketmq
                 }
 
                 var statusMessage = response.Status.Message;
-                Logger.Info($"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
+                Logger.Info(
+                    $"Failed to send heartbeat, endpoints={item}, code={code}, statusMessage={statusMessage}, clientId={ClientId}");
             }
         }
-        
-        
 
 
         public grpc.Metadata Sign()
diff --git a/csharp/rocketmq-client-csharp/ClientManager.cs b/csharp/rocketmq-client-csharp/ClientManager.cs
index 3eef2fe6..967a2ac1 100644
--- a/csharp/rocketmq-client-csharp/ClientManager.cs
+++ b/csharp/rocketmq-client-csharp/ClientManager.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using System;
 using System.Threading;
 using System.Threading.Tasks;
@@ -78,7 +78,7 @@ namespace Org.Apache.Rocketmq
             _clientLock.EnterReadLock();
             try
             {
-                List<Task> tasks = new List<Task>();
+                var tasks = new List<Task>();
                 foreach (var item in _rpcClients)
                 {
                     tasks.Add(item.Value.Shutdown());
@@ -92,56 +92,57 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public grpc::AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> Telemetry(
+        public grpc::AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand> Telemetry(
             Endpoints endpoints)
         {
             return GetRpcClient(endpoints).Telemetry(_client.Sign());
         }
 
-        public async Task<rmq.QueryRouteResponse> QueryRoute(Endpoints endpoints, rmq.QueryRouteRequest request,
+        public async Task<Proto.QueryRouteResponse> QueryRoute(Endpoints endpoints, Proto.QueryRouteRequest request,
             TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).QueryRoute(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq.HeartbeatResponse> Heartbeat(Endpoints endpoints, rmq.HeartbeatRequest request,
+        public async Task<Proto.HeartbeatResponse> Heartbeat(Endpoints endpoints, Proto.HeartbeatRequest request,
             TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).Heartbeat(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq.NotifyClientTerminationResponse> NotifyClientTermination(Endpoints endpoints,
-            rmq.NotifyClientTerminationRequest request, TimeSpan timeout)
+        public async Task<Proto.NotifyClientTerminationResponse> NotifyClientTermination(Endpoints endpoints,
+            Proto.NotifyClientTerminationRequest request, TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).NotifyClientTermination(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq::SendMessageResponse> SendMessage(Endpoints endpoints, rmq::SendMessageRequest request,
+        public async Task<Proto::SendMessageResponse> SendMessage(Endpoints endpoints,
+            Proto::SendMessageRequest request,
             TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).SendMessage(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq::QueryAssignmentResponse> QueryAssignment(Endpoints endpoints,
-            rmq.QueryAssignmentRequest request, TimeSpan timeout)
+        public async Task<Proto::QueryAssignmentResponse> QueryAssignment(Endpoints endpoints,
+            Proto.QueryAssignmentRequest request, TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).QueryAssignment(_client.Sign(), request, timeout);
         }
 
-        public async Task<List<rmq::ReceiveMessageResponse>> ReceiveMessage(Endpoints endpoints,
-            rmq.ReceiveMessageRequest request, TimeSpan timeout)
+        public async Task<List<Proto::ReceiveMessageResponse>> ReceiveMessage(Endpoints endpoints,
+            Proto.ReceiveMessageRequest request, TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).ReceiveMessage(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq::AckMessageResponse> AckMessage(Endpoints endpoints,
-            rmq.AckMessageRequest request, TimeSpan timeout)
+        public async Task<Proto::AckMessageResponse> AckMessage(Endpoints endpoints,
+            Proto.AckMessageRequest request, TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).AckMessage(_client.Sign(), request, timeout);
         }
 
-        public async Task<rmq::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Endpoints endpoints,
-            rmq.ChangeInvisibleDurationRequest request, TimeSpan timeout)
+        public async Task<Proto::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Endpoints endpoints,
+            Proto.ChangeInvisibleDurationRequest request, TimeSpan timeout)
         {
             return await GetRpcClient(endpoints).ChangeInvisibleDuration(_client.Sign(), request, timeout);
         }
diff --git a/csharp/rocketmq-client-csharp/Consumer.cs b/csharp/rocketmq-client-csharp/Consumer.cs
new file mode 100644
index 00000000..25df84d3
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/Consumer.cs
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+using System.Collections.Generic;
+using System.Linq;
+using System.Threading.Tasks;
+using Google.Protobuf.WellKnownTypes;
+using Proto = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public abstract class Consumer : Client
+    {
+        protected readonly string ConsumerGroup;
+
+        protected Consumer(ClientConfig clientConfig, string consumerGroup, ICollection<string> topics) : base(
+            clientConfig, topics)
+        {
+            ConsumerGroup = consumerGroup;
+        }
+        
+        protected async Task<ReceiveMessageResult> ReceiveMessage(Proto.ReceiveMessageRequest request, MessageQueue mq,
+            TimeSpan awaitDuration)
+        {
+            var tolerance = ClientConfig.RequestTimeout;
+            var timeout = tolerance.Add(awaitDuration);
+            var response = await ClientManager.ReceiveMessage(mq.Broker.Endpoints, request, timeout);
+            var status = new Proto.Status()
+            {
+                Code = Proto.Code.InternalServerError,
+                Message = "Status was not set by server"
+            };
+            var messageList = new List<Proto.Message>();
+            foreach (var entry in response)
+            {
+                switch (entry.ContentCase)
+                {
+                    case Proto.ReceiveMessageResponse.ContentOneofCase.Status:
+                        status = entry.Status;
+                        break;
+                    case Proto.ReceiveMessageResponse.ContentOneofCase.Message:
+                        messageList.Add(entry.Message);
+                        break;
+                    case Proto.ReceiveMessageResponse.ContentOneofCase.DeliveryTimestamp:
+                    case Proto.ReceiveMessageResponse.ContentOneofCase.None:
+                    default:
+                        break;
+                }
+            }
+
+            var messages = messageList.Select(message => MessageView.FromProtobuf(message, mq)).ToList();
+            StatusChecker.Check(status, request);
+            return new ReceiveMessageResult(mq.Broker.Endpoints, messages);
+        }
+
+        private static Proto.FilterExpression WrapFilterExpression(FilterExpression filterExpression)
+        {
+            var filterType = Proto.FilterType.Tag;
+            if (ExpressionType.Sql92.Equals(filterExpression.Type))
+            {
+                filterType = Proto.FilterType.Sql;
+            }
+
+            return new Proto.FilterExpression
+            {
+                Type = filterType,
+                Expression = filterExpression.Expression
+            };
+        }
+
+        protected static Proto.ReceiveMessageRequest WrapReceiveMessageRequest(int batchSize, MessageQueue mq,
+            FilterExpression filterExpression, TimeSpan invisibleDuration)
+        {
+            return new Proto.ReceiveMessageRequest()
+            {
+                MessageQueue = mq.ToProtobuf(),
+                FilterExpression = WrapFilterExpression(filterExpression),
+                BatchSize = batchSize,
+                AutoRenew = false,
+                InvisibleDuration = Duration.FromTimeSpan(invisibleDuration)
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Error/StatusChecker.cs b/csharp/rocketmq-client-csharp/Error/StatusChecker.cs
deleted file mode 100644
index 55f56d57..00000000
--- a/csharp/rocketmq-client-csharp/Error/StatusChecker.cs
+++ /dev/null
@@ -1,41 +0,0 @@
-using Apache.Rocketmq.V2;
-using Google.Protobuf;
-
-namespace Org.Apache.Rocketmq.Error
-{
-    public class StatusChecker
-    {
-        public static void check(Status status, IMessage message)
-        {
-            // var code = status.Code;
-            // switch (code)
-            // {
-            //     case Code.Ok:
-            //     case Code.MultipleResults:
-            //         return;
-            //     case Code.BadRequest:
-            //     case Code.IllegalAccessPoint:
-            //     case Code.IllegalTopic:
-            //     case Code.IllegalConsumerGroup:
-            //     case Code.IllegalMessageTag:
-            //     case Code.IllegalMessageKey:
-            //     case Code.IllegalMessageGroup:
-            //     case Code.IllegalMessagePropertyKey:
-            //     case Code.InvalidTransactionId:
-            //     case Code.IllegalMessageId:
-            //     case Code.IllegalFilterExpression:
-            //     case Code.IllegalInvisibleTime:
-            //     case Code.IllegalDeliveryTime:
-            //     case Code.InvalidReceiptHandle:
-            //     case Code.MessagePropertyConflictWithType:
-            //     case Code.UnrecognizedClientType:
-            //     case Code.MessageCorrupted:
-            //     case Code.ClientIdRequired:
-            //     case Code.IllegalPollingTime:
-            //         throw new BadRequestException(code)
-            //
-            //     case ILLEGAL_POLLING_TIME:
-            // }
-        }
-    }
-}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/FilterExpression.cs b/csharp/rocketmq-client-csharp/FilterExpression.cs
index 3bd432da..07a4af5a 100644
--- a/csharp/rocketmq-client-csharp/FilterExpression.cs
+++ b/csharp/rocketmq-client-csharp/FilterExpression.cs
@@ -25,6 +25,12 @@ namespace Org.Apache.Rocketmq
             Type = type;
         }
 
+        public FilterExpression(string expression)
+        {
+            Expression = expression;
+            Type = ExpressionType.Tag;
+        }
+
         public ExpressionType Type { get; }
         public string Expression { get; }
     }
diff --git a/csharp/rocketmq-client-csharp/IClientManager.cs b/csharp/rocketmq-client-csharp/IClientManager.cs
index f2e48e36..df2035ab 100644
--- a/csharp/rocketmq-client-csharp/IClientManager.cs
+++ b/csharp/rocketmq-client-csharp/IClientManager.cs
@@ -25,15 +25,48 @@ namespace Org.Apache.Rocketmq
 {
     public interface IClientManager
     {
+        /// <summary>
+        /// Establish a telemetry channel between client and remote endpoints.
+        /// </summary>
+        /// <param name="endpoints">The target endpoints.</param>
+        /// <returns>gRPC bi-directional stream.</returns>
         AsyncDuplexStreamingCall<TelemetryCommand, TelemetryCommand> Telemetry(Endpoints endpoints);
 
+        /// <summary>
+        /// Query topic route info from remote.
+        /// </summary>
+        /// <param name="endpoints">The target endpoints.</param>
+        /// <param name="request">gRPC request of querying topic route.</param>
+        /// <param name="timeout">Request max duration.</param>
+        /// <returns>Task of response.</returns>
         Task<QueryRouteResponse> QueryRoute(Endpoints endpoints, QueryRouteRequest request, TimeSpan timeout);
 
+        /// <summary>
+        /// Send heartbeat to remote endpoints.
+        /// </summary>
+        /// <param name="endpoints">The target endpoints.</param>
+        /// <param name="request">gRPC request of heartbeat.</param>
+        /// <param name="timeout">Request max duration.</param>
+        /// <returns>Task of response.</returns>
         Task<HeartbeatResponse> Heartbeat(Endpoints endpoints, HeartbeatRequest request, TimeSpan timeout);
 
+        /// <summary>
+        /// Notify client's termination.
+        /// </summary>
+        /// <param name="endpoints">The target endpoints.</param>
+        /// <param name="request">gRPC request of notifying client's termination.</param>
+        /// <param name="timeout">Request max duration.</param>
+        /// <returns>Task of response.</returns>
         Task<NotifyClientTerminationResponse> NotifyClientTermination(Endpoints endpoints,
             NotifyClientTerminationRequest request, TimeSpan timeout);
 
+        /// <summary>
+        /// Send message to remote endpoints.
+        /// </summary>
+        /// <param name="endpoints"></param>
+        /// <param name="request"></param>
+        /// <param name="timeout"></param>
+        /// <returns></returns>
         Task<SendMessageResponse> SendMessage(Endpoints endpoints, SendMessageRequest request,
             TimeSpan timeout);
 
diff --git a/csharp/rocketmq-client-csharp/MessageQueue.cs b/csharp/rocketmq-client-csharp/MessageQueue.cs
index cd6f0ce3..580fbafe 100644
--- a/csharp/rocketmq-client-csharp/MessageQueue.cs
+++ b/csharp/rocketmq-client-csharp/MessageQueue.cs
@@ -16,23 +16,20 @@
  */
 
 using System.Collections.Generic;
-using rmq = Apache.Rocketmq.V2;
+using System.Linq;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class MessageQueue
     {
-        public MessageQueue(rmq::MessageQueue messageQueue)
+        public MessageQueue(Proto::MessageQueue messageQueue)
         {
             TopicResource = new Resource(messageQueue.Topic);
             QueueId = messageQueue.Id;
             Permission = PermissionHelper.FromProtobuf(messageQueue.Permission);
-            var messageTypes = new List<MessageType>();
-            foreach (var acceptMessageType in messageQueue.AcceptMessageTypes)
-            {
-                var messageType = MessageTypeHelper.FromProtobuf(acceptMessageType);
-                messageTypes.Add(messageType);
-            }
+            var messageTypes = messageQueue.AcceptMessageTypes
+                .Select(MessageTypeHelper.FromProtobuf).ToList();
 
             AcceptMessageTypes = messageTypes;
             Broker = new Broker(messageQueue.Broker);
@@ -57,5 +54,19 @@ namespace Org.Apache.Rocketmq
         {
             return $"{Broker.Name}.{TopicResource}.{QueueId}";
         }
+
+        public Proto.MessageQueue ToProtobuf()
+        {
+            var messageTypes = AcceptMessageTypes.Select(MessageTypeHelper.ToProtobuf).ToList();
+
+            return new Proto.MessageQueue
+            {
+                Topic = TopicResource.ToProtobuf(),
+                Id = QueueId,
+                Permission = PermissionHelper.ToProtobuf(Permission),
+                Broker = Broker.ToProtobuf(),
+                AcceptMessageTypes = { messageTypes }
+            };
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/MessageView.cs b/csharp/rocketmq-client-csharp/MessageView.cs
index b790fcb9..dfb45e45 100644
--- a/csharp/rocketmq-client-csharp/MessageView.cs
+++ b/csharp/rocketmq-client-csharp/MessageView.cs
@@ -15,9 +15,10 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using System;
 using System.Collections.Generic;
+using System.Linq;
 using System.Security.Cryptography;
 using NLog;
 
@@ -30,14 +31,14 @@ namespace Org.Apache.Rocketmq
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
-        private readonly rmq.MessageQueue _messageQueue;
-        private readonly string _receiptHandle;
+        internal readonly MessageQueue MessageQueue;
+        internal readonly string ReceiptHandle;
         private readonly long _offset;
         private readonly bool _corrupted;
 
         internal MessageView(string messageId, string topic, byte[] body, string tag, string messageGroup,
             DateTime deliveryTime, List<string> keys, Dictionary<string, string> properties, string bornHost,
-            DateTime bornTime, int deliveryAttempt, rmq.MessageQueue messageQueue, string receiptHandle, long offset,
+            DateTime bornTime, int deliveryAttempt, MessageQueue messageQueue, string receiptHandle, long offset,
             bool corrupted)
         {
             MessageId = messageId;
@@ -51,8 +52,8 @@ namespace Org.Apache.Rocketmq
             BornHost = bornHost;
             BornTime = bornTime;
             DeliveryAttempt = deliveryAttempt;
-            _messageQueue = messageQueue;
-            _receiptHandle = receiptHandle;
+            MessageQueue = messageQueue;
+            ReceiptHandle = receiptHandle;
             _offset = offset;
             _corrupted = corrupted;
         }
@@ -79,7 +80,7 @@ namespace Org.Apache.Rocketmq
 
         public int DeliveryAttempt { get; }
 
-        public static MessageView FromProtobuf(rmq.Message message, rmq.MessageQueue messageQueue)
+        public static MessageView FromProtobuf(Proto.Message message, MessageQueue messageQueue)
         {
             var topic = message.Topic.Name;
             var systemProperties = message.SystemProperties;
@@ -87,11 +88,11 @@ namespace Org.Apache.Rocketmq
             var bodyDigest = systemProperties.BodyDigest;
             var checkSum = bodyDigest.Checksum;
             var raw = message.Body.ToByteArray();
-            bool corrupted = false;
+            var corrupted = false;
             var type = bodyDigest.Type;
             switch (type)
             {
-                case rmq.DigestType.Crc32:
+                case Proto.DigestType.Crc32:
                 {
                     var expectedCheckSum = Force.Crc32.Crc32Algorithm.Compute(raw, 0, raw.Length).ToString("X");
                     if (!expectedCheckSum.Equals(checkSum))
@@ -101,7 +102,7 @@ namespace Org.Apache.Rocketmq
 
                     break;
                 }
-                case rmq.DigestType.Md5:
+                case Proto.DigestType.Md5:
                 {
                     var expectedCheckSum = Convert.ToHexString(MD5.HashData(raw));
                     if (!expectedCheckSum.Equals(checkSum))
@@ -111,7 +112,7 @@ namespace Org.Apache.Rocketmq
 
                     break;
                 }
-                case rmq.DigestType.Sha1:
+                case Proto.DigestType.Sha1:
                 {
                     var expectedCheckSum = Convert.ToHexString(SHA1.HashData(raw));
                     if (!expectedCheckSum.Equals(checkSum))
@@ -121,6 +122,7 @@ namespace Org.Apache.Rocketmq
 
                     break;
                 }
+                case Proto.DigestType.Unspecified:
                 default:
                 {
                     Logger.Error(
@@ -131,18 +133,19 @@ namespace Org.Apache.Rocketmq
             }
 
             var bodyEncoding = systemProperties.BodyEncoding;
-            byte[] body = raw;
+            var body = raw;
             switch (bodyEncoding)
             {
-                case rmq.Encoding.Gzip:
+                case Proto.Encoding.Gzip:
                 {
                     body = Utilities.DecompressBytesGzip(message.Body.ToByteArray());
                     break;
                 }
-                case rmq.Encoding.Identity:
+                case Proto.Encoding.Identity:
                 {
                     break;
                 }
+                case Proto.Encoding.Unspecified:
                 default:
                 {
                     Logger.Error($"Unsupported message encoding algorithm," +
@@ -151,25 +154,22 @@ namespace Org.Apache.Rocketmq
                 }
             }
 
-            string tag = systemProperties.HasTag ? systemProperties.Tag : null;
-            string messageGroup = systemProperties.HasMessageGroup ? systemProperties.MessageGroup : null;
+            var tag = systemProperties.HasTag ? systemProperties.Tag : null;
+            var messageGroup = systemProperties.HasMessageGroup ? systemProperties.MessageGroup : null;
             var deliveryTime = systemProperties.DeliveryTimestamp.ToDateTime();
-            List<string> keys = new List<string>();
-            foreach (var key in systemProperties.Keys)
-            {
-                keys.Add(key);
-            }
+            var keys = systemProperties.Keys.ToList();
 
             var bornHost = systemProperties.BornHost;
             var bornTime = systemProperties.BornTimestamp.ToDateTime();
             var deliveryAttempt = systemProperties.DeliveryAttempt;
             var queueOffset = systemProperties.QueueOffset;
-            Dictionary<string, string> properties = new Dictionary<string, string>();
+            var properties = new Dictionary<string, string>();
             foreach (var (key, value) in message.UserProperties)
             {
                 properties.Add(key, value);
             }
 
+
             var receiptHandle = systemProperties.ReceiptHandle;
             return new MessageView(messageId, topic, body, tag, messageGroup, deliveryTime, keys, properties, bornHost,
                 bornTime, deliveryAttempt, messageQueue, receiptHandle, queueOffset, corrupted);
diff --git a/csharp/rocketmq-client-csharp/MqEncoding.cs b/csharp/rocketmq-client-csharp/MqEncoding.cs
index 27dfb052..ce6f4779 100644
--- a/csharp/rocketmq-client-csharp/MqEncoding.cs
+++ b/csharp/rocketmq-client-csharp/MqEncoding.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -27,13 +27,13 @@ namespace Org.Apache.Rocketmq
 
     public static class EncodingHelper
     {
-        public static rmq.Encoding ToProtobuf(MqEncoding mqEncoding)
+        public static Proto.Encoding ToProtobuf(MqEncoding mqEncoding)
         {
             return mqEncoding switch
             {
-                MqEncoding.Gzip => rmq.Encoding.Gzip,
-                MqEncoding.Identity => rmq.Encoding.Identity,
-                _ => rmq.Encoding.Unspecified
+                MqEncoding.Gzip => Proto.Encoding.Gzip,
+                MqEncoding.Identity => Proto.Encoding.Identity,
+                _ => Proto.Encoding.Unspecified
             };
         }
     }
diff --git a/csharp/rocketmq-client-csharp/Permission.cs b/csharp/rocketmq-client-csharp/Permission.cs
index eedd6247..20d2828d 100644
--- a/csharp/rocketmq-client-csharp/Permission.cs
+++ b/csharp/rocketmq-client-csharp/Permission.cs
@@ -46,8 +46,24 @@ namespace Org.Apache.Rocketmq
                     throw new InternalErrorException("Permission is not specified");
             }
         }
-        
-        public static bool IsWritable(Permission permission) {
+
+        public static Proto.Permission ToProtobuf(Permission permission)
+        {
+            switch (permission)
+            {
+                case Permission.Read:
+                    return Proto.Permission.Read;
+                case Permission.Write:
+                    return Proto.Permission.Write;
+                case Permission.ReadWrite:
+                    return Proto.Permission.ReadWrite;
+                default:
+                    throw new InternalErrorException("Permission is not specified");
+            }
+        }
+
+        public static bool IsWritable(Permission permission)
+        {
             switch (permission)
             {
                 case Permission.Write:
@@ -74,6 +90,4 @@ namespace Org.Apache.Rocketmq
             }
         }
     }
-    
-
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index fc41926b..62387a98 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -25,7 +25,7 @@ using NLog;
 
 namespace Org.Apache.Rocketmq
 {
-    public class Producer : Client, IProducer
+    public class Producer : Client
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
         private readonly ConcurrentDictionary<string /* topic */, PublishingLoadBalancer> _publishingRouteDataCache;
@@ -42,7 +42,7 @@ namespace Org.Apache.Rocketmq
         }
 
         private Producer(ClientConfig clientConfig, ConcurrentDictionary<string, bool> topics, int maxAttempts) :
-            base(clientConfig, topics)
+            base(clientConfig, topics.Keys)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
             _publishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
@@ -54,7 +54,7 @@ namespace Org.Apache.Rocketmq
         {
             foreach (var topic in topics)
             {
-                Topics[topic] = false;
+                Topics.Add(topic);
             }
         }
 
@@ -94,7 +94,7 @@ namespace Org.Apache.Rocketmq
             return publishingLoadBalancer;
         }
 
-        protected override void OnTopicRouteDataFetched0(string topic, TopicRouteData topicRouteData)
+        protected override void OnTopicRouteDataUpdated0(string topic, TopicRouteData topicRouteData)
         {
             var publishingLoadBalancer = new PublishingLoadBalancer(topicRouteData);
             _publishingRouteDataCache.TryAdd(topic, publishingLoadBalancer);
diff --git a/csharp/rocketmq-client-csharp/Publishing.cs b/csharp/rocketmq-client-csharp/Publishing.cs
index ffedd177..055e36ce 100644
--- a/csharp/rocketmq-client-csharp/Publishing.cs
+++ b/csharp/rocketmq-client-csharp/Publishing.cs
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using System.Collections.Generic;
 
 namespace Org.Apache.Rocketmq
@@ -23,7 +23,7 @@ namespace Org.Apache.Rocketmq
     // Settings for publishing
     public class Publishing
     {
-        public List<rmq::Resource> Topics { get; set; }
+        public List<Proto::Resource> Topics { get; set; }
         public int CompressBodyThreshold { get; set; }
 
         public int MaxBodySize { get; set; }
diff --git a/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
index c33bc7dc..2133af5a 100644
--- a/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
+++ b/csharp/rocketmq-client-csharp/PublishingLoadBalancer.cs
@@ -18,23 +18,24 @@
 using System;
 using System.Collections.Generic;
 using System.Linq;
-using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class PublishingLoadBalancer
     {
         private readonly List<MessageQueue> _messageQueues;
+
+        // TODO
         private int _roundRobinIndex;
 
         public PublishingLoadBalancer(TopicRouteData route)
         {
             _messageQueues = new List<MessageQueue>();
-            foreach (var messageQueue in route.MessageQueues.Where(messageQueue =>
+            foreach (var mq in route.MessageQueues.Where(messageQueue =>
                          PermissionHelper.IsWritable(messageQueue.Permission) &&
                          Utilities.MasterBrokerId == messageQueue.Broker.Id))
             {
-                _messageQueues.Add(messageQueue);
+                _messageQueues.Add(mq);
             }
 
             var random = new Random();
@@ -70,21 +71,23 @@ namespace Org.Apache.Rocketmq
                 }
             }
 
-            if (candidates.Count != 0) return candidates;
+            if (candidates.Count != 0)
+            {
+                return candidates;
+            }
+
+            foreach (var mq in _messageQueues.Select(_ => Utilities.GetPositiveMod(next++, _messageQueues.Count))
+                         .Select(positiveMod => _messageQueues[positiveMod]))
             {
-                foreach (var mq in _messageQueues.Select(_ => Utilities.GetPositiveMod(next++, _messageQueues.Count))
-                             .Select(positiveMod => _messageQueues[positiveMod]))
+                if (!candidateBrokerNames.Contains(mq.Broker.Name))
                 {
-                    if (!candidateBrokerNames.Contains(mq.Broker.Name))
-                    {
-                        candidateBrokerNames.Add(mq.Broker.Name);
-                        candidates.Add(mq);
-                    }
+                    candidateBrokerNames.Add(mq.Broker.Name);
+                    candidates.Add(mq);
+                }
 
-                    if (candidates.Count >= count)
-                    {
-                        break;
-                    }
+                if (candidates.Count >= count)
+                {
+                    break;
                 }
             }
 
diff --git a/csharp/rocketmq-client-csharp/PublishingSettings.cs b/csharp/rocketmq-client-csharp/PublishingSettings.cs
index 023b0be3..b9f8f454 100644
--- a/csharp/rocketmq-client-csharp/PublishingSettings.cs
+++ b/csharp/rocketmq-client-csharp/PublishingSettings.cs
@@ -17,7 +17,7 @@
 
 using System;
 using System.Collections.Concurrent;
-using System.Collections.Generic;
+using System.Linq;
 using Google.Protobuf.WellKnownTypes;
 using Proto = Apache.Rocketmq.V2;
 
@@ -28,8 +28,8 @@ namespace Org.Apache.Rocketmq
         private volatile int _maxBodySizeBytes = 4 * 1024 * 1024;
         private volatile bool _validateMessageType = true;
 
-        public PublishingSettings(string clientId, Endpoints accessPoint, ExponentialBackoffRetryPolicy retryPolicy,
-            TimeSpan requestTimeout, ConcurrentDictionary<string, bool> topics) : base(clientId, ClientType.Producer, accessPoint,
+        public PublishingSettings(string clientId, Endpoints endpoints, ExponentialBackoffRetryPolicy retryPolicy,
+            TimeSpan requestTimeout, ConcurrentDictionary<string, bool> topics) : base(clientId, ClientType.Producer, endpoints,
             retryPolicy, requestTimeout)
         {
             Topics = topics;
@@ -54,14 +54,7 @@ namespace Org.Apache.Rocketmq
 
         public override Proto.Settings ToProtobuf()
         {
-            List<Proto.Resource> topics = new List<Proto.Resource>();
-            foreach (var topic in Topics)
-            {
-                topics.Add(new Proto.Resource
-                {
-                    Name = topic.Key
-                });
-            }
+            var topics = Topics.Select(topic => new Proto.Resource { Name = topic.Key }).ToList();
 
             var publishing = new Proto.Publishing();
             publishing.Topics.Add(topics);
@@ -69,7 +62,7 @@ namespace Org.Apache.Rocketmq
             return new Proto.Settings
             {
                 Publishing = publishing,
-                AccessPoint = AccessPoint.ToProtobuf(),
+                AccessPoint = Endpoints.ToProtobuf(),
                 ClientType = ClientTypeHelper.ToProtobuf(ClientType),
                 RequestTimeout = Duration.FromTimeSpan(RequestTimeout),
                 BackoffPolicy = RetryPolicy.ToProtobuf(),
diff --git a/csharp/rocketmq-client-csharp/IProducer.cs b/csharp/rocketmq-client-csharp/ReceiveMessageResult.cs
similarity index 71%
rename from csharp/rocketmq-client-csharp/IProducer.cs
rename to csharp/rocketmq-client-csharp/ReceiveMessageResult.cs
index 420af202..1898838b 100644
--- a/csharp/rocketmq-client-csharp/IProducer.cs
+++ b/csharp/rocketmq-client-csharp/ReceiveMessageResult.cs
@@ -15,16 +15,20 @@
  * limitations under the License.
  */
 
-using System.Threading.Tasks;
+using System.Collections.Generic;
 
 namespace Org.Apache.Rocketmq
 {
-    public interface IProducer
+    public class ReceiveMessageResult
     {
-        Task Start();
+        public ReceiveMessageResult(Endpoints endpoints, List<MessageView> messages)
+        {
+            Endpoints = endpoints;
+            Messages = messages;
+        }
 
-        Task Shutdown();
+        public Endpoints Endpoints { get; }
 
-        Task<SendReceipt> Send(Message message);
+        public List<MessageView> Messages { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Resource.cs b/csharp/rocketmq-client-csharp/Resource.cs
index a1825f15..76c0ba9e 100644
--- a/csharp/rocketmq-client-csharp/Resource.cs
+++ b/csharp/rocketmq-client-csharp/Resource.cs
@@ -11,6 +11,12 @@ namespace Org.Apache.Rocketmq
             Name = resource.Name;
         }
 
+        public Resource(string name)
+        {
+            Namespace = "";
+            Name = name;
+        }
+
         public string Namespace { get; }
         public string Name { get; }
 
@@ -22,7 +28,7 @@ namespace Org.Apache.Rocketmq
                 Name = Name
             };
         }
-        
+
         public override string ToString()
         {
             return String.IsNullOrEmpty(Namespace) ? Name : $"{Namespace}.{Name}";
diff --git a/csharp/rocketmq-client-csharp/RpcClient.cs b/csharp/rocketmq-client-csharp/RpcClient.cs
index de28c184..bf45410c 100644
--- a/csharp/rocketmq-client-csharp/RpcClient.cs
+++ b/csharp/rocketmq-client-csharp/RpcClient.cs
@@ -21,7 +21,7 @@ using System.Net.Http;
 using System.Net.Security;
 using System.Threading;
 using System.Threading.Tasks;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 using Grpc.Core;
 using Grpc.Core.Interceptors;
 using Grpc.Net.Client;
@@ -32,7 +32,7 @@ namespace Org.Apache.Rocketmq
     public class RpcClient : IRpcClient
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
-        private readonly rmq::MessagingService.MessagingServiceClient _stub;
+        private readonly Proto::MessagingService.MessagingServiceClient _stub;
         private readonly GrpcChannel _channel;
         private readonly string _target;
 
@@ -44,7 +44,7 @@ namespace Org.Apache.Rocketmq
                 HttpHandler = CreateHttpHandler()
             });
             var invoker = _channel.Intercept(new ClientLoggerInterceptor());
-            _stub = new rmq::MessagingService.MessagingServiceClient(invoker);
+            _stub = new Proto::MessagingService.MessagingServiceClient(invoker);
         }
 
         public async Task Shutdown()
@@ -74,14 +74,14 @@ namespace Org.Apache.Rocketmq
             return handler;
         }
 
-        public AsyncDuplexStreamingCall<rmq::TelemetryCommand, rmq::TelemetryCommand> Telemetry(Metadata metadata)
+        public AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand> Telemetry(Metadata metadata)
         {
             var deadline = DateTime.UtcNow.Add(TimeSpan.FromDays(3650));
             var callOptions = new CallOptions(metadata, deadline);
             return _stub.Telemetry(callOptions);
         }
 
-        public async Task<rmq::QueryRouteResponse> QueryRoute(Metadata metadata, rmq::QueryRouteRequest request,
+        public async Task<Proto::QueryRouteResponse> QueryRoute(Metadata metadata, Proto::QueryRouteRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -92,7 +92,7 @@ namespace Org.Apache.Rocketmq
         }
 
 
-        public async Task<rmq::HeartbeatResponse> Heartbeat(Metadata metadata, rmq::HeartbeatRequest request,
+        public async Task<Proto::HeartbeatResponse> Heartbeat(Metadata metadata, Proto::HeartbeatRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -102,7 +102,7 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::SendMessageResponse> SendMessage(Metadata metadata, rmq::SendMessageRequest request,
+        public async Task<Proto::SendMessageResponse> SendMessage(Metadata metadata, Proto::SendMessageRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -112,8 +112,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::QueryAssignmentResponse> QueryAssignment(Metadata metadata,
-            rmq::QueryAssignmentRequest request,
+        public async Task<Proto::QueryAssignmentResponse> QueryAssignment(Metadata metadata,
+            Proto::QueryAssignmentRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -123,14 +123,14 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<List<rmq::ReceiveMessageResponse>> ReceiveMessage(Metadata metadata,
-            rmq::ReceiveMessageRequest request, TimeSpan timeout)
+        public async Task<List<Proto::ReceiveMessageResponse>> ReceiveMessage(Metadata metadata,
+            Proto::ReceiveMessageRequest request, TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
             var call = _stub.ReceiveMessage(request, callOptions);
             Logger.Debug($"ReceiveMessageRequest has been written to {_target}");
-            var result = new List<rmq::ReceiveMessageResponse>();
+            var result = new List<Proto::ReceiveMessageResponse>();
             var stream = call.ResponseStream;
             while (await stream.MoveNext())
             {
@@ -143,7 +143,7 @@ namespace Org.Apache.Rocketmq
             return result;
         }
 
-        public async Task<rmq::AckMessageResponse> AckMessage(Metadata metadata, rmq::AckMessageRequest request,
+        public async Task<Proto::AckMessageResponse> AckMessage(Metadata metadata, Proto::AckMessageRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -153,8 +153,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Metadata metadata,
-            rmq::ChangeInvisibleDurationRequest request,
+        public async Task<Proto::ChangeInvisibleDurationResponse> ChangeInvisibleDuration(Metadata metadata,
+            Proto::ChangeInvisibleDurationRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -164,9 +164,9 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::ForwardMessageToDeadLetterQueueResponse> ForwardMessageToDeadLetterQueue(
+        public async Task<Proto::ForwardMessageToDeadLetterQueueResponse> ForwardMessageToDeadLetterQueue(
             Metadata metadata,
-            rmq::ForwardMessageToDeadLetterQueueRequest request, TimeSpan timeout)
+            Proto::ForwardMessageToDeadLetterQueueRequest request, TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
@@ -175,8 +175,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::EndTransactionResponse> EndTransaction(Metadata metadata,
-            rmq::EndTransactionRequest request,
+        public async Task<Proto::EndTransactionResponse> EndTransaction(Metadata metadata,
+            Proto::EndTransactionRequest request,
             TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
@@ -186,8 +186,8 @@ namespace Org.Apache.Rocketmq
             return await call.ResponseAsync;
         }
 
-        public async Task<rmq::NotifyClientTerminationResponse> NotifyClientTermination(Metadata metadata,
-            rmq::NotifyClientTerminationRequest request, TimeSpan timeout)
+        public async Task<Proto::NotifyClientTerminationResponse> NotifyClientTermination(Metadata metadata,
+            Proto::NotifyClientTerminationRequest request, TimeSpan timeout)
         {
             var deadline = DateTime.UtcNow.Add(timeout);
             var callOptions = new CallOptions(metadata, deadline);
diff --git a/csharp/rocketmq-client-csharp/Settings.cs b/csharp/rocketmq-client-csharp/Settings.cs
index 7716fc2d..491aa564 100644
--- a/csharp/rocketmq-client-csharp/Settings.cs
+++ b/csharp/rocketmq-client-csharp/Settings.cs
@@ -24,25 +24,25 @@ namespace Org.Apache.Rocketmq
     {
         protected readonly string ClientId;
         protected readonly ClientType ClientType;
-        protected readonly Endpoints AccessPoint;
+        protected readonly Endpoints Endpoints;
         protected volatile IRetryPolicy RetryPolicy;
         protected readonly TimeSpan RequestTimeout;
 
-        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, IRetryPolicy retryPolicy,
+        public Settings(string clientId, ClientType clientType, Endpoints endpoints, IRetryPolicy retryPolicy,
             TimeSpan requestTimeout)
         {
             ClientId = clientId;
             ClientType = clientType;
-            AccessPoint = accessPoint;
+            Endpoints = endpoints;
             RetryPolicy = retryPolicy;
             RequestTimeout = requestTimeout;
         }
 
-        public Settings(string clientId, ClientType clientType, Endpoints accessPoint, TimeSpan requestTimeout)
+        public Settings(string clientId, ClientType clientType, Endpoints endpoints, TimeSpan requestTimeout)
         {
             ClientId = clientId;
             ClientType = clientType;
-            AccessPoint = accessPoint;
+            Endpoints = endpoints;
             RetryPolicy = null;
             RequestTimeout = requestTimeout;
         }
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
new file mode 100644
index 00000000..1a0f0ec2
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -0,0 +1,192 @@
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using System.Threading;
+using System.Threading.Tasks;
+using Google.Protobuf.WellKnownTypes;
+using Proto = Apache.Rocketmq.V2;
+using NLog;
+using Org.Apache.Rocketmq.Error;
+
+namespace Org.Apache.Rocketmq
+{
+    public class SimpleConsumer : Consumer
+    {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+        private readonly ConcurrentDictionary<string /* topic */, SubscriptionLoadBalancer> _subscriptionRouteDataCache;
+        private readonly ConcurrentDictionary<string /* topic */, FilterExpression> _subscriptionExpressions;
+        private readonly TimeSpan _awaitDuration;
+        private readonly SimpleSubscriptionSettings _simpleSubscriptionSettings;
+        private int _topicRoundRobinIndex;
+
+        public SimpleConsumer(ClientConfig clientConfig, string consumerGroup, TimeSpan awaitDuration,
+            Dictionary<string, FilterExpression> subscriptionExpressions) : this(clientConfig, consumerGroup,
+            awaitDuration, new ConcurrentDictionary<string, FilterExpression>(subscriptionExpressions))
+        {
+        }
+
+        private SimpleConsumer(ClientConfig clientConfig, string consumerGroup, TimeSpan awaitDuration,
+            ConcurrentDictionary<string, FilterExpression> subscriptionExpressions) : base(clientConfig, consumerGroup,
+            subscriptionExpressions.Keys)
+        {
+            _awaitDuration = awaitDuration;
+            _subscriptionRouteDataCache = new ConcurrentDictionary<string, SubscriptionLoadBalancer>();
+            _subscriptionExpressions = subscriptionExpressions;
+            _simpleSubscriptionSettings = new SimpleSubscriptionSettings(ClientId, clientConfig.Endpoints,
+                ConsumerGroup, clientConfig.RequestTimeout, awaitDuration, subscriptionExpressions);
+            _topicRoundRobinIndex = 0;
+        }
+
+        public async Task Subscribe(string topic, FilterExpression filterExpression)
+        {
+            // TODO: check running status.
+            await GetSubscriptionLoadBalancer(topic);
+            _subscriptionExpressions.TryAdd(topic, filterExpression);
+        }
+
+        public void Unsubscribe(string topic)
+        {
+            _subscriptionExpressions.TryRemove(topic, out _);
+        }
+
+        public override async Task Start()
+        {
+            Logger.Info($"Begin to start the rocketmq simple consumer, clientId={ClientId}");
+            await base.Start();
+            Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+        }
+
+        public override async Task Shutdown()
+        {
+            Logger.Info($"Begin to shutdown the rocketmq simple consumer, clientId={ClientId}");
+            await base.Shutdown();
+            Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+        }
+
+        protected override Proto.HeartbeatRequest WrapHeartbeatRequest()
+        {
+            return new Proto::HeartbeatRequest
+            {
+                ClientType = Proto.ClientType.SimpleConsumer
+            };
+        }
+
+        protected override void OnTopicRouteDataUpdated0(string topic, TopicRouteData topicRouteData)
+        {
+            var subscriptionLoadBalancer = new SubscriptionLoadBalancer(topicRouteData);
+            _subscriptionRouteDataCache.TryAdd(topic, subscriptionLoadBalancer);
+        }
+
+        public override Proto.Settings GetSettings()
+        {
+            return _simpleSubscriptionSettings.ToProtobuf();
+        }
+
+        public override void OnSettingsCommand(Endpoints endpoints, Proto.Settings settings)
+        {
+            _simpleSubscriptionSettings.Sync(settings);
+        }
+
+
+        private async Task<SubscriptionLoadBalancer> GetSubscriptionLoadBalancer(string topic)
+        {
+            if (_subscriptionRouteDataCache.TryGetValue(topic, out var subscriptionLoadBalancer))
+            {
+                return subscriptionLoadBalancer;
+            }
+
+            var topicRouteData = await FetchTopicRoute(topic);
+            subscriptionLoadBalancer = new SubscriptionLoadBalancer(topicRouteData);
+            _subscriptionRouteDataCache.TryAdd(topic, subscriptionLoadBalancer);
+
+            return subscriptionLoadBalancer;
+        }
+
+
+        public async Task<List<MessageView>> Receive(int maxMessageNum, TimeSpan invisibleDuration)
+        {
+            if (maxMessageNum <= 0)
+            {
+                throw new InternalErrorException("maxMessageNum must be greater than 0");
+            }
+
+            var copy = new ConcurrentDictionary<string, FilterExpression>(_subscriptionExpressions);
+            var topics = new List<string>(copy.Keys);
+            if (topics.Count <= 0)
+            {
+                throw new ArgumentException("There is no topic to receive message");
+            }
+
+            var index = Utilities.GetPositiveMod(Interlocked.Increment(ref _topicRoundRobinIndex), topics.Count);
+            var topic = topics[index];
+            var filterExpression = _subscriptionExpressions[topic];
+            var subscriptionLoadBalancer = await GetSubscriptionLoadBalancer(topic);
+
+            var mq = subscriptionLoadBalancer.TakeMessageQueue();
+            var request = WrapReceiveMessageRequest(maxMessageNum, mq, filterExpression, invisibleDuration);
+            var receiveMessageResult = await ReceiveMessage(request, mq, _awaitDuration);
+            return receiveMessageResult.Messages;
+        }
+
+        public async void ChangeInvisibleDuration(MessageView messageView, TimeSpan invisibleDuration)
+        {
+            var request = WrapChangeInvisibleDuration(messageView, invisibleDuration);
+            var response = await ClientManager.ChangeInvisibleDuration(messageView.MessageQueue.Broker.Endpoints,
+                request, ClientConfig.RequestTimeout);
+            StatusChecker.Check(response.Status, request);
+        }
+
+
+        public async Task Ack(MessageView messageView)
+        {
+            var request = WrapAckMessageRequest(messageView);
+            var response = await ClientManager.AckMessage(messageView.MessageQueue.Broker.Endpoints, request,
+                ClientConfig.RequestTimeout);
+            StatusChecker.Check(response.Status, request);
+        }
+
+        private Proto.AckMessageRequest WrapAckMessageRequest(MessageView messageView)
+        {
+            var topicResource = new Proto.Resource
+            {
+                Name = messageView.Topic
+            };
+            var entry = new Proto.AckMessageEntry
+            {
+                MessageId = messageView.MessageId,
+                ReceiptHandle = messageView.ReceiptHandle,
+            };
+            return new Proto.AckMessageRequest
+            {
+                Group = GetProtobufGroup(),
+                Topic = topicResource,
+                Entries = { entry }
+            };
+        }
+
+        private Proto.ChangeInvisibleDurationRequest WrapChangeInvisibleDuration(MessageView messageView,
+            TimeSpan invisibleDuration)
+        {
+            var topicResource = new Proto.Resource
+            {
+                Name = messageView.Topic
+            };
+            return new Proto.ChangeInvisibleDurationRequest
+            {
+                Topic = topicResource,
+                Group = GetProtobufGroup(),
+                ReceiptHandle = messageView.ReceiptHandle,
+                InvisibleDuration = Duration.FromTimeSpan(invisibleDuration),
+                MessageId = messageView.MessageId
+            };
+        }
+
+        private Proto.Resource GetProtobufGroup()
+        {
+            return new Proto.Resource()
+            {
+                Name = ConsumerGroup
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs b/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
new file mode 100644
index 00000000..a6a409d7
--- /dev/null
+++ b/csharp/rocketmq-client-csharp/SimpleSubscriptionSettings.cs
@@ -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.
+ */
+
+using System;
+using System.Collections.Concurrent;
+using System.Collections.Generic;
+using Google.Protobuf.WellKnownTypes;
+using NLog;
+using Proto = Apache.Rocketmq.V2;
+
+namespace Org.Apache.Rocketmq
+{
+    public class SimpleSubscriptionSettings : Settings
+    {
+        private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
+
+        private readonly Resource _group;
+        private readonly TimeSpan _longPollingTimeout;
+        private readonly ConcurrentDictionary<string /* topic */, FilterExpression> _subscriptionExpressions;
+
+        public SimpleSubscriptionSettings(string clientId, Endpoints endpoints, string consumerGroup,
+            TimeSpan requestTimeout, TimeSpan longPollingTimeout,
+            ConcurrentDictionary<string, FilterExpression> subscriptionExpressions) : base(
+            clientId, ClientType.SimpleConsumer, endpoints, requestTimeout)
+        {
+            _group = new Resource(consumerGroup);
+            _longPollingTimeout = longPollingTimeout;
+            _subscriptionExpressions = subscriptionExpressions;
+        }
+
+        public override void Sync(Proto::Settings settings)
+        {
+            // TODO
+        }
+
+        public override Proto.Settings ToProtobuf()
+        {
+            var subscriptionEntries = new List<Proto.SubscriptionEntry>();
+            foreach (var (key, value) in _subscriptionExpressions)
+            {
+                var topic = new Proto.Resource()
+                {
+                    Name = key,
+                };
+                var subscriptionEntry = new Proto.SubscriptionEntry();
+                var filterExpression = new Proto.FilterExpression();
+                switch (value.Type)
+                {
+                    case ExpressionType.Tag:
+                        filterExpression.Type = Proto.FilterType.Tag;
+                        break;
+                    case ExpressionType.Sql92:
+                        filterExpression.Type = Proto.FilterType.Sql;
+                        break;
+                    default:
+                        Logger.Warn($"[Bug] Unrecognized filter type={value.Type} for simple consumer");
+                        break;
+                }
+
+                filterExpression.Expression = value.Expression;
+                subscriptionEntry.Topic = topic;
+                subscriptionEntries.Add(subscriptionEntry);
+            }
+
+            var subscription = new Proto.Subscription
+            {
+                Group = _group.ToProtobuf(),
+                Subscriptions = { subscriptionEntries },
+                LongPollingTimeout = Duration.FromTimeSpan(_longPollingTimeout)
+            };
+            return new Proto.Settings
+            {
+                AccessPoint = Endpoints.ToProtobuf(),
+                ClientType = ClientTypeHelper.ToProtobuf(ClientType),
+                RequestTimeout = Duration.FromTimeSpan(RequestTimeout),
+                Subscription = subscription,
+                UserAgent = UserAgent.Instance.ToProtobuf()
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/StatusChecker.cs b/csharp/rocketmq-client-csharp/StatusChecker.cs
index 641fd097..2802fd78 100644
--- a/csharp/rocketmq-client-csharp/StatusChecker.cs
+++ b/csharp/rocketmq-client-csharp/StatusChecker.cs
@@ -31,6 +31,7 @@ namespace Org.Apache.Rocketmq
             var statusCode = status.Code;
 
             var statusMessage = status.Message;
+            // TODO
             switch (statusCode)
             {
                 case Proto.Code.Ok:
diff --git a/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs b/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
index b77da833..a8b1963e 100644
--- a/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
+++ b/csharp/rocketmq-client-csharp/SubscriptionLoadBalancer.cs
@@ -15,37 +15,35 @@
  * limitations under the License.
  */
 
+using System;
 using System.Collections.Generic;
-using System.Threading;
-using rmq = Apache.Rocketmq.V2;
+using System.Linq;
 
 namespace Org.Apache.Rocketmq
 {
     internal sealed class SubscriptionLoadBalancer
     {
-        public List<rmq.Assignment> Assignments { get; private set; }
-        private uint index = 0;
+        private readonly List<MessageQueue> _messageQueues;
+        private int _roundRobinIndex;
 
-        public SubscriptionLoadBalancer(List<rmq.Assignment> assignments)
+        public SubscriptionLoadBalancer(TopicRouteData topicRouteData)
         {
-            Assignments = assignments;
-        }
+            _messageQueues = new List<MessageQueue>();
+            foreach (var mq in topicRouteData.MessageQueues.Where(mq => PermissionHelper.IsReadable(mq.Permission))
+                         .Where(mq => Utilities.MasterBrokerId == mq.Broker.Id))
+            {
+                _messageQueues.Add(mq);
+            }
 
-        private SubscriptionLoadBalancer(uint oldIndex, List<rmq.Assignment> assignments)
-        {
-            index = oldIndex;
-            Assignments = assignments;
-        }
-
-        public SubscriptionLoadBalancer Update(List<rmq.Assignment> newAssignments)
-        {
-            return new SubscriptionLoadBalancer(index, newAssignments);
+            var random = new Random();
+            _roundRobinIndex = random.Next(0, _messageQueues.Count);
         }
 
-        public rmq.MessageQueue TakeMessageQueue()
+        public MessageQueue TakeMessageQueue()
         {
-            var i = Interlocked.Increment(ref index);
-            return Assignments[(int)(i % Assignments.Count)].MessageQueue;
+            var next = ++_roundRobinIndex;
+            var index = Utilities.GetPositiveMod(next, _messageQueues.Count);
+            return _messageQueues[index];
         }
     }
-}
+}
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/TopicRouteData.cs b/csharp/rocketmq-client-csharp/TopicRouteData.cs
index 2be2b9a5..885db5f6 100644
--- a/csharp/rocketmq-client-csharp/TopicRouteData.cs
+++ b/csharp/rocketmq-client-csharp/TopicRouteData.cs
@@ -18,19 +18,15 @@
 using System;
 using System.Collections.Generic;
 using System.Linq;
-using rmq = Apache.Rocketmq.V2;
+using Proto = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
     public class TopicRouteData : IEquatable<TopicRouteData>
     {
-        public TopicRouteData(List<rmq::MessageQueue> messageQueues)
+        public TopicRouteData(IEnumerable<Proto.MessageQueue> messageQueues)
         {
-            var messageQueuesList = new List<MessageQueue>();
-            foreach (var mq in messageQueues)
-            {
-                messageQueuesList.Add(new MessageQueue(mq));
-            }
+            var messageQueuesList = messageQueues.Select(mq => new MessageQueue(mq)).ToList();
 
             MessageQueues = messageQueuesList;
         }
diff --git a/csharp/rocketmq-client-csharp/Utilities.cs b/csharp/rocketmq-client-csharp/Utilities.cs
index 592f364e..d032ae1e 100644
--- a/csharp/rocketmq-client-csharp/Utilities.cs
+++ b/csharp/rocketmq-client-csharp/Utilities.cs
@@ -22,7 +22,6 @@ using System;
 using System.IO;
 using System.IO.Compression;
 using System.Threading;
-using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
 {
@@ -81,10 +80,10 @@ namespace Org.Apache.Rocketmq
 
         public static string ByteArrayToHexString(byte[] bytes)
         {
-            StringBuilder result = new StringBuilder(bytes.Length * 2);
+            var result = new StringBuilder(bytes.Length * 2);
             const string hexAlphabet = "0123456789ABCDEF";
 
-            foreach (byte b in bytes)
+            foreach (var b in bytes)
             {
                 result.Append(hexAlphabet[(int)(b >> 4)]);
                 result.Append(hexAlphabet[(int)(b & 0xF)]);
diff --git a/csharp/tests/UnitTest1.cs b/csharp/tests/UnitTest1.cs
index af13d3dc..2f6b468a 100644
--- a/csharp/tests/UnitTest1.cs
+++ b/csharp/tests/UnitTest1.cs
@@ -14,11 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-using Microsoft.VisualStudio.TestTools.UnitTesting;
-using Org.Apache.Rocketmq;
-using Grpc.Net.Client;
-using rmq = Apache.Rocketmq.V2;
 
+using Microsoft.VisualStudio.TestTools.UnitTesting;
+using Proto = Apache.Rocketmq.V2;
 using System;
 using System.Collections.Concurrent;
 using System.Collections.Generic;
@@ -31,33 +29,32 @@ namespace tests
         [TestMethod]
         public void TestMethod1()
         {
-            rmq::Permission perm = rmq::Permission.None;
+            Proto::Permission perm = Proto::Permission.None;
             switch (perm)
             {
-                case rmq::Permission.None:
-                    {
-                        Console.WriteLine("None");
-                        break;
-                    }
-
-                case rmq::Permission.Read:
-                    {
-                        Console.WriteLine("Read");
-                        break;
-                    }
+                case Proto::Permission.None:
+                {
+                    Console.WriteLine("None");
+                    break;
+                }
 
-                case rmq::Permission.Write:
-                    {
-                        Console.WriteLine("Write");
-                        break;
-                    }
+                case Proto::Permission.Read:
+                {
+                    Console.WriteLine("Read");
+                    break;
+                }
 
-                case rmq::Permission.ReadWrite:
-                    {
-                        Console.WriteLine("ReadWrite");
-                        break;
-                    }
+                case Proto::Permission.Write:
+                {
+                    Console.WriteLine("Write");
+                    break;
+                }
 
+                case Proto::Permission.ReadWrite:
+                {
+                    Console.WriteLine("ReadWrite");
+                    break;
+                }
             }
         }
 
@@ -82,4 +79,4 @@ namespace tests
             Assert.AreEqual(1, list.Count);
         }
     }
-}
+}
\ No newline at end of file


[rocketmq-clients] 18/28: Add more method for IClientConfig

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit d7a0a5c6a62a7a6b9e4e3d1050cf45b5694dfb95
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Fri Feb 17 10:47:51 2023 +0800

    Add more method for IClientConfig
---
 csharp/rocketmq-client-csharp/Client.cs         | 6 ++++--
 csharp/rocketmq-client-csharp/ClientConfig.cs   | 4 ++--
 csharp/rocketmq-client-csharp/IClientConfig.cs  | 6 ++++++
 csharp/rocketmq-client-csharp/Producer.cs       | 2 +-
 csharp/rocketmq-client-csharp/Session.cs        | 9 ++++++---
 csharp/rocketmq-client-csharp/SimpleConsumer.cs | 2 +-
 6 files changed, 20 insertions(+), 9 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 8e49851a..9311b48a 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -48,6 +48,7 @@ namespace Org.Apache.Rocketmq
         private readonly CancellationTokenSource _statsCts;
 
         protected readonly ClientConfig ClientConfig;
+        protected readonly Endpoints Endpoints;
         protected readonly IClientManager ClientManager;
         protected readonly string ClientId;
 
@@ -61,6 +62,7 @@ namespace Org.Apache.Rocketmq
         protected Client(ClientConfig clientConfig)
         {
             ClientConfig = clientConfig;
+            Endpoints = new Endpoints(clientConfig.Endpoints);
             ClientId = Utilities.GetClientId();
 
             ClientManager = new ClientManager(this);
@@ -288,11 +290,11 @@ namespace Org.Apache.Rocketmq
                 {
                     Name = topic
                 },
-                Endpoints = ClientConfig.Endpoints.ToProtobuf()
+                Endpoints = Endpoints.ToProtobuf()
             };
 
             var response =
-                await ClientManager.QueryRoute(ClientConfig.Endpoints, request, ClientConfig.RequestTimeout);
+                await ClientManager.QueryRoute(Endpoints, request, ClientConfig.RequestTimeout);
             var code = response.Status.Code;
             if (!Proto.Code.Ok.Equals(code))
             {
diff --git a/csharp/rocketmq-client-csharp/ClientConfig.cs b/csharp/rocketmq-client-csharp/ClientConfig.cs
index 7e434eae..82ee8403 100644
--- a/csharp/rocketmq-client-csharp/ClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/ClientConfig.cs
@@ -25,13 +25,13 @@ namespace Org.Apache.Rocketmq
         public ClientConfig(string endpoints)
         {
             RequestTimeout = TimeSpan.FromSeconds(3);
-            Endpoints = new Endpoints(endpoints);
+            Endpoints = endpoints;
         }
 
         public ICredentialsProvider CredentialsProvider { get; set; }
 
         public TimeSpan RequestTimeout { get; set; }
 
-        public Endpoints Endpoints { get; }
+        public string Endpoints { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/IClientConfig.cs b/csharp/rocketmq-client-csharp/IClientConfig.cs
index a50bdf93..d1b7ffe6 100644
--- a/csharp/rocketmq-client-csharp/IClientConfig.cs
+++ b/csharp/rocketmq-client-csharp/IClientConfig.cs
@@ -15,10 +15,16 @@
  * limitations under the License.
  */
 
+using System;
+
 namespace Org.Apache.Rocketmq
 {
     public interface IClientConfig
     {
         ICredentialsProvider CredentialsProvider { get; }
+
+        TimeSpan RequestTimeout { get; }
+
+        string Endpoints { get; }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index f9825d32..c4e78e91 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -47,7 +47,7 @@ namespace Org.Apache.Rocketmq
             base(clientConfig)
         {
             var retryPolicy = ExponentialBackoffRetryPolicy.ImmediatelyRetryPolicy(maxAttempts);
-            PublishingSettings = new PublishingSettings(ClientId, clientConfig.Endpoints, retryPolicy,
+            PublishingSettings = new PublishingSettings(ClientId, Endpoints, retryPolicy,
                 clientConfig.RequestTimeout, publishingTopics);
             _publishingRouteDataCache = new ConcurrentDictionary<string, PublishingLoadBalancer>();
             _publishingTopics = publishingTopics;
diff --git a/csharp/rocketmq-client-csharp/Session.cs b/csharp/rocketmq-client-csharp/Session.cs
index 24ff9c7c..0d35be0a 100644
--- a/csharp/rocketmq-client-csharp/Session.cs
+++ b/csharp/rocketmq-client-csharp/Session.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+using System;
 using System.Threading;
 using System.Threading.Channels;
 using System.Threading.Tasks;
@@ -30,6 +31,8 @@ namespace Org.Apache.Rocketmq
     {
         private static readonly Logger Logger = MqLogManager.Instance.GetCurrentClassLogger();
 
+        private static readonly TimeSpan SettingsInitializationTimeout = TimeSpan.FromSeconds(3);
+
         private readonly grpc::AsyncDuplexStreamingCall<Proto::TelemetryCommand, Proto::TelemetryCommand>
             _streamingCall;
 
@@ -57,12 +60,12 @@ namespace Org.Apache.Rocketmq
         {
             var writer = _streamingCall.RequestStream;
             await writer.WriteAsync(telemetryCommand);
-        } 
+        }
 
         public async Task SyncSettings(bool awaitResp)
         {
-            // TODO
-            await _semaphore.WaitAsync();
+            // Add more buffer time.
+            await _semaphore.WaitAsync(_client.GetClientConfig().RequestTimeout.Add(SettingsInitializationTimeout));
             try
             {
                 var writer = _streamingCall.RequestStream;
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index cdd5aa64..f6abd5ea 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -48,7 +48,7 @@ namespace Org.Apache.Rocketmq
             _awaitDuration = awaitDuration;
             _subscriptionRouteDataCache = new ConcurrentDictionary<string, SubscriptionLoadBalancer>();
             _subscriptionExpressions = subscriptionExpressions;
-            _simpleSubscriptionSettings = new SimpleSubscriptionSettings(ClientId, clientConfig.Endpoints,
+            _simpleSubscriptionSettings = new SimpleSubscriptionSettings(ClientId, Endpoints,
                 ConsumerGroup, clientConfig.RequestTimeout, awaitDuration, subscriptionExpressions);
             _topicRoundRobinIndex = 0;
         }


[rocketmq-clients] 24/28: Add state machine for rocketmq producer/simpleConsumer

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit f5c2878af5c8e542148f81180c35ce47735d23a5
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Wed Feb 22 17:52:56 2023 +0800

    Add state machine for rocketmq producer/simpleConsumer
---
 csharp/rocketmq-client-csharp/Client.cs         |  4 +++
 csharp/rocketmq-client-csharp/Producer.cs       | 34 +++++++++++++++----
 csharp/rocketmq-client-csharp/SimpleConsumer.cs | 45 ++++++++++++++++++++-----
 csharp/rocketmq-client-csharp/State.cs          |  7 +++-
 4 files changed, 74 insertions(+), 16 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index e0a4c553..a9cd093a 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -58,6 +58,8 @@ namespace Org.Apache.Rocketmq
         private readonly Dictionary<Endpoints, Session> _sessionsTable;
         private readonly ReaderWriterLockSlim _sessionLock;
 
+        protected volatile State State;
+
         protected Client(ClientConfig clientConfig)
         {
             ClientConfig = clientConfig;
@@ -75,6 +77,8 @@ namespace Org.Apache.Rocketmq
 
             _sessionsTable = new Dictionary<Endpoints, Session>();
             _sessionLock = new ReaderWriterLockSlim();
+
+            State = State.New;
         }
 
         public virtual async Task Start()
diff --git a/csharp/rocketmq-client-csharp/Producer.cs b/csharp/rocketmq-client-csharp/Producer.cs
index c4e78e91..d376d14c 100644
--- a/csharp/rocketmq-client-csharp/Producer.cs
+++ b/csharp/rocketmq-client-csharp/Producer.cs
@@ -73,16 +73,36 @@ namespace Org.Apache.Rocketmq
 
         public override async Task Start()
         {
-            Logger.Info($"Begin to start the rocketmq producer, clientId={ClientId}");
-            await base.Start();
-            Logger.Info($"The rocketmq producer starts successfully, clientId={ClientId}");
+            try
+            {
+                State = State.Starting;
+                Logger.Info($"Begin to start the rocketmq producer, clientId={ClientId}");
+                await base.Start();
+                Logger.Info($"The rocketmq producer starts successfully, clientId={ClientId}");
+                State = State.Running;
+            }
+            catch (Exception)
+            {
+                State = State.Failed;
+                throw;
+            }
         }
 
         public override async Task Shutdown()
         {
-            Logger.Info($"Begin to shutdown the rocketmq producer, clientId={ClientId}");
-            await base.Shutdown();
-            Logger.Info($"Shutdown the rocketmq producer successfully, clientId={ClientId}");
+            try
+            {
+                State = State.Stopping;
+                Logger.Info($"Begin to shutdown the rocketmq producer, clientId={ClientId}");
+                await base.Shutdown();
+                Logger.Info($"Shutdown the rocketmq producer successfully, clientId={ClientId}");
+                State = State.Terminated;
+            }
+            catch (Exception)
+            {
+                State = State.Failed;
+                throw;
+            }
         }
 
         protected override Proto::HeartbeatRequest WrapHeartbeatRequest()
@@ -165,7 +185,7 @@ namespace Org.Apache.Rocketmq
 
         public async Task<SendReceipt> Send(Message message, ITransaction transaction)
         {
-            var tx = (Transaction) transaction;
+            var tx = (Transaction)transaction;
             var publishingMessage = tx.TryAddMessage(message);
             var sendReceipt = await Send(message, true);
             tx.TryAddReceipt(publishingMessage, sendReceipt);
diff --git a/csharp/rocketmq-client-csharp/SimpleConsumer.cs b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
index f6abd5ea..bf9614e1 100644
--- a/csharp/rocketmq-client-csharp/SimpleConsumer.cs
+++ b/csharp/rocketmq-client-csharp/SimpleConsumer.cs
@@ -55,30 +55,59 @@ namespace Org.Apache.Rocketmq
 
         public async Task Subscribe(string topic, FilterExpression filterExpression)
         {
-            // TODO: check running status.
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Simple consumer is not running");
+            }
+
             await GetSubscriptionLoadBalancer(topic);
             _subscriptionExpressions.TryAdd(topic, filterExpression);
         }
 
         public void Unsubscribe(string topic)
         {
+            if (State.Running != State)
+            {
+                throw new InvalidOperationException("Simple consumer is not running");
+            }
+
             _subscriptionExpressions.TryRemove(topic, out _);
         }
 
         public override async Task Start()
         {
-            Logger.Info($"Begin to start the rocketmq simple consumer, clientId={ClientId}");
-            await base.Start();
-            Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+            try
+            {
+                State = State.Starting;
+                Logger.Info($"Begin to start the rocketmq simple consumer, clientId={ClientId}");
+                await base.Start();
+                Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+                State = State.Running;
+            }
+            catch (Exception)
+            {
+                State = State.Failed;
+                throw;
+            }
         }
 
         public override async Task Shutdown()
         {
-            Logger.Info($"Begin to shutdown the rocketmq simple consumer, clientId={ClientId}");
-            await base.Shutdown();
-            Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+            try
+            {
+                State = State.Stopping;
+                Logger.Info($"Begin to shutdown the rocketmq simple consumer, clientId={ClientId}");
+                await base.Shutdown();
+                Logger.Info($"The rocketmq simple consumer starts successfully, clientId={ClientId}");
+                State = State.Terminated;
+            }
+            catch (Exception)
+            {
+                State = State.Failed;
+                throw;
+            }
         }
-        
+
         protected override IEnumerable<string> GetTopics()
         {
             return _subscriptionExpressions.Keys;
diff --git a/csharp/rocketmq-client-csharp/State.cs b/csharp/rocketmq-client-csharp/State.cs
index 1dbd6b30..e353df50 100644
--- a/csharp/rocketmq-client-csharp/State.cs
+++ b/csharp/rocketmq-client-csharp/State.cs
@@ -19,6 +19,11 @@ namespace Org.Apache.Rocketmq
 {
     public enum State
     {
-        
+        New,
+        Starting,
+        Running,
+        Stopping,
+        Terminated,
+        Failed
     }
 }
\ No newline at end of file


[rocketmq-clients] 02/28: Format log output

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 1cc2e59c9b9434042f839f64276ae94e90512079
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Tue Feb 7 17:41:26 2023 +0800

    Format log output
---
 csharp/examples/ProducerNormalMessageExample.cs | 18 ++++++++++--------
 csharp/rocketmq-client-csharp/Client.cs         |  2 ++
 csharp/rocketmq-client-csharp/MessageQueue.cs   |  7 +++++--
 csharp/rocketmq-client-csharp/Resource.cs       |  6 ++++++
 csharp/rocketmq-client-csharp/TopicRouteData.cs |  4 +++-
 5 files changed, 26 insertions(+), 11 deletions(-)

diff --git a/csharp/examples/ProducerNormalMessageExample.cs b/csharp/examples/ProducerNormalMessageExample.cs
index 7aade855..c22bc695 100644
--- a/csharp/examples/ProducerNormalMessageExample.cs
+++ b/csharp/examples/ProducerNormalMessageExample.cs
@@ -29,25 +29,27 @@ namespace examples
 
         internal static async Task QuickStart()
         {
-            string accessKey = "5jFk0wK7OU6Uq395";
-            string secretKey = "V1u8z19URHs4o6RQ";
+            const string accessKey = "5jFk0wK7OU6Uq395";
+            const string secretKey = "V1u8z19URHs4o6RQ";
             // Credential provider is optional for client configuration.
             var credentialsProvider = new StaticCredentialsProvider(accessKey, secretKey);
-            string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
-            var clientConfig = new ClientConfig(endpoints);
-            clientConfig.CredentialsProvider = credentialsProvider;
+            const string endpoints = "rmq-cn-7mz30qjc71a.cn-hangzhou.rmq.aliyuncs.com:8080";
+            var clientConfig = new ClientConfig(endpoints)
+            {
+                CredentialsProvider = credentialsProvider
+            };
             // In most case, you don't need to create too many producers, single pattern is recommended.
             var producer = new Producer(clientConfig);
             
-            string topic = "lingchu_normal_topic";
+            const string topic = "lingchu_normal_topic";
             producer.SetTopics(topic);
             // Set the topic name(s), which is optional but recommended. It makes producer could prefetch
             // the topic route before message publishing.
 
             await producer.Start();
             // Define your message body.
-            byte[] bytes = Encoding.UTF8.GetBytes("foobar");
-            string tag = "yourMessageTagA";
+            var bytes = Encoding.UTF8.GetBytes("foobar");
+            const string tag = "yourMessageTagA";
             // You could set multiple keys for the single message.
             var keys = new List<string>
             {
diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index a63dc038..c4392e2d 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -213,6 +213,8 @@ namespace Org.Apache.Rocketmq
         {
             var topicRouteData = await FetchTopicRoute0(topic);
             await OnTopicRouteDataFetched(topic, topicRouteData);
+            Logger.Info(
+                $"Fetch topic route successfully, clientId={ClientId}, topic={topic}, topicRouteData={topicRouteData}");
             return topicRouteData;
         }
 
diff --git a/csharp/rocketmq-client-csharp/MessageQueue.cs b/csharp/rocketmq-client-csharp/MessageQueue.cs
index b7a6f922..385e392c 100644
--- a/csharp/rocketmq-client-csharp/MessageQueue.cs
+++ b/csharp/rocketmq-client-csharp/MessageQueue.cs
@@ -35,7 +35,10 @@ namespace Org.Apache.Rocketmq
         {
             get { return TopicResource.Name; }
         }
-        
-        
+
+        public override string ToString()
+        {
+            return $"{Broker.Name}.{TopicResource}.{QueueId}";
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/Resource.cs b/csharp/rocketmq-client-csharp/Resource.cs
index aeca3e4e..5af67d1e 100644
--- a/csharp/rocketmq-client-csharp/Resource.cs
+++ b/csharp/rocketmq-client-csharp/Resource.cs
@@ -1,3 +1,4 @@
+using System;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
@@ -21,5 +22,10 @@ namespace Org.Apache.Rocketmq
                 Name = Name
             };
         }
+        
+        public override string ToString()
+        {
+            return String.IsNullOrEmpty(Namespace) ? Name : $"{Namespace}.{Name}";
+        }
     }
 }
\ No newline at end of file
diff --git a/csharp/rocketmq-client-csharp/TopicRouteData.cs b/csharp/rocketmq-client-csharp/TopicRouteData.cs
index 3c5689a4..05418eaa 100644
--- a/csharp/rocketmq-client-csharp/TopicRouteData.cs
+++ b/csharp/rocketmq-client-csharp/TopicRouteData.cs
@@ -17,6 +17,7 @@
 
 using System;
 using System.Collections.Generic;
+using System.Linq;
 using rmq = Apache.Rocketmq.V2;
 
 namespace Org.Apache.Rocketmq
@@ -59,7 +60,8 @@ namespace Org.Apache.Rocketmq
 
         public override string ToString()
         {
-            return $"{nameof(MessageQueues)}: {MessageQueues}";
+            var mqs = MessageQueues.Select(mq => mq.ToString()).ToList();
+            return $"[{string.Join(",", mqs)}]";
         }
     }
 }
\ No newline at end of file


[rocketmq-clients] 20/28: Polish code

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

aaronai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/rocketmq-clients.git

commit 2f734d6327cca4e6cf323b732f464a458a2b2ee5
Author: Aaron Ai <ya...@gmail.com>
AuthorDate: Mon Feb 20 19:14:15 2023 +0800

    Polish code
---
 csharp/rocketmq-client-csharp/Client.cs  | 12 +++---------
 csharp/rocketmq-client-csharp/IClient.cs |  2 --
 2 files changed, 3 insertions(+), 11 deletions(-)

diff --git a/csharp/rocketmq-client-csharp/Client.cs b/csharp/rocketmq-client-csharp/Client.cs
index 5a2cd69f..24e1e4ac 100644
--- a/csharp/rocketmq-client-csharp/Client.cs
+++ b/csharp/rocketmq-client-csharp/Client.cs
@@ -54,7 +54,6 @@ namespace Org.Apache.Rocketmq
 
         protected readonly ConcurrentDictionary<Endpoints, bool> Isolated;
         private readonly ConcurrentDictionary<string, TopicRouteData> _topicRouteCache;
-        private readonly CancellationTokenSource _telemetryCts;
 
         private readonly Dictionary<Endpoints, Session> _sessionsTable;
         private readonly ReaderWriterLockSlim _sessionLock;
@@ -72,7 +71,6 @@ namespace Org.Apache.Rocketmq
             _topicRouteUpdateCts = new CancellationTokenSource();
             _settingsSyncCts = new CancellationTokenSource();
             _heartbeatCts = new CancellationTokenSource();
-            _telemetryCts = new CancellationTokenSource();
             _statsCts = new CancellationTokenSource();
 
             _sessionsTable = new Dictionary<Endpoints, Session>();
@@ -99,9 +97,10 @@ namespace Org.Apache.Rocketmq
         public virtual async Task Shutdown()
         {
             Logger.Debug($"Begin to shutdown rocketmq client, clientId={ClientId}");
-            _topicRouteUpdateCts.Cancel();
             _heartbeatCts.Cancel();
-            _telemetryCts.Cancel();
+            _topicRouteUpdateCts.Cancel();
+            _settingsSyncCts.Cancel();
+            _statsCts.Cancel();
             NotifyClientTermination();
             await ClientManager.Shutdown();
             Logger.Debug($"Shutdown the rocketmq client successfully, clientId={ClientId}");
@@ -411,11 +410,6 @@ namespace Org.Apache.Rocketmq
             }
         }
 
-        public CancellationTokenSource TelemetryCts()
-        {
-            return _telemetryCts;
-        }
-
         public abstract Settings GetSettings();
 
         public string GetClientId()
diff --git a/csharp/rocketmq-client-csharp/IClient.cs b/csharp/rocketmq-client-csharp/IClient.cs
index fc4c0127..3d20d3d5 100644
--- a/csharp/rocketmq-client-csharp/IClient.cs
+++ b/csharp/rocketmq-client-csharp/IClient.cs
@@ -23,8 +23,6 @@ namespace Org.Apache.Rocketmq
 {
     public interface IClient
     {
-        CancellationTokenSource TelemetryCts();
-
         ClientConfig GetClientConfig();
 
         Settings GetSettings();