From 28b98b096f7104f08658a116525b0812b9a14367 Mon Sep 17 00:00:00 2001 From: tangjie Date: Fri, 7 Jul 2017 09:56:15 +0800 Subject: [PATCH 01/88] [ROCKETMQ-198] Go-Client's incomplete implement. Author: tangjie Closes #22 from StyleTang/go-client-all. --- rocketmq-go/clean_expire_msg_controller.go | 49 ++ rocketmq-go/docs/roadmap.md | 105 +++-- rocketmq-go/example/consumer_example.go | 50 +-- .../example/producer_consumer_example.go | 77 ++++ rocketmq-go/example/producer_example.go | 40 ++ rocketmq-go/model/config/consumer_config.go | 58 ++- rocketmq-go/model/config/producer_config.go | 46 ++ rocketmq-go/model/constant/config.go | 2 +- rocketmq-go/model/constant/mix_all.go | 3 - .../header/pull_message_request_header.go | 12 +- .../header/send_message_request_header.go | 29 +- rocketmq-go/model/message/message_queue.go | 84 ---- rocketmq-go/model/process_queue.go | 421 ------------------ rocketmq-go/model/process_queue_info.go | 8 - rocketmq-go/model/response_code.go | 4 +- rocketmq-go/model/send_result.go | 13 +- rocketmq-go/model/topic_publishInfo.go | 76 ---- rocketmq-go/model/topic_publish_info.go | 6 - rocketmq-go/model/topic_route_data.go | 105 ----- rocketmq-go/mq_client_manage.go | 261 +++++++++++ rocketmq-go/mq_client_manager.go | 90 ---- rocketmq-go/mq_consumer.go | 74 --- rocketmq-go/mq_producer.go | 53 ++- rocketmq-go/mq_push_consumer.go | 153 +++++++ rocketmq-go/pull_message_controller.go | 329 ++++++++++++++ ...etmq_config.go => rebalance_controller.go} | 17 +- .../allocate_message_averagely.go | 80 ++++ .../allocate_message_averagely_by_circle.go | 79 ++++ .../allocate_message_by_config.go} | 12 +- .../allocate_message_by_machine_room.go | 80 ++++ .../allocate_message_queue_strategy.go | 27 ++ .../service/consume_message_service.go | 24 +- rocketmq-go/service/mq_client.go | 12 +- rocketmq-go/service/mq_fault_strategy.go | 49 ++ rocketmq-go/service/offset_store.go | 163 +++++++ rocketmq-go/service/producer_service.go | 222 ++++++++- .../service/producer_service_for_send_back.go | 115 +++++ rocketmq-go/service/rebalance.go | 307 +++++++++++++ rocketmq-go/tasks.go | 68 +++ rocketmq-go/util/compress_util.go | 63 +++ rocketmq-go/util/concurrent_map.go | 16 + .../util/message_client_id_generator.go | 4 +- rocketmq-go/util/message_properties.go | 47 ++ .../regex_util.go} | 20 +- 44 files changed, 2517 insertions(+), 1036 deletions(-) create mode 100644 rocketmq-go/clean_expire_msg_controller.go create mode 100644 rocketmq-go/example/producer_consumer_example.go delete mode 100644 rocketmq-go/model/message/message_queue.go delete mode 100644 rocketmq-go/model/topic_publishInfo.go create mode 100644 rocketmq-go/mq_client_manage.go delete mode 100644 rocketmq-go/mq_client_manager.go delete mode 100644 rocketmq-go/mq_consumer.go create mode 100644 rocketmq-go/mq_push_consumer.go create mode 100644 rocketmq-go/pull_message_controller.go rename rocketmq-go/{model/config/rocketmq_config.go => rebalance_controller.go} (68%) create mode 100644 rocketmq-go/service/allocate_message/allocate_message_averagely.go create mode 100644 rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go rename rocketmq-go/service/{offset_store_service.go => allocate_message/allocate_message_by_config.go} (67%) create mode 100644 rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go create mode 100644 rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go create mode 100644 rocketmq-go/service/mq_fault_strategy.go create mode 100644 rocketmq-go/service/offset_store.go create mode 100644 rocketmq-go/service/producer_service_for_send_back.go create mode 100644 rocketmq-go/service/rebalance.go create mode 100644 rocketmq-go/tasks.go create mode 100644 rocketmq-go/util/compress_util.go create mode 100644 rocketmq-go/util/message_properties.go rename rocketmq-go/{service/rebalance_service.go => util/regex_util.go} (75%) diff --git a/rocketmq-go/clean_expire_msg_controller.go b/rocketmq-go/clean_expire_msg_controller.go new file mode 100644 index 000000000..73bb35b3c --- /dev/null +++ b/rocketmq-go/clean_expire_msg_controller.go @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rocketmq + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "time" +) + +type CleanExpireMsgController struct { + mqClient service.RocketMqClient + clientFactory *ClientFactory +} + +func NewCleanExpireMsgController(mqClient service.RocketMqClient, clientFactory *ClientFactory) *CleanExpireMsgController { + return &CleanExpireMsgController{ + mqClient: mqClient, + clientFactory: clientFactory, + } +} + +func (self *CleanExpireMsgController) Start() { + for _, consumer := range self.clientFactory.ConsumerTable { + go func() { + cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) + //cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) + for { + <-cleanExpireMsgTimer.C + consumer.CleanExpireMsg() + cleanExpireMsgTimer.Reset(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) + //cleanExpireMsgTimer.Reset(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) + } + }() + } +} diff --git a/rocketmq-go/docs/roadmap.md b/rocketmq-go/docs/roadmap.md index 0db903326..a69c561f0 100644 --- a/rocketmq-go/docs/roadmap.md +++ b/rocketmq-go/docs/roadmap.md @@ -1,45 +1,64 @@ # RoadMap-Milestone1 ## Consumer -- [ ] ConsumerType - - [ ] PushConsumer -- [ ] MessageListener - - [ ] Concurrently -- [ ] MessageModel - - [ ] CLUSTERING -- [ ] OffsetStore - - [ ] RemoteBrokerOffsetStore -- [ ] RebalanceService -- [ ] PullMessageService -- [ ] ConsumeMessageService -- [ ] AllocateMessageQueueStrategy - - [ ] AllocateMessageQueueAveragely -- [ ] Other - - [ ] Config - - [ ] ZIP - - [ ] ConsumeFromWhere - - [ ] CONSUME_FROM_LAST_OFFSET - - [ ] CONSUME_FROM_FIRST_OFFSET - - [ ] CONSUME_FROM_TIMESTAMP - - [ ] Retry(sendMessageBack) - - [ ] TimeOut(clearExpiredMessage) - - [ ] ACK(partSuccess) - - [ ] FlowControl(messageCanNotConsume) +- [x] ConsumerType + - [x] PushConsumer +- [x] MessageListener + - [x] Concurrently +- [x] MessageModel + - [x] CLUSTERING +- [x] OffsetStore + - [x] RemoteBrokerOffsetStore +- [x] RebalanceService +- [x] PullMessageService +- [x] ConsumeMessageService +- [x] AllocateMessageQueueStrategy + - [x] AllocateMessageQueueAveragely +- [x] Other + - [x] Config + - [x] ZIP + - [x] ConsumeFromWhere + - [x] CONSUME_FROM_LAST_OFFSET + - [x] CONSUME_FROM_FIRST_OFFSET + - [x] CONSUME_FROM_TIMESTAMP + - [x] Retry(sendMessageBack) + - [x] TimeOut(clearExpiredMessage) + - [x] ACK(partSuccess) + - [x] FlowControl(messageCanNotConsume) + +## Producer +- [x] ProducerType + - [x] DefaultProducer +- [x] API + - [x] Send + - [x] Sync +- [x] Other + - [x] DelayMessage + - [x] Config + - [x] MessageId Generate + - [x] CompressMsg + - [x] TimeOut + - [x] LoadBalance + - [x] DefaultTopic + - [x] VipChannel + - [x] MQFaultStrategy + ## Manager -- [ ] Controller - - [ ] PullMessageController -- [ ] Task - - [ ] Heartbeat - - [ ] UpdateTopicRouteInfoFromNameServer - - [ ] PersistAllConsumerOffset - - [ ] ClearExpiredMessage(form consumer consumeMessageService) -- [ ] ClientRemotingProcessor - - [ ] CHECK_TRANSACTION_STATE - - [ ] NOTIFY_CONSUMER_IDS_CHANGED - - [ ] RESET_CONSUMER_CLIENT_OFFSET - - [ ] GET_CONSUMER_STATUS_FROM_CLIENT - - [ ] GET_CONSUMER_RUNNING_INFO - - [ ] CONSUME_MESSAGE_DIRECTLY +- [x] Controller + - [x] PullMessageController +- [x] Task + - [x] UpdateTopicRouteInfo + - [x] Heartbeat + - [x] Rebalance + - [x] PullMessage + - [x] CleanExpireMsg +- [x] ClientRemotingProcessor + - [x] CHECK_TRANSACTION_STATE + - [x] NOTIFY_CONSUMER_IDS_CHANGED + - [x] RESET_CONSUMER_CLIENT_OFFSET + - [x] GET_CONSUMER_STATUS_FROM_CLIENT + - [x] GET_CONSUMER_RUNNING_INFO + - [x] CONSUME_MESSAGE_DIRECTLY ## Remoting - [x] MqClientRequest @@ -122,13 +141,11 @@ - [ ] RebalanceController - [ ] PullMessageController - [ ] Task - - [ ] PollNameServer + - [ ] UpdateTopicRouteInfo - [ ] Heartbeat - - [ ] UpdateTopicRouteInfoFromNameServer - - [ ] CleanOfflineBroker - - [ ] PersistAllConsumerOffset - - [ ] ClearExpiredMessage(form consumer consumeMessageService) - - [ ] UploadFilterClassSource(FromHeartBeat/But Golang Not Easy To do this(Java Source)) + - [ ] Rebalance + - [ ] PullMessage + - [ ] CleanExpireMsg - [ ] ClientRemotingProcessor - [ ] CHECK_TRANSACTION_STATE - [ ] NOTIFY_CONSUMER_IDS_CHANGED diff --git a/rocketmq-go/example/consumer_example.go b/rocketmq-go/example/consumer_example.go index af74c01fc..7c94e589c 100644 --- a/rocketmq-go/example/consumer_example.go +++ b/rocketmq-go/example/consumer_example.go @@ -17,39 +17,37 @@ package main import ( - "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/golang/glog" + "time" ) func main() { - // create a mqClientManager instance - var mqClientConfig = &rocketmq.MqClientConfig{} - var mqClientManager = rocketmq.NewMqClientManager(mqClientConfig) - - // create rocketMq consumer - var consumerConfig = &rocketmq.MqConsumerConfig{} - var consumer1 = rocketmq.NewDefaultMQPushConsumer("testGroup", consumerConfig) - consumer1.Subscribe("testTopic", "*") - consumer1.RegisterMessageListener(func(msgs []model.MessageExt) model.ConsumeConcurrentlyResult { - var index = -1 - for i, msg := range msgs { - // your code here,for example,print msg - glog.Info(msg) - var err = errors.New("error") - if err != nil { - break - } - index = i + var ( + testTopic = "GoLang" + ) + var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + comsumer1.ConsumerConfig.PullInterval = 0 + comsumer1.ConsumerConfig.ConsumeTimeout = 1 + comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 + comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() + comsumer1.Subscribe(testTopic, "*") + comsumer1.RegisterMessageListener(func(msgs []model.MessageExt) model.ConsumeConcurrentlyResult { + for _, msg := range msgs { + glog.Info(msg.BornTimestamp) } - return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: index} + glog.Info("look message len ", len(msgs)) + return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} }) - - //register consumer to mqClientManager - mqClientManager.RegisterConsumer(consumer1) - - //start it - mqClientManager.Start() + var clienConfig = &config.ClientConfig{} + clienConfig.SetNameServerAddress("120.55.113.35:9876") + rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) + rocketMqManager.RegistConsumer(comsumer1) + rocketMqManager.Start() + select {} + rocketMqManager.ShutDown() } diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go new file mode 100644 index 000000000..0d8e45586 --- /dev/null +++ b/rocketmq-go/example/producer_consumer_example.go @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go" //todo todo I want only import this + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" + "net/http" + _ "net/http/pprof" + "time" +) + +func main() { + go func() { + http.ListenAndServe("localhost:6060", nil) + }() + var ( + testTopic = "GoLang" + ) + var producer1 = rocketmq.NewDefaultMQProducer("Test1") + producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 + var producer2 = rocketmq.NewDefaultMQProducer("Test2") + var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + comsumer1.ConsumerConfig.PullInterval = 0 + comsumer1.ConsumerConfig.ConsumeTimeout = 1 + comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 + comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() + comsumer1.Subscribe(testTopic, "*") + comsumer1.RegisterMessageListener(func(msgs []model.MessageExt) model.ConsumeConcurrentlyResult { + for _, msg := range msgs { + glog.Info(msg.BornTimestamp) + } + glog.Info("look message len ", len(msgs)) + return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} + }) + var clienConfig = &config.ClientConfig{} + clienConfig.SetNameServerAddress("120.55.113.35:9876") + rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) + rocketMqManager.RegistProducer(producer1) + rocketMqManager.RegistProducer(producer2) + rocketMqManager.RegistConsumer(comsumer1) + rocketMqManager.Start() + for i := 0; i < 10000000; i++ { + var message = &model.Message{} + message.Topic = testTopic + message.SetKeys([]string{"xxx"}) + message.SetTag("1122") + message.Body = []byte("hellAXXWord" + util.IntToString(i)) + + xx, ee := producer1.Send(message) + if ee != nil { + glog.Error(ee) + continue + } + glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + } + select {} + rocketMqManager.ShutDown() +} diff --git a/rocketmq-go/example/producer_example.go b/rocketmq-go/example/producer_example.go index bda2941b9..acc201178 100644 --- a/rocketmq-go/example/producer_example.go +++ b/rocketmq-go/example/producer_example.go @@ -15,3 +15,43 @@ * limitations under the License. */ package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" + _ "net/http/pprof" +) + +func main() { + var ( + testTopic = "GoLang" + ) + var producer1 = rocketmq.NewDefaultMQProducer("Test1") + producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 + var producer2 = rocketmq.NewDefaultMQProducer("Test2") + var clienConfig = &config.ClientConfig{} + clienConfig.SetNameServerAddress("120.55.113.35:9876") + rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) + rocketMqManager.RegistProducer(producer1) + rocketMqManager.RegistProducer(producer2) + rocketMqManager.Start() + for i := 0; i < 1000; i++ { + var message = &model.Message{} + message.Topic = testTopic + message.SetKeys([]string{"xxx"}) + message.SetTag("1122") + message.Body = []byte("hellAXXWord" + util.IntToString(i)) + + xx, ee := producer1.Send(message) + if ee != nil { + glog.Error(ee) + continue + } + glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + } + select {} + rocketMqManager.ShutDown() +} diff --git a/rocketmq-go/model/config/consumer_config.go b/rocketmq-go/model/config/consumer_config.go index 25f758554..b6a6f32ba 100644 --- a/rocketmq-go/model/config/consumer_config.go +++ b/rocketmq-go/model/config/consumer_config.go @@ -18,22 +18,30 @@ package config import "time" -type RocketMqConsumerConfig struct { - ConsumeFromWhere string - /** - * Minimum consumer thread number - */ - //consumeThreadMin int - // /** - // * Max consumer thread number - // */ - //consumeThreadMax int +/** + * Delay some time when exception occur + */ +const PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION int64 = 3000 - /** - * Threshold for dynamic adjustment of the number of thread pool - */ - //adjustThreadPoolNumsThreshold int // = 100000; +/** + * Flow control interval + */ +const PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL int64 = 50 +//consume from where +//first consume from the last offset +const CONSUME_FROM_LAST_OFFSET string = "CONSUME_FROM_LAST_OFFSET" + +//first consume from the first offset +const CONSUME_FROM_FIRST_OFFSET string = "CONSUME_FROM_FIRST_OFFSET" + +//first consume from the time +const CONSUME_FROM_TIMESTAMP string = "CONSUME_FROM_TIMESTAMP" + +//consume from where + +type RocketMqConsumerConfig struct { + ConsumeFromWhere string /** * Concurrently max span offset.it has no effect on sequential consumption */ @@ -94,6 +102,26 @@ type RocketMqConsumerConfig struct { } func NewRocketMqConsumerConfig() (consumerConfig *RocketMqConsumerConfig) { - consumerConfig = &RocketMqConsumerConfig{} + consumerConfig = &RocketMqConsumerConfig{ + ConsumeFromWhere: CONSUME_FROM_LAST_OFFSET, + ConsumeConcurrentlyMaxSpan: 2000, + PullThresholdForQueue: 1000, + PullInterval: 0, + ConsumeMessageBatchMaxSize: 1, + PullBatchSize: 32, + PostSubscriptionWhenPull: false, + UnitMode: false, + MaxReconsumeTimes: 16, + SuspendCurrentQueueTimeMillis: 1000, + ConsumeTimeout: 15, + ConsumeTimestamp: time.Now().Add(-30 * time.Minute), + + // use custom or constants.don't suggest to change + PullTimeDelayMillsWhenException: PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION, + PullTimeDelayMillsWhenFlowControl: PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL, + PullTimeDelayMillsWhenSuspend: 1000, + BrokerSuspendMaxTimeMillis: 1000 * 15, + ConsumerTimeoutMillisWhenSuspend: 1000 * 30, + } return } diff --git a/rocketmq-go/model/config/producer_config.go b/rocketmq-go/model/config/producer_config.go index ce109fb42..3bf484448 100644 --- a/rocketmq-go/model/config/producer_config.go +++ b/rocketmq-go/model/config/producer_config.go @@ -17,4 +17,50 @@ package config type RocketMqProducerConfig struct { + SendMsgTimeout int64 //done + //private int sendMsgTimeout = 3000; + CompressMsgBodyOverHowMuch int //done + //private int compressMsgBodyOverHowmuch = 1024 * 4; + ZipCompressLevel int //done + //private int zipCompressLevel = Integer.parseInt(System.getProperty(MixAll.MESSAGE_COMPRESS_LEVEL, "5")); + /** + * Just for testing or demo program + */ + // private String createTopicKey = MixAll.DEFAULT_TOPIC; + + //DefaultTopicQueueNums int + ////private volatile int defaultTopicQueueNums = 4; + + RetryTimesWhenSendFailed int + //private int retryTimesWhenSendFailed = 2; + RetryTimesWhenSendAsyncFailed int + //private int retryTimesWhenSendAsyncFailed = 2; + // + RetryAnotherBrokerWhenNotStoreOK bool + //private boolean retryAnotherBrokerWhenNotStoreOK = false; + MaxMessageSize int + //private int maxMessageSize = 1024 * 1024 * 4; // 4M + + //for MQFaultStrategy todo to be done + SendLatencyFaultEnable bool //false + LatencyMax []int64 //= {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L}; + NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; +} + +//set defaultValue +func NewProducerConfig() (producerConfig *RocketMqProducerConfig) { + producerConfig = &RocketMqProducerConfig{ + SendMsgTimeout: 3000, + CompressMsgBodyOverHowMuch: 1024 * 4, + ZipCompressLevel: 5, + MaxMessageSize: 1024 * 1024 * 4, // 4M + + RetryTimesWhenSendFailed: 2, + RetryTimesWhenSendAsyncFailed: 2, // + RetryAnotherBrokerWhenNotStoreOK: false, + SendLatencyFaultEnable: false, + LatencyMax: []int64{50, 100, 550, 1000, 2000, 3000, 15000}, + NotAvailableDuration: []int64{0, 0, 30000, 60000, 120000, 180000, 600000}, + } + return } diff --git a/rocketmq-go/model/constant/config.go b/rocketmq-go/model/constant/config.go index c48dfa591..5f7f5dbba 100644 --- a/rocketmq-go/model/constant/config.go +++ b/rocketmq-go/model/constant/config.go @@ -26,4 +26,4 @@ var USE_HEADER_SERIALIZETYPE = JSON_SERIALIZE var REMOTING_COMMAND_FLAG = 0 var REMOTING_COMMAND_LANGUAGE = "OTHER" -var REMOTING_COMMAND_VERSION int16 = 137 +var REMOTING_COMMAND_VERSION int16 = 213 diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index 6abaabe4e..2c8f0a749 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -44,9 +44,6 @@ const ( CID_ONSAPI_PULL_GROUP = "CID_ONSAPI_PULL" CID_RMQ_SYS_PREFIX = "CID_RMQ_SYS_" - //public static final List LocalInetAddrs = getLocalInetAddress() - //Localhost = localhost() - //DEFAULT_CHARSET = "UTF-8" MASTER_ID int64 = 0 CURRENT_JVM_PID diff --git a/rocketmq-go/model/header/pull_message_request_header.go b/rocketmq-go/model/header/pull_message_request_header.go index 013379674..f341b49c1 100644 --- a/rocketmq-go/model/header/pull_message_request_header.go +++ b/rocketmq-go/model/header/pull_message_request_header.go @@ -17,8 +17,18 @@ package header type PullMessageRequestHeader struct { + ConsumerGroup string `json:"consumerGroup"` + Topic string `json:"topic"` + QueueId int32 `json:"queueId"` + QueueOffset int64 `json:"queueOffset"` + MaxMsgNums int32 `json:"maxMsgNums"` + SysFlag int32 `json:"sysFlag"` + CommitOffset int64 `json:"commitOffset"` + SuspendTimeoutMillis int64 `json:"suspendTimeoutMillis"` + Subscription string `json:"subscription"` + SubVersion int64 `json:"subVersion"` } -func (header *PullMessageRequestHeader) FromMap(headerMap map[string]interface{}) { +func (self *PullMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/send_message_request_header.go b/rocketmq-go/model/header/send_message_request_header.go index 5c828a828..80f17ccea 100644 --- a/rocketmq-go/model/header/send_message_request_header.go +++ b/rocketmq-go/model/header/send_message_request_header.go @@ -17,21 +17,20 @@ package header type SendMessageRequestHeader struct { - //CommandCustomHeader - ProducerGroup string - Topic string - DefaultTopic string - DefaultTopicQueueNum int - QueueID int - SysFlag int - BornTimestamp int - Flag int - Properties string - ReconsumeTimes int - UnitMode bool - MaxReconsumeTimes int + ProducerGroup string `json:"producerGroup"` + Topic string `json:"topic"` + DefaultTopic string `json:"defaultTopic"` + DefaultTopicQueueNums int `json:"defaultTopicQueueNums"` + QueueId int32 `json:"queueId"` + SysFlag int `json:"sysFlag"` + BornTimestamp int64 `json:"bornTimestamp"` + Flag int `json:"flag"` + Properties string `json:"properties"` + ReconsumeTimes int `json:"reconsumeTimes"` + UnitMode bool `json:"unitMode"` + MaxReconsumeTimes int `json:"maxReconsumeTimes"` } -func (header *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { - //TODO +func (self *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { + return } diff --git a/rocketmq-go/model/message/message_queue.go b/rocketmq-go/model/message/message_queue.go deleted file mode 100644 index 20b47be8a..000000000 --- a/rocketmq-go/model/message/message_queue.go +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package message - -type MessageQueue struct { - topic string - brokerName string - queueId int32 -} - -func NewMessageQueue(topic string, brokerName string, queueId int32) *MessageQueue { - return &MessageQueue{ - topic: topic, - brokerName: brokerName, - queueId: queueId, - } -} - -func (queue *MessageQueue) clone() *MessageQueue { - no := new(MessageQueue) - no.topic = queue.topic - no.queueId = queue.queueId - no.brokerName = queue.brokerName - return no -} - -func (queue MessageQueue) BrokerName() string { - return queue.brokerName -} - -func (queue *MessageQueue) QueueID() int32 { - return queue.queueId -} - -type MessageQueues []*MessageQueue - -func (queues MessageQueues) Less(i, j int) bool { - imq := queues[i] - jmq := queues[j] - - if imq.topic < jmq.topic { - return true - } - - if imq.topic < jmq.topic { - return false - } - - if imq.brokerName < jmq.brokerName { - return true - } - - if imq.brokerName < jmq.brokerName { - return false - } - - if imq.queueId < jmq.queueId { - return true - } - - return false -} - -func (queues MessageQueues) Swap(i, j int) { - queues[i], queues[j] = queues[j], queues[i] -} - -func (queues MessageQueues) Len() int { - return len(queues) -} diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index 285cbdabc..e0075e088 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -191,424 +191,3 @@ func (self *ProcessQueue) PutMessage(msgs []MessageExt) (dispatchToConsume bool) } return } - -//func (self *ProcessQueue) TakeMessages(batchSize int) (messageToConsumeList []MessageExt) { -// defer self.lockTreeMap.Unlock() -// self.lockTreeMap.Lock() -// self.lastConsumeTimestamp = time.Now() -// it := self.msgTreeMap.Iterator() -// nowIndex := 0 -// for it.Next() { -// offset, message := it.Key(), it.Value() -// if (nowIndex >= batchSize) { -// break -// } -// self.msgTreeMap.Remove(offset) -// self.msgTreeMapToBeConsume.Put(offset, message) -// //messageToConsumeList = append(messageToConsumeList, message) -// } -// if (len(messageToConsumeList) == 0) { -// self.consuming = false -// } -// return -//} - -/** -# -public final static long RebalanceLockMaxLiveTime =Long.parseLong(System.getProperty("rocketmq.client.rebalance.lockMaxLiveTime", "30000")); -public final static long RebalanceLockInterval = Long.parseLong(System.getProperty("rocketmq.client.rebalance.lockInterval", "20000")); -#并发消费过期的 - case CONSUME_PASSIVELY: - pq.setDropped(true); - if (this.removeUnnecessaryMessageQueue(mq, pq)) { - it.remove(); - changed = true; - log.error("[BUG]doRebalance, {}, remove unnecessary mq, {}, because pull is pause, so try to fixed it", - consumerGroup, mq); - } - break; -private final static long PullMaxIdleTime = Long.parseLong(System.getProperty("rocketmq.client.pull.pullMaxIdleTime", "120000")); -private final Logger log = ClientLogger.getLog(); -private final ReadWriteLock lockTreeMap = new ReentrantReadWriteLock(); - -private final TreeMap msgTreeMap = new TreeMap(); -private final AtomicLong msgCount = new AtomicLong(); -private final Lock lockConsume = new ReentrantLock(); -private final TreeMap msgTreeMapTemp = new TreeMap(); -private final AtomicLong tryUnlockTimes = new AtomicLong(0); -private volatile long queueOffsetMax = 0L; -private volatile boolean dropped = false; -private volatile long lastPullTimestamp = System.currentTimeMillis(); -private volatile long lastConsumeTimestamp = System.currentTimeMillis(); -private volatile boolean locked = false; -private volatile long lastLockTimestamp = System.currentTimeMillis(); -private volatile boolean consuming = false; -private volatile long msgAccCnt = 0; - - public boolean isLockExpired() { - boolean result = (System.currentTimeMillis() - this.lastLockTimestamp) > RebalanceLockMaxLiveTime; - return result; - } - - - public boolean isPullExpired() { - boolean result = (System.currentTimeMillis() - this.lastPullTimestamp) > PullMaxIdleTime; - return result; - } - -param pushConsumer -cleanExpiredMsg(DefaultMQPushConsumer pushConsumer) { -if (pushConsumer.getDefaultMQPushConsumerImpl().isConsumeOrderly()) { -return; -} - -int loop = msgTreeMap.size() < 16 ? msgTreeMap.size() : 16; -for (int i = 0; i < loop; i++) { -MessageExt msg = null; -try { -this.lockTreeMap.readLock().lockInterruptibly(); -try { -if (!msgTreeMap.isEmpty() && System.currentTimeMillis() - Long.parseLong(MessageAccessor.getConsumeStartTimeStamp(msgTreeMap.firstEntry().getValue())) > pushConsumer.getConsumeTimeout() * 60 * 1000) { -msg = msgTreeMap.firstEntry().getValue(); -} else { - -break; -} -} finally { -this.lockTreeMap.readLock().unlock(); -} -} catch (InterruptedException e) { -log.error("getExpiredMsg exception", e); -} - -try { - -pushConsumer.sendMessageBack(msg, 3); -log.info("send expire msg back. topic={}, msgId={}, storeHost={}, queueId={}, queueOffset={}", msg.getTopic(), msg.getMsgId(), msg.getStoreHost(), msg.getQueueId(), msg.getQueueOffset()); -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -if (!msgTreeMap.isEmpty() && msg.getQueueOffset() == msgTreeMap.firstKey()) { -try { -msgTreeMap.remove(msgTreeMap.firstKey()); -} catch (Exception e) { -log.error("send expired msg exception", e); -} -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("getExpiredMsg exception", e); -} -} catch (Exception e) { -log.error("send expired msg exception", e); -} -} -} - - -public boolean putMessage(final List msgs) { -boolean dispatchToConsume = false; -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -int validMsgCnt = 0; -for (MessageExt msg : msgs) { -MessageExt old = msgTreeMap.put(msg.getQueueOffset(), msg); -if (null == old) { -validMsgCnt++; -this.queueOffsetMax = msg.getQueueOffset(); -} -} -msgCount.addAndGet(validMsgCnt); - -if (!msgTreeMap.isEmpty() && !this.consuming) { -dispatchToConsume = true; -this.consuming = true; -} - -if (!msgs.isEmpty()) { -MessageExt messageExt = msgs.get(msgs.size() - 1); -String property = messageExt.getProperty(MessageConst.PROPERTY_MAX_OFFSET); -if (property != null) { -long accTotal = Long.parseLong(property) - messageExt.getQueueOffset(); -if (accTotal > 0) { -this.msgAccCnt = accTotal; -} -} -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("putMessage exception", e); -} - -return dispatchToConsume; -} - - -public long getMaxSpan() { -try { -this.lockTreeMap.readLock().lockInterruptibly(); -try { -if (!this.msgTreeMap.isEmpty()) { -return this.msgTreeMap.lastKey() - this.msgTreeMap.firstKey(); -} -} finally { -this.lockTreeMap.readLock().unlock(); -} -} catch (InterruptedException e) { -log.error("getMaxSpan exception", e); -} - -return 0; -} - - -public long removeMessage(final List msgs) { //treeMap是维护了没有消费的 为了处理过期使用 -long result = -1; -final long now = System.currentTimeMillis(); -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -this.lastConsumeTimestamp = now; -try { -if (!msgTreeMap.isEmpty()) { -result = this.queueOffsetMax + 1; -int removedCnt = 0; -for (MessageExt msg : msgs) { -MessageExt prev = msgTreeMap.remove(msg.getQueueOffset()); -if (prev != null) { -removedCnt--; -} -} -msgCount.addAndGet(removedCnt); - -if (!msgTreeMap.isEmpty()) { -result = msgTreeMap.firstKey(); -} -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (Throwable t) { -log.error("removeMessage exception", t); -} - -return result; -} - - -public TreeMap getMsgTreeMap() { -return msgTreeMap; -} - - -public AtomicLong getMsgCount() { -return msgCount; -} - - -public boolean isDropped() { -return dropped; -} - - -public void setDropped(boolean dropped) { -this.dropped = dropped; -} - -public boolean isLocked() { -return locked; -} - -public void setLocked(boolean locked) { -this.locked = locked; -} - -public void rollback() { -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -this.msgTreeMap.putAll(this.msgTreeMapTemp); -this.msgTreeMapTemp.clear(); -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("rollback exception", e); -} -} - - -public long commit() { -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -Long offset = this.msgTreeMapTemp.lastKey(); -msgCount.addAndGet(this.msgTreeMapTemp.size() * (-1)); -this.msgTreeMapTemp.clear(); -if (offset != null) { -return offset + 1; -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("commit exception", e); -} - -return -1; -} - - -public void makeMessageToCosumeAgain(List msgs) { -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -for (MessageExt msg : msgs) { -this.msgTreeMapTemp.remove(msg.getQueueOffset()); -this.msgTreeMap.put(msg.getQueueOffset(), msg); -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("makeMessageToCosumeAgain exception", e); -} -} - - -public List takeMessags(final int batchSize) { -List result = new ArrayList(batchSize); -final long now = System.currentTimeMillis(); -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -this.lastConsumeTimestamp = now; -try { -if (!this.msgTreeMap.isEmpty()) { -for (int i = 0; i < batchSize; i++) { -Map.Entry entry = this.msgTreeMap.pollFirstEntry(); -if (entry != null) { -result.add(entry.getValue()); -msgTreeMapTemp.put(entry.getKey(), entry.getValue()); -} else { -break; -} -} -} - -if (result.isEmpty()) { -consuming = false; -} -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("take Messages exception", e); -} - -return result; -} - - -public boolean hasTempMessage() { -try { -this.lockTreeMap.readLock().lockInterruptibly(); -try { -return !this.msgTreeMap.isEmpty(); -} finally { -this.lockTreeMap.readLock().unlock(); -} -} catch (InterruptedException e) { -} - -return true; -} - - -public void clear() { -try { -this.lockTreeMap.writeLock().lockInterruptibly(); -try { -this.msgTreeMap.clear(); -this.msgTreeMapTemp.clear(); -this.msgCount.set(0); -this.queueOffsetMax = 0L; -} finally { -this.lockTreeMap.writeLock().unlock(); -} -} catch (InterruptedException e) { -log.error("rollback exception", e); -} -} - - - - -public void setLastLockTimestamp(long lastLockTimestamp) { -this.lastLockTimestamp = lastLockTimestamp; -} - - -public Lock getLockConsume() { -return lockConsume; -} - - - - -public void setLastPullTimestamp(long lastPullTimestamp) { -this.lastPullTimestamp = lastPullTimestamp; -} - - -public long getMsgAccCnt() { -return msgAccCnt; -} - - - -public long getTryUnlockTimes() { -return this.tryUnlockTimes.get(); -} - - -public void incTryUnlockTimes() { -this.tryUnlockTimes.incrementAndGet(); -} - - -public void fillProcessQueueInfo(final ProcessQueueInfo info) { -try { -this.lockTreeMap.readLock().lockInterruptibly(); - -if (!this.msgTreeMap.isEmpty()) { -info.setCachedMsgMinOffset(this.msgTreeMap.firstKey()); -info.setCachedMsgMaxOffset(this.msgTreeMap.lastKey()); -info.setCachedMsgCount(this.msgTreeMap.size()); -} - -if (!this.msgTreeMapTemp.isEmpty()) { -info.setTransactionMsgMinOffset(this.msgTreeMapTemp.firstKey()); -info.setTransactionMsgMaxOffset(this.msgTreeMapTemp.lastKey()); -info.setTransactionMsgCount(this.msgTreeMapTemp.size()); -} - -info.setLocked(this.locked); -info.setTryUnlockTimes(this.tryUnlockTimes.get()); -info.setLastLockTimestamp(this.lastLockTimestamp); - -info.setDroped(this.dropped); -info.setLastPullTimestamp(this.lastPullTimestamp); -info.setLastConsumeTimestamp(this.lastConsumeTimestamp); -} catch (Exception e) { -} finally { -this.lockTreeMap.readLock().unlock(); -} -} - - - -*/ diff --git a/rocketmq-go/model/process_queue_info.go b/rocketmq-go/model/process_queue_info.go index 6bd71bd6a..c221ef8a9 100644 --- a/rocketmq-go/model/process_queue_info.go +++ b/rocketmq-go/model/process_queue_info.go @@ -35,11 +35,3 @@ type ProcessQueueInfo struct { LastPullTimestamp int64 `json:"lastPullTimestamp"` LastConsumeTimestamp int64 `json:"lastConsumeTimestamp"` } - -//func (self ProcessQueueInfo) BuildFromProcessQueue(processQueue ProcessQueue) (processQueueInfo ProcessQueueInfo) { -// processQueueInfo = ProcessQueueInfo{} -// //processQueueInfo.CommitOffset = -// processQueueInfo.CachedMsgCount = processQueue.GetMsgCount() -// processQueueInfo.CachedMsgCount -// return -//} diff --git a/rocketmq-go/model/response_code.go b/rocketmq-go/model/response_code.go index ed40a6dc0..a42120e26 100644 --- a/rocketmq-go/model/response_code.go +++ b/rocketmq-go/model/response_code.go @@ -28,11 +28,11 @@ const ( // transaction failed, because of add db failed TransactionFailed = 4 // Broker flush disk timeout - FlushDiskTimeout = 10 + //FlushDiskTimeout = 10 // Broker slave unavailable, just for sync double write SlaveNotAvailable = 11 // Broker write slave timeout, just for sync double write - FlushSlaveTimeout = 12 + //FlushSlaveTimeout = 12 // Broker illegal message MessageIllegal = 13 // Broker, Namesrv not available,maybe service is closing or incorrect permission diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go index 4d3b31f53..857b6c402 100644 --- a/rocketmq-go/model/send_result.go +++ b/rocketmq-go/model/send_result.go @@ -18,22 +18,21 @@ package model import ( "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type SendStatus int const ( SendOK SendStatus = iota - //FlushDiskTimeout - //FlushSlaveTimeout + FlushDiskTimeout + FlushSlaveTimeout SlaveNotAvaliable ) type SendResult struct { sendStatus SendStatus msgID string - messageQueue *message.MessageQueue + messageQueue MessageQueue queueOffset int64 transactionID string offsetMsgID string @@ -41,7 +40,7 @@ type SendResult struct { traceOn bool } -func NewSendResult(status SendStatus, msgID, offsetID string, queue *message.MessageQueue, queueOffset int64) *SendResult { +func NewSendResult(status SendStatus, msgID, offsetID string, queue MessageQueue, queueOffset int64) *SendResult { return &SendResult{ sendStatus: status, msgID: msgID, @@ -87,11 +86,11 @@ func (result *SendResult) SetSendStatus(status SendStatus) { result.sendStatus = status } -func (result *SendResult) MessageQueue() *message.MessageQueue { +func (result *SendResult) MessageQueue() MessageQueue { return result.messageQueue } -func (result *SendResult) SetMessageQueue(queue *message.MessageQueue) { +func (result *SendResult) SetMessageQueue(queue MessageQueue) { result.messageQueue = queue } diff --git a/rocketmq-go/model/topic_publishInfo.go b/rocketmq-go/model/topic_publishInfo.go deleted file mode 100644 index b5f9e3756..000000000 --- a/rocketmq-go/model/topic_publishInfo.go +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package model - -import ( -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -) - -//type TopicPublishInfo struct { -// orderTopic bool -// havaTopicRouteInfo bool -// messageQueueList []*message.MessageQueue -// topicRouteData *TopicRouteData -//} -// -//func (info *TopicPublishInfo) SetOrderTopic(b bool) { -// info.orderTopic = b -//} -// -//func (info *TopicPublishInfo) Ok() bool { -// return false -//} -// -//func (info *TopicPublishInfo) MessageQueueList() []*message.MessageQueue { -// return info.messageQueueList -//} -// -//func (info *TopicPublishInfo) HaveTopicRouteInfo() bool { -// return info.havaTopicRouteInfo -//} -// -//func (info *TopicPublishInfo) SetHaveTopicRouteInfo(b bool) { -// info.havaTopicRouteInfo = b -//} -// -//func (info *TopicPublishInfo) TopicRouteData() *TopicRouteData { -// return info.topicRouteData -//} -// -//func (info *TopicPublishInfo) SetTopicRouteData(routeDate *TopicRouteData) { -// info.topicRouteData = routeDate -//} -// -//func (info *TopicPublishInfo) SelectOneMessageQueue() *message.MessageQueue { -// return nil //TODO -//} -// -//func (info *TopicPublishInfo) selectOneMessageQueueWithBroker(brokerName string) *message.MessageQueue { -// if brokerName == "" { -// return info.SelectOneMessageQueue() -// } -// return nil //TODO -//} -// -//func (info *TopicPublishInfo) QueueIdByBroker(brokerName string) int { -// return 0 //TODO -//} -// -//func (info *TopicPublishInfo) String() string { -// return "" -//} diff --git a/rocketmq-go/model/topic_publish_info.go b/rocketmq-go/model/topic_publish_info.go index 14ec088ce..26a541cd3 100644 --- a/rocketmq-go/model/topic_publish_info.go +++ b/rocketmq-go/model/topic_publish_info.go @@ -29,12 +29,6 @@ type TopicPublishInfo struct { topicQueueIndex int32 } -//private boolean orderTopic = false; -//private boolean haveTopicRouterInfo = false; -//private List messageQueueList = new ArrayList(); -//private volatile ThreadLocalIndex sendWhichQueue = new ThreadLocalIndex(0); // todo -//private TopicRouteData topicRouteData; - func (self *TopicPublishInfo) JudgeTopicPublishInfoOk() (bIsTopicOk bool) { bIsTopicOk = (len(self.MessageQueueList) > 0) return diff --git a/rocketmq-go/model/topic_route_data.go b/rocketmq-go/model/topic_route_data.go index 348479fb4..9c1ab27cb 100644 --- a/rocketmq-go/model/topic_route_data.go +++ b/rocketmq-go/model/topic_route_data.go @@ -18,114 +18,9 @@ package model import ( - //"fmt" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "sync" ) -// -//type BrokerData struct { -//} -// -//type TopicRouteData struct { -// orderTopicConf string -// queueDatas []*message.MessageQueue -// brokerDatas []*BrokerData -// filterServerTable map[string][]string -//} -// -//func NewTopicRouteData() *TopicRouteData { -// return &TopicRouteData{} -//} -// -//func (route *TopicRouteData) CloneTopicRouteData() (clonedRouteData *TopicRouteData) { -// clonedRouteData = &TopicRouteData{ -// route.orderTopicConf, -// route.queueDatas, -// route.brokerDatas, -// route.filterServerTable, -// } -// // TODO: to complete -// return -//} -// -//func (route *TopicRouteData) QueueDatas() []*message.MessageQueue { -// return route.queueDatas -//} -// -//func (route *TopicRouteData) SetQueueDatas(data []*message.MessageQueue) { -// route.queueDatas = data -//} -// -//func (route *TopicRouteData) BrokerDatas() []*BrokerData { -// return route.brokerDatas -//} -// -//func (route *TopicRouteData) SetBrokerDatas(data []*BrokerData) { -// route.brokerDatas = data -//} -// -//func (route *TopicRouteData) FilterServerTable() map[string][]string { -// return route.filterServerTable -//} -// -//func (route *TopicRouteData) SetFilterServerTable(data map[string][]string) { -// route.filterServerTable = data -//} -// -//func (route *TopicRouteData) OrderTopicConf() string { -// return route.orderTopicConf -//} -// -//func (route *TopicRouteData) SetOrderTopicConf(s string) { -// route.orderTopicConf = s -//} -// -//func (route *TopicRouteData) HashCode() (result int) { -// prime := 31 -// result = 1 -// result *= prime -// // TODO -// -// return -//} -// -//func (route *TopicRouteData) Equals(route1 interface{}) bool { -// if route == nil { -// return true -// } -// if route1 == nil { -// return false -// } -// //value, ok := route1.(TopicRouteData) -// //if !ok { -// // return false -// //} -// // TODO -// //if route.brokerDatas == nil && value.brokerDatas != nil || len(route.brokerDatas) != len(value.brokerDatas) { -// // return false -// //} -// // -// //if route.orderTopicConf == "" && value.orderTopicConf != "" || route.orderTopicConf != value.orderTopicConf { -// // return false -// //} -// // -// //if route.queueDatas == nil && value.queueDatas != nil || route.queueDatas != value.queueDatas { -// // return false -// //} -// // -// //if route.filterServerTable == nil && value.filterServerTable != nil || -// // route.filterServerTable != value.filterServerTable { -// // return false -// //} -// return true -//} -// -//func (route *TopicRouteData) String() string { -// return fmt.Sprintf("TopicRouteData [orderTopicConf=%s, queueDatas=%s, brokerDatas=%s, filterServerTable=%s]", -// route.orderTopicConf, route.queueDatas, route.brokerDatas, route.filterServerTable) -//} - type TopicRouteData struct { OrderTopicConf string QueueDatas []*QueueData diff --git a/rocketmq-go/mq_client_manage.go b/rocketmq-go/mq_client_manage.go new file mode 100644 index 000000000..790311604 --- /dev/null +++ b/rocketmq-go/mq_client_manage.go @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rocketmq + +import ( + "encoding/json" + "errors" + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" + "github.com/golang/glog" + "strings" + "sync" + "time" +) + +//@see com.alibaba.rocketmq.client.impl.factory.MQClientInstance +type MqClientManager struct { + rocketMqManagerLock sync.Mutex + //ClientId string + BootTimestamp int64 + + clientFactory *ClientFactory + + NamesrvLock sync.Mutex + HeartBeatLock sync.Mutex + //all producer and consumer use this + mqClient service.RocketMqClient + //all producer and consumer use this + //private final ClientRemotingProcessor clientRemotingProcessor; + // private final PullMessageService pullMessageService; + //private final RebalanceService rebalanceService; + // private final ConsumerStatsManager consumerStatsManager; + // private final AtomicLong storeTimesTotal = new AtomicLong(0); + ServiceState int + + //should be here because need all producer consumer + pullMessageController *PullMessageController + cleanExpireMsgController *CleanExpireMsgController + rebalanceControllr *RebalanceController + //should be here because need all producer consumer + defaultProducerService *service.DefaultProducerService //for send back message +} + +func MqClientManagerInit(clientConfig *config.ClientConfig) (rocketMqManager *MqClientManager) { + rocketMqManager = &MqClientManager{} + rocketMqManager.BootTimestamp = time.Now().Unix() + rocketMqManager.clientFactory = ClientFactoryInit() + rocketMqManager.mqClient = service.MqClientInit(clientConfig, rocketMqManager.InitClientRequestProcessor()) // todo todo todo + rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) + rocketMqManager.cleanExpireMsgController = NewCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) + rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) + + return +} + +//CHECK_TRANSACTION_STATE +//NOTIFY_CONSUMER_IDS_CHANGED +//RESET_CONSUMER_CLIENT_OFFSET +//GET_CONSUMER_STATUS_FROM_CLIENT +//GET_CONSUMER_RUNNING_INFO +//CONSUME_MESSAGE_DIRECTLY +func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { + clientRequestProcessor = func(cmd *remoting.RemotingCommand) (response *remoting.RemotingCommand) { + switch cmd.Code { + case remoting.CHECK_TRANSACTION_STATE: + glog.V(2).Info("receive_request_code CHECK_TRANSACTION_STATE") + // todo this version don't impl this + break + case remoting.NOTIFY_CONSUMER_IDS_CHANGED: + glog.V(1).Info("receive_request_code NOTIFY_CONSUMER_IDS_CHANGED") + self.rebalanceControllr.doRebalance() + break + case remoting.RESET_CONSUMER_CLIENT_OFFSET: // struct json key supported + glog.V(2).Info("receive_request_code RESET_CONSUMER_CLIENT_OFFSET") + glog.V(2).Info("op=look cmd body", string(cmd.Body)) + var resetOffsetRequestHeader = &header.ResetOffsetRequestHeader{} + if cmd.ExtFields != nil { + resetOffsetRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct + glog.V(2).Info("op=look ResetOffsetRequestHeader", resetOffsetRequestHeader) + resetOffsetBody := &model.ResetOffsetBody{} + err := resetOffsetBody.Decode(cmd.Body) + if err != nil { + return + } + glog.V(2).Info("op=look resetOffsetBody xxxxx", resetOffsetBody) + self.resetConsumerOffset(resetOffsetRequestHeader.Topic, resetOffsetRequestHeader.Group, resetOffsetBody.OffsetTable) + } + break + case remoting.GET_CONSUMER_STATUS_FROM_CLIENT: // useless we can use GET_CONSUMER_RUNNING_INFO instead + glog.V(2).Info("receive_request_code GET_CONSUMER_STATUS_FROM_CLIENT") + break + case remoting.GET_CONSUMER_RUNNING_INFO: + glog.V(2).Info("receive_request_code GET_CONSUMER_RUNNING_INFO") + var getConsumerRunningInfoRequestHeader = &header.GetConsumerRunningInfoRequestHeader{} + if cmd.ExtFields != nil { + getConsumerRunningInfoRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct + consumerRunningInfo := model.ConsumerRunningInfo{} + consumerRunningInfo.Properties = map[string]string{} + defaultMQPushConsumer := self.clientFactory.ConsumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] + consumerConfigMap := structs.Map(defaultMQPushConsumer.ConsumerConfig) // todo test + for key, value := range consumerConfigMap { + consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) + } + + consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.GetRemotingClient().GetNamesrvAddrList(), ";") + consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.GetMqTableInfo() + + glog.V(2).Info("op=look consumerRunningInfo", consumerRunningInfo) + jsonByte, err := consumerRunningInfo.Encode() + glog.V(2).Info("op=enCode jsonByte", string(jsonByte)) + if err != nil { + glog.Error(err) + return + } + response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) + } + + break + case remoting.CONSUME_MESSAGE_DIRECTLY: + glog.V(2).Info("receive_request_code CONSUME_MESSAGE_DIRECTLY") + var consumeMessageDirectlyResultRequestHeader = &header.ConsumeMessageDirectlyResultRequestHeader{} + if cmd.ExtFields != nil { + consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) + messageExt := &DecodeMessage(cmd.Body)[0] + glog.V(2).Info("op=look", messageExt) + defaultMQPushConsumer := self.clientFactory.ConsumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] + consumeResult, err := defaultMQPushConsumer.consumeMessageService.ConsumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) + if err != nil { + return + } + jsonByte, err := json.Marshal(consumeResult) + if err != nil { + glog.Error(err) + return + } + response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) + } + default: + glog.Error("illeage requestCode ", cmd.Code) + } + return + } + return +} +func (self *MqClientManager) RegistProducer(producer *DefaultMQProducer) { + producer.producerService = service.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, self.mqClient) + self.clientFactory.ProducerTable[producer.producerGroup] = producer + return +} + +func (self *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { + consumer := self.clientFactory.ConsumerTable[group] + if consumer == nil { + glog.Error("resetConsumerOffset beacuse consumer not online,group=", group) + return + } + consumer.resetOffset(offsetTable) +} +func (self *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { + if self.defaultProducerService == nil { + self.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, config.NewProducerConfig(), self.mqClient) + } + consumer.mqClient = self.mqClient + consumer.offsetStore = service.RemoteOffsetStoreInit(consumer.consumerGroup, self.mqClient) + self.clientFactory.ConsumerTable[consumer.consumerGroup] = consumer + consumer.rebalance = service.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) + + fmt.Println(consumer.consumeMessageService) + + consumer.consumeMessageService.Init(consumer.consumerGroup, self.mqClient, consumer.offsetStore, self.defaultProducerService, consumer.ConsumerConfig) + return +} + +func (self *MqClientManager) Start() { + //self.SendHeartbeatToAllBrokerWithLock()//we should send heartbeat first + self.StartAllScheduledTask() +} + +func (self MqClientManager) ShutDown() { + +} + +type ClientFactory struct { + ProducerTable map[string]*DefaultMQProducer //group|RocketMQProducer + ConsumerTable map[string]*DefaultMQPushConsumer //group|Consumer +} + +func ClientFactoryInit() (clientFactory *ClientFactory) { + clientFactory = &ClientFactory{} + clientFactory.ProducerTable = make(map[string]*DefaultMQProducer) + clientFactory.ConsumerTable = make(map[string]*DefaultMQPushConsumer) + return +} + +//heart beat +func (self MqClientManager) SendHeartbeatToAllBrokerWithLock() error { + heartbeatData := self.prepareHeartbeatData() + if len(heartbeatData.ConsumerDataSet) == 0 { + return errors.New("send heartbeat error") + } + self.mqClient.SendHeartbeatToAllBroker(heartbeatData) + return nil +} + +//routeInfo +func (self MqClientManager) UpdateTopicRouteInfoFromNameServer() { + var topicSet []string + for _, consumer := range self.clientFactory.ConsumerTable { + for key, _ := range consumer.subscription { + topicSet = append(topicSet, key) + } + } + topicSet = append(topicSet, self.mqClient.GetPublishTopicList()...) + for _, topic := range topicSet { + self.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + + } +} + +func (self MqClientManager) prepareHeartbeatData() *model.HeartbeatData { + heartbeatData := new(model.HeartbeatData) + heartbeatData.ClientId = self.mqClient.GetClientId() + heartbeatData.ConsumerDataSet = make([]*model.ConsumerData, 0) + heartbeatData.ProducerDataSet = make([]*model.ProducerData, 0) + for group, consumer := range self.clientFactory.ConsumerTable { + consumerData := new(model.ConsumerData) + consumerData.GroupName = group + consumerData.ConsumeType = consumer.consumeType + consumerData.ConsumeFromWhere = consumer.ConsumerConfig.ConsumeFromWhere + consumerData.MessageModel = consumer.messageModel + consumerData.SubscriptionDataSet = consumer.Subscriptions() + consumerData.UnitMode = consumer.unitMode + heartbeatData.ConsumerDataSet = append(heartbeatData.ConsumerDataSet, consumerData) + } + for group := range self.clientFactory.ProducerTable { + producerData := new(model.ProducerData) + producerData.GroupName = group + heartbeatData.ProducerDataSet = append(heartbeatData.ProducerDataSet, producerData) + } + return heartbeatData +} diff --git a/rocketmq-go/mq_client_manager.go b/rocketmq-go/mq_client_manager.go deleted file mode 100644 index 731158ffe..000000000 --- a/rocketmq-go/mq_client_manager.go +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rocketmq - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" - "sync" - "time" -) - -type MqClientManager struct { - clientFactory *ClientFactory - rocketMqClient service.RocketMqClient - pullMessageController *PullMessageController - defaultProducerService RocketMQProducer //for send back message - - rocketMqManagerLock sync.Mutex - //ClientId string - BootTimestamp int64 - - NamesrvLock sync.Mutex - HeartBeatLock sync.Mutex - //rebalanceControllr *RebalanceController -} - -type MqClientConfig struct { -} - -func NewMqClientManager(clientConfig *MqClientConfig) (rocketMqManager *MqClientManager) { - rocketMqManager = &MqClientManager{} - rocketMqManager.BootTimestamp = time.Now().Unix() - rocketMqManager.clientFactory = clientFactoryInit() - //rocketMqManager.rocketMqClient = - //rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) - //rocketMqManager.cleanExpireMsgController = NewCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) - //rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) - - return -} - -func (self *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { - return -} - -func (self *MqClientManager) RegisterConsumer(consumer RocketMQConsumer) { - // todo check config - //if (self.defaultProducerService == nil) { - // self.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, mq_config.NewProducerConfig(), self.mqClient) - //} - return -} - -func (self *MqClientManager) Start() { - //self.SendHeartbeatToAllBrokerWithLock()//we should send heartbeat first - self.startAllScheduledTask() -} -func (manager *MqClientManager) startAllScheduledTask() { - -} - -func clientFactoryInit() (clientFactory *ClientFactory) { - clientFactory = &ClientFactory{} - clientFactory.ProducerTable = make(map[string]RocketMQProducer) - clientFactory.ConsumerTable = make(map[string]RocketMQConsumer) - return -} - -type ClientFactory struct { - ProducerTable map[string]RocketMQProducer //group|RocketMQProducer - ConsumerTable map[string]RocketMQConsumer //group|Consumer -} - -type PullMessageController struct { - rocketMqClient service.RocketMqClient - clientFactory *ClientFactory -} diff --git a/rocketmq-go/mq_consumer.go b/rocketmq-go/mq_consumer.go deleted file mode 100644 index 711253774..000000000 --- a/rocketmq-go/mq_consumer.go +++ /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. - */ -package rocketmq - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" -) - -type RocketMQConsumer interface { -} - -type MqConsumerConfig struct { -} -type DefaultMQPushConsumer struct { - offsetStore service.OffsetStore //for consumer's offset - mqClient service.RocketMqClient - rebalance *service.Rebalance //Rebalance's impl depend on offsetStore - consumeMessageService service.ConsumeMessageService - consumerConfig *MqConsumerConfig - - consumerGroup string - //consumeFromWhere string - consumeType string - messageModel string - unitMode bool - - subscription map[string]string //topic|subExpression - subscriptionTag map[string][]string // we use it filter again - // 分配策略 - pause bool //when reset offset we need pause -} - -func NewDefaultMQPushConsumer(consumerGroup string, mqConsumerConfig *MqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { - defaultMQPushConsumer = &DefaultMQPushConsumer{} - defaultMQPushConsumer.consumerConfig = mqConsumerConfig - return -} - -func (self *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { - self.consumeMessageService = service.NewConsumeMessageConcurrentlyServiceImpl(messageListener) -} -func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { - //self.subscription[topic] = subExpression - //if len(subExpression) == 0 || subExpression == "*" { - // return - //} - //tags := strings.Split(subExpression, "||") - //tagsList := []string{} - //for _, tag := range tags { - // t := strings.TrimSpace(tag) - // if len(t) == 0 { - // continue - // } - // tagsList = append(tagsList, t) - //} - //if len(tagsList) > 0 { - // self.subscriptionTag[topic] = tagsList - //} -} diff --git a/rocketmq-go/mq_producer.go b/rocketmq-go/mq_producer.go index d1a011b85..098377d85 100644 --- a/rocketmq-go/mq_producer.go +++ b/rocketmq-go/mq_producer.go @@ -1,31 +1,40 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ package rocketmq -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" +import () +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" +) type RocketMQProducer interface { + Send(message *model.Message) (sendResult *model.SendResult, err error) + SendWithTimeout(message *model.Message, timeout int) (sendResult *model.SendResult, err error) + //SendAsync(message *model.Message) (sendResult *model.SendResult,err error) + //SendAsyncWithTimeout(message *model.Message) (sendResult *model.SendResult,err error) + //SendOneWay(message *model.Message) (sendResult *model.SendResult,err error) } +type DefaultMQProducer struct { + producerGroup string + ProducerConfig *config.RocketMqProducerConfig -type MqProducerConfig struct { + producerService service.ProducerService } -type DefaultMQProducer struct { - producerGroup string - mqProducerConfig *MqProducerConfig - producerService service.ProducerService +func NewDefaultMQProducer(producerGroup string) (rocketMQProducer *DefaultMQProducer) { + rocketMQProducer = &DefaultMQProducer{ + producerGroup: producerGroup, + ProducerConfig: config.NewProducerConfig(), + } + return +} + +func (self *DefaultMQProducer) Send(message *model.Message) (sendResult *model.SendResult, err error) { + sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) + return +} +func (self *DefaultMQProducer) SendWithTimeout(message *model.Message, timeout int64) (sendResult *model.SendResult, err error) { + sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) + return } diff --git a/rocketmq-go/mq_push_consumer.go b/rocketmq-go/mq_push_consumer.go new file mode 100644 index 000000000..245bbe43b --- /dev/null +++ b/rocketmq-go/mq_push_consumer.go @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rocketmq + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/golang/glog" + "strings" + "time" +) + +type Consumer interface { + RegisterMessageListener(listener model.MessageListener) + Subscribe(topic string, subExpression string) +} + +type DefaultMQPushConsumer struct { + consumerGroup string + //consumeFromWhere string + consumeType string + messageModel string + unitMode bool + + subscription map[string]string //topic|subExpression + subscriptionTag map[string][]string // we use it filter again + offsetStore service.OffsetStore + mqClient service.RocketMqClient + rebalance *service.Rebalance + pause bool //when reset offset we need pause + consumeMessageService service.ConsumeMessageService + ConsumerConfig *config.RocketMqConsumerConfig +} + +func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *DefaultMQPushConsumer) { + defaultMQPushConsumer = &DefaultMQPushConsumer{ + consumerGroup: consumerGroup, + //consumeFromWhere:"CONSUME_FROM_FIRST_OFFSET", //todo use config + consumeType: "CONSUME_PASSIVELY", + messageModel: "CLUSTERING", + pause: false} + defaultMQPushConsumer.subscription = make(map[string]string) + defaultMQPushConsumer.subscriptionTag = make(map[string][]string) + defaultMQPushConsumer.ConsumerConfig = config.NewRocketMqConsumerConfig() + return +} +func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { + self.subscription[topic] = subExpression + if len(subExpression) == 0 || subExpression == "*" { + return + } + tags := strings.Split(subExpression, "||") + tagsList := []string{} + for _, tag := range tags { + t := strings.TrimSpace(tag) + if len(t) == 0 { + continue + } + tagsList = append(tagsList, t) + } + if len(tagsList) > 0 { + self.subscriptionTag[topic] = tagsList + } +} + +func (self *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { + self.consumeMessageService = service.NewConsumeMessageConcurrentlyServiceImpl(messageListener) +} + +func (self *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { + self.pause = true + glog.Info("now we ClearProcessQueue 0 ", offsetTable) + + self.rebalance.ClearProcessQueue(offsetTable) + glog.Info("now we ClearProcessQueue", offsetTable) + go func() { + waitTime := time.NewTimer(10 * time.Second) + <-waitTime.C + defer func() { + self.pause = false + self.rebalance.DoRebalance() + }() + + for messageQueue, offset := range offsetTable { + processQueue := self.rebalance.GetProcessQueue(messageQueue) + if processQueue == nil || offset < 0 { + continue + } + glog.Info("now we UpdateOffset", messageQueue, offset) + self.offsetStore.UpdateOffset(&messageQueue, offset, false) + self.rebalance.RemoveProcessQueue(&messageQueue) + } + }() +} + +func (self *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { + subscriptions := make([]*model.SubscriptionData, 0) + for _, subscription := range self.rebalance.SubscriptionInner { + subscriptions = append(subscriptions, subscription) + } + return subscriptions +} + +func (self *DefaultMQPushConsumer) CleanExpireMsg() { + nowTime := int64(time.Now().UnixNano()) / 1000000 //will cause nowTime - consumeStartTime <0 ,but no matter + messageQueueList, processQueueList := self.rebalance.GetProcessQueueList() + for messageQueueIndex, processQueue := range processQueueList { + loop := processQueue.GetMsgCount() + if loop > 16 { + loop = 16 + } + for i := 0; i < loop; i++ { + _, message := processQueue.GetMinMessageInTree() + if message == nil { + break + } + consumeStartTime := message.GetConsumeStartTime() + maxDiffTime := self.ConsumerConfig.ConsumeTimeout * 1000 * 60 + //maxDiffTime := self.ConsumerConfig.ConsumeTimeout + glog.V(2).Info("look message.GetConsumeStartTime()", consumeStartTime) + glog.V(2).Infof("look diff %d %d", nowTime-consumeStartTime, maxDiffTime) + //if(nowTime - consumeStartTime <0){ + // panic("nowTime - consumeStartTime <0") + //} + if nowTime-consumeStartTime < maxDiffTime { + break + } + glog.Info("look now we send expire message back", message.Topic, message.MsgId) + err := self.consumeMessageService.SendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) + if err != nil { + glog.Error("op=send_expire_message_back_error", err) + continue + } + processQueue.DeleteExpireMsg(int(message.QueueOffset)) + } + } + return +} diff --git a/rocketmq-go/pull_message_controller.go b/rocketmq-go/pull_message_controller.go new file mode 100644 index 000000000..320cc3112 --- /dev/null +++ b/rocketmq-go/pull_message_controller.go @@ -0,0 +1,329 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rocketmq + +import ( + "bytes" + "compress/zlib" + "encoding/binary" + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" + "io/ioutil" + "strconv" + "time" +) + +type PullMessageController struct { + mqClient service.RocketMqClient + clientFactory *ClientFactory +} + +func NewPullMessageController(mqClient service.RocketMqClient, clientFactory *ClientFactory) *PullMessageController { + return &PullMessageController{ + mqClient: mqClient, + clientFactory: clientFactory, + } +} + +func (self *PullMessageController) Start() { + go func() { + for { + pullRequest := self.mqClient.DequeuePullMessageRequest() + self.pullMessage(pullRequest) + } + }() +} + +func (self *PullMessageController) needDelayPullMessage(mqPushConsumer *DefaultMQPushConsumer, pullRequest *model.PullRequest) (needDelayTime int64) { + if pullRequest.ProcessQueue.GetMsgCount() > mqPushConsumer.ConsumerConfig.PullThresholdForQueue { + return mqPushConsumer.ConsumerConfig.PullTimeDelayMillsWhenFlowControl + } + if pullRequest.ProcessQueue.GetMaxSpan() > mqPushConsumer.ConsumerConfig.ConsumeConcurrentlyMaxSpan { + return mqPushConsumer.ConsumerConfig.PullTimeDelayMillsWhenFlowControl + } + return +} + +func (self *PullMessageController) pullMessageLater(pullRequest *model.PullRequest, millisecond int64) { + go func() { + timeoutTimer := time.NewTimer(time.Duration(millisecond) * time.Millisecond) + <-timeoutTimer.C + self.pullMessage(pullRequest) + }() + return +} + +func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { + defaultMQPullConsumer := self.clientFactory.ConsumerTable[pullRequest.ConsumerGroup] + if pullRequest.ProcessQueue.IsDropped() { + return + } + + //pullRequest.ProcessQueue.SetLastPullTimestamp(System.currentTimeMillis()); + // state ok + // isPause + + delayPullTime := self.needDelayPullMessage(defaultMQPullConsumer, pullRequest) + if delayPullTime > 0 { + self.pullMessageLater(pullRequest, delayPullTime) + return + } + commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, service.READ_FROM_MEMORY) + + subscriptionData, ok := defaultMQPullConsumer.rebalance.SubscriptionInner[pullRequest.MessageQueue.Topic] + if !ok { + self.pullMessageLater(pullRequest, defaultMQPullConsumer.ConsumerConfig.PullTimeDelayMillsWhenException) + return + } + + var sysFlag int32 = 0 + if commitOffsetValue > 0 { + sysFlag |= constant.FLAG_COMMIT_OFFSET + } + sysFlag |= constant.FLAG_SUSPEND + sysFlag |= constant.FLAG_SUBSCRIPTION + requestHeader := new(header.PullMessageRequestHeader) + requestHeader.ConsumerGroup = pullRequest.ConsumerGroup + requestHeader.Topic = pullRequest.MessageQueue.Topic + requestHeader.QueueId = pullRequest.MessageQueue.QueueId + requestHeader.QueueOffset = pullRequest.NextOffset + + requestHeader.CommitOffset = commitOffsetValue + requestHeader.SuspendTimeoutMillis = defaultMQPullConsumer.ConsumerConfig.BrokerSuspendMaxTimeMillis + requestHeader.MaxMsgNums = int32(defaultMQPullConsumer.ConsumerConfig.PullBatchSize) + requestHeader.SubVersion = subscriptionData.SubVersion + requestHeader.Subscription = subscriptionData.SubString + + requestHeader.SysFlag = sysFlag + + pullCallback := func(responseFuture *remoting.ResponseFuture) { + var nextBeginOffset int64 = pullRequest.NextOffset + + if responseFuture != nil { + responseCommand := responseFuture.ResponseCommand + if responseCommand.Code == remoting.SUCCESS && len(responseCommand.Body) > 0 { + //FOUND + var err error + pullResult := responseCommand.ExtFields + if ok { + if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { + if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { + nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) + if err != nil { + glog.Error(err) + return + } + } + } + } + msgs := DecodeMessage(responseFuture.ResponseCommand.Body) + + msgs = FilterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) + if len(msgs) == 0 { + if pullRequest.ProcessQueue.GetMsgCount() == 0 { + defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) + } + } + // + pullRequest.ProcessQueue.PutMessage(msgs) + defaultMQPullConsumer.consumeMessageService.SubmitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) + } else { + //glog.Error(fmt.Sprintf("pull message error,code=%d,body=%s", responseCommand.Code, string(responseCommand.Body))) + var err error // change the offset , use nextBeginOffset + pullResult := responseCommand.ExtFields + if ok { + if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { + if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { + nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) + if err != nil { + glog.Error(err) + } + } + } + } + if responseCommand.Code == remoting.PULL_NOT_FOUND || responseCommand.Code == remoting.PULL_RETRY_IMMEDIATELY { + //NO_NEW_MSG //NO_MATCHED_MSG + if pullRequest.ProcessQueue.GetMsgCount() == 0 { + defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) + } + //update offset increase only + //failedPullRequest, _ := json.Marshal(pullRequest) + //glog.Error("the pull request offset illegal", string(failedPullRequest)) + } else if responseCommand.Code == remoting.PULL_OFFSET_MOVED { + //OFFSET_ILLEGAL + glog.Error(fmt.Sprintf("PULL_OFFSET_MOVED,code=%d,body=%s", responseCommand.Code, string(responseCommand.Body))) + pullRequest.ProcessQueue.SetDrop(true) + go func() { + executeTaskLater := time.NewTimer(10 * time.Second) + <-executeTaskLater.C + defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, false) + defaultMQPullConsumer.rebalance.RemoveProcessQueue(pullRequest.MessageQueue) + }() + } else { + glog.Errorf("illegal response code. pull message error,code=%d,request=%v OFFSET_ILLEGAL", responseCommand.Code, requestHeader) + glog.Error(pullRequest.MessageQueue) + time.Sleep(1 * time.Second) + } + } + } else { + glog.Error("responseFuture is nil") + } + + if pullRequest.ProcessQueue.IsDropped() { + return + } + nextPullRequest := &model.PullRequest{ + ConsumerGroup: pullRequest.ConsumerGroup, + NextOffset: nextBeginOffset, + MessageQueue: pullRequest.MessageQueue, + ProcessQueue: pullRequest.ProcessQueue, + } + if defaultMQPullConsumer.ConsumerConfig.PullInterval > 0 { + go func() { + nextPullTime := time.NewTimer(time.Duration(defaultMQPullConsumer.ConsumerConfig.PullInterval) * time.Millisecond) + <-nextPullTime.C + self.mqClient.EnqueuePullMessageRequest(nextPullRequest) + }() + } else { + self.mqClient.EnqueuePullMessageRequest(nextPullRequest) + } + } + glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) + self.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) +} +func FilterMessageAgainByTags(msgExts []model.MessageExt, subscriptionTagList []string) (result []model.MessageExt) { + result = msgExts + if len(subscriptionTagList) == 0 { + return + } + result = []model.MessageExt{} + for _, msg := range msgExts { + for _, tag := range subscriptionTagList { + if tag == msg.GetTag() { + result = append(result, msg) + break + } + } + } + return +} + +func (self *PullMessageController) consumerPullMessageAsync(brokerName string, requestHeader remoting.CustomerHeader, invokeCallback remoting.InvokeCallback) { + brokerAddr, _, found := self.mqClient.FindBrokerAddressInSubscribe(brokerName, 0, false) + if found { + remotingCommand := remoting.NewRemotingCommand(remoting.PULL_MESSAGE, requestHeader) + self.mqClient.GetRemotingClient().InvokeAsync(brokerAddr, remotingCommand, 1000, invokeCallback) + } +} + +func DecodeMessage(data []byte) []model.MessageExt { + buf := bytes.NewBuffer(data) + var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 + var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 + var topicLen byte + var topic, body, properties, bornHost, storeHost []byte + var propertiesLength int16 + + var propertiesmap = make(map[string]string) + + msgs := []model.MessageExt{} + for buf.Len() > 0 { + msg := model.MessageExt{Message: &model.Message{}} + binary.Read(buf, binary.BigEndian, &storeSize) + binary.Read(buf, binary.BigEndian, &magicCode) + binary.Read(buf, binary.BigEndian, &bodyCRC) + binary.Read(buf, binary.BigEndian, &queueId) + binary.Read(buf, binary.BigEndian, &flag) + binary.Read(buf, binary.BigEndian, &queueOffset) + binary.Read(buf, binary.BigEndian, &physicOffset) + binary.Read(buf, binary.BigEndian, &sysFlag) + binary.Read(buf, binary.BigEndian, &bornTimeStamp) + bornHost = make([]byte, 4) + binary.Read(buf, binary.BigEndian, &bornHost) + binary.Read(buf, binary.BigEndian, &bornPort) + binary.Read(buf, binary.BigEndian, &storeTimestamp) + storeHost = make([]byte, 4) + binary.Read(buf, binary.BigEndian, &storeHost) + binary.Read(buf, binary.BigEndian, &storePort) + binary.Read(buf, binary.BigEndian, &reconsumeTimes) + binary.Read(buf, binary.BigEndian, &preparedTransactionOffset) + binary.Read(buf, binary.BigEndian, &bodyLength) + if bodyLength > 0 { + body = make([]byte, bodyLength) + binary.Read(buf, binary.BigEndian, body) + if (sysFlag & constant.CompressedFlag) == constant.CompressedFlag { + b := bytes.NewReader(body) + z, err := zlib.NewReader(b) + if err != nil { + glog.Error(err) + return nil + } + body, err = ioutil.ReadAll(z) + z.Close() + if err != nil { + glog.Error(err) + return nil + } + } + } + binary.Read(buf, binary.BigEndian, &topicLen) + topic = make([]byte, int(topicLen)) + binary.Read(buf, binary.BigEndian, &topic) + binary.Read(buf, binary.BigEndian, &propertiesLength) + if propertiesLength > 0 { + properties = make([]byte, propertiesLength) + binary.Read(buf, binary.BigEndian, &properties) + propertiesmap = util.String2MessageProperties(string(properties)) + } + + if magicCode != -626843481 { + glog.Errorf("magic code is error %d", magicCode) + return nil + } + + msg.Topic = string(topic) + msg.QueueId = queueId + msg.SysFlag = sysFlag + msg.QueueOffset = queueOffset + msg.BodyCRC = bodyCRC + msg.StoreSize = storeSize + msg.BornTimestamp = bornTimeStamp + msg.ReconsumeTimes = reconsumeTimes + msg.Flag = int(flag) + msg.CommitLogOffset = physicOffset + msg.StoreTimestamp = storeTimestamp + msg.PreparedTransactionOffset = preparedTransactionOffset + msg.Body = body + msg.Properties = propertiesmap + + // < 3.5.8 use messageOffsetId + // >= 3.5.8 use clientUniqMsgId + msg.MsgId = msg.GetMsgUniqueKey() + if len(msg.MsgId) == 0 { + msg.MsgId = util.GeneratorMessageOffsetId(storeHost, storePort, msg.CommitLogOffset) + } + msgs = append(msgs, msg) + } + + return msgs +} diff --git a/rocketmq-go/model/config/rocketmq_config.go b/rocketmq-go/rebalance_controller.go similarity index 68% rename from rocketmq-go/model/config/rocketmq_config.go rename to rocketmq-go/rebalance_controller.go index 56e89b9bf..d6d4001eb 100644 --- a/rocketmq-go/model/config/rocketmq_config.go +++ b/rocketmq-go/rebalance_controller.go @@ -14,7 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package config +package rocketmq -type RocketMqClientConfig struct { +type RebalanceController struct { + clientFactory *ClientFactory +} + +func NewRebalanceController(clientFactory *ClientFactory) *RebalanceController { + return &RebalanceController{ + clientFactory: clientFactory, + } +} + +func (self *RebalanceController) doRebalance() { + for _, consumer := range self.clientFactory.ConsumerTable { + consumer.rebalance.DoRebalance() + } } diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely.go b/rocketmq-go/service/allocate_message/allocate_message_averagely.go new file mode 100644 index 000000000..cdfe77541 --- /dev/null +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely.go @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service_allocate_message + +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type AllocateMessageQueueAveragely struct{} + +func (self *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { + + if currentCID == "" { + return nil, errors.New("currentCID is empty") + } + + if mqAll == nil || len(mqAll) == 0 { + return nil, errors.New("mqAll is nil or mqAll empty") + } + + if cidAll == nil || len(cidAll) == 0 { + return nil, errors.New("cidAll is nil or cidAll empty") + } + + result := make([]model.MessageQueue, 0) + for i, cid := range cidAll { + if cid == currentCID { + mqLen := len(mqAll) + cidLen := len(cidAll) + mod := mqLen % cidLen + var averageSize int + if mqLen < cidLen { + averageSize = 1 + } else { + if mod > 0 && i < mod { + averageSize = mqLen/cidLen + 1 + } else { + averageSize = mqLen / cidLen + } + } + + var startIndex int + if mod > 0 && i < mod { + startIndex = i * averageSize + } else { + startIndex = i*averageSize + mod + } + + var min int + if averageSize > mqLen-startIndex { + min = mqLen - startIndex + } else { + min = averageSize + } + + for j := 0; j < min; j++ { + result = append(result, *mqAll[(startIndex+j)%mqLen]) + } + return result, nil + + } + } + + return nil, errors.New("cant't find currentCID") +} diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go new file mode 100644 index 000000000..cdfd668fb --- /dev/null +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service_allocate_message + +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type AllocateMessageQueueAveragelyByCircle struct{} + +func (self *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { + if currentCID == "" { + return nil, errors.New("currentCID is empty") + } + + if mqAll == nil || len(mqAll) == 0 { + return nil, errors.New("mqAll is nil or mqAll empty") + } + + if cidAll == nil || len(cidAll) == 0 { + return nil, errors.New("cidAll is nil or cidAll empty") + } + + result := make([]model.MessageQueue, 0) + for i, cid := range cidAll { + if cid == currentCID { + mqLen := len(mqAll) + cidLen := len(cidAll) + mod := mqLen % cidLen + var averageSize int + if mqLen < cidLen { + averageSize = 1 + } else { + if mod > 0 && i < mod { + averageSize = mqLen/cidLen + 1 + } else { + averageSize = mqLen / cidLen + } + } + + var startIndex int + if mod > 0 && i < mod { + startIndex = i * averageSize + } else { + startIndex = i*averageSize + mod + } + + var min int + if averageSize > mqLen-startIndex { + min = mqLen - startIndex + } else { + min = averageSize + } + + for j := 0; j < min; j++ { + result = append(result, *mqAll[(startIndex+j)%mqLen]) + } + return result, nil + + } + } + + return nil, errors.New("cant't find currentCID") +} diff --git a/rocketmq-go/service/offset_store_service.go b/rocketmq-go/service/allocate_message/allocate_message_by_config.go similarity index 67% rename from rocketmq-go/service/offset_store_service.go rename to rocketmq-go/service/allocate_message/allocate_message_by_config.go index 302b41257..2046ffd0e 100644 --- a/rocketmq-go/service/offset_store_service.go +++ b/rocketmq-go/service/allocate_message/allocate_message_by_config.go @@ -14,8 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package service +package service_allocate_message -type OffsetStore struct { - mqClient RocketMqClient +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + +type AllocateMessageQueueByConfig struct { + messageQueueList []model.MessageQueue +} + +func (self *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { + return self.messageQueueList, nil } diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go new file mode 100644 index 000000000..6fe1cbb9f --- /dev/null +++ b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service_allocate_message + +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type AllocateMessageQueueByMachineRoom struct { +} + +func (self *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { + if currentCID == "" { + return nil, errors.New("currentCID is empty") + } + + if mqAll == nil || len(mqAll) == 0 { + return nil, errors.New("mqAll is nil or mqAll empty") + } + + if cidAll == nil || len(cidAll) == 0 { + return nil, errors.New("cidAll is nil or cidAll empty") + } + + result := make([]model.MessageQueue, 0) + for i, cid := range cidAll { + if cid == currentCID { + mqLen := len(mqAll) + cidLen := len(cidAll) + mod := mqLen % cidLen + var averageSize int + if mqLen < cidLen { + averageSize = 1 + } else { + if mod > 0 && i < mod { + averageSize = mqLen/cidLen + 1 + } else { + averageSize = mqLen / cidLen + } + } + + var startIndex int + if mod > 0 && i < mod { + startIndex = i * averageSize + } else { + startIndex = i*averageSize + mod + } + + var min int + if averageSize > mqLen-startIndex { + min = mqLen - startIndex + } else { + min = averageSize + } + + for j := 0; j < min; j++ { + result = append(result, *mqAll[(startIndex+j)%mqLen]) + } + return result, nil + + } + } + + return nil, errors.New("cant't find currentCID") +} diff --git a/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go b/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go new file mode 100644 index 000000000..e838c7bf9 --- /dev/null +++ b/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service_allocate_message + +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + +type AllocateMessageQueueStrategy interface { + Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) +} + +func GetAllocateMessageQueueStrategyByConfig(allocateMessageQueueStrategy string) AllocateMessageQueueStrategy { + return new(AllocateMessageQueueAveragely) +} diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index 09be61c20..e4a08df40 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -35,22 +35,22 @@ type ConsumeMessageService interface { } type ConsumeMessageConcurrentlyServiceImpl struct { - consumerGroup string - messageListener model.MessageListener - //sendMessageBackProducerService SendMessageBackProducerService //for send retry Message - offsetStore OffsetStore - consumerConfig *config.RocketMqConsumerConfig + consumerGroup string + messageListener model.MessageListener + sendMessageBackProducerService SendMessageBackProducerService //for send retry Message + offsetStore OffsetStore + consumerConfig *config.RocketMqConsumerConfig } func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListener) (consumeService ConsumeMessageService) { - //consumeService = &ConsumeMessageConcurrentlyServiceImpl{messageListener:messageListener, sendMessageBackProducerService:&SendMessageBackProducerServiceImpl{}} + consumeService = &ConsumeMessageConcurrentlyServiceImpl{messageListener: messageListener, sendMessageBackProducerService: &SendMessageBackProducerServiceImpl{}} return } func (self *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) { self.consumerGroup = consumerGroup self.offsetStore = offsetStore - //self.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient,defaultProducerService,consumerConfig) + self.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) self.consumerConfig = consumerConfig } @@ -74,7 +74,7 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []m } func (self *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) { - //err = self.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) + err = self.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } @@ -128,10 +128,10 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result m if len(failedMessages) > 0 { self.SubmitConsumeRequest(failedMessages, processQueue, messageQueue, true) } - //commitOffset := processQueue.RemoveMessage(successMessages) - //if (commitOffset > 0 && ! processQueue.IsDropped()) { - // self.offsetStore.UpdateOffset(messageQueue, commitOffset, true) - //} + commitOffset := processQueue.RemoveMessage(successMessages) + if commitOffset > 0 && !processQueue.IsDropped() { + self.offsetStore.UpdateOffset(messageQueue, commitOffset, true) + } } diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 8bbfe79ca..366aa3fcb 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -59,10 +59,9 @@ type RocketMqClient interface { var DEFAULT_TIMEOUT int64 = 6000 -// common type MqClientImpl struct { ClientId string - remotingClient remoting.RemotingClient + remotingClient *remoting.DefalutRemotingClient TopicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData BrokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address TopicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this @@ -134,7 +133,7 @@ func (self *MqClientImpl) GetPublishTopicList() []string { } return publishTopicList } -func (self *MqClientImpl) GetRemotingClient() remoting.RemotingClient { +func (self *MqClientImpl) GetRemotingClient() *remoting.DefalutRemotingClient { return self.remotingClient } @@ -147,7 +146,7 @@ func (self *MqClientImpl) DequeuePullMessageRequest() (pullRequest *model.PullRe } func (self *MqClientImpl) ClearExpireResponse() { - //self.remotingClient.ClearExpireResponse() + self.remotingClient.ClearExpireResponse() } func (self *MqClientImpl) FetchMasterBrokerAddress(brokerName string) (masterAddress string) { @@ -199,10 +198,9 @@ func (self MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMi return nil, err } if response.Code == remoting.SUCCESS { - //todo it's dirty topicRouteData := new(model.TopicRouteData) bodyjson := strings.Replace(string(response.Body), ",0:", ",\"0\":", -1) - bodyjson = strings.Replace(bodyjson, ",1:", ",\"1\":", -1) // fastJson的key没有引号 需要通用的方法 + bodyjson = strings.Replace(bodyjson, ",1:", ",\"1\":", -1) // fastJson key is string todo todo bodyjson = strings.Replace(bodyjson, "{0:", "{\"0\":", -1) bodyjson = strings.Replace(bodyjson, "{1:", "{\"1\":", -1) err = json.Unmarshal([]byte(bodyjson), topicRouteData) @@ -291,7 +289,7 @@ func (self MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData //update pubInfo for each topicPublishInfo := model.BuildTopicPublishInfoFromTopicRoteData(topic, topicRouteData) - self.TopicPublishInfoTable.Set(topic, topicPublishInfo) + self.TopicPublishInfoTable.Set(topic, topicPublishInfo) // todo mqList := model.BuildTopicSubscribeInfoFromRoteData(topic, topicRouteData) self.TopicSubscribeInfoTable.Set(topic, mqList) diff --git a/rocketmq-go/service/mq_fault_strategy.go b/rocketmq-go/service/mq_fault_strategy.go new file mode 100644 index 000000000..852ab98a7 --- /dev/null +++ b/rocketmq-go/service/mq_fault_strategy.go @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service + +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type MQFaultStrategy struct { +} + +//if first select : random one +//if has error broker before ,skip the err broker +func selectOneMessageQueue(topicPublishInfo *model.TopicPublishInfo, lastFailedBroker string) (mqQueue model.MessageQueue, err error) { + queueIndex := topicPublishInfo.FetchQueueIndex() + queues := topicPublishInfo.MessageQueueList + if len(lastFailedBroker) == 0 { + mqQueue = queues[queueIndex] + return + } + for i := 0; i < len(queues); i++ { + nowQueueIndex := queueIndex + i + if nowQueueIndex >= len(queues) { + nowQueueIndex = nowQueueIndex - len(queues) + } + if lastFailedBroker == queues[nowQueueIndex].BrokerName { + continue + } + mqQueue = queues[nowQueueIndex] + return + } + err = errors.New("send to [" + lastFailedBroker + "] fail,no other broker") + return +} diff --git a/rocketmq-go/service/offset_store.go b/rocketmq-go/service/offset_store.go new file mode 100644 index 000000000..0bfe64067 --- /dev/null +++ b/rocketmq-go/service/offset_store.go @@ -0,0 +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. + */ +package service + +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/golang/glog" + "strconv" + "sync" +) + +const ( + MEMORY_FIRST_THEN_STORE = 0 + READ_FROM_MEMORY = 1 + READ_FROM_STORE = 2 +) + +type OffsetStore interface { + UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) + ReadOffset(mq *model.MessageQueue, readType int) int64 + Persist(mq *model.MessageQueue) + RemoveOffset(mq *model.MessageQueue) +} +type RemoteOffsetStore struct { + groupName string + mqClient RocketMqClient + offsetTable map[model.MessageQueue]int64 + offsetTableLock sync.RWMutex +} + +func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStore { + offsetStore := new(RemoteOffsetStore) + offsetStore.groupName = groupName + offsetStore.mqClient = mqClient + offsetStore.offsetTable = make(map[model.MessageQueue]int64) + return offsetStore +} +func (self *RemoteOffsetStore) RemoveOffset(mq *model.MessageQueue) { + defer self.offsetTableLock.Unlock() + self.offsetTableLock.Lock() + delete(self.offsetTable, *mq) +} + +func (self *RemoteOffsetStore) Persist(mq *model.MessageQueue) { + brokerAddr := self.mqClient.FetchMasterBrokerAddress(mq.BrokerName) + if len(brokerAddr) == 0 { + self.mqClient.TryToFindTopicPublishInfo(mq.Topic) + brokerAddr = self.mqClient.FetchMasterBrokerAddress(mq.BrokerName) + } + self.offsetTableLock.RLock() + offset := self.offsetTable[*mq] + self.offsetTableLock.RUnlock() + updateConsumerOffsetRequestHeader := &header.UpdateConsumerOffsetRequestHeader{ConsumerGroup: self.groupName, Topic: mq.Topic, QueueId: mq.QueueId, CommitOffset: offset} + requestCommand := remoting.NewRemotingCommand(remoting.UPDATE_CONSUMER_OFFSET, updateConsumerOffsetRequestHeader) + self.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) +} + +func (self *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) int64 { + + switch readType { + case MEMORY_FIRST_THEN_STORE: + case READ_FROM_MEMORY: + self.offsetTableLock.RLock() + offset, ok := self.offsetTable[*mq] + self.offsetTableLock.RUnlock() + if ok { + return offset + } else { + return -1 + } + case READ_FROM_STORE: + offset, err := self.fetchConsumeOffsetFromBroker(mq) + if err != nil { + glog.Error(err) + return -1 + } + glog.V(2).Info("READ_FROM_STORE", offset) + self.UpdateOffset(mq, offset, false) + return offset + } + + return -1 + +} + +func (self *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *model.MessageQueue) (int64, error) { + brokerAddr, _, found := self.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) + + if !found { + brokerAddr, _, found = self.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) + } + + if found { + requestHeader := &header.QueryConsumerOffsetRequestHeader{} + requestHeader.Topic = mq.Topic + requestHeader.QueueId = mq.QueueId + requestHeader.ConsumerGroup = self.groupName + return self.queryConsumerOffset(brokerAddr, requestHeader, 3000) + } + + return -1, errors.New("fetch consumer offset error") +} + +func (self RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *header.QueryConsumerOffsetRequestHeader, timeoutMillis int64) (int64, error) { + remotingCommand := remoting.NewRemotingCommand(remoting.QUERY_CONSUMER_OFFSET, requestHeader) + reponse, err := self.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) + if err != nil { + glog.Error(err) + return -1, err + } + + if reponse.Code == remoting.QUERY_NOT_FOUND { + return -1, nil + } + + if offsetInter, ok := reponse.ExtFields["offset"]; ok { + if offsetStr, ok := offsetInter.(string); ok { + offset, err := strconv.ParseInt(offsetStr, 10, 64) + if err != nil { + glog.Error(err) + return -1, err + } + return offset, nil + + } + } + glog.Error(requestHeader, reponse) + return -1, errors.New("query offset error") +} + +func (self *RemoteOffsetStore) UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { + defer self.offsetTableLock.Unlock() + self.offsetTableLock.Lock() + if mq != nil { + if increaseOnly { + offsetOld := self.offsetTable[*mq] + if offsetOld >= offset { + return + } + self.offsetTable[*mq] = offset + } else { + self.offsetTable[*mq] = offset + } + + } +} diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/service/producer_service.go index a684b2767..2f2a7b685 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/service/producer_service.go @@ -16,13 +16,227 @@ */ package service -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" +import ( + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" + "time" +) type ProducerService interface { + CheckConfig() (err error) + SendDefaultImpl(message *model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } type DefaultProducerService struct { - producerGroup string - producerConfig *config.RocketMqProducerConfig - mqClient RocketMqClient + producerGroup string + producerConfig *config.RocketMqProducerConfig + mqClient RocketMqClient + mqFaultStrategy MQFaultStrategy +} + +func NewDefaultProducerService(producerGroup string, producerConfig *config.RocketMqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { + defaultProducerService = &DefaultProducerService{ + mqClient: mqClient, + producerGroup: producerGroup, + producerConfig: producerConfig, + } + defaultProducerService.CheckConfig() + return +} +func (self *DefaultProducerService) CheckConfig() (err error) { + // todo check if not pass panic + return +} + +func (self *DefaultProducerService) SendDefaultImpl(message *model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { + var ( + topicPublishInfo *model.TopicPublishInfo + ) + err = self.checkMessage(message) + if err != nil { + return + } + topicPublishInfo, err = self.mqClient.TryToFindTopicPublishInfo(message.Topic) + if err != nil { + return + } + if topicPublishInfo.JudgeTopicPublishInfoOk() == false { + err = errors.New("topicPublishInfo is error,topic=" + message.Topic) + return + } + glog.V(2).Info("op=look topicPublishInfo", topicPublishInfo) + //if(!ok) return error + sendResult, err = self.sendMsgUseTopicPublishInfo(message, communicationMode, sendCallback, topicPublishInfo, timeout) + return +} + +func (self *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *model.Message, timeout int64) (sendResult *model.SendResult, err error) { + remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) + var response *remoting.RemotingCommand + response, err = self.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) + if err != nil { + glog.Error(err) + return + } + sendResult, err = processSendResponse(brokerAddr, message, response) + return +} +func processSendResponse(brokerName string, message *model.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { + sendResult = &model.SendResult{} + switch response.Code { + case remoting.FLUSH_DISK_TIMEOUT: + { + sendResult.SetSendStatus(model.FlushDiskTimeout) + break + } + case remoting.FLUSH_SLAVE_TIMEOUT: + { + sendResult.SetSendStatus(model.FlushSlaveTimeout) + break + } + case remoting.SLAVE_NOT_AVAILABLE: + { + sendResult.SetSendStatus(model.SlaveNotAvaliable) + break + } + case remoting.SUCCESS: + { + sendResult.SetSendStatus(model.SendOK) + break + } + default: + err = errors.New("response.Code error code") + return + } + var responseHeader = &header.SendMessageResponseHeader{} + if response.ExtFields != nil { + responseHeader.FromMap(response.ExtFields) //change map[string]interface{} into CustomerHeader struct + } + sendResult.SetMsgID(message.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) + sendResult.SetOffsetMsgID(responseHeader.MsgId) + sendResult.SetQueueOffset(responseHeader.QueueOffset) + sendResult.SetTransactionID(responseHeader.TransactionId) + messageQueue := model.MessageQueue{Topic: message.Topic, BrokerName: brokerName, + QueueId: responseHeader.QueueId} + sendResult.SetMessageQueue(messageQueue) + var regionId = responseHeader.MsgRegion + if len(regionId) == 0 { + regionId = "DefaultRegion" + } + sendResult.SetRegionID(regionId) + return +} + +func (self *DefaultProducerService) checkMessage(message *model.Message) (err error) { + if message == nil { + err = errors.New("message is nil") + return + } + if len(message.Topic) == 0 { + err = errors.New("topic is empty") + return + } + if message.Topic == constant.DEFAULT_TOPIC { + err = errors.New("the topic[" + message.Topic + "] is conflict with default topic.") + return + } + + if len(message.Topic) > constant.MAX_MESSAGE_TOPIC_SIZE { + err = errors.New("the specified topic is longer than topic max length 255.") + return + } + //todo todo public static final String VALID_PATTERN_STR = ""; + + if !util.MatchString(message.Topic, `^[%|a-zA-Z0-9_-]+$`) { + err = errors.New("the specified topic[" + message.Topic + "] contains illegal characters") + return + } + if len(message.Body) == 0 { + err = errors.New("messageBody is empty") + return + } + if len(message.Body) > self.producerConfig.MaxMessageSize { + err = errors.New("messageBody is large than " + util.IntToString(self.producerConfig.MaxMessageSize)) + return + } + return +} + +func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { + var ( + sendTotalTime int + messageQueue model.MessageQueue + ) + + sendTotalTime = 1 + var lastFailedBroker = "" + //todo transaction + // todo retry + for i := 0; i < sendTotalTime; i++ { + messageQueue, err = selectOneMessageQueue(topicPublishInfo, lastFailedBroker) + if err != nil { + return + } + sendResult, err = self.doSendMessage(message, messageQueue, communicationMode, sendCallback, topicPublishInfo, timeout) + if err != nil { + // todo retry + return + } + } + return +} + +func (self *DefaultProducerService) doSendMessage(message *model.Message, messageQueue model.MessageQueue, + communicationMode string, sendCallback string, + topicPublishInfo *model.TopicPublishInfo, + timeout int64) (sendResult *model.SendResult, err error) { + var ( + brokerAddr string + sysFlag int + compressMessageFlag int + ) + compressMessageFlag, err = self.tryToCompressMessage(message) + if err != nil { + return + } + sysFlag = sysFlag | compressMessageFlag + brokerAddr = self.mqClient.FetchMasterBrokerAddress(messageQueue.BrokerName) + if len(brokerAddr) == 0 { + err = errors.New("The broker[" + messageQueue.BrokerName + "] not exist") + return + } + message.GeneratorMsgUniqueKey() + sendMessageHeader := &header.SendMessageRequestHeader{ + ProducerGroup: self.producerGroup, + Topic: message.Topic, + DefaultTopic: constant.DEFAULT_TOPIC, + DefaultTopicQueueNums: 4, + QueueId: messageQueue.QueueId, + SysFlag: sysFlag, + BornTimestamp: time.Now().UnixNano() / 1000000, + Flag: message.Flag, + Properties: util.MessageProperties2String(message.Properties), + + UnitMode: false, + ReconsumeTimes: message.GetReconsumeTimes(), + MaxReconsumeTimes: message.GetMaxReconsumeTimes(), + } + sendResult, err = self.producerSendMessageRequest(brokerAddr, sendMessageHeader, message, timeout) + return +} + +func (self *DefaultProducerService) tryToCompressMessage(message *model.Message) (compressedFlag int, err error) { + if len(message.Body) < self.producerConfig.CompressMsgBodyOverHowMuch { + compressedFlag = 0 + return + } + compressedFlag = int(constant.CompressedFlag) + message.Body, err = util.CompressWithLevel(message.Body, self.producerConfig.ZipCompressLevel) + return } diff --git a/rocketmq-go/service/producer_service_for_send_back.go b/rocketmq-go/service/producer_service_for_send_back.go new file mode 100644 index 000000000..290da274c --- /dev/null +++ b/rocketmq-go/service/producer_service_for_send_back.go @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service + +import ( + "encoding/json" + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/golang/glog" +) + +type SendMessageBackProducerService interface { + SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) + InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) +} + +type SendMessageBackProducerServiceImpl struct { + mqClient RocketMqClient + defaultProducerService *DefaultProducerService // one namesvr only one + consumerGroup string + consumerConfig *config.RocketMqConsumerConfig //one mq group have one +} + +// send to original broker,if fail send a new retry message +func (self *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) { + glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) + err = self.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) // todo use + if err == nil { + return + } + glog.Error(err) + err = self.sendRetryMessageBack(messageExt) + return +} + +func (self *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *model.MessageExt) error { + // todo build a retry topic todo check todo check + retryMessage := &model.Message{} + originMessageId := messageExt.GetOriginMessageId() + retryMessage.Properties = messageExt.Properties + retryMessage.SetOriginMessageId(originMessageId) + retryMessage.Flag = messageExt.Flag + retryMessage.Topic = constant.RETRY_GROUP_TOPIC_PREFIX + self.consumerGroup + retryMessage.Body = messageExt.Body + retryMessage.SetRetryTopic(messageExt.Topic) + retryMessage.SetReconsumeTime(messageExt.GetReconsumeTimes() + 1) + retryMessage.SetMaxReconsumeTimes(self.consumerConfig.MaxReconsumeTimes) + retryMessage.SetDelayTimeLevel(3 + messageExt.GetReconsumeTimes()) + pp, _ := json.Marshal(retryMessage) + glog.Info("look retryMessage ", string(pp), string(messageExt.Body)) + sendResult, err := self.defaultProducerService.SendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", self.defaultProducerService.producerConfig.SendMsgTimeout) + if err != nil { + glog.Error(err) + return err + } + xx, _ := json.Marshal(sendResult) + glog.Info("look retryMessage result", string(xx)) + // todo need check send result + return nil + +} + +func (self *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) { + self.mqClient = mqClient + self.consumerGroup = consumerGroup + self.defaultProducerService = defaultProducerService + self.consumerConfig = consumerConfig +} + +func (self *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *model.MessageExt, delayLayLevel int) (err error) { + if len(brokerName) == 0 { + err = errors.New("broker can't be empty") + glog.Error(err) + return + } + brokerAddr := self.mqClient.FetchMasterBrokerAddress(brokerName) + sendMsgBackHeader := &header.ConsumerSendMsgBackRequestHeader{ + Offset: messageExt.CommitLogOffset, + Group: self.consumerGroup, + DelayLevel: 0, //Message consume retry strategy
-1,no retry,put into DLQ directly
0,broker control retry frequency
>0,client control retry frequency + OriginMsgId: messageExt.MsgId, + OriginTopic: messageExt.Topic, + UnitMode: false, + MaxReconsumeTimes: int32(self.consumerConfig.MaxReconsumeTimes), + } + remotingCommand := remoting.NewRemotingCommand(remoting.CONSUMER_SEND_MSG_BACK, sendMsgBackHeader) + response, invokeErr := self.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, 5000) + if invokeErr != nil { + err = invokeErr + return + } + if response == nil || response.Code != remoting.SUCCESS { + glog.Error("sendMsgBackRemarkError", response.Remark) + err = errors.New("send Message back error") + } + return +} diff --git a/rocketmq-go/service/rebalance.go b/rocketmq-go/service/rebalance.go new file mode 100644 index 000000000..8f4f4fbef --- /dev/null +++ b/rocketmq-go/service/rebalance.go @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package service + +import ( + "encoding/json" + "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service/allocate_message" + "github.com/golang/glog" + "sort" + "strings" + "sync" + "time" +) + +type Rebalance struct { + groupName string + messageModel string + topicSubscribeInfoTableLock sync.RWMutex + SubscriptionInner map[string]*model.SubscriptionData + subscriptionInnerLock sync.RWMutex + mqClient RocketMqClient + allocateMessageQueueStrategy service_allocate_message.AllocateMessageQueueStrategy + processQueueTable map[model.MessageQueue]*model.ProcessQueue // both subscribe topic and retry group + processQueueTableLock sync.RWMutex + mutex sync.Mutex + offsetStore OffsetStore + consumerConfig *config.RocketMqConsumerConfig +} + +func (self *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { + defer self.processQueueTableLock.RUnlock() + self.processQueueTableLock.RLock() + mqTable := map[model.MessageQueue]model.ProcessQueueInfo{} + for messageQueue, processQueue := range self.processQueueTable { + mqTable[messageQueue] = processQueue.ChangeToProcessQueueInfo() + } + return mqTable +} + +func (self *Rebalance) GetProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { + defer self.processQueueTableLock.RUnlock() + self.processQueueTableLock.RLock() + return self.processQueueTable[messageQueue] +} + +func (self *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int64) { + defer self.processQueueTableLock.Unlock() + self.processQueueTableLock.Lock() + for mq, _ := range offsetTable { + processQueue, ok := self.processQueueTable[mq] + if !ok { + continue + } + processQueue.Clear() + } + +} + +func (self *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { + defer self.processQueueTableLock.RUnlock() + self.processQueueTableLock.RLock() + for messageQueue, processQueue := range self.processQueueTable { + processQueueList = append(processQueueList, processQueue) + messageQueueList = append(messageQueueList, messageQueue) + } + return +} + +//removeUnnecessaryMessageQueue you should drop it first +func (self *Rebalance) RemoveProcessQueue(messageQueue *model.MessageQueue) { + self.offsetStore.Persist(messageQueue) + self.offsetStore.RemoveOffset(messageQueue) + self.removeMessageQueueFromMap(*messageQueue) +} +func (self *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { + defer self.processQueueTableLock.Unlock() + self.processQueueTableLock.Lock() + delete(self.processQueueTable, messageQueue) + +} + +func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *config.RocketMqConsumerConfig) *Rebalance { + subscriptionInner := make(map[string]*model.SubscriptionData) + for topic, subExpression := range subscription { + subData := &model.SubscriptionData{ + Topic: topic, + SubString: subExpression, + SubVersion: time.Now().Unix(), + } + subscriptionInner[topic] = subData + } + // put retry + retryTopic := constant.RETRY_GROUP_TOPIC_PREFIX + groupName + subscriptionInner[retryTopic] = &model.SubscriptionData{ + Topic: retryTopic, + SubString: "*", + SubVersion: time.Now().Unix(), + } + return &Rebalance{ + groupName: groupName, + mqClient: mqClient, + offsetStore: offsetStore, + SubscriptionInner: subscriptionInner, + allocateMessageQueueStrategy: service_allocate_message.GetAllocateMessageQueueStrategyByConfig("default"), + messageModel: "CLUSTERING", + processQueueTable: make(map[model.MessageQueue]*model.ProcessQueue), + consumerConfig: consumerConfig, + } +} + +func (self *Rebalance) DoRebalance() { + self.mutex.Lock() + defer self.mutex.Unlock() + for topic, _ := range self.SubscriptionInner { + self.rebalanceByTopic(topic) + } +} + +type ConsumerIdSorter []string + +func (self ConsumerIdSorter) Len() int { + return len(self) +} +func (self ConsumerIdSorter) Swap(i, j int) { + self[i], self[j] = self[j], self[i] +} +func (self ConsumerIdSorter) Less(i, j int) bool { + if self[i] < self[j] { + return true + } + return false +} + +func (self *Rebalance) rebalanceByTopic(topic string) error { + var cidAll []string + cidAll, err := self.findConsumerIdList(topic, self.groupName) + if err != nil { + glog.Error(err) + return err + } + self.topicSubscribeInfoTableLock.RLock() + mqs := self.mqClient.GetTopicSubscribeInfo(topic) + self.topicSubscribeInfoTableLock.RUnlock() + if len(mqs) > 0 && len(cidAll) > 0 { + var messageQueues model.MessageQueues = mqs + var consumerIdSorter ConsumerIdSorter = cidAll + + sort.Sort(messageQueues) + sort.Sort(consumerIdSorter) + } + allocateResult, err := self.allocateMessageQueueStrategy.Allocate(self.groupName, self.mqClient.GetClientId(), mqs, cidAll) + + if err != nil { + glog.Error(err) + return err + } + + glog.V(2).Infof("rebalance topic[%s]", topic) + self.updateProcessQueueTableInRebalance(topic, allocateResult) + return nil +} + +func (self *Rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { + defer self.processQueueTableLock.RUnlock() + self.processQueueTableLock.RLock() + self.removeTheQueueDontBelongHere(topic, mqSet) + self.putTheQueueToProcessQueueTable(topic, mqSet) + +} +func (self *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { + // there is n^2 todo improve + for key, value := range self.processQueueTable { + if topic != key.Topic { + continue + } + needDelete := true + for _, messageQueueItem := range mqSet { + if key == messageQueueItem { + needDelete = false + // todo if expire + break + } + } + if needDelete { + value.SetDrop(true) + delete(self.processQueueTable, key) + } + } +} + +func (self *Rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { + for index, mq := range mqSet { + _, ok := self.processQueueTable[mq] + if !ok { + pullRequest := new(model.PullRequest) + pullRequest.ConsumerGroup = self.groupName + pullRequest.MessageQueue = &mqSet[index] + pullRequest.NextOffset = self.computePullFromWhere(&mq) // todo use remote offset + pullRequest.ProcessQueue = model.NewProcessQueue() + self.processQueueTable[mq] = pullRequest.ProcessQueue + self.mqClient.EnqueuePullMessageRequest(pullRequest) + } + } + +} +func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { + var result int64 = -1 + lastOffset := self.offsetStore.ReadOffset(mq, READ_FROM_STORE) + switch self.consumerConfig.ConsumeFromWhere { + case config.CONSUME_FROM_LAST_OFFSET: + if lastOffset >= 0 { + result = lastOffset + } else { + if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { + result = 0 + } else { + result = self.mqClient.GetMaxOffset(mq) + } + } + break + case config.CONSUME_FROM_FIRST_OFFSET: + if lastOffset >= 0 { + result = lastOffset + } else { + result = 0 // use the begin offset + } + break + case config.CONSUME_FROM_TIMESTAMP: + if lastOffset >= 0 { + result = lastOffset + } else { + if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { + result = 0 + } else { + result = self.mqClient.SearchOffset(mq, self.consumerConfig.ConsumeTimestamp) + } + } + break + default: + + } + + return result +} + +func (self *Rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { + brokerAddr, ok := self.mqClient.FindBrokerAddrByTopic(topic) + if !ok { + err := self.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + if err != nil { + glog.Error(err) + } + brokerAddr, ok = self.mqClient.FindBrokerAddrByTopic(topic) + } + + if ok { + return self.getConsumerIdListByGroup(brokerAddr, groupName, 3000) + } + + return nil, errors.New("can't find broker") + +} + +func (self *Rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, timeoutMillis int64) ([]string, error) { + requestHeader := new(header.GetConsumerListByGroupRequestHeader) + requestHeader.ConsumerGroup = consumerGroup + + request := remoting.NewRemotingCommand(remoting.GET_CONSUMER_LIST_BY_GROUP, requestHeader) + + response, err := self.mqClient.GetRemotingClient().InvokeSync(addr, request, timeoutMillis) + if err != nil { + glog.Error(err) + return nil, err + } + if response.Code == remoting.SUCCESS { + getConsumerListByGroupResponseBody := new(header.GetConsumerListByGroupResponseBody) + bodyjson := strings.Replace(string(response.Body), "0:", "\"0\":", -1) + bodyjson = strings.Replace(bodyjson, "1:", "\"1\":", -1) + err := json.Unmarshal([]byte(bodyjson), getConsumerListByGroupResponseBody) + if err != nil { + glog.Error(err) + return nil, err + } + return getConsumerListByGroupResponseBody.ConsumerIdList, nil + } + return nil, errors.New("getConsumerIdListByGroup error=" + response.Remark) +} diff --git a/rocketmq-go/tasks.go b/rocketmq-go/tasks.go new file mode 100644 index 000000000..604222f21 --- /dev/null +++ b/rocketmq-go/tasks.go @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package rocketmq + +import ( + "math/rand" + "time" +) + +type TaskManager struct { +} + +func (self MqClientManager) StartAllScheduledTask() { + rand.Seed(time.Now().UnixNano()) + go func() { + updateTopicRouteTimer := time.NewTimer(5 * time.Second) + for { + <-updateTopicRouteTimer.C + self.UpdateTopicRouteInfoFromNameServer() + updateTopicRouteTimer.Reset(5 * time.Second) + } + }() + + go func() { + heartbeatTimer := time.NewTimer(10 * time.Second) + for { + <-heartbeatTimer.C + self.SendHeartbeatToAllBrokerWithLock() + heartbeatTimer.Reset(5 * time.Second) + } + }() + + go func() { + rebalanceTimer := time.NewTimer(15 * time.Second) + for { + <-rebalanceTimer.C + self.rebalanceControllr.doRebalance() + rebalanceTimer.Reset(30 * time.Second) + } + }() + + go func() { + timeoutTimer := time.NewTimer(3 * time.Second) + for { + <-timeoutTimer.C + self.mqClient.ClearExpireResponse() + timeoutTimer.Reset(time.Second) + } + }() + self.pullMessageController.Start() + + //cleanExpireMsg + self.cleanExpireMsgController.Start() +} diff --git a/rocketmq-go/util/compress_util.go b/rocketmq-go/util/compress_util.go new file mode 100644 index 000000000..061791167 --- /dev/null +++ b/rocketmq-go/util/compress_util.go @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "bytes" + "compress/zlib" + "github.com/golang/glog" + "io/ioutil" +) + +func UnCompress(body []byte) (unCompressBody []byte, err error) { + b := bytes.NewReader(body) + z, err := zlib.NewReader(b) + if err != nil { + glog.Error(err) + return + } + defer z.Close() + unCompressBody, err = ioutil.ReadAll(z) + if err != nil { + glog.Error(err) + } + return +} +func Compress(body []byte) (compressBody []byte, err error) { + var in bytes.Buffer + w := zlib.NewWriter(&in) + _, err = w.Write(body) + w.Close() + compressBody = in.Bytes() + return +} + +func CompressWithLevel(body []byte, level int) (compressBody []byte, err error) { + var ( + in bytes.Buffer + w *zlib.Writer + ) + //w := zlib.NewWriter(&in) + w, err = zlib.NewWriterLevel(&in, level) + if err != nil { + return + } + _, err = w.Write(body) + w.Close() + compressBody = in.Bytes() + return +} diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go index 2fbe9bf64..9d3e27360 100644 --- a/rocketmq-go/util/concurrent_map.go +++ b/rocketmq-go/util/concurrent_map.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package util import ( diff --git a/rocketmq-go/util/message_client_id_generator.go b/rocketmq-go/util/message_client_id_generator.go index df4cfb68f..23293c0dd 100644 --- a/rocketmq-go/util/message_client_id_generator.go +++ b/rocketmq-go/util/message_client_id_generator.go @@ -88,13 +88,13 @@ func getStartAndNextStartTime() (thisMonthFirstDay12 int64, nextMonthFirstDay12 now := time.Now() year := now.Year() month := now.Month() - thisMonthFirstDay12 = time.Date(year, month, 1, 12, 0, 0, 0, time.Local).UnixNano() + thisMonthFirstDay12 = time.Date(year, month, 1, 0, 0, 0, 0, time.Local).UnixNano() month = month + 1 if month > 12 { month = month - 12 year = year + 1 } - nextMonthFirstDay12 = time.Date(year, month, 1, 12, 0, 0, 0, time.Local).UnixNano() + nextMonthFirstDay12 = time.Date(year, month, 1, 0, 0, 0, 0, time.Local).UnixNano() return } func bytes2string(bytes []byte) (ret string) { diff --git a/rocketmq-go/util/message_properties.go b/rocketmq-go/util/message_properties.go new file mode 100644 index 000000000..59fd5b8cf --- /dev/null +++ b/rocketmq-go/util/message_properties.go @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "strings" +) + +//char 1 and 2 from java code +var NAME_VALUE_SEPARATOR = string(rune(1)) + +var PROPERTY_SEPARATOR = string(rune(2)) + +func MessageProperties2String(propertiesMap map[string]string) (ret string) { + for key, value := range propertiesMap { + ret = ret + key + NAME_VALUE_SEPARATOR + value + PROPERTY_SEPARATOR + } + return +} + +func String2MessageProperties(properties string) (ret map[string]string) { + ret = make(map[string]string) + for _, nameValueStr := range strings.Split(properties, PROPERTY_SEPARATOR) { + nameValuePair := strings.Split(nameValueStr, NAME_VALUE_SEPARATOR) + nameValueLen := len(nameValuePair) + if nameValueLen != 2 { + //glog.Error("nameValuePair is error", nameValueStr) + continue + } + ret[nameValuePair[0]] = nameValuePair[1] + } + return +} diff --git a/rocketmq-go/service/rebalance_service.go b/rocketmq-go/util/regex_util.go similarity index 75% rename from rocketmq-go/service/rebalance_service.go rename to rocketmq-go/util/regex_util.go index acdcdd663..53574524b 100644 --- a/rocketmq-go/service/rebalance_service.go +++ b/rocketmq-go/util/regex_util.go @@ -14,12 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package service +package util -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" +import ( + "regexp" +) -type Rebalance struct { - mqClient RocketMqClient - offsetStore OffsetStore - consumerConfig config.RocketMqClientConfig +//var regexpMap map[string]*regexp.Regexp +//var rwMutex sync.RWMutex + +// todo improve +func MatchString(value, pattern string) bool { + re, err := regexp.Compile(pattern) + if err != nil { + return false + } + return re.MatchString(value) } From 2ebd1a02fc20089856e0157339d791c8c195c383 Mon Sep 17 00:00:00 2001 From: tangjie Date: Wed, 9 Aug 2017 21:45:20 +0800 Subject: [PATCH 02/88] use localhost test --- rocketmq-go/example/producer_consumer_example.go | 2 +- rocketmq-go/example/rocketmq_client_example.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go index 0d8e45586..4b78839a1 100644 --- a/rocketmq-go/example/producer_consumer_example.go +++ b/rocketmq-go/example/producer_consumer_example.go @@ -52,7 +52,7 @@ func main() { return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} }) var clienConfig = &config.ClientConfig{} - clienConfig.SetNameServerAddress("120.55.113.35:9876") + clienConfig.SetNameServerAddress("127.0.0.1:9876") rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) rocketMqManager.RegistProducer(producer1) rocketMqManager.RegistProducer(producer2) diff --git a/rocketmq-go/example/rocketmq_client_example.go b/rocketmq-go/example/rocketmq_client_example.go index c6828c8a7..5e922bf2b 100644 --- a/rocketmq-go/example/rocketmq_client_example.go +++ b/rocketmq-go/example/rocketmq_client_example.go @@ -26,7 +26,7 @@ import ( func main() { var clienConfig = config.NewClientConfig() - clienConfig.SetNameServerAddress("120.55.113.35:9876") + clienConfig.SetNameServerAddress("127.0.0.1:9876") //use json serializer var mqClient = service.MqClientInit(clienConfig, nil) From 5ae0244ded1934860f2ed4906519bacbcccfe76e Mon Sep 17 00:00:00 2001 From: tangjie Date: Wed, 9 Aug 2017 23:22:15 +0800 Subject: [PATCH 03/88] user only use "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" --- .../config => api/model}/client_config.go | 2 +- .../model/consume_concurrently_result.go | 2 +- .../config => api/model}/consumer_config.go | 2 +- rocketmq-go/{ => api}/model/message.go | 2 +- rocketmq-go/{ => api}/model/message_ext.go | 2 +- .../config => api/model}/producer_config.go | 2 +- rocketmq-go/api/rocketmq_consumer.go | 15 +++++ rocketmq-go/api/rocketmq_manage.go | 31 ++++++++++ rocketmq-go/api/rocketmq_producer.go | 16 +++++ rocketmq-go/example/consumer_example.go | 52 ++++++++--------- .../example/producer_consumer_example.go | 34 +++++------ rocketmq-go/example/producer_example.go | 58 +++++++++---------- .../example/rocketmq_client_example.go | 2 +- .../clean_expire_msg_controller.go | 0 rocketmq-go/{ => manage}/mq_client_manage.go | 6 +- rocketmq-go/manage/mq_producer.go | 33 +++++++++++ rocketmq-go/{ => manage}/mq_push_consumer.go | 20 ++++--- .../{ => manage}/pull_message_controller.go | 11 ++-- .../{ => manage}/rebalance_controller.go | 0 rocketmq-go/{ => manage}/tasks.go | 0 rocketmq-go/model/message_listener.go | 4 +- rocketmq-go/model/process_queue.go | 9 +-- rocketmq-go/mq_producer.go | 40 ------------- rocketmq-go/remoting/remoting_client.go | 6 +- rocketmq-go/service/client_api.go | 6 +- .../service/consume_message_service.go | 32 +++++----- rocketmq-go/service/mq_client.go | 4 +- rocketmq-go/service/producer_service.go | 22 +++---- .../service/producer_service_for_send_back.go | 19 +++--- rocketmq-go/service/rebalance.go | 12 ++-- 30 files changed, 250 insertions(+), 194 deletions(-) rename rocketmq-go/{model/config => api/model}/client_config.go (99%) rename rocketmq-go/{ => api}/model/consume_concurrently_result.go (97%) rename rocketmq-go/{model/config => api/model}/consumer_config.go (99%) rename rocketmq-go/{ => api}/model/message.go (99%) rename rocketmq-go/{ => api}/model/message_ext.go (98%) rename rocketmq-go/{model/config => api/model}/producer_config.go (98%) create mode 100644 rocketmq-go/api/rocketmq_consumer.go create mode 100644 rocketmq-go/api/rocketmq_manage.go create mode 100644 rocketmq-go/api/rocketmq_producer.go rename rocketmq-go/{ => manage}/clean_expire_msg_controller.go (100%) rename rocketmq-go/{ => manage}/mq_client_manage.go (97%) create mode 100644 rocketmq-go/manage/mq_producer.go rename rocketmq-go/{ => manage}/mq_push_consumer.go (90%) rename rocketmq-go/{ => manage}/pull_message_controller.go (96%) rename rocketmq-go/{ => manage}/rebalance_controller.go (100%) rename rocketmq-go/{ => manage}/tasks.go (100%) delete mode 100644 rocketmq-go/mq_producer.go diff --git a/rocketmq-go/model/config/client_config.go b/rocketmq-go/api/model/client_config.go similarity index 99% rename from rocketmq-go/model/config/client_config.go rename to rocketmq-go/api/model/client_config.go index 8a415d858..76f455b9f 100644 --- a/rocketmq-go/model/config/client_config.go +++ b/rocketmq-go/api/model/client_config.go @@ -15,7 +15,7 @@ * limitations under the License. */ -package config +package rocketmq_api_model import ( "bytes" diff --git a/rocketmq-go/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go similarity index 97% rename from rocketmq-go/model/consume_concurrently_result.go rename to rocketmq-go/api/model/consume_concurrently_result.go index 6e4df7bf1..2b7cae092 100644 --- a/rocketmq-go/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package model +package rocketmq_api_model const ( CONSUME_SUCCESS = "CONSUME_SUCCESS" diff --git a/rocketmq-go/model/config/consumer_config.go b/rocketmq-go/api/model/consumer_config.go similarity index 99% rename from rocketmq-go/model/config/consumer_config.go rename to rocketmq-go/api/model/consumer_config.go index b6a6f32ba..8d4dbbbb4 100644 --- a/rocketmq-go/model/config/consumer_config.go +++ b/rocketmq-go/api/model/consumer_config.go @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package config +package rocketmq_api_model import "time" diff --git a/rocketmq-go/model/message.go b/rocketmq-go/api/model/message.go similarity index 99% rename from rocketmq-go/model/message.go rename to rocketmq-go/api/model/message.go index 0cb3d97e9..4e443377d 100644 --- a/rocketmq-go/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package model +package rocketmq_api_model import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" diff --git a/rocketmq-go/model/message_ext.go b/rocketmq-go/api/model/message_ext.go similarity index 98% rename from rocketmq-go/model/message_ext.go rename to rocketmq-go/api/model/message_ext.go index 9a3aacb29..4fd956c62 100644 --- a/rocketmq-go/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package model +package rocketmq_api_model import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" diff --git a/rocketmq-go/model/config/producer_config.go b/rocketmq-go/api/model/producer_config.go similarity index 98% rename from rocketmq-go/model/config/producer_config.go rename to rocketmq-go/api/model/producer_config.go index 3bf484448..a79eae9e9 100644 --- a/rocketmq-go/model/config/producer_config.go +++ b/rocketmq-go/api/model/producer_config.go @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package config +package rocketmq_api_model type RocketMqProducerConfig struct { SendMsgTimeout int64 //done diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go new file mode 100644 index 000000000..1d7857562 --- /dev/null +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -0,0 +1,15 @@ +package rocketmq_api + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type RocketMQConsumer interface { + RegisterMessageListener(listener model.MessageListener) + Subscribe(topic string, subExpression string) +} + +func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { + return rocketmq.NewDefaultMQPushConsumer(producerGroup) +} diff --git a/rocketmq-go/api/rocketmq_manage.go b/rocketmq-go/api/rocketmq_manage.go new file mode 100644 index 000000000..8d0cad422 --- /dev/null +++ b/rocketmq-go/api/rocketmq_manage.go @@ -0,0 +1,31 @@ +package rocketmq_api + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" +) + +type RocketMQController struct { + rocketMqManager *rocketmq.MqClientManager +} + +func InitRocketMQController(clientConfig *rocketmq_api_model.ClientConfig) (rocketMQController *RocketMQController) { + rocketMQController = &RocketMQController{} + rocketMQController.rocketMqManager = rocketmq.MqClientManagerInit(clientConfig) + return + +} +func (self *RocketMQController) RegistProducer(producer RocketMQProducer) { + self.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) +} + +func (self *RocketMQController) RegistConsumer(consumer RocketMQConsumer) { + self.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) +} +func (self *RocketMQController) Start() { + self.rocketMqManager.Start() +} + +func (self *RocketMQController) ShutDown() { + self.rocketMqManager.ShutDown() +} diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go new file mode 100644 index 000000000..1e792ffdb --- /dev/null +++ b/rocketmq-go/api/rocketmq_producer.go @@ -0,0 +1,16 @@ +package rocketmq_api + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +) + +type RocketMQProducer interface { + Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) + SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) +} + +func NewDefaultMQProducer(producerGroup string) (r RocketMQProducer) { + return rocketmq.NewDefaultMQProducer(producerGroup) +} diff --git a/rocketmq-go/example/consumer_example.go b/rocketmq-go/example/consumer_example.go index 7c94e589c..990393894 100644 --- a/rocketmq-go/example/consumer_example.go +++ b/rocketmq-go/example/consumer_example.go @@ -18,36 +18,36 @@ package main import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/golang/glog" "time" ) func main() { - - var ( - testTopic = "GoLang" - ) - var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") - comsumer1.ConsumerConfig.PullInterval = 0 - comsumer1.ConsumerConfig.ConsumeTimeout = 1 - comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 - comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() - comsumer1.Subscribe(testTopic, "*") - comsumer1.RegisterMessageListener(func(msgs []model.MessageExt) model.ConsumeConcurrentlyResult { - for _, msg := range msgs { - glog.Info(msg.BornTimestamp) - } - glog.Info("look message len ", len(msgs)) - return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} - }) - var clienConfig = &config.ClientConfig{} - clienConfig.SetNameServerAddress("120.55.113.35:9876") - rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) - rocketMqManager.RegistConsumer(comsumer1) - rocketMqManager.Start() - select {} - rocketMqManager.ShutDown() + // + //var ( + // testTopic = "GoLang" + //) + //var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + //comsumer1.ConsumerConfig.PullInterval = 0 + //comsumer1.ConsumerConfig.ConsumeTimeout = 1 + //comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 + //comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + //comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() + //comsumer1.Subscribe(testTopic, "*") + //comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) model.ConsumeConcurrentlyResult { + // for _, msg := range msgs { + // glog.Info(msg.BornTimestamp) + // } + // glog.Info("look message len ", len(msgs)) + // return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} + //}) + //var clienConfig = &rocketmq_api_model.ClientConfig{} + //clienConfig.SetNameServerAddress("127.0.0.1:9876") + //rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) + //rocketMqManager.RegistConsumer(comsumer1) + //rocketMqManager.Start() + //select {} + //rocketMqManager.ShutDown() } diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go index 4b78839a1..8508897bf 100644 --- a/rocketmq-go/example/producer_consumer_example.go +++ b/rocketmq-go/example/producer_consumer_example.go @@ -17,14 +17,12 @@ package main import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go" //todo todo I want only import this - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" "net/http" _ "net/http/pprof" - "time" ) func main() { @@ -34,36 +32,32 @@ func main() { var ( testTopic = "GoLang" ) - var producer1 = rocketmq.NewDefaultMQProducer("Test1") - producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 - var producer2 = rocketmq.NewDefaultMQProducer("Test2") - var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") - comsumer1.ConsumerConfig.PullInterval = 0 - comsumer1.ConsumerConfig.ConsumeTimeout = 1 - comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 - comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() + var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") + //producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 + var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") + var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") comsumer1.Subscribe(testTopic, "*") - comsumer1.RegisterMessageListener(func(msgs []model.MessageExt) model.ConsumeConcurrentlyResult { + comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { for _, msg := range msgs { glog.Info(msg.BornTimestamp) } glog.Info("look message len ", len(msgs)) - return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} }) - var clienConfig = &config.ClientConfig{} + var clienConfig = &rocketmq_api_model.ClientConfig{} clienConfig.SetNameServerAddress("127.0.0.1:9876") - rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) + //clienConfig// todo + rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) rocketMqManager.RegistProducer(producer1) rocketMqManager.RegistProducer(producer2) rocketMqManager.RegistConsumer(comsumer1) rocketMqManager.Start() for i := 0; i < 10000000; i++ { - var message = &model.Message{} + var message = &rocketmq_api_model.Message{} message.Topic = testTopic message.SetKeys([]string{"xxx"}) message.SetTag("1122") - message.Body = []byte("hellAXXWord" + util.IntToString(i)) + message.Body = []byte("hellAXXWord") xx, ee := producer1.Send(message) if ee != nil { diff --git a/rocketmq-go/example/producer_example.go b/rocketmq-go/example/producer_example.go index acc201178..f54b3fc52 100644 --- a/rocketmq-go/example/producer_example.go +++ b/rocketmq-go/example/producer_example.go @@ -18,40 +18,40 @@ package main import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" _ "net/http/pprof" ) func main() { - var ( - testTopic = "GoLang" - ) - var producer1 = rocketmq.NewDefaultMQProducer("Test1") - producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 - var producer2 = rocketmq.NewDefaultMQProducer("Test2") - var clienConfig = &config.ClientConfig{} - clienConfig.SetNameServerAddress("120.55.113.35:9876") - rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) - rocketMqManager.RegistProducer(producer1) - rocketMqManager.RegistProducer(producer2) - rocketMqManager.Start() - for i := 0; i < 1000; i++ { - var message = &model.Message{} - message.Topic = testTopic - message.SetKeys([]string{"xxx"}) - message.SetTag("1122") - message.Body = []byte("hellAXXWord" + util.IntToString(i)) - - xx, ee := producer1.Send(message) - if ee != nil { - glog.Error(ee) - continue - } - glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - } - select {} - rocketMqManager.ShutDown() + //var ( + // testTopic = "GoLang" + //) + //var producer1 = rocketmq.NewDefaultMQProducer("Test1") + //producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 + //var producer2 = rocketmq.NewDefaultMQProducer("Test2") + //var clientConfig = &rocketmq_api_model.ClientConfig{} + //clientConfig.SetNameServerAddress("127.0.0.1:9876") + //rocketMqManager := rocketmq.MqClientManagerInit(clientConfig) + //rocketMqManager.RegistProducer(producer1) + //rocketMqManager.RegistProducer(producer2) + //rocketMqManager.Start() + //for i := 0; i < 1000; i++ { + // var message = &model.Message{} + // message.Topic = testTopic + // message.SetKeys([]string{"xxx"}) + // message.SetTag("1122") + // message.Body = []byte("hellAXXWord" + util.IntToString(i)) + // + // xx, ee := producer1.Send(message) + // if ee != nil { + // glog.Error(ee) + // continue + // } + // glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + //} + //select {} + //rocketMqManager.ShutDown() } diff --git a/rocketmq-go/example/rocketmq_client_example.go b/rocketmq-go/example/rocketmq_client_example.go index 5e922bf2b..ca6ac5404 100644 --- a/rocketmq-go/example/rocketmq_client_example.go +++ b/rocketmq-go/example/rocketmq_client_example.go @@ -18,7 +18,7 @@ package main import ( "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" ) diff --git a/rocketmq-go/clean_expire_msg_controller.go b/rocketmq-go/manage/clean_expire_msg_controller.go similarity index 100% rename from rocketmq-go/clean_expire_msg_controller.go rename to rocketmq-go/manage/clean_expire_msg_controller.go diff --git a/rocketmq-go/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go similarity index 97% rename from rocketmq-go/mq_client_manage.go rename to rocketmq-go/manage/mq_client_manage.go index 790311604..1c45a3e8c 100644 --- a/rocketmq-go/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -20,8 +20,8 @@ import ( "encoding/json" "errors" "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -61,7 +61,7 @@ type MqClientManager struct { defaultProducerService *service.DefaultProducerService //for send back message } -func MqClientManagerInit(clientConfig *config.ClientConfig) (rocketMqManager *MqClientManager) { +func MqClientManagerInit(clientConfig *rocketmq_api_model.ClientConfig) (rocketMqManager *MqClientManager) { rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() rocketMqManager.clientFactory = ClientFactoryInit() @@ -178,7 +178,7 @@ func (self *MqClientManager) resetConsumerOffset(topic, group string, offsetTabl } func (self *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { if self.defaultProducerService == nil { - self.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, config.NewProducerConfig(), self.mqClient) + self.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmq_api_model.NewProducerConfig(), self.mqClient) } consumer.mqClient = self.mqClient consumer.offsetStore = service.RemoteOffsetStoreInit(consumer.consumerGroup, self.mqClient) diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go new file mode 100644 index 000000000..5b12f3a97 --- /dev/null +++ b/rocketmq-go/manage/mq_producer.go @@ -0,0 +1,33 @@ +package rocketmq + +import () +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" +) + +type DefaultMQProducer struct { + producerGroup string + ProducerConfig *rocketmq_api_model.RocketMqProducerConfig + + producerService service.ProducerService +} + +func NewDefaultMQProducer(producerGroup string) (rocketMQProducer *DefaultMQProducer) { + rocketMQProducer = &DefaultMQProducer{ + producerGroup: producerGroup, + ProducerConfig: rocketmq_api_model.NewProducerConfig(), + } + return +} + +func (self *DefaultMQProducer) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) { + sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) + return +} +func (self *DefaultMQProducer) SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { + sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) + return +} diff --git a/rocketmq-go/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go similarity index 90% rename from rocketmq-go/mq_push_consumer.go rename to rocketmq-go/manage/mq_push_consumer.go index 245bbe43b..e7b0b9bd7 100644 --- a/rocketmq-go/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -17,19 +17,14 @@ package rocketmq import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" "github.com/golang/glog" "strings" "time" ) -type Consumer interface { - RegisterMessageListener(listener model.MessageListener) - Subscribe(topic string, subExpression string) -} - type DefaultMQPushConsumer struct { consumerGroup string //consumeFromWhere string @@ -44,7 +39,7 @@ type DefaultMQPushConsumer struct { rebalance *service.Rebalance pause bool //when reset offset we need pause consumeMessageService service.ConsumeMessageService - ConsumerConfig *config.RocketMqConsumerConfig + ConsumerConfig *rocketmq_api_model.RocketMqConsumerConfig } func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *DefaultMQPushConsumer) { @@ -56,7 +51,16 @@ func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *Defa pause: false} defaultMQPushConsumer.subscription = make(map[string]string) defaultMQPushConsumer.subscriptionTag = make(map[string][]string) - defaultMQPushConsumer.ConsumerConfig = config.NewRocketMqConsumerConfig() + defaultMQPushConsumer.ConsumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() + + //for test + comsumer1 := defaultMQPushConsumer + comsumer1.ConsumerConfig.PullInterval = 0 + comsumer1.ConsumerConfig.ConsumeTimeout = 1 + comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 + comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() + return } func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { diff --git a/rocketmq-go/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go similarity index 96% rename from rocketmq-go/pull_message_controller.go rename to rocketmq-go/manage/pull_message_controller.go index 320cc3112..d99a9a7a0 100644 --- a/rocketmq-go/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -21,6 +21,7 @@ import ( "compress/zlib" "encoding/binary" "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" @@ -211,12 +212,12 @@ func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) self.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } -func FilterMessageAgainByTags(msgExts []model.MessageExt, subscriptionTagList []string) (result []model.MessageExt) { +func FilterMessageAgainByTags(msgExts []rocketmq_api_model.MessageExt, subscriptionTagList []string) (result []rocketmq_api_model.MessageExt) { result = msgExts if len(subscriptionTagList) == 0 { return } - result = []model.MessageExt{} + result = []rocketmq_api_model.MessageExt{} for _, msg := range msgExts { for _, tag := range subscriptionTagList { if tag == msg.GetTag() { @@ -236,7 +237,7 @@ func (self *PullMessageController) consumerPullMessageAsync(brokerName string, r } } -func DecodeMessage(data []byte) []model.MessageExt { +func DecodeMessage(data []byte) []rocketmq_api_model.MessageExt { buf := bytes.NewBuffer(data) var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 @@ -246,9 +247,9 @@ func DecodeMessage(data []byte) []model.MessageExt { var propertiesmap = make(map[string]string) - msgs := []model.MessageExt{} + msgs := []rocketmq_api_model.MessageExt{} for buf.Len() > 0 { - msg := model.MessageExt{Message: &model.Message{}} + msg := rocketmq_api_model.MessageExt{Message: &rocketmq_api_model.Message{}} binary.Read(buf, binary.BigEndian, &storeSize) binary.Read(buf, binary.BigEndian, &magicCode) binary.Read(buf, binary.BigEndian, &bodyCRC) diff --git a/rocketmq-go/rebalance_controller.go b/rocketmq-go/manage/rebalance_controller.go similarity index 100% rename from rocketmq-go/rebalance_controller.go rename to rocketmq-go/manage/rebalance_controller.go diff --git a/rocketmq-go/tasks.go b/rocketmq-go/manage/tasks.go similarity index 100% rename from rocketmq-go/tasks.go rename to rocketmq-go/manage/tasks.go diff --git a/rocketmq-go/model/message_listener.go b/rocketmq-go/model/message_listener.go index 7ad2054fc..1e166c4a5 100644 --- a/rocketmq-go/model/message_listener.go +++ b/rocketmq-go/model/message_listener.go @@ -16,4 +16,6 @@ */ package model -type MessageListener func(msgs []MessageExt) ConsumeConcurrentlyResult +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + +type MessageListener func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index e0075e088..8a94b134f 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -17,6 +17,7 @@ package model import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/emirpasic/gods/maps/treemap" @@ -107,7 +108,7 @@ func (self *ProcessQueue) DeleteExpireMsg(queueOffset int) { } } -func (self *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *MessageExt) { +func (self *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq_api_model.MessageExt) { defer self.lockTreeMap.Unlock() self.lockTreeMap.Lock() key, value := self.msgTreeMap.Min() @@ -116,7 +117,7 @@ func (self *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *Messa } offset = key.(int) - message := value.(MessageExt) + message := value.(rocketmq_api_model.MessageExt) messagePoint = &message return } @@ -140,7 +141,7 @@ func (self *ProcessQueue) GetMaxSpan() int { return maxOffset - minOffset } -func (self *ProcessQueue) RemoveMessage(msgs []MessageExt) (offset int64) { +func (self *ProcessQueue) RemoveMessage(msgs []rocketmq_api_model.MessageExt) (offset int64) { now := time.Now() offset = -1 defer self.lockTreeMap.Unlock() @@ -161,7 +162,7 @@ func (self *ProcessQueue) RemoveMessage(msgs []MessageExt) (offset int64) { return } -func (self *ProcessQueue) PutMessage(msgs []MessageExt) (dispatchToConsume bool) { +func (self *ProcessQueue) PutMessage(msgs []rocketmq_api_model.MessageExt) (dispatchToConsume bool) { dispatchToConsume = false msgsLen := len(msgs) if msgsLen == 0 { diff --git a/rocketmq-go/mq_producer.go b/rocketmq-go/mq_producer.go deleted file mode 100644 index 098377d85..000000000 --- a/rocketmq-go/mq_producer.go +++ /dev/null @@ -1,40 +0,0 @@ -package rocketmq - -import () -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" -) - -type RocketMQProducer interface { - Send(message *model.Message) (sendResult *model.SendResult, err error) - SendWithTimeout(message *model.Message, timeout int) (sendResult *model.SendResult, err error) - //SendAsync(message *model.Message) (sendResult *model.SendResult,err error) - //SendAsyncWithTimeout(message *model.Message) (sendResult *model.SendResult,err error) - //SendOneWay(message *model.Message) (sendResult *model.SendResult,err error) -} -type DefaultMQProducer struct { - producerGroup string - ProducerConfig *config.RocketMqProducerConfig - - producerService service.ProducerService -} - -func NewDefaultMQProducer(producerGroup string) (rocketMQProducer *DefaultMQProducer) { - rocketMQProducer = &DefaultMQProducer{ - producerGroup: producerGroup, - ProducerConfig: config.NewProducerConfig(), - } - return -} - -func (self *DefaultMQProducer) Send(message *model.Message) (sendResult *model.SendResult, err error) { - sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) - return -} -func (self *DefaultMQProducer) SendWithTimeout(message *model.Message, timeout int64) (sendResult *model.SendResult, err error) { - sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) - return -} diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 206fdcfce..13a5eda55 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -20,7 +20,7 @@ import ( "bytes" "encoding/binary" "errors" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "math/rand" @@ -38,7 +38,7 @@ type RemotingClient interface { } type DefalutRemotingClient struct { clientId string - clientConfig *config.ClientConfig + clientConfig *rocketmq_api_model.ClientConfig connTable map[string]net.Conn connTableLock sync.RWMutex @@ -55,7 +55,7 @@ type DefalutRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } -func RemotingClientInit(clientConfig *config.ClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefalutRemotingClient) { +func RemotingClientInit(clientConfig *rocketmq_api_model.ClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefalutRemotingClient) { client = &DefalutRemotingClient{} client.connTable = map[string]net.Conn{} client.responseTable = util.New() diff --git a/rocketmq-go/service/client_api.go b/rocketmq-go/service/client_api.go index 6901f4ccf..d7b839fea 100644 --- a/rocketmq-go/service/client_api.go +++ b/rocketmq-go/service/client_api.go @@ -18,8 +18,8 @@ package service import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" ) @@ -41,10 +41,10 @@ type MQClientAPI struct { topAddress *TopAddress crp *ClientRemotingProcessor nameServerAddress string - config *config.ClientConfig + config *rocketmq_api_model.ClientConfig } -//func NewMQClientAPI(cfg *config.ClientConfig, processor *ClientRemotingProcessor, hook remoting.RPCHook) *MQClientAPI { +//func NewMQClientAPI(cfg *rocketmq_api_model.ClientConfig, processor *ClientRemotingProcessor, hook remoting.RPCHook) *MQClientAPI { // api := &MQClientAPI{ // remotingClient: &remoting.RemotingClient{}, //TODO // topAddress: &TopAddress{}, // TODO diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index e4a08df40..18ec57229 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -17,8 +17,8 @@ package service import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/golang/glog" "time" @@ -27,11 +27,11 @@ import ( type ConsumeMessageService interface { //ConsumeMessageDirectlyResult consumeMessageDirectly(final MessageExt msg, final String brokerName); - Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) - SubmitConsumeRequest(msgs []model.MessageExt, processQueue *model.ProcessQueue, + Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) + SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) - SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) - ConsumeMessageDirectly(messageExt *model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) + SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) + ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) } type ConsumeMessageConcurrentlyServiceImpl struct { @@ -39,7 +39,7 @@ type ConsumeMessageConcurrentlyServiceImpl struct { messageListener model.MessageListener sendMessageBackProducerService SendMessageBackProducerService //for send retry Message offsetStore OffsetStore - consumerConfig *config.RocketMqConsumerConfig + consumerConfig *rocketmq_api_model.RocketMqConsumerConfig } func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListener) (consumeService ConsumeMessageService) { @@ -47,14 +47,14 @@ func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListe return } -func (self *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) { +func (self *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { self.consumerGroup = consumerGroup self.offsetStore = offsetStore self.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) self.consumerConfig = consumerConfig } -func (self *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (self *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i @@ -73,14 +73,14 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []m return } -func (self *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (self *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { err = self.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } -func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { +func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { start := time.Now().UnixNano() / 1000000 - consumeResult := self.messageListener([]model.MessageExt{*messageExt}) + consumeResult := self.messageListener([]rocketmq_api_model.MessageExt{*messageExt}) consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false consumeMessageDirectlyResult.SpentTimeMills = time.Now().UnixNano()/1000000 - start @@ -93,7 +93,7 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messag return } -func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result model.ConsumeConcurrentlyResult, msgs []model.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmq_api_model.ConsumeConcurrentlyResult, msgs []rocketmq_api_model.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return @@ -102,7 +102,7 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result m return } ackIndex := result.AckIndex - if model.CONSUME_SUCCESS == result.ConsumeConcurrentlyStatus { + if rocketmq_api_model.CONSUME_SUCCESS == result.ConsumeConcurrentlyStatus { if ackIndex >= len(msgs) { ackIndex = len(msgs) - 1 } else { @@ -111,8 +111,8 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result m } } } - var failedMessages []model.MessageExt - successMessages := []model.MessageExt{} + var failedMessages []rocketmq_api_model.MessageExt + successMessages := []rocketmq_api_model.MessageExt{} if ackIndex >= 0 { successMessages = msgs[:ackIndex+1] } @@ -135,7 +135,7 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result m } -func transformMessageToConsume(consumerGroup string, msgs []model.MessageExt) []model.MessageExt { +func transformMessageToConsume(consumerGroup string, msgs []rocketmq_api_model.MessageExt) []rocketmq_api_model.MessageExt { retryTopicName := constant.RETRY_GROUP_TOPIC_PREFIX + consumerGroup for _, msg := range msgs { diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 366aa3fcb..587f42d9b 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -20,8 +20,8 @@ import ( "encoding/json" "errors" "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -69,7 +69,7 @@ type MqClientImpl struct { PullRequestQueue chan *model.PullRequest //todo move } -func MqClientInit(clientConfig *config.ClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { +func MqClientInit(clientConfig *rocketmq_api_model.ClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} mqClientImpl.ClientId = buildMqClientImplId() mqClientImpl.TopicRouteTable = util.New() // make(map[string]*model.TopicRouteData) diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/service/producer_service.go index 2f2a7b685..1f3607b18 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/service/producer_service.go @@ -18,8 +18,8 @@ package service import ( "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -30,17 +30,17 @@ import ( type ProducerService interface { CheckConfig() (err error) - SendDefaultImpl(message *model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } type DefaultProducerService struct { producerGroup string - producerConfig *config.RocketMqProducerConfig + producerConfig *rocketmq_api_model.RocketMqProducerConfig mqClient RocketMqClient mqFaultStrategy MQFaultStrategy } -func NewDefaultProducerService(producerGroup string, producerConfig *config.RocketMqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { +func NewDefaultProducerService(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { defaultProducerService = &DefaultProducerService{ mqClient: mqClient, producerGroup: producerGroup, @@ -54,7 +54,7 @@ func (self *DefaultProducerService) CheckConfig() (err error) { return } -func (self *DefaultProducerService) SendDefaultImpl(message *model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (self *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) @@ -76,7 +76,7 @@ func (self *DefaultProducerService) SendDefaultImpl(message *model.Message, comm return } -func (self *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *model.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (self *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) var response *remoting.RemotingCommand response, err = self.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -87,7 +87,7 @@ func (self *DefaultProducerService) producerSendMessageRequest(brokerAddr string sendResult, err = processSendResponse(brokerAddr, message, response) return } -func processSendResponse(brokerName string, message *model.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { +func processSendResponse(brokerName string, message *rocketmq_api_model.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { sendResult = &model.SendResult{} switch response.Code { case remoting.FLUSH_DISK_TIMEOUT: @@ -133,7 +133,7 @@ func processSendResponse(brokerName string, message *model.Message, response *re return } -func (self *DefaultProducerService) checkMessage(message *model.Message) (err error) { +func (self *DefaultProducerService) checkMessage(message *rocketmq_api_model.Message) (err error) { if message == nil { err = errors.New("message is nil") return @@ -168,7 +168,7 @@ func (self *DefaultProducerService) checkMessage(message *model.Message) (err er return } -func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int messageQueue model.MessageQueue @@ -192,7 +192,7 @@ func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *model.Me return } -func (self *DefaultProducerService) doSendMessage(message *model.Message, messageQueue model.MessageQueue, +func (self *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Message, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -231,7 +231,7 @@ func (self *DefaultProducerService) doSendMessage(message *model.Message, messag return } -func (self *DefaultProducerService) tryToCompressMessage(message *model.Message) (compressedFlag int, err error) { +func (self *DefaultProducerService) tryToCompressMessage(message *rocketmq_api_model.Message) (compressedFlag int, err error) { if len(message.Body) < self.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return diff --git a/rocketmq-go/service/producer_service_for_send_back.go b/rocketmq-go/service/producer_service_for_send_back.go index 290da274c..f1543b912 100644 --- a/rocketmq-go/service/producer_service_for_send_back.go +++ b/rocketmq-go/service/producer_service_for_send_back.go @@ -19,8 +19,7 @@ package service import ( "encoding/json" "errors" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -28,19 +27,19 @@ import ( ) type SendMessageBackProducerService interface { - SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) - InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) + SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) + InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) } type SendMessageBackProducerServiceImpl struct { mqClient RocketMqClient defaultProducerService *DefaultProducerService // one namesvr only one consumerGroup string - consumerConfig *config.RocketMqConsumerConfig //one mq group have one + consumerConfig *rocketmq_api_model.RocketMqConsumerConfig //one mq group have one } // send to original broker,if fail send a new retry message -func (self *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *model.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (self *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) err = self.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) // todo use if err == nil { @@ -51,9 +50,9 @@ func (self *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *mode return } -func (self *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *model.MessageExt) error { +func (self *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmq_api_model.MessageExt) error { // todo build a retry topic todo check todo check - retryMessage := &model.Message{} + retryMessage := &rocketmq_api_model.Message{} originMessageId := messageExt.GetOriginMessageId() retryMessage.Properties = messageExt.Properties retryMessage.SetOriginMessageId(originMessageId) @@ -78,14 +77,14 @@ func (self *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt } -func (self *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *config.RocketMqConsumerConfig) { +func (self *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { self.mqClient = mqClient self.consumerGroup = consumerGroup self.defaultProducerService = defaultProducerService self.consumerConfig = consumerConfig } -func (self *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *model.MessageExt, delayLayLevel int) (err error) { +func (self *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmq_api_model.MessageExt, delayLayLevel int) (err error) { if len(brokerName) == 0 { err = errors.New("broker can't be empty") glog.Error(err) diff --git a/rocketmq-go/service/rebalance.go b/rocketmq-go/service/rebalance.go index 8f4f4fbef..8c0cbb4fd 100644 --- a/rocketmq-go/service/rebalance.go +++ b/rocketmq-go/service/rebalance.go @@ -19,8 +19,8 @@ package service import ( "encoding/json" "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/config" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -44,7 +44,7 @@ type Rebalance struct { processQueueTableLock sync.RWMutex mutex sync.Mutex offsetStore OffsetStore - consumerConfig *config.RocketMqConsumerConfig + consumerConfig *rocketmq_api_model.RocketMqConsumerConfig } func (self *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { @@ -99,7 +99,7 @@ func (self *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue } -func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *config.RocketMqConsumerConfig) *Rebalance { +func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) *Rebalance { subscriptionInner := make(map[string]*model.SubscriptionData) for topic, subExpression := range subscription { subData := &model.SubscriptionData{ @@ -227,7 +227,7 @@ func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { var result int64 = -1 lastOffset := self.offsetStore.ReadOffset(mq, READ_FROM_STORE) switch self.consumerConfig.ConsumeFromWhere { - case config.CONSUME_FROM_LAST_OFFSET: + case rocketmq_api_model.CONSUME_FROM_LAST_OFFSET: if lastOffset >= 0 { result = lastOffset } else { @@ -238,14 +238,14 @@ func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { } } break - case config.CONSUME_FROM_FIRST_OFFSET: + case rocketmq_api_model.CONSUME_FROM_FIRST_OFFSET: if lastOffset >= 0 { result = lastOffset } else { result = 0 // use the begin offset } break - case config.CONSUME_FROM_TIMESTAMP: + case rocketmq_api_model.CONSUME_FROM_TIMESTAMP: if lastOffset >= 0 { result = lastOffset } else { From b115a9c3054079786a765014699bf8c604039148 Mon Sep 17 00:00:00 2001 From: tangjie Date: Wed, 9 Aug 2017 23:29:22 +0800 Subject: [PATCH 04/88] consumer and producer can getConfig --- rocketmq-go/api/rocketmq_consumer.go | 2 ++ rocketmq-go/api/rocketmq_producer.go | 1 + rocketmq-go/example/producer_consumer_example.go | 12 +++++++++++- rocketmq-go/manage/mq_producer.go | 4 +++- rocketmq-go/manage/mq_push_consumer.go | 10 +++------- 5 files changed, 20 insertions(+), 9 deletions(-) diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 1d7857562..d98cdde92 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -3,11 +3,13 @@ package rocketmq_api import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) type RocketMQConsumer interface { RegisterMessageListener(listener model.MessageListener) Subscribe(topic string, subExpression string) + GetConsumerConfig()( *rocketmq_api_model.RocketMqConsumerConfig) } func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 1e792ffdb..66b33a1eb 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -7,6 +7,7 @@ import ( ) type RocketMQProducer interface { + GetProducerConfig()( *rocketmq_api_model.RocketMqProducerConfig) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) } diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go index 8508897bf..d3eee231d 100644 --- a/rocketmq-go/example/producer_consumer_example.go +++ b/rocketmq-go/example/producer_consumer_example.go @@ -23,6 +23,7 @@ import ( "github.com/golang/glog" "net/http" _ "net/http/pprof" + "time" ) func main() { @@ -33,9 +34,18 @@ func main() { testTopic = "GoLang" ) var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") - //producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 + producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + //for test + consumerConfig := comsumer1.GetConsumerConfig() + consumerConfig.PullInterval = 0 + consumerConfig.ConsumeTimeout = 1 + consumerConfig.ConsumeMessageBatchMaxSize = 16 + consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + consumerConfig.ConsumeTimestamp = time.Now() + + comsumer1.Subscribe(testTopic, "*") comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { for _, msg := range msgs { diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index 5b12f3a97..9fa7acd6b 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -22,7 +22,9 @@ func NewDefaultMQProducer(producerGroup string) (rocketMQProducer *DefaultMQProd } return } - +func (self *DefaultMQProducer) GetProducerConfig()( *rocketmq_api_model.RocketMqProducerConfig){ +return self.ProducerConfig +} func (self *DefaultMQProducer) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) return diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index e7b0b9bd7..1113b2991 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -53,16 +53,12 @@ func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *Defa defaultMQPushConsumer.subscriptionTag = make(map[string][]string) defaultMQPushConsumer.ConsumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() - //for test - comsumer1 := defaultMQPushConsumer - comsumer1.ConsumerConfig.PullInterval = 0 - comsumer1.ConsumerConfig.ConsumeTimeout = 1 - comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 - comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() return } +func (self *DefaultMQPushConsumer) GetConsumerConfig()( *rocketmq_api_model.RocketMqConsumerConfig){ + return self.ConsumerConfig +} func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { self.subscription[topic] = subExpression if len(subExpression) == 0 || subExpression == "*" { From 208f2a2be0f889ec281381b0445df6b36f1f5e45 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 20:52:28 +0800 Subject: [PATCH 05/88] add .travis.yml --- .travis.yml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 000000000..00d42047d --- /dev/null +++ b/.travis.yml @@ -0,0 +1,15 @@ +matrix: + include: + - language: java + jdk: + - oraclejdk7 + script: + - travis_retry mvn -B clean + - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report + +notifications: + email: + recipients: + - styletang.me@gmail.com + on_success: change + on_failure: always \ No newline at end of file From 3e8fc27ed985462762bf5938c174efe68c4b6fa8 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 20:59:29 +0800 Subject: [PATCH 06/88] .travis.yml jdk --- .travis.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 00d42047d..0ef919e0b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,8 +1,9 @@ matrix: include: - language: java - jdk: - - oraclejdk7 + jdk: oraclejdk7 + before_script: + - cd rocketmq-console script: - travis_retry mvn -B clean - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report From c3a99dbde1ec53831d281902220610c0a0f645c4 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 21:03:16 +0800 Subject: [PATCH 07/88] .travis.yml jdk --- .travis.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 0ef919e0b..167c42d12 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,9 +1,10 @@ matrix: include: - language: java - jdk: oraclejdk7 + jdk: + - oraclejdk7 before_script: - - cd rocketmq-console + - cd rocketmq-console script: - travis_retry mvn -B clean - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report From 903a3399368da10951926623389e1a0352f0ecf8 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 21:20:20 +0800 Subject: [PATCH 08/88] .travis.yml add go test --- .travis.yml | 12 ++- rocketmq-go/api/model/producer_config_test.go | 7 ++ rocketmq-go/api/rocketmq_manage_test.go | 7 ++ rocketmq-go/example/consumer_example.go | 10 +-- rocketmq-go/example/producer_example.go | 10 +-- .../example/rocketmq_client_example.go | 34 ++++---- rocketmq-go/producer_consumer_example.go | 81 +++++++++++++++++++ rocketmq-go/producer_consumer_example_test.go | 14 ++++ 8 files changed, 146 insertions(+), 29 deletions(-) create mode 100644 rocketmq-go/api/model/producer_config_test.go create mode 100644 rocketmq-go/api/rocketmq_manage_test.go create mode 100644 rocketmq-go/producer_consumer_example.go create mode 100644 rocketmq-go/producer_consumer_example_test.go diff --git a/.travis.yml b/.travis.yml index 167c42d12..94d9a8281 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,8 +1,16 @@ matrix: + include: + language: go + go: + - 1.7.x + go_import_path: apache/incubator-rocketmq-externals/rocketmq-go + before_script: + - cd rocketmq-go + script: + - go test -v ./... include: - language: java - jdk: - - oraclejdk7 + jdk: oraclejdk7 before_script: - cd rocketmq-console script: diff --git a/rocketmq-go/api/model/producer_config_test.go b/rocketmq-go/api/model/producer_config_test.go new file mode 100644 index 000000000..34aba084f --- /dev/null +++ b/rocketmq-go/api/model/producer_config_test.go @@ -0,0 +1,7 @@ +package rocketmq_api_model + +import "testing" + +func TestB(t *testing.T) { + +} diff --git a/rocketmq-go/api/rocketmq_manage_test.go b/rocketmq-go/api/rocketmq_manage_test.go new file mode 100644 index 000000000..dae00e8d0 --- /dev/null +++ b/rocketmq-go/api/rocketmq_manage_test.go @@ -0,0 +1,7 @@ +package rocketmq_api + +import "testing" + +func TestA(t *testing.T) { + +} diff --git a/rocketmq-go/example/consumer_example.go b/rocketmq-go/example/consumer_example.go index 990393894..984eccbe7 100644 --- a/rocketmq-go/example/consumer_example.go +++ b/rocketmq-go/example/consumer_example.go @@ -17,11 +17,11 @@ package main import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/golang/glog" - "time" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + //"github.com/golang/glog" + //"time" ) func main() { diff --git a/rocketmq-go/example/producer_example.go b/rocketmq-go/example/producer_example.go index f54b3fc52..ab3620cc8 100644 --- a/rocketmq-go/example/producer_example.go +++ b/rocketmq-go/example/producer_example.go @@ -17,11 +17,11 @@ package main import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "github.com/golang/glog" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + //"github.com/golang/glog" _ "net/http/pprof" ) diff --git a/rocketmq-go/example/rocketmq_client_example.go b/rocketmq-go/example/rocketmq_client_example.go index ca6ac5404..a78479b4a 100644 --- a/rocketmq-go/example/rocketmq_client_example.go +++ b/rocketmq-go/example/rocketmq_client_example.go @@ -17,24 +17,24 @@ package main import ( - "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + //"fmt" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" ) func main() { - - var clienConfig = config.NewClientConfig() - clienConfig.SetNameServerAddress("127.0.0.1:9876") - - //use json serializer - var mqClient = service.MqClientInit(clienConfig, nil) - fmt.Println(mqClient.TryToFindTopicPublishInfo("GoLang")) - //&{false true [{GoLang broker-a 0} {GoLang broker-a 1} {GoLang broker-a 2} {GoLang broker-a 3}] 0xc420016800 0} - - //use rocketmq serializer - constant.USE_HEADER_SERIALIZETYPE = constant.ROCKETMQ_SERIALIZE - var mqClient2 = service.MqClientInit(clienConfig, nil) - fmt.Println(mqClient2.TryToFindTopicPublishInfo("GoLang")) + // + //var clienConfig = config.NewClientConfig() + //clienConfig.SetNameServerAddress("127.0.0.1:9876") + // + ////use json serializer + //var mqClient = service.MqClientInit(clienConfig, nil) + //fmt.Println(mqClient.TryToFindTopicPublishInfo("GoLang")) + ////&{false true [{GoLang broker-a 0} {GoLang broker-a 1} {GoLang broker-a 2} {GoLang broker-a 3}] 0xc420016800 0} + // + ////use rocketmq serializer + //constant.USE_HEADER_SERIALIZETYPE = constant.ROCKETMQ_SERIALIZE + //var mqClient2 = service.MqClientInit(clienConfig, nil) + //fmt.Println(mqClient2.TryToFindTopicPublishInfo("GoLang")) } diff --git a/rocketmq-go/producer_consumer_example.go b/rocketmq-go/producer_consumer_example.go new file mode 100644 index 000000000..d3eee231d --- /dev/null +++ b/rocketmq-go/producer_consumer_example.go @@ -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. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + + "github.com/golang/glog" + "net/http" + _ "net/http/pprof" + "time" +) + +func main() { + go func() { + http.ListenAndServe("localhost:6060", nil) + }() + var ( + testTopic = "GoLang" + ) + var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") + producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 + var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") + var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + //for test + consumerConfig := comsumer1.GetConsumerConfig() + consumerConfig.PullInterval = 0 + consumerConfig.ConsumeTimeout = 1 + consumerConfig.ConsumeMessageBatchMaxSize = 16 + consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + consumerConfig.ConsumeTimestamp = time.Now() + + + comsumer1.Subscribe(testTopic, "*") + comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + for _, msg := range msgs { + glog.Info(msg.BornTimestamp) + } + glog.Info("look message len ", len(msgs)) + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} + }) + var clienConfig = &rocketmq_api_model.ClientConfig{} + clienConfig.SetNameServerAddress("127.0.0.1:9876") + //clienConfig// todo + rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) + rocketMqManager.RegistProducer(producer1) + rocketMqManager.RegistProducer(producer2) + rocketMqManager.RegistConsumer(comsumer1) + rocketMqManager.Start() + for i := 0; i < 10000000; i++ { + var message = &rocketmq_api_model.Message{} + message.Topic = testTopic + message.SetKeys([]string{"xxx"}) + message.SetTag("1122") + message.Body = []byte("hellAXXWord") + + xx, ee := producer1.Send(message) + if ee != nil { + glog.Error(ee) + continue + } + glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + } + select {} + rocketMqManager.ShutDown() +} diff --git a/rocketmq-go/producer_consumer_example_test.go b/rocketmq-go/producer_consumer_example_test.go new file mode 100644 index 000000000..69efb4553 --- /dev/null +++ b/rocketmq-go/producer_consumer_example_test.go @@ -0,0 +1,14 @@ +package main + +import "testing" +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +) +func TestAdda(t *testing.T) { + var clienConfig = &rocketmq_api_model.ClientConfig{} + clienConfig.SetNameServerAddress("127.0.0.1:9876") + testa := rocketmq_api.InitRocketMQController(clienConfig); + fmt.Print(testa) +} From d8ef4cba93f0c01b0af95bed493121c7f4b4a7a4 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 21:22:27 +0800 Subject: [PATCH 09/88] .travis.yml remove java --- .travis.yml | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/.travis.yml b/.travis.yml index 94d9a8281..bdaf62fe0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -8,14 +8,14 @@ matrix: - cd rocketmq-go script: - go test -v ./... - include: - - language: java - jdk: oraclejdk7 - before_script: - - cd rocketmq-console - script: - - travis_retry mvn -B clean - - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report +# include: +# - language: java +# jdk: oraclejdk7 +# before_script: +# - cd rocketmq-console +# script: +# - travis_retry mvn -B clean +# - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report notifications: email: From 6ec7976dfe004d7f170990a1c7f487440767fa8a Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 21:54:31 +0800 Subject: [PATCH 10/88] .travis.yml test java --- .travis.yml | 37 +++++++++++++++++++++---------------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/.travis.yml b/.travis.yml index bdaf62fe0..3459f5ff9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,21 +1,26 @@ matrix: - include: - language: go - go: - - 1.7.x - go_import_path: apache/incubator-rocketmq-externals/rocketmq-go - before_script: - - cd rocketmq-go - script: - - go test -v ./... # include: -# - language: java -# jdk: oraclejdk7 -# before_script: -# - cd rocketmq-console -# script: -# - travis_retry mvn -B clean -# - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report +# language: go +# go: +# - 1.7.x +# go_import_path: apache/incubator-rocketmq-externals/rocketmq-go +# before_script: +# - cd rocketmq-go +# script: +# - go test -v ./... + + + include: + - language: java + jdk: + - openjdk7 + - oraclejdk7 + - oraclejdk8 + before_script: + - cd rocketmq-console + script: + - travis_retry mvn -B clean + - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report notifications: email: From 6cec9d4f4b17fa0829d48ac9f677a9d59c79831a Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 22:00:40 +0800 Subject: [PATCH 11/88] .travis.yml test java --- .travis.yml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 3459f5ff9..2c5cef216 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,13 +12,12 @@ matrix: include: - language: java - jdk: - - openjdk7 - - oraclejdk7 - - oraclejdk8 +# jdk: +# - oraclejdk7 before_script: - cd rocketmq-console script: + - jdk_switcher use oraclejdk7 - travis_retry mvn -B clean - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report From 82f7c87ce14f4c04abd9314720a967a8b3e96f1a Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 22:09:21 +0800 Subject: [PATCH 12/88] openjdk7 --- .travis.yml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 2c5cef216..c27c86cb8 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,12 +12,11 @@ matrix: include: - language: java -# jdk: -# - oraclejdk7 + jdk: + - openjdk7 before_script: - cd rocketmq-console script: - - jdk_switcher use oraclejdk7 - travis_retry mvn -B clean - travis_retry mvn -B package findbugs:findbugs jacoco:report coveralls:report From 713950086700f7c05e3a269f1fed072ea0eb8efd Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 22:12:48 +0800 Subject: [PATCH 13/88] - os: linux env: CUSTOM_JDK="oraclejdk7" --- .travis.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index c27c86cb8..b2f41cc19 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,8 +12,8 @@ matrix: include: - language: java - jdk: - - openjdk7 + - os: linux + env: CUSTOM_JDK="oraclejdk7" before_script: - cd rocketmq-console script: From 7c4d36a8bcdf50543f8f3fcfbf20ed2e0bfd11ac Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 22:22:56 +0800 Subject: [PATCH 14/88] travis language: java --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index b2f41cc19..989c7a4da 100644 --- a/.travis.yml +++ b/.travis.yml @@ -12,7 +12,7 @@ matrix: include: - language: java - - os: linux + os: linux env: CUSTOM_JDK="oraclejdk7" before_script: - cd rocketmq-console From 9c43201ed790320e0707dcb6f3fe953dff3794a9 Mon Sep 17 00:00:00 2001 From: tangjie Date: Thu, 10 Aug 2017 22:36:27 +0800 Subject: [PATCH 15/88] README.md --- rocketmq-console/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-console/README.md b/rocketmq-console/README.md index 30dd67cc7..2857f9131 100644 --- a/rocketmq-console/README.md +++ b/rocketmq-console/README.md @@ -1,4 +1,4 @@ -## RocketMQ-Console-Ng[![Build Status](https://travis-ci.org/rocketmq/rocketmq-console-ng.svg?branch=master)](https://travis-ci.org/rocketmq/rocketmq-console-ng) [![Coverage Status](https://coveralls.io/repos/github/rocketmq/rocketmq-console-ng/badge.svg?branch=master)](https://coveralls.io/github/rocketmq/rocketmq-console-ng?branch=master) +## RocketMQ-Console-Ng[![Build Status](https://travis-ci.org/StyleTang/incubator-rocketmq-externals.svg?branch=master)](https://travis-ci.org/StyleTang/incubator-rocketmq-externals) [![Coverage Status](https://coveralls.io/repos/github/StyleTang/incubator-rocketmq-externals/badge.svg?branch=master)](https://coveralls.io/github/StyleTang/incubator-rocketmq-externals?branch=master) [![License](https://img.shields.io/badge/license-Apache%202-4EB1BA.svg)](https://www.apache.org/licenses/LICENSE-2.0.html) ## How To Install From 119b6bc5de7ef0aa9e67cba9a533971726c9824d Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 10 Aug 2017 23:47:52 +0800 Subject: [PATCH 16/88] update go client code of api --- .../api/model/consume_concurrently_result.go | 8 +- rocketmq-go/api/model/consumer_config.go | 19 ++- rocketmq-go/api/model/producer_config.go | 24 +--- ...ketmq_manage.go => rocketmq_controller.go} | 0 ...ge_test.go => rocketmq_controller_test.go} | 0 .../example/producer_consumer_example.go | 116 +++++++++--------- rocketmq-go/model/heart_beat.go | 4 +- rocketmq-go/producer_consumer_example.go | 2 +- .../service/consume_message_service.go | 3 +- 9 files changed, 85 insertions(+), 91 deletions(-) rename rocketmq-go/api/{rocketmq_manage.go => rocketmq_controller.go} (100%) rename rocketmq-go/api/{rocketmq_manage_test.go => rocketmq_controller_test.go} (100%) diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go index 2b7cae092..070e68d80 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -16,12 +16,14 @@ */ package rocketmq_api_model +type ConsumeStatus int + const ( - CONSUME_SUCCESS = "CONSUME_SUCCESS" - RECONSUME_LATER = "RECONSUME_LATER" + CONSUME_SUCCESS ConsumeStatus = iota + RECONSUME_LATER ) type ConsumeConcurrentlyResult struct { - ConsumeConcurrentlyStatus string + ConsumeConcurrentlyStatus ConsumeStatus AckIndex int } diff --git a/rocketmq-go/api/model/consumer_config.go b/rocketmq-go/api/model/consumer_config.go index 8d4dbbbb4..3b79d7ae2 100644 --- a/rocketmq-go/api/model/consumer_config.go +++ b/rocketmq-go/api/model/consumer_config.go @@ -29,19 +29,28 @@ const PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION int64 = 3000 const PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL int64 = 50 //consume from where +type ConsumeFromWhere int + //first consume from the last offset -const CONSUME_FROM_LAST_OFFSET string = "CONSUME_FROM_LAST_OFFSET" +const( + CONSUME_FROM_LAST_OFFSET ConsumeFromWhere = iota //first consume from the first offset -const CONSUME_FROM_FIRST_OFFSET string = "CONSUME_FROM_FIRST_OFFSET" + CONSUME_FROM_FIRST_OFFSET //first consume from the time -const CONSUME_FROM_TIMESTAMP string = "CONSUME_FROM_TIMESTAMP" + CONSUME_FROM_TIMESTAMP +) + + + -//consume from where type RocketMqConsumerConfig struct { - ConsumeFromWhere string + /** + * consume from where + */ + ConsumeFromWhere ConsumeFromWhere /** * Concurrently max span offset.it has no effect on sequential consumption */ diff --git a/rocketmq-go/api/model/producer_config.go b/rocketmq-go/api/model/producer_config.go index a79eae9e9..233e2d013 100644 --- a/rocketmq-go/api/model/producer_config.go +++ b/rocketmq-go/api/model/producer_config.go @@ -17,31 +17,13 @@ package rocketmq_api_model type RocketMqProducerConfig struct { - SendMsgTimeout int64 //done - //private int sendMsgTimeout = 3000; - CompressMsgBodyOverHowMuch int //done - //private int compressMsgBodyOverHowmuch = 1024 * 4; - ZipCompressLevel int //done - //private int zipCompressLevel = Integer.parseInt(System.getProperty(MixAll.MESSAGE_COMPRESS_LEVEL, "5")); - /** - * Just for testing or demo program - */ - // private String createTopicKey = MixAll.DEFAULT_TOPIC; - - //DefaultTopicQueueNums int - ////private volatile int defaultTopicQueueNums = 4; - + SendMsgTimeout int64 + CompressMsgBodyOverHowMuch int + ZipCompressLevel int RetryTimesWhenSendFailed int - //private int retryTimesWhenSendFailed = 2; RetryTimesWhenSendAsyncFailed int - //private int retryTimesWhenSendAsyncFailed = 2; - // RetryAnotherBrokerWhenNotStoreOK bool - //private boolean retryAnotherBrokerWhenNotStoreOK = false; MaxMessageSize int - //private int maxMessageSize = 1024 * 1024 * 4; // 4M - - //for MQFaultStrategy todo to be done SendLatencyFaultEnable bool //false LatencyMax []int64 //= {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L}; NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; diff --git a/rocketmq-go/api/rocketmq_manage.go b/rocketmq-go/api/rocketmq_controller.go similarity index 100% rename from rocketmq-go/api/rocketmq_manage.go rename to rocketmq-go/api/rocketmq_controller.go diff --git a/rocketmq-go/api/rocketmq_manage_test.go b/rocketmq-go/api/rocketmq_controller_test.go similarity index 100% rename from rocketmq-go/api/rocketmq_manage_test.go rename to rocketmq-go/api/rocketmq_controller_test.go diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go index d3eee231d..8e1357a9e 100644 --- a/rocketmq-go/example/producer_consumer_example.go +++ b/rocketmq-go/example/producer_consumer_example.go @@ -17,65 +17,65 @@ package main import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - - "github.com/golang/glog" - "net/http" - _ "net/http/pprof" - "time" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + // + //"github.com/golang/glog" + //"net/http" + //_ "net/http/pprof" + //"time" ) func main() { - go func() { - http.ListenAndServe("localhost:6060", nil) - }() - var ( - testTopic = "GoLang" - ) - var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") - producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 - var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") - var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") - //for test - consumerConfig := comsumer1.GetConsumerConfig() - consumerConfig.PullInterval = 0 - consumerConfig.ConsumeTimeout = 1 - consumerConfig.ConsumeMessageBatchMaxSize = 16 - consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - consumerConfig.ConsumeTimestamp = time.Now() - - - comsumer1.Subscribe(testTopic, "*") - comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { - for _, msg := range msgs { - glog.Info(msg.BornTimestamp) - } - glog.Info("look message len ", len(msgs)) - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} - }) - var clienConfig = &rocketmq_api_model.ClientConfig{} - clienConfig.SetNameServerAddress("127.0.0.1:9876") - //clienConfig// todo - rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) - rocketMqManager.RegistProducer(producer1) - rocketMqManager.RegistProducer(producer2) - rocketMqManager.RegistConsumer(comsumer1) - rocketMqManager.Start() - for i := 0; i < 10000000; i++ { - var message = &rocketmq_api_model.Message{} - message.Topic = testTopic - message.SetKeys([]string{"xxx"}) - message.SetTag("1122") - message.Body = []byte("hellAXXWord") - - xx, ee := producer1.Send(message) - if ee != nil { - glog.Error(ee) - continue - } - glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - } - select {} - rocketMqManager.ShutDown() + //go func() { + // http.ListenAndServe("localhost:6060", nil) + //}() + //var ( + // testTopic = "GoLang" + //) + //var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") + //producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 + //var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") + //var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + ////for test + //consumerConfig := comsumer1.GetConsumerConfig() + //consumerConfig.PullInterval = 0 + //consumerConfig.ConsumeTimeout = 1 + //consumerConfig.ConsumeMessageBatchMaxSize = 16 + //consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + //consumerConfig.ConsumeTimestamp = time.Now() + // + // + //comsumer1.Subscribe(testTopic, "*") + //comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + // for _, msg := range msgs { + // glog.Info(msg.BornTimestamp) + // } + // glog.Info("look message len ", len(msgs)) + // return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} + //}) + //var clienConfig = &rocketmq_api_model.ClientConfig{} + //clienConfig.SetNameServerAddress("127.0.0.1:9876") + ////clienConfig// todo + //rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) + //rocketMqManager.RegistProducer(producer1) + //rocketMqManager.RegistProducer(producer2) + //rocketMqManager.RegistConsumer(comsumer1) + //rocketMqManager.Start() + //for i := 0; i < 10000000; i++ { + // var message = &rocketmq_api_model.Message{} + // message.Topic = testTopic + // message.SetKeys([]string{"xxx"}) + // message.SetTag("1122") + // message.Body = []byte("hellAXXWord") + // + // xx, ee := producer1.Send(message) + // if ee != nil { + // glog.Error(ee) + // continue + // } + // glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + //} + //select {} + //rocketMqManager.ShutDown() } diff --git a/rocketmq-go/model/heart_beat.go b/rocketmq-go/model/heart_beat.go index fc5eded24..5e206f31a 100644 --- a/rocketmq-go/model/heart_beat.go +++ b/rocketmq-go/model/heart_beat.go @@ -16,11 +16,13 @@ */ package model +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + type ConsumerData struct { GroupName string ConsumeType string MessageModel string - ConsumeFromWhere string + ConsumeFromWhere rocketmq_api_model.ConsumeFromWhere SubscriptionDataSet []*SubscriptionData UnitMode bool } diff --git a/rocketmq-go/producer_consumer_example.go b/rocketmq-go/producer_consumer_example.go index d3eee231d..c16eec4a6 100644 --- a/rocketmq-go/producer_consumer_example.go +++ b/rocketmq-go/producer_consumer_example.go @@ -42,7 +42,7 @@ func main() { consumerConfig.PullInterval = 0 consumerConfig.ConsumeTimeout = 1 consumerConfig.ConsumeMessageBatchMaxSize = 16 - consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" + consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP consumerConfig.ConsumeTimestamp = time.Now() diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index 18ec57229..19a92360d 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -84,9 +84,8 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messag consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false consumeMessageDirectlyResult.SpentTimeMills = time.Now().UnixNano()/1000000 - start - if consumeResult.ConsumeConcurrentlyStatus == "CONSUME_SUCCESS" && consumeResult.AckIndex >= 0 { + if consumeResult.ConsumeConcurrentlyStatus == rocketmq_api_model.CONSUME_SUCCESS && consumeResult.AckIndex >= 0 { consumeMessageDirectlyResult.ConsumeResult = "CR_SUCCESS" - } else { consumeMessageDirectlyResult.ConsumeResult = "CR_THROW_EXCEPTION" } From 664caa2a14573f615bbfaf454cf7e2a59b304525 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 12 Aug 2017 14:36:58 +0800 Subject: [PATCH 17/88] CurrentTimeMillisInt64 --- rocketmq-go/api/model/message_ext.go | 4 +- rocketmq-go/api/rocketmq_consumer.go | 2 +- rocketmq-go/example/consumer_example.go | 53 ------------------- rocketmq-go/manage/mq_client_manage.go | 1 + rocketmq-go/manage/mq_push_consumer.go | 3 +- .../service/consume_message_service.go | 6 +-- rocketmq-go/service/mq_client.go | 3 +- rocketmq-go/service/producer_service.go | 3 +- .../util/message_client_id_generator.go | 6 +-- rocketmq-go/util/time_util.go | 15 ++++++ 10 files changed, 28 insertions(+), 68 deletions(-) delete mode 100644 rocketmq-go/example/consumer_example.go create mode 100644 rocketmq-go/util/time_util.go diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 4fd956c62..bf8d328e0 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -20,8 +20,6 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "math" - "strconv" - "time" ) type MessageExt struct { @@ -64,7 +62,7 @@ func (self *MessageExt) SetConsumeStartTime() { if self.Properties == nil { self.Properties = make(map[string]string) } - nowTime := strconv.FormatInt(time.Now().UnixNano()/1000000, 10) + nowTime := util.CurrentTimeMillisStr() self.Properties[constant.PROPERTY_KEYS] = nowTime self.propertyConsumeStartTimestamp = nowTime return diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index d98cdde92..caf9f299f 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -9,7 +9,7 @@ import ( type RocketMQConsumer interface { RegisterMessageListener(listener model.MessageListener) Subscribe(topic string, subExpression string) - GetConsumerConfig()( *rocketmq_api_model.RocketMqConsumerConfig) + GetConsumerConfig() (*rocketmq_api_model.RocketMqConsumerConfig) } func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { diff --git a/rocketmq-go/example/consumer_example.go b/rocketmq-go/example/consumer_example.go deleted file mode 100644 index 984eccbe7..000000000 --- a/rocketmq-go/example/consumer_example.go +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package main - -import ( - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - //"github.com/golang/glog" - //"time" -) - -func main() { - // - //var ( - // testTopic = "GoLang" - //) - //var comsumer1 = rocketmq.NewDefaultMQPushConsumer(testTopic + "-StyleTang") - //comsumer1.ConsumerConfig.PullInterval = 0 - //comsumer1.ConsumerConfig.ConsumeTimeout = 1 - //comsumer1.ConsumerConfig.ConsumeMessageBatchMaxSize = 16 - //comsumer1.ConsumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - //comsumer1.ConsumerConfig.ConsumeTimestamp = time.Now() - //comsumer1.Subscribe(testTopic, "*") - //comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) model.ConsumeConcurrentlyResult { - // for _, msg := range msgs { - // glog.Info(msg.BornTimestamp) - // } - // glog.Info("look message len ", len(msgs)) - // return model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: model.CONSUME_SUCCESS, AckIndex: len(msgs)} - //}) - //var clienConfig = &rocketmq_api_model.ClientConfig{} - //clienConfig.SetNameServerAddress("127.0.0.1:9876") - //rocketMqManager := rocketmq.MqClientManagerInit(clienConfig) - //rocketMqManager.RegistConsumer(comsumer1) - //rocketMqManager.Start() - //select {} - //rocketMqManager.ShutDown() -} diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index 1c45a3e8c..5f3126d25 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -198,6 +198,7 @@ func (self *MqClientManager) Start() { func (self MqClientManager) ShutDown() { + } type ClientFactory struct { diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index 1113b2991..00b280d21 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -23,6 +23,7 @@ import ( "github.com/golang/glog" "strings" "time" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" ) type DefaultMQPushConsumer struct { @@ -117,7 +118,7 @@ func (self *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { } func (self *DefaultMQPushConsumer) CleanExpireMsg() { - nowTime := int64(time.Now().UnixNano()) / 1000000 //will cause nowTime - consumeStartTime <0 ,but no matter + nowTime := util.CurrentTimeMillisInt64()//will cause nowTime - consumeStartTime <0 ,but no matter messageQueueList, processQueueList := self.rebalance.GetProcessQueueList() for messageQueueIndex, processQueue := range processQueueList { loop := processQueue.GetMsgCount() diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index 19a92360d..e85ed592e 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -21,7 +21,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/golang/glog" - "time" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" ) type ConsumeMessageService interface { @@ -79,11 +79,11 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *r } func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { - start := time.Now().UnixNano() / 1000000 + start := util.CurrentTimeMillisInt64() consumeResult := self.messageListener([]rocketmq_api_model.MessageExt{*messageExt}) consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false - consumeMessageDirectlyResult.SpentTimeMills = time.Now().UnixNano()/1000000 - start + consumeMessageDirectlyResult.SpentTimeMills = util.CurrentTimeMillisInt64() - start if consumeResult.ConsumeConcurrentlyStatus == rocketmq_api_model.CONSUME_SUCCESS && consumeResult.AckIndex >= 0 { consumeMessageDirectlyResult.ConsumeResult = "CR_SUCCESS" } else { diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 587f42d9b..8f9286e4d 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -101,7 +101,6 @@ func (self *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { } queryOffsetResponseHeader := header.QueryOffsetResponseHeader{} queryOffsetResponseHeader.FromMap(response.ExtFields) - glog.Info("op=look max offset result", string(response.Body)) return queryOffsetResponseHeader.Offset } func (self *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) int64 { @@ -110,7 +109,7 @@ func (self *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) i self.TryToFindTopicPublishInfo(mq.Topic) brokerAddr = self.FetchMasterBrokerAddress(mq.BrokerName) } - timeStamp := time.UnixNano() / 1000000 + timeStamp := util.CurrentTimeMillisInt64() searchOffsetRequestHeader := &header.SearchOffsetRequestHeader{Topic: mq.Topic, QueueId: mq.QueueId, Timestamp: timeStamp} remotingCmd := remoting.NewRemotingCommand(remoting.SEARCH_OFFSET_BY_TIMESTAMP, searchOffsetRequestHeader) response, err := self.remotingClient.InvokeSync(brokerAddr, remotingCmd, DEFAULT_TIMEOUT) diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/service/producer_service.go index 1f3607b18..836254093 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/service/producer_service.go @@ -25,7 +25,6 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" - "time" ) type ProducerService interface { @@ -219,7 +218,7 @@ func (self *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Me DefaultTopicQueueNums: 4, QueueId: messageQueue.QueueId, SysFlag: sysFlag, - BornTimestamp: time.Now().UnixNano() / 1000000, + BornTimestamp: util.CurrentTimeMillisInt64(), Flag: message.Flag, Properties: util.MessageProperties2String(message.Properties), diff --git a/rocketmq-go/util/message_client_id_generator.go b/rocketmq-go/util/message_client_id_generator.go index 23293c0dd..c0c53df80 100644 --- a/rocketmq-go/util/message_client_id_generator.go +++ b/rocketmq-go/util/message_client_id_generator.go @@ -28,8 +28,8 @@ import ( var ( counter int16 = 0 - startTime int64 //this month's first day 12 hour. for example. 2017-01-01 12:00:00 - nextStartTime int64 //next month's first day 12 hour. for example. 2017-02-01 12:00:00 + startTime int64 + nextStartTime int64 idPrefix string lock sync.Mutex ) @@ -40,7 +40,7 @@ var ( //4 bytes for classloaderid(for java,go put 0) //2 bytes for counter, -//4 bytes for timediff, //(time.Now().UnixNano() - startTime) / 1000000) divide 1000000 because golang is different with java +//4 bytes for timediff, //(time.Now().UnixNano() - startTime) / 1000000) divide 1000000 because use time millis func GeneratorMessageClientId() (uniqMessageId string) { defer lock.Unlock() lock.Lock() diff --git a/rocketmq-go/util/time_util.go b/rocketmq-go/util/time_util.go new file mode 100644 index 000000000..ab8987d6a --- /dev/null +++ b/rocketmq-go/util/time_util.go @@ -0,0 +1,15 @@ +package util + +import ( + "time" + "strconv" +) + +func CurrentTimeMillisInt64() (ret int64) { + ret = time.Now().UnixNano() / 1000000 + return +} +func CurrentTimeMillisStr() (ret string) { + ret = strconv.FormatInt(CurrentTimeMillisInt64(), 10) + return +} From 34760b647dffb3fa427ac898d475e8b814596e10 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 12 Aug 2017 15:05:49 +0800 Subject: [PATCH 18/88] go api config --- rocketmq-go/aa/producer_consumer_example.go | 80 +++++++++++++++++++ rocketmq-go/api/model/consumer_config.go | 16 ++-- rocketmq-go/api/model/message_ext.go | 3 + rocketmq-go/api/model/producer_config.go | 18 ++--- rocketmq-go/api/rocketmq_consumer.go | 10 ++- rocketmq-go/api/rocketmq_controller.go | 3 - rocketmq-go/api/rocketmq_producer.go | 5 +- .../example/producer_consumer_example.go | 14 ++-- .../example/rocketmq_client_example.go | 8 +- rocketmq-go/manage/mq_client_manage.go | 4 - rocketmq-go/manage/mq_producer.go | 8 +- rocketmq-go/manage/mq_push_consumer.go | 12 +-- rocketmq-go/producer_consumer_example.go | 1 - rocketmq-go/producer_consumer_example_test.go | 5 +- .../service/consume_message_service.go | 2 +- rocketmq-go/util/time_util.go | 2 +- 16 files changed, 129 insertions(+), 62 deletions(-) create mode 100644 rocketmq-go/aa/producer_consumer_example.go diff --git a/rocketmq-go/aa/producer_consumer_example.go b/rocketmq-go/aa/producer_consumer_example.go new file mode 100644 index 000000000..0eb4a6d09 --- /dev/null +++ b/rocketmq-go/aa/producer_consumer_example.go @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + + "github.com/golang/glog" + "net/http" + _ "net/http/pprof" + "time" +) + +func main() { + go func() { + http.ListenAndServe("localhost:6060", nil) + }() + var ( + testTopic = "GoLang" + ) + + var producerConfig = rocketmq_api_model.NewProducerConfig() + producerConfig.CompressMsgBodyOverHowMuch = 1 + var producer1 = rocketmq_api.NewDefaultMQProducer("Test1", producerConfig) + var producer2 = rocketmq_api.NewDefaultMQProducer("Test2", producerConfig) + var consumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() + //for test + consumerConfig.PullInterval = 0 + consumerConfig.ConsumeTimeout = 1 + consumerConfig.ConsumeMessageBatchMaxSize = 16 + consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP + consumerConfig.ConsumeTimestamp = time.Now() + var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic+"-StyleTang", consumerConfig) + comsumer1.Subscribe(testTopic, "*") + comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + for _, msg := range msgs { + glog.Info(msg.BornTimestamp) + } + glog.Info("look message len ", len(msgs)) + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} + }) + var clienConfig = &rocketmq_api_model.ClientConfig{} + clienConfig.SetNameServerAddress("127.0.0.1:9876") + //clienConfig// todo + rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) + rocketMqManager.RegistProducer(producer1) + rocketMqManager.RegistProducer(producer2) + rocketMqManager.RegistConsumer(comsumer1) + rocketMqManager.Start() + for i := 0; i < 10000000; i++ { + var message = &rocketmq_api_model.Message{} + message.Topic = testTopic + message.SetKeys([]string{"xxx"}) + message.SetTag("1122") + message.Body = []byte("hellAXXWord") + + xx, ee := producer1.Send(message) + if ee != nil { + glog.Error(ee) + continue + } + glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) + } + select {} +} diff --git a/rocketmq-go/api/model/consumer_config.go b/rocketmq-go/api/model/consumer_config.go index 3b79d7ae2..c28bf5056 100644 --- a/rocketmq-go/api/model/consumer_config.go +++ b/rocketmq-go/api/model/consumer_config.go @@ -32,20 +32,16 @@ const PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL int64 = 50 type ConsumeFromWhere int //first consume from the last offset -const( - CONSUME_FROM_LAST_OFFSET ConsumeFromWhere = iota +const ( + CONSUME_FROM_LAST_OFFSET ConsumeFromWhere = iota -//first consume from the first offset - CONSUME_FROM_FIRST_OFFSET + //first consume from the first offset + CONSUME_FROM_FIRST_OFFSET -//first consume from the time - CONSUME_FROM_TIMESTAMP + //first consume from the time + CONSUME_FROM_TIMESTAMP ) - - - - type RocketMqConsumerConfig struct { /** * consume from where diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index bf8d328e0..6381b6dae 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -67,3 +67,6 @@ func (self *MessageExt) SetConsumeStartTime() { self.propertyConsumeStartTimestamp = nowTime return } + + + diff --git a/rocketmq-go/api/model/producer_config.go b/rocketmq-go/api/model/producer_config.go index 233e2d013..708495bba 100644 --- a/rocketmq-go/api/model/producer_config.go +++ b/rocketmq-go/api/model/producer_config.go @@ -17,16 +17,16 @@ package rocketmq_api_model type RocketMqProducerConfig struct { - SendMsgTimeout int64 - CompressMsgBodyOverHowMuch int - ZipCompressLevel int - RetryTimesWhenSendFailed int - RetryTimesWhenSendAsyncFailed int + SendMsgTimeout int64 + CompressMsgBodyOverHowMuch int + ZipCompressLevel int + RetryTimesWhenSendFailed int + RetryTimesWhenSendAsyncFailed int RetryAnotherBrokerWhenNotStoreOK bool - MaxMessageSize int - SendLatencyFaultEnable bool //false - LatencyMax []int64 //= {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L}; - NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; + MaxMessageSize int + SendLatencyFaultEnable bool //false + LatencyMax []int64 //= {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L}; + NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; } //set defaultValue diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index caf9f299f..08232d851 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -1,17 +1,19 @@ package rocketmq_api import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) type RocketMQConsumer interface { RegisterMessageListener(listener model.MessageListener) Subscribe(topic string, subExpression string) - GetConsumerConfig() (*rocketmq_api_model.RocketMqConsumerConfig) } -func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { - return rocketmq.NewDefaultMQPushConsumer(producerGroup) +//func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { +// return rocketmq.NewDefaultMQPushConsumer(producerGroup) +//} +func NewDefaultMQPushConsumer(producerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (r RocketMQConsumer) { + return rocketmq.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_controller.go b/rocketmq-go/api/rocketmq_controller.go index 8d0cad422..ef7603d6d 100644 --- a/rocketmq-go/api/rocketmq_controller.go +++ b/rocketmq-go/api/rocketmq_controller.go @@ -26,6 +26,3 @@ func (self *RocketMQController) Start() { self.rocketMqManager.Start() } -func (self *RocketMQController) ShutDown() { - self.rocketMqManager.ShutDown() -} diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 66b33a1eb..18c06f7dd 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -7,11 +7,10 @@ import ( ) type RocketMQProducer interface { - GetProducerConfig()( *rocketmq_api_model.RocketMqProducerConfig) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) } -func NewDefaultMQProducer(producerGroup string) (r RocketMQProducer) { - return rocketmq.NewDefaultMQProducer(producerGroup) +func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (r RocketMQProducer) { + return rocketmq.NewDefaultMQProducer(producerGroup, producerConfig) } diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go index 8e1357a9e..bf2dc2391 100644 --- a/rocketmq-go/example/producer_consumer_example.go +++ b/rocketmq-go/example/producer_consumer_example.go @@ -17,13 +17,13 @@ package main import ( - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - // - //"github.com/golang/glog" - //"net/http" - //_ "net/http/pprof" - //"time" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +// +//"github.com/golang/glog" +//"net/http" +//_ "net/http/pprof" +//"time" ) func main() { diff --git a/rocketmq-go/example/rocketmq_client_example.go b/rocketmq-go/example/rocketmq_client_example.go index a78479b4a..3f5f7439d 100644 --- a/rocketmq-go/example/rocketmq_client_example.go +++ b/rocketmq-go/example/rocketmq_client_example.go @@ -17,10 +17,10 @@ package main import ( - //"fmt" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" +//"fmt" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" ) func main() { diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index 5f3126d25..9b1e215ab 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -196,10 +196,6 @@ func (self *MqClientManager) Start() { self.StartAllScheduledTask() } -func (self MqClientManager) ShutDown() { - - -} type ClientFactory struct { ProducerTable map[string]*DefaultMQProducer //group|RocketMQProducer diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index 9fa7acd6b..cc0116d84 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -15,16 +15,14 @@ type DefaultMQProducer struct { producerService service.ProducerService } -func NewDefaultMQProducer(producerGroup string) (rocketMQProducer *DefaultMQProducer) { +func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (rocketMQProducer *DefaultMQProducer) { rocketMQProducer = &DefaultMQProducer{ producerGroup: producerGroup, - ProducerConfig: rocketmq_api_model.NewProducerConfig(), + ProducerConfig: producerConfig, } return } -func (self *DefaultMQProducer) GetProducerConfig()( *rocketmq_api_model.RocketMqProducerConfig){ -return self.ProducerConfig -} + func (self *DefaultMQProducer) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) return diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index 00b280d21..d802f10ba 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -20,10 +20,10 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strings" "time" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" ) type DefaultMQPushConsumer struct { @@ -43,7 +43,7 @@ type DefaultMQPushConsumer struct { ConsumerConfig *rocketmq_api_model.RocketMqConsumerConfig } -func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *DefaultMQPushConsumer) { +func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { defaultMQPushConsumer = &DefaultMQPushConsumer{ consumerGroup: consumerGroup, //consumeFromWhere:"CONSUME_FROM_FIRST_OFFSET", //todo use config @@ -52,14 +52,10 @@ func NewDefaultMQPushConsumer(consumerGroup string) (defaultMQPushConsumer *Defa pause: false} defaultMQPushConsumer.subscription = make(map[string]string) defaultMQPushConsumer.subscriptionTag = make(map[string][]string) - defaultMQPushConsumer.ConsumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() - + defaultMQPushConsumer.ConsumerConfig = consumerConfig return } -func (self *DefaultMQPushConsumer) GetConsumerConfig()( *rocketmq_api_model.RocketMqConsumerConfig){ - return self.ConsumerConfig -} func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { self.subscription[topic] = subExpression if len(subExpression) == 0 || subExpression == "*" { @@ -118,7 +114,7 @@ func (self *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { } func (self *DefaultMQPushConsumer) CleanExpireMsg() { - nowTime := util.CurrentTimeMillisInt64()//will cause nowTime - consumeStartTime <0 ,but no matter + nowTime := util.CurrentTimeMillisInt64() //will cause nowTime - consumeStartTime <0 ,but no matter messageQueueList, processQueueList := self.rebalance.GetProcessQueueList() for messageQueueIndex, processQueue := range processQueueList { loop := processQueue.GetMsgCount() diff --git a/rocketmq-go/producer_consumer_example.go b/rocketmq-go/producer_consumer_example.go index c16eec4a6..2480a6464 100644 --- a/rocketmq-go/producer_consumer_example.go +++ b/rocketmq-go/producer_consumer_example.go @@ -45,7 +45,6 @@ func main() { consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP consumerConfig.ConsumeTimestamp = time.Now() - comsumer1.Subscribe(testTopic, "*") comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { for _, msg := range msgs { diff --git a/rocketmq-go/producer_consumer_example_test.go b/rocketmq-go/producer_consumer_example_test.go index 69efb4553..105888bc0 100644 --- a/rocketmq-go/producer_consumer_example_test.go +++ b/rocketmq-go/producer_consumer_example_test.go @@ -2,13 +2,14 @@ package main import "testing" import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) + func TestAdda(t *testing.T) { var clienConfig = &rocketmq_api_model.ClientConfig{} clienConfig.SetNameServerAddress("127.0.0.1:9876") - testa := rocketmq_api.InitRocketMQController(clienConfig); + testa := rocketmq_api.InitRocketMQController(clienConfig) fmt.Print(testa) } diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index e85ed592e..3a9b57a9b 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -20,8 +20,8 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/golang/glog" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" ) type ConsumeMessageService interface { diff --git a/rocketmq-go/util/time_util.go b/rocketmq-go/util/time_util.go index ab8987d6a..690bb636c 100644 --- a/rocketmq-go/util/time_util.go +++ b/rocketmq-go/util/time_util.go @@ -1,8 +1,8 @@ package util import ( - "time" "strconv" + "time" ) func CurrentTimeMillisInt64() (ret int64) { From 3d01651a2e571a00a05ee780b144d2985ff4b4d7 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 12 Aug 2017 18:00:03 +0800 Subject: [PATCH 19/88] update rocketmq's example --- rocketmq-go/aa/producer_consumer_example.go | 80 ------------------ rocketmq-go/api/model/mq_client_config.go | 25 ++++++ ...nsumer_config.go => mq_consumer_config.go} | 0 ...oducer_config.go => mq_producer_config.go} | 0 ...fig_test.go => mq_producer_config_test.go} | 0 rocketmq-go/api/rocketmq_clent_instance.go | 33 ++++++++ ...est.go => rocketmq_clent_instance_test.go} | 0 rocketmq-go/api/rocketmq_consumer.go | 9 ++- rocketmq-go/api/rocketmq_controller.go | 28 ------- rocketmq-go/api/rocketmq_producer.go | 6 +- .../example/producer_consumer_example.go | 81 ------------------- rocketmq-go/example/producer_example.go | 57 ------------- .../example/rocketmq_client_example.go | 40 --------- rocketmq-go/example/simple_consumer.go | 54 +++++++++++++ rocketmq-go/example/simple_producer.go | 44 ++++++++++ .../example/simple_producer_consumer.go | 63 +++++++++++++++ rocketmq-go/manage/mq_client_manage.go | 2 +- rocketmq-go/{api => }/model/client_config.go | 2 +- rocketmq-go/producer_consumer_example.go | 24 +++--- rocketmq-go/producer_consumer_example_test.go | 4 +- rocketmq-go/remoting/remoting_client.go | 8 +- rocketmq-go/service/client_api.go | 2 +- rocketmq-go/service/mq_client.go | 2 +- 23 files changed, 249 insertions(+), 315 deletions(-) delete mode 100644 rocketmq-go/aa/producer_consumer_example.go create mode 100644 rocketmq-go/api/model/mq_client_config.go rename rocketmq-go/api/model/{consumer_config.go => mq_consumer_config.go} (100%) rename rocketmq-go/api/model/{producer_config.go => mq_producer_config.go} (100%) rename rocketmq-go/api/model/{producer_config_test.go => mq_producer_config_test.go} (100%) create mode 100644 rocketmq-go/api/rocketmq_clent_instance.go rename rocketmq-go/api/{rocketmq_controller_test.go => rocketmq_clent_instance_test.go} (100%) delete mode 100644 rocketmq-go/api/rocketmq_controller.go delete mode 100644 rocketmq-go/example/producer_consumer_example.go delete mode 100644 rocketmq-go/example/producer_example.go delete mode 100644 rocketmq-go/example/rocketmq_client_example.go create mode 100644 rocketmq-go/example/simple_consumer.go create mode 100644 rocketmq-go/example/simple_producer.go create mode 100644 rocketmq-go/example/simple_producer_consumer.go rename rocketmq-go/{api => }/model/client_config.go (99%) diff --git a/rocketmq-go/aa/producer_consumer_example.go b/rocketmq-go/aa/producer_consumer_example.go deleted file mode 100644 index 0eb4a6d09..000000000 --- a/rocketmq-go/aa/producer_consumer_example.go +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package main - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - - "github.com/golang/glog" - "net/http" - _ "net/http/pprof" - "time" -) - -func main() { - go func() { - http.ListenAndServe("localhost:6060", nil) - }() - var ( - testTopic = "GoLang" - ) - - var producerConfig = rocketmq_api_model.NewProducerConfig() - producerConfig.CompressMsgBodyOverHowMuch = 1 - var producer1 = rocketmq_api.NewDefaultMQProducer("Test1", producerConfig) - var producer2 = rocketmq_api.NewDefaultMQProducer("Test2", producerConfig) - var consumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() - //for test - consumerConfig.PullInterval = 0 - consumerConfig.ConsumeTimeout = 1 - consumerConfig.ConsumeMessageBatchMaxSize = 16 - consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP - consumerConfig.ConsumeTimestamp = time.Now() - var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic+"-StyleTang", consumerConfig) - comsumer1.Subscribe(testTopic, "*") - comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { - for _, msg := range msgs { - glog.Info(msg.BornTimestamp) - } - glog.Info("look message len ", len(msgs)) - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} - }) - var clienConfig = &rocketmq_api_model.ClientConfig{} - clienConfig.SetNameServerAddress("127.0.0.1:9876") - //clienConfig// todo - rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) - rocketMqManager.RegistProducer(producer1) - rocketMqManager.RegistProducer(producer2) - rocketMqManager.RegistConsumer(comsumer1) - rocketMqManager.Start() - for i := 0; i < 10000000; i++ { - var message = &rocketmq_api_model.Message{} - message.Topic = testTopic - message.SetKeys([]string{"xxx"}) - message.SetTag("1122") - message.Body = []byte("hellAXXWord") - - xx, ee := producer1.Send(message) - if ee != nil { - glog.Error(ee) - continue - } - glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - } - select {} -} diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go new file mode 100644 index 000000000..2d801ddb8 --- /dev/null +++ b/rocketmq-go/api/model/mq_client_config.go @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package rocketmq_api_model + + +// client common config +type MqClientConfig struct { + NameServerAddress string // NameServerAddress split by ; +} + diff --git a/rocketmq-go/api/model/consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go similarity index 100% rename from rocketmq-go/api/model/consumer_config.go rename to rocketmq-go/api/model/mq_consumer_config.go diff --git a/rocketmq-go/api/model/producer_config.go b/rocketmq-go/api/model/mq_producer_config.go similarity index 100% rename from rocketmq-go/api/model/producer_config.go rename to rocketmq-go/api/model/mq_producer_config.go diff --git a/rocketmq-go/api/model/producer_config_test.go b/rocketmq-go/api/model/mq_producer_config_test.go similarity index 100% rename from rocketmq-go/api/model/producer_config_test.go rename to rocketmq-go/api/model/mq_producer_config_test.go diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go new file mode 100644 index 000000000..7075e4f7a --- /dev/null +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -0,0 +1,33 @@ +package rocketmq_api + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" +) + +type RocketMQClientInstance interface { + RegisterProducer(producer RocketMQProducer) + RegisterConsumer(consumer RocketMQConsumer) + Start() +} + +type RocketMQClientInstanceImpl struct { + rocketMqManager *rocketmq.MqClientManager +} + +func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance RocketMQClientInstance) { + var mqClientConfig = &rocketmq_api_model.MqClientConfig{NameServerAddress: nameServerAddress} + rocketMQClientInstance = &RocketMQClientInstanceImpl{rocketMqManager: rocketmq.MqClientManagerInit(mqClientConfig)} + return +} + +func (self *RocketMQClientInstanceImpl) RegisterProducer(producer RocketMQProducer) { + self.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) +} + +func (self *RocketMQClientInstanceImpl) RegisterConsumer(consumer RocketMQConsumer) { + self.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) +} +func (self *RocketMQClientInstanceImpl) Start() { + self.rocketMqManager.Start() +} diff --git a/rocketmq-go/api/rocketmq_controller_test.go b/rocketmq-go/api/rocketmq_clent_instance_test.go similarity index 100% rename from rocketmq-go/api/rocketmq_controller_test.go rename to rocketmq-go/api/rocketmq_clent_instance_test.go diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 08232d851..6b691d74c 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -11,9 +11,10 @@ type RocketMQConsumer interface { Subscribe(topic string, subExpression string) } -//func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { -// return rocketmq.NewDefaultMQPushConsumer(producerGroup) -//} -func NewDefaultMQPushConsumer(producerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (r RocketMQConsumer) { +func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { + return rocketmq.NewDefaultMQPushConsumer(producerGroup, rocketmq_api_model.NewRocketMqConsumerConfig()) +} + +func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (r RocketMQConsumer) { return rocketmq.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_controller.go b/rocketmq-go/api/rocketmq_controller.go deleted file mode 100644 index ef7603d6d..000000000 --- a/rocketmq-go/api/rocketmq_controller.go +++ /dev/null @@ -1,28 +0,0 @@ -package rocketmq_api - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" -) - -type RocketMQController struct { - rocketMqManager *rocketmq.MqClientManager -} - -func InitRocketMQController(clientConfig *rocketmq_api_model.ClientConfig) (rocketMQController *RocketMQController) { - rocketMQController = &RocketMQController{} - rocketMQController.rocketMqManager = rocketmq.MqClientManagerInit(clientConfig) - return - -} -func (self *RocketMQController) RegistProducer(producer RocketMQProducer) { - self.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) -} - -func (self *RocketMQController) RegistConsumer(consumer RocketMQConsumer) { - self.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) -} -func (self *RocketMQController) Start() { - self.rocketMqManager.Start() -} - diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 18c06f7dd..94e0e2963 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -11,6 +11,10 @@ type RocketMQProducer interface { SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) } -func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (r RocketMQProducer) { +func NewDefaultMQProducer(producerGroup string) (r RocketMQProducer) { + return rocketmq.NewDefaultMQProducer(producerGroup, rocketmq_api_model.NewProducerConfig()) +} + +func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (r RocketMQProducer) { return rocketmq.NewDefaultMQProducer(producerGroup, producerConfig) } diff --git a/rocketmq-go/example/producer_consumer_example.go b/rocketmq-go/example/producer_consumer_example.go deleted file mode 100644 index bf2dc2391..000000000 --- a/rocketmq-go/example/producer_consumer_example.go +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package main - -import ( -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" -// -//"github.com/golang/glog" -//"net/http" -//_ "net/http/pprof" -//"time" -) - -func main() { - //go func() { - // http.ListenAndServe("localhost:6060", nil) - //}() - //var ( - // testTopic = "GoLang" - //) - //var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") - //producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 - //var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") - //var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") - ////for test - //consumerConfig := comsumer1.GetConsumerConfig() - //consumerConfig.PullInterval = 0 - //consumerConfig.ConsumeTimeout = 1 - //consumerConfig.ConsumeMessageBatchMaxSize = 16 - //consumerConfig.ConsumeFromWhere = "CONSUME_FROM_TIMESTAMP" - //consumerConfig.ConsumeTimestamp = time.Now() - // - // - //comsumer1.Subscribe(testTopic, "*") - //comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { - // for _, msg := range msgs { - // glog.Info(msg.BornTimestamp) - // } - // glog.Info("look message len ", len(msgs)) - // return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} - //}) - //var clienConfig = &rocketmq_api_model.ClientConfig{} - //clienConfig.SetNameServerAddress("127.0.0.1:9876") - ////clienConfig// todo - //rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) - //rocketMqManager.RegistProducer(producer1) - //rocketMqManager.RegistProducer(producer2) - //rocketMqManager.RegistConsumer(comsumer1) - //rocketMqManager.Start() - //for i := 0; i < 10000000; i++ { - // var message = &rocketmq_api_model.Message{} - // message.Topic = testTopic - // message.SetKeys([]string{"xxx"}) - // message.SetTag("1122") - // message.Body = []byte("hellAXXWord") - // - // xx, ee := producer1.Send(message) - // if ee != nil { - // glog.Error(ee) - // continue - // } - // glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - //} - //select {} - //rocketMqManager.ShutDown() -} diff --git a/rocketmq-go/example/producer_example.go b/rocketmq-go/example/producer_example.go deleted file mode 100644 index ab3620cc8..000000000 --- a/rocketmq-go/example/producer_example.go +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package main - -import ( - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - //"github.com/golang/glog" - _ "net/http/pprof" -) - -func main() { - //var ( - // testTopic = "GoLang" - //) - //var producer1 = rocketmq.NewDefaultMQProducer("Test1") - //producer1.ProducerConfig.CompressMsgBodyOverHowMuch = 1 - //var producer2 = rocketmq.NewDefaultMQProducer("Test2") - //var clientConfig = &rocketmq_api_model.ClientConfig{} - //clientConfig.SetNameServerAddress("127.0.0.1:9876") - //rocketMqManager := rocketmq.MqClientManagerInit(clientConfig) - //rocketMqManager.RegistProducer(producer1) - //rocketMqManager.RegistProducer(producer2) - //rocketMqManager.Start() - //for i := 0; i < 1000; i++ { - // var message = &model.Message{} - // message.Topic = testTopic - // message.SetKeys([]string{"xxx"}) - // message.SetTag("1122") - // message.Body = []byte("hellAXXWord" + util.IntToString(i)) - // - // xx, ee := producer1.Send(message) - // if ee != nil { - // glog.Error(ee) - // continue - // } - // glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - //} - //select {} - //rocketMqManager.ShutDown() -} diff --git a/rocketmq-go/example/rocketmq_client_example.go b/rocketmq-go/example/rocketmq_client_example.go deleted file mode 100644 index 3f5f7439d..000000000 --- a/rocketmq-go/example/rocketmq_client_example.go +++ /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. - */ -package main - -import ( -//"fmt" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/config" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" -) - -func main() { - // - //var clienConfig = config.NewClientConfig() - //clienConfig.SetNameServerAddress("127.0.0.1:9876") - // - ////use json serializer - //var mqClient = service.MqClientInit(clienConfig, nil) - //fmt.Println(mqClient.TryToFindTopicPublishInfo("GoLang")) - ////&{false true [{GoLang broker-a 0} {GoLang broker-a 1} {GoLang broker-a 2} {GoLang broker-a 3}] 0xc420016800 0} - // - ////use rocketmq serializer - //constant.USE_HEADER_SERIALIZETYPE = constant.ROCKETMQ_SERIALIZE - //var mqClient2 = service.MqClientInit(clienConfig, nil) - //fmt.Println(mqClient2.TryToFindTopicPublishInfo("GoLang")) -} diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go new file mode 100644 index 000000000..b997347d9 --- /dev/null +++ b/rocketmq-go/example/simple_consumer.go @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" +) + +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + testTopic = "GoLangRocketMQ" + testConsumerGroup = "TestConsumerGroup" + ) + // init rocketMQClientInstance + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + + // 1.init rocketMQConsumer + // 2.subscribe topic and register our function to message listener + // 3.register it + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "*") + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) + // call your function + successIndex = index + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + + // start rocketMQ client instance + rocketMQClientInstance.Start() + + select {} +} diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go new file mode 100644 index 000000000..2582da193 --- /dev/null +++ b/rocketmq-go/example/simple_producer.go @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" +) + +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestProducerGroup" + ) + // init rocketMQClientInstance + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + // init rocketMQProducer and register it + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + + // start rocketMQ client instance + rocketMQClientInstance.Start() + + //start send test message + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) +} diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go new file mode 100644 index 000000000..a3e8dc36f --- /dev/null +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" +) + +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestProducerGroup" + testConsumerGroup = "TestConsumerGroup" + ) + // init rocketMQClientInstance + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + // init rocketMQProducer and register it + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + + // 1.init rocketMQConsumer + // 2.subscribe topic and register our function to message listener + // 3.register it + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "*") + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) + // call your function + successIndex = index + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + + // start rocketMQ client instance + rocketMQClientInstance.Start() + + //start send test message + for { + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + } +} diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index 9b1e215ab..d7764c8e7 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -61,7 +61,7 @@ type MqClientManager struct { defaultProducerService *service.DefaultProducerService //for send back message } -func MqClientManagerInit(clientConfig *rocketmq_api_model.ClientConfig) (rocketMqManager *MqClientManager) { +func MqClientManagerInit(clientConfig *rocketmq_api_model.MqClientConfig) (rocketMqManager *MqClientManager) { rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() rocketMqManager.clientFactory = ClientFactoryInit() diff --git a/rocketmq-go/api/model/client_config.go b/rocketmq-go/model/client_config.go similarity index 99% rename from rocketmq-go/api/model/client_config.go rename to rocketmq-go/model/client_config.go index 76f455b9f..017c6d91a 100644 --- a/rocketmq-go/api/model/client_config.go +++ b/rocketmq-go/model/client_config.go @@ -15,7 +15,7 @@ * limitations under the License. */ -package rocketmq_api_model +package model import ( "bytes" diff --git a/rocketmq-go/producer_consumer_example.go b/rocketmq-go/producer_consumer_example.go index 2480a6464..5d1d4841c 100644 --- a/rocketmq-go/producer_consumer_example.go +++ b/rocketmq-go/producer_consumer_example.go @@ -33,18 +33,18 @@ func main() { var ( testTopic = "GoLang" ) - var producer1 = rocketmq_api.NewDefaultMQProducer("Test1") - producer1.GetProducerConfig().CompressMsgBodyOverHowMuch = 1 - var producer2 = rocketmq_api.NewDefaultMQProducer("Test2") - var comsumer1 = rocketmq_api.NewDefaultMQPushConsumer(testTopic + "-StyleTang") + + var producerConfig = rocketmq_api_model.NewProducerConfig() + producerConfig.CompressMsgBodyOverHowMuch = 1 + var producer1 = rocketmq_api.NewDefaultMQProducerWithCustomConfig("Test1", producerConfig) + var consumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() //for test - consumerConfig := comsumer1.GetConsumerConfig() consumerConfig.PullInterval = 0 consumerConfig.ConsumeTimeout = 1 consumerConfig.ConsumeMessageBatchMaxSize = 16 consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP consumerConfig.ConsumeTimestamp = time.Now() - + var comsumer1 = rocketmq_api.NewDefaultMQPushConsumerWithCustomConfig(testTopic+"-StyleTang", consumerConfig) comsumer1.Subscribe(testTopic, "*") comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { for _, msg := range msgs { @@ -53,13 +53,10 @@ func main() { glog.Info("look message len ", len(msgs)) return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} }) - var clienConfig = &rocketmq_api_model.ClientConfig{} - clienConfig.SetNameServerAddress("127.0.0.1:9876") - //clienConfig// todo - rocketMqManager := rocketmq_api.InitRocketMQController(clienConfig) - rocketMqManager.RegistProducer(producer1) - rocketMqManager.RegistProducer(producer2) - rocketMqManager.RegistConsumer(comsumer1) + nameServerAddress := "127.0.0.1:9876" + rocketMqManager := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + rocketMqManager.RegisterProducer(producer1) + rocketMqManager.RegisterConsumer(comsumer1) rocketMqManager.Start() for i := 0; i < 10000000; i++ { var message = &rocketmq_api_model.Message{} @@ -76,5 +73,4 @@ func main() { glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) } select {} - rocketMqManager.ShutDown() } diff --git a/rocketmq-go/producer_consumer_example_test.go b/rocketmq-go/producer_consumer_example_test.go index 105888bc0..b900c9462 100644 --- a/rocketmq-go/producer_consumer_example_test.go +++ b/rocketmq-go/producer_consumer_example_test.go @@ -8,8 +8,8 @@ import ( ) func TestAdda(t *testing.T) { - var clienConfig = &rocketmq_api_model.ClientConfig{} + var clienConfig = &rocketmq_api_model.MqClientConfig{} clienConfig.SetNameServerAddress("127.0.0.1:9876") - testa := rocketmq_api.InitRocketMQController(clienConfig) + testa := rocketmq_api.InitRocketMQClientInstance(clienConfig) fmt.Print(testa) } diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 13a5eda55..021dd1f21 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -38,7 +38,7 @@ type RemotingClient interface { } type DefalutRemotingClient struct { clientId string - clientConfig *rocketmq_api_model.ClientConfig + clientConfig *rocketmq_api_model.MqClientConfig connTable map[string]net.Conn connTableLock sync.RWMutex @@ -55,13 +55,13 @@ type DefalutRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } -func RemotingClientInit(clientConfig *rocketmq_api_model.ClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefalutRemotingClient) { +func RemotingClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefalutRemotingClient) { client = &DefalutRemotingClient{} client.connTable = map[string]net.Conn{} client.responseTable = util.New() client.clientConfig = clientConfig - client.namesrvAddrList = strings.Split(clientConfig.NameServerAddress(), ";") + client.namesrvAddrList = strings.Split(clientConfig.NameServerAddress, ";") client.namesrvAddrSelectedIndex = -1 client.clientRequestProcessor = clientRequestProcessor client.serializerHandler = NewSerializerHandler() @@ -236,7 +236,7 @@ func (self *DefalutRemotingClient) getNamesvrConn() (conn net.Conn, err error) { return } } - err = errors.New("all namesvrAddress can't use!,address:" + self.clientConfig.NameServerAddress()) + err = errors.New("all namesvrAddress can't use!,address:" + self.clientConfig.NameServerAddress) return } func (self *DefalutRemotingClient) createAndHandleTcpConn(address string) (conn net.Conn, err error) { diff --git a/rocketmq-go/service/client_api.go b/rocketmq-go/service/client_api.go index d7b839fea..adf8a60f4 100644 --- a/rocketmq-go/service/client_api.go +++ b/rocketmq-go/service/client_api.go @@ -41,7 +41,7 @@ type MQClientAPI struct { topAddress *TopAddress crp *ClientRemotingProcessor nameServerAddress string - config *rocketmq_api_model.ClientConfig + config *rocketmq_api_model.MqClientConfig } //func NewMQClientAPI(cfg *rocketmq_api_model.ClientConfig, processor *ClientRemotingProcessor, hook remoting.RPCHook) *MQClientAPI { diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 8f9286e4d..47cf52e5b 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -69,7 +69,7 @@ type MqClientImpl struct { PullRequestQueue chan *model.PullRequest //todo move } -func MqClientInit(clientConfig *rocketmq_api_model.ClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { +func MqClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} mqClientImpl.ClientId = buildMqClientImplId() mqClientImpl.TopicRouteTable = util.New() // make(map[string]*model.TopicRouteData) From 5bc4e16dbef3ad496e82eae68e995ceccc613333 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 12 Aug 2017 18:05:58 +0800 Subject: [PATCH 20/88] update rocketmq's example --- rocketmq-go/api/model/message.go | 4 +--- rocketmq-go/api/model/message_ext.go | 5 +---- rocketmq-go/api/model/mq_client_config.go | 4 ---- rocketmq-go/api/model/mq_producer_config.go | 1 - rocketmq-go/api/model/mq_producer_config_test.go | 16 ++++++++++++++++ rocketmq-go/api/rocketmq_clent_instance.go | 16 ++++++++++++++++ rocketmq-go/api/rocketmq_clent_instance_test.go | 16 ++++++++++++++++ rocketmq-go/api/rocketmq_consumer.go | 16 ++++++++++++++++ rocketmq-go/api/rocketmq_producer.go | 16 ++++++++++++++++ rocketmq-go/manage/mq_client_manage.go | 1 - 10 files changed, 82 insertions(+), 13 deletions(-) diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 4e443377d..f3394afd9 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -47,7 +47,7 @@ func (self *Message) SetKeys(keys []string) { if self.Properties == nil { self.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, KEY_SEPARATOR) + self.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") } func (self *Message) SetDelayTimeLevel(delayTimeLevel int) { @@ -130,5 +130,3 @@ func (self *Message) GetMaxReconsumeTimes() (maxConsumeTime int) { } return } - -var KEY_SEPARATOR string = " " diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 6381b6dae..a747361b5 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -38,7 +38,7 @@ type MessageExt struct { ReconsumeTimes int32 PreparedTransactionOffset int64 - propertyConsumeStartTimestamp string // race condition + propertyConsumeStartTimestamp string } func (self *MessageExt) GetOriginMessageId() string { @@ -67,6 +67,3 @@ func (self *MessageExt) SetConsumeStartTime() { self.propertyConsumeStartTimestamp = nowTime return } - - - diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 2d801ddb8..7ceb84c45 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -14,12 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package rocketmq_api_model - -// client common config type MqClientConfig struct { NameServerAddress string // NameServerAddress split by ; } - diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index 708495bba..334ef9806 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -29,7 +29,6 @@ type RocketMqProducerConfig struct { NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; } -//set defaultValue func NewProducerConfig() (producerConfig *RocketMqProducerConfig) { producerConfig = &RocketMqProducerConfig{ SendMsgTimeout: 3000, diff --git a/rocketmq-go/api/model/mq_producer_config_test.go b/rocketmq-go/api/model/mq_producer_config_test.go index 34aba084f..0aac8fce7 100644 --- a/rocketmq-go/api/model/mq_producer_config_test.go +++ b/rocketmq-go/api/model/mq_producer_config_test.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package rocketmq_api_model import "testing" diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 7075e4f7a..49417ab41 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package rocketmq_api import ( diff --git a/rocketmq-go/api/rocketmq_clent_instance_test.go b/rocketmq-go/api/rocketmq_clent_instance_test.go index dae00e8d0..c7401d21b 100644 --- a/rocketmq-go/api/rocketmq_clent_instance_test.go +++ b/rocketmq-go/api/rocketmq_clent_instance_test.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package rocketmq_api import "testing" diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 6b691d74c..fb0ee3999 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package rocketmq_api import ( diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 94e0e2963..b087b93d1 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package rocketmq_api import ( diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index d7764c8e7..5254cd00a 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -196,7 +196,6 @@ func (self *MqClientManager) Start() { self.StartAllScheduledTask() } - type ClientFactory struct { ProducerTable map[string]*DefaultMQProducer //group|RocketMQProducer ConsumerTable map[string]*DefaultMQPushConsumer //group|Consumer From 5301148c82e07a84742cd8649e5ce4e620f6aeec Mon Sep 17 00:00:00 2001 From: Rock1994 <806714022@qq.com> Date: Sat, 12 Aug 2017 21:54:32 +0800 Subject: [PATCH 21/88] package util unit test --- rocketmq-go/util/compress_util_test.go | 38 +++++++++ rocketmq-go/util/ip_test.go | 37 +++++++++ rocketmq-go/util/json_util_test.go | 39 ++++++++++ .../util/message_client_id_generator_test.go | 39 ++++++++++ rocketmq-go/util/message_properties_test.go | 46 +++++++++++ rocketmq-go/util/regex_util_test.go | 37 +++++++++ rocketmq-go/util/string_util_test.go | 78 +++++++++++++++++++ 7 files changed, 314 insertions(+) create mode 100644 rocketmq-go/util/compress_util_test.go create mode 100644 rocketmq-go/util/ip_test.go create mode 100644 rocketmq-go/util/json_util_test.go create mode 100644 rocketmq-go/util/message_client_id_generator_test.go create mode 100644 rocketmq-go/util/message_properties_test.go create mode 100644 rocketmq-go/util/regex_util_test.go create mode 100644 rocketmq-go/util/string_util_test.go diff --git a/rocketmq-go/util/compress_util_test.go b/rocketmq-go/util/compress_util_test.go new file mode 100644 index 000000000..ada07af6d --- /dev/null +++ b/rocketmq-go/util/compress_util_test.go @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" +) + +func TestCompressWithLevel(t *testing.T) { + bytes := []byte{120, 156, 202, 72, 205, 201, 201, 215, 81, 40, 207, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147} + body1, error1 := CompressWithLevel(bytes, 5) + if len(body1) > len(bytes) && error1 != nil { + t.Errorf("CompressWithLevel failed : cannot transfer byte[]") + } + body2, error2 := CompressWithLevel(nil, 10) + if body2 != nil && error2 != nil { + t.Errorf("CompressWithLevel failed : cannot handle error") + } +} diff --git a/rocketmq-go/util/ip_test.go b/rocketmq-go/util/ip_test.go new file mode 100644 index 000000000..52caaf026 --- /dev/null +++ b/rocketmq-go/util/ip_test.go @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" + "strings" +) + +func TestGetIp4Bytes(t *testing.T) { + ip := GetIp4Bytes() + if ip == nil && len(ip) != 4 { + t.Error("GetIp4Bytes failed") + } +} + +func TestGetLocalIp4(t *testing.T) { + ip4 := GetLocalIp4() + split := strings.Split(ip4, ".") + if len(split) != 4 { + t.Errorf("GetLocalIp4 failed") + } +} diff --git a/rocketmq-go/util/json_util_test.go b/rocketmq-go/util/json_util_test.go new file mode 100644 index 000000000..6b4f2a262 --- /dev/null +++ b/rocketmq-go/util/json_util_test.go @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" +) + +func TestGetKvStringMap(t *testing.T) { + s1 := "{\"key1\":\"value1\",\"key2\":\"value2\"}" + kvMap, err := GetKvStringMap(s1) + if kvMap["\"key1\""] != "\"value1\"" || kvMap["\"key2\""] != "\"value2\"" || err != nil { + t.Errorf("TestGetKvStringMap failed : cannot transfer normal json") + } + s2 := "\"key1\":\"value1\",\"key2\":\"value2\"}" + kvMap2, err2 := GetKvStringMap(s2) + if len(kvMap2) != 0 || err2.Error() != "json not start with {" { + t.Errorf("TestGetKvStringMap failed : cannot found json error") + } + s3 := "{key1:value1,key2:value2}" + kvMap3, err3 := GetKvStringMap(s3) + if len(kvMap3) != 0 || err3.Error() != "INVALID JSON" { + t.Errorf("TestGetKvStringMap failed : cannot found invalidjson") + } +} diff --git a/rocketmq-go/util/message_client_id_generator_test.go b/rocketmq-go/util/message_client_id_generator_test.go new file mode 100644 index 000000000..df082c144 --- /dev/null +++ b/rocketmq-go/util/message_client_id_generator_test.go @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" +) + +func TestGeneratorMessageClientId(t *testing.T) { + id1 := GeneratorMessageClientId() + if id1 == "" { + t.Errorf("TestGeneratorMessageClientId failed") + } + id2 := GeneratorMessageClientId() + if id2 == "" || id1 == id2 { + t.Errorf("TestGeneratorMessageClientId failed : create same clientId") + } +} + +func TestGeneratorMessageOffsetId(t *testing.T) { + id := GeneratorMessageOffsetId([]byte{111}, 1, 1) + if id != "6F000000010000000000000001" { + t.Errorf("TestGeneratorMessageOffsetId failed") + } +} diff --git a/rocketmq-go/util/message_properties_test.go b/rocketmq-go/util/message_properties_test.go new file mode 100644 index 000000000..1de8a295e --- /dev/null +++ b/rocketmq-go/util/message_properties_test.go @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" +) + +func TestMessageProperties2String(t *testing.T) { + stringMap := make(map[string]string) + stringMap["key1"] = "value1" + stringMap["key2"] = "value2" + var NAME_VALUE_SEPARATOR = string(rune(1)) + var PROPERTY_SEPARATOR = string(rune(2)) + s1 := "key1" + NAME_VALUE_SEPARATOR + "value1" + PROPERTY_SEPARATOR + "key2" + + NAME_VALUE_SEPARATOR + "value2" + PROPERTY_SEPARATOR + ret := MessageProperties2String(stringMap) + if ret != s1 { + t.Errorf("TestMessageProperties2String failed") + } +} + +func TestString2MessageProperties(t *testing.T) { + var NAME_VALUE_SEPARATOR = string(rune(1)) + var PROPERTY_SEPARATOR = string(rune(2)) + s1 := "key1" + NAME_VALUE_SEPARATOR + "value1" + PROPERTY_SEPARATOR + "key2" + + NAME_VALUE_SEPARATOR + "value2" + mapString := String2MessageProperties(s1) + if mapString["key1"] != "value1" || mapString["key2"] != "value2" { + t.Errorf("TestString2MessageProperties failed") + } +} diff --git a/rocketmq-go/util/regex_util_test.go b/rocketmq-go/util/regex_util_test.go new file mode 100644 index 000000000..fc228776b --- /dev/null +++ b/rocketmq-go/util/regex_util_test.go @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" +) + +func TestMatchString(t *testing.T) { + s1 := "123" + s2 := "123qwe" + pattern := "^[0-9]+$" + notPattern := "\\123\\" + result1 := MatchString(s1, pattern) + result2 := MatchString(s2, pattern) + if !result1 || result2 { + t.Errorf("TestMatchString failed : cannot match") + } + result3 := MatchString(s1, notPattern) + if result3 { + t.Errorf("TestMatchString failed : cannot find pattern mistake") + } +} diff --git a/rocketmq-go/util/string_util_test.go b/rocketmq-go/util/string_util_test.go new file mode 100644 index 000000000..537cdab78 --- /dev/null +++ b/rocketmq-go/util/string_util_test.go @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package util + +import ( + "testing" + "strings" +) + +func TestStrToIntWithDefaultValue(t *testing.T) { + result1 := StrToIntWithDefaultValue("1", 0) + if result1 != 1 { + t.Errorf("StrToIntWithDefaultValue failed : cannot transfer string 1 to int 1") + } + result2 := StrToIntWithDefaultValue("", 2) + if result2 != 2 { + t.Errorf("StrToIntWithDefaultValue failed : cannot use default value") + } +} + +func TestReadString(t *testing.T) { + var i interface{} = "testReadString" + ret := ReadString(i) + if !strings.EqualFold(ret, "testReadString") { + t.Errorf("TestReadString failed : cannot transfer string") + } + ret2 := ReadString(nil) + if ret2 != "" { + t.Errorf("TestReadString failed : cannot transfer nil to \"\""); + } +} + +func TestIntToString(t *testing.T) { + ret := IntToString(1) + if ret != "1" { + t.Errorf("TestIntToString failed : cannot tansfer int 1 to string1") + } +} + +func TestStrToInt32WithDefaultValue(t *testing.T) { + i1 := int32(2147483646) + i2 := int32(2147483647) + result1 := StrToInt32WithDefaultValue("2147483646", i2) + if result1 != i1 { + t.Errorf("StrToIntWithDefaultValue failed : cannot transfer string 2147483646 to int32 2147483646") + } + result2 := StrToInt32WithDefaultValue("2147483648", i2) + if result2 != i2 { + t.Errorf("StrToIntWithDefaultValue failed : cannot use default value") + } +} + +func TestStrToInt64WithDefaultValue(t *testing.T) { + i1 := int64(9223372036854775806) + i2 := int64(9223372036854775807) + result1 := StrToInt64WithDefaultValue("9223372036854775806", i2) + if result1 != i1 { + t.Errorf("StrToIntWithDefaultValue failed : cannot transfer string 9223372036854775806 to int64 9223372036854775806") + } + result2 := StrToInt64WithDefaultValue("9223372036854775808", i2) + if result2 != i2 { + t.Errorf("StrToIntWithDefaultValue failed : use default value") + } +} From 6165d4ea36c9c1c0992db7a9a111739b985dee55 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 12 Aug 2017 23:33:48 +0800 Subject: [PATCH 22/88] make SerializeType can be config --- rocketmq-go/api/model/mq_client_config.go | 20 ++- rocketmq-go/api/model/mq_producer_config.go | 8 +- rocketmq-go/api/rocketmq_clent_instance.go | 6 +- rocketmq-go/api/rocketmq_consumer.go | 2 +- rocketmq-go/api/rocketmq_producer.go | 2 +- rocketmq-go/model/constant/config.go | 8 - rocketmq-go/producer_consumer_example.go | 76 -------- rocketmq-go/producer_consumer_example_test.go | 15 -- rocketmq-go/remoting/json_serializable.go | 4 +- .../remoting/json_serializable_test.go | 57 ++++++ rocketmq-go/remoting/remoting_client.go | 167 +++++++++--------- rocketmq-go/remoting/remoting_client_test.go | 31 ++++ rocketmq-go/remoting/rocketmq_serializable.go | 21 ++- .../remoting/rocketmq_serializable_test.go | 40 +++++ rocketmq-go/remoting/serializable.go | 25 +-- rocketmq-go/service/mq_client.go | 6 +- 16 files changed, 272 insertions(+), 216 deletions(-) delete mode 100644 rocketmq-go/producer_consumer_example.go delete mode 100644 rocketmq-go/producer_consumer_example_test.go create mode 100644 rocketmq-go/remoting/json_serializable_test.go create mode 100644 rocketmq-go/remoting/remoting_client_test.go create mode 100644 rocketmq-go/remoting/rocketmq_serializable_test.go diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 7ceb84c45..5966a902e 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -16,6 +16,24 @@ */ package rocketmq_api_model +//-------SerializeType------- +// default serialize type is JSON_SERIALIZE, but ROCKETMQ_SERIALIZE(need version >= ?) is faster +type SerializeType byte + +const ( + JSON_SERIALIZE SerializeType = iota + ROCKETMQ_SERIALIZE +) type MqClientConfig struct { - NameServerAddress string // NameServerAddress split by ; + // NameServerAddress split by ; + NameServerAddress string + ClientSerializeType SerializeType +} + +func NewMqClientConfig(nameServerAddress string) (mqClientConfig *MqClientConfig) { + mqClientConfig = &MqClientConfig{ + NameServerAddress: nameServerAddress, + ClientSerializeType: JSON_SERIALIZE, + } + return } diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index 334ef9806..a2cc7ce13 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -24,9 +24,9 @@ type RocketMqProducerConfig struct { RetryTimesWhenSendAsyncFailed int RetryAnotherBrokerWhenNotStoreOK bool MaxMessageSize int - SendLatencyFaultEnable bool //false - LatencyMax []int64 //= {50L, 100L, 550L, 1000L, 2000L, 3000L, 15000L}; - NotAvailableDuration []int64 // {0L, 0L, 30000L, 60000L, 120000L, 180000L, 600000L}; + SendLatencyFaultEnable bool + LatencyMax []int64 + NotAvailableDuration []int64 } func NewProducerConfig() (producerConfig *RocketMqProducerConfig) { @@ -37,7 +37,7 @@ func NewProducerConfig() (producerConfig *RocketMqProducerConfig) { MaxMessageSize: 1024 * 1024 * 4, // 4M RetryTimesWhenSendFailed: 2, - RetryTimesWhenSendAsyncFailed: 2, // + RetryTimesWhenSendAsyncFailed: 2, RetryAnotherBrokerWhenNotStoreOK: false, SendLatencyFaultEnable: false, LatencyMax: []int64{50, 100, 550, 1000, 2000, 3000, 15000}, diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 49417ab41..7cc0fb6bb 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -32,11 +32,15 @@ type RocketMQClientInstanceImpl struct { } func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance RocketMQClientInstance) { - var mqClientConfig = &rocketmq_api_model.MqClientConfig{NameServerAddress: nameServerAddress} + mqClientConfig := rocketmq_api_model.NewMqClientConfig(nameServerAddress) + return InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) +} +func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmq_api_model.MqClientConfig) (rocketMQClientInstance RocketMQClientInstance) { rocketMQClientInstance = &RocketMQClientInstanceImpl{rocketMqManager: rocketmq.MqClientManagerInit(mqClientConfig)} return } + func (self *RocketMQClientInstanceImpl) RegisterProducer(producer RocketMQProducer) { self.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) } diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index fb0ee3999..a883e9280 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -28,7 +28,7 @@ type RocketMQConsumer interface { } func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { - return rocketmq.NewDefaultMQPushConsumer(producerGroup, rocketmq_api_model.NewRocketMqConsumerConfig()) + return NewDefaultMQPushConsumerWithCustomConfig(producerGroup, rocketmq_api_model.NewRocketMqConsumerConfig()) } func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (r RocketMQConsumer) { diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index b087b93d1..579026679 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -28,7 +28,7 @@ type RocketMQProducer interface { } func NewDefaultMQProducer(producerGroup string) (r RocketMQProducer) { - return rocketmq.NewDefaultMQProducer(producerGroup, rocketmq_api_model.NewProducerConfig()) + return NewDefaultMQProducerWithCustomConfig(producerGroup, rocketmq_api_model.NewProducerConfig()) } func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (r RocketMQProducer) { diff --git a/rocketmq-go/model/constant/config.go b/rocketmq-go/model/constant/config.go index 5f7f5dbba..80c57bb51 100644 --- a/rocketmq-go/model/constant/config.go +++ b/rocketmq-go/model/constant/config.go @@ -16,14 +16,6 @@ */ package constant -//-------SerializeType------- -var JSON_SERIALIZE byte = 0 -var ROCKETMQ_SERIALIZE byte = 1 - -//-------SerializeType------- - -var USE_HEADER_SERIALIZETYPE = JSON_SERIALIZE - var REMOTING_COMMAND_FLAG = 0 var REMOTING_COMMAND_LANGUAGE = "OTHER" var REMOTING_COMMAND_VERSION int16 = 213 diff --git a/rocketmq-go/producer_consumer_example.go b/rocketmq-go/producer_consumer_example.go deleted file mode 100644 index 5d1d4841c..000000000 --- a/rocketmq-go/producer_consumer_example.go +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package main - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - - "github.com/golang/glog" - "net/http" - _ "net/http/pprof" - "time" -) - -func main() { - go func() { - http.ListenAndServe("localhost:6060", nil) - }() - var ( - testTopic = "GoLang" - ) - - var producerConfig = rocketmq_api_model.NewProducerConfig() - producerConfig.CompressMsgBodyOverHowMuch = 1 - var producer1 = rocketmq_api.NewDefaultMQProducerWithCustomConfig("Test1", producerConfig) - var consumerConfig = rocketmq_api_model.NewRocketMqConsumerConfig() - //for test - consumerConfig.PullInterval = 0 - consumerConfig.ConsumeTimeout = 1 - consumerConfig.ConsumeMessageBatchMaxSize = 16 - consumerConfig.ConsumeFromWhere = rocketmq_api_model.CONSUME_FROM_TIMESTAMP - consumerConfig.ConsumeTimestamp = time.Now() - var comsumer1 = rocketmq_api.NewDefaultMQPushConsumerWithCustomConfig(testTopic+"-StyleTang", consumerConfig) - comsumer1.Subscribe(testTopic, "*") - comsumer1.RegisterMessageListener(func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { - for _, msg := range msgs { - glog.Info(msg.BornTimestamp) - } - glog.Info("look message len ", len(msgs)) - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: len(msgs)} - }) - nameServerAddress := "127.0.0.1:9876" - rocketMqManager := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - rocketMqManager.RegisterProducer(producer1) - rocketMqManager.RegisterConsumer(comsumer1) - rocketMqManager.Start() - for i := 0; i < 10000000; i++ { - var message = &rocketmq_api_model.Message{} - message.Topic = testTopic - message.SetKeys([]string{"xxx"}) - message.SetTag("1122") - message.Body = []byte("hellAXXWord") - - xx, ee := producer1.Send(message) - if ee != nil { - glog.Error(ee) - continue - } - glog.V(0).Infof("sendMessageResutl messageId[%s] err[%s]", xx.MsgID(), ee) - } - select {} -} diff --git a/rocketmq-go/producer_consumer_example_test.go b/rocketmq-go/producer_consumer_example_test.go deleted file mode 100644 index b900c9462..000000000 --- a/rocketmq-go/producer_consumer_example_test.go +++ /dev/null @@ -1,15 +0,0 @@ -package main - -import "testing" -import ( - "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" -) - -func TestAdda(t *testing.T) { - var clienConfig = &rocketmq_api_model.MqClientConfig{} - clienConfig.SetNameServerAddress("127.0.0.1:9876") - testa := rocketmq_api.InitRocketMQClientInstance(clienConfig) - fmt.Print(testa) -} diff --git a/rocketmq-go/remoting/json_serializable.go b/rocketmq-go/remoting/json_serializable.go index c2c5ea0f8..dc839b590 100644 --- a/rocketmq-go/remoting/json_serializable.go +++ b/rocketmq-go/remoting/json_serializable.go @@ -23,14 +23,14 @@ import ( type JsonSerializer struct { } -func (self *JsonSerializer) EncodeHeaderData(command *RemotingCommand) []byte { +func (j *JsonSerializer) EncodeHeaderData(command *RemotingCommand) []byte { buf, err := json.Marshal(command) if err != nil { return nil } return buf } -func (self *JsonSerializer) DecodeRemoteCommand(header, body []byte) *RemotingCommand { +func (j *JsonSerializer) DecodeRemoteCommand(header, body []byte) *RemotingCommand { cmd := &RemotingCommand{} cmd.ExtFields = make(map[string]interface{}) err := json.Unmarshal(header, cmd) diff --git a/rocketmq-go/remoting/json_serializable_test.go b/rocketmq-go/remoting/json_serializable_test.go new file mode 100644 index 000000000..1089c04ce --- /dev/null +++ b/rocketmq-go/remoting/json_serializable_test.go @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package remoting_test + +import ( + "testing" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" +) + +var testJson = "{\"code\":1,\"language\":\"GO\",\"version\":1,\"opaque\":1,\"flag\":1,\"remark\":\"remark\",\"extFields\":{\"key1\":\"str\",\"key2\":1},\"body\":\"AQIDBA==\"}" + +func TestEncodeHeaderData(t *testing.T) { + testMap := make(map[string]interface{}) + testMap["key1"] = "str" + testMap["key2"] = 1 + command := &remoting.RemotingCommand{ + Code: 1, + Language: "GO", + Version: 1, + Opaque: 1, + Flag: 1, + Remark: "remark", + ExtFields: testMap, + Body: []byte{1,2,3,4}, + } + jsonSerializer := remoting.JsonSerializer{} + + resultJson := jsonSerializer.EncodeHeaderData(command) + if testJson != string(resultJson) { + t.Errorf("resultJson is not equals testJson resultJson=%s ", resultJson) + } +} + +func TestDecodeRemoteCommand(t *testing.T) { + jsonSerializer := remoting.JsonSerializer{} + testByte := []byte(testJson) + remotingCommand := jsonSerializer.DecodeRemoteCommand(testByte, []byte{1, 2, 3,4}) + if remotingCommand.Language != "GO" || remotingCommand.Remark != "remark" { + t.Error("TestDecodeRemoteCommand fail reslutData") + }else { + t.Log("TestDecodeRemoteCommandSuccess") + } +} diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 021dd1f21..f2ce87b03 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -36,7 +36,7 @@ type RemotingClient interface { InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error } -type DefalutRemotingClient struct { +type DefaultRemotingClient struct { clientId string clientConfig *rocketmq_api_model.MqClientConfig @@ -55,8 +55,8 @@ type DefalutRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } -func RemotingClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefalutRemotingClient) { - client = &DefalutRemotingClient{} +func RemotingClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { + client = &DefaultRemotingClient{} client.connTable = map[string]net.Conn{} client.responseTable = util.New() client.clientConfig = clientConfig @@ -64,13 +64,13 @@ func RemotingClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientR client.namesrvAddrList = strings.Split(clientConfig.NameServerAddress, ";") client.namesrvAddrSelectedIndex = -1 client.clientRequestProcessor = clientRequestProcessor - client.serializerHandler = NewSerializerHandler() + client.serializerHandler = NewSerializerHandler(clientConfig.ClientSerializeType) return } -func (self *DefalutRemotingClient) InvokeSync(addr string, request *RemotingCommand, timeoutMillis int64) (remotingCommand *RemotingCommand, err error) { +func (drc *DefaultRemotingClient) InvokeSync(addr string, request *RemotingCommand, timeoutMillis int64) (remotingCommand *RemotingCommand, err error) { var conn net.Conn - conn, err = self.GetOrCreateConn(addr) + conn, err = drc.GetOrCreateConn(addr) response := &ResponseFuture{ SendRequestOK: false, Opaque: request.Opaque, @@ -78,10 +78,10 @@ func (self *DefalutRemotingClient) InvokeSync(addr string, request *RemotingComm BeginTimestamp: time.Now().Unix(), Done: make(chan bool), } - header := self.serializerHandler.EncodeHeader(request) + header := drc.serializerHandler.EncodeHeader(request) body := request.Body - self.SetResponse(request.Opaque, response) - err = self.sendRequest(header, body, conn, addr) + drc.SetResponse(request.Opaque, response) + err = drc.sendRequest(header, body, conn, addr) if err != nil { glog.Error(err) return @@ -95,8 +95,8 @@ func (self *DefalutRemotingClient) InvokeSync(addr string, request *RemotingComm return } } -func (self *DefalutRemotingClient) InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error { - conn, err := self.GetOrCreateConn(addr) +func (drc *DefaultRemotingClient) InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error { + conn, err := drc.GetOrCreateConn(addr) if err != nil { return err } @@ -107,24 +107,24 @@ func (self *DefalutRemotingClient) InvokeAsync(addr string, request *RemotingCom BeginTimestamp: time.Now().Unix(), InvokeCallback: invokeCallback, } - self.SetResponse(request.Opaque, response) - header := self.serializerHandler.EncodeHeader(request) + drc.SetResponse(request.Opaque, response) + header := drc.serializerHandler.EncodeHeader(request) body := request.Body - err = self.sendRequest(header, body, conn, addr) + err = drc.sendRequest(header, body, conn, addr) if err != nil { glog.Error(err) return err } return err } -func (self *DefalutRemotingClient) InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error { - conn, err := self.GetOrCreateConn(addr) +func (drc *DefaultRemotingClient) InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error { + conn, err := drc.GetOrCreateConn(addr) if err != nil { return err } - header := self.serializerHandler.EncodeHeader(request) + header := drc.serializerHandler.EncodeHeader(request) body := request.Body - err = self.sendRequest(header, body, conn, addr) + err = drc.sendRequest(header, body, conn, addr) if err != nil { glog.Error(err) return err @@ -132,7 +132,7 @@ func (self *DefalutRemotingClient) InvokeOneWay(addr string, request *RemotingCo return err } -func (self *DefalutRemotingClient) sendRequest(header, body []byte, conn net.Conn, addr string) error { +func (drc *DefaultRemotingClient) sendRequest(header, body []byte, conn net.Conn, addr string) error { var requestBytes []byte requestBytes = append(requestBytes, header...) if body != nil && len(body) > 0 { @@ -142,21 +142,21 @@ func (self *DefalutRemotingClient) sendRequest(header, body []byte, conn net.Con if err != nil { glog.Error(err) if len(addr) > 0 { - self.ReleaseConn(addr, conn) + drc.ReleaseConn(addr, conn) } return err } return nil } -func (self *DefalutRemotingClient) GetNamesrvAddrList() []string { - return self.namesrvAddrList +func (drc *DefaultRemotingClient) GetNamesrvAddrList() []string { + return drc.namesrvAddrList } -func (self *DefalutRemotingClient) SetResponse(index int32, response *ResponseFuture) { - self.responseTable.Set(strconv.Itoa(int(index)), response) +func (drc *DefaultRemotingClient) SetResponse(index int32, response *ResponseFuture) { + drc.responseTable.Set(strconv.Itoa(int(index)), response) } -func (self *DefalutRemotingClient) getResponse(index int32) (response *ResponseFuture, err error) { - obj, ok := self.responseTable.Get(strconv.Itoa(int(index))) +func (drc *DefaultRemotingClient) getResponse(index int32) (response *ResponseFuture, err error) { + obj, ok := drc.responseTable.Get(strconv.Itoa(int(index))) if !ok { err = errors.New("get conn from responseTable error") return @@ -164,102 +164,102 @@ func (self *DefalutRemotingClient) getResponse(index int32) (response *ResponseF response = obj.(*ResponseFuture) return } -func (self *DefalutRemotingClient) removeResponse(index int32) { - self.responseTable.Remove(strconv.Itoa(int(index))) +func (drc *DefaultRemotingClient) removeResponse(index int32) { + drc.responseTable.Remove(strconv.Itoa(int(index))) } -func (self *DefalutRemotingClient) GetOrCreateConn(address string) (conn net.Conn, err error) { +func (drc *DefaultRemotingClient) GetOrCreateConn(address string) (conn net.Conn, err error) { if len(address) == 0 { - conn, err = self.getNamesvrConn() + conn, err = drc.getNamesvrConn() return } - conn = self.GetConn(address) + conn = drc.GetConn(address) if conn != nil { return } - conn, err = self.CreateConn(address) + conn, err = drc.CreateConn(address) return } -func (self *DefalutRemotingClient) GetConn(address string) (conn net.Conn) { - self.connTableLock.RLock() - conn = self.connTable[address] - self.connTableLock.RUnlock() +func (drc *DefaultRemotingClient) GetConn(address string) (conn net.Conn) { + drc.connTableLock.RLock() + conn = drc.connTable[address] + drc.connTableLock.RUnlock() return } -func (self *DefalutRemotingClient) CreateConn(address string) (conn net.Conn, err error) { - defer self.connTableLock.Unlock() - self.connTableLock.Lock() - conn = self.connTable[address] +func (drc *DefaultRemotingClient) CreateConn(address string) (conn net.Conn, err error) { + defer drc.connTableLock.Unlock() + drc.connTableLock.Lock() + conn = drc.connTable[address] if conn != nil { return } - conn, err = self.createAndHandleTcpConn(address) - self.connTable[address] = conn + conn, err = drc.createAndHandleTcpConn(address) + drc.connTable[address] = conn return } -func (self *DefalutRemotingClient) getNamesvrConn() (conn net.Conn, err error) { - self.namesvrLockRW.RLock() - address := self.namesrvAddrSelectedAddr - self.namesvrLockRW.RUnlock() +func (drc *DefaultRemotingClient) getNamesvrConn() (conn net.Conn, err error) { + drc.namesvrLockRW.RLock() + address := drc.namesrvAddrSelectedAddr + drc.namesvrLockRW.RUnlock() if len(address) != 0 { - conn = self.GetConn(address) + conn = drc.GetConn(address) if conn != nil { return } } - defer self.namesvrLockRW.Unlock() - self.namesvrLockRW.Lock() + defer drc.namesvrLockRW.Unlock() + drc.namesvrLockRW.Lock() //already connected by another write lock owner - address = self.namesrvAddrSelectedAddr + address = drc.namesrvAddrSelectedAddr if len(address) != 0 { - conn = self.GetConn(address) + conn = drc.GetConn(address) if conn != nil { return } } - addressCount := len(self.namesrvAddrList) - if self.namesrvAddrSelectedIndex < 0 { - self.namesrvAddrSelectedIndex = rand.Intn(addressCount) + addressCount := len(drc.namesrvAddrList) + if drc.namesrvAddrSelectedIndex < 0 { + drc.namesrvAddrSelectedIndex = rand.Intn(addressCount) } for i := 1; i <= addressCount; i++ { - selectedIndex := (self.namesrvAddrSelectedIndex + i) % addressCount - selectAddress := self.namesrvAddrList[selectedIndex] + selectedIndex := (drc.namesrvAddrSelectedIndex + i) % addressCount + selectAddress := drc.namesrvAddrList[selectedIndex] if len(selectAddress) == 0 { continue } - conn, err = self.CreateConn(selectAddress) + conn, err = drc.CreateConn(selectAddress) if err == nil { - self.namesrvAddrSelectedAddr = selectAddress - self.namesrvAddrSelectedIndex = selectedIndex + drc.namesrvAddrSelectedAddr = selectAddress + drc.namesrvAddrSelectedIndex = selectedIndex return } } - err = errors.New("all namesvrAddress can't use!,address:" + self.clientConfig.NameServerAddress) + err = errors.New("all namesvrAddress can't use!,address:" + drc.clientConfig.NameServerAddress) return } -func (self *DefalutRemotingClient) createAndHandleTcpConn(address string) (conn net.Conn, err error) { +func (drc *DefaultRemotingClient) createAndHandleTcpConn(address string) (conn net.Conn, err error) { conn, err = net.Dial("tcp", address) if err != nil { glog.Error(err) return nil, err } - go self.handlerReceiveLoop(conn, address) //handler连接 处理这个连接返回的结果 + go drc.handlerReceiveLoop(conn, address) //handler连接 处理这个连接返回的结果 return } -func (self *DefalutRemotingClient) ReleaseConn(addr string, conn net.Conn) { - defer self.connTableLock.Unlock() +func (drc *DefaultRemotingClient) ReleaseConn(addr string, conn net.Conn) { + defer drc.connTableLock.Unlock() conn.Close() - self.connTableLock.Lock() - delete(self.connTable, addr) + drc.connTableLock.Lock() + delete(drc.connTable, addr) } -func (self *DefalutRemotingClient) handlerReceiveLoop(conn net.Conn, addr string) (err error) { +func (drc *DefaultRemotingClient) handlerReceiveLoop(conn net.Conn, addr string) (err error) { defer func() { //when for is break releaseConn glog.Error(err, addr) - self.ReleaseConn(addr, conn) + drc.ReleaseConn(addr, conn) }() b := make([]byte, 1024) var length, headerLength, bodyLength int32 @@ -313,35 +313,35 @@ func (self *DefalutRemotingClient) handlerReceiveLoop(conn net.Conn, addr string } else { _, err = buf.Read(body) } - go self.handlerReceivedMessage(conn, headerSerializableType, header, body) + go drc.handlerReceivedMessage(conn, headerSerializableType, header, body) } } } -func (self *DefalutRemotingClient) handlerReceivedMessage(conn net.Conn, headerSerializableType byte, headBytes []byte, bodyBytes []byte) { - cmd := self.serializerHandler.DecodeRemoteCommand(headerSerializableType, headBytes, bodyBytes) +func (drc *DefaultRemotingClient) handlerReceivedMessage(conn net.Conn, headerSerializableType byte, headBytes []byte, bodyBytes []byte) { + cmd := drc.serializerHandler.DecodeRemoteCommand(headerSerializableType, headBytes, bodyBytes) if cmd.IsResponseType() { - self.handlerResponse(cmd) + drc.handlerResponse(cmd) return } - go self.handlerRequest(conn, cmd) + go drc.handlerRequest(conn, cmd) } -func (self *DefalutRemotingClient) handlerRequest(conn net.Conn, cmd *RemotingCommand) { - responseCommand := self.clientRequestProcessor(cmd) +func (drc *DefaultRemotingClient) handlerRequest(conn net.Conn, cmd *RemotingCommand) { + responseCommand := drc.clientRequestProcessor(cmd) if responseCommand == nil { return } responseCommand.Opaque = cmd.Opaque responseCommand.MarkResponseType() - header := self.serializerHandler.EncodeHeader(responseCommand) + header := drc.serializerHandler.EncodeHeader(responseCommand) body := responseCommand.Body - err := self.sendRequest(header, body, conn, "") + err := drc.sendRequest(header, body, conn, "") if err != nil { glog.Error(err) } } -func (self *DefalutRemotingClient) handlerResponse(cmd *RemotingCommand) { - response, err := self.getResponse(cmd.Opaque) - self.removeResponse(cmd.Opaque) +func (drc *DefaultRemotingClient) handlerResponse(cmd *RemotingCommand) { + response, err := drc.getResponse(cmd.Opaque) + drc.removeResponse(cmd.Opaque) if err != nil { return } @@ -355,12 +355,11 @@ func (self *DefalutRemotingClient) handlerResponse(cmd *RemotingCommand) { } } -func (self *DefalutRemotingClient) ClearExpireResponse() { - for seq, responseObj := range self.responseTable.Items() { +func (drc *DefaultRemotingClient) ClearExpireResponse() { + for seq, responseObj := range drc.responseTable.Items() { response := responseObj.(*ResponseFuture) if (response.BeginTimestamp + 30) <= time.Now().Unix() { - //30 mins expire - self.responseTable.Remove(seq) + drc.responseTable.Remove(seq) if response.InvokeCallback != nil { response.InvokeCallback(nil) glog.Warningf("remove time out request %v", response) diff --git a/rocketmq-go/remoting/remoting_client_test.go b/rocketmq-go/remoting/remoting_client_test.go new file mode 100644 index 000000000..9359b403f --- /dev/null +++ b/rocketmq-go/remoting/remoting_client_test.go @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package remoting_test + + +//func TestClearExpireResponse(t *testing.T) { +// for seq, responseObj := range drc.responseTable.Items() { +// response := responseObj.(*ResponseFuture) +// if (response.BeginTimestamp + 30) <= time.Now().Unix() { +// drc.responseTable.Remove(seq) +// if response.InvokeCallback != nil { +// response.InvokeCallback(nil) +// glog.Warningf("remove time out request %v", response) +// } +// } +// } +//} diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index 22dc41f65..e1bb685b2 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -26,6 +26,13 @@ import ( type RocketMqSerializer struct { } +type itemType int8 + +const ( + key_item itemType = iota + value_item +) + func (self *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { var ( remarkBytes []byte @@ -68,7 +75,7 @@ func (self *RocketMqSerializer) DecodeRemoteCommand(headerArray, body []byte) (c // LanguageCode language var LanguageCodeNope byte binary.Read(buf, binary.BigEndian, &LanguageCodeNope) - cmd.Language = constant.REMOTING_COMMAND_LANGUAGE //todo use code from remote + cmd.Language = constant.REMOTING_COMMAND_LANGUAGE // int version(~32767) binary.Read(buf, binary.BigEndian, &cmd.Version) // int opaque @@ -83,8 +90,6 @@ func (self *RocketMqSerializer) DecodeRemoteCommand(headerArray, body []byte) (c binary.Read(buf, binary.BigEndian, &remarkData) cmd.Remark = string(remarkData) } - //map ext - // HashMap extFields binary.Read(buf, binary.BigEndian, &extFieldsLen) if extFieldsLen > 0 { var extFieldsData = make([]byte, extFieldsLen) @@ -114,21 +119,21 @@ func customHeaderDeserialize(extFiledDataBytes []byte) (extFiledMap map[string]i extFiledMap = make(map[string]interface{}) buf := bytes.NewBuffer(extFiledDataBytes) for buf.Len() > 0 { - var key = getItemFormExtFiledDataBytes(buf, "key") - var value = getItemFormExtFiledDataBytes(buf, "value") + var key = getItemFormExtFiledDataBytes(buf, key_item) + var value = getItemFormExtFiledDataBytes(buf, value_item) extFiledMap[key] = value } return } -func getItemFormExtFiledDataBytes(buff *bytes.Buffer, itemType string) (item string) { - if itemType == "key" { +func getItemFormExtFiledDataBytes(buff *bytes.Buffer, iType itemType) (item string) { + if iType == key_item { var len int16 binary.Read(buff, binary.BigEndian, &len) var data = make([]byte, len) binary.Read(buff, binary.BigEndian, &data) item = string(data) } - if itemType == "value" { + if iType == value_item { var len int32 binary.Read(buff, binary.BigEndian, &len) var data = make([]byte, len) diff --git a/rocketmq-go/remoting/rocketmq_serializable_test.go b/rocketmq-go/remoting/rocketmq_serializable_test.go new file mode 100644 index 000000000..1d1a3d6a3 --- /dev/null +++ b/rocketmq-go/remoting/rocketmq_serializable_test.go @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package remoting_test + +import ( + "testing" +) + +func TestEncodeHeaderDataRocketMq(t *testing.T) { + +} +func TestDecodeRemoteCommandRocketMq(t *testing.T) { + +} +func TestRocketMqCustomHeaderSerialize(t *testing.T) { + +} + +func TestCustomHeaderDeserialize(t *testing.T) { + +} + +func TestGetItemFormExtFiledDataBytes(t *testing.T) { + +} + diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index 24420cd9d..2a425e7d3 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -19,12 +19,13 @@ package remoting import ( "bytes" "encoding/binary" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/golang/glog" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) type SerializerHandler struct { - serializer Serializer //which serializer this client use, depend on constant.USE_HEADER_SERIALIZETYPE + serializeType rocketmq_api_model.SerializeType + serializer Serializer //which serializer this client use, depend on constant.USE_HEADER_SERIALIZE_TYPE } type Serializer interface { @@ -35,14 +36,14 @@ type Serializer interface { var JSON_SERIALIZER = &JsonSerializer{} var ROCKETMQ_SERIALIZER = &RocketMqSerializer{} -func NewSerializerHandler() SerializerHandler { - serializerHandler := SerializerHandler{} - switch constant.USE_HEADER_SERIALIZETYPE { - case constant.JSON_SERIALIZE: +func NewSerializerHandler(serializeType rocketmq_api_model.SerializeType) SerializerHandler { + serializerHandler := SerializerHandler{serializeType: serializeType} + switch serializeType { + case rocketmq_api_model.JSON_SERIALIZE: serializerHandler.serializer = JSON_SERIALIZER break - case constant.ROCKETMQ_SERIALIZE: + case rocketmq_api_model.ROCKETMQ_SERIALIZE: serializerHandler.serializer = ROCKETMQ_SERIALIZER break default: @@ -58,19 +59,19 @@ func (self *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { length += len(request.Body) } buf := bytes.NewBuffer([]byte{}) - binary.Write(buf, binary.BigEndian, int32(length)) // len - binary.Write(buf, binary.BigEndian, int32(len(headerData)|(int(constant.USE_HEADER_SERIALIZETYPE)<<24))) // header len + binary.Write(buf, binary.BigEndian, int32(length)) // len + binary.Write(buf, binary.BigEndian, int32(len(headerData)|(int(self.serializeType)<<24))) // header len buf.Write(headerData) return buf.Bytes() } func (self *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { var serializer Serializer - switch headerSerializableType { - case constant.JSON_SERIALIZE: + switch rocketmq_api_model.SerializeType(headerSerializableType) { + case rocketmq_api_model.JSON_SERIALIZE: serializer = JSON_SERIALIZER break - case constant.ROCKETMQ_SERIALIZE: + case rocketmq_api_model.ROCKETMQ_SERIALIZE: serializer = ROCKETMQ_SERIALIZER break default: diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 47cf52e5b..23816791c 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -40,7 +40,7 @@ import ( //4.heartbeat type RocketMqClient interface { GetClientId() (clientId string) - GetRemotingClient() (remotingClient *remoting.DefalutRemotingClient) + GetRemotingClient() (remotingClient *remoting.DefaultRemotingClient) GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) GetPublishTopicList() []string FetchMasterBrokerAddress(brokerName string) (masterAddress string) @@ -61,7 +61,7 @@ var DEFAULT_TIMEOUT int64 = 6000 type MqClientImpl struct { ClientId string - remotingClient *remoting.DefalutRemotingClient + remotingClient *remoting.DefaultRemotingClient TopicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData BrokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address TopicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this @@ -132,7 +132,7 @@ func (self *MqClientImpl) GetPublishTopicList() []string { } return publishTopicList } -func (self *MqClientImpl) GetRemotingClient() *remoting.DefalutRemotingClient { +func (self *MqClientImpl) GetRemotingClient() *remoting.DefaultRemotingClient { return self.remotingClient } From 925ab25aefb798263f4d209421159be36a091b3b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 13 Aug 2017 20:39:58 +0800 Subject: [PATCH 23/88] update api --- rocketmq-go/api/model/message.go | 100 +++++++++--------- rocketmq-go/api/model/message_ext.go | 24 ++--- rocketmq-go/api/model/mq_client_config.go | 3 +- .../api/model/mq_producer_config_test.go | 23 ---- rocketmq-go/api/rocketmq_clent_instance.go | 13 ++- .../api/rocketmq_clent_instance_test.go | 23 ---- .../remoting/json_serializable_test.go | 8 +- rocketmq-go/remoting/remoting_client_test.go | 1 - rocketmq-go/remoting/rocketmq_serializable.go | 2 +- .../remoting/rocketmq_serializable_test.go | 1 - rocketmq-go/remoting/serializable.go | 2 +- rocketmq-go/util/compress_util_test.go | 10 +- rocketmq-go/util/ip_test.go | 2 +- rocketmq-go/util/string_util_test.go | 4 +- 14 files changed, 84 insertions(+), 132 deletions(-) delete mode 100644 rocketmq-go/api/model/mq_producer_config_test.go delete mode 100644 rocketmq-go/api/rocketmq_clent_instance_test.go diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index f3394afd9..81a9d9f7b 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -30,82 +30,82 @@ type Message struct { Body []byte } -func (self *Message) SetTag(tag string) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetTag(tag string) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_TAGS] = tag + m.Properties[constant.PROPERTY_TAGS] = tag } -func (self *Message) GetTag() (tag string) { - if self.Properties != nil { - tag = self.Properties[constant.PROPERTY_TAGS] +func (m *Message) GetTag() (tag string) { + if m.Properties != nil { + tag = m.Properties[constant.PROPERTY_TAGS] } return } -func (self *Message) SetKeys(keys []string) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetKeys(keys []string) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") + m.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") } -func (self *Message) SetDelayTimeLevel(delayTimeLevel int) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetDelayTimeLevel(delayTimeLevel int) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) + m.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) } -func (self *Message) SetWaitStoreMsgOK(waitStoreMsgOK bool) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetWaitStoreMsgOK(waitStoreMsgOK bool) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) + m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) } -func (self *Message) GeneratorMsgUniqueKey() { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) GeneratorMsgUniqueKey() { + if m.Properties == nil { + m.Properties = make(map[string]string) } - if len(self.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { + if len(m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { return } - self.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() + m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() } -func (self *MessageExt) GetMsgUniqueKey() string { - if self.Properties != nil { - originMessageId := self.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] +func (m *MessageExt) GetMsgUniqueKey() string { + if m.Properties != nil { + originMessageId := m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] if len(originMessageId) > 0 { return originMessageId } } - return self.MsgId + return m.MsgId } -func (self *Message) SetOriginMessageId(messageId string) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetOriginMessageId(messageId string) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId + m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId } -func (self *Message) SetRetryTopic(retryTopic string) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetRetryTopic(retryTopic string) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic + m.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic } -func (self *Message) SetReconsumeTime(reConsumeTime int) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetReconsumeTime(reConsumeTime int) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) + m.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) } -func (self *Message) GetReconsumeTimes() (reConsumeTime int) { +func (m *Message) GetReconsumeTimes() (reConsumeTime int) { reConsumeTime = 0 - if self.Properties != nil { - reConsumeTimeStr := self.Properties[constant.PROPERTY_RECONSUME_TIME] + if m.Properties != nil { + reConsumeTimeStr := m.Properties[constant.PROPERTY_RECONSUME_TIME] if len(reConsumeTimeStr) > 0 { reConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) } @@ -113,17 +113,17 @@ func (self *Message) GetReconsumeTimes() (reConsumeTime int) { return } -func (self *Message) SetMaxReconsumeTimes(maxConsumeTime int) { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *Message) SetMaxReconsumeTimes(maxConsumeTime int) { + if m.Properties == nil { + m.Properties = make(map[string]string) } - self.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) + m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) } -func (self *Message) GetMaxReconsumeTimes() (maxConsumeTime int) { +func (m *Message) GetMaxReconsumeTimes() (maxConsumeTime int) { maxConsumeTime = 0 - if self.Properties != nil { - reConsumeTimeStr := self.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] + if m.Properties != nil { + reConsumeTimeStr := m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] if len(reConsumeTimeStr) > 0 { maxConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) } diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index a747361b5..dcc857a2e 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -41,29 +41,29 @@ type MessageExt struct { propertyConsumeStartTimestamp string } -func (self *MessageExt) GetOriginMessageId() string { - if self.Properties != nil { - originMessageId := self.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] +func (m *MessageExt) GetOriginMessageId() string { + if m.Properties != nil { + originMessageId := m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] if len(originMessageId) > 0 { return originMessageId } } - return self.MsgId + return m.MsgId } -func (self *MessageExt) GetConsumeStartTime() int64 { - if len(self.propertyConsumeStartTimestamp) > 0 { - return util.StrToInt64WithDefaultValue(self.propertyConsumeStartTimestamp, -1) +func (m *MessageExt) GetConsumeStartTime() int64 { + if len(m.propertyConsumeStartTimestamp) > 0 { + return util.StrToInt64WithDefaultValue(m.propertyConsumeStartTimestamp, -1) } return math.MaxInt64 } -func (self *MessageExt) SetConsumeStartTime() { - if self.Properties == nil { - self.Properties = make(map[string]string) +func (m *MessageExt) SetConsumeStartTime() { + if m.Properties == nil { + m.Properties = make(map[string]string) } nowTime := util.CurrentTimeMillisStr() - self.Properties[constant.PROPERTY_KEYS] = nowTime - self.propertyConsumeStartTimestamp = nowTime + m.Properties[constant.PROPERTY_KEYS] = nowTime + m.propertyConsumeStartTimestamp = nowTime return } diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 5966a902e..77f081192 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -21,9 +21,10 @@ package rocketmq_api_model type SerializeType byte const ( - JSON_SERIALIZE SerializeType = iota + JSON_SERIALIZE SerializeType = iota ROCKETMQ_SERIALIZE ) + type MqClientConfig struct { // NameServerAddress split by ; NameServerAddress string diff --git a/rocketmq-go/api/model/mq_producer_config_test.go b/rocketmq-go/api/model/mq_producer_config_test.go deleted file mode 100644 index 0aac8fce7..000000000 --- a/rocketmq-go/api/model/mq_producer_config_test.go +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rocketmq_api_model - -import "testing" - -func TestB(t *testing.T) { - -} diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 7cc0fb6bb..41977e403 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -40,14 +40,13 @@ func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmq_a return } - -func (self *RocketMQClientInstanceImpl) RegisterProducer(producer RocketMQProducer) { - self.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) +func (r *RocketMQClientInstanceImpl) RegisterProducer(producer RocketMQProducer) { + r.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) } -func (self *RocketMQClientInstanceImpl) RegisterConsumer(consumer RocketMQConsumer) { - self.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) +func (r *RocketMQClientInstanceImpl) RegisterConsumer(consumer RocketMQConsumer) { + r.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) } -func (self *RocketMQClientInstanceImpl) Start() { - self.rocketMqManager.Start() +func (r *RocketMQClientInstanceImpl) Start() { + r.rocketMqManager.Start() } diff --git a/rocketmq-go/api/rocketmq_clent_instance_test.go b/rocketmq-go/api/rocketmq_clent_instance_test.go deleted file mode 100644 index c7401d21b..000000000 --- a/rocketmq-go/api/rocketmq_clent_instance_test.go +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package rocketmq_api - -import "testing" - -func TestA(t *testing.T) { - -} diff --git a/rocketmq-go/remoting/json_serializable_test.go b/rocketmq-go/remoting/json_serializable_test.go index 1089c04ce..287e45a7c 100644 --- a/rocketmq-go/remoting/json_serializable_test.go +++ b/rocketmq-go/remoting/json_serializable_test.go @@ -17,8 +17,8 @@ package remoting_test import ( - "testing" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" + "testing" ) var testJson = "{\"code\":1,\"language\":\"GO\",\"version\":1,\"opaque\":1,\"flag\":1,\"remark\":\"remark\",\"extFields\":{\"key1\":\"str\",\"key2\":1},\"body\":\"AQIDBA==\"}" @@ -35,7 +35,7 @@ func TestEncodeHeaderData(t *testing.T) { Flag: 1, Remark: "remark", ExtFields: testMap, - Body: []byte{1,2,3,4}, + Body: []byte{1, 2, 3, 4}, } jsonSerializer := remoting.JsonSerializer{} @@ -48,10 +48,10 @@ func TestEncodeHeaderData(t *testing.T) { func TestDecodeRemoteCommand(t *testing.T) { jsonSerializer := remoting.JsonSerializer{} testByte := []byte(testJson) - remotingCommand := jsonSerializer.DecodeRemoteCommand(testByte, []byte{1, 2, 3,4}) + remotingCommand := jsonSerializer.DecodeRemoteCommand(testByte, []byte{1, 2, 3, 4}) if remotingCommand.Language != "GO" || remotingCommand.Remark != "remark" { t.Error("TestDecodeRemoteCommand fail reslutData") - }else { + } else { t.Log("TestDecodeRemoteCommandSuccess") } } diff --git a/rocketmq-go/remoting/remoting_client_test.go b/rocketmq-go/remoting/remoting_client_test.go index 9359b403f..f1d85ada8 100644 --- a/rocketmq-go/remoting/remoting_client_test.go +++ b/rocketmq-go/remoting/remoting_client_test.go @@ -16,7 +16,6 @@ */ package remoting_test - //func TestClearExpireResponse(t *testing.T) { // for seq, responseObj := range drc.responseTable.Items() { // response := responseObj.(*ResponseFuture) diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index e1bb685b2..4bc9c146c 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -29,7 +29,7 @@ type RocketMqSerializer struct { type itemType int8 const ( - key_item itemType = iota + key_item itemType = iota value_item ) diff --git a/rocketmq-go/remoting/rocketmq_serializable_test.go b/rocketmq-go/remoting/rocketmq_serializable_test.go index 1d1a3d6a3..58048afa9 100644 --- a/rocketmq-go/remoting/rocketmq_serializable_test.go +++ b/rocketmq-go/remoting/rocketmq_serializable_test.go @@ -37,4 +37,3 @@ func TestCustomHeaderDeserialize(t *testing.T) { func TestGetItemFormExtFiledDataBytes(t *testing.T) { } - diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index 2a425e7d3..130f99bb9 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -19,8 +19,8 @@ package remoting import ( "bytes" "encoding/binary" - "github.com/golang/glog" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" ) type SerializerHandler struct { diff --git a/rocketmq-go/util/compress_util_test.go b/rocketmq-go/util/compress_util_test.go index ada07af6d..ccef8f5fc 100644 --- a/rocketmq-go/util/compress_util_test.go +++ b/rocketmq-go/util/compress_util_test.go @@ -22,11 +22,11 @@ import ( func TestCompressWithLevel(t *testing.T) { bytes := []byte{120, 156, 202, 72, 205, 201, 201, 215, 81, 40, 207, - 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, - 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, - 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, - 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, - 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147} + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147, + 47, 202, 73, 225, 2, 4, 0, 0, 255, 255, 33, 231, 4, 147} body1, error1 := CompressWithLevel(bytes, 5) if len(body1) > len(bytes) && error1 != nil { t.Errorf("CompressWithLevel failed : cannot transfer byte[]") diff --git a/rocketmq-go/util/ip_test.go b/rocketmq-go/util/ip_test.go index 52caaf026..3e6e8684f 100644 --- a/rocketmq-go/util/ip_test.go +++ b/rocketmq-go/util/ip_test.go @@ -17,8 +17,8 @@ package util import ( - "testing" "strings" + "testing" ) func TestGetIp4Bytes(t *testing.T) { diff --git a/rocketmq-go/util/string_util_test.go b/rocketmq-go/util/string_util_test.go index 537cdab78..f6b29fdea 100644 --- a/rocketmq-go/util/string_util_test.go +++ b/rocketmq-go/util/string_util_test.go @@ -17,8 +17,8 @@ package util import ( - "testing" "strings" + "testing" ) func TestStrToIntWithDefaultValue(t *testing.T) { @@ -40,7 +40,7 @@ func TestReadString(t *testing.T) { } ret2 := ReadString(nil) if ret2 != "" { - t.Errorf("TestReadString failed : cannot transfer nil to \"\""); + t.Errorf("TestReadString failed : cannot transfer nil to \"\"") } } From 275ae2ef2025adc18c0a850bb194eda8c56f313a Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 13 Aug 2017 22:05:40 +0800 Subject: [PATCH 24/88] refactor code --- rocketmq-go/manage/mq_client_manage.go | 104 ++-- rocketmq-go/manage/mq_producer.go | 1 - rocketmq-go/manage/mq_push_consumer.go | 63 +-- rocketmq-go/manage/pull_message_controller.go | 54 +- rocketmq-go/manage/tasks.go | 14 +- rocketmq-go/model/client_config.go | 337 ++++++------ rocketmq-go/model/constant/mix_all.go | 14 +- .../model/consume_message_directly_result.go | 27 +- rocketmq-go/model/consumer_running_info.go | 18 +- ..._message_directly_result_request_header.go | 10 +- .../consumer_send_msg_back_request_header.go | 2 +- .../header/get_consumer_list_by_group.go | 4 +- ...et_consumer_running_info_request_header.go | 6 +- .../header/get_max_offset_request_header.go | 2 +- .../header/get_max_offset_response_header.go | 4 +- .../header/get_route_info_request_header.go | 2 +- .../header/pull_message_request_header.go | 2 +- .../query_consumer_offset_request_header.go | 2 +- .../header/reset_offset_request_header.go | 10 +- .../header/search_offset_request_header.go | 8 +- .../header/send_message_request_header.go | 2 +- .../update_consumer_offset_request_header.go | 10 +- rocketmq-go/model/message/message.go | 511 +++++++++--------- rocketmq-go/model/message/message_constant.go | 193 +++---- rocketmq-go/model/message_queue.go | 30 +- rocketmq-go/model/process_queue.go | 125 +++-- rocketmq-go/model/pull_result.go | 155 +++--- rocketmq-go/model/query_result.go | 93 ++-- rocketmq-go/model/reset_offset_body.go | 6 +- rocketmq-go/model/topic_publish_info.go | 11 +- rocketmq-go/remoting/remoting_client.go | 10 +- rocketmq-go/remoting/remoting_command.go | 9 +- rocketmq-go/remoting/rocketmq_serializable.go | 4 +- rocketmq-go/remoting/serializable.go | 10 +- .../allocate_message_averagely.go | 2 +- .../allocate_message_averagely_by_circle.go | 2 +- .../allocate_message_by_config.go | 4 +- .../allocate_message_by_machine_room.go | 2 +- rocketmq-go/service/client_api.go | 85 --- rocketmq-go/service/client_error_code.go | 26 - .../service/consume_message_service.go | 42 +- rocketmq-go/service/mq_client.go | 126 ++--- rocketmq-go/service/offset_store.go | 64 +-- rocketmq-go/service/producer_service.go | 41 +- .../service/producer_service_for_send_back.go | 38 +- rocketmq-go/service/rebalance.go | 125 +++-- .../test_send_and_receive_compress_message.go | 53 ++ rocketmq-go/util/compress_util.go | 9 +- rocketmq-go/util/regex_util.go | 1 - rocketmq-go/util/time_util_test.go | 12 + 50 files changed, 1186 insertions(+), 1299 deletions(-) delete mode 100644 rocketmq-go/service/client_api.go delete mode 100644 rocketmq-go/service/client_error_code.go create mode 100644 rocketmq-go/test/test_send_and_receive_compress_message.go create mode 100644 rocketmq-go/util/time_util_test.go diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index 5254cd00a..a7110a2e3 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -33,32 +33,18 @@ import ( "time" ) -//@see com.alibaba.rocketmq.client.impl.factory.MQClientInstance type MqClientManager struct { - rocketMqManagerLock sync.Mutex - //ClientId string - BootTimestamp int64 - - clientFactory *ClientFactory - - NamesrvLock sync.Mutex - HeartBeatLock sync.Mutex - //all producer and consumer use this - mqClient service.RocketMqClient - //all producer and consumer use this - //private final ClientRemotingProcessor clientRemotingProcessor; - // private final PullMessageService pullMessageService; - //private final RebalanceService rebalanceService; - // private final ConsumerStatsManager consumerStatsManager; - // private final AtomicLong storeTimesTotal = new AtomicLong(0); - ServiceState int - - //should be here because need all producer consumer + rocketMqManagerLock sync.Mutex + BootTimestamp int64 + clientFactory *ClientFactory + NamesrvLock sync.Mutex + HeartBeatLock sync.Mutex + mqClient service.RocketMqClient + ServiceState int pullMessageController *PullMessageController cleanExpireMsgController *CleanExpireMsgController rebalanceControllr *RebalanceController - //should be here because need all producer consumer - defaultProducerService *service.DefaultProducerService //for send back message + defaultProducerService *service.DefaultProducerService } func MqClientManagerInit(clientConfig *rocketmq_api_model.MqClientConfig) (rocketMqManager *MqClientManager) { @@ -72,23 +58,15 @@ func MqClientManagerInit(clientConfig *rocketmq_api_model.MqClientConfig) (rocke return } - -//CHECK_TRANSACTION_STATE -//NOTIFY_CONSUMER_IDS_CHANGED -//RESET_CONSUMER_CLIENT_OFFSET -//GET_CONSUMER_STATUS_FROM_CLIENT -//GET_CONSUMER_RUNNING_INFO -//CONSUME_MESSAGE_DIRECTLY -func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { +func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { clientRequestProcessor = func(cmd *remoting.RemotingCommand) (response *remoting.RemotingCommand) { switch cmd.Code { case remoting.CHECK_TRANSACTION_STATE: glog.V(2).Info("receive_request_code CHECK_TRANSACTION_STATE") - // todo this version don't impl this break case remoting.NOTIFY_CONSUMER_IDS_CHANGED: - glog.V(1).Info("receive_request_code NOTIFY_CONSUMER_IDS_CHANGED") - self.rebalanceControllr.doRebalance() + glog.V(2).Info("receive_request_code NOTIFY_CONSUMER_IDS_CHANGED") + m.rebalanceControllr.doRebalance() break case remoting.RESET_CONSUMER_CLIENT_OFFSET: // struct json key supported glog.V(2).Info("receive_request_code RESET_CONSUMER_CLIENT_OFFSET") @@ -103,7 +81,7 @@ func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcesso return } glog.V(2).Info("op=look resetOffsetBody xxxxx", resetOffsetBody) - self.resetConsumerOffset(resetOffsetRequestHeader.Topic, resetOffsetRequestHeader.Group, resetOffsetBody.OffsetTable) + m.resetConsumerOffset(resetOffsetRequestHeader.Topic, resetOffsetRequestHeader.Group, resetOffsetBody.OffsetTable) } break case remoting.GET_CONSUMER_STATUS_FROM_CLIENT: // useless we can use GET_CONSUMER_RUNNING_INFO instead @@ -116,8 +94,8 @@ func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcesso getConsumerRunningInfoRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct consumerRunningInfo := model.ConsumerRunningInfo{} consumerRunningInfo.Properties = map[string]string{} - defaultMQPushConsumer := self.clientFactory.ConsumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] - consumerConfigMap := structs.Map(defaultMQPushConsumer.ConsumerConfig) // todo test + defaultMQPushConsumer := m.clientFactory.ConsumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] + consumerConfigMap := structs.Map(defaultMQPushConsumer.ConsumerConfig) for key, value := range consumerConfigMap { consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) } @@ -143,7 +121,7 @@ func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcesso consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) messageExt := &DecodeMessage(cmd.Body)[0] glog.V(2).Info("op=look", messageExt) - defaultMQPushConsumer := self.clientFactory.ConsumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] + defaultMQPushConsumer := m.clientFactory.ConsumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] consumeResult, err := defaultMQPushConsumer.consumeMessageService.ConsumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) if err != nil { return @@ -162,38 +140,38 @@ func (self *MqClientManager) InitClientRequestProcessor() (clientRequestProcesso } return } -func (self *MqClientManager) RegistProducer(producer *DefaultMQProducer) { - producer.producerService = service.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, self.mqClient) - self.clientFactory.ProducerTable[producer.producerGroup] = producer +func (m *MqClientManager) RegistProducer(producer *DefaultMQProducer) { + producer.producerService = service.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) + m.clientFactory.ProducerTable[producer.producerGroup] = producer return } -func (self *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { - consumer := self.clientFactory.ConsumerTable[group] +func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { + consumer := m.clientFactory.ConsumerTable[group] if consumer == nil { glog.Error("resetConsumerOffset beacuse consumer not online,group=", group) return } consumer.resetOffset(offsetTable) } -func (self *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { - if self.defaultProducerService == nil { - self.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmq_api_model.NewProducerConfig(), self.mqClient) +func (m *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { + if m.defaultProducerService == nil { + m.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmq_api_model.NewProducerConfig(), m.mqClient) } - consumer.mqClient = self.mqClient - consumer.offsetStore = service.RemoteOffsetStoreInit(consumer.consumerGroup, self.mqClient) - self.clientFactory.ConsumerTable[consumer.consumerGroup] = consumer + consumer.mqClient = m.mqClient + consumer.offsetStore = service.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) + m.clientFactory.ConsumerTable[consumer.consumerGroup] = consumer consumer.rebalance = service.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) fmt.Println(consumer.consumeMessageService) - consumer.consumeMessageService.Init(consumer.consumerGroup, self.mqClient, consumer.offsetStore, self.defaultProducerService, consumer.ConsumerConfig) + consumer.consumeMessageService.Init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) return } -func (self *MqClientManager) Start() { - //self.SendHeartbeatToAllBrokerWithLock()//we should send heartbeat first - self.StartAllScheduledTask() +func (m *MqClientManager) Start() { + //d.SendHeartbeatToAllBrokerWithLock()//we should send heartbeat first todo check + m.StartAllScheduledTask() } type ClientFactory struct { @@ -209,36 +187,36 @@ func ClientFactoryInit() (clientFactory *ClientFactory) { } //heart beat -func (self MqClientManager) SendHeartbeatToAllBrokerWithLock() error { - heartbeatData := self.prepareHeartbeatData() +func (m MqClientManager) SendHeartbeatToAllBrokerWithLock() error { + heartbeatData := m.prepareHeartbeatData() if len(heartbeatData.ConsumerDataSet) == 0 { return errors.New("send heartbeat error") } - self.mqClient.SendHeartbeatToAllBroker(heartbeatData) + m.mqClient.SendHeartbeatToAllBroker(heartbeatData) return nil } //routeInfo -func (self MqClientManager) UpdateTopicRouteInfoFromNameServer() { +func (m MqClientManager) UpdateTopicRouteInfoFromNameServer() { var topicSet []string - for _, consumer := range self.clientFactory.ConsumerTable { + for _, consumer := range m.clientFactory.ConsumerTable { for key, _ := range consumer.subscription { topicSet = append(topicSet, key) } } - topicSet = append(topicSet, self.mqClient.GetPublishTopicList()...) + topicSet = append(topicSet, m.mqClient.GetPublishTopicList()...) for _, topic := range topicSet { - self.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + m.mqClient.UpdateTopicRouteInfoFromNameServer(topic) } } -func (self MqClientManager) prepareHeartbeatData() *model.HeartbeatData { +func (m MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData := new(model.HeartbeatData) - heartbeatData.ClientId = self.mqClient.GetClientId() + heartbeatData.ClientId = m.mqClient.GetClientId() heartbeatData.ConsumerDataSet = make([]*model.ConsumerData, 0) heartbeatData.ProducerDataSet = make([]*model.ProducerData, 0) - for group, consumer := range self.clientFactory.ConsumerTable { + for group, consumer := range m.clientFactory.ConsumerTable { consumerData := new(model.ConsumerData) consumerData.GroupName = group consumerData.ConsumeType = consumer.consumeType @@ -248,7 +226,7 @@ func (self MqClientManager) prepareHeartbeatData() *model.HeartbeatData { consumerData.UnitMode = consumer.unitMode heartbeatData.ConsumerDataSet = append(heartbeatData.ConsumerDataSet, consumerData) } - for group := range self.clientFactory.ProducerTable { + for group := range m.clientFactory.ProducerTable { producerData := new(model.ProducerData) producerData.GroupName = group heartbeatData.ProducerDataSet = append(heartbeatData.ProducerDataSet, producerData) diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index cc0116d84..634e5ddec 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -1,6 +1,5 @@ package rocketmq -import () import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index d802f10ba..e724db09f 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -27,18 +27,16 @@ import ( ) type DefaultMQPushConsumer struct { - consumerGroup string - //consumeFromWhere string - consumeType string - messageModel string - unitMode bool - + consumerGroup string + consumeType string + messageModel string + unitMode bool subscription map[string]string //topic|subExpression subscriptionTag map[string][]string // we use it filter again offsetStore service.OffsetStore mqClient service.RocketMqClient rebalance *service.Rebalance - pause bool //when reset offset we need pause + pause bool consumeMessageService service.ConsumeMessageService ConsumerConfig *rocketmq_api_model.RocketMqConsumerConfig } @@ -46,18 +44,17 @@ type DefaultMQPushConsumer struct { func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { defaultMQPushConsumer = &DefaultMQPushConsumer{ consumerGroup: consumerGroup, - //consumeFromWhere:"CONSUME_FROM_FIRST_OFFSET", //todo use config - consumeType: "CONSUME_PASSIVELY", - messageModel: "CLUSTERING", - pause: false} + consumeType: "CONSUME_PASSIVELY", + messageModel: "CLUSTERING", + pause: false} defaultMQPushConsumer.subscription = make(map[string]string) defaultMQPushConsumer.subscriptionTag = make(map[string][]string) defaultMQPushConsumer.ConsumerConfig = consumerConfig return } -func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { - self.subscription[topic] = subExpression +func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { + d.subscription[topic] = subExpression if len(subExpression) == 0 || subExpression == "*" { return } @@ -71,51 +68,51 @@ func (self *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) tagsList = append(tagsList, t) } if len(tagsList) > 0 { - self.subscriptionTag[topic] = tagsList + d.subscriptionTag[topic] = tagsList } } -func (self *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { - self.consumeMessageService = service.NewConsumeMessageConcurrentlyServiceImpl(messageListener) +func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { + d.consumeMessageService = service.NewConsumeMessageConcurrentlyServiceImpl(messageListener) } -func (self *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { - self.pause = true - glog.Info("now we ClearProcessQueue 0 ", offsetTable) +func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { + d.pause = true + glog.V(2).Info("now we ClearProcessQueue 0 ", offsetTable) - self.rebalance.ClearProcessQueue(offsetTable) - glog.Info("now we ClearProcessQueue", offsetTable) + d.rebalance.ClearProcessQueue(offsetTable) + glog.V(2).Info("now we ClearProcessQueue", offsetTable) go func() { waitTime := time.NewTimer(10 * time.Second) <-waitTime.C defer func() { - self.pause = false - self.rebalance.DoRebalance() + d.pause = false + d.rebalance.DoRebalance() }() for messageQueue, offset := range offsetTable { - processQueue := self.rebalance.GetProcessQueue(messageQueue) + processQueue := d.rebalance.GetProcessQueue(messageQueue) if processQueue == nil || offset < 0 { continue } glog.Info("now we UpdateOffset", messageQueue, offset) - self.offsetStore.UpdateOffset(&messageQueue, offset, false) - self.rebalance.RemoveProcessQueue(&messageQueue) + d.offsetStore.UpdateOffset(&messageQueue, offset, false) + d.rebalance.RemoveProcessQueue(&messageQueue) } }() } -func (self *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { +func (d *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { subscriptions := make([]*model.SubscriptionData, 0) - for _, subscription := range self.rebalance.SubscriptionInner { + for _, subscription := range d.rebalance.SubscriptionInner { subscriptions = append(subscriptions, subscription) } return subscriptions } -func (self *DefaultMQPushConsumer) CleanExpireMsg() { +func (d *DefaultMQPushConsumer) CleanExpireMsg() { nowTime := util.CurrentTimeMillisInt64() //will cause nowTime - consumeStartTime <0 ,but no matter - messageQueueList, processQueueList := self.rebalance.GetProcessQueueList() + messageQueueList, processQueueList := d.rebalance.GetProcessQueueList() for messageQueueIndex, processQueue := range processQueueList { loop := processQueue.GetMsgCount() if loop > 16 { @@ -127,8 +124,8 @@ func (self *DefaultMQPushConsumer) CleanExpireMsg() { break } consumeStartTime := message.GetConsumeStartTime() - maxDiffTime := self.ConsumerConfig.ConsumeTimeout * 1000 * 60 - //maxDiffTime := self.ConsumerConfig.ConsumeTimeout + maxDiffTime := d.ConsumerConfig.ConsumeTimeout * 1000 * 60 + //maxDiffTime := d.ConsumerConfig.ConsumeTimeout glog.V(2).Info("look message.GetConsumeStartTime()", consumeStartTime) glog.V(2).Infof("look diff %d %d", nowTime-consumeStartTime, maxDiffTime) //if(nowTime - consumeStartTime <0){ @@ -138,7 +135,7 @@ func (self *DefaultMQPushConsumer) CleanExpireMsg() { break } glog.Info("look now we send expire message back", message.Topic, message.MsgId) - err := self.consumeMessageService.SendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) + err := d.consumeMessageService.SendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) if err != nil { glog.Error("op=send_expire_message_back_error", err) continue diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index d99a9a7a0..560be85e3 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -18,7 +18,6 @@ package rocketmq import ( "bytes" - "compress/zlib" "encoding/binary" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" @@ -29,7 +28,6 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" - "io/ioutil" "strconv" "time" ) @@ -46,16 +44,16 @@ func NewPullMessageController(mqClient service.RocketMqClient, clientFactory *Cl } } -func (self *PullMessageController) Start() { +func (p *PullMessageController) Start() { go func() { for { - pullRequest := self.mqClient.DequeuePullMessageRequest() - self.pullMessage(pullRequest) + pullRequest := p.mqClient.DequeuePullMessageRequest() + p.pullMessage(pullRequest) } }() } -func (self *PullMessageController) needDelayPullMessage(mqPushConsumer *DefaultMQPushConsumer, pullRequest *model.PullRequest) (needDelayTime int64) { +func (p *PullMessageController) needDelayPullMessage(mqPushConsumer *DefaultMQPushConsumer, pullRequest *model.PullRequest) (needDelayTime int64) { if pullRequest.ProcessQueue.GetMsgCount() > mqPushConsumer.ConsumerConfig.PullThresholdForQueue { return mqPushConsumer.ConsumerConfig.PullTimeDelayMillsWhenFlowControl } @@ -65,35 +63,30 @@ func (self *PullMessageController) needDelayPullMessage(mqPushConsumer *DefaultM return } -func (self *PullMessageController) pullMessageLater(pullRequest *model.PullRequest, millisecond int64) { +func (p *PullMessageController) pullMessageLater(pullRequest *model.PullRequest, millisecond int64) { go func() { timeoutTimer := time.NewTimer(time.Duration(millisecond) * time.Millisecond) <-timeoutTimer.C - self.pullMessage(pullRequest) + p.pullMessage(pullRequest) }() return } -func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { - defaultMQPullConsumer := self.clientFactory.ConsumerTable[pullRequest.ConsumerGroup] +func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { + defaultMQPullConsumer := p.clientFactory.ConsumerTable[pullRequest.ConsumerGroup] if pullRequest.ProcessQueue.IsDropped() { return } - - //pullRequest.ProcessQueue.SetLastPullTimestamp(System.currentTimeMillis()); - // state ok - // isPause - - delayPullTime := self.needDelayPullMessage(defaultMQPullConsumer, pullRequest) + delayPullTime := p.needDelayPullMessage(defaultMQPullConsumer, pullRequest) if delayPullTime > 0 { - self.pullMessageLater(pullRequest, delayPullTime) + p.pullMessageLater(pullRequest, delayPullTime) return } commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, service.READ_FROM_MEMORY) subscriptionData, ok := defaultMQPullConsumer.rebalance.SubscriptionInner[pullRequest.MessageQueue.Topic] if !ok { - self.pullMessageLater(pullRequest, defaultMQPullConsumer.ConsumerConfig.PullTimeDelayMillsWhenException) + p.pullMessageLater(pullRequest, defaultMQPullConsumer.ConsumerConfig.PullTimeDelayMillsWhenException) return } @@ -123,7 +116,6 @@ func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { if responseFuture != nil { responseCommand := responseFuture.ResponseCommand if responseCommand.Code == remoting.SUCCESS && len(responseCommand.Body) > 0 { - //FOUND var err error pullResult := responseCommand.ExtFields if ok { @@ -149,7 +141,6 @@ func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { pullRequest.ProcessQueue.PutMessage(msgs) defaultMQPullConsumer.consumeMessageService.SubmitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) } else { - //glog.Error(fmt.Sprintf("pull message error,code=%d,body=%s", responseCommand.Code, string(responseCommand.Body))) var err error // change the offset , use nextBeginOffset pullResult := responseCommand.ExtFields if ok { @@ -203,14 +194,14 @@ func (self *PullMessageController) pullMessage(pullRequest *model.PullRequest) { go func() { nextPullTime := time.NewTimer(time.Duration(defaultMQPullConsumer.ConsumerConfig.PullInterval) * time.Millisecond) <-nextPullTime.C - self.mqClient.EnqueuePullMessageRequest(nextPullRequest) + p.mqClient.EnqueuePullMessageRequest(nextPullRequest) }() } else { - self.mqClient.EnqueuePullMessageRequest(nextPullRequest) + p.mqClient.EnqueuePullMessageRequest(nextPullRequest) } } glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) - self.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) + p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } func FilterMessageAgainByTags(msgExts []rocketmq_api_model.MessageExt, subscriptionTagList []string) (result []rocketmq_api_model.MessageExt) { result = msgExts @@ -229,11 +220,11 @@ func FilterMessageAgainByTags(msgExts []rocketmq_api_model.MessageExt, subscript return } -func (self *PullMessageController) consumerPullMessageAsync(brokerName string, requestHeader remoting.CustomerHeader, invokeCallback remoting.InvokeCallback) { - brokerAddr, _, found := self.mqClient.FindBrokerAddressInSubscribe(brokerName, 0, false) +func (p *PullMessageController) consumerPullMessageAsync(brokerName string, requestHeader remoting.CustomerHeader, invokeCallback remoting.InvokeCallback) { + brokerAddr, _, found := p.mqClient.FindBrokerAddressInSubscribe(brokerName, 0, false) if found { remotingCommand := remoting.NewRemotingCommand(remoting.PULL_MESSAGE, requestHeader) - self.mqClient.GetRemotingClient().InvokeAsync(brokerAddr, remotingCommand, 1000, invokeCallback) + p.mqClient.GetRemotingClient().InvokeAsync(brokerAddr, remotingCommand, 1000, invokeCallback) } } @@ -273,14 +264,8 @@ func DecodeMessage(data []byte) []rocketmq_api_model.MessageExt { body = make([]byte, bodyLength) binary.Read(buf, binary.BigEndian, body) if (sysFlag & constant.CompressedFlag) == constant.CompressedFlag { - b := bytes.NewReader(body) - z, err := zlib.NewReader(b) - if err != nil { - glog.Error(err) - return nil - } - body, err = ioutil.ReadAll(z) - z.Close() + var err error + body, err = util.UnCompress(body) if err != nil { glog.Error(err) return nil @@ -316,7 +301,6 @@ func DecodeMessage(data []byte) []rocketmq_api_model.MessageExt { msg.PreparedTransactionOffset = preparedTransactionOffset msg.Body = body msg.Properties = propertiesmap - // < 3.5.8 use messageOffsetId // >= 3.5.8 use clientUniqMsgId msg.MsgId = msg.GetMsgUniqueKey() diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/manage/tasks.go index 604222f21..70b4672c0 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/manage/tasks.go @@ -24,13 +24,13 @@ import ( type TaskManager struct { } -func (self MqClientManager) StartAllScheduledTask() { +func (m MqClientManager) StartAllScheduledTask() { rand.Seed(time.Now().UnixNano()) go func() { updateTopicRouteTimer := time.NewTimer(5 * time.Second) for { <-updateTopicRouteTimer.C - self.UpdateTopicRouteInfoFromNameServer() + m.UpdateTopicRouteInfoFromNameServer() updateTopicRouteTimer.Reset(5 * time.Second) } }() @@ -39,7 +39,7 @@ func (self MqClientManager) StartAllScheduledTask() { heartbeatTimer := time.NewTimer(10 * time.Second) for { <-heartbeatTimer.C - self.SendHeartbeatToAllBrokerWithLock() + m.SendHeartbeatToAllBrokerWithLock() heartbeatTimer.Reset(5 * time.Second) } }() @@ -48,7 +48,7 @@ func (self MqClientManager) StartAllScheduledTask() { rebalanceTimer := time.NewTimer(15 * time.Second) for { <-rebalanceTimer.C - self.rebalanceControllr.doRebalance() + m.rebalanceControllr.doRebalance() rebalanceTimer.Reset(30 * time.Second) } }() @@ -57,12 +57,12 @@ func (self MqClientManager) StartAllScheduledTask() { timeoutTimer := time.NewTimer(3 * time.Second) for { <-timeoutTimer.C - self.mqClient.ClearExpireResponse() + m.mqClient.ClearExpireResponse() timeoutTimer.Reset(time.Second) } }() - self.pullMessageController.Start() + m.pullMessageController.Start() //cleanExpireMsg - self.cleanExpireMsgController.Start() + m.cleanExpireMsgController.Start() } diff --git a/rocketmq-go/model/client_config.go b/rocketmq-go/model/client_config.go index 017c6d91a..e3c2fab7b 100644 --- a/rocketmq-go/model/client_config.go +++ b/rocketmq-go/model/client_config.go @@ -1,170 +1,171 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// package model -import ( - "bytes" - "time" -) - -// client common config -type ClientConfig struct { - nameServerAddress string // only this is in use - - clientIP string - instanceName string - clientCallbackExecutorThreads int // TODO: clientCallbackExecutorThreads - // Pulling topic information interval from the named server - pullNameServerInterval time.Duration // default 30 - // Heartbeat interval in microseconds with message broker - heartbeatBrokerInterval time.Duration // default 30 - // Offset persistent interval for consumer - persistConsumerOffsetInterval time.Duration // default 5 - unitMode bool - unitName string - vipChannelEnabled bool -} - -func NewClientConfig() *ClientConfig { - return &ClientConfig{ - unitMode: false, - pullNameServerInterval: time.Second * 30, - heartbeatBrokerInterval: time.Second * 30, - persistConsumerOffsetInterval: time.Second * 30, - } -} - -func (config *ClientConfig) BuildMQClientId() string { - var buffer bytes.Buffer - buffer.WriteString(config.clientIP) - buffer.WriteString("@") - buffer.WriteString(config.instanceName) - if config.unitName != "" { - buffer.WriteString("@") - buffer.WriteString(config.unitName) - } - return buffer.String() -} - -func (config *ClientConfig) ChangeInstanceNameToPID() { - // TODO -} - -func (config *ClientConfig) ResetClientConfig(cfg *ClientConfig) { - // TODO -} - -func (config *ClientConfig) CloneClientConfig() *ClientConfig { - return &ClientConfig{ - nameServerAddress: config.nameServerAddress, - clientIP: config.clientIP, - instanceName: config.instanceName, - clientCallbackExecutorThreads: config.clientCallbackExecutorThreads, - pullNameServerInterval: config.pullNameServerInterval, - heartbeatBrokerInterval: config.heartbeatBrokerInterval, - persistConsumerOffsetInterval: config.persistConsumerOffsetInterval, - unitMode: config.unitMode, - unitName: config.unitName, - vipChannelEnabled: config.vipChannelEnabled, - } -} - -func (config *ClientConfig) ClientIP() string { - return config.clientIP -} - -func (config *ClientConfig) SetClientIP(s string) { - config.clientIP = s -} - -func (config *ClientConfig) InstanceName() string { - return config.instanceName -} - -func (config *ClientConfig) SetInstanceName(s string) { - config.instanceName = s -} - -func (config *ClientConfig) NameServerAddress() string { - return config.nameServerAddress -} - -func (config *ClientConfig) SetNameServerAddress(s string) { - config.nameServerAddress = s -} - -func (config *ClientConfig) ClientCallbackExecutorThreads() int { - return config.clientCallbackExecutorThreads -} - -func (config *ClientConfig) SetClientCallbackExecutorThreads(threads int) { - config.clientCallbackExecutorThreads = threads -} - -func (config *ClientConfig) PullNameServerInteval() time.Duration { - return config.pullNameServerInterval -} - -func (config *ClientConfig) SetPullNameServerInteval(interval time.Duration) { - config.pullNameServerInterval = interval -} - -func (config *ClientConfig) HeartbeatBrokerInterval() time.Duration { - return config.heartbeatBrokerInterval -} - -func (config *ClientConfig) SetHeartbeatBrokerInterval(interval time.Duration) { - config.heartbeatBrokerInterval = interval -} - -func (config *ClientConfig) PersistConsumerOffsetInterval() time.Duration { - return config.persistConsumerOffsetInterval -} - -func (config *ClientConfig) SetPersistConsumerOffsetInterval(interval time.Duration) { - config.persistConsumerOffsetInterval = interval -} - -func (config *ClientConfig) UnitName() string { - return config.unitName -} - -func (config *ClientConfig) SetUnitName(name string) { - config.unitName = name -} - -func (config *ClientConfig) UnitMode() bool { - return config.unitMode -} - -func (config *ClientConfig) SetUnitMode(mode bool) { - config.unitMode = mode -} - -func (config *ClientConfig) VipChannelEnabled() bool { - return config.vipChannelEnabled -} - -func (config *ClientConfig) SetVipChannelEnabled(enable bool) { - config.vipChannelEnabled = enable -} - -func (config *ClientConfig) String() string { - //TODO - return "" -} +// +//import ( +// "bytes" +// "time" +//) +// +//// client common config +//type ClientConfig struct { +// nameServerAddress string // only this is in use +// +// clientIP string +// instanceName string +// clientCallbackExecutorThreads int // TODO: clientCallbackExecutorThreads +// // Pulling topic information interval from the named server +// pullNameServerInterval time.Duration // default 30 +// // Heartbeat interval in microseconds with message broker +// heartbeatBrokerInterval time.Duration // default 30 +// // Offset persistent interval for consumer +// persistConsumerOffsetInterval time.Duration // default 5 +// unitMode bool +// unitName string +// vipChannelEnabled bool +//} +// +//func NewClientConfig() *ClientConfig { +// return &ClientConfig{ +// unitMode: false, +// pullNameServerInterval: time.Second * 30, +// heartbeatBrokerInterval: time.Second * 30, +// persistConsumerOffsetInterval: time.Second * 30, +// } +//} +// +//func (config *ClientConfig) BuildMQClientId() string { +// var buffer bytes.Buffer +// buffer.WriteString(config.clientIP) +// buffer.WriteString("@") +// buffer.WriteString(config.instanceName) +// if config.unitName != "" { +// buffer.WriteString("@") +// buffer.WriteString(config.unitName) +// } +// return buffer.String() +//} +// +//func (config *ClientConfig) ChangeInstanceNameToPID() { +// // TODO +//} +// +//func (config *ClientConfig) ResetClientConfig(cfg *ClientConfig) { +// // TODO +//} +// +//func (config *ClientConfig) CloneClientConfig() *ClientConfig { +// return &ClientConfig{ +// nameServerAddress: config.nameServerAddress, +// clientIP: config.clientIP, +// instanceName: config.instanceName, +// clientCallbackExecutorThreads: config.clientCallbackExecutorThreads, +// pullNameServerInterval: config.pullNameServerInterval, +// heartbeatBrokerInterval: config.heartbeatBrokerInterval, +// persistConsumerOffsetInterval: config.persistConsumerOffsetInterval, +// unitMode: config.unitMode, +// unitName: config.unitName, +// vipChannelEnabled: config.vipChannelEnabled, +// } +//} +// +//func (config *ClientConfig) ClientIP() string { +// return config.clientIP +//} +// +//func (config *ClientConfig) SetClientIP(s string) { +// config.clientIP = s +//} +// +//func (config *ClientConfig) InstanceName() string { +// return config.instanceName +//} +// +//func (config *ClientConfig) SetInstanceName(s string) { +// config.instanceName = s +//} +// +//func (config *ClientConfig) NameServerAddress() string { +// return config.nameServerAddress +//} +// +//func (config *ClientConfig) SetNameServerAddress(s string) { +// config.nameServerAddress = s +//} +// +//func (config *ClientConfig) ClientCallbackExecutorThreads() int { +// return config.clientCallbackExecutorThreads +//} +// +//func (config *ClientConfig) SetClientCallbackExecutorThreads(threads int) { +// config.clientCallbackExecutorThreads = threads +//} +// +//func (config *ClientConfig) PullNameServerInteval() time.Duration { +// return config.pullNameServerInterval +//} +// +//func (config *ClientConfig) SetPullNameServerInteval(interval time.Duration) { +// config.pullNameServerInterval = interval +//} +// +//func (config *ClientConfig) HeartbeatBrokerInterval() time.Duration { +// return config.heartbeatBrokerInterval +//} +// +//func (config *ClientConfig) SetHeartbeatBrokerInterval(interval time.Duration) { +// config.heartbeatBrokerInterval = interval +//} +// +//func (config *ClientConfig) PersistConsumerOffsetInterval() time.Duration { +// return config.persistConsumerOffsetInterval +//} +// +//func (config *ClientConfig) SetPersistConsumerOffsetInterval(interval time.Duration) { +// config.persistConsumerOffsetInterval = interval +//} +// +//func (config *ClientConfig) UnitName() string { +// return config.unitName +//} +// +//func (config *ClientConfig) SetUnitName(name string) { +// config.unitName = name +//} +// +//func (config *ClientConfig) UnitMode() bool { +// return config.unitMode +//} +// +//func (config *ClientConfig) SetUnitMode(mode bool) { +// config.unitMode = mode +//} +// +//func (config *ClientConfig) VipChannelEnabled() bool { +// return config.vipChannelEnabled +//} +// +//func (config *ClientConfig) SetVipChannelEnabled(enable bool) { +// config.vipChannelEnabled = enable +//} +// +//func (config *ClientConfig) String() string { +// //TODO +// return "" +//} diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index 2c8f0a749..44fdd4a97 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -17,15 +17,11 @@ package constant const ( - ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME" - ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir" - NAMESRV_ADDR_ENV = "NAMESRV_ADDR" - NAMESRV_ADDR_PROPERTY = "rocketmq.namesrv.addr" - MESSAGE_COMPRESS_LEVEL = "rocketmq.message.compressLevel" - //WS_DOMAIN_NAME = System.getProperty("rocketmq.namesrv.domain", "jmenv.tbsite.net") - //WS_DOMAIN_SUBGROUP = System.getProperty("rocketmq.namesrv.domain.subgroup", "nsaddr") - // http://jmenv.tbsite.net:8080/rocketmq/nsaddr - //WS_ADDR = "http://" + WS_DOMAIN_NAME + ":8080/rocketmq/" + WS_DOMAIN_SUBGROUP + ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME" + ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir" + NAMESRV_ADDR_ENV = "NAMESRV_ADDR" + NAMESRV_ADDR_PROPERTY = "rocketmq.namesrv.addr" + MESSAGE_COMPRESS_LEVEL = "rocketmq.message.compressLevel" DEFAULT_TOPIC = "TBW102" BENCHMARK_TOPIC = "BenchmarkTest" DEFAULT_PRODUCER_GROUP = "DEFAULT_PRODUCER" diff --git a/rocketmq-go/model/consume_message_directly_result.go b/rocketmq-go/model/consume_message_directly_result.go index a9af32e43..8dcea7cba 100644 --- a/rocketmq-go/model/consume_message_directly_result.go +++ b/rocketmq-go/model/consume_message_directly_result.go @@ -16,16 +16,21 @@ */ package model +type ConsumeResultType string + +const ( + CR_SUCCESS ConsumeResultType = "CR_SUCCESS" + CR_LATER ConsumeResultType = "CR_LATER" + CR_ROLLBACK ConsumeResultType = "CR_ROLLBACK" + CR_COMMIT ConsumeResultType = "CR_COMMIT" + CR_THROW_EXCEPTION ConsumeResultType = "CR_THROW_EXCEPTION" + CR_RETURN_NULL ConsumeResultType = "CR_RETURN_NULL" +) + type ConsumeMessageDirectlyResult struct { - Order bool `json:"order"` - AutoCommit bool `json:"autoCommit"` - //CR_SUCCESS, - //CR_LATER, - //CR_ROLLBACK, - //CR_COMMIT, - //CR_THROW_EXCEPTION, - //CR_RETURN_NULL, - ConsumeResult string `json:"consumeResult"` - Remark string `json:"remark"` - SpentTimeMills int64 `json:"spentTimeMills"` + Order bool `json:"order"` + AutoCommit bool `json:"autoCommit"` + ConsumeResult ConsumeResultType `json:"consumeResult"` + Remark string `json:"remark"` + SpentTimeMills int64 `json:"spentTimeMills"` } diff --git a/rocketmq-go/model/consumer_running_info.go b/rocketmq-go/model/consumer_running_info.go index 80c39ae77..2d1bf1282 100644 --- a/rocketmq-go/model/consumer_running_info.go +++ b/rocketmq-go/model/consumer_running_info.go @@ -21,23 +21,15 @@ import "encoding/json" type ConsumerRunningInfo struct { Properties map[string]string `json:"properties"` MqTable map[MessageQueue]ProcessQueueInfo `json:"mqTable"` - // todo - //private TreeSet subscriptionSet = new TreeSet(); - // - //private TreeMap mqTable = new TreeMap(); - // - //private TreeMap statusTable = new TreeMap(); - // - //private String jstack; } -func (self *ConsumerRunningInfo) Encode() (jsonByte []byte, err error) { +func (c *ConsumerRunningInfo) Encode() (jsonByte []byte, err error) { mqTableJsonStr := "{" first := true var keyJson []byte var valueJson []byte - for key, value := range self.MqTable { + for key, value := range c.MqTable { keyJson, err = json.Marshal(key) if err != nil { return @@ -54,14 +46,14 @@ func (self *ConsumerRunningInfo) Encode() (jsonByte []byte, err error) { } mqTableJsonStr = mqTableJsonStr + "}" var propertiesJson []byte - propertiesJson, err = json.Marshal(self.Properties) + propertiesJson, err = json.Marshal(c.Properties) if err != nil { return } - jsonByte = self.formatEncode("\"properties\"", string(propertiesJson), "\"mqTable\"", string(mqTableJsonStr)) + jsonByte = c.formatEncode("\"properties\"", string(propertiesJson), "\"mqTable\"", string(mqTableJsonStr)) return } -func (self *ConsumerRunningInfo) formatEncode(kVList ...string) []byte { +func (c *ConsumerRunningInfo) formatEncode(kVList ...string) []byte { jsonStr := "{" first := true for i := 0; i+1 < len(kVList); i += 2 { diff --git a/rocketmq-go/model/header/consume_message_directly_result_request_header.go b/rocketmq-go/model/header/consume_message_directly_result_request_header.go index a593d51c3..9c2d77eb3 100644 --- a/rocketmq-go/model/header/consume_message_directly_result_request_header.go +++ b/rocketmq-go/model/header/consume_message_directly_result_request_header.go @@ -23,10 +23,10 @@ type ConsumeMessageDirectlyResultRequestHeader struct { BrokerName string `json:"brokerName"` } -func (self *ConsumeMessageDirectlyResultRequestHeader) FromMap(headerMap map[string]interface{}) { - self.ConsumerGroup = headerMap["consumerGroup"].(string) - self.ClientId = headerMap["clientId"].(string) - self.MsgId = headerMap["msgId"].(string) - self.BrokerName = headerMap["brokerName"].(string) +func (c *ConsumeMessageDirectlyResultRequestHeader) FromMap(headerMap map[string]interface{}) { + c.ConsumerGroup = headerMap["consumerGroup"].(string) + c.ClientId = headerMap["clientId"].(string) + c.MsgId = headerMap["msgId"].(string) + c.BrokerName = headerMap["brokerName"].(string) return } diff --git a/rocketmq-go/model/header/consumer_send_msg_back_request_header.go b/rocketmq-go/model/header/consumer_send_msg_back_request_header.go index 4e101c6c4..727914f63 100644 --- a/rocketmq-go/model/header/consumer_send_msg_back_request_header.go +++ b/rocketmq-go/model/header/consumer_send_msg_back_request_header.go @@ -26,6 +26,6 @@ type ConsumerSendMsgBackRequestHeader struct { MaxReconsumeTimes int32 } -func (self *ConsumerSendMsgBackRequestHeader) FromMap(headerMap map[string]interface{}) { +func (c *ConsumerSendMsgBackRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/get_consumer_list_by_group.go b/rocketmq-go/model/header/get_consumer_list_by_group.go index e06e1fa08..97b8304a6 100644 --- a/rocketmq-go/model/header/get_consumer_list_by_group.go +++ b/rocketmq-go/model/header/get_consumer_list_by_group.go @@ -20,7 +20,7 @@ type GetConsumerListByGroupRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` } -func (self *GetConsumerListByGroupRequestHeader) FromMap(headerMap map[string]interface{}) { +func (g *GetConsumerListByGroupRequestHeader) FromMap(headerMap map[string]interface{}) { return } @@ -28,6 +28,6 @@ type GetConsumerListByGroupResponseBody struct { ConsumerIdList []string } -func (self *GetConsumerListByGroupResponseBody) FromMap(headerMap map[string]interface{}) { +func (g *GetConsumerListByGroupResponseBody) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/get_consumer_running_info_request_header.go b/rocketmq-go/model/header/get_consumer_running_info_request_header.go index 5e7487fc1..2d5ab5459 100644 --- a/rocketmq-go/model/header/get_consumer_running_info_request_header.go +++ b/rocketmq-go/model/header/get_consumer_running_info_request_header.go @@ -22,8 +22,8 @@ type GetConsumerRunningInfoRequestHeader struct { JstackEnable bool `json:"jstackEnable"` } -func (self *GetConsumerRunningInfoRequestHeader) FromMap(headerMap map[string]interface{}) { - self.ConsumerGroup = headerMap["consumerGroup"].(string) - self.ClientId = headerMap["clientId"].(string) +func (g *GetConsumerRunningInfoRequestHeader) FromMap(headerMap map[string]interface{}) { + g.ConsumerGroup = headerMap["consumerGroup"].(string) + g.ClientId = headerMap["clientId"].(string) return } diff --git a/rocketmq-go/model/header/get_max_offset_request_header.go b/rocketmq-go/model/header/get_max_offset_request_header.go index 6d4723e6a..db8b765a6 100644 --- a/rocketmq-go/model/header/get_max_offset_request_header.go +++ b/rocketmq-go/model/header/get_max_offset_request_header.go @@ -21,6 +21,6 @@ type GetMaxOffsetRequestHeader struct { QueueId int32 `json:"queueId"` } -func (self *GetMaxOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { +func (g *GetMaxOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/get_max_offset_response_header.go b/rocketmq-go/model/header/get_max_offset_response_header.go index eea6c2cc0..59dd9baad 100644 --- a/rocketmq-go/model/header/get_max_offset_response_header.go +++ b/rocketmq-go/model/header/get_max_offset_response_header.go @@ -22,7 +22,7 @@ type QueryOffsetResponseHeader struct { Offset int64 `json:"offset"` } -func (self *QueryOffsetResponseHeader) FromMap(headerMap map[string]interface{}) { - self.Offset = util.StrToInt64WithDefaultValue(headerMap["offset"].(string), -1) +func (q *QueryOffsetResponseHeader) FromMap(headerMap map[string]interface{}) { + q.Offset = util.StrToInt64WithDefaultValue(headerMap["offset"].(string), -1) return } diff --git a/rocketmq-go/model/header/get_route_info_request_header.go b/rocketmq-go/model/header/get_route_info_request_header.go index 7c33c2520..2ebc20b73 100644 --- a/rocketmq-go/model/header/get_route_info_request_header.go +++ b/rocketmq-go/model/header/get_route_info_request_header.go @@ -20,6 +20,6 @@ type GetRouteInfoRequestHeader struct { Topic string `json:"topic"` } -func (self *GetRouteInfoRequestHeader) FromMap(headerMap map[string]interface{}) { +func (g *GetRouteInfoRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/pull_message_request_header.go b/rocketmq-go/model/header/pull_message_request_header.go index f341b49c1..966a9cd5b 100644 --- a/rocketmq-go/model/header/pull_message_request_header.go +++ b/rocketmq-go/model/header/pull_message_request_header.go @@ -29,6 +29,6 @@ type PullMessageRequestHeader struct { SubVersion int64 `json:"subVersion"` } -func (self *PullMessageRequestHeader) FromMap(headerMap map[string]interface{}) { +func (p *PullMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/query_consumer_offset_request_header.go b/rocketmq-go/model/header/query_consumer_offset_request_header.go index ed455e7ee..bdd693cc7 100644 --- a/rocketmq-go/model/header/query_consumer_offset_request_header.go +++ b/rocketmq-go/model/header/query_consumer_offset_request_header.go @@ -22,6 +22,6 @@ type QueryConsumerOffsetRequestHeader struct { QueueId int32 `json:"queueId"` } -func (self *QueryConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { +func (q *QueryConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/reset_offset_request_header.go b/rocketmq-go/model/header/reset_offset_request_header.go index 642b60015..9dd8ee601 100644 --- a/rocketmq-go/model/header/reset_offset_request_header.go +++ b/rocketmq-go/model/header/reset_offset_request_header.go @@ -28,10 +28,10 @@ type ResetOffsetRequestHeader struct { IsForce bool `json:"isForce"` } -func (self *ResetOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { - self.Group = headerMap["group"].(string) - self.Topic = headerMap["topic"].(string) - self.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) - self.IsForce, _ = strconv.ParseBool(headerMap["isForce"].(string)) +func (r *ResetOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { + r.Group = headerMap["group"].(string) + r.Topic = headerMap["topic"].(string) + r.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) + r.IsForce, _ = strconv.ParseBool(headerMap["isForce"].(string)) return } diff --git a/rocketmq-go/model/header/search_offset_request_header.go b/rocketmq-go/model/header/search_offset_request_header.go index 5088eac55..50944c1b9 100644 --- a/rocketmq-go/model/header/search_offset_request_header.go +++ b/rocketmq-go/model/header/search_offset_request_header.go @@ -25,9 +25,9 @@ type SearchOffsetRequestHeader struct { Timestamp int64 `json:"timestamp"` } -func (self *SearchOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { - self.Topic = headerMap["topic"].(string) - self.Topic = headerMap["queueId"].(string) - self.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) +func (s *SearchOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { + s.Topic = headerMap["topic"].(string) + s.Topic = headerMap["queueId"].(string) + s.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) return } diff --git a/rocketmq-go/model/header/send_message_request_header.go b/rocketmq-go/model/header/send_message_request_header.go index 80f17ccea..5eeea6053 100644 --- a/rocketmq-go/model/header/send_message_request_header.go +++ b/rocketmq-go/model/header/send_message_request_header.go @@ -31,6 +31,6 @@ type SendMessageRequestHeader struct { MaxReconsumeTimes int `json:"maxReconsumeTimes"` } -func (self *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { +func (s *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/model/header/update_consumer_offset_request_header.go b/rocketmq-go/model/header/update_consumer_offset_request_header.go index 42612db30..0826059d3 100644 --- a/rocketmq-go/model/header/update_consumer_offset_request_header.go +++ b/rocketmq-go/model/header/update_consumer_offset_request_header.go @@ -25,10 +25,10 @@ type UpdateConsumerOffsetRequestHeader struct { CommitOffset int64 `json:"commitOffset"` } -func (self *UpdateConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { - self.ConsumerGroup = headerMap["consumerGroup"].(string) - self.QueueId = util.StrToInt32WithDefaultValue(util.ReadString(headerMap["queueId"]), 0) - self.CommitOffset = util.StrToInt64WithDefaultValue(headerMap["commitOffset"].(string), -1) - self.Topic = util.ReadString(headerMap["topic"]) +func (u *UpdateConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { + u.ConsumerGroup = headerMap["consumerGroup"].(string) + u.QueueId = util.StrToInt32WithDefaultValue(util.ReadString(headerMap["queueId"]), 0) + u.CommitOffset = util.StrToInt64WithDefaultValue(headerMap["commitOffset"].(string), -1) + u.Topic = util.ReadString(headerMap["topic"]) return } diff --git a/rocketmq-go/model/message/message.go b/rocketmq-go/model/message/message.go index 1dcd3657e..7e42543ac 100644 --- a/rocketmq-go/model/message/message.go +++ b/rocketmq-go/model/message/message.go @@ -1,264 +1,265 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ package message -import ( - "bytes" - "compress/zlib" - "encoding/binary" - "encoding/json" - "fmt" - "github.com/golang/glog" - "io/ioutil" -) - -const ( - CompressedFlag = 1 << 0 - MultiTagsFlag = 1 << 1 - TransactionNotType = 0 << 2 - TransactionPreparedType = 1 << 2 - TransactionCommitType = 2 << 2 - TransactionRollbackType = 3 << 2 -) - -const ( - NameValueSeparator = 1 + iota - PropertySeparator -) - -const ( - CharacterMaxLength = 255 -) - -type Message struct { - Topic string - Flag int32 - properties map[string]string - Body []byte -} - -func NewDefultMessage(topic string, body []byte) *Message { - return NewMessage(topic, "", "", 0, body, true) -} - -type MessageExt struct { - Message - QueueId int32 - StoreSize int32 - QueueOffset int64 - SysFlag int32 - BornTimestamp int64 - // bornHost - StoreTimestamp int64 - // storeHost - MsgId string - CommitLogOffset int64 - BodyCRC int32 - ReconsumeTimes int32 - PreparedTransactionOffset int64 -} - -func (msg *Message) encodeMessage() []byte { - // TODO - return nil -} - -func decodeMessage(data []byte) []*MessageExt { - buf := bytes.NewBuffer(data) - var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 - var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 - var topicLen byte - var topic, body, properties, bornHost, storeHost []byte - var propertiesLength int16 - - var propertiesMap map[string]string - - msgs := make([]*MessageExt, 0, 32) - for buf.Len() > 0 { - msg := new(MessageExt) - binary.Read(buf, binary.BigEndian, &storeSize) - binary.Read(buf, binary.BigEndian, &magicCode) - binary.Read(buf, binary.BigEndian, &bodyCRC) - binary.Read(buf, binary.BigEndian, &queueId) - binary.Read(buf, binary.BigEndian, &flag) - binary.Read(buf, binary.BigEndian, &queueOffset) - binary.Read(buf, binary.BigEndian, &physicOffset) - binary.Read(buf, binary.BigEndian, &sysFlag) - binary.Read(buf, binary.BigEndian, &bornTimeStamp) - bornHost = make([]byte, 4) - binary.Read(buf, binary.BigEndian, &bornHost) - binary.Read(buf, binary.BigEndian, &bornPort) - binary.Read(buf, binary.BigEndian, &storeTimestamp) - storeHost = make([]byte, 4) - binary.Read(buf, binary.BigEndian, &storeHost) - binary.Read(buf, binary.BigEndian, &storePort) - binary.Read(buf, binary.BigEndian, &reconsumeTimes) - binary.Read(buf, binary.BigEndian, &preparedTransactionOffset) - binary.Read(buf, binary.BigEndian, &bodyLength) - if bodyLength > 0 { - body = make([]byte, bodyLength) - binary.Read(buf, binary.BigEndian, body) - - if (sysFlag & CompressedFlag) == CompressedFlag { - b := bytes.NewReader(body) - z, err := zlib.NewReader(b) - if err != nil { - fmt.Println(err) - return nil - } - - body, err = ioutil.ReadAll(z) - if err != nil { - fmt.Println(err) - return nil - } - z.Close() - } - - } - binary.Read(buf, binary.BigEndian, &topicLen) - topic = make([]byte, 0) - binary.Read(buf, binary.BigEndian, &topic) - binary.Read(buf, binary.BigEndian, &propertiesLength) - if propertiesLength > 0 { - properties = make([]byte, propertiesLength) - binary.Read(buf, binary.BigEndian, &properties) - propertiesMap = make(map[string]string) - json.Unmarshal(properties, &propertiesMap) - } - - if magicCode != -626843481 { - fmt.Printf("magic code is error %d", magicCode) - return nil - } - - msg.Topic = string(topic) - msg.QueueId = queueId - msg.SysFlag = sysFlag - msg.QueueOffset = queueOffset - msg.BodyCRC = bodyCRC - msg.StoreSize = storeSize - msg.BornTimestamp = bornTimeStamp - msg.ReconsumeTimes = reconsumeTimes - msg.Flag = flag - //msg.commitLogOffset=physicOffset - msg.StoreTimestamp = storeTimestamp - msg.PreparedTransactionOffset = preparedTransactionOffset - msg.Body = body - msg.properties = propertiesMap - - msgs = append(msgs, msg) - } - - return msgs -} - -func messageProperties2String(properties map[string]string) string { - StringBuilder := bytes.NewBuffer([]byte{}) - if properties != nil && len(properties) != 0 { - for k, v := range properties { - binary.Write(StringBuilder, binary.BigEndian, k) // 4 - binary.Write(StringBuilder, binary.BigEndian, NameValueSeparator) // 4 - binary.Write(StringBuilder, binary.BigEndian, v) // 4 - binary.Write(StringBuilder, binary.BigEndian, PropertySeparator) // 4 - } - } - return StringBuilder.String() -} - -//func (msg Message) checkMessage(producer *DefaultProducer) (err error) { -// if err = checkTopic(msg.Topic); err != nil { -// if len(msg.Body) == 0 { -// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body is null") -// } else if len(msg.Body) > producer.maxMessageSize { -// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body size over max value, MAX:" + strconv.Itoa(producer.maxMessageSize)) +// +//import ( +// "bytes" +// "compress/zlib" +// "encoding/binary" +// "encoding/json" +// "fmt" +// "github.com/golang/glog" +// "io/ioutil" +//) +// +//const ( +// CompressedFlag = 1 << 0 +// MultiTagsFlag = 1 << 1 +// TransactionNotType = 0 << 2 +// TransactionPreparedType = 1 << 2 +// TransactionCommitType = 2 << 2 +// TransactionRollbackType = 3 << 2 +//) +// +//const ( +// NameValueSeparator = 1 + iota +// PropertySeparator +//) +// +//const ( +// CharacterMaxLength = 255 +//) +// +//type Message struct { +// Topic string +// Flag int32 +// properties map[string]string +// Body []byte +//} +// +//func NewDefultMessage(topic string, body []byte) *Message { +// return NewMessage(topic, "", "", 0, body, true) +//} +// +//type MessageExt struct { +// Message +// QueueId int32 +// StoreSize int32 +// QueueOffset int64 +// SysFlag int32 +// BornTimestamp int64 +// // bornHost +// StoreTimestamp int64 +// // storeHost +// MsgId string +// CommitLogOffset int64 +// BodyCRC int32 +// ReconsumeTimes int32 +// PreparedTransactionOffset int64 +//} +// +//func (msg *Message) encodeMessage() []byte { +// // TODO +// return nil +//} +// +//func decodeMessage(data []byte) []*MessageExt { +// buf := bytes.NewBuffer(data) +// var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 +// var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 +// var topicLen byte +// var topic, body, properties, bornHost, storeHost []byte +// var propertiesLength int16 +// +// var propertiesMap map[string]string +// +// msgs := make([]*MessageExt, 0, 32) +// for buf.Len() > 0 { +// msg := new(MessageExt) +// binary.Read(buf, binary.BigEndian, &storeSize) +// binary.Read(buf, binary.BigEndian, &magicCode) +// binary.Read(buf, binary.BigEndian, &bodyCRC) +// binary.Read(buf, binary.BigEndian, &queueId) +// binary.Read(buf, binary.BigEndian, &flag) +// binary.Read(buf, binary.BigEndian, &queueOffset) +// binary.Read(buf, binary.BigEndian, &physicOffset) +// binary.Read(buf, binary.BigEndian, &sysFlag) +// binary.Read(buf, binary.BigEndian, &bornTimeStamp) +// bornHost = make([]byte, 4) +// binary.Read(buf, binary.BigEndian, &bornHost) +// binary.Read(buf, binary.BigEndian, &bornPort) +// binary.Read(buf, binary.BigEndian, &storeTimestamp) +// storeHost = make([]byte, 4) +// binary.Read(buf, binary.BigEndian, &storeHost) +// binary.Read(buf, binary.BigEndian, &storePort) +// binary.Read(buf, binary.BigEndian, &reconsumeTimes) +// binary.Read(buf, binary.BigEndian, &preparedTransactionOffset) +// binary.Read(buf, binary.BigEndian, &bodyLength) +// if bodyLength > 0 { +// body = make([]byte, bodyLength) +// binary.Read(buf, binary.BigEndian, body) +// +// if (sysFlag & CompressedFlag) == CompressedFlag { +// b := bytes.NewReader(body) +// z, err := zlib.NewReader(b) +// if err != nil { +// fmt.Println(err) +// return nil +// } +// +// body, err = ioutil.ReadAll(z) +// if err != nil { +// fmt.Println(err) +// return nil +// } +// z.Close() +// } +// // } +// binary.Read(buf, binary.BigEndian, &topicLen) +// topic = make([]byte, 0) +// binary.Read(buf, binary.BigEndian, &topic) +// binary.Read(buf, binary.BigEndian, &propertiesLength) +// if propertiesLength > 0 { +// properties = make([]byte, propertiesLength) +// binary.Read(buf, binary.BigEndian, &properties) +// propertiesMap = make(map[string]string) +// json.Unmarshal(properties, &propertiesMap) +// } +// +// if magicCode != -626843481 { +// fmt.Printf("magic code is error %d", magicCode) +// return nil +// } +// +// msg.Topic = string(topic) +// msg.QueueId = queueId +// msg.SysFlag = sysFlag +// msg.QueueOffset = queueOffset +// msg.BodyCRC = bodyCRC +// msg.StoreSize = storeSize +// msg.BornTimestamp = bornTimeStamp +// msg.ReconsumeTimes = reconsumeTimes +// msg.Flag = flag +// //msg.commitLogOffset=physicOffset +// msg.StoreTimestamp = storeTimestamp +// msg.PreparedTransactionOffset = preparedTransactionOffset +// msg.Body = body +// msg.properties = propertiesMap +// +// msgs = append(msgs, msg) // } -// return +// +// return msgs //} - -//func checkTopic(topic string) (err error) { -// if topic == "" { -// err = errors.New("the specified topic is blank") +// +//func messageProperties2String(properties map[string]string) string { +// StringBuilder := bytes.NewBuffer([]byte{}) +// if properties != nil && len(properties) != 0 { +// for k, v := range properties { +// binary.Write(StringBuilder, binary.BigEndian, k) // 4 +// binary.Write(StringBuilder, binary.BigEndian, NameValueSeparator) // 4 +// binary.Write(StringBuilder, binary.BigEndian, v) // 4 +// binary.Write(StringBuilder, binary.BigEndian, PropertySeparator) // 4 +// } +// } +// return StringBuilder.String() +//} +// +////func (msg Message) checkMessage(producer *DefaultProducer) (err error) { +//// if err = checkTopic(msg.Topic); err != nil { +//// if len(msg.Body) == 0 { +//// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body is null") +//// } else if len(msg.Body) > producer.maxMessageSize { +//// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body size over max value, MAX:" + strconv.Itoa(producer.maxMessageSize)) +//// } +//// } +//// return +////} +// +////func checkTopic(topic string) (err error) { +//// if topic == "" { +//// err = errors.New("the specified topic is blank") +//// } +//// if len(topic) > CharacterMaxLength { +//// err = errors.New("the specified topic is longer than topic max length 255") +//// } +//// if topic == DefaultTopic { +//// err = errors.New("the topic[" + topic + "] is conflict with default topic") +//// } +//// return +////} +// +//func NewMessage(topic, tags, keys string, flag int32, body []byte, waitStoreMsgOK bool) *Message { +// message := &Message{ +// Topic: topic, +// Flag: flag, +// Body: body, // } -// if len(topic) > CharacterMaxLength { -// err = errors.New("the specified topic is longer than topic max length 255") +// +// if tags != "" { +// message.SetTags(tags) // } -// if topic == DefaultTopic { -// err = errors.New("the topic[" + topic + "] is conflict with default topic") +// +// if keys != "" { +// message.SetKeys(keys) // } -// return +// +// message.SetWaitStoreMsgOK(waitStoreMsgOK) +// return message +//} +// +//func (msg *Message) SetTags(t string) { +// msg.putProperty(MessageConst.PropertyTags, t) +//} +// +//func (msg *Message) SetKeys(k string) { +// msg.putProperty(MessageConst.PropertyKeys, k) +//} +// +//func (msg *Message) SetWaitStoreMsgOK(b bool) { +// +//} +// +//func (msg *Message) Property() map[string]string { +// return msg.properties +//} +// +//func (msg *Message) putProperty(k, v string) { +// if msg.properties == nil { +// msg.properties = make(map[string]string) +// } +// if v, found := msg.properties[k]; !found { +// msg.properties[k] = v +// } else { +// glog.Infof("Message put peoperties key: %s existed.", k) +// } +//} +// +//func (msg *Message) removeProperty(k, v string) string { +// if v, ok := msg.properties[k]; ok { +// delete(msg.properties, k) +// return v +// } +// return "" +//} +// +//func (msg *Message) String() string { +// return fmt.Sprintf("Message [topic=%s, flag=%s, properties=%s, body=%s]", +// msg.Topic, msg.Flag, msg.properties, msg.Body) //} - -func NewMessage(topic, tags, keys string, flag int32, body []byte, waitStoreMsgOK bool) *Message { - message := &Message{ - Topic: topic, - Flag: flag, - Body: body, - } - - if tags != "" { - message.SetTags(tags) - } - - if keys != "" { - message.SetKeys(keys) - } - - message.SetWaitStoreMsgOK(waitStoreMsgOK) - return message -} - -func (msg *Message) SetTags(t string) { - msg.putProperty(MessageConst.PropertyTags, t) -} - -func (msg *Message) SetKeys(k string) { - msg.putProperty(MessageConst.PropertyKeys, k) -} - -func (msg *Message) SetWaitStoreMsgOK(b bool) { - -} - -func (msg *Message) Property() map[string]string { - return msg.properties -} - -func (msg *Message) putProperty(k, v string) { - if msg.properties == nil { - msg.properties = make(map[string]string) - } - if v, found := msg.properties[k]; !found { - msg.properties[k] = v - } else { - glog.Infof("Message put peoperties key: %s existed.", k) - } -} - -func (msg *Message) removeProperty(k, v string) string { - if v, ok := msg.properties[k]; ok { - delete(msg.properties, k) - return v - } - return "" -} - -func (msg *Message) String() string { - return fmt.Sprintf("Message [topic=%s, flag=%s, properties=%s, body=%s]", - msg.Topic, msg.Flag, msg.properties, msg.Body) -} diff --git a/rocketmq-go/model/message/message_constant.go b/rocketmq-go/model/message/message_constant.go index 3991c7e3c..b1d731e04 100644 --- a/rocketmq-go/model/message/message_constant.go +++ b/rocketmq-go/model/message/message_constant.go @@ -1,98 +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. - */ - +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// package message -type messageConst struct { - PropertyKeys string - PropertyTags string - PropertyWaitStoreMsgOk string - PropertyDelayTimeLevel string - PropertyRetryTopic string - PropertyRealTopic string - PropertyRealQueueId string - PropertyTransactionPrepared string - PropertyProducerGroup string - PropertyMinOffset string - PropertyMaxOffset string - PropertyBuyerId string - PropertyOriginMessageId string - PropertyTransferFlag string - PropertyCorrectionFlag string - PropertyMq2Flag string - PropertyReconsumeTime string - PropertyMsgRegion string - PropertyUniqClientMessageIdKeyidx string - PropertyMaxReconsumeTimes string - PropertyConsumeStartTimeStamp string - - KeySeparator string - systemKeySet []string -} - -var MessageConst = &messageConst{ - PropertyKeys: "KEYS", - PropertyTags: "TAGS", - PropertyWaitStoreMsgOk: "WAIT", - PropertyDelayTimeLevel: "DELAY", - PropertyRetryTopic: "RETRY_TOPIC", - PropertyRealTopic: "REAL_TOPIC", - PropertyRealQueueId: "REAL_QID", - PropertyTransactionPrepared: "TRAN_MSG", - PropertyProducerGroup: "PGROUP", - PropertyMinOffset: "MIN_OFFSET", - PropertyMaxOffset: "MAX_OFFSET", - PropertyBuyerId: "BUYER_ID", - PropertyOriginMessageId: "ORIGIN_MESSAGE_ID", - PropertyTransferFlag: "TRANSFER_FLAG", - PropertyCorrectionFlag: "CORRECTION_FLAG", - PropertyMq2Flag: "MQ2_FLAG", - PropertyReconsumeTime: "RECONSUME_TIME", - PropertyMsgRegion: "MSG_REGION", - PropertyUniqClientMessageIdKeyidx: "UNIQ_KEY", - PropertyMaxReconsumeTimes: "MAX_RECONSUME_TIMES", - PropertyConsumeStartTimeStamp: "CONSUME_START_TIME", - - KeySeparator: "", -} - -func init() { - var systemKeySet = []string{} - systemKeySet = append(systemKeySet, MessageConst.PropertyKeys) - systemKeySet = append(systemKeySet, MessageConst.PropertyTags) - systemKeySet = append(systemKeySet, MessageConst.PropertyWaitStoreMsgOk) - systemKeySet = append(systemKeySet, MessageConst.PropertyDelayTimeLevel) - systemKeySet = append(systemKeySet, MessageConst.PropertyRetryTopic) - systemKeySet = append(systemKeySet, MessageConst.PropertyRealTopic) - systemKeySet = append(systemKeySet, MessageConst.PropertyRealQueueId) - systemKeySet = append(systemKeySet, MessageConst.PropertyTransactionPrepared) - systemKeySet = append(systemKeySet, MessageConst.PropertyProducerGroup) - systemKeySet = append(systemKeySet, MessageConst.PropertyMinOffset) - systemKeySet = append(systemKeySet, MessageConst.PropertyMaxOffset) - systemKeySet = append(systemKeySet, MessageConst.PropertyBuyerId) - systemKeySet = append(systemKeySet, MessageConst.PropertyOriginMessageId) - systemKeySet = append(systemKeySet, MessageConst.PropertyTransferFlag) - systemKeySet = append(systemKeySet, MessageConst.PropertyCorrectionFlag) - systemKeySet = append(systemKeySet, MessageConst.PropertyMq2Flag) - systemKeySet = append(systemKeySet, MessageConst.PropertyReconsumeTime) - systemKeySet = append(systemKeySet, MessageConst.PropertyMsgRegion) - systemKeySet = append(systemKeySet, MessageConst.PropertyUniqClientMessageIdKeyidx) - systemKeySet = append(systemKeySet, MessageConst.PropertyMaxReconsumeTimes) - systemKeySet = append(systemKeySet, MessageConst.PropertyConsumeStartTimeStamp) - - MessageConst.systemKeySet = systemKeySet -} +// +//type messageConst struct { +// PropertyKeys string +// PropertyTags string +// PropertyWaitStoreMsgOk string +// PropertyDelayTimeLevel string +// PropertyRetryTopic string +// PropertyRealTopic string +// PropertyRealQueueId string +// PropertyTransactionPrepared string +// PropertyProducerGroup string +// PropertyMinOffset string +// PropertyMaxOffset string +// PropertyBuyerId string +// PropertyOriginMessageId string +// PropertyTransferFlag string +// PropertyCorrectionFlag string +// PropertyMq2Flag string +// PropertyReconsumeTime string +// PropertyMsgRegion string +// PropertyUniqClientMessageIdKeyidx string +// PropertyMaxReconsumeTimes string +// PropertyConsumeStartTimeStamp string +// +// KeySeparator string +// systemKeySet []string +//} +// +//var MessageConst = &messageConst{ +// PropertyKeys: "KEYS", +// PropertyTags: "TAGS", +// PropertyWaitStoreMsgOk: "WAIT", +// PropertyDelayTimeLevel: "DELAY", +// PropertyRetryTopic: "RETRY_TOPIC", +// PropertyRealTopic: "REAL_TOPIC", +// PropertyRealQueueId: "REAL_QID", +// PropertyTransactionPrepared: "TRAN_MSG", +// PropertyProducerGroup: "PGROUP", +// PropertyMinOffset: "MIN_OFFSET", +// PropertyMaxOffset: "MAX_OFFSET", +// PropertyBuyerId: "BUYER_ID", +// PropertyOriginMessageId: "ORIGIN_MESSAGE_ID", +// PropertyTransferFlag: "TRANSFER_FLAG", +// PropertyCorrectionFlag: "CORRECTION_FLAG", +// PropertyMq2Flag: "MQ2_FLAG", +// PropertyReconsumeTime: "RECONSUME_TIME", +// PropertyMsgRegion: "MSG_REGION", +// PropertyUniqClientMessageIdKeyidx: "UNIQ_KEY", +// PropertyMaxReconsumeTimes: "MAX_RECONSUME_TIMES", +// PropertyConsumeStartTimeStamp: "CONSUME_START_TIME", +// +// KeySeparator: "", +//} +// +//func init() { +// var systemKeySet = []string{} +// systemKeySet = append(systemKeySet, MessageConst.PropertyKeys) +// systemKeySet = append(systemKeySet, MessageConst.PropertyTags) +// systemKeySet = append(systemKeySet, MessageConst.PropertyWaitStoreMsgOk) +// systemKeySet = append(systemKeySet, MessageConst.PropertyDelayTimeLevel) +// systemKeySet = append(systemKeySet, MessageConst.PropertyRetryTopic) +// systemKeySet = append(systemKeySet, MessageConst.PropertyRealTopic) +// systemKeySet = append(systemKeySet, MessageConst.PropertyRealQueueId) +// systemKeySet = append(systemKeySet, MessageConst.PropertyTransactionPrepared) +// systemKeySet = append(systemKeySet, MessageConst.PropertyProducerGroup) +// systemKeySet = append(systemKeySet, MessageConst.PropertyMinOffset) +// systemKeySet = append(systemKeySet, MessageConst.PropertyMaxOffset) +// systemKeySet = append(systemKeySet, MessageConst.PropertyBuyerId) +// systemKeySet = append(systemKeySet, MessageConst.PropertyOriginMessageId) +// systemKeySet = append(systemKeySet, MessageConst.PropertyTransferFlag) +// systemKeySet = append(systemKeySet, MessageConst.PropertyCorrectionFlag) +// systemKeySet = append(systemKeySet, MessageConst.PropertyMq2Flag) +// systemKeySet = append(systemKeySet, MessageConst.PropertyReconsumeTime) +// systemKeySet = append(systemKeySet, MessageConst.PropertyMsgRegion) +// systemKeySet = append(systemKeySet, MessageConst.PropertyUniqClientMessageIdKeyidx) +// systemKeySet = append(systemKeySet, MessageConst.PropertyMaxReconsumeTimes) +// systemKeySet = append(systemKeySet, MessageConst.PropertyConsumeStartTimeStamp) +// +// MessageConst.systemKeySet = systemKeySet +//} diff --git a/rocketmq-go/model/message_queue.go b/rocketmq-go/model/message_queue.go index 27d70a6ab..fc5c8608f 100644 --- a/rocketmq-go/model/message_queue.go +++ b/rocketmq-go/model/message_queue.go @@ -22,19 +22,19 @@ type MessageQueue struct { QueueId int32 `json:"queueId"` } -func (self *MessageQueue) clone() *MessageQueue { +func (m *MessageQueue) clone() *MessageQueue { no := new(MessageQueue) - no.Topic = self.Topic - no.QueueId = self.QueueId - no.BrokerName = self.BrokerName + no.Topic = m.Topic + no.QueueId = m.QueueId + no.BrokerName = m.BrokerName return no } type MessageQueues []*MessageQueue -func (self MessageQueues) Less(i, j int) bool { - imq := self[i] - jmq := self[j] +func (m MessageQueues) Less(i, j int) bool { + imq := m[i] + jmq := m[j] if imq.Topic < jmq.Topic { return true @@ -55,22 +55,22 @@ func (self MessageQueues) Less(i, j int) bool { } } -func (self MessageQueues) Swap(i, j int) { - self[i], self[j] = self[j], self[i] +func (m MessageQueues) Swap(i, j int) { + m[i], m[j] = m[j], m[i] } -func (self MessageQueues) Len() int { - return len(self) +func (m MessageQueues) Len() int { + return len(m) } -func (self MessageQueue) Equals(messageQueue *MessageQueue) bool { - if self.QueueId != messageQueue.QueueId { +func (m MessageQueue) Equals(messageQueue *MessageQueue) bool { + if m.QueueId != messageQueue.QueueId { return false } - if self.Topic != messageQueue.Topic { + if m.Topic != messageQueue.Topic { return false } - if self.BrokerName != messageQueue.BrokerName { + if m.BrokerName != messageQueue.BrokerName { return false } return true diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index 8a94b134f..a3e178f8e 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -51,67 +51,64 @@ func NewProcessQueue() (processQueue *ProcessQueue) { return } -func (self *ProcessQueue) GetMsgCount() int { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - return self.msgCount +func (p *ProcessQueue) GetMsgCount() int { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + return p.msgCount } -func (self *ProcessQueue) Clear() { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - self.SetDrop(true) - self.msgTreeMap.Clear() - self.msgCount = 0 - self.queueOffsetMax = 0 +func (p *ProcessQueue) Clear() { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + p.SetDrop(true) + p.msgTreeMap.Clear() + p.msgCount = 0 + p.queueOffsetMax = 0 } -func (self *ProcessQueue) ChangeToProcessQueueInfo() (processQueueInfo ProcessQueueInfo) { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() +func (p *ProcessQueue) ChangeToProcessQueueInfo() (processQueueInfo ProcessQueueInfo) { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() processQueueInfo = ProcessQueueInfo{} minOffset := -1 maxOffset := -1 - minKey, _ := self.msgTreeMap.Min() + minKey, _ := p.msgTreeMap.Min() if minKey != nil { minOffset = minKey.(int) } - maxKey, _ := self.msgTreeMap.Max() + maxKey, _ := p.msgTreeMap.Max() if maxKey != nil { maxOffset = maxKey.(int) } - processQueueInfo.CachedMsgCount = int32(self.msgCount) + processQueueInfo.CachedMsgCount = int32(p.msgCount) processQueueInfo.CachedMsgMinOffset = int64(maxOffset) processQueueInfo.CachedMsgMaxOffset = int64(minOffset) - //processQueueInfo.CommitOffset = -123 // todo - processQueueInfo.Droped = self.dropped - processQueueInfo.LastConsumeTimestamp = self.lastConsumeTimestamp.UnixNano() - processQueueInfo.LastPullTimestamp = self.lastPullTimestamp.UnixNano() - //processQueueInfo. - + processQueueInfo.Droped = p.dropped + processQueueInfo.LastConsumeTimestamp = p.lastConsumeTimestamp.UnixNano() + processQueueInfo.LastPullTimestamp = p.lastPullTimestamp.UnixNano() return } -func (self *ProcessQueue) DeleteExpireMsg(queueOffset int) { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - key, _ := self.msgTreeMap.Min() +func (p *ProcessQueue) DeleteExpireMsg(queueOffset int) { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + key, _ := p.msgTreeMap.Min() if key == nil { return } offset := key.(int) - glog.Infof("look min key and offset %d %s", offset, queueOffset) + glog.V(2).Infof("look min key and offset %d %s", offset, queueOffset) if queueOffset == offset { - self.msgTreeMap.Remove(queueOffset) - self.msgCount = self.msgTreeMap.Size() + p.msgTreeMap.Remove(queueOffset) + p.msgCount = p.msgTreeMap.Size() } } -func (self *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq_api_model.MessageExt) { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - key, value := self.msgTreeMap.Min() +func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq_api_model.MessageExt) { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + key, value := p.msgTreeMap.Min() if key == nil || value == nil { return } @@ -122,72 +119,72 @@ func (self *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocke return } -func (self *ProcessQueue) SetDrop(drop bool) { - self.dropped = drop +func (p *ProcessQueue) SetDrop(drop bool) { + p.dropped = drop } -func (self *ProcessQueue) IsDropped() bool { - return self.dropped +func (p *ProcessQueue) IsDropped() bool { + return p.dropped } -func (self *ProcessQueue) GetMaxSpan() int { - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - if self.msgTreeMap.Empty() { +func (p *ProcessQueue) GetMaxSpan() int { + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + if p.msgTreeMap.Empty() { return 0 } - minKey, _ := self.msgTreeMap.Min() + minKey, _ := p.msgTreeMap.Min() minOffset := minKey.(int) - maxKey, _ := self.msgTreeMap.Max() + maxKey, _ := p.msgTreeMap.Max() maxOffset := maxKey.(int) return maxOffset - minOffset } -func (self *ProcessQueue) RemoveMessage(msgs []rocketmq_api_model.MessageExt) (offset int64) { +func (p *ProcessQueue) RemoveMessage(msgs []rocketmq_api_model.MessageExt) (offset int64) { now := time.Now() offset = -1 - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() - self.lastConsumeTimestamp = now - if self.msgCount > 0 { - maxKey, _ := self.msgTreeMap.Max() + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() + p.lastConsumeTimestamp = now + if p.msgCount > 0 { + maxKey, _ := p.msgTreeMap.Max() offset = int64(maxKey.(int)) + 1 for _, msg := range msgs { - self.msgTreeMap.Remove(int(msg.QueueOffset)) + p.msgTreeMap.Remove(int(msg.QueueOffset)) } - self.msgCount = self.msgTreeMap.Size() - if self.msgCount > 0 { - minKey, _ := self.msgTreeMap.Min() + p.msgCount = p.msgTreeMap.Size() + if p.msgCount > 0 { + minKey, _ := p.msgTreeMap.Min() offset = int64(minKey.(int)) } } return } -func (self *ProcessQueue) PutMessage(msgs []rocketmq_api_model.MessageExt) (dispatchToConsume bool) { +func (p *ProcessQueue) PutMessage(msgs []rocketmq_api_model.MessageExt) (dispatchToConsume bool) { dispatchToConsume = false msgsLen := len(msgs) if msgsLen == 0 { return } - defer self.lockTreeMap.Unlock() - self.lockTreeMap.Lock() + defer p.lockTreeMap.Unlock() + p.lockTreeMap.Lock() for _, msg := range msgs { - self.msgTreeMap.Put(int(msg.QueueOffset), msg) + p.msgTreeMap.Put(int(msg.QueueOffset), msg) } - self.msgCount = self.msgTreeMap.Size() - maxOffset, _ := self.msgTreeMap.Max() - self.queueOffsetMax = int64(maxOffset.(int)) - if self.msgCount > 0 && !self.consuming { + p.msgCount = p.msgTreeMap.Size() + maxOffset, _ := p.msgTreeMap.Max() + p.queueOffsetMax = int64(maxOffset.(int)) + if p.msgCount > 0 && !p.consuming { dispatchToConsume = true - self.consuming = true + p.consuming = true } lastMsg := msgs[msgsLen-1] remoteMaxOffset := util.StrToInt64WithDefaultValue(lastMsg.Properties[constant.PROPERTY_MAX_OFFSET], -1) if remoteMaxOffset > 0 { accTotal := remoteMaxOffset - lastMsg.QueueOffset if accTotal > 0 { - self.msgAccCnt = accTotal + p.msgAccCnt = accTotal } } return diff --git a/rocketmq-go/model/pull_result.go b/rocketmq-go/model/pull_result.go index b34a2d9a9..694b88f92 100644 --- a/rocketmq-go/model/pull_result.go +++ b/rocketmq-go/model/pull_result.go @@ -1,79 +1,80 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// package model -import ( - "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -) - -type PullStatus int - -const ( - Found PullStatus = iota - NoNewMsg - NoMatchedMsg - OffsetIllegal -) - -type PullResult struct { - pullStatus PullStatus - nextBeginOffset int64 - minOffset int64 - maxOffset int64 - msgFoundList []*message.MessageExt -} - -func NewPullResult(ps PullStatus, next, min, max int64, list []*message.MessageExt) *PullResult { - return &PullResult{ - ps, - next, - min, - max, - list, - } -} - -func (result *PullResult) PullStatus() PullStatus { - return result.pullStatus -} - -func (result *PullResult) NextBeginOffset() int64 { - return result.nextBeginOffset -} - -func (result *PullResult) MaxOffset() int64 { - return result.maxOffset -} - -func (result *PullResult) MinOffset() int64 { - return result.minOffset -} - -func (result *PullResult) MsgFoundList() []*message.MessageExt { - return result.msgFoundList -} - -func (result *PullResult) SetMsgFoundList(list []*message.MessageExt) { - result.msgFoundList = list -} - -func (result *PullResult) String() string { - return fmt.Sprintf("PullResult [pullStatus=%s, nextBeginOffset=%s, minOffset=%s, maxOffset=%s, msgFoundList=%s]", - result.pullStatus, result.nextBeginOffset, result.minOffset, result.maxOffset, len(result.msgFoundList)) -} +// +//import ( +// "fmt" +// "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" +//) +// +//type PullStatus int +// +//const ( +// Found PullStatus = iota +// NoNewMsg +// NoMatchedMsg +// OffsetIllegal +//) +// +//type PullResult struct { +// pullStatus PullStatus +// nextBeginOffset int64 +// minOffset int64 +// maxOffset int64 +// msgFoundList []*message.MessageExt +//} +// +//func NewPullResult(ps PullStatus, next, min, max int64, list []*message.MessageExt) *PullResult { +// return &PullResult{ +// ps, +// next, +// min, +// max, +// list, +// } +//} +// +//func (result *PullResult) PullStatus() PullStatus { +// return result.pullStatus +//} +// +//func (result *PullResult) NextBeginOffset() int64 { +// return result.nextBeginOffset +//} +// +//func (result *PullResult) MaxOffset() int64 { +// return result.maxOffset +//} +// +//func (result *PullResult) MinOffset() int64 { +// return result.minOffset +//} +// +//func (result *PullResult) MsgFoundList() []*message.MessageExt { +// return result.msgFoundList +//} +// +//func (result *PullResult) SetMsgFoundList(list []*message.MessageExt) { +// result.msgFoundList = list +//} +// +//func (result *PullResult) String() string { +// return fmt.Sprintf("PullResult [pullStatus=%s, nextBeginOffset=%s, minOffset=%s, maxOffset=%s, msgFoundList=%s]", +// result.pullStatus, result.nextBeginOffset, result.minOffset, result.maxOffset, len(result.msgFoundList)) +//} diff --git a/rocketmq-go/model/query_result.go b/rocketmq-go/model/query_result.go index b9e923691..9cdf9a360 100644 --- a/rocketmq-go/model/query_result.go +++ b/rocketmq-go/model/query_result.go @@ -1,48 +1,49 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - +///* +// * Licensed to the Apache Software Foundation (ASF) under one or more +// * contributor license agreements. See the NOTICE file distributed with +// * this work for additional information regarding copyright ownership. +// * The ASF licenses this file to You under the Apache License, Version 2.0 +// * (the "License"); you may not use this file except in compliance with +// * the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// package model -import ( - "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -) - -type QueryResult struct { - indexLastUpdateTimestamp int64 - messageList []*message.MessageExt -} - -func NewQueryResult(timestamp int64, list []*message.MessageExt) *QueryResult { - return &QueryResult{ - indexLastUpdateTimestamp: timestamp, - messageList: list, - } -} - -func (qr *QueryResult) IndexLastUpdateTimestamp() int64 { - return qr.indexLastUpdateTimestamp -} - -func (qr *QueryResult) MessageList() []*message.MessageExt { //TODO: address? - return qr.messageList -} - -func (qr *QueryResult) String() string { - return fmt.Sprintf("QueryResult [indexLastUpdateTimestamp=%s, messageList=%s]", - qr.indexLastUpdateTimestamp, qr.messageList) -} +// +//import ( +// "fmt" +// "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" +//) +// +//type QueryResult struct { +// indexLastUpdateTimestamp int64 +// messageList []*message.MessageExt +//} +// +//func NewQueryResult(timestamp int64, list []*message.MessageExt) *QueryResult { +// return &QueryResult{ +// indexLastUpdateTimestamp: timestamp, +// messageList: list, +// } +//} +// +//func (qr *QueryResult) IndexLastUpdateTimestamp() int64 { +// return qr.indexLastUpdateTimestamp +//} +// +//func (qr *QueryResult) MessageList() []*message.MessageExt { //TODO: address? +// return qr.messageList +//} +// +//func (qr *QueryResult) String() string { +// return fmt.Sprintf("QueryResult [indexLastUpdateTimestamp=%s, messageList=%s]", +// qr.indexLastUpdateTimestamp, qr.messageList) +//} diff --git a/rocketmq-go/model/reset_offset_body.go b/rocketmq-go/model/reset_offset_body.go index 1a0221d81..d2db83255 100644 --- a/rocketmq-go/model/reset_offset_body.go +++ b/rocketmq-go/model/reset_offset_body.go @@ -26,8 +26,8 @@ type ResetOffsetBody struct { OffsetTable map[MessageQueue]int64 `json:"offsetTable"` } -func (self *ResetOffsetBody) Decode(data []byte) (err error) { - self.OffsetTable = map[MessageQueue]int64{} +func (r *ResetOffsetBody) Decode(data []byte) (err error) { + r.OffsetTable = map[MessageQueue]int64{} var kvMap map[string]string kvMap, err = util.GetKvStringMap(string(data)) if err != nil { @@ -49,7 +49,7 @@ func (self *ResetOffsetBody) Decode(data []byte) (err error) { if err != nil { return } - self.OffsetTable[*messageQueue] = offset + r.OffsetTable[*messageQueue] = offset } return } diff --git a/rocketmq-go/model/topic_publish_info.go b/rocketmq-go/model/topic_publish_info.go index 26a541cd3..6fa6f0cf9 100644 --- a/rocketmq-go/model/topic_publish_info.go +++ b/rocketmq-go/model/topic_publish_info.go @@ -29,14 +29,14 @@ type TopicPublishInfo struct { topicQueueIndex int32 } -func (self *TopicPublishInfo) JudgeTopicPublishInfoOk() (bIsTopicOk bool) { - bIsTopicOk = (len(self.MessageQueueList) > 0) +func (t *TopicPublishInfo) JudgeTopicPublishInfoOk() (bIsTopicOk bool) { + bIsTopicOk = (len(t.MessageQueueList) > 0) return } -func (self *TopicPublishInfo) FetchQueueIndex() (index int) { - qLen := len(self.MessageQueueList) +func (t *TopicPublishInfo) FetchQueueIndex() (index int) { + qLen := len(t.MessageQueueList) if qLen > 0 { - qIndex := atomic.AddInt32(&self.topicQueueIndex, 1) + qIndex := atomic.AddInt32(&t.topicQueueIndex, 1) qIndex = qIndex % int32(qLen) index = int(qIndex) } @@ -62,7 +62,6 @@ func BuildTopicSubscribeInfoFromRoteData(topic string, topicRouteData *TopicRout } func BuildTopicPublishInfoFromTopicRoteData(topic string, topicRouteData *TopicRouteData) (topicPublishInfo *TopicPublishInfo) { - // all order topic is false todo change topicPublishInfo = &TopicPublishInfo{ TopicRouteDataInstance: topicRouteData, OrderTopic: false, diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index f2ce87b03..7fbb20c58 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -43,14 +43,12 @@ type DefaultRemotingClient struct { connTable map[string]net.Conn connTableLock sync.RWMutex - responseTable util.ConcurrentMap //map[int32]*ResponseFuture - processorTable util.ConcurrentMap //map[int]ClientRequestProcessor //requestCode|ClientRequestProcessor - // protected final HashMap> processorTable = - //new HashMap>(64); + responseTable util.ConcurrentMap //map[int32]*ResponseFuture + processorTable util.ConcurrentMap //map[int]ClientRequestProcessor //requestCode|ClientRequestProcessor namesrvAddrList []string namesrvAddrSelectedAddr string - namesrvAddrSelectedIndex int //how to chose. done - namesvrLockRW sync.RWMutex // + namesrvAddrSelectedIndex int + namesvrLockRW sync.RWMutex clientRequestProcessor ClientRequestProcessor //mange register the processor here serializerHandler SerializerHandler //rocketmq encode decode } diff --git a/rocketmq-go/remoting/remoting_command.go b/rocketmq-go/remoting/remoting_command.go index a8361bd04..748117488 100644 --- a/rocketmq-go/remoting/remoting_command.go +++ b/rocketmq-go/remoting/remoting_command.go @@ -31,7 +31,6 @@ var RPC_ONEWAY int = 1 // 0, RPC var RESPONSE_TYPE int = 1 type RemotingCommand struct { - //header Code int16 `json:"code"` Language string `json:"language"` //int 8 Version int16 `json:"version"` @@ -61,9 +60,9 @@ func NewRemotingCommandWithBody(commandCode int16, customerHeader CustomerHeader return remotingCommand } -func (self *RemotingCommand) IsResponseType() bool { - return self.Flag&(RESPONSE_TYPE) == RESPONSE_TYPE +func (r *RemotingCommand) IsResponseType() bool { + return r.Flag&(RESPONSE_TYPE) == RESPONSE_TYPE } -func (self *RemotingCommand) MarkResponseType() { - self.Flag = (self.Flag | RESPONSE_TYPE) +func (r *RemotingCommand) MarkResponseType() { + r.Flag = (r.Flag | RESPONSE_TYPE) } diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index 4bc9c146c..4e59f228c 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -33,7 +33,7 @@ const ( value_item ) -func (self *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { +func (r *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { var ( remarkBytes []byte remarkBytesLen int @@ -67,7 +67,7 @@ func (self *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { return buf.Bytes() } -func (self *RocketMqSerializer) DecodeRemoteCommand(headerArray, body []byte) (cmd *RemotingCommand) { +func (r *RocketMqSerializer) DecodeRemoteCommand(headerArray, body []byte) (cmd *RemotingCommand) { cmd = &RemotingCommand{} buf := bytes.NewBuffer(headerArray) // int code(~32767) diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index 130f99bb9..cb7f2e11d 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -51,21 +51,21 @@ func NewSerializerHandler(serializeType rocketmq_api_model.SerializeType) Serial } return serializerHandler } -func (self *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { +func (s *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { length := 4 - headerData := self.serializer.EncodeHeaderData(request) + headerData := s.serializer.EncodeHeaderData(request) length += len(headerData) if request.Body != nil { length += len(request.Body) } buf := bytes.NewBuffer([]byte{}) - binary.Write(buf, binary.BigEndian, int32(length)) // len - binary.Write(buf, binary.BigEndian, int32(len(headerData)|(int(self.serializeType)<<24))) // header len + binary.Write(buf, binary.BigEndian, int32(length)) // len + binary.Write(buf, binary.BigEndian, int32(len(headerData)|(int(s.serializeType)<<24))) // header len buf.Write(headerData) return buf.Bytes() } -func (self *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { +func (s *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { var serializer Serializer switch rocketmq_api_model.SerializeType(headerSerializableType) { case rocketmq_api_model.JSON_SERIALIZE: diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely.go b/rocketmq-go/service/allocate_message/allocate_message_averagely.go index cdfe77541..103bade8c 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_averagely.go +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely.go @@ -23,7 +23,7 @@ import ( type AllocateMessageQueueAveragely struct{} -func (self *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go index cdfd668fb..a93070ffc 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go @@ -23,7 +23,7 @@ import ( type AllocateMessageQueueAveragelyByCircle struct{} -func (self *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") } diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_config.go b/rocketmq-go/service/allocate_message/allocate_message_by_config.go index 2046ffd0e..d55424a54 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_by_config.go +++ b/rocketmq-go/service/allocate_message/allocate_message_by_config.go @@ -22,6 +22,6 @@ type AllocateMessageQueueByConfig struct { messageQueueList []model.MessageQueue } -func (self *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { - return self.messageQueueList, nil +func (a *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { + return a.messageQueueList, nil } diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go index 6fe1cbb9f..ff92eb783 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go +++ b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go @@ -24,7 +24,7 @@ import ( type AllocateMessageQueueByMachineRoom struct { } -func (self *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") } diff --git a/rocketmq-go/service/client_api.go b/rocketmq-go/service/client_api.go deleted file mode 100644 index adf8a60f4..000000000 --- a/rocketmq-go/service/client_api.go +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package service - -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" -) - -var sendSmartMsg bool = true // TODO get from system env - -type TopAddress struct { -} - -type ClientRemotingProcessor interface { -} - -func init() { - // TODO -} - -type MQClientAPI struct { - remotingClient *remoting.RemotingClient - topAddress *TopAddress - crp *ClientRemotingProcessor - nameServerAddress string - config *rocketmq_api_model.MqClientConfig -} - -//func NewMQClientAPI(cfg *rocketmq_api_model.ClientConfig, processor *ClientRemotingProcessor, hook remoting.RPCHook) *MQClientAPI { -// api := &MQClientAPI{ -// remotingClient: &remoting.RemotingClient{}, //TODO -// topAddress: &TopAddress{}, // TODO -// crp: processor, -// config: cfg, -// } -// -// // TODO register -// return api -//} -// -//func (api *MQClientAPI) SendMessage(addr, brokerName string, -// msg message.Message, requestHeader header.SendMessageRequestHeader, timeout int64) *model.SendResult { -// var request *remoting.RemotingCommand -// request = remoting.CreateRemotingCommand(model.SendMsg, &requestHeader) -// request.SetBody(msg.Body) -// return api.sendMessageSync(addr, brokerName, msg, timeout, request) -//} - -func (api *MQClientAPI) sendMessageSync(addr, brokerName string, - msg message.Message, - timeout int64, - request *remoting.RemotingCommand) *model.SendResult { - response := api.invokeSync(addr, request, timeout) - if response == nil { - panic("invokeSync panci!") - } - return nil - // TODO return api.processSendResponse(brokerName, msg, response) -} - -func (api *MQClientAPI) invokeSync(addr string, cmd *remoting.RemotingCommand, timeout int64) *remoting.RemotingCommand { - return nil -} - -func (api *MQClientAPI) processSendResponse(name string, msg message.Message, cmd *remoting.RemotingCommand) *remoting.RemotingCommand { - return nil -} diff --git a/rocketmq-go/service/client_error_code.go b/rocketmq-go/service/client_error_code.go deleted file mode 100644 index d03163862..000000000 --- a/rocketmq-go/service/client_error_code.go +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package service - -const ( - ConnecBrokerError = 10001 - AccessBrokerTimeoutError = 10002 - BrokerNotExistError = 10003 - NoNameServerError = 10004 - NotFoundTopicError = 10005 -) diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index 3a9b57a9b..5808aa044 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -25,8 +25,6 @@ import ( ) type ConsumeMessageService interface { - //ConsumeMessageDirectlyResult consumeMessageDirectly(final MessageExt msg, final String brokerName); - Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) @@ -47,52 +45,52 @@ func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListe return } -func (self *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { - self.consumerGroup = consumerGroup - self.offsetStore = offsetStore - self.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) - self.consumerConfig = consumerConfig +func (c *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { + c.consumerGroup = consumerGroup + c.offsetStore = offsetStore + c.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) + c.consumerConfig = consumerConfig } -func (self *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i - end := i + self.consumerConfig.ConsumeMessageBatchMaxSize + end := i + c.consumerConfig.ConsumeMessageBatchMaxSize if end > msgsLen { end = msgsLen } go func() { glog.V(2).Infof("look slice begin %d end %d msgsLen %d", begin, end, msgsLen) - batchMsgs := transformMessageToConsume(self.consumerGroup, msgs[begin:end]) - consumeState := self.messageListener(batchMsgs) - self.processConsumeResult(consumeState, batchMsgs, messageQueue, processQueue) + batchMsgs := transformMessageToConsume(c.consumerGroup, msgs[begin:end]) + consumeState := c.messageListener(batchMsgs) + c.processConsumeResult(consumeState, batchMsgs, messageQueue, processQueue) }() i = end } return } -func (self *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { - err = self.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) +func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { + err = c.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } -func (self *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { +func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { start := util.CurrentTimeMillisInt64() - consumeResult := self.messageListener([]rocketmq_api_model.MessageExt{*messageExt}) + consumeResult := c.messageListener([]rocketmq_api_model.MessageExt{*messageExt}) consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false consumeMessageDirectlyResult.SpentTimeMills = util.CurrentTimeMillisInt64() - start if consumeResult.ConsumeConcurrentlyStatus == rocketmq_api_model.CONSUME_SUCCESS && consumeResult.AckIndex >= 0 { - consumeMessageDirectlyResult.ConsumeResult = "CR_SUCCESS" + consumeMessageDirectlyResult.ConsumeResult = model.CR_SUCCESS } else { - consumeMessageDirectlyResult.ConsumeResult = "CR_THROW_EXCEPTION" + consumeMessageDirectlyResult.ConsumeResult = model.CR_THROW_EXCEPTION } return } -func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmq_api_model.ConsumeConcurrentlyResult, msgs []rocketmq_api_model.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmq_api_model.ConsumeConcurrentlyResult, msgs []rocketmq_api_model.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return @@ -116,7 +114,7 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result r successMessages = msgs[:ackIndex+1] } for i := ackIndex + 1; i < len(msgs); i++ { - err := self.SendMessageBack(&msgs[i], 0, messageQueue.BrokerName) + err := c.SendMessageBack(&msgs[i], 0, messageQueue.BrokerName) if err != nil { msgs[i].ReconsumeTimes = msgs[i].ReconsumeTimes + 1 failedMessages = append(failedMessages, msgs[i]) @@ -125,11 +123,11 @@ func (self *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result r } } if len(failedMessages) > 0 { - self.SubmitConsumeRequest(failedMessages, processQueue, messageQueue, true) + c.SubmitConsumeRequest(failedMessages, processQueue, messageQueue, true) } commitOffset := processQueue.RemoveMessage(successMessages) if commitOffset > 0 && !processQueue.IsDropped() { - self.offsetStore.UpdateOffset(messageQueue, commitOffset, true) + c.offsetStore.UpdateOffset(messageQueue, commitOffset, true) } } diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 23816791c..418564be3 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -62,11 +62,11 @@ var DEFAULT_TIMEOUT int64 = 6000 type MqClientImpl struct { ClientId string remotingClient *remoting.DefaultRemotingClient - TopicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData - BrokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address - TopicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this - TopicSubscribeInfoTable util.ConcurrentMap //map[string][]*model.MessageQueue //topic | MessageQueue - PullRequestQueue chan *model.PullRequest //todo move + TopicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData + BrokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address + TopicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this + TopicSubscribeInfoTable util.ConcurrentMap //map[string][]*model.MessageQueue //topic | MessageQueue + PullRequestQueue chan *model.PullRequest } func MqClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { @@ -80,22 +80,22 @@ func MqClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequest mqClientImpl.PullRequestQueue = make(chan *model.PullRequest, 1024) return } -func (self *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) { - value, ok := self.TopicSubscribeInfoTable.Get(topic) +func (m *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) { + value, ok := m.TopicSubscribeInfoTable.Get(topic) if ok { messageQueueList = value.([]*model.MessageQueue) } return } -func (self *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { - brokerAddr := self.FetchMasterBrokerAddress(mq.BrokerName) +func (m *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { + brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - self.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = self.FetchMasterBrokerAddress(mq.BrokerName) + m.TryToFindTopicPublishInfo(mq.Topic) + brokerAddr = m.FetchMasterBrokerAddress(mq.BrokerName) } getMaxOffsetRequestHeader := &header.GetMaxOffsetRequestHeader{Topic: mq.Topic, QueueId: mq.QueueId} remotingCmd := remoting.NewRemotingCommand(remoting.GET_MAX_OFFSET, getMaxOffsetRequestHeader) - response, err := self.remotingClient.InvokeSync(brokerAddr, remotingCmd, DEFAULT_TIMEOUT) + response, err := m.remotingClient.InvokeSync(brokerAddr, remotingCmd, DEFAULT_TIMEOUT) if err != nil { return -1 } @@ -103,71 +103,70 @@ func (self *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { queryOffsetResponseHeader.FromMap(response.ExtFields) return queryOffsetResponseHeader.Offset } -func (self *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) int64 { - brokerAddr := self.FetchMasterBrokerAddress(mq.BrokerName) +func (m *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) int64 { + brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - self.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = self.FetchMasterBrokerAddress(mq.BrokerName) + m.TryToFindTopicPublishInfo(mq.Topic) + brokerAddr = m.FetchMasterBrokerAddress(mq.BrokerName) } timeStamp := util.CurrentTimeMillisInt64() searchOffsetRequestHeader := &header.SearchOffsetRequestHeader{Topic: mq.Topic, QueueId: mq.QueueId, Timestamp: timeStamp} remotingCmd := remoting.NewRemotingCommand(remoting.SEARCH_OFFSET_BY_TIMESTAMP, searchOffsetRequestHeader) - response, err := self.remotingClient.InvokeSync(brokerAddr, remotingCmd, DEFAULT_TIMEOUT) + response, err := m.remotingClient.InvokeSync(brokerAddr, remotingCmd, DEFAULT_TIMEOUT) if err != nil { return -1 } queryOffsetResponseHeader := header.QueryOffsetResponseHeader{} queryOffsetResponseHeader.FromMap(response.ExtFields) - glog.Info("op=look search offset result", string(response.Body)) return queryOffsetResponseHeader.Offset } -func (self *MqClientImpl) GetClientId() string { - return self.ClientId +func (m *MqClientImpl) GetClientId() string { + return m.ClientId } -func (self *MqClientImpl) GetPublishTopicList() []string { +func (m *MqClientImpl) GetPublishTopicList() []string { var publishTopicList []string - for _, topic := range self.TopicPublishInfoTable.Keys() { + for _, topic := range m.TopicPublishInfoTable.Keys() { publishTopicList = append(publishTopicList, topic) } return publishTopicList } -func (self *MqClientImpl) GetRemotingClient() *remoting.DefaultRemotingClient { - return self.remotingClient +func (m *MqClientImpl) GetRemotingClient() *remoting.DefaultRemotingClient { + return m.remotingClient } -func (self *MqClientImpl) EnqueuePullMessageRequest(pullRequest *model.PullRequest) { - self.PullRequestQueue <- pullRequest +func (m *MqClientImpl) EnqueuePullMessageRequest(pullRequest *model.PullRequest) { + m.PullRequestQueue <- pullRequest } -func (self *MqClientImpl) DequeuePullMessageRequest() (pullRequest *model.PullRequest) { - pullRequest = <-self.PullRequestQueue +func (m *MqClientImpl) DequeuePullMessageRequest() (pullRequest *model.PullRequest) { + pullRequest = <-m.PullRequestQueue return } -func (self *MqClientImpl) ClearExpireResponse() { - self.remotingClient.ClearExpireResponse() +func (m *MqClientImpl) ClearExpireResponse() { + m.remotingClient.ClearExpireResponse() } -func (self *MqClientImpl) FetchMasterBrokerAddress(brokerName string) (masterAddress string) { - value, ok := self.BrokerAddrTable.Get(brokerName) +func (m *MqClientImpl) FetchMasterBrokerAddress(brokerName string) (masterAddress string) { + value, ok := m.BrokerAddrTable.Get(brokerName) if ok { masterAddress = value.(map[string]string)["0"] } return } -func (self *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) { - value, ok := self.TopicPublishInfoTable.Get(topic) +func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) { + value, ok := m.TopicPublishInfoTable.Get(topic) if ok { topicPublicInfo = value.(*model.TopicPublishInfo) } if topicPublicInfo == nil || !topicPublicInfo.JudgeTopicPublishInfoOk() { - self.TopicPublishInfoTable.Set(topic, &model.TopicPublishInfo{HaveTopicRouterInfo: false}) - err = self.UpdateTopicRouteInfoFromNameServer(topic) + m.TopicPublishInfoTable.Set(topic, &model.TopicPublishInfo{HaveTopicRouterInfo: false}) + err = m.UpdateTopicRouteInfoFromNameServer(topic) if err != nil { glog.Warning(err) // if updateRouteInfo error, maybe we can use the defaultTopic } - value, ok := self.TopicPublishInfoTable.Get(topic) + value, ok := m.TopicPublishInfoTable.Get(topic) if ok { topicPublicInfo = value.(*model.TopicPublishInfo) } @@ -176,9 +175,9 @@ func (self *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicIn return } //try to use the defaultTopic - err = self.UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic) + err = m.UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic) - defaultValue, defaultValueOk := self.TopicPublishInfoTable.Get(topic) + defaultValue, defaultValueOk := m.TopicPublishInfoTable.Get(topic) if defaultValueOk { topicPublicInfo = defaultValue.(*model.TopicPublishInfo) } @@ -186,12 +185,12 @@ func (self *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicIn return } -func (self MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMillis int64) (*model.TopicRouteData, error) { +func (m MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMillis int64) (*model.TopicRouteData, error) { requestHeader := &header.GetRouteInfoRequestHeader{ Topic: topic, } var remotingCommand = remoting.NewRemotingCommand(remoting.GET_ROUTEINTO_BY_TOPIC, requestHeader) - response, err := self.remotingClient.InvokeSync("", remotingCommand, timeoutMillis) + response, err := m.remotingClient.InvokeSync("", remotingCommand, timeoutMillis) if err != nil { return nil, err @@ -213,15 +212,14 @@ func (self MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMi } } -func (self MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { +func (m MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { slave = false found = false - value, ok := self.BrokerAddrTable.Get(brokerName) + value, ok := m.BrokerAddrTable.Get(brokerName) if !ok { return } brokerMap := value.(map[string]string) - //self.brokerAddrTableLock.RUnlock() brokerAddr, ok = brokerMap[util.IntToString(brokerId)] slave = (brokerId != 0) found = ok @@ -237,26 +235,24 @@ func (self MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerI return } -func (self MqClientImpl) UpdateTopicRouteInfoFromNameServer(topic string) (err error) { +func (m MqClientImpl) UpdateTopicRouteInfoFromNameServer(topic string) (err error) { var ( topicRouteData *model.TopicRouteData ) //namesvr lock - //topicRouteData = this.MqClientImplAPIImpl.getTopicRouteInfoFromNameServer(topic, 1000 * 3); - topicRouteData, err = self.GetTopicRouteInfoFromNameServer(topic, 1000*3) + topicRouteData, err = m.GetTopicRouteInfoFromNameServer(topic, 1000*3) if err != nil { return } - self.updateTopicRouteInfoLocal(topic, topicRouteData) + m.updateTopicRouteInfoLocal(topic, topicRouteData) return } -func (self MqClientImpl) UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) { +func (m MqClientImpl) UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) { var ( topicRouteData *model.TopicRouteData ) //namesvr lock - //topicRouteData = this.MqClientImplAPIImpl.getTopicRouteInfoFromNameServer(topic, 1000 * 3); - topicRouteData, err = self.GetTopicRouteInfoFromNameServer(constant.DEFAULT_TOPIC, 1000*3) + topicRouteData, err = m.GetTopicRouteInfoFromNameServer(constant.DEFAULT_TOPIC, 1000*3) if err != nil { return } @@ -269,10 +265,10 @@ func (self MqClientImpl) UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic queueData.ReadQueueNums = defaultQueueData queueData.WriteQueueNums = defaultQueueData } - self.updateTopicRouteInfoLocal(topic, topicRouteData) + m.updateTopicRouteInfoLocal(topic, topicRouteData) return } -func (self MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData *model.TopicRouteData) (err error) { +func (m MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData *model.TopicRouteData) (err error) { if topicRouteData == nil { return } @@ -283,21 +279,21 @@ func (self MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData } //update brokerAddrTable for _, brokerData := range topicRouteData.BrokerDatas { - self.BrokerAddrTable.Set(brokerData.BrokerName, brokerData.BrokerAddrs) + m.BrokerAddrTable.Set(brokerData.BrokerName, brokerData.BrokerAddrs) } //update pubInfo for each topicPublishInfo := model.BuildTopicPublishInfoFromTopicRoteData(topic, topicRouteData) - self.TopicPublishInfoTable.Set(topic, topicPublishInfo) // todo + m.TopicPublishInfoTable.Set(topic, topicPublishInfo) mqList := model.BuildTopicSubscribeInfoFromRoteData(topic, topicRouteData) - self.TopicSubscribeInfoTable.Set(topic, mqList) - self.TopicRouteTable.Set(topic, topicRouteData) + m.TopicSubscribeInfoTable.Set(topic, mqList) + m.TopicRouteTable.Set(topic, topicRouteData) return } -func (self MqClientImpl) FindBrokerAddrByTopic(topic string) (addr string, ok bool) { - value, findValue := self.TopicRouteTable.Get(topic) +func (m MqClientImpl) FindBrokerAddrByTopic(topic string) (addr string, ok bool) { + value, findValue := m.TopicRouteTable.Get(topic) if !findValue { return "", false } @@ -324,8 +320,8 @@ func buildMqClientImplId() (clientId string) { return } -func (self MqClientImpl) sendHeartBeat(addr string, remotingCommand *remoting.RemotingCommand, timeoutMillis int64) error { - remotingCommand, err := self.remotingClient.InvokeSync(addr, remotingCommand, timeoutMillis) +func (m MqClientImpl) sendHeartBeat(addr string, remotingCommand *remoting.RemotingCommand, timeoutMillis int64) error { + remotingCommand, err := m.remotingClient.InvokeSync(addr, remotingCommand, timeoutMillis) if err != nil { glog.Error(err) } else { @@ -336,10 +332,8 @@ func (self MqClientImpl) sendHeartBeat(addr string, remotingCommand *remoting.Re return err } -func (self MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) { - //self.brokerAddrTableLock.RLock() - - for _, brokerTable := range self.BrokerAddrTable.Items() { +func (m MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) { + for _, brokerTable := range m.BrokerAddrTable.Items() { for brokerId, addr := range brokerTable.(map[string]string) { if len(addr) == 0 || brokerId != "0" { continue @@ -352,7 +346,7 @@ func (self MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.Heartbeat glog.V(2).Info("send heartbeat to broker look data[", string(data)+"]") remotingCommand := remoting.NewRemotingCommandWithBody(remoting.HEART_BEAT, nil, data) glog.V(2).Info("send heartbeat to broker[", addr+"]") - self.sendHeartBeat(addr, remotingCommand, 3000) + m.sendHeartBeat(addr, remotingCommand, 3000) } } diff --git a/rocketmq-go/service/offset_store.go b/rocketmq-go/service/offset_store.go index 0bfe64067..c2f64a619 100644 --- a/rocketmq-go/service/offset_store.go +++ b/rocketmq-go/service/offset_store.go @@ -52,47 +52,47 @@ func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStor offsetStore.offsetTable = make(map[model.MessageQueue]int64) return offsetStore } -func (self *RemoteOffsetStore) RemoveOffset(mq *model.MessageQueue) { - defer self.offsetTableLock.Unlock() - self.offsetTableLock.Lock() - delete(self.offsetTable, *mq) +func (r *RemoteOffsetStore) RemoveOffset(mq *model.MessageQueue) { + defer r.offsetTableLock.Unlock() + r.offsetTableLock.Lock() + delete(r.offsetTable, *mq) } -func (self *RemoteOffsetStore) Persist(mq *model.MessageQueue) { - brokerAddr := self.mqClient.FetchMasterBrokerAddress(mq.BrokerName) +func (r *RemoteOffsetStore) Persist(mq *model.MessageQueue) { + brokerAddr := r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - self.mqClient.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = self.mqClient.FetchMasterBrokerAddress(mq.BrokerName) + r.mqClient.TryToFindTopicPublishInfo(mq.Topic) + brokerAddr = r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) } - self.offsetTableLock.RLock() - offset := self.offsetTable[*mq] - self.offsetTableLock.RUnlock() - updateConsumerOffsetRequestHeader := &header.UpdateConsumerOffsetRequestHeader{ConsumerGroup: self.groupName, Topic: mq.Topic, QueueId: mq.QueueId, CommitOffset: offset} + r.offsetTableLock.RLock() + offset := r.offsetTable[*mq] + r.offsetTableLock.RUnlock() + updateConsumerOffsetRequestHeader := &header.UpdateConsumerOffsetRequestHeader{ConsumerGroup: r.groupName, Topic: mq.Topic, QueueId: mq.QueueId, CommitOffset: offset} requestCommand := remoting.NewRemotingCommand(remoting.UPDATE_CONSUMER_OFFSET, updateConsumerOffsetRequestHeader) - self.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) + r.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) } -func (self *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) int64 { +func (r *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) int64 { switch readType { case MEMORY_FIRST_THEN_STORE: case READ_FROM_MEMORY: - self.offsetTableLock.RLock() - offset, ok := self.offsetTable[*mq] - self.offsetTableLock.RUnlock() + r.offsetTableLock.RLock() + offset, ok := r.offsetTable[*mq] + r.offsetTableLock.RUnlock() if ok { return offset } else { return -1 } case READ_FROM_STORE: - offset, err := self.fetchConsumeOffsetFromBroker(mq) + offset, err := r.fetchConsumeOffsetFromBroker(mq) if err != nil { glog.Error(err) return -1 } glog.V(2).Info("READ_FROM_STORE", offset) - self.UpdateOffset(mq, offset, false) + r.UpdateOffset(mq, offset, false) return offset } @@ -100,27 +100,27 @@ func (self *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) } -func (self *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *model.MessageQueue) (int64, error) { - brokerAddr, _, found := self.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) +func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *model.MessageQueue) (int64, error) { + brokerAddr, _, found := r.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) if !found { - brokerAddr, _, found = self.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) + brokerAddr, _, found = r.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) } if found { requestHeader := &header.QueryConsumerOffsetRequestHeader{} requestHeader.Topic = mq.Topic requestHeader.QueueId = mq.QueueId - requestHeader.ConsumerGroup = self.groupName - return self.queryConsumerOffset(brokerAddr, requestHeader, 3000) + requestHeader.ConsumerGroup = r.groupName + return r.queryConsumerOffset(brokerAddr, requestHeader, 3000) } return -1, errors.New("fetch consumer offset error") } -func (self RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *header.QueryConsumerOffsetRequestHeader, timeoutMillis int64) (int64, error) { +func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *header.QueryConsumerOffsetRequestHeader, timeoutMillis int64) (int64, error) { remotingCommand := remoting.NewRemotingCommand(remoting.QUERY_CONSUMER_OFFSET, requestHeader) - reponse, err := self.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) + reponse, err := r.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) if err != nil { glog.Error(err) return -1, err @@ -145,18 +145,18 @@ func (self RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *he return -1, errors.New("query offset error") } -func (self *RemoteOffsetStore) UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { - defer self.offsetTableLock.Unlock() - self.offsetTableLock.Lock() +func (r *RemoteOffsetStore) UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { + defer r.offsetTableLock.Unlock() + r.offsetTableLock.Lock() if mq != nil { if increaseOnly { - offsetOld := self.offsetTable[*mq] + offsetOld := r.offsetTable[*mq] if offsetOld >= offset { return } - self.offsetTable[*mq] = offset + r.offsetTable[*mq] = offset } else { - self.offsetTable[*mq] = offset + r.offsetTable[*mq] = offset } } diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/service/producer_service.go index 836254093..1342b7969 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/service/producer_service.go @@ -48,20 +48,20 @@ func NewDefaultProducerService(producerGroup string, producerConfig *rocketmq_ap defaultProducerService.CheckConfig() return } -func (self *DefaultProducerService) CheckConfig() (err error) { +func (d *DefaultProducerService) CheckConfig() (err error) { // todo check if not pass panic return } -func (self *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) - err = self.checkMessage(message) + err = d.checkMessage(message) if err != nil { return } - topicPublishInfo, err = self.mqClient.TryToFindTopicPublishInfo(message.Topic) + topicPublishInfo, err = d.mqClient.TryToFindTopicPublishInfo(message.Topic) if err != nil { return } @@ -71,14 +71,14 @@ func (self *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model. } glog.V(2).Info("op=look topicPublishInfo", topicPublishInfo) //if(!ok) return error - sendResult, err = self.sendMsgUseTopicPublishInfo(message, communicationMode, sendCallback, topicPublishInfo, timeout) + sendResult, err = d.sendMsgUseTopicPublishInfo(message, communicationMode, sendCallback, topicPublishInfo, timeout) return } -func (self *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) var response *remoting.RemotingCommand - response, err = self.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) + response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) if err != nil { glog.Error(err) return @@ -132,7 +132,7 @@ func processSendResponse(brokerName string, message *rocketmq_api_model.Message, return } -func (self *DefaultProducerService) checkMessage(message *rocketmq_api_model.Message) (err error) { +func (d *DefaultProducerService) checkMessage(message *rocketmq_api_model.Message) (err error) { if message == nil { err = errors.New("message is nil") return @@ -150,7 +150,6 @@ func (self *DefaultProducerService) checkMessage(message *rocketmq_api_model.Mes err = errors.New("the specified topic is longer than topic max length 255.") return } - //todo todo public static final String VALID_PATTERN_STR = ""; if !util.MatchString(message.Topic, `^[%|a-zA-Z0-9_-]+$`) { err = errors.New("the specified topic[" + message.Topic + "] contains illegal characters") @@ -160,14 +159,14 @@ func (self *DefaultProducerService) checkMessage(message *rocketmq_api_model.Mes err = errors.New("messageBody is empty") return } - if len(message.Body) > self.producerConfig.MaxMessageSize { - err = errors.New("messageBody is large than " + util.IntToString(self.producerConfig.MaxMessageSize)) + if len(message.Body) > d.producerConfig.MaxMessageSize { + err = errors.New("messageBody is large than " + util.IntToString(d.producerConfig.MaxMessageSize)) return } return } -func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int messageQueue model.MessageQueue @@ -182,7 +181,7 @@ func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq if err != nil { return } - sendResult, err = self.doSendMessage(message, messageQueue, communicationMode, sendCallback, topicPublishInfo, timeout) + sendResult, err = d.doSendMessage(message, messageQueue, communicationMode, sendCallback, topicPublishInfo, timeout) if err != nil { // todo retry return @@ -191,7 +190,7 @@ func (self *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq return } -func (self *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Message, messageQueue model.MessageQueue, +func (d *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Message, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -200,19 +199,19 @@ func (self *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Me sysFlag int compressMessageFlag int ) - compressMessageFlag, err = self.tryToCompressMessage(message) + compressMessageFlag, err = d.tryToCompressMessage(message) if err != nil { return } sysFlag = sysFlag | compressMessageFlag - brokerAddr = self.mqClient.FetchMasterBrokerAddress(messageQueue.BrokerName) + brokerAddr = d.mqClient.FetchMasterBrokerAddress(messageQueue.BrokerName) if len(brokerAddr) == 0 { err = errors.New("The broker[" + messageQueue.BrokerName + "] not exist") return } message.GeneratorMsgUniqueKey() sendMessageHeader := &header.SendMessageRequestHeader{ - ProducerGroup: self.producerGroup, + ProducerGroup: d.producerGroup, Topic: message.Topic, DefaultTopic: constant.DEFAULT_TOPIC, DefaultTopicQueueNums: 4, @@ -226,16 +225,16 @@ func (self *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Me ReconsumeTimes: message.GetReconsumeTimes(), MaxReconsumeTimes: message.GetMaxReconsumeTimes(), } - sendResult, err = self.producerSendMessageRequest(brokerAddr, sendMessageHeader, message, timeout) + sendResult, err = d.producerSendMessageRequest(brokerAddr, sendMessageHeader, message, timeout) return } -func (self *DefaultProducerService) tryToCompressMessage(message *rocketmq_api_model.Message) (compressedFlag int, err error) { - if len(message.Body) < self.producerConfig.CompressMsgBodyOverHowMuch { +func (d *DefaultProducerService) tryToCompressMessage(message *rocketmq_api_model.Message) (compressedFlag int, err error) { + if len(message.Body) < d.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return } compressedFlag = int(constant.CompressedFlag) - message.Body, err = util.CompressWithLevel(message.Body, self.producerConfig.ZipCompressLevel) + message.Body, err = util.CompressWithLevel(message.Body, d.producerConfig.ZipCompressLevel) return } diff --git a/rocketmq-go/service/producer_service_for_send_back.go b/rocketmq-go/service/producer_service_for_send_back.go index f1543b912..910d6bfee 100644 --- a/rocketmq-go/service/producer_service_for_send_back.go +++ b/rocketmq-go/service/producer_service_for_send_back.go @@ -39,69 +39,67 @@ type SendMessageBackProducerServiceImpl struct { } // send to original broker,if fail send a new retry message -func (self *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) - err = self.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) // todo use + err = s.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) if err == nil { return } glog.Error(err) - err = self.sendRetryMessageBack(messageExt) + err = s.sendRetryMessageBack(messageExt) return } -func (self *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmq_api_model.MessageExt) error { - // todo build a retry topic todo check todo check +func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmq_api_model.MessageExt) error { retryMessage := &rocketmq_api_model.Message{} originMessageId := messageExt.GetOriginMessageId() retryMessage.Properties = messageExt.Properties retryMessage.SetOriginMessageId(originMessageId) retryMessage.Flag = messageExt.Flag - retryMessage.Topic = constant.RETRY_GROUP_TOPIC_PREFIX + self.consumerGroup + retryMessage.Topic = constant.RETRY_GROUP_TOPIC_PREFIX + s.consumerGroup retryMessage.Body = messageExt.Body retryMessage.SetRetryTopic(messageExt.Topic) retryMessage.SetReconsumeTime(messageExt.GetReconsumeTimes() + 1) - retryMessage.SetMaxReconsumeTimes(self.consumerConfig.MaxReconsumeTimes) + retryMessage.SetMaxReconsumeTimes(s.consumerConfig.MaxReconsumeTimes) retryMessage.SetDelayTimeLevel(3 + messageExt.GetReconsumeTimes()) pp, _ := json.Marshal(retryMessage) glog.Info("look retryMessage ", string(pp), string(messageExt.Body)) - sendResult, err := self.defaultProducerService.SendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", self.defaultProducerService.producerConfig.SendMsgTimeout) + sendResult, err := s.defaultProducerService.SendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", s.defaultProducerService.producerConfig.SendMsgTimeout) if err != nil { glog.Error(err) return err } xx, _ := json.Marshal(sendResult) - glog.Info("look retryMessage result", string(xx)) - // todo need check send result + glog.V(2).Info("look retryMessage result", string(xx)) return nil } -func (self *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { - self.mqClient = mqClient - self.consumerGroup = consumerGroup - self.defaultProducerService = defaultProducerService - self.consumerConfig = consumerConfig +func (s *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { + s.mqClient = mqClient + s.consumerGroup = consumerGroup + s.defaultProducerService = defaultProducerService + s.consumerConfig = consumerConfig } -func (self *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmq_api_model.MessageExt, delayLayLevel int) (err error) { +func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmq_api_model.MessageExt, delayLayLevel int) (err error) { if len(brokerName) == 0 { err = errors.New("broker can't be empty") glog.Error(err) return } - brokerAddr := self.mqClient.FetchMasterBrokerAddress(brokerName) + brokerAddr := s.mqClient.FetchMasterBrokerAddress(brokerName) sendMsgBackHeader := &header.ConsumerSendMsgBackRequestHeader{ Offset: messageExt.CommitLogOffset, - Group: self.consumerGroup, + Group: s.consumerGroup, DelayLevel: 0, //Message consume retry strategy
-1,no retry,put into DLQ directly
0,broker control retry frequency
>0,client control retry frequency OriginMsgId: messageExt.MsgId, OriginTopic: messageExt.Topic, UnitMode: false, - MaxReconsumeTimes: int32(self.consumerConfig.MaxReconsumeTimes), + MaxReconsumeTimes: int32(s.consumerConfig.MaxReconsumeTimes), } remotingCommand := remoting.NewRemotingCommand(remoting.CONSUMER_SEND_MSG_BACK, sendMsgBackHeader) - response, invokeErr := self.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, 5000) + response, invokeErr := s.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, 5000) if invokeErr != nil { err = invokeErr return diff --git a/rocketmq-go/service/rebalance.go b/rocketmq-go/service/rebalance.go index 8c0cbb4fd..c2d186a80 100644 --- a/rocketmq-go/service/rebalance.go +++ b/rocketmq-go/service/rebalance.go @@ -47,27 +47,27 @@ type Rebalance struct { consumerConfig *rocketmq_api_model.RocketMqConsumerConfig } -func (self *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { - defer self.processQueueTableLock.RUnlock() - self.processQueueTableLock.RLock() +func (r *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { + defer r.processQueueTableLock.RUnlock() + r.processQueueTableLock.RLock() mqTable := map[model.MessageQueue]model.ProcessQueueInfo{} - for messageQueue, processQueue := range self.processQueueTable { + for messageQueue, processQueue := range r.processQueueTable { mqTable[messageQueue] = processQueue.ChangeToProcessQueueInfo() } return mqTable } -func (self *Rebalance) GetProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { - defer self.processQueueTableLock.RUnlock() - self.processQueueTableLock.RLock() - return self.processQueueTable[messageQueue] +func (r *Rebalance) GetProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { + defer r.processQueueTableLock.RUnlock() + r.processQueueTableLock.RLock() + return r.processQueueTable[messageQueue] } -func (self *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int64) { - defer self.processQueueTableLock.Unlock() - self.processQueueTableLock.Lock() +func (r *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int64) { + defer r.processQueueTableLock.Unlock() + r.processQueueTableLock.Lock() for mq, _ := range offsetTable { - processQueue, ok := self.processQueueTable[mq] + processQueue, ok := r.processQueueTable[mq] if !ok { continue } @@ -76,10 +76,10 @@ func (self *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int6 } -func (self *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { - defer self.processQueueTableLock.RUnlock() - self.processQueueTableLock.RLock() - for messageQueue, processQueue := range self.processQueueTable { +func (r *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { + defer r.processQueueTableLock.RUnlock() + r.processQueueTableLock.RLock() + for messageQueue, processQueue := range r.processQueueTable { processQueueList = append(processQueueList, processQueue) messageQueueList = append(messageQueueList, messageQueue) } @@ -87,15 +87,15 @@ func (self *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQu } //removeUnnecessaryMessageQueue you should drop it first -func (self *Rebalance) RemoveProcessQueue(messageQueue *model.MessageQueue) { - self.offsetStore.Persist(messageQueue) - self.offsetStore.RemoveOffset(messageQueue) - self.removeMessageQueueFromMap(*messageQueue) +func (r *Rebalance) RemoveProcessQueue(messageQueue *model.MessageQueue) { + r.offsetStore.Persist(messageQueue) + r.offsetStore.RemoveOffset(messageQueue) + r.removeMessageQueueFromMap(*messageQueue) } -func (self *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { - defer self.processQueueTableLock.Unlock() - self.processQueueTableLock.Lock() - delete(self.processQueueTable, messageQueue) +func (r *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { + defer r.processQueueTableLock.Unlock() + r.processQueueTableLock.Lock() + delete(r.processQueueTable, messageQueue) } @@ -128,11 +128,11 @@ func NewRebalance(groupName string, subscription map[string]string, mqClient Roc } } -func (self *Rebalance) DoRebalance() { - self.mutex.Lock() - defer self.mutex.Unlock() - for topic, _ := range self.SubscriptionInner { - self.rebalanceByTopic(topic) +func (r *Rebalance) DoRebalance() { + r.mutex.Lock() + defer r.mutex.Unlock() + for topic, _ := range r.SubscriptionInner { + r.rebalanceByTopic(topic) } } @@ -151,16 +151,16 @@ func (self ConsumerIdSorter) Less(i, j int) bool { return false } -func (self *Rebalance) rebalanceByTopic(topic string) error { +func (r *Rebalance) rebalanceByTopic(topic string) error { var cidAll []string - cidAll, err := self.findConsumerIdList(topic, self.groupName) + cidAll, err := r.findConsumerIdList(topic, r.groupName) if err != nil { glog.Error(err) return err } - self.topicSubscribeInfoTableLock.RLock() - mqs := self.mqClient.GetTopicSubscribeInfo(topic) - self.topicSubscribeInfoTableLock.RUnlock() + r.topicSubscribeInfoTableLock.RLock() + mqs := r.mqClient.GetTopicSubscribeInfo(topic) + r.topicSubscribeInfoTableLock.RUnlock() if len(mqs) > 0 && len(cidAll) > 0 { var messageQueues model.MessageQueues = mqs var consumerIdSorter ConsumerIdSorter = cidAll @@ -168,7 +168,7 @@ func (self *Rebalance) rebalanceByTopic(topic string) error { sort.Sort(messageQueues) sort.Sort(consumerIdSorter) } - allocateResult, err := self.allocateMessageQueueStrategy.Allocate(self.groupName, self.mqClient.GetClientId(), mqs, cidAll) + allocateResult, err := r.allocateMessageQueueStrategy.Allocate(r.groupName, r.mqClient.GetClientId(), mqs, cidAll) if err != nil { glog.Error(err) @@ -176,20 +176,20 @@ func (self *Rebalance) rebalanceByTopic(topic string) error { } glog.V(2).Infof("rebalance topic[%s]", topic) - self.updateProcessQueueTableInRebalance(topic, allocateResult) + r.updateProcessQueueTableInRebalance(topic, allocateResult) return nil } -func (self *Rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { - defer self.processQueueTableLock.RUnlock() - self.processQueueTableLock.RLock() - self.removeTheQueueDontBelongHere(topic, mqSet) - self.putTheQueueToProcessQueueTable(topic, mqSet) +func (r *Rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { + defer r.processQueueTableLock.RUnlock() + r.processQueueTableLock.RLock() + r.removeTheQueueDontBelongHere(topic, mqSet) + r.putTheQueueToProcessQueueTable(topic, mqSet) } -func (self *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { +func (r *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { // there is n^2 todo improve - for key, value := range self.processQueueTable { + for key, value := range r.processQueueTable { if topic != key.Topic { continue } @@ -197,36 +197,35 @@ func (self *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model. for _, messageQueueItem := range mqSet { if key == messageQueueItem { needDelete = false - // todo if expire break } } if needDelete { value.SetDrop(true) - delete(self.processQueueTable, key) + delete(r.processQueueTable, key) } } } -func (self *Rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { +func (r *Rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { for index, mq := range mqSet { - _, ok := self.processQueueTable[mq] + _, ok := r.processQueueTable[mq] if !ok { pullRequest := new(model.PullRequest) - pullRequest.ConsumerGroup = self.groupName + pullRequest.ConsumerGroup = r.groupName pullRequest.MessageQueue = &mqSet[index] - pullRequest.NextOffset = self.computePullFromWhere(&mq) // todo use remote offset + pullRequest.NextOffset = r.computePullFromWhere(&mq) pullRequest.ProcessQueue = model.NewProcessQueue() - self.processQueueTable[mq] = pullRequest.ProcessQueue - self.mqClient.EnqueuePullMessageRequest(pullRequest) + r.processQueueTable[mq] = pullRequest.ProcessQueue + r.mqClient.EnqueuePullMessageRequest(pullRequest) } } } -func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { +func (r *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { var result int64 = -1 - lastOffset := self.offsetStore.ReadOffset(mq, READ_FROM_STORE) - switch self.consumerConfig.ConsumeFromWhere { + lastOffset := r.offsetStore.ReadOffset(mq, READ_FROM_STORE) + switch r.consumerConfig.ConsumeFromWhere { case rocketmq_api_model.CONSUME_FROM_LAST_OFFSET: if lastOffset >= 0 { result = lastOffset @@ -234,7 +233,7 @@ func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { result = 0 } else { - result = self.mqClient.GetMaxOffset(mq) + result = r.mqClient.GetMaxOffset(mq) } } break @@ -252,7 +251,7 @@ func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { result = 0 } else { - result = self.mqClient.SearchOffset(mq, self.consumerConfig.ConsumeTimestamp) + result = r.mqClient.SearchOffset(mq, r.consumerConfig.ConsumeTimestamp) } } break @@ -263,31 +262,31 @@ func (self *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { return result } -func (self *Rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { - brokerAddr, ok := self.mqClient.FindBrokerAddrByTopic(topic) +func (r *Rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { + brokerAddr, ok := r.mqClient.FindBrokerAddrByTopic(topic) if !ok { - err := self.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + err := r.mqClient.UpdateTopicRouteInfoFromNameServer(topic) if err != nil { glog.Error(err) } - brokerAddr, ok = self.mqClient.FindBrokerAddrByTopic(topic) + brokerAddr, ok = r.mqClient.FindBrokerAddrByTopic(topic) } if ok { - return self.getConsumerIdListByGroup(brokerAddr, groupName, 3000) + return r.getConsumerIdListByGroup(brokerAddr, groupName, 3000) } return nil, errors.New("can't find broker") } -func (self *Rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, timeoutMillis int64) ([]string, error) { +func (r *Rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, timeoutMillis int64) ([]string, error) { requestHeader := new(header.GetConsumerListByGroupRequestHeader) requestHeader.ConsumerGroup = consumerGroup request := remoting.NewRemotingCommand(remoting.GET_CONSUMER_LIST_BY_GROUP, requestHeader) - response, err := self.mqClient.GetRemotingClient().InvokeSync(addr, request, timeoutMillis) + response, err := r.mqClient.GetRemotingClient().InvokeSync(addr, request, timeoutMillis) if err != nil { glog.Error(err) return nil, err diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go new file mode 100644 index 000000000..cf8c9b8cf --- /dev/null +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -0,0 +1,53 @@ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/golang/glog" +) + +func main() { + chResult := make(chan bool, 1) + var ( + nameServerAddress = "127.0.0.1:9876" + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestCompressProducerGroup" + testConsumerGroup = "TestCompressConsumerGroup" + ) + var bigMessageBody = "test_string" + for i := 0; i < 16; i++ { + bigMessageBody += bigMessageBody + } + //bigMessageBody len will be 720896,it will be compressed + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + producerConfig := rocketmq_api_model.NewProducerConfig() + producerConfig.CompressMsgBodyOverHowMuch = 500 + var producer = rocketmq_api.NewDefaultMQProducerWithCustomConfig(testProducerGroup, producerConfig) + rocketMQClientInstance.RegisterProducer(producer) + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "compress_message_test") + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + if msg.SysFlag&constant.CompressedFlag != constant.CompressedFlag { + panic("message not be compressed") + } + if string(msg.Body) != bigMessageBody { + panic("message not be unCompressed") + } + glog.Info("Test compress and tag success") + successIndex = index + + } + chResult <- true + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + rocketMQClientInstance.Start() + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(bigMessageBody)} + message.SetTag("compress_message_test") + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + <-chResult +} diff --git a/rocketmq-go/util/compress_util.go b/rocketmq-go/util/compress_util.go index 061791167..8925afa90 100644 --- a/rocketmq-go/util/compress_util.go +++ b/rocketmq-go/util/compress_util.go @@ -28,14 +28,11 @@ func UnCompress(body []byte) (unCompressBody []byte, err error) { z, err := zlib.NewReader(b) if err != nil { glog.Error(err) - return + return nil, err } - defer z.Close() unCompressBody, err = ioutil.ReadAll(z) - if err != nil { - glog.Error(err) - } - return + z.Close() + return unCompressBody, nil } func Compress(body []byte) (compressBody []byte, err error) { var in bytes.Buffer diff --git a/rocketmq-go/util/regex_util.go b/rocketmq-go/util/regex_util.go index 53574524b..966a359a4 100644 --- a/rocketmq-go/util/regex_util.go +++ b/rocketmq-go/util/regex_util.go @@ -23,7 +23,6 @@ import ( //var regexpMap map[string]*regexp.Regexp //var rwMutex sync.RWMutex -// todo improve func MatchString(value, pattern string) bool { re, err := regexp.Compile(pattern) if err != nil { diff --git a/rocketmq-go/util/time_util_test.go b/rocketmq-go/util/time_util_test.go new file mode 100644 index 000000000..ce2a65edb --- /dev/null +++ b/rocketmq-go/util/time_util_test.go @@ -0,0 +1,12 @@ +package util_test + +import ( + "testing" +) + +func TestCurrentTimeMillisInt64(t *testing.T) { + return +} +func TestCurrentTimeMillisStr(t *testing.T) { + return +} From 5863f41f476020e47115526ea0d4289e2d98141b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 13 Aug 2017 23:15:11 +0800 Subject: [PATCH 25/88] add test case: retry message compress message message tag --- .../test/test_consume_message_retry.go | 61 +++++++++++++++++++ .../test_send_and_receive_compress_message.go | 7 ++- .../test/test_send_and_receive_tag_message.go | 56 +++++++++++++++++ 3 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 rocketmq-go/test/test_consume_message_retry.go create mode 100644 rocketmq-go/test/test_send_and_receive_tag_message.go diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go new file mode 100644 index 000000000..c309fbed0 --- /dev/null +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -0,0 +1,61 @@ +package main + +import ( + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" + "time" +) + +//test consume message, first and second time consume error,third time consume success +func main() { + chResult := make(chan bool, 3) + var ( + nameServerAddress = "127.0.0.1:9876" + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestRetryProducerGroup" + testConsumerGroup = "TestRetryConsumerGroup" + tag = "RetryTestTag" + testMessageBody = "RetryTestMessageBody" + consumeTime = 0 + ) + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, tag) + fmt.Println(tag) + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, message := range messageList { + if string(message.Body) != testMessageBody { + panic("message.Body is wrong message.Body=" + string(message.Body) + " testMessageBody=" + testMessageBody + " tag=" + message.GetTag()) + } + if consumeTime < 2 { + consumeTime++ + chResult <- true + glog.Info("test consume fail") + break + } + glog.Info("test consume success") + chResult <- true + successIndex = index + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + rocketMQClientInstance.Start() + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(testMessageBody)} + message.SetTag(tag) + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + for i := 0; i < 3; i++ { + select { + case <-chResult: + case <-time.After(time.Second * 50): + panic("receive tag message timeout") + } + } + glog.Info("Test tag message success") +} diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index cf8c9b8cf..49c774b56 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -5,6 +5,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/golang/glog" + "time" ) func main() { @@ -49,5 +50,9 @@ func main() { message.SetTag("compress_message_test") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) - <-chResult + select { + case <-chResult: + case <-time.After(time.Second * 30): + panic("receive compressed message timeout") + } } diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go new file mode 100644 index 000000000..542b276b7 --- /dev/null +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -0,0 +1,56 @@ +package main + +import ( + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" + "time" +) + +func main() { + chResult := make(chan bool, 3) + var ( + nameServerAddress = "127.0.0.1:9876" + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestTagProducerGroup" + testConsumerGroup = "TestTagConsumerGroup" + ) + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "tag0 || tag2||tag4") + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + if msg.GetTag() != "tag0" && msg.GetTag() != "tag2" && msg.GetTag() != "tag4" { + panic("receive message not belong here tag=" + msg.GetTag()) + } + fmt.Println("got " + msg.GetTag()) + chResult <- true + successIndex = index + + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + rocketMQClientInstance.Start() + for i := 0; i < 5; i++ { + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello world")} + message.SetTag("tag" + util.IntToString(i)) + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + } + for i := 0; i < 3; i++ { + select { + case <-chResult: + + case <-time.After(time.Second * 30): + panic("receive tag message timeout") + } + } + glog.Info("Test tag message success") + +} From 57cef1e045616d044aaa227dc1083bf3aaa2aee0 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 14 Aug 2017 23:51:58 +0800 Subject: [PATCH 26/88] test_delay_message_send_and_receive --- .../test_delay_message_send_and_receive.go | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 rocketmq-go/test/test_delay_message_send_and_receive.go diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go new file mode 100644 index 000000000..6c932f34c --- /dev/null +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -0,0 +1,60 @@ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" + "time" +) + +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestDelayProducerGroup" + testConsumerGroup = "TestDelayConsumerGroup" + tag = "TestDelayMessageTag" + ) + var messageId string + var startTime time.Time + chResult := make(chan bool, 1) + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, tag) + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + endTime := time.Now() + if msg.MsgId != messageId { + panic("messageId is wrong " + msg.MsgId) + } + costSeconds := endTime.Unix() - startTime.Unix() + if costSeconds < 14 || costSeconds > 16 { + panic("delay time message is error ") + } + chResult <- true + successIndex = index + + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + rocketMQClientInstance.Start() + <-time.After(time.Second * 30) // wait + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello world")} + message.SetTag(tag) + message.SetDelayTimeLevel(3) // cost 15 second + result, err := producer.Send(message) + startTime = time.Now() + messageId = result.MsgID() + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err); + select { + case <-chResult: + case <-time.After(time.Second * 30): + panic("receive tag message timeout") + } + glog.Info("Test tag message success") + +} From 5cdad85666ae8482f4f37e8f0c5a16d3dbe14d9c Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 15 Aug 2017 00:01:54 +0800 Subject: [PATCH 27/88] test_rocketmq_serialize_message_send_and_receive --- ...etmq_serialize_message_send_and_receive.go | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go new file mode 100644 index 000000000..a2e0a44b1 --- /dev/null +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -0,0 +1,56 @@ +package main + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" + "time" +) + +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestSerializeProducerGroup" + testConsumerGroup = "TestSerializeConsumerGroup" + tag = "TestSerializeMessageTag" + messageBody = "testMessageBody_testMessageBody" + messageCount = 100 + ) + chResult := make(chan bool, messageCount) + mqClientConfig := rocketmq_api_model.NewMqClientConfig(nameServerAddress) + mqClientConfig.ClientSerializeType = rocketmq_api_model.ROCKETMQ_SERIALIZE + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, tag) + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + if (msg.GetTag() == tag && messageBody == string(messageBody)) { + chResult <- true + } + successIndex = index + + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + rocketMQClientInstance.Start() + for i := 0; i < messageCount; i++ { + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(messageBody)} + message.SetTag(tag) + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err); + } + for i := 0; i < messageCount; i++ { + select { + case <-chResult: + case <-time.After(time.Second * 30): + panic("receive tag message timeout") + } + } + glog.Info("Test tag message success") + +} From 3e827483995344e667eec30f88bb8de540ec512b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 15 Aug 2017 22:52:33 +0800 Subject: [PATCH 28/88] test_rocketmq_serialize_message_send_and_receive --- rocketmq-go/LICENSE | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/rocketmq-go/LICENSE b/rocketmq-go/LICENSE index b67d90910..7f77f44e7 100644 --- a/rocketmq-go/LICENSE +++ b/rocketmq-go/LICENSE @@ -1,4 +1,4 @@ -Apache License + Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ @@ -15,7 +15,7 @@ Apache License "Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the + "control" means (properties) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. @@ -178,7 +178,7 @@ Apache License APPENDIX: How to apply the Apache License to your work. To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "{}" + boilerplate notice, with the fields enclosed by brackets "[]" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a @@ -186,7 +186,7 @@ Apache License same "printed page" as the copyright notice for easier identification within third-party archives. - Copyright {} + Copyright [yyyy] [name of copyright owner] Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -198,4 +198,4 @@ Apache License distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and - limitations under the License. + limitations under the License. \ No newline at end of file From 669c3fe39c7033848a33b40881264d41af88b187 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 15 Aug 2017 22:55:36 +0800 Subject: [PATCH 29/88] LICENSE --- LICENSE | 201 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 201 insertions(+) create mode 100644 LICENSE diff --git a/LICENSE b/LICENSE new file mode 100644 index 000000000..7f77f44e7 --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (properties) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. \ No newline at end of file From a7c5e5ffdb8f17f7b6e9b0bf888ee91a4aead44f Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 15 Aug 2017 23:42:28 +0800 Subject: [PATCH 30/88] add go rocketmq client design --- rocketmq-go/README.md | 138 +++++++++++++++++- rocketmq-go/docs/checklist.md | 2 + .../docs/consume-message-activity.puml | 26 ++++ .../docs/heartbeat-timing-diagram.puml | 7 + rocketmq-go/docs/package.puml | 17 ++- .../docs/pull-message-timing-diagram.puml | 21 +++ .../docs/rebalance-timing-diagram.puml | 16 ++ rocketmq-go/docs/roadmap.md | 105 ++++++------- ...update-topic-routeInfo-timing-diagram.puml | 10 ++ 9 files changed, 272 insertions(+), 70 deletions(-) create mode 100644 rocketmq-go/docs/consume-message-activity.puml create mode 100644 rocketmq-go/docs/heartbeat-timing-diagram.puml create mode 100644 rocketmq-go/docs/pull-message-timing-diagram.puml create mode 100644 rocketmq-go/docs/rebalance-timing-diagram.puml create mode 100644 rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml diff --git a/rocketmq-go/README.md b/rocketmq-go/README.md index 900e9189b..43f7c0b26 100644 --- a/rocketmq-go/README.md +++ b/rocketmq-go/README.md @@ -1,4 +1,134 @@ -# RocketMQ Go SDK -some code refer to below repos: -* https://github.com/didapinchegit/go_rocket_mq -* https://github.com/sevennt/go_rocket_mq/ \ No newline at end of file +# RocketMQ Go SDK Millstone1 Detail Design + +## Example +``` +func main() { + var ( + nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + testTopic = "GoLangRocketMQ" + testProducerGroup = "TestProducerGroup" + testConsumerGroup = "TestConsumerGroup" + ) + // init rocketMQClientInstance + rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + // init rocketMQProducer and register it + var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance.RegisterProducer(producer) + + // 1.init rocketMQConsumer + // 2.subscribe topic and register our function to message listener + // 3.register it + var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "*") + consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) + // call your function + successIndex = index + } + return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + + // start rocketMQ client instance + rocketMQClientInstance.Start() + + //start send test message + for { + var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} + result, err := producer.Send(message) + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + } + +``` + +# Go RocketMQ Client's Arch + +![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/svg/ZLNDRk8m4BxdANokL1w06qNzOTK4Dg7g4RdEGACcDjWE4jjotvNsaVfEVOKTsp7vPIa7bF5yy-MRySo4vLGf8WLx0rtiLXin2dVJF0EkGyhf1kHxe43kCmQ9fXg2Oy1w4OiopqDG2k5JmRKKUMxYJjYAO3J9Sy6GfCB-BA54LeadcyFx8BDJSaUP5X8XnGxvLLc0NDAN7D1UI96MpDlT5uOd_9aiQg0dwW39CI0zJYiAynjmkimUCwM16p83gJ0I2g4plXd5rOFd8OHaV2_U83bmLbiJrJBd79xf0UtJI_k4eYWeJmqZAvKMnGFG52IQ4dObA3qLALXBRR4kuCm1XKuPrcwTRQm-JWj8v7wIfeODuSO_MpXrIbFE84A8KpVC5Zi9M6U6HHBIyXej3B8zU8K4zLS94_qAf03zAjAHmzxVBbVJUPGyXRVnAbbEba_9wYwUXwlfu-msMWw0ugSecaNtgrbqDtVkqSYH7Ur_Hsa2CgDvzWla0-gmJKpSxuSIlFRwqzYsQhZhU8v149YAgIrbu7i3-oMuE3Jawlhw_02C6G8f5Zmu2x44TJT_Fy8FIXrHirX8j_z94-cZMqYJuMGnUvPqkqNNc5mAcA_N2dI2gk0Rw1XUQ6uwxnlOwh2gT-9Ect6u4OQkpDocJnXJ8S9Sp_0Sr-KmGSrKX2kmHLbdfp0z_wd4JGy6N0hsaBeoFUgsgw5omFk_TMtXmrKreGdmj3g-eUpDDr85kH3SLNKUsLctxKlqzqJCDRRlYjZcwApDFig0cgkmgb6VQNZ33S6lR74wMoRej5zZLjijM7sIRUw3--CikZNoWLrGGkEA5LiYrkTQceo_Fd-Rhz07FIZs8UmNwElpps0i6NFGNvz_bSyEdt4aE31pwjUx__Jy0m00) + +# How Go RocketMQ Client Works + + +1. create a new rocketMqMange instance(nameServerAddr ...) + +2. create a new consumer instance(topic/tag/listener ...)(now only support cluster/concurrent) + +3. consumer register to rocketMqMange + +4. rocketMqMange start + +* register ClientRequestProcessor + * CHECK_TRANSACTION_STATE + * NOTIFY_CONSUMER_IDS_CHANGED + * RESET_CONSUMER_CLIENT_OFFSET + * GET_CONSUMER_STATUS_FROM_CLIENT + * GET_CONSUMER_RUNNING_INFO + * CONSUME_MESSAGE_DIRECTLY + +* Start All Task + * updateTopicRouteInfo + * heartbeat + * rebalance + * pullMessage + * cleanExpireMsg (Non-major) + +## All Tasks + + ### updateTopicRouteInfo + + update Topic Route Info by consumer subscription data (topic route info data get from name server) + + put them into local memory(BrokerAddrTable/TopicPublishInfoTable/TopicSubscribeInfoTable/TopicRouteTable) + +![update-topic-routeInfo-timing-diagram](http://www.plantuml.com/plantuml/proxy?src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/go-client-detail-design/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml) + + ### heartbeat: + + prepare heartbeat data(all consumer and producer data in this client) + + send it to all brokers.(broker data is from BrokerAddrTable) + + (only broker know the distribution of the consumers we can rebalance) + +![heartbeat-timing-diagram](http://www.plantuml.com/plantuml/proxy?src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/go-client-detail-design/rocketmq-go/docs/heartbeat-timing-diagram.puml) + + ### rebalance + + for each MqClientManager.ClientFactory's consumers,invoke consumer.rebalance's DoRebalance method + + (after rebalance we can know the (topic/consumer group) should consume from which broker which queue) + + put them into local memory(processQueueTable) + + enqueue pull message request (chan *model.PullRequest) + + ![rebalance-timing-diagram](http://www.plantuml.com/plantuml/svg/XL7DQiCm3BxdANJSO7s170gZi55OsTOMTdPi9J4uaclBbBtz72TBCoZiPblVdpuViL5EaKROR8-_vxhb0AXq3yBUQh04fzH47QmNorGjmCtsSDavYoHrQydic68QCEpDcutoKCXFNU3a7-zoNb7E8sOMRt1FBK-qFuHdvrWhmGF6g3hyJ9Zm926_TD-rceTmjT93le6UOu0kDZ260KMc32yZEM_KyjhXjdho9ejz1DRPh3YTLUDoiWLIAIUmk2eWlCwg3Jgc3gItSGcnTdblsuXo4WvOQnvyoaR9kPV0mrUF0QiLO5LJG6M0o-XkZKYJlSzQC4mTGS3y6AL25n76pyb99nYn_9lqreT1XtdDWlIhLYeaymC0) + + ### pullMessage + + dequeue pull message request and pull message from broker,when get messages to consume, + put them into consume request,consume request handler will call the listener consume the message + + enqueue a new pull message request and commit our consume offset to broker + + #### pullMessage + + ![pull-message-timing-diagram](http://www.plantuml.com/plantuml/svg/dPHHZzem3CVV-odynYO1Uwyc9AvNRfCet5hG7b7Qkb2HDbpY7BTlFoSK1bgPnDxYsFdPVyT9YhcGeYqGHfFql0v5HQX1Ntn7X2qI24Z4uMk2neWj_h1eSVYgLS6sDoP1UaM3LojbYcyM3KMg9QsaP6W8aDkwPDRXZnz4Mx9DGEfwsrE3Viu_4XntjKIGIXs0n1w1TWWrOGEg-elkdAtVxMJrfnjDdhJQemvB1KOrIBkwtOB85TTSINM4uXJwfJbHeDJgC1wFeQv0xOV2_6eBdmNE0PLM3UGU6fpOBAatTrW8FfUBOZ_cx4wC1saqLb8W9C5ikLuytokSryOssCdBKB_NVCF6varDdQyxTO_GNnL-O6495_X1Lm51RxhHP5uRmXPrmgrJPVYf2uizH6cMHyNETT7jUf5TepxpjCZoxEcmhWpE2xupj-ZWrhodNoDPtLuI6X9aJJ1mtOoMYsoTn9ji7KLnbWM3EvBwmS40fK58upDcFbt5AU-svRtUD6-HhB6bq72Grru9dk3oCYlkxeSyIOPgrkkSG_TOwfQVIsEsJ-oU-HF1GwNspG1KV1ctpCUW6XlrVhf1OmltDrnak4VklX7dOpm_nyeW3UsX58IT5VZkBPQRHVnpasGl3ytavN0oNKNVukV_12ndionURRxFv_7BTFuWe2r_0m00) + + #### consumeMessage(submitConsumeRequest) + + ![consume-message-activity](http://www.plantuml.com/plantuml/svg/VL5FQy8m5B_tKxoR4PdWjKvaAnu4Ey7Eqv0HsnSDQvCkBztcjzzBxT3si1m2UVdzJSXBvwEuGY9vmeqc3mlmJXfIrbNfTVng4skegN0UnGu38htXqQ0JT_pnFD8A1EEc7IlpqZS4YmMCakrBjazNxza-ILPPDbgEmP_HWBWWZIE0MEOVQrFW3ti4XQVkE8-m90HXx13rCECxKsWLnTJaEUVeih4XL7IYjnjw0hC3Lm3DLt_32VE_pxfaSGsFBMDQeZdvyxAr8XP_Pl0EgIb3zJ3eBC9Sj1xwmBK1j19z_B3VTGsJJcLTCwqd5VhU24fomk9VVFi6lBTbeZYtLQNzLYbgFynXPRymw_cAGGpMuMH7fdKVjyFF1icBldk0DNKXFLxLrsPZSrccxE0kujwNaUHj_Gi0) + + ### cleanExpireMsg (Non-major) + +when message cost too many time,we will drop this message(send message back) (for example 30 mins) + + + +# Go RocketMQ Client's Roadmap + [Go RocketMQ Client's Roadmap](https://github.com/StyleTang/incubator-rocketmq-externals/blob/master/rocketmq-go/docs/roadmap.md) + +# Go RocketMQ Client's Check List + +## todo \ No newline at end of file diff --git a/rocketmq-go/docs/checklist.md b/rocketmq-go/docs/checklist.md index 0412f864c..a5ae9a3c4 100644 --- a/rocketmq-go/docs/checklist.md +++ b/rocketmq-go/docs/checklist.md @@ -1 +1,3 @@ # Test Check List + +## todo \ No newline at end of file diff --git a/rocketmq-go/docs/consume-message-activity.puml b/rocketmq-go/docs/consume-message-activity.puml new file mode 100644 index 000000000..12756e520 --- /dev/null +++ b/rocketmq-go/docs/consume-message-activity.puml @@ -0,0 +1,26 @@ +@startuml + +start +:messageList = transformMessageToConsume; +:ret = MessageListener(messageList); +if (ret.ConsumeStatus == 'CONSUME_SUCCESS'?) then (yes) + :ackIndex = ret.AckIndex; +else (no) + :ackIndex = -1; +endif +:successMessages = msgs[:ackIndex + 1]; +: i = ackIndex+1; +while (i < msgs.length?) + :sendBackResult = sendMessageBack(msgs[i]); + if(sendBackResult?) then (yes) + :successMessages.append(msgs[i]); + else + :failedMessages.append(msgs[i]); + endif +endwhile +:commitOffset = removeSuccessMessage(successMessages); +:submitConsumeRequest(failedMessages); +:updateOffsetInLocalMemory(commitOffset); +stop +@enduml + diff --git a/rocketmq-go/docs/heartbeat-timing-diagram.puml b/rocketmq-go/docs/heartbeat-timing-diagram.puml new file mode 100644 index 000000000..a8341b55f --- /dev/null +++ b/rocketmq-go/docs/heartbeat-timing-diagram.puml @@ -0,0 +1,7 @@ +@startuml +MqClientManager ->ClientFactory:prepareHeartbeatData(producer/consumer info of this client) +MqClientManager->MqClient:SendHeartbeatToAllBroker + +@enduml + + diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index f34dd38f9..cc688728c 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -6,6 +6,8 @@ serviceState } class rocketmq_go.PullMessageController{ +} +class service.AllocateMessageQueueStrategy{ } class rocketmq_go.ClientFactory{ mqConsumerTable @@ -13,14 +15,14 @@ mqProducerTable } class service.MqConsumer{ serviceState -offsetStore } class service.MqProducer{ serviceState } class service.MqClient{ - + } + class service.MessageListener{ } class remoting.RemotingClient{ invokeSync @@ -56,7 +58,7 @@ rocketmq_go.PullMessageController *-- rocketmq_go.ClientFactory:contains rocketmq_go.ClientFactory *-- MqProducer:contains rocketmq_go.ClientFactory *-- MqConsumer:contains MqProducer *-- service.MqClient :contains -MqConsumer *-- service.MqClient : contains +MqConsumer *-- service.PullAPIWrapper : contains MqConsumer *-- service.OffsetStore : contains MqConsumer *-- service.Rebalance : contains MqConsumer *-- service.ConsumeMessageService : contains @@ -64,12 +66,17 @@ MqConsumer *-- service.ConsumeMessageService : contains } namespace service{ - MqClient o-- remoting.RemotingClient:contains + MqClientAPI o-- remoting.RemotingClient:contains OffsetStore o-- MqClient + PullAPIWrapper o-- MqClient + MqClient o-- MqClientAPI + PullAPIWrapper o-- MqClientAPI Rebalance o-- MqClient Rebalance o-- OffsetStore + Rebalance o-- AllocateMessageQueueStrategy ConsumeMessageService o-- SendMessageBackProducerService ConsumeMessageService o-- OffsetStore + ConsumeMessageService o-- MessageListener } namespace remoting { @@ -89,5 +96,5 @@ namespace rocketmq_go{ note top of remoting.RemotingClient :(sync|aysc|oneWay) note top of remoting :net,serialize,connect,request response -note top of service.MqClient :mq common method + @enduml \ No newline at end of file diff --git a/rocketmq-go/docs/pull-message-timing-diagram.puml b/rocketmq-go/docs/pull-message-timing-diagram.puml new file mode 100644 index 000000000..6ae4ba269 --- /dev/null +++ b/rocketmq-go/docs/pull-message-timing-diagram.puml @@ -0,0 +1,21 @@ +@startuml +Rebalance ->Rebalance:after rebalance we get a Map +Rebalance -> PullRequestController : buid a pullRequest and put to pullRequestQueue +PullRequestController -> PullRequestController : while loop take the item of pullRequestQueue then pullMessage +PullRequestController->Broker:do something about flow control/message drop... if ok,async pull message from broker +Broker ->PullRequestController:pull message call back. +alt FOUND +PullRequestController->PullRequestController:put Message into processQueue +PullRequestController->ConsumeMessageService:submitConsumeRequest +else NO_NEW_MSG +PullRequestController->PullRequestController +else NO_MATCHED_MSG +PullRequestController->PullRequestController +else OFFSET_ILLEGAL +PullRequestController->PullRequestController +end +PullRequestController->PullRequestController:executeNextPullRequest + +@enduml + + diff --git a/rocketmq-go/docs/rebalance-timing-diagram.puml b/rocketmq-go/docs/rebalance-timing-diagram.puml new file mode 100644 index 000000000..15a404188 --- /dev/null +++ b/rocketmq-go/docs/rebalance-timing-diagram.puml @@ -0,0 +1,16 @@ +@startuml +Rebalance ->Rebalance:for each topic in Subscription +Rebalance->MqClient:findConsumerIdList(topic,group) +Rebalance->MqClient:GetTopicSubscribeInfo(topic) +Rebalance->AllocateMessageQueueStrategy:allocate with all consumer client List and all queue info +Rebalance->Rebalance:updateProcessQueueTableInRebalance +alt allocate a new queue belong this consumer +Rebalance->OffsetStore:computePullFromWhere +OffsetStore->MqClient:read offset +Rebalance->MqClient:enqueue pull message request +else find a queue not belong this consumer +Rebalance->Rebalance:delete this processQueue +end +@enduml + + diff --git a/rocketmq-go/docs/roadmap.md b/rocketmq-go/docs/roadmap.md index a69c561f0..0db903326 100644 --- a/rocketmq-go/docs/roadmap.md +++ b/rocketmq-go/docs/roadmap.md @@ -1,64 +1,45 @@ # RoadMap-Milestone1 ## Consumer -- [x] ConsumerType - - [x] PushConsumer -- [x] MessageListener - - [x] Concurrently -- [x] MessageModel - - [x] CLUSTERING -- [x] OffsetStore - - [x] RemoteBrokerOffsetStore -- [x] RebalanceService -- [x] PullMessageService -- [x] ConsumeMessageService -- [x] AllocateMessageQueueStrategy - - [x] AllocateMessageQueueAveragely -- [x] Other - - [x] Config - - [x] ZIP - - [x] ConsumeFromWhere - - [x] CONSUME_FROM_LAST_OFFSET - - [x] CONSUME_FROM_FIRST_OFFSET - - [x] CONSUME_FROM_TIMESTAMP - - [x] Retry(sendMessageBack) - - [x] TimeOut(clearExpiredMessage) - - [x] ACK(partSuccess) - - [x] FlowControl(messageCanNotConsume) - -## Producer -- [x] ProducerType - - [x] DefaultProducer -- [x] API - - [x] Send - - [x] Sync -- [x] Other - - [x] DelayMessage - - [x] Config - - [x] MessageId Generate - - [x] CompressMsg - - [x] TimeOut - - [x] LoadBalance - - [x] DefaultTopic - - [x] VipChannel - - [x] MQFaultStrategy - +- [ ] ConsumerType + - [ ] PushConsumer +- [ ] MessageListener + - [ ] Concurrently +- [ ] MessageModel + - [ ] CLUSTERING +- [ ] OffsetStore + - [ ] RemoteBrokerOffsetStore +- [ ] RebalanceService +- [ ] PullMessageService +- [ ] ConsumeMessageService +- [ ] AllocateMessageQueueStrategy + - [ ] AllocateMessageQueueAveragely +- [ ] Other + - [ ] Config + - [ ] ZIP + - [ ] ConsumeFromWhere + - [ ] CONSUME_FROM_LAST_OFFSET + - [ ] CONSUME_FROM_FIRST_OFFSET + - [ ] CONSUME_FROM_TIMESTAMP + - [ ] Retry(sendMessageBack) + - [ ] TimeOut(clearExpiredMessage) + - [ ] ACK(partSuccess) + - [ ] FlowControl(messageCanNotConsume) ## Manager -- [x] Controller - - [x] PullMessageController -- [x] Task - - [x] UpdateTopicRouteInfo - - [x] Heartbeat - - [x] Rebalance - - [x] PullMessage - - [x] CleanExpireMsg -- [x] ClientRemotingProcessor - - [x] CHECK_TRANSACTION_STATE - - [x] NOTIFY_CONSUMER_IDS_CHANGED - - [x] RESET_CONSUMER_CLIENT_OFFSET - - [x] GET_CONSUMER_STATUS_FROM_CLIENT - - [x] GET_CONSUMER_RUNNING_INFO - - [x] CONSUME_MESSAGE_DIRECTLY +- [ ] Controller + - [ ] PullMessageController +- [ ] Task + - [ ] Heartbeat + - [ ] UpdateTopicRouteInfoFromNameServer + - [ ] PersistAllConsumerOffset + - [ ] ClearExpiredMessage(form consumer consumeMessageService) +- [ ] ClientRemotingProcessor + - [ ] CHECK_TRANSACTION_STATE + - [ ] NOTIFY_CONSUMER_IDS_CHANGED + - [ ] RESET_CONSUMER_CLIENT_OFFSET + - [ ] GET_CONSUMER_STATUS_FROM_CLIENT + - [ ] GET_CONSUMER_RUNNING_INFO + - [ ] CONSUME_MESSAGE_DIRECTLY ## Remoting - [x] MqClientRequest @@ -141,11 +122,13 @@ - [ ] RebalanceController - [ ] PullMessageController - [ ] Task - - [ ] UpdateTopicRouteInfo + - [ ] PollNameServer - [ ] Heartbeat - - [ ] Rebalance - - [ ] PullMessage - - [ ] CleanExpireMsg + - [ ] UpdateTopicRouteInfoFromNameServer + - [ ] CleanOfflineBroker + - [ ] PersistAllConsumerOffset + - [ ] ClearExpiredMessage(form consumer consumeMessageService) + - [ ] UploadFilterClassSource(FromHeartBeat/But Golang Not Easy To do this(Java Source)) - [ ] ClientRemotingProcessor - [ ] CHECK_TRANSACTION_STATE - [ ] NOTIFY_CONSUMER_IDS_CHANGED diff --git a/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml b/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml new file mode 100644 index 000000000..df26aa06c --- /dev/null +++ b/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml @@ -0,0 +1,10 @@ +@startuml +MqClientManager ->ClientFactory:get all topics from consumer.subscription +MqClientManager -> MqClient:all topics from topic publish info +MqClientManager -> MqClient:for each topic,UpdateTopicRouteInfoFromNameServer +MqClient -> RemotingClient:GetTopicRouteInfoFromNameServer +MqClient -> MqClient:updateTopicRouteInfoLocal \nBrokerAddrTable\nTopicPublishInfoTable\nTopicSubscribeInfoTable\nTopicRouteTable + +@enduml + + From 8830e6412101d900f6004b619d549802dbd55ae3 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 16 Aug 2017 00:02:07 +0800 Subject: [PATCH 31/88] go fmt --- rocketmq-go/test/test_delay_message_send_and_receive.go | 2 +- .../test_rocketmq_serialize_message_send_and_receive.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index 6c932f34c..a962f1239 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -49,7 +49,7 @@ func main() { result, err := producer.Send(message) startTime = time.Now() messageId = result.MsgID() - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err); + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) select { case <-chResult: case <-time.After(time.Second * 30): diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index a2e0a44b1..7fa2e2bf8 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -14,7 +14,7 @@ func main() { testProducerGroup = "TestSerializeProducerGroup" testConsumerGroup = "TestSerializeConsumerGroup" tag = "TestSerializeMessageTag" - messageBody = "testMessageBody_testMessageBody" + messageBody = "testMessageBody_testMessageBody" messageCount = 100 ) chResult := make(chan bool, messageCount) @@ -28,7 +28,7 @@ func main() { consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - if (msg.GetTag() == tag && messageBody == string(messageBody)) { + if msg.GetTag() == tag && messageBody == string(messageBody) { chResult <- true } successIndex = index @@ -42,7 +42,7 @@ func main() { var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(messageBody)} message.SetTag(tag) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err); + glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } for i := 0; i < messageCount; i++ { select { From d96abe345edd194fb9dfae82ec539f0568beabf7 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 16 Aug 2017 00:04:50 +0800 Subject: [PATCH 32/88] go report --- rocketmq-go/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-go/README.md b/rocketmq-go/README.md index 43f7c0b26..2310206dc 100644 --- a/rocketmq-go/README.md +++ b/rocketmq-go/README.md @@ -1,4 +1,4 @@ -# RocketMQ Go SDK Millstone1 Detail Design +# RocketMQ Go SDK Millstone1 Detail Design[![Go Report Card](https://goreportcard.com/badge/github.com/StyleTang/incubator-rocketmq-externals)](https://goreportcard.com/report/github.com/StyleTang/incubator-rocketmq-externals) ## Example ``` From 30a0e49ebd7fd82be2605093c0f70ecb635772bf Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 16 Aug 2017 00:25:57 +0800 Subject: [PATCH 33/88] License format --- rocketmq-go/api/model/message.go | 29 +++++++++++++++-------------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 81a9d9f7b..6891f0e3d 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -1,19 +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. +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + package rocketmq_api_model import ( From 5531aebadbf7cbec7506304c704c064b382d678b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 17 Aug 2017 21:46:03 +0800 Subject: [PATCH 34/88] for golint --- .../api/model/consume_concurrently_result.go | 31 +++++++++-------- rocketmq-go/api/model/message.go | 2 +- rocketmq-go/api/model/message_ext.go | 31 +++++++++-------- rocketmq-go/api/model/mq_client_config.go | 31 +++++++++-------- rocketmq-go/api/model/mq_consumer_config.go | 31 +++++++++-------- rocketmq-go/api/model/mq_producer_config.go | 31 +++++++++-------- rocketmq-go/api/rocketmq_clent_instance.go | 31 +++++++++-------- rocketmq-go/api/rocketmq_consumer.go | 31 +++++++++-------- rocketmq-go/api/rocketmq_producer.go | 31 +++++++++-------- rocketmq-go/example/simple_consumer.go | 31 +++++++++-------- rocketmq-go/example/simple_producer.go | 31 +++++++++-------- .../example/simple_producer_consumer.go | 31 +++++++++-------- .../manage/clean_expire_msg_controller.go | 31 +++++++++-------- rocketmq-go/manage/mq_client_manage.go | 31 +++++++++-------- rocketmq-go/manage/mq_producer.go | 17 ++++++++++ rocketmq-go/manage/mq_push_consumer.go | 31 +++++++++-------- rocketmq-go/manage/pull_message_controller.go | 31 +++++++++-------- rocketmq-go/manage/rebalance_controller.go | 31 +++++++++-------- rocketmq-go/manage/tasks.go | 31 +++++++++-------- rocketmq-go/model/client_config.go | 34 +++++++++---------- rocketmq-go/model/constant/config.go | 31 +++++++++-------- rocketmq-go/model/constant/message_const.go | 31 +++++++++-------- .../model/constant/message_sys_flag.go | 31 +++++++++-------- rocketmq-go/model/constant/mix_all.go | 31 +++++++++-------- rocketmq-go/model/constant/perm.go | 31 +++++++++-------- rocketmq-go/model/constant/pull_sys_flag.go | 31 +++++++++-------- .../model/consume_message_directly_result.go | 31 +++++++++-------- rocketmq-go/model/consumer_running_info.go | 31 +++++++++-------- ..._message_directly_result_request_header.go | 31 +++++++++-------- .../consumer_send_msg_back_request_header.go | 31 +++++++++-------- .../header/get_consumer_list_by_group.go | 31 +++++++++-------- ...et_consumer_running_info_request_header.go | 31 +++++++++-------- .../header/get_max_offset_request_header.go | 31 +++++++++-------- .../header/get_max_offset_response_header.go | 31 +++++++++-------- .../header/get_route_info_request_header.go | 31 +++++++++-------- .../header/pull_message_request_header.go | 31 +++++++++-------- .../query_consumer_offset_request_header.go | 31 +++++++++-------- .../header/reset_offset_request_header.go | 31 +++++++++-------- .../header/search_offset_request_header.go | 31 +++++++++-------- .../header/send_message_request_header.go | 31 +++++++++-------- .../header/send_message_response_header.go | 31 +++++++++-------- .../update_consumer_offset_request_header.go | 31 +++++++++-------- rocketmq-go/model/heart_beat.go | 31 +++++++++-------- rocketmq-go/model/message/message.go | 33 +++++++++--------- rocketmq-go/model/message/message_constant.go | 34 +++++++++---------- rocketmq-go/model/message_listener.go | 31 +++++++++-------- rocketmq-go/model/message_queue.go | 31 +++++++++-------- rocketmq-go/model/process_queue.go | 31 +++++++++-------- rocketmq-go/model/process_queue_info.go | 31 +++++++++-------- rocketmq-go/model/pull_request.go | 31 +++++++++-------- rocketmq-go/model/pull_result.go | 34 +++++++++---------- rocketmq-go/model/query_result.go | 34 +++++++++---------- rocketmq-go/model/request_code.go | 30 ++++++++-------- rocketmq-go/model/reset_offset_body.go | 31 +++++++++-------- rocketmq-go/model/response_code.go | 31 +++++++++-------- rocketmq-go/model/send_result.go | 31 +++++++++-------- rocketmq-go/model/subscription_data.go | 31 +++++++++-------- rocketmq-go/model/topic_publish_info.go | 31 +++++++++-------- rocketmq-go/model/topic_route_data.go | 30 ++++++++-------- rocketmq-go/remoting/communication_mode.go | 30 ++++++++-------- rocketmq-go/remoting/custom_header.go | 31 +++++++++-------- rocketmq-go/remoting/json_serializable.go | 31 +++++++++-------- .../remoting/json_serializable_test.go | 31 +++++++++-------- rocketmq-go/remoting/remoting_client.go | 31 +++++++++-------- rocketmq-go/remoting/remoting_client_test.go | 31 +++++++++-------- rocketmq-go/remoting/remoting_command.go | 31 +++++++++-------- rocketmq-go/remoting/request_code.go | 31 +++++++++-------- rocketmq-go/remoting/request_processor.go | 31 +++++++++-------- rocketmq-go/remoting/response_code.go | 31 +++++++++-------- rocketmq-go/remoting/response_future.go | 31 +++++++++-------- rocketmq-go/remoting/rocketmq_serializable.go | 31 +++++++++-------- .../remoting/rocketmq_serializable_test.go | 31 +++++++++-------- rocketmq-go/remoting/rpchook.go | 30 ++++++++-------- rocketmq-go/remoting/serializable.go | 31 +++++++++-------- .../allocate_message_averagely.go | 31 +++++++++-------- .../allocate_message_averagely_by_circle.go | 31 +++++++++-------- .../allocate_message_by_config.go | 31 +++++++++-------- .../allocate_message_by_machine_room.go | 31 +++++++++-------- .../allocate_message_queue_strategy.go | 31 +++++++++-------- .../service/consume_message_service.go | 31 +++++++++-------- rocketmq-go/service/mq_client.go | 31 +++++++++-------- rocketmq-go/service/mq_fault_strategy.go | 31 +++++++++-------- rocketmq-go/service/offset_store.go | 31 +++++++++-------- rocketmq-go/service/producer_service.go | 31 +++++++++-------- .../service/producer_service_for_send_back.go | 31 +++++++++-------- rocketmq-go/service/rebalance.go | 31 +++++++++-------- .../test/test_consume_message_retry.go | 17 ++++++++++ .../test_delay_message_send_and_receive.go | 17 ++++++++++ ...etmq_serialize_message_send_and_receive.go | 17 ++++++++++ .../test_send_and_receive_compress_message.go | 17 ++++++++++ .../test/test_send_and_receive_tag_message.go | 17 ++++++++++ rocketmq-go/util/compress_util.go | 31 +++++++++-------- rocketmq-go/util/compress_util_test.go | 31 +++++++++-------- rocketmq-go/util/concurrent_map.go | 31 +++++++++-------- rocketmq-go/util/ip.go | 31 +++++++++-------- rocketmq-go/util/ip_test.go | 31 +++++++++-------- rocketmq-go/util/json_util.go | 31 +++++++++-------- rocketmq-go/util/json_util_test.go | 31 +++++++++-------- .../util/message_client_id_generator.go | 31 +++++++++-------- .../util/message_client_id_generator_test.go | 31 +++++++++-------- rocketmq-go/util/message_properties.go | 31 +++++++++-------- rocketmq-go/util/message_properties_test.go | 31 +++++++++-------- rocketmq-go/util/regex_util.go | 31 +++++++++-------- rocketmq-go/util/regex_util_test.go | 31 +++++++++-------- rocketmq-go/util/string_util.go | 31 +++++++++-------- rocketmq-go/util/string_util_test.go | 31 +++++++++-------- rocketmq-go/util/structs/field.go | 17 ++++++++++ rocketmq-go/util/structs/structs.go | 18 +++++++++- rocketmq-go/util/structs/tags.go | 17 ++++++++++ rocketmq-go/util/time_util.go | 17 ++++++++++ rocketmq-go/util/time_util_test.go | 17 ++++++++++ 111 files changed, 1773 insertions(+), 1496 deletions(-) diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go index 070e68d80..329aafcbc 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api_model type ConsumeStatus int diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 6891f0e3d..d73124d89 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -13,7 +13,7 @@ distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. - */ +*/ package rocketmq_api_model diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index dcc857a2e..2eb680bc4 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api_model import ( diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 77f081192..c83cbe396 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api_model //-------SerializeType------- diff --git a/rocketmq-go/api/model/mq_consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go index c28bf5056..0b5e59c4c 100644 --- a/rocketmq-go/api/model/mq_consumer_config.go +++ b/rocketmq-go/api/model/mq_consumer_config.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api_model import "time" diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index a2cc7ce13..664d95ec8 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api_model type RocketMqProducerConfig struct { diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 41977e403..0bbe18dba 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api import ( diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index a883e9280..22afed520 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api import ( diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 579026679..1cff17352 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq_api import ( diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index b997347d9..1fcb3b8f2 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package main import ( diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index 2582da193..03c1f168a 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package main import ( diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index a3e8dc36f..27c5140e1 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package main import ( diff --git a/rocketmq-go/manage/clean_expire_msg_controller.go b/rocketmq-go/manage/clean_expire_msg_controller.go index 73bb35b3c..ceed735c8 100644 --- a/rocketmq-go/manage/clean_expire_msg_controller.go +++ b/rocketmq-go/manage/clean_expire_msg_controller.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq import ( diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index a7110a2e3..db9c66f3a 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq import ( diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index 634e5ddec..b490d6299 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -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. +*/ + package rocketmq import ( diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index e724db09f..c012818e6 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq import ( diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index 560be85e3..6202d990e 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq import ( diff --git a/rocketmq-go/manage/rebalance_controller.go b/rocketmq-go/manage/rebalance_controller.go index d6d4001eb..3855a1080 100644 --- a/rocketmq-go/manage/rebalance_controller.go +++ b/rocketmq-go/manage/rebalance_controller.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq type RebalanceController struct { diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/manage/tasks.go index 70b4672c0..227f98cd1 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/manage/tasks.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package rocketmq import ( diff --git a/rocketmq-go/model/client_config.go b/rocketmq-go/model/client_config.go index e3c2fab7b..ca58dfb45 100644 --- a/rocketmq-go/model/client_config.go +++ b/rocketmq-go/model/client_config.go @@ -1,20 +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. -// */ -// +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model // diff --git a/rocketmq-go/model/constant/config.go b/rocketmq-go/model/constant/config.go index 80c57bb51..1aba15407 100644 --- a/rocketmq-go/model/constant/config.go +++ b/rocketmq-go/model/constant/config.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant var REMOTING_COMMAND_FLAG = 0 diff --git a/rocketmq-go/model/constant/message_const.go b/rocketmq-go/model/constant/message_const.go index 402d328c5..14ec59651 100644 --- a/rocketmq-go/model/constant/message_const.go +++ b/rocketmq-go/model/constant/message_const.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant const ( diff --git a/rocketmq-go/model/constant/message_sys_flag.go b/rocketmq-go/model/constant/message_sys_flag.go index a53c4fd96..f9bd5bff8 100644 --- a/rocketmq-go/model/constant/message_sys_flag.go +++ b/rocketmq-go/model/constant/message_sys_flag.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant const ( diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index 44fdd4a97..d0f9c3fad 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant const ( diff --git a/rocketmq-go/model/constant/perm.go b/rocketmq-go/model/constant/perm.go index 962d9894a..6bcc034ae 100644 --- a/rocketmq-go/model/constant/perm.go +++ b/rocketmq-go/model/constant/perm.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant const ( diff --git a/rocketmq-go/model/constant/pull_sys_flag.go b/rocketmq-go/model/constant/pull_sys_flag.go index 0a2921cdd..f06b39fbc 100644 --- a/rocketmq-go/model/constant/pull_sys_flag.go +++ b/rocketmq-go/model/constant/pull_sys_flag.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package constant const ( diff --git a/rocketmq-go/model/consume_message_directly_result.go b/rocketmq-go/model/consume_message_directly_result.go index 8dcea7cba..92cf1a8a2 100644 --- a/rocketmq-go/model/consume_message_directly_result.go +++ b/rocketmq-go/model/consume_message_directly_result.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model type ConsumeResultType string diff --git a/rocketmq-go/model/consumer_running_info.go b/rocketmq-go/model/consumer_running_info.go index 2d1bf1282..e327275c6 100644 --- a/rocketmq-go/model/consumer_running_info.go +++ b/rocketmq-go/model/consumer_running_info.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import "encoding/json" diff --git a/rocketmq-go/model/header/consume_message_directly_result_request_header.go b/rocketmq-go/model/header/consume_message_directly_result_request_header.go index 9c2d77eb3..9b99039dd 100644 --- a/rocketmq-go/model/header/consume_message_directly_result_request_header.go +++ b/rocketmq-go/model/header/consume_message_directly_result_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type ConsumeMessageDirectlyResultRequestHeader struct { diff --git a/rocketmq-go/model/header/consumer_send_msg_back_request_header.go b/rocketmq-go/model/header/consumer_send_msg_back_request_header.go index 727914f63..5ac822278 100644 --- a/rocketmq-go/model/header/consumer_send_msg_back_request_header.go +++ b/rocketmq-go/model/header/consumer_send_msg_back_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type ConsumerSendMsgBackRequestHeader struct { diff --git a/rocketmq-go/model/header/get_consumer_list_by_group.go b/rocketmq-go/model/header/get_consumer_list_by_group.go index 97b8304a6..69aa394ed 100644 --- a/rocketmq-go/model/header/get_consumer_list_by_group.go +++ b/rocketmq-go/model/header/get_consumer_list_by_group.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type GetConsumerListByGroupRequestHeader struct { diff --git a/rocketmq-go/model/header/get_consumer_running_info_request_header.go b/rocketmq-go/model/header/get_consumer_running_info_request_header.go index 2d5ab5459..e9e02a661 100644 --- a/rocketmq-go/model/header/get_consumer_running_info_request_header.go +++ b/rocketmq-go/model/header/get_consumer_running_info_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type GetConsumerRunningInfoRequestHeader struct { diff --git a/rocketmq-go/model/header/get_max_offset_request_header.go b/rocketmq-go/model/header/get_max_offset_request_header.go index db8b765a6..2393fe35f 100644 --- a/rocketmq-go/model/header/get_max_offset_request_header.go +++ b/rocketmq-go/model/header/get_max_offset_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type GetMaxOffsetRequestHeader struct { diff --git a/rocketmq-go/model/header/get_max_offset_response_header.go b/rocketmq-go/model/header/get_max_offset_response_header.go index 59dd9baad..5e695d840 100644 --- a/rocketmq-go/model/header/get_max_offset_response_header.go +++ b/rocketmq-go/model/header/get_max_offset_response_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" diff --git a/rocketmq-go/model/header/get_route_info_request_header.go b/rocketmq-go/model/header/get_route_info_request_header.go index 2ebc20b73..91eefc9ef 100644 --- a/rocketmq-go/model/header/get_route_info_request_header.go +++ b/rocketmq-go/model/header/get_route_info_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type GetRouteInfoRequestHeader struct { diff --git a/rocketmq-go/model/header/pull_message_request_header.go b/rocketmq-go/model/header/pull_message_request_header.go index 966a9cd5b..73ae576a6 100644 --- a/rocketmq-go/model/header/pull_message_request_header.go +++ b/rocketmq-go/model/header/pull_message_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type PullMessageRequestHeader struct { diff --git a/rocketmq-go/model/header/query_consumer_offset_request_header.go b/rocketmq-go/model/header/query_consumer_offset_request_header.go index bdd693cc7..86eed07d6 100644 --- a/rocketmq-go/model/header/query_consumer_offset_request_header.go +++ b/rocketmq-go/model/header/query_consumer_offset_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type QueryConsumerOffsetRequestHeader struct { diff --git a/rocketmq-go/model/header/reset_offset_request_header.go b/rocketmq-go/model/header/reset_offset_request_header.go index 9dd8ee601..934ef28e3 100644 --- a/rocketmq-go/model/header/reset_offset_request_header.go +++ b/rocketmq-go/model/header/reset_offset_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header import ( diff --git a/rocketmq-go/model/header/search_offset_request_header.go b/rocketmq-go/model/header/search_offset_request_header.go index 50944c1b9..6226f5ff9 100644 --- a/rocketmq-go/model/header/search_offset_request_header.go +++ b/rocketmq-go/model/header/search_offset_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header import () diff --git a/rocketmq-go/model/header/send_message_request_header.go b/rocketmq-go/model/header/send_message_request_header.go index 5eeea6053..41d4b3e87 100644 --- a/rocketmq-go/model/header/send_message_request_header.go +++ b/rocketmq-go/model/header/send_message_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type SendMessageRequestHeader struct { diff --git a/rocketmq-go/model/header/send_message_response_header.go b/rocketmq-go/model/header/send_message_response_header.go index 9ddacb29f..c2bc50d3c 100644 --- a/rocketmq-go/model/header/send_message_response_header.go +++ b/rocketmq-go/model/header/send_message_response_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header type SendMessageResponseHeader struct { diff --git a/rocketmq-go/model/header/update_consumer_offset_request_header.go b/rocketmq-go/model/header/update_consumer_offset_request_header.go index 0826059d3..287240cbe 100644 --- a/rocketmq-go/model/header/update_consumer_offset_request_header.go +++ b/rocketmq-go/model/header/update_consumer_offset_request_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package header import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" diff --git a/rocketmq-go/model/heart_beat.go b/rocketmq-go/model/heart_beat.go index 5e206f31a..4a2ff1010 100644 --- a/rocketmq-go/model/heart_beat.go +++ b/rocketmq-go/model/heart_beat.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" diff --git a/rocketmq-go/model/message/message.go b/rocketmq-go/model/message/message.go index 7e42543ac..7ed484150 100644 --- a/rocketmq-go/model/message/message.go +++ b/rocketmq-go/model/message/message.go @@ -1,19 +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. -// */ +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package message // diff --git a/rocketmq-go/model/message/message_constant.go b/rocketmq-go/model/message/message_constant.go index b1d731e04..a358ce8e0 100644 --- a/rocketmq-go/model/message/message_constant.go +++ b/rocketmq-go/model/message/message_constant.go @@ -1,20 +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. -// */ -// +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package message // diff --git a/rocketmq-go/model/message_listener.go b/rocketmq-go/model/message_listener.go index 1e166c4a5..b4ab7f7ed 100644 --- a/rocketmq-go/model/message_listener.go +++ b/rocketmq-go/model/message_listener.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" diff --git a/rocketmq-go/model/message_queue.go b/rocketmq-go/model/message_queue.go index fc5c8608f..5f6219663 100644 --- a/rocketmq-go/model/message_queue.go +++ b/rocketmq-go/model/message_queue.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model type MessageQueue struct { diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index a3e178f8e..c565c7205 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import ( diff --git a/rocketmq-go/model/process_queue_info.go b/rocketmq-go/model/process_queue_info.go index c221ef8a9..6b49616fb 100644 --- a/rocketmq-go/model/process_queue_info.go +++ b/rocketmq-go/model/process_queue_info.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model type ProcessQueueInfo struct { diff --git a/rocketmq-go/model/pull_request.go b/rocketmq-go/model/pull_request.go index bc1a46f2d..1ae0e35f3 100644 --- a/rocketmq-go/model/pull_request.go +++ b/rocketmq-go/model/pull_request.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model type PullRequest struct { diff --git a/rocketmq-go/model/pull_result.go b/rocketmq-go/model/pull_result.go index 694b88f92..e9ec9fd11 100644 --- a/rocketmq-go/model/pull_result.go +++ b/rocketmq-go/model/pull_result.go @@ -1,20 +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. -// */ -// +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model // diff --git a/rocketmq-go/model/query_result.go b/rocketmq-go/model/query_result.go index 9cdf9a360..be47f83af 100644 --- a/rocketmq-go/model/query_result.go +++ b/rocketmq-go/model/query_result.go @@ -1,20 +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. -// */ -// +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model // diff --git a/rocketmq-go/model/request_code.go b/rocketmq-go/model/request_code.go index 495c1a7ec..c70fa1e25 100644 --- a/rocketmq-go/model/request_code.go +++ b/rocketmq-go/model/request_code.go @@ -1,19 +1,19 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http:// www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ package model diff --git a/rocketmq-go/model/reset_offset_body.go b/rocketmq-go/model/reset_offset_body.go index d2db83255..274148f59 100644 --- a/rocketmq-go/model/reset_offset_body.go +++ b/rocketmq-go/model/reset_offset_body.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import ( diff --git a/rocketmq-go/model/response_code.go b/rocketmq-go/model/response_code.go index a42120e26..03a0fb2ac 100644 --- a/rocketmq-go/model/response_code.go +++ b/rocketmq-go/model/response_code.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model const ( diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go index 857b6c402..2cccd0c8e 100644 --- a/rocketmq-go/model/send_result.go +++ b/rocketmq-go/model/send_result.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import ( diff --git a/rocketmq-go/model/subscription_data.go b/rocketmq-go/model/subscription_data.go index ce5ae74e3..90a135a44 100644 --- a/rocketmq-go/model/subscription_data.go +++ b/rocketmq-go/model/subscription_data.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model type SubscriptionData struct { diff --git a/rocketmq-go/model/topic_publish_info.go b/rocketmq-go/model/topic_publish_info.go index 6fa6f0cf9..bccaf2a3d 100644 --- a/rocketmq-go/model/topic_publish_info.go +++ b/rocketmq-go/model/topic_publish_info.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package model import ( diff --git a/rocketmq-go/model/topic_route_data.go b/rocketmq-go/model/topic_route_data.go index 9c1ab27cb..0241411a2 100644 --- a/rocketmq-go/model/topic_route_data.go +++ b/rocketmq-go/model/topic_route_data.go @@ -1,19 +1,19 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ package model diff --git a/rocketmq-go/remoting/communication_mode.go b/rocketmq-go/remoting/communication_mode.go index 338095505..bc084a57a 100644 --- a/rocketmq-go/remoting/communication_mode.go +++ b/rocketmq-go/remoting/communication_mode.go @@ -1,19 +1,19 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ package remoting diff --git a/rocketmq-go/remoting/custom_header.go b/rocketmq-go/remoting/custom_header.go index 04a46befe..c85935cc7 100644 --- a/rocketmq-go/remoting/custom_header.go +++ b/rocketmq-go/remoting/custom_header.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting type CustomerHeader interface { diff --git a/rocketmq-go/remoting/json_serializable.go b/rocketmq-go/remoting/json_serializable.go index dc839b590..a32e0ffd2 100644 --- a/rocketmq-go/remoting/json_serializable.go +++ b/rocketmq-go/remoting/json_serializable.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting import ( diff --git a/rocketmq-go/remoting/json_serializable_test.go b/rocketmq-go/remoting/json_serializable_test.go index 287e45a7c..ab8e5a3fe 100644 --- a/rocketmq-go/remoting/json_serializable_test.go +++ b/rocketmq-go/remoting/json_serializable_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting_test import ( diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 7fbb20c58..274d203da 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting import ( diff --git a/rocketmq-go/remoting/remoting_client_test.go b/rocketmq-go/remoting/remoting_client_test.go index f1d85ada8..0a3baaf9e 100644 --- a/rocketmq-go/remoting/remoting_client_test.go +++ b/rocketmq-go/remoting/remoting_client_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting_test //func TestClearExpireResponse(t *testing.T) { diff --git a/rocketmq-go/remoting/remoting_command.go b/rocketmq-go/remoting/remoting_command.go index 748117488..4df511d13 100644 --- a/rocketmq-go/remoting/remoting_command.go +++ b/rocketmq-go/remoting/remoting_command.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting import ( diff --git a/rocketmq-go/remoting/request_code.go b/rocketmq-go/remoting/request_code.go index 52965d5d3..b6ded0cd9 100644 --- a/rocketmq-go/remoting/request_code.go +++ b/rocketmq-go/remoting/request_code.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting const ( diff --git a/rocketmq-go/remoting/request_processor.go b/rocketmq-go/remoting/request_processor.go index eec8cd8d1..1c71b0970 100644 --- a/rocketmq-go/remoting/request_processor.go +++ b/rocketmq-go/remoting/request_processor.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting type ClientRequestProcessor func(remotingCommand *RemotingCommand) (responseCommand *RemotingCommand) diff --git a/rocketmq-go/remoting/response_code.go b/rocketmq-go/remoting/response_code.go index 6a49c77f9..2b5682768 100644 --- a/rocketmq-go/remoting/response_code.go +++ b/rocketmq-go/remoting/response_code.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting const ( diff --git a/rocketmq-go/remoting/response_future.go b/rocketmq-go/remoting/response_future.go index 1bece6d4f..a1c3b8945 100644 --- a/rocketmq-go/remoting/response_future.go +++ b/rocketmq-go/remoting/response_future.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting type ResponseFuture struct { diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index 4e59f228c..3c1ab38ca 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting import ( diff --git a/rocketmq-go/remoting/rocketmq_serializable_test.go b/rocketmq-go/remoting/rocketmq_serializable_test.go index 58048afa9..91d5f08e0 100644 --- a/rocketmq-go/remoting/rocketmq_serializable_test.go +++ b/rocketmq-go/remoting/rocketmq_serializable_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting_test import ( diff --git a/rocketmq-go/remoting/rpchook.go b/rocketmq-go/remoting/rpchook.go index 150ffe071..92d7cf122 100644 --- a/rocketmq-go/remoting/rpchook.go +++ b/rocketmq-go/remoting/rpchook.go @@ -1,19 +1,19 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ package remoting diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index cb7f2e11d..ee0efd6e6 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package remoting import ( diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely.go b/rocketmq-go/service/allocate_message/allocate_message_averagely.go index 103bade8c..572666701 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_averagely.go +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service_allocate_message import ( diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go index a93070ffc..0dacb7a8e 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go +++ b/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service_allocate_message import ( diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_config.go b/rocketmq-go/service/allocate_message/allocate_message_by_config.go index d55424a54..934a4fb9a 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_by_config.go +++ b/rocketmq-go/service/allocate_message/allocate_message_by_config.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service_allocate_message import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go index ff92eb783..b470ba6d9 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go +++ b/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service_allocate_message import ( diff --git a/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go b/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go index e838c7bf9..9c9444440 100644 --- a/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go +++ b/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service_allocate_message import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/service/consume_message_service.go index 5808aa044..9963c615c 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/service/consume_message_service.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/service/mq_client.go index 418564be3..dc829a16f 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/service/mq_client.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/mq_fault_strategy.go b/rocketmq-go/service/mq_fault_strategy.go index 852ab98a7..9e29adb65 100644 --- a/rocketmq-go/service/mq_fault_strategy.go +++ b/rocketmq-go/service/mq_fault_strategy.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/offset_store.go b/rocketmq-go/service/offset_store.go index c2f64a619..6164f9e17 100644 --- a/rocketmq-go/service/offset_store.go +++ b/rocketmq-go/service/offset_store.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/service/producer_service.go index 1342b7969..6e5190b43 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/service/producer_service.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/producer_service_for_send_back.go b/rocketmq-go/service/producer_service_for_send_back.go index 910d6bfee..d1a8a4458 100644 --- a/rocketmq-go/service/producer_service_for_send_back.go +++ b/rocketmq-go/service/producer_service_for_send_back.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/service/rebalance.go b/rocketmq-go/service/rebalance.go index c2d186a80..45308e340 100644 --- a/rocketmq-go/service/rebalance.go +++ b/rocketmq-go/service/rebalance.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package service import ( diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index c309fbed0..8878b4415 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -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. +*/ + package main import ( diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index a962f1239..8d62affdf 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -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. +*/ + package main import ( diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index 7fa2e2bf8..bdcea6d7d 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -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. +*/ + package main import ( diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index 49c774b56..3e9147562 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -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. +*/ + package main import ( diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index 542b276b7..1218aded6 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -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. +*/ + package main import ( diff --git a/rocketmq-go/util/compress_util.go b/rocketmq-go/util/compress_util.go index 8925afa90..06eb6ed94 100644 --- a/rocketmq-go/util/compress_util.go +++ b/rocketmq-go/util/compress_util.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/compress_util_test.go b/rocketmq-go/util/compress_util_test.go index ccef8f5fc..2673feb06 100644 --- a/rocketmq-go/util/compress_util_test.go +++ b/rocketmq-go/util/compress_util_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go index 9d3e27360..2d81a76d1 100644 --- a/rocketmq-go/util/concurrent_map.go +++ b/rocketmq-go/util/concurrent_map.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/ip.go b/rocketmq-go/util/ip.go index f87000db9..874b532b6 100644 --- a/rocketmq-go/util/ip.go +++ b/rocketmq-go/util/ip.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/ip_test.go b/rocketmq-go/util/ip_test.go index 3e6e8684f..ee543a5f2 100644 --- a/rocketmq-go/util/ip_test.go +++ b/rocketmq-go/util/ip_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/json_util.go b/rocketmq-go/util/json_util.go index 91fdc5d3d..5fa6716c5 100644 --- a/rocketmq-go/util/json_util.go +++ b/rocketmq-go/util/json_util.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/json_util_test.go b/rocketmq-go/util/json_util_test.go index 6b4f2a262..7440ca755 100644 --- a/rocketmq-go/util/json_util_test.go +++ b/rocketmq-go/util/json_util_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/message_client_id_generator.go b/rocketmq-go/util/message_client_id_generator.go index c0c53df80..895cdf68f 100644 --- a/rocketmq-go/util/message_client_id_generator.go +++ b/rocketmq-go/util/message_client_id_generator.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/message_client_id_generator_test.go b/rocketmq-go/util/message_client_id_generator_test.go index df082c144..3e16c8644 100644 --- a/rocketmq-go/util/message_client_id_generator_test.go +++ b/rocketmq-go/util/message_client_id_generator_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/message_properties.go b/rocketmq-go/util/message_properties.go index 59fd5b8cf..30a40318b 100644 --- a/rocketmq-go/util/message_properties.go +++ b/rocketmq-go/util/message_properties.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/message_properties_test.go b/rocketmq-go/util/message_properties_test.go index 1de8a295e..8f430d7fc 100644 --- a/rocketmq-go/util/message_properties_test.go +++ b/rocketmq-go/util/message_properties_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/regex_util.go b/rocketmq-go/util/regex_util.go index 966a359a4..1527e4042 100644 --- a/rocketmq-go/util/regex_util.go +++ b/rocketmq-go/util/regex_util.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/regex_util_test.go b/rocketmq-go/util/regex_util_test.go index fc228776b..f056fe022 100644 --- a/rocketmq-go/util/regex_util_test.go +++ b/rocketmq-go/util/regex_util_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/string_util.go b/rocketmq-go/util/string_util.go index 7e31e00ee..3318cdcc3 100644 --- a/rocketmq-go/util/string_util.go +++ b/rocketmq-go/util/string_util.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import "strconv" diff --git a/rocketmq-go/util/string_util_test.go b/rocketmq-go/util/string_util_test.go index f6b29fdea..86975671f 100644 --- a/rocketmq-go/util/string_util_test.go +++ b/rocketmq-go/util/string_util_test.go @@ -1,19 +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. - */ +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package util import ( diff --git a/rocketmq-go/util/structs/field.go b/rocketmq-go/util/structs/field.go index e69783230..97721208b 100644 --- a/rocketmq-go/util/structs/field.go +++ b/rocketmq-go/util/structs/field.go @@ -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. +*/ + package structs import ( diff --git a/rocketmq-go/util/structs/structs.go b/rocketmq-go/util/structs/structs.go index 87b79702c..5cce7c86b 100644 --- a/rocketmq-go/util/structs/structs.go +++ b/rocketmq-go/util/structs/structs.go @@ -1,4 +1,20 @@ -// Package structs contains various utilities functions to work with structs. +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package structs import ( diff --git a/rocketmq-go/util/structs/tags.go b/rocketmq-go/util/structs/tags.go index 8859341c1..2dcf66b32 100644 --- a/rocketmq-go/util/structs/tags.go +++ b/rocketmq-go/util/structs/tags.go @@ -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. +*/ + package structs import "strings" diff --git a/rocketmq-go/util/time_util.go b/rocketmq-go/util/time_util.go index 690bb636c..80f962d23 100644 --- a/rocketmq-go/util/time_util.go +++ b/rocketmq-go/util/time_util.go @@ -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. +*/ + package util import ( diff --git a/rocketmq-go/util/time_util_test.go b/rocketmq-go/util/time_util_test.go index ce2a65edb..42320a04e 100644 --- a/rocketmq-go/util/time_util_test.go +++ b/rocketmq-go/util/time_util_test.go @@ -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. +*/ + package util_test import ( From 97a475c3e8bf8985290991549690530743347e53 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 17 Aug 2017 21:52:55 +0800 Subject: [PATCH 35/88] misspell --- rocketmq-go/manage/mq_client_manage.go | 2 +- rocketmq-go/service/offset_store.go | 8 ++++---- rocketmq-go/util/concurrent_map.go | 2 -- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index db9c66f3a..c7fb31e24 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -150,7 +150,7 @@ func (m *MqClientManager) RegistProducer(producer *DefaultMQProducer) { func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { consumer := m.clientFactory.ConsumerTable[group] if consumer == nil { - glog.Error("resetConsumerOffset beacuse consumer not online,group=", group) + glog.Error("resetConsumerOffset because consumer not online,group=", group) return } consumer.resetOffset(offsetTable) diff --git a/rocketmq-go/service/offset_store.go b/rocketmq-go/service/offset_store.go index 6164f9e17..40b51ab0f 100644 --- a/rocketmq-go/service/offset_store.go +++ b/rocketmq-go/service/offset_store.go @@ -121,17 +121,17 @@ func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *model.MessageQueue) func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *header.QueryConsumerOffsetRequestHeader, timeoutMillis int64) (int64, error) { remotingCommand := remoting.NewRemotingCommand(remoting.QUERY_CONSUMER_OFFSET, requestHeader) - reponse, err := r.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) + response, err := r.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) if err != nil { glog.Error(err) return -1, err } - if reponse.Code == remoting.QUERY_NOT_FOUND { + if response.Code == remoting.QUERY_NOT_FOUND { return -1, nil } - if offsetInter, ok := reponse.ExtFields["offset"]; ok { + if offsetInter, ok := response.ExtFields["offset"]; ok { if offsetStr, ok := offsetInter.(string); ok { offset, err := strconv.ParseInt(offsetStr, 10, 64) if err != nil { @@ -142,7 +142,7 @@ func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *heade } } - glog.Error(requestHeader, reponse) + glog.Error(requestHeader, response) return -1, errors.New("query offset error") } diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go index 2d81a76d1..0b70e918c 100644 --- a/rocketmq-go/util/concurrent_map.go +++ b/rocketmq-go/util/concurrent_map.go @@ -162,8 +162,6 @@ type Tuple struct { } // Returns an iterator which could be used in a for range loop. -// -// Deprecated: using IterBuffered() will get a better performence func (m ConcurrentMap) Iter() <-chan Tuple { ch := make(chan Tuple) go func() { From 46d95c90406646d99c0a058e68eeb0b5cb4dc764 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 17 Aug 2017 22:18:54 +0800 Subject: [PATCH 36/88] change package name for golint --- .../api/model/consume_concurrently_result.go | 2 +- rocketmq-go/api/model/message.go | 2 +- rocketmq-go/api/model/message_ext.go | 2 +- rocketmq-go/api/model/mq_client_config.go | 2 +- rocketmq-go/api/model/mq_consumer_config.go | 8 ++--- rocketmq-go/api/model/mq_producer_config.go | 8 ++--- rocketmq-go/api/rocketmq_clent_instance.go | 26 +++++++------- rocketmq-go/api/rocketmq_consumer.go | 12 +++---- rocketmq-go/api/rocketmq_producer.go | 16 ++++----- rocketmq-go/example/simple_consumer.go | 8 ++--- rocketmq-go/example/simple_producer.go | 6 ++-- .../example/simple_producer_consumer.go | 12 +++---- .../allocate_message_averagely.go | 0 .../allocate_message_averagely_by_circle.go | 0 .../allocate_message_by_config.go | 0 .../allocate_message_by_machine_room.go | 0 .../allocate_message_queue_strategy.go | 0 .../consume_message_service.go | 34 +++++++++---------- rocketmq-go/{service => kernel}/mq_client.go | 4 +-- .../{service => kernel}/mq_fault_strategy.go | 2 +- .../{service => kernel}/offset_store.go | 2 +- .../{service => kernel}/producer_service.go | 22 ++++++------ .../producer_service_for_send_back.go | 18 +++++----- rocketmq-go/{service => kernel}/rebalance.go | 14 ++++---- .../manage/clean_expire_msg_controller.go | 8 ++--- rocketmq-go/manage/mq_client_manage.go | 20 +++++------ rocketmq-go/manage/mq_producer.go | 14 ++++---- rocketmq-go/manage/mq_push_consumer.go | 18 +++++----- rocketmq-go/manage/pull_message_controller.go | 20 +++++------ rocketmq-go/manage/rebalance_controller.go | 2 +- rocketmq-go/manage/tasks.go | 2 +- rocketmq-go/model/heart_beat.go | 2 +- rocketmq-go/model/message_listener.go | 2 +- rocketmq-go/model/process_queue.go | 8 ++--- rocketmq-go/remoting/remoting_client.go | 4 +-- rocketmq-go/remoting/serializable.go | 14 ++++---- .../test/test_consume_message_retry.go | 12 +++---- .../test_delay_message_send_and_receive.go | 12 +++---- ...etmq_serialize_message_send_and_receive.go | 16 ++++----- .../test_send_and_receive_compress_message.go | 14 ++++---- .../test/test_send_and_receive_tag_message.go | 12 +++---- 41 files changed, 190 insertions(+), 190 deletions(-) rename rocketmq-go/{service => kernel}/allocate_message/allocate_message_averagely.go (100%) rename rocketmq-go/{service => kernel}/allocate_message/allocate_message_averagely_by_circle.go (100%) rename rocketmq-go/{service => kernel}/allocate_message/allocate_message_by_config.go (100%) rename rocketmq-go/{service => kernel}/allocate_message/allocate_message_by_machine_room.go (100%) rename rocketmq-go/{service => kernel}/allocate_message/allocate_message_queue_strategy.go (100%) rename rocketmq-go/{service => kernel}/consume_message_service.go (76%) rename rocketmq-go/{service => kernel}/mq_client.go (98%) rename rocketmq-go/{service => kernel}/mq_fault_strategy.go (99%) rename rocketmq-go/{service => kernel}/offset_store.go (99%) rename rocketmq-go/{service => kernel}/producer_service.go (86%) rename rocketmq-go/{service => kernel}/producer_service_for_send_back.go (87%) rename rocketmq-go/{service => kernel}/rebalance.go (96%) diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go index 329aafcbc..1a0d65be6 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm type ConsumeStatus int diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index d73124d89..cd9de627a 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 2eb680bc4..1e481f167 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index c83cbe396..4b5813ee7 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm //-------SerializeType------- // default serialize type is JSON_SERIALIZE, but ROCKETMQ_SERIALIZE(need version >= ?) is faster diff --git a/rocketmq-go/api/model/mq_consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go index 0b5e59c4c..a0ec836de 100644 --- a/rocketmq-go/api/model/mq_consumer_config.go +++ b/rocketmq-go/api/model/mq_consumer_config.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm import "time" @@ -43,7 +43,7 @@ const ( CONSUME_FROM_TIMESTAMP ) -type RocketMqConsumerConfig struct { +type MqConsumerConfig struct { /** * consume from where */ @@ -107,8 +107,8 @@ type RocketMqConsumerConfig struct { ConsumeTimestamp time.Time //when use CONSUME_FROM_TIMESTAMP } -func NewRocketMqConsumerConfig() (consumerConfig *RocketMqConsumerConfig) { - consumerConfig = &RocketMqConsumerConfig{ +func NewRocketMqConsumerConfig() (consumerConfig *MqConsumerConfig) { + consumerConfig = &MqConsumerConfig{ ConsumeFromWhere: CONSUME_FROM_LAST_OFFSET, ConsumeConcurrentlyMaxSpan: 2000, PullThresholdForQueue: 1000, diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index 664d95ec8..735436391 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -15,9 +15,9 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api_model +package rocketmqm -type RocketMqProducerConfig struct { +type MqProducerConfig struct { SendMsgTimeout int64 CompressMsgBodyOverHowMuch int ZipCompressLevel int @@ -30,8 +30,8 @@ type RocketMqProducerConfig struct { NotAvailableDuration []int64 } -func NewProducerConfig() (producerConfig *RocketMqProducerConfig) { - producerConfig = &RocketMqProducerConfig{ +func NewProducerConfig() (producerConfig *MqProducerConfig) { + producerConfig = &MqProducerConfig{ SendMsgTimeout: 3000, CompressMsgBodyOverHowMuch: 1024 * 4, ZipCompressLevel: 5, diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 0bbe18dba..24c25fb2c 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -15,38 +15,38 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api +package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" ) -type RocketMQClientInstance interface { - RegisterProducer(producer RocketMQProducer) - RegisterConsumer(consumer RocketMQConsumer) +type MQClientInstance interface { + RegisterProducer(producer MQProducer) + RegisterConsumer(consumer MQConsumer) Start() } type RocketMQClientInstanceImpl struct { - rocketMqManager *rocketmq.MqClientManager + rocketMqManager *manage.MqClientManager } -func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance RocketMQClientInstance) { - mqClientConfig := rocketmq_api_model.NewMqClientConfig(nameServerAddress) +func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance MQClientInstance) { + mqClientConfig := rocketmqm.NewMqClientConfig(nameServerAddress) return InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) } -func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmq_api_model.MqClientConfig) (rocketMQClientInstance RocketMQClientInstance) { - rocketMQClientInstance = &RocketMQClientInstanceImpl{rocketMqManager: rocketmq.MqClientManagerInit(mqClientConfig)} +func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmqm.MqClientConfig) (rocketMQClientInstance MQClientInstance) { + rocketMQClientInstance = &RocketMQClientInstanceImpl{rocketMqManager: manage.MqClientManagerInit(mqClientConfig)} return } -func (r *RocketMQClientInstanceImpl) RegisterProducer(producer RocketMQProducer) { - r.rocketMqManager.RegistProducer(producer.(*rocketmq.DefaultMQProducer)) +func (r *RocketMQClientInstanceImpl) RegisterProducer(producer MQProducer) { + r.rocketMqManager.RegistProducer(producer.(*manage.DefaultMQProducer)) } -func (r *RocketMQClientInstanceImpl) RegisterConsumer(consumer RocketMQConsumer) { - r.rocketMqManager.RegistConsumer(consumer.(*rocketmq.DefaultMQPushConsumer)) +func (r *RocketMQClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { + r.rocketMqManager.RegistConsumer(consumer.(*manage.DefaultMQPushConsumer)) } func (r *RocketMQClientInstanceImpl) Start() { r.rocketMqManager.Start() diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 22afed520..7ffd846f0 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api +package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" @@ -23,15 +23,15 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) -type RocketMQConsumer interface { +type MQConsumer interface { RegisterMessageListener(listener model.MessageListener) Subscribe(topic string, subExpression string) } -func NewDefaultMQPushConsumer(producerGroup string) (r RocketMQConsumer) { - return NewDefaultMQPushConsumerWithCustomConfig(producerGroup, rocketmq_api_model.NewRocketMqConsumerConfig()) +func NewDefaultMQPushConsumer(producerGroup string) (r MQConsumer) { + return NewDefaultMQPushConsumerWithCustomConfig(producerGroup, rocketmqm.NewRocketMqConsumerConfig()) } -func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (r RocketMQConsumer) { - return rocketmq.NewDefaultMQPushConsumer(producerGroup, consumerConfig) +func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (r MQConsumer) { + return manage.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 1cff17352..1883997b1 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq_api +package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" @@ -23,15 +23,15 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) -type RocketMQProducer interface { - Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) - SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) +type MQProducer interface { + Send(message *rocketmqm.Message) (sendResult *model.SendResult, err error) + SendWithTimeout(message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) } -func NewDefaultMQProducer(producerGroup string) (r RocketMQProducer) { - return NewDefaultMQProducerWithCustomConfig(producerGroup, rocketmq_api_model.NewProducerConfig()) +func NewDefaultMQProducer(producerGroup string) (r MQProducer) { + return NewDefaultMQProducerWithCustomConfig(producerGroup, rocketmqm.NewProducerConfig()) } -func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (r RocketMQProducer) { - return rocketmq.NewDefaultMQProducer(producerGroup, producerConfig) +func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (r MQProducer) { + return manage.NewDefaultMQProducer(producerGroup, producerConfig) } diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 1fcb3b8f2..2456bb7f6 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -30,21 +30,21 @@ func main() { testConsumerGroup = "TestConsumerGroup" ) // init rocketMQClientInstance - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) // 1.init rocketMQConsumer // 2.subscribe topic and register our function to message listener // 3.register it - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "*") - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) // call your function successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index 03c1f168a..5b257447d 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -30,16 +30,16 @@ func main() { testProducerGroup = "TestProducerGroup" ) // init rocketMQClientInstance - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) // init rocketMQProducer and register it - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) // start rocketMQ client instance rocketMQClientInstance.Start() //start send test message - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello World")} result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 27c5140e1..844824877 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -31,24 +31,24 @@ func main() { testConsumerGroup = "TestConsumerGroup" ) // init rocketMQClientInstance - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) // init rocketMQProducer and register it - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) // 1.init rocketMQConsumer // 2.subscribe topic and register our function to message listener // 3.register it - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "*") - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) // call your function successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) @@ -57,7 +57,7 @@ func main() { //start send test message for { - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello World")} result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely.go b/rocketmq-go/kernel/allocate_message/allocate_message_averagely.go similarity index 100% rename from rocketmq-go/service/allocate_message/allocate_message_averagely.go rename to rocketmq-go/kernel/allocate_message/allocate_message_averagely.go diff --git a/rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go b/rocketmq-go/kernel/allocate_message/allocate_message_averagely_by_circle.go similarity index 100% rename from rocketmq-go/service/allocate_message/allocate_message_averagely_by_circle.go rename to rocketmq-go/kernel/allocate_message/allocate_message_averagely_by_circle.go diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_config.go b/rocketmq-go/kernel/allocate_message/allocate_message_by_config.go similarity index 100% rename from rocketmq-go/service/allocate_message/allocate_message_by_config.go rename to rocketmq-go/kernel/allocate_message/allocate_message_by_config.go diff --git a/rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go b/rocketmq-go/kernel/allocate_message/allocate_message_by_machine_room.go similarity index 100% rename from rocketmq-go/service/allocate_message/allocate_message_by_machine_room.go rename to rocketmq-go/kernel/allocate_message/allocate_message_by_machine_room.go diff --git a/rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go b/rocketmq-go/kernel/allocate_message/allocate_message_queue_strategy.go similarity index 100% rename from rocketmq-go/service/allocate_message/allocate_message_queue_strategy.go rename to rocketmq-go/kernel/allocate_message/allocate_message_queue_strategy.go diff --git a/rocketmq-go/service/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go similarity index 76% rename from rocketmq-go/service/consume_message_service.go rename to rocketmq-go/kernel/consume_message_service.go index 9963c615c..6fe02307e 100644 --- a/rocketmq-go/service/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" @@ -26,11 +26,11 @@ import ( ) type ConsumeMessageService interface { - Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) - SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, + Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) + SubmitConsumeRequest(msgs []rocketmqm.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) - SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) - ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) + SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) + ConsumeMessageDirectly(messageExt *rocketmqm.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) } type ConsumeMessageConcurrentlyServiceImpl struct { @@ -38,7 +38,7 @@ type ConsumeMessageConcurrentlyServiceImpl struct { messageListener model.MessageListener sendMessageBackProducerService SendMessageBackProducerService //for send retry Message offsetStore OffsetStore - consumerConfig *rocketmq_api_model.RocketMqConsumerConfig + consumerConfig *rocketmqm.MqConsumerConfig } func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListener) (consumeService ConsumeMessageService) { @@ -46,14 +46,14 @@ func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListe return } -func (c *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { +func (c *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { c.consumerGroup = consumerGroup c.offsetStore = offsetStore c.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) c.consumerConfig = consumerConfig } -func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmq_api_model.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmqm.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i @@ -72,18 +72,18 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rock return } -func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) { err = c.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } -func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmq_api_model.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { +func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmqm.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { start := util.CurrentTimeMillisInt64() - consumeResult := c.messageListener([]rocketmq_api_model.MessageExt{*messageExt}) + consumeResult := c.messageListener([]rocketmqm.MessageExt{*messageExt}) consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false consumeMessageDirectlyResult.SpentTimeMills = util.CurrentTimeMillisInt64() - start - if consumeResult.ConsumeConcurrentlyStatus == rocketmq_api_model.CONSUME_SUCCESS && consumeResult.AckIndex >= 0 { + if consumeResult.ConsumeConcurrentlyStatus == rocketmqm.CONSUME_SUCCESS && consumeResult.AckIndex >= 0 { consumeMessageDirectlyResult.ConsumeResult = model.CR_SUCCESS } else { consumeMessageDirectlyResult.ConsumeResult = model.CR_THROW_EXCEPTION @@ -91,7 +91,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageEx return } -func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmq_api_model.ConsumeConcurrentlyResult, msgs []rocketmq_api_model.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []rocketmqm.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return @@ -100,7 +100,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock return } ackIndex := result.AckIndex - if rocketmq_api_model.CONSUME_SUCCESS == result.ConsumeConcurrentlyStatus { + if rocketmqm.CONSUME_SUCCESS == result.ConsumeConcurrentlyStatus { if ackIndex >= len(msgs) { ackIndex = len(msgs) - 1 } else { @@ -109,8 +109,8 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } } } - var failedMessages []rocketmq_api_model.MessageExt - successMessages := []rocketmq_api_model.MessageExt{} + var failedMessages []rocketmqm.MessageExt + successMessages := []rocketmqm.MessageExt{} if ackIndex >= 0 { successMessages = msgs[:ackIndex+1] } @@ -133,7 +133,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } -func transformMessageToConsume(consumerGroup string, msgs []rocketmq_api_model.MessageExt) []rocketmq_api_model.MessageExt { +func transformMessageToConsume(consumerGroup string, msgs []rocketmqm.MessageExt) []rocketmqm.MessageExt { retryTopicName := constant.RETRY_GROUP_TOPIC_PREFIX + consumerGroup for _, msg := range msgs { diff --git a/rocketmq-go/service/mq_client.go b/rocketmq-go/kernel/mq_client.go similarity index 98% rename from rocketmq-go/service/mq_client.go rename to rocketmq-go/kernel/mq_client.go index dc829a16f..6ed480677 100644 --- a/rocketmq-go/service/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "encoding/json" @@ -70,7 +70,7 @@ type MqClientImpl struct { PullRequestQueue chan *model.PullRequest } -func MqClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { +func MqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} mqClientImpl.ClientId = buildMqClientImplId() mqClientImpl.TopicRouteTable = util.New() // make(map[string]*model.TopicRouteData) diff --git a/rocketmq-go/service/mq_fault_strategy.go b/rocketmq-go/kernel/mq_fault_strategy.go similarity index 99% rename from rocketmq-go/service/mq_fault_strategy.go rename to rocketmq-go/kernel/mq_fault_strategy.go index 9e29adb65..abc12da3d 100644 --- a/rocketmq-go/service/mq_fault_strategy.go +++ b/rocketmq-go/kernel/mq_fault_strategy.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "errors" diff --git a/rocketmq-go/service/offset_store.go b/rocketmq-go/kernel/offset_store.go similarity index 99% rename from rocketmq-go/service/offset_store.go rename to rocketmq-go/kernel/offset_store.go index 40b51ab0f..665d933fd 100644 --- a/rocketmq-go/service/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "errors" diff --git a/rocketmq-go/service/producer_service.go b/rocketmq-go/kernel/producer_service.go similarity index 86% rename from rocketmq-go/service/producer_service.go rename to rocketmq-go/kernel/producer_service.go index 6e5190b43..835e68fbb 100644 --- a/rocketmq-go/service/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "errors" @@ -30,17 +30,17 @@ import ( type ProducerService interface { CheckConfig() (err error) - SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + SendDefaultImpl(message *rocketmqm.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } type DefaultProducerService struct { producerGroup string - producerConfig *rocketmq_api_model.RocketMqProducerConfig + producerConfig *rocketmqm.MqProducerConfig mqClient RocketMqClient mqFaultStrategy MQFaultStrategy } -func NewDefaultProducerService(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { +func NewDefaultProducerService(producerGroup string, producerConfig *rocketmqm.MqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { defaultProducerService = &DefaultProducerService{ mqClient: mqClient, producerGroup: producerGroup, @@ -54,7 +54,7 @@ func (d *DefaultProducerService) CheckConfig() (err error) { return } -func (d *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) @@ -76,7 +76,7 @@ func (d *DefaultProducerService) SendDefaultImpl(message *rocketmq_api_model.Mes return } -func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -87,7 +87,7 @@ func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, s sendResult, err = processSendResponse(brokerAddr, message, response) return } -func processSendResponse(brokerName string, message *rocketmq_api_model.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { +func processSendResponse(brokerName string, message *rocketmqm.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { sendResult = &model.SendResult{} switch response.Code { case remoting.FLUSH_DISK_TIMEOUT: @@ -133,7 +133,7 @@ func processSendResponse(brokerName string, message *rocketmq_api_model.Message, return } -func (d *DefaultProducerService) checkMessage(message *rocketmq_api_model.Message) (err error) { +func (d *DefaultProducerService) checkMessage(message *rocketmqm.Message) (err error) { if message == nil { err = errors.New("message is nil") return @@ -167,7 +167,7 @@ func (d *DefaultProducerService) checkMessage(message *rocketmq_api_model.Messag return } -func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq_api_model.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int messageQueue model.MessageQueue @@ -191,7 +191,7 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmq_ap return } -func (d *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Message, messageQueue model.MessageQueue, +func (d *DefaultProducerService) doSendMessage(message *rocketmqm.Message, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -230,7 +230,7 @@ func (d *DefaultProducerService) doSendMessage(message *rocketmq_api_model.Messa return } -func (d *DefaultProducerService) tryToCompressMessage(message *rocketmq_api_model.Message) (compressedFlag int, err error) { +func (d *DefaultProducerService) tryToCompressMessage(message *rocketmqm.Message) (compressedFlag int, err error) { if len(message.Body) < d.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return diff --git a/rocketmq-go/service/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go similarity index 87% rename from rocketmq-go/service/producer_service_for_send_back.go rename to rocketmq-go/kernel/producer_service_for_send_back.go index d1a8a4458..4e9171a90 100644 --- a/rocketmq-go/service/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "encoding/json" @@ -28,19 +28,19 @@ import ( ) type SendMessageBackProducerService interface { - SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) - InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) + SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) + InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) } type SendMessageBackProducerServiceImpl struct { mqClient RocketMqClient defaultProducerService *DefaultProducerService // one namesvr only one consumerGroup string - consumerConfig *rocketmq_api_model.RocketMqConsumerConfig //one mq group have one + consumerConfig *rocketmqm.MqConsumerConfig //one mq group have one } // send to original broker,if fail send a new retry message -func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmq_api_model.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) { glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) err = s.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) if err == nil { @@ -51,8 +51,8 @@ func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketm return } -func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmq_api_model.MessageExt) error { - retryMessage := &rocketmq_api_model.Message{} +func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmqm.MessageExt) error { + retryMessage := &rocketmqm.Message{} originMessageId := messageExt.GetOriginMessageId() retryMessage.Properties = messageExt.Properties retryMessage.SetOriginMessageId(originMessageId) @@ -76,14 +76,14 @@ func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *ro } -func (s *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) { +func (s *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { s.mqClient = mqClient s.consumerGroup = consumerGroup s.defaultProducerService = defaultProducerService s.consumerConfig = consumerConfig } -func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmq_api_model.MessageExt, delayLayLevel int) (err error) { +func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmqm.MessageExt, delayLayLevel int) (err error) { if len(brokerName) == 0 { err = errors.New("broker can't be empty") glog.Error(err) diff --git a/rocketmq-go/service/rebalance.go b/rocketmq-go/kernel/rebalance.go similarity index 96% rename from rocketmq-go/service/rebalance.go rename to rocketmq-go/kernel/rebalance.go index 45308e340..2458f17f6 100644 --- a/rocketmq-go/service/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service +package kernel import ( "encoding/json" @@ -25,7 +25,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service/allocate_message" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate_message" "github.com/golang/glog" "sort" "strings" @@ -45,7 +45,7 @@ type Rebalance struct { processQueueTableLock sync.RWMutex mutex sync.Mutex offsetStore OffsetStore - consumerConfig *rocketmq_api_model.RocketMqConsumerConfig + consumerConfig *rocketmqm.MqConsumerConfig } func (r *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { @@ -100,7 +100,7 @@ func (r *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { } -func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) *Rebalance { +func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmqm.MqConsumerConfig) *Rebalance { subscriptionInner := make(map[string]*model.SubscriptionData) for topic, subExpression := range subscription { subData := &model.SubscriptionData{ @@ -227,7 +227,7 @@ func (r *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { var result int64 = -1 lastOffset := r.offsetStore.ReadOffset(mq, READ_FROM_STORE) switch r.consumerConfig.ConsumeFromWhere { - case rocketmq_api_model.CONSUME_FROM_LAST_OFFSET: + case rocketmqm.CONSUME_FROM_LAST_OFFSET: if lastOffset >= 0 { result = lastOffset } else { @@ -238,14 +238,14 @@ func (r *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { } } break - case rocketmq_api_model.CONSUME_FROM_FIRST_OFFSET: + case rocketmqm.CONSUME_FROM_FIRST_OFFSET: if lastOffset >= 0 { result = lastOffset } else { result = 0 // use the begin offset } break - case rocketmq_api_model.CONSUME_FROM_TIMESTAMP: + case rocketmqm.CONSUME_FROM_TIMESTAMP: if lastOffset >= 0 { result = lastOffset } else { diff --git a/rocketmq-go/manage/clean_expire_msg_controller.go b/rocketmq-go/manage/clean_expire_msg_controller.go index ceed735c8..3a2f9eef7 100644 --- a/rocketmq-go/manage/clean_expire_msg_controller.go +++ b/rocketmq-go/manage/clean_expire_msg_controller.go @@ -15,19 +15,19 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "time" ) type CleanExpireMsgController struct { - mqClient service.RocketMqClient + mqClient kernel.RocketMqClient clientFactory *ClientFactory } -func NewCleanExpireMsgController(mqClient service.RocketMqClient, clientFactory *ClientFactory) *CleanExpireMsgController { +func NewCleanExpireMsgController(mqClient kernel.RocketMqClient, clientFactory *ClientFactory) *CleanExpireMsgController { return &CleanExpireMsgController{ mqClient: mqClient, clientFactory: clientFactory, diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index c7fb31e24..eafe62ba3 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( "encoding/json" @@ -26,7 +26,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" "github.com/golang/glog" "strings" @@ -40,19 +40,19 @@ type MqClientManager struct { clientFactory *ClientFactory NamesrvLock sync.Mutex HeartBeatLock sync.Mutex - mqClient service.RocketMqClient + mqClient kernel.RocketMqClient ServiceState int pullMessageController *PullMessageController cleanExpireMsgController *CleanExpireMsgController rebalanceControllr *RebalanceController - defaultProducerService *service.DefaultProducerService + defaultProducerService *kernel.DefaultProducerService } -func MqClientManagerInit(clientConfig *rocketmq_api_model.MqClientConfig) (rocketMqManager *MqClientManager) { +func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManager *MqClientManager) { rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() rocketMqManager.clientFactory = ClientFactoryInit() - rocketMqManager.mqClient = service.MqClientInit(clientConfig, rocketMqManager.InitClientRequestProcessor()) // todo todo todo + rocketMqManager.mqClient = kernel.MqClientInit(clientConfig, rocketMqManager.InitClientRequestProcessor()) // todo todo todo rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.cleanExpireMsgController = NewCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) @@ -142,7 +142,7 @@ func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor r return } func (m *MqClientManager) RegistProducer(producer *DefaultMQProducer) { - producer.producerService = service.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) + producer.producerService = kernel.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) m.clientFactory.ProducerTable[producer.producerGroup] = producer return } @@ -157,12 +157,12 @@ func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable m } func (m *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { if m.defaultProducerService == nil { - m.defaultProducerService = service.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmq_api_model.NewProducerConfig(), m.mqClient) + m.defaultProducerService = kernel.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) } consumer.mqClient = m.mqClient - consumer.offsetStore = service.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) + consumer.offsetStore = kernel.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) m.clientFactory.ConsumerTable[consumer.consumerGroup] = consumer - consumer.rebalance = service.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) + consumer.rebalance = kernel.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) fmt.Println(consumer.consumeMessageService) diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index b490d6299..07a7c4388 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -15,23 +15,23 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" ) type DefaultMQProducer struct { producerGroup string - ProducerConfig *rocketmq_api_model.RocketMqProducerConfig + ProducerConfig *rocketmqm.MqProducerConfig - producerService service.ProducerService + producerService kernel.ProducerService } -func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmq_api_model.RocketMqProducerConfig) (rocketMQProducer *DefaultMQProducer) { +func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (rocketMQProducer *DefaultMQProducer) { rocketMQProducer = &DefaultMQProducer{ producerGroup: producerGroup, ProducerConfig: producerConfig, @@ -39,11 +39,11 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmq_api_mod return } -func (self *DefaultMQProducer) Send(message *rocketmq_api_model.Message) (sendResult *model.SendResult, err error) { +func (self *DefaultMQProducer) Send(message *rocketmqm.Message) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) return } -func (self *DefaultMQProducer) SendWithTimeout(message *rocketmq_api_model.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (self *DefaultMQProducer) SendWithTimeout(message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) return } diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index c012818e6..65a7d6a39 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -15,12 +15,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strings" @@ -34,15 +34,15 @@ type DefaultMQPushConsumer struct { unitMode bool subscription map[string]string //topic|subExpression subscriptionTag map[string][]string // we use it filter again - offsetStore service.OffsetStore - mqClient service.RocketMqClient - rebalance *service.Rebalance + offsetStore kernel.OffsetStore + mqClient kernel.RocketMqClient + rebalance *kernel.Rebalance pause bool - consumeMessageService service.ConsumeMessageService - ConsumerConfig *rocketmq_api_model.RocketMqConsumerConfig + consumeMessageService kernel.ConsumeMessageService + ConsumerConfig *rocketmqm.MqConsumerConfig } -func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmq_api_model.RocketMqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { +func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { defaultMQPushConsumer = &DefaultMQPushConsumer{ consumerGroup: consumerGroup, consumeType: "CONSUME_PASSIVELY", @@ -74,7 +74,7 @@ func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { } func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { - d.consumeMessageService = service.NewConsumeMessageConcurrentlyServiceImpl(messageListener) + d.consumeMessageService = kernel.NewConsumeMessageConcurrentlyServiceImpl(messageListener) } func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index 6202d990e..7766ccff0 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( "bytes" @@ -26,7 +26,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/service" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strconv" @@ -34,11 +34,11 @@ import ( ) type PullMessageController struct { - mqClient service.RocketMqClient + mqClient kernel.RocketMqClient clientFactory *ClientFactory } -func NewPullMessageController(mqClient service.RocketMqClient, clientFactory *ClientFactory) *PullMessageController { +func NewPullMessageController(mqClient kernel.RocketMqClient, clientFactory *ClientFactory) *PullMessageController { return &PullMessageController{ mqClient: mqClient, clientFactory: clientFactory, @@ -83,7 +83,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { p.pullMessageLater(pullRequest, delayPullTime) return } - commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, service.READ_FROM_MEMORY) + commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, kernel.READ_FROM_MEMORY) subscriptionData, ok := defaultMQPullConsumer.rebalance.SubscriptionInner[pullRequest.MessageQueue.Topic] if !ok { @@ -204,12 +204,12 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } -func FilterMessageAgainByTags(msgExts []rocketmq_api_model.MessageExt, subscriptionTagList []string) (result []rocketmq_api_model.MessageExt) { +func FilterMessageAgainByTags(msgExts []rocketmqm.MessageExt, subscriptionTagList []string) (result []rocketmqm.MessageExt) { result = msgExts if len(subscriptionTagList) == 0 { return } - result = []rocketmq_api_model.MessageExt{} + result = []rocketmqm.MessageExt{} for _, msg := range msgExts { for _, tag := range subscriptionTagList { if tag == msg.GetTag() { @@ -229,7 +229,7 @@ func (p *PullMessageController) consumerPullMessageAsync(brokerName string, requ } } -func DecodeMessage(data []byte) []rocketmq_api_model.MessageExt { +func DecodeMessage(data []byte) []rocketmqm.MessageExt { buf := bytes.NewBuffer(data) var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 @@ -239,9 +239,9 @@ func DecodeMessage(data []byte) []rocketmq_api_model.MessageExt { var propertiesmap = make(map[string]string) - msgs := []rocketmq_api_model.MessageExt{} + msgs := []rocketmqm.MessageExt{} for buf.Len() > 0 { - msg := rocketmq_api_model.MessageExt{Message: &rocketmq_api_model.Message{}} + msg := rocketmqm.MessageExt{Message: &rocketmqm.Message{}} binary.Read(buf, binary.BigEndian, &storeSize) binary.Read(buf, binary.BigEndian, &magicCode) binary.Read(buf, binary.BigEndian, &bodyCRC) diff --git a/rocketmq-go/manage/rebalance_controller.go b/rocketmq-go/manage/rebalance_controller.go index 3855a1080..b1f9d864f 100644 --- a/rocketmq-go/manage/rebalance_controller.go +++ b/rocketmq-go/manage/rebalance_controller.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage type RebalanceController struct { clientFactory *ClientFactory diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/manage/tasks.go index 227f98cd1..f5b6f43a3 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/manage/tasks.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmq +package manage import ( "math/rand" diff --git a/rocketmq-go/model/heart_beat.go b/rocketmq-go/model/heart_beat.go index 4a2ff1010..87b49eca8 100644 --- a/rocketmq-go/model/heart_beat.go +++ b/rocketmq-go/model/heart_beat.go @@ -23,7 +23,7 @@ type ConsumerData struct { GroupName string ConsumeType string MessageModel string - ConsumeFromWhere rocketmq_api_model.ConsumeFromWhere + ConsumeFromWhere rocketmqm.ConsumeFromWhere SubscriptionDataSet []*SubscriptionData UnitMode bool } diff --git a/rocketmq-go/model/message_listener.go b/rocketmq-go/model/message_listener.go index b4ab7f7ed..f7e415de9 100644 --- a/rocketmq-go/model/message_listener.go +++ b/rocketmq-go/model/message_listener.go @@ -19,4 +19,4 @@ package model import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" -type MessageListener func(msgs []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult +type MessageListener func(msgs []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index c565c7205..68dcf714c 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -106,7 +106,7 @@ func (p *ProcessQueue) DeleteExpireMsg(queueOffset int) { } } -func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq_api_model.MessageExt) { +func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmqm.MessageExt) { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() key, value := p.msgTreeMap.Min() @@ -115,7 +115,7 @@ func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq } offset = key.(int) - message := value.(rocketmq_api_model.MessageExt) + message := value.(rocketmqm.MessageExt) messagePoint = &message return } @@ -139,7 +139,7 @@ func (p *ProcessQueue) GetMaxSpan() int { return maxOffset - minOffset } -func (p *ProcessQueue) RemoveMessage(msgs []rocketmq_api_model.MessageExt) (offset int64) { +func (p *ProcessQueue) RemoveMessage(msgs []rocketmqm.MessageExt) (offset int64) { now := time.Now() offset = -1 defer p.lockTreeMap.Unlock() @@ -160,7 +160,7 @@ func (p *ProcessQueue) RemoveMessage(msgs []rocketmq_api_model.MessageExt) (offs return } -func (p *ProcessQueue) PutMessage(msgs []rocketmq_api_model.MessageExt) (dispatchToConsume bool) { +func (p *ProcessQueue) PutMessage(msgs []rocketmqm.MessageExt) (dispatchToConsume bool) { dispatchToConsume = false msgsLen := len(msgs) if msgsLen == 0 { diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 274d203da..6e533d869 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -39,7 +39,7 @@ type RemotingClient interface { } type DefaultRemotingClient struct { clientId string - clientConfig *rocketmq_api_model.MqClientConfig + clientConfig *rocketmqm.MqClientConfig connTable map[string]net.Conn connTableLock sync.RWMutex @@ -54,7 +54,7 @@ type DefaultRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } -func RemotingClientInit(clientConfig *rocketmq_api_model.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { +func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { client = &DefaultRemotingClient{} client.connTable = map[string]net.Conn{} client.responseTable = util.New() diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index ee0efd6e6..4b57bc61e 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -25,7 +25,7 @@ import ( ) type SerializerHandler struct { - serializeType rocketmq_api_model.SerializeType + serializeType rocketmqm.SerializeType serializer Serializer //which serializer this client use, depend on constant.USE_HEADER_SERIALIZE_TYPE } @@ -37,14 +37,14 @@ type Serializer interface { var JSON_SERIALIZER = &JsonSerializer{} var ROCKETMQ_SERIALIZER = &RocketMqSerializer{} -func NewSerializerHandler(serializeType rocketmq_api_model.SerializeType) SerializerHandler { +func NewSerializerHandler(serializeType rocketmqm.SerializeType) SerializerHandler { serializerHandler := SerializerHandler{serializeType: serializeType} switch serializeType { - case rocketmq_api_model.JSON_SERIALIZE: + case rocketmqm.JSON_SERIALIZE: serializerHandler.serializer = JSON_SERIALIZER break - case rocketmq_api_model.ROCKETMQ_SERIALIZE: + case rocketmqm.ROCKETMQ_SERIALIZE: serializerHandler.serializer = ROCKETMQ_SERIALIZER break default: @@ -68,11 +68,11 @@ func (s *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { func (s *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { var serializer Serializer - switch rocketmq_api_model.SerializeType(headerSerializableType) { - case rocketmq_api_model.JSON_SERIALIZE: + switch rocketmqm.SerializeType(headerSerializableType) { + case rocketmqm.JSON_SERIALIZE: serializer = JSON_SERIALIZER break - case rocketmq_api_model.ROCKETMQ_SERIALIZE: + case rocketmqm.ROCKETMQ_SERIALIZE: serializer = ROCKETMQ_SERIALIZER break default: diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index 8878b4415..ea82edd92 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -37,13 +37,13 @@ func main() { testMessageBody = "RetryTestMessageBody" consumeTime = 0 ) - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) fmt.Println(tag) - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, message := range messageList { if string(message.Body) != testMessageBody { @@ -59,11 +59,11 @@ func main() { chResult <- true successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(testMessageBody)} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(testMessageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index 8d62affdf..e4347de61 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -35,12 +35,12 @@ func main() { var messageId string var startTime time.Time chResult := make(chan bool, 1) - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { endTime := time.Now() @@ -55,12 +55,12 @@ func main() { successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() <-time.After(time.Second * 30) // wait - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello world")} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello world")} message.SetTag(tag) message.SetDelayTimeLevel(3) // cost 15 second result, err := producer.Send(message) diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index bdcea6d7d..a7d5796c3 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -35,14 +35,14 @@ func main() { messageCount = 100 ) chResult := make(chan bool, messageCount) - mqClientConfig := rocketmq_api_model.NewMqClientConfig(nameServerAddress) - mqClientConfig.ClientSerializeType = rocketmq_api_model.ROCKETMQ_SERIALIZE - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + mqClientConfig := rocketmqm.NewMqClientConfig(nameServerAddress) + mqClientConfig.ClientSerializeType = rocketmqm.ROCKETMQ_SERIALIZE + rocketMQClientInstance := rocketmq.InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.GetTag() == tag && messageBody == string(messageBody) { @@ -51,12 +51,12 @@ func main() { successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < messageCount; i++ { - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(messageBody)} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(messageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index 3e9147562..c401aecee 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -38,14 +38,14 @@ func main() { bigMessageBody += bigMessageBody } //bigMessageBody len will be 720896,it will be compressed - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - producerConfig := rocketmq_api_model.NewProducerConfig() + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + producerConfig := rocketmqm.NewProducerConfig() producerConfig.CompressMsgBodyOverHowMuch = 500 - var producer = rocketmq_api.NewDefaultMQProducerWithCustomConfig(testProducerGroup, producerConfig) + var producer = rocketmq.NewDefaultMQProducerWithCustomConfig(testProducerGroup, producerConfig) rocketMQClientInstance.RegisterProducer(producer) - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "compress_message_test") - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.SysFlag&constant.CompressedFlag != constant.CompressedFlag { @@ -59,11 +59,11 @@ func main() { } chResult <- true - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte(bigMessageBody)} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(bigMessageBody)} message.SetTag("compress_message_test") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index 1218aded6..a88ae2dfb 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -34,12 +34,12 @@ func main() { testProducerGroup = "TestTagProducerGroup" testConsumerGroup = "TestTagConsumerGroup" ) - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + var producer = rocketmq.NewDefaultMQProducer(testProducerGroup) rocketMQClientInstance.RegisterProducer(producer) - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "tag0 || tag2||tag4") - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.GetTag() != "tag0" && msg.GetTag() != "tag2" && msg.GetTag() != "tag4" { @@ -50,12 +50,12 @@ func main() { successIndex = index } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < 5; i++ { - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello world")} + var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello world")} message.SetTag("tag" + util.IntToString(i)) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) From 5319eba0baa412074719c9ba512f33c96ab2a2c6 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 17 Aug 2017 22:21:25 +0800 Subject: [PATCH 37/88] change package name for golint --- rocketmq-go/api/model/consume_concurrently_result.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go index 1a0d65be6..f22808167 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -20,7 +20,13 @@ package rocketmqm type ConsumeStatus int const ( + /** + consume success + */ CONSUME_SUCCESS ConsumeStatus = iota + /** + consume fail + */ RECONSUME_LATER ) From b9b0d60780315244cddcebed707ce55afa1b3a49 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 17 Aug 2017 22:56:34 +0800 Subject: [PATCH 38/88] make api easy to use --- .../api/model/consume_concurrently_result.go | 8 +- rocketmq-go/api/model/message.go | 65 +++-- rocketmq-go/api/model/message_ext.go | 2 +- rocketmq-go/api/model/mq_consumer_config.go | 2 +- rocketmq-go/api/rocketmq_producer.go | 4 +- rocketmq-go/example/simple_producer.go | 2 +- .../example/simple_producer_consumer.go | 2 +- rocketmq-go/kernel/consume_message_service.go | 6 +- rocketmq-go/kernel/producer_service.go | 17 +- .../kernel/producer_service_for_send_back.go | 6 +- rocketmq-go/manage/mq_producer.go | 4 +- rocketmq-go/manage/pull_message_controller.go | 4 +- rocketmq-go/model/message/message.go | 266 ------------------ .../test/test_consume_message_retry.go | 4 +- .../test_delay_message_send_and_receive.go | 2 +- ...etmq_serialize_message_send_and_receive.go | 4 +- .../test_send_and_receive_compress_message.go | 2 +- .../test/test_send_and_receive_tag_message.go | 8 +- 18 files changed, 83 insertions(+), 325 deletions(-) delete mode 100644 rocketmq-go/model/message/message.go diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/api/model/consume_concurrently_result.go index f22808167..61a460b4f 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/api/model/consume_concurrently_result.go @@ -20,13 +20,9 @@ package rocketmqm type ConsumeStatus int const ( - /** - consume success - */ + //consume success CONSUME_SUCCESS ConsumeStatus = iota - /** - consume fail - */ + //consume fail RECONSUME_LATER ) diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index cd9de627a..2aa29047c 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -20,50 +20,69 @@ package rocketmqm import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "strconv" "strings" ) -type Message struct { +type Message interface { + SetTag(tag string) + Tag() (tag string) + SetKeys(keys []string) + SetDelayTimeLevel(delayTimeLevel int) +} + +func NewMessage() (message Message) { + message = &MessageImpl{} + return +} + +type MessageImpl struct { Topic string Flag int Properties map[string]string Body []byte } -func (m *Message) SetTag(tag string) { +//set message tag +func (m *MessageImpl) SetTag(tag string) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_TAGS] = tag } -func (m *Message) GetTag() (tag string) { + +//get message tag from Properties +func (m *MessageImpl) Tag() (tag string) { if m.Properties != nil { tag = m.Properties[constant.PROPERTY_TAGS] } return } -func (m *Message) SetKeys(keys []string) { +//set message key +func (m *MessageImpl) SetKeys(keys []string) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") } -func (m *Message) SetDelayTimeLevel(delayTimeLevel int) { +//SetDelayTimeLevel +func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) } -func (m *Message) SetWaitStoreMsgOK(waitStoreMsgOK bool) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) -} -func (m *Message) GeneratorMsgUniqueKey() { + +////SetWaitStoreMsgOK +//func (m *MessageImpl) SetWaitStoreMsgOK(waitStoreMsgOK bool) { +// if m.Properties == nil { +// m.Properties = make(map[string]string) +// } +// m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) +//} +//GeneratorMsgUniqueKey only use by system +func (m *MessageImpl) GeneratorMsgUniqueKey() { if m.Properties == nil { m.Properties = make(map[string]string) } @@ -73,6 +92,7 @@ func (m *Message) GeneratorMsgUniqueKey() { m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() } +//GetMsgUniqueKey only use by system func (m *MessageExt) GetMsgUniqueKey() string { if m.Properties != nil { originMessageId := m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] @@ -83,27 +103,32 @@ func (m *MessageExt) GetMsgUniqueKey() string { return m.MsgId } -func (m *Message) SetOriginMessageId(messageId string) { +//only use by system +func (m *MessageImpl) SetOriginMessageId(messageId string) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId } -func (m *Message) SetRetryTopic(retryTopic string) { +//only use by system +func (m *MessageImpl) SetRetryTopic(retryTopic string) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic } -func (m *Message) SetReconsumeTime(reConsumeTime int) { + +//only use by system +func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) } -func (m *Message) GetReconsumeTimes() (reConsumeTime int) { +//only use by system +func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { reConsumeTime = 0 if m.Properties != nil { reConsumeTimeStr := m.Properties[constant.PROPERTY_RECONSUME_TIME] @@ -114,14 +139,16 @@ func (m *Message) GetReconsumeTimes() (reConsumeTime int) { return } -func (m *Message) SetMaxReconsumeTimes(maxConsumeTime int) { +//only use by system +func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { if m.Properties == nil { m.Properties = make(map[string]string) } m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) } -func (m *Message) GetMaxReconsumeTimes() (maxConsumeTime int) { +//only use by system +func (m *MessageImpl) GetMaxReconsumeTimes() (maxConsumeTime int) { maxConsumeTime = 0 if m.Properties != nil { reConsumeTimeStr := m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 1e481f167..26e89efcb 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -24,7 +24,7 @@ import ( ) type MessageExt struct { - *Message + *MessageImpl QueueId int32 StoreSize int32 QueueOffset int64 diff --git a/rocketmq-go/api/model/mq_consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go index a0ec836de..db17e1ee6 100644 --- a/rocketmq-go/api/model/mq_consumer_config.go +++ b/rocketmq-go/api/model/mq_consumer_config.go @@ -57,7 +57,7 @@ type MqConsumerConfig struct { */ PullThresholdForQueue int //= 1000; /** - * Message pull Interval + * MessageImpl pull Interval */ PullInterval int64 //= 0; /** diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 1883997b1..2a35edfc7 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -24,8 +24,8 @@ import ( ) type MQProducer interface { - Send(message *rocketmqm.Message) (sendResult *model.SendResult, err error) - SendWithTimeout(message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) + Send(message *rocketmqm.MessageImpl) (sendResult *model.SendResult, err error) + SendWithTimeout(message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) } func NewDefaultMQProducer(producerGroup string) (r MQProducer) { diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index 5b257447d..f80b9f0af 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -39,7 +39,7 @@ func main() { rocketMQClientInstance.Start() //start send test message - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello World")} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello World")} result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 844824877..c85839f89 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -57,7 +57,7 @@ func main() { //start send test message for { - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello World")} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello World")} result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 6fe02307e..25440f851 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -36,7 +36,7 @@ type ConsumeMessageService interface { type ConsumeMessageConcurrentlyServiceImpl struct { consumerGroup string messageListener model.MessageListener - sendMessageBackProducerService SendMessageBackProducerService //for send retry Message + sendMessageBackProducerService SendMessageBackProducerService //for send retry MessageImpl offsetStore OffsetStore consumerConfig *rocketmqm.MqConsumerConfig } @@ -138,10 +138,10 @@ func transformMessageToConsume(consumerGroup string, msgs []rocketmqm.MessageExt for _, msg := range msgs { //reset retry topic name - if msg.Message.Topic == retryTopicName { + if msg.MessageImpl.Topic == retryTopicName { retryTopic := msg.Properties[constant.PROPERTY_RETRY_TOPIC] if len(retryTopic) > 0 { - msg.Message.Topic = retryTopic + msg.MessageImpl.Topic = retryTopic } } //set consume start time diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 835e68fbb..d6d1fd3bb 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -30,7 +30,7 @@ import ( type ProducerService interface { CheckConfig() (err error) - SendDefaultImpl(message *rocketmqm.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + SendDefaultImpl(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } type DefaultProducerService struct { @@ -54,7 +54,7 @@ func (d *DefaultProducerService) CheckConfig() (err error) { return } -func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.Message, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) @@ -76,7 +76,7 @@ func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.Message, com return } -func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -87,7 +87,7 @@ func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, s sendResult, err = processSendResponse(brokerAddr, message, response) return } -func processSendResponse(brokerName string, message *rocketmqm.Message, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { +func processSendResponse(brokerName string, message *rocketmqm.MessageImpl, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { sendResult = &model.SendResult{} switch response.Code { case remoting.FLUSH_DISK_TIMEOUT: @@ -133,7 +133,7 @@ func processSendResponse(brokerName string, message *rocketmqm.Message, response return } -func (d *DefaultProducerService) checkMessage(message *rocketmqm.Message) (err error) { +func (d *DefaultProducerService) checkMessage(message *rocketmqm.MessageImpl) (err error) { if message == nil { err = errors.New("message is nil") return @@ -167,7 +167,7 @@ func (d *DefaultProducerService) checkMessage(message *rocketmqm.Message) (err e return } -func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.Message, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int messageQueue model.MessageQueue @@ -191,7 +191,8 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.M return } -func (d *DefaultProducerService) doSendMessage(message *rocketmqm.Message, messageQueue model.MessageQueue, + +func (d *DefaultProducerService) doSendMessage(message *rocketmqm.MessageImpl, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -230,7 +231,7 @@ func (d *DefaultProducerService) doSendMessage(message *rocketmqm.Message, messa return } -func (d *DefaultProducerService) tryToCompressMessage(message *rocketmqm.Message) (compressedFlag int, err error) { +func (d *DefaultProducerService) tryToCompressMessage(message *rocketmqm.MessageImpl) (compressedFlag int, err error) { if len(message.Body) < d.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 4e9171a90..72bf7c2fe 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -52,7 +52,7 @@ func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketm } func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmqm.MessageExt) error { - retryMessage := &rocketmqm.Message{} + retryMessage := &rocketmqm.MessageImpl{} originMessageId := messageExt.GetOriginMessageId() retryMessage.Properties = messageExt.Properties retryMessage.SetOriginMessageId(originMessageId) @@ -93,7 +93,7 @@ func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName sendMsgBackHeader := &header.ConsumerSendMsgBackRequestHeader{ Offset: messageExt.CommitLogOffset, Group: s.consumerGroup, - DelayLevel: 0, //Message consume retry strategy
-1,no retry,put into DLQ directly
0,broker control retry frequency
>0,client control retry frequency + DelayLevel: 0, //MessageImpl consume retry strategy
-1,no retry,put into DLQ directly
0,broker control retry frequency
>0,client control retry frequency OriginMsgId: messageExt.MsgId, OriginTopic: messageExt.Topic, UnitMode: false, @@ -107,7 +107,7 @@ func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName } if response == nil || response.Code != remoting.SUCCESS { glog.Error("sendMsgBackRemarkError", response.Remark) - err = errors.New("send Message back error") + err = errors.New("send MessageImpl back error") } return } diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index 07a7c4388..a10be5a7b 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -39,11 +39,11 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProd return } -func (self *DefaultMQProducer) Send(message *rocketmqm.Message) (sendResult *model.SendResult, err error) { +func (self *DefaultMQProducer) Send(message *rocketmqm.MessageImpl) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) return } -func (self *DefaultMQProducer) SendWithTimeout(message *rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (self *DefaultMQProducer) SendWithTimeout(message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) return } diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index 7766ccff0..b0a5ab315 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -212,7 +212,7 @@ func FilterMessageAgainByTags(msgExts []rocketmqm.MessageExt, subscriptionTagLis result = []rocketmqm.MessageExt{} for _, msg := range msgExts { for _, tag := range subscriptionTagList { - if tag == msg.GetTag() { + if tag == msg.Tag() { result = append(result, msg) break } @@ -241,7 +241,7 @@ func DecodeMessage(data []byte) []rocketmqm.MessageExt { msgs := []rocketmqm.MessageExt{} for buf.Len() > 0 { - msg := rocketmqm.MessageExt{Message: &rocketmqm.Message{}} + msg := rocketmqm.MessageExt{MessageImpl: &rocketmqm.MessageImpl{}} binary.Read(buf, binary.BigEndian, &storeSize) binary.Read(buf, binary.BigEndian, &magicCode) binary.Read(buf, binary.BigEndian, &bodyCRC) diff --git a/rocketmq-go/model/message/message.go b/rocketmq-go/model/message/message.go deleted file mode 100644 index 7ed484150..000000000 --- a/rocketmq-go/model/message/message.go +++ /dev/null @@ -1,266 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package message - -// -//import ( -// "bytes" -// "compress/zlib" -// "encoding/binary" -// "encoding/json" -// "fmt" -// "github.com/golang/glog" -// "io/ioutil" -//) -// -//const ( -// CompressedFlag = 1 << 0 -// MultiTagsFlag = 1 << 1 -// TransactionNotType = 0 << 2 -// TransactionPreparedType = 1 << 2 -// TransactionCommitType = 2 << 2 -// TransactionRollbackType = 3 << 2 -//) -// -//const ( -// NameValueSeparator = 1 + iota -// PropertySeparator -//) -// -//const ( -// CharacterMaxLength = 255 -//) -// -//type Message struct { -// Topic string -// Flag int32 -// properties map[string]string -// Body []byte -//} -// -//func NewDefultMessage(topic string, body []byte) *Message { -// return NewMessage(topic, "", "", 0, body, true) -//} -// -//type MessageExt struct { -// Message -// QueueId int32 -// StoreSize int32 -// QueueOffset int64 -// SysFlag int32 -// BornTimestamp int64 -// // bornHost -// StoreTimestamp int64 -// // storeHost -// MsgId string -// CommitLogOffset int64 -// BodyCRC int32 -// ReconsumeTimes int32 -// PreparedTransactionOffset int64 -//} -// -//func (msg *Message) encodeMessage() []byte { -// // TODO -// return nil -//} -// -//func decodeMessage(data []byte) []*MessageExt { -// buf := bytes.NewBuffer(data) -// var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 -// var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 -// var topicLen byte -// var topic, body, properties, bornHost, storeHost []byte -// var propertiesLength int16 -// -// var propertiesMap map[string]string -// -// msgs := make([]*MessageExt, 0, 32) -// for buf.Len() > 0 { -// msg := new(MessageExt) -// binary.Read(buf, binary.BigEndian, &storeSize) -// binary.Read(buf, binary.BigEndian, &magicCode) -// binary.Read(buf, binary.BigEndian, &bodyCRC) -// binary.Read(buf, binary.BigEndian, &queueId) -// binary.Read(buf, binary.BigEndian, &flag) -// binary.Read(buf, binary.BigEndian, &queueOffset) -// binary.Read(buf, binary.BigEndian, &physicOffset) -// binary.Read(buf, binary.BigEndian, &sysFlag) -// binary.Read(buf, binary.BigEndian, &bornTimeStamp) -// bornHost = make([]byte, 4) -// binary.Read(buf, binary.BigEndian, &bornHost) -// binary.Read(buf, binary.BigEndian, &bornPort) -// binary.Read(buf, binary.BigEndian, &storeTimestamp) -// storeHost = make([]byte, 4) -// binary.Read(buf, binary.BigEndian, &storeHost) -// binary.Read(buf, binary.BigEndian, &storePort) -// binary.Read(buf, binary.BigEndian, &reconsumeTimes) -// binary.Read(buf, binary.BigEndian, &preparedTransactionOffset) -// binary.Read(buf, binary.BigEndian, &bodyLength) -// if bodyLength > 0 { -// body = make([]byte, bodyLength) -// binary.Read(buf, binary.BigEndian, body) -// -// if (sysFlag & CompressedFlag) == CompressedFlag { -// b := bytes.NewReader(body) -// z, err := zlib.NewReader(b) -// if err != nil { -// fmt.Println(err) -// return nil -// } -// -// body, err = ioutil.ReadAll(z) -// if err != nil { -// fmt.Println(err) -// return nil -// } -// z.Close() -// } -// -// } -// binary.Read(buf, binary.BigEndian, &topicLen) -// topic = make([]byte, 0) -// binary.Read(buf, binary.BigEndian, &topic) -// binary.Read(buf, binary.BigEndian, &propertiesLength) -// if propertiesLength > 0 { -// properties = make([]byte, propertiesLength) -// binary.Read(buf, binary.BigEndian, &properties) -// propertiesMap = make(map[string]string) -// json.Unmarshal(properties, &propertiesMap) -// } -// -// if magicCode != -626843481 { -// fmt.Printf("magic code is error %d", magicCode) -// return nil -// } -// -// msg.Topic = string(topic) -// msg.QueueId = queueId -// msg.SysFlag = sysFlag -// msg.QueueOffset = queueOffset -// msg.BodyCRC = bodyCRC -// msg.StoreSize = storeSize -// msg.BornTimestamp = bornTimeStamp -// msg.ReconsumeTimes = reconsumeTimes -// msg.Flag = flag -// //msg.commitLogOffset=physicOffset -// msg.StoreTimestamp = storeTimestamp -// msg.PreparedTransactionOffset = preparedTransactionOffset -// msg.Body = body -// msg.properties = propertiesMap -// -// msgs = append(msgs, msg) -// } -// -// return msgs -//} -// -//func messageProperties2String(properties map[string]string) string { -// StringBuilder := bytes.NewBuffer([]byte{}) -// if properties != nil && len(properties) != 0 { -// for k, v := range properties { -// binary.Write(StringBuilder, binary.BigEndian, k) // 4 -// binary.Write(StringBuilder, binary.BigEndian, NameValueSeparator) // 4 -// binary.Write(StringBuilder, binary.BigEndian, v) // 4 -// binary.Write(StringBuilder, binary.BigEndian, PropertySeparator) // 4 -// } -// } -// return StringBuilder.String() -//} -// -////func (msg Message) checkMessage(producer *DefaultProducer) (err error) { -//// if err = checkTopic(msg.Topic); err != nil { -//// if len(msg.Body) == 0 { -//// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body is null") -//// } else if len(msg.Body) > producer.maxMessageSize { -//// err = errors.New("ResponseCode:" + strconv.Itoa(MsgIllegal) + ", the message body size over max value, MAX:" + strconv.Itoa(producer.maxMessageSize)) -//// } -//// } -//// return -////} -// -////func checkTopic(topic string) (err error) { -//// if topic == "" { -//// err = errors.New("the specified topic is blank") -//// } -//// if len(topic) > CharacterMaxLength { -//// err = errors.New("the specified topic is longer than topic max length 255") -//// } -//// if topic == DefaultTopic { -//// err = errors.New("the topic[" + topic + "] is conflict with default topic") -//// } -//// return -////} -// -//func NewMessage(topic, tags, keys string, flag int32, body []byte, waitStoreMsgOK bool) *Message { -// message := &Message{ -// Topic: topic, -// Flag: flag, -// Body: body, -// } -// -// if tags != "" { -// message.SetTags(tags) -// } -// -// if keys != "" { -// message.SetKeys(keys) -// } -// -// message.SetWaitStoreMsgOK(waitStoreMsgOK) -// return message -//} -// -//func (msg *Message) SetTags(t string) { -// msg.putProperty(MessageConst.PropertyTags, t) -//} -// -//func (msg *Message) SetKeys(k string) { -// msg.putProperty(MessageConst.PropertyKeys, k) -//} -// -//func (msg *Message) SetWaitStoreMsgOK(b bool) { -// -//} -// -//func (msg *Message) Property() map[string]string { -// return msg.properties -//} -// -//func (msg *Message) putProperty(k, v string) { -// if msg.properties == nil { -// msg.properties = make(map[string]string) -// } -// if v, found := msg.properties[k]; !found { -// msg.properties[k] = v -// } else { -// glog.Infof("Message put peoperties key: %s existed.", k) -// } -//} -// -//func (msg *Message) removeProperty(k, v string) string { -// if v, ok := msg.properties[k]; ok { -// delete(msg.properties, k) -// return v -// } -// return "" -//} -// -//func (msg *Message) String() string { -// return fmt.Sprintf("Message [topic=%s, flag=%s, properties=%s, body=%s]", -// msg.Topic, msg.Flag, msg.properties, msg.Body) -//} diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index ea82edd92..5ea8a2572 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -47,7 +47,7 @@ func main() { successIndex := -1 for index, message := range messageList { if string(message.Body) != testMessageBody { - panic("message.Body is wrong message.Body=" + string(message.Body) + " testMessageBody=" + testMessageBody + " tag=" + message.GetTag()) + panic("message.Body is wrong message.Body=" + string(message.Body) + " testMessageBody=" + testMessageBody + " tag=" + message.Tag()) } if consumeTime < 2 { consumeTime++ @@ -63,7 +63,7 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(testMessageBody)} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(testMessageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index e4347de61..c0dade4e8 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -60,7 +60,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() <-time.After(time.Second * 30) // wait - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello world")} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello world")} message.SetTag(tag) message.SetDelayTimeLevel(3) // cost 15 second result, err := producer.Send(message) diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index a7d5796c3..cb81145d5 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -45,7 +45,7 @@ func main() { consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - if msg.GetTag() == tag && messageBody == string(messageBody) { + if msg.Tag() == tag && messageBody == string(messageBody) { chResult <- true } successIndex = index @@ -56,7 +56,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < messageCount; i++ { - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(messageBody)} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(messageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index c401aecee..079c5c836 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -63,7 +63,7 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte(bigMessageBody)} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(bigMessageBody)} message.SetTag("compress_message_test") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index a88ae2dfb..46a938747 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -42,10 +42,10 @@ func main() { consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - if msg.GetTag() != "tag0" && msg.GetTag() != "tag2" && msg.GetTag() != "tag4" { - panic("receive message not belong here tag=" + msg.GetTag()) + if msg.Tag() != "tag0" && msg.Tag() != "tag2" && msg.Tag() != "tag4" { + panic("receive message not belong here tag=" + msg.Tag()) } - fmt.Println("got " + msg.GetTag()) + fmt.Println("got " + msg.Tag()) chResult <- true successIndex = index @@ -55,7 +55,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < 5; i++ { - var message = &rocketmqm.Message{Topic: testTopic, Body: []byte("hello world")} + var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello world")} message.SetTag("tag" + util.IntToString(i)) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) From 6d509f24010806215b7ca95715fdd5ef82f4ac55 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Fri, 18 Aug 2017 00:09:50 +0800 Subject: [PATCH 39/88] make api easy to use --- rocketmq-go/api/model/message.go | 159 ++--------------- rocketmq-go/api/model/message_ext.go | 73 +------- .../{ => api}/model/message_listener.go | 6 +- rocketmq-go/api/rocketmq_clent_instance.go | 10 +- rocketmq-go/api/rocketmq_consumer.go | 3 +- rocketmq-go/api/rocketmq_producer.go | 4 +- rocketmq-go/example/simple_consumer.go | 63 +++---- rocketmq-go/example/simple_producer.go | 4 +- .../example/simple_producer_consumer.go | 6 +- rocketmq-go/kernel/consume_message_service.go | 39 +++-- rocketmq-go/kernel/producer_service.go | 17 +- .../kernel/producer_service_for_send_back.go | 11 +- rocketmq-go/manage/mq_producer.go | 9 +- rocketmq-go/manage/mq_push_consumer.go | 2 +- rocketmq-go/manage/pull_message_controller.go | 12 +- rocketmq-go/model/message/message_ext_impl.go | 94 ++++++++++ rocketmq-go/model/message/message_impl.go | 163 ++++++++++++++++++ rocketmq-go/model/process_queue.go | 12 +- rocketmq-go/model/pull_result.go | 8 +- rocketmq-go/model/query_result.go | 6 +- .../test/test_consume_message_retry.go | 8 +- .../test_delay_message_send_and_receive.go | 4 +- ...etmq_serialize_message_send_and_receive.go | 4 +- .../test_send_and_receive_compress_message.go | 6 +- .../test/test_send_and_receive_tag_message.go | 4 +- 25 files changed, 403 insertions(+), 324 deletions(-) rename rocketmq-go/{ => api}/model/message_listener.go (80%) create mode 100644 rocketmq-go/model/message/message_ext_impl.go create mode 100644 rocketmq-go/model/message/message_impl.go diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 2aa29047c..74ab475e0 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -1,160 +1,23 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - package rocketmqm -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "strings" -) +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" type Message interface { + //set message topic + SetTopic(tag string) + //set message tag SetTag(tag string) + //get message tag Tag() (tag string) + //set message key SetKeys(keys []string) + // set delay time level SetDelayTimeLevel(delayTimeLevel int) + // set body + SetBody([]byte) } -func NewMessage() (message Message) { - message = &MessageImpl{} - return -} - -type MessageImpl struct { - Topic string - Flag int - Properties map[string]string - Body []byte -} - -//set message tag -func (m *MessageImpl) SetTag(tag string) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_TAGS] = tag -} - -//get message tag from Properties -func (m *MessageImpl) Tag() (tag string) { - if m.Properties != nil { - tag = m.Properties[constant.PROPERTY_TAGS] - } - return -} - -//set message key -func (m *MessageImpl) SetKeys(keys []string) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") -} - -//SetDelayTimeLevel -func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) -} - -////SetWaitStoreMsgOK -//func (m *MessageImpl) SetWaitStoreMsgOK(waitStoreMsgOK bool) { -// if m.Properties == nil { -// m.Properties = make(map[string]string) -// } -// m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) -//} -//GeneratorMsgUniqueKey only use by system -func (m *MessageImpl) GeneratorMsgUniqueKey() { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - if len(m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { - return - } - m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() -} - -//GetMsgUniqueKey only use by system -func (m *MessageExt) GetMsgUniqueKey() string { - if m.Properties != nil { - originMessageId := m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] - if len(originMessageId) > 0 { - return originMessageId - } - } - return m.MsgId -} - -//only use by system -func (m *MessageImpl) SetOriginMessageId(messageId string) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId -} - -//only use by system -func (m *MessageImpl) SetRetryTopic(retryTopic string) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic -} - -//only use by system -func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) -} - -//only use by system -func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { - reConsumeTime = 0 - if m.Properties != nil { - reConsumeTimeStr := m.Properties[constant.PROPERTY_RECONSUME_TIME] - if len(reConsumeTimeStr) > 0 { - reConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) - } - } - return -} - -//only use by system -func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) -} - -//only use by system -func (m *MessageImpl) GetMaxReconsumeTimes() (maxConsumeTime int) { - maxConsumeTime = 0 - if m.Properties != nil { - reConsumeTimeStr := m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] - if len(reConsumeTimeStr) > 0 { - maxConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) - } - } +func NewMessage() (msg Message) { + msg = message.NewMessageImpl() return } diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 26e89efcb..5acb418ba 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,70 +1,13 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - package rocketmqm +type MessageExt interface { + //get message topic + Topic2()(tag string) + //get message tag + Tag2() (tag string) -import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "math" -) - -type MessageExt struct { - *MessageImpl - QueueId int32 - StoreSize int32 - QueueOffset int64 - SysFlag int32 - BornTimestamp int64 - BornHost string - StoreTimestamp int64 - StoreHost string - MsgId string - CommitLogOffset int64 - BodyCRC int32 - ReconsumeTimes int32 - PreparedTransactionOffset int64 - - propertyConsumeStartTimestamp string + // get body + Body2()([]byte) + MsgId2()(string) } -func (m *MessageExt) GetOriginMessageId() string { - if m.Properties != nil { - originMessageId := m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] - if len(originMessageId) > 0 { - return originMessageId - } - } - return m.MsgId -} - -func (m *MessageExt) GetConsumeStartTime() int64 { - if len(m.propertyConsumeStartTimestamp) > 0 { - return util.StrToInt64WithDefaultValue(m.propertyConsumeStartTimestamp, -1) - } - return math.MaxInt64 -} -func (m *MessageExt) SetConsumeStartTime() { - if m.Properties == nil { - m.Properties = make(map[string]string) - } - nowTime := util.CurrentTimeMillisStr() - m.Properties[constant.PROPERTY_KEYS] = nowTime - m.propertyConsumeStartTimestamp = nowTime - return -} diff --git a/rocketmq-go/model/message_listener.go b/rocketmq-go/api/model/message_listener.go similarity index 80% rename from rocketmq-go/model/message_listener.go rename to rocketmq-go/api/model/message_listener.go index f7e415de9..c365f442b 100644 --- a/rocketmq-go/model/message_listener.go +++ b/rocketmq-go/api/model/message_listener.go @@ -15,8 +15,6 @@ See the License for the specific language governing permissions and limitations under the License. */ -package model +package rocketmqm -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - -type MessageListener func(msgs []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult +type MessageListener func(msgs []MessageExt) ConsumeConcurrentlyResult diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 24c25fb2c..a75451d60 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -28,7 +28,7 @@ type MQClientInstance interface { Start() } -type RocketMQClientInstanceImpl struct { +type ClientInstanceImpl struct { rocketMqManager *manage.MqClientManager } @@ -37,17 +37,17 @@ func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstanc return InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) } func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmqm.MqClientConfig) (rocketMQClientInstance MQClientInstance) { - rocketMQClientInstance = &RocketMQClientInstanceImpl{rocketMqManager: manage.MqClientManagerInit(mqClientConfig)} + rocketMQClientInstance = &ClientInstanceImpl{rocketMqManager: manage.MqClientManagerInit(mqClientConfig)} return } -func (r *RocketMQClientInstanceImpl) RegisterProducer(producer MQProducer) { +func (r *ClientInstanceImpl) RegisterProducer(producer MQProducer) { r.rocketMqManager.RegistProducer(producer.(*manage.DefaultMQProducer)) } -func (r *RocketMQClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { +func (r *ClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { r.rocketMqManager.RegistConsumer(consumer.(*manage.DefaultMQPushConsumer)) } -func (r *RocketMQClientInstanceImpl) Start() { +func (r *ClientInstanceImpl) Start() { r.rocketMqManager.Start() } diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 7ffd846f0..986e6e55b 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -20,11 +20,10 @@ package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) type MQConsumer interface { - RegisterMessageListener(listener model.MessageListener) + RegisterMessageListener(listener rocketmqm.MessageListener) Subscribe(topic string, subExpression string) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 2a35edfc7..bd9cbdab3 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -24,8 +24,8 @@ import ( ) type MQProducer interface { - Send(message *rocketmqm.MessageImpl) (sendResult *model.SendResult, err error) - SendWithTimeout(message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) + Send(message rocketmqm.Message ) (sendResult *model.SendResult, err error) + SendWithTimeout(message rocketmqm.Message , timeout int64) (sendResult *model.SendResult, err error) } func NewDefaultMQProducer(producerGroup string) (r MQProducer) { diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 2456bb7f6..1dc8c28b8 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -18,38 +18,39 @@ limitations under the License. package main import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/golang/glog" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + //"github.com/golang/glog" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) func main() { - var ( - nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) - testTopic = "GoLangRocketMQ" - testConsumerGroup = "TestConsumerGroup" - ) - // init rocketMQClientInstance - rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) - - // 1.init rocketMQConsumer - // 2.subscribe topic and register our function to message listener - // 3.register it - var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) - consumer.Subscribe(testTopic, "*") - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { - successIndex := -1 - for index, msg := range messageList { - glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) - // call your function - successIndex = index - } - return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} - }) - rocketMQClientInstance.RegisterConsumer(consumer) - - // start rocketMQ client instance - rocketMQClientInstance.Start() - - select {} + //var ( + // nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + // testTopic = "GoLangRocketMQ" + // testConsumerGroup = "TestConsumerGroup" + //) + //// init rocketMQClientInstance + //rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + // + //// 1.init rocketMQConsumer + //// 2.subscribe topic and register our function to message listener + //// 3.register it + //var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) + //consumer.Subscribe(testTopic, "*") + //consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + // successIndex := -1 + // for index, msg := range messageList { + // glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) + // // call your function + // successIndex = index + // } + // return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} + //}) + //rocketMQClientInstance.RegisterConsumer(consumer) + // + //// start rocketMQ client instance + //rocketMQClientInstance.Start() + // + //select {} } diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index f80b9f0af..fd6cf7e0e 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -39,7 +39,9 @@ func main() { rocketMQClientInstance.Start() //start send test message - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello World")} + var message =rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte("hello World")) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index c85839f89..ae7c76463 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -44,7 +44,7 @@ func main() { consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId2(), string(msg.Body2())) // call your function successIndex = index } @@ -57,7 +57,9 @@ func main() { //start send test message for { - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello World")} + var message =rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte("hello World")) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 25440f851..fb69317df 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -23,25 +23,26 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ConsumeMessageService interface { Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) - SubmitConsumeRequest(msgs []rocketmqm.MessageExt, processQueue *model.ProcessQueue, + SubmitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) - SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) - ConsumeMessageDirectly(messageExt *rocketmqm.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) + SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) + ConsumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) } type ConsumeMessageConcurrentlyServiceImpl struct { consumerGroup string - messageListener model.MessageListener + messageListener rocketmqm.MessageListener sendMessageBackProducerService SendMessageBackProducerService //for send retry MessageImpl offsetStore OffsetStore consumerConfig *rocketmqm.MqConsumerConfig } -func NewConsumeMessageConcurrentlyServiceImpl(messageListener model.MessageListener) (consumeService ConsumeMessageService) { +func NewConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageListener) (consumeService ConsumeMessageService) { consumeService = &ConsumeMessageConcurrentlyServiceImpl{messageListener: messageListener, sendMessageBackProducerService: &SendMessageBackProducerServiceImpl{}} return } @@ -53,7 +54,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqCli c.consumerConfig = consumerConfig } -func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rocketmqm.MessageExt, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i @@ -64,22 +65,32 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []rock go func() { glog.V(2).Infof("look slice begin %d end %d msgsLen %d", begin, end, msgsLen) batchMsgs := transformMessageToConsume(c.consumerGroup, msgs[begin:end]) - consumeState := c.messageListener(batchMsgs) + + consumeState := c.messageListener(c.convert2ConsumeType(batchMsgs)) c.processConsumeResult(consumeState, batchMsgs, messageQueue, processQueue) }() i = end } return } +func (c *ConsumeMessageConcurrentlyServiceImpl) convert2ConsumeType(msgs []message.MessageExtImpl) (ret []rocketmqm.MessageExt) { + msgLen := len(msgs) + ret = make([]rocketmqm.MessageExt, msgLen) + + for i := 0; i < msgLen; i++ { + ret[i] = rocketmqm.MessageExt(&msgs[i]) + } + return +} -func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { err = c.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } -func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *rocketmqm.MessageExt, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { +func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { start := util.CurrentTimeMillisInt64() - consumeResult := c.messageListener([]rocketmqm.MessageExt{*messageExt}) + consumeResult := c.messageListener(c.convert2ConsumeType([]message.MessageExtImpl{*messageExt})) consumeMessageDirectlyResult.AutoCommit = true consumeMessageDirectlyResult.Order = false consumeMessageDirectlyResult.SpentTimeMills = util.CurrentTimeMillisInt64() - start @@ -91,7 +102,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageEx return } -func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []rocketmqm.MessageExt, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []message.MessageExtImpl, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return @@ -109,8 +120,8 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } } } - var failedMessages []rocketmqm.MessageExt - successMessages := []rocketmqm.MessageExt{} + var failedMessages []message.MessageExtImpl + successMessages := []message.MessageExtImpl{} if ackIndex >= 0 { successMessages = msgs[:ackIndex+1] } @@ -133,7 +144,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } -func transformMessageToConsume(consumerGroup string, msgs []rocketmqm.MessageExt) []rocketmqm.MessageExt { +func transformMessageToConsume(consumerGroup string, msgs []message.MessageExtImpl) []message.MessageExtImpl { retryTopicName := constant.RETRY_GROUP_TOPIC_PREFIX + consumerGroup for _, msg := range msgs { diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index d6d1fd3bb..7e489cce8 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -26,11 +26,12 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ProducerService interface { CheckConfig() (err error) - SendDefaultImpl(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + SendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } type DefaultProducerService struct { @@ -54,7 +55,7 @@ func (d *DefaultProducerService) CheckConfig() (err error) { return } -func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) SendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) @@ -76,7 +77,7 @@ func (d *DefaultProducerService) SendDefaultImpl(message *rocketmqm.MessageImpl, return } -func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -87,7 +88,7 @@ func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, s sendResult, err = processSendResponse(brokerAddr, message, response) return } -func processSendResponse(brokerName string, message *rocketmqm.MessageImpl, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { +func processSendResponse(brokerName string, message *message.MessageImpl, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { sendResult = &model.SendResult{} switch response.Code { case remoting.FLUSH_DISK_TIMEOUT: @@ -133,7 +134,7 @@ func processSendResponse(brokerName string, message *rocketmqm.MessageImpl, resp return } -func (d *DefaultProducerService) checkMessage(message *rocketmqm.MessageImpl) (err error) { +func (d *DefaultProducerService) checkMessage(message *message.MessageImpl) (err error) { if message == nil { err = errors.New("message is nil") return @@ -167,7 +168,7 @@ func (d *DefaultProducerService) checkMessage(message *rocketmqm.MessageImpl) (e return } -func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int messageQueue model.MessageQueue @@ -192,7 +193,7 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *rocketmqm.M } -func (d *DefaultProducerService) doSendMessage(message *rocketmqm.MessageImpl, messageQueue model.MessageQueue, +func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -231,7 +232,7 @@ func (d *DefaultProducerService) doSendMessage(message *rocketmqm.MessageImpl, m return } -func (d *DefaultProducerService) tryToCompressMessage(message *rocketmqm.MessageImpl) (compressedFlag int, err error) { +func (d *DefaultProducerService) tryToCompressMessage(message *message.MessageImpl) (compressedFlag int, err error) { if len(message.Body) < d.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 72bf7c2fe..465b51745 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -25,10 +25,11 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type SendMessageBackProducerService interface { - SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) + SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) } @@ -40,7 +41,7 @@ type SendMessageBackProducerServiceImpl struct { } // send to original broker,if fail send a new retry message -func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketmqm.MessageExt, delayLayLevel int, brokerName string) (err error) { +func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) err = s.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) if err == nil { @@ -51,8 +52,8 @@ func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *rocketm return } -func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *rocketmqm.MessageExt) error { - retryMessage := &rocketmqm.MessageImpl{} +func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *message.MessageExtImpl) error { + retryMessage := &message.MessageImpl{} originMessageId := messageExt.GetOriginMessageId() retryMessage.Properties = messageExt.Properties retryMessage.SetOriginMessageId(originMessageId) @@ -83,7 +84,7 @@ func (s *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService( s.consumerConfig = consumerConfig } -func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *rocketmqm.MessageExt, delayLayLevel int) (err error) { +func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *message.MessageExtImpl, delayLayLevel int) (err error) { if len(brokerName) == 0 { err = errors.New("broker can't be empty") glog.Error(err) diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index a10be5a7b..03b72c535 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -22,6 +22,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type DefaultMQProducer struct { @@ -39,11 +40,11 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProd return } -func (self *DefaultMQProducer) Send(message *rocketmqm.MessageImpl) (sendResult *model.SendResult, err error) { - sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", self.ProducerConfig.SendMsgTimeout) +func (d *DefaultMQProducer) Send(msg rocketmqm.Message) (sendResult *model.SendResult, err error) { + sendResult, err = d.producerService.SendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", d.ProducerConfig.SendMsgTimeout) return } -func (self *DefaultMQProducer) SendWithTimeout(message *rocketmqm.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { - sendResult, err = self.producerService.SendDefaultImpl(message, constant.COMMUNICATIONMODE_SYNC, "", timeout) +func (d *DefaultMQProducer) SendWithTimeout(msg rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { + sendResult, err = d.producerService.SendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", timeout) return } diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index 65a7d6a39..dfd1944f3 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -73,7 +73,7 @@ func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { } } -func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener model.MessageListener) { +func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener rocketmqm.MessageListener) { d.consumeMessageService = kernel.NewConsumeMessageConcurrentlyServiceImpl(messageListener) } diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index b0a5ab315..6a2c74403 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -21,7 +21,6 @@ import ( "bytes" "encoding/binary" "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" @@ -31,6 +30,7 @@ import ( "github.com/golang/glog" "strconv" "time" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type PullMessageController struct { @@ -204,12 +204,12 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } -func FilterMessageAgainByTags(msgExts []rocketmqm.MessageExt, subscriptionTagList []string) (result []rocketmqm.MessageExt) { +func FilterMessageAgainByTags(msgExts []message.MessageExtImpl, subscriptionTagList []string) (result []message.MessageExtImpl) { result = msgExts if len(subscriptionTagList) == 0 { return } - result = []rocketmqm.MessageExt{} + result = []message.MessageExtImpl{} for _, msg := range msgExts { for _, tag := range subscriptionTagList { if tag == msg.Tag() { @@ -229,7 +229,7 @@ func (p *PullMessageController) consumerPullMessageAsync(brokerName string, requ } } -func DecodeMessage(data []byte) []rocketmqm.MessageExt { +func DecodeMessage(data []byte) []message.MessageExtImpl { buf := bytes.NewBuffer(data) var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 @@ -239,9 +239,9 @@ func DecodeMessage(data []byte) []rocketmqm.MessageExt { var propertiesmap = make(map[string]string) - msgs := []rocketmqm.MessageExt{} + msgs := []message.MessageExtImpl{} for buf.Len() > 0 { - msg := rocketmqm.MessageExt{MessageImpl: &rocketmqm.MessageImpl{}} + msg := message.MessageExtImpl{MessageImpl: &message.MessageImpl{}} binary.Read(buf, binary.BigEndian, &storeSize) binary.Read(buf, binary.BigEndian, &magicCode) binary.Read(buf, binary.BigEndian, &bodyCRC) diff --git a/rocketmq-go/model/message/message_ext_impl.go b/rocketmq-go/model/message/message_ext_impl.go new file mode 100644 index 000000000..b4f65064d --- /dev/null +++ b/rocketmq-go/model/message/message_ext_impl.go @@ -0,0 +1,94 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package message + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "math" +) + + + + +type MessageExtImpl struct { + *MessageImpl + QueueId int32 + StoreSize int32 + QueueOffset int64 + SysFlag int32 + BornTimestamp int64 + BornHost string + StoreTimestamp int64 + StoreHost string + MsgId string + CommitLogOffset int64 + BodyCRC int32 + ReconsumeTimes int32 + PreparedTransactionOffset int64 + + propertyConsumeStartTimestamp string +} + +//get message topic +func (m *MessageExtImpl)Topic2()(topic string){ + topic = m.Topic + return +} +//get message tag +func (m *MessageExtImpl)Tag2() (tag string){ + tag = m.Tag() + return +} + +// get body +func (m *MessageExtImpl)Body2()(body []byte){ + body = m.Body + return +} +func (m *MessageExtImpl)MsgId2()(msgId string){ + msgId = m.MsgId + return +} + +func (m *MessageExtImpl) GetOriginMessageId() string { + if m.Properties != nil { + originMessageId := m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] + if len(originMessageId) > 0 { + return originMessageId + } + } + return m.MsgId +} + +func (m *MessageExtImpl) GetConsumeStartTime() int64 { + if len(m.propertyConsumeStartTimestamp) > 0 { + return util.StrToInt64WithDefaultValue(m.propertyConsumeStartTimestamp, -1) + } + return math.MaxInt64 +} + +func (m *MessageExtImpl) SetConsumeStartTime() { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + nowTime := util.CurrentTimeMillisStr() + m.Properties[constant.PROPERTY_KEYS] = nowTime + m.propertyConsumeStartTimestamp = nowTime + return +} diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go new file mode 100644 index 000000000..e65b6fd13 --- /dev/null +++ b/rocketmq-go/model/message/message_impl.go @@ -0,0 +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. +*/ + +package message + +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "strings" +) + + + +type MessageImpl struct { + Topic string + Flag int + Properties map[string]string + Body []byte +} + +func NewMessageImpl()(message *MessageImpl) { + message = &MessageImpl{} + return +} + +func (m *MessageImpl) SetTopic(topic string) { + m.Topic = topic +} + +func (m *MessageImpl) SetBody(body []byte) { + m.Body = body +} + +//set message tag +func (m *MessageImpl) SetTag(tag string) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_TAGS] = tag +} + +//get message tag from Properties +func (m *MessageImpl) Tag() (tag string) { + if m.Properties != nil { + tag = m.Properties[constant.PROPERTY_TAGS] + } + return +} + +//set message key +func (m *MessageImpl) SetKeys(keys []string) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") +} + +//SetDelayTimeLevel +func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) +} + +////SetWaitStoreMsgOK +//func (m *MessageImpl) SetWaitStoreMsgOK(waitStoreMsgOK bool) { +// if m.Properties == nil { +// m.Properties = make(map[string]string) +// } +// m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) +//} +//GeneratorMsgUniqueKey only use by system +func (m *MessageImpl) GeneratorMsgUniqueKey() { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + if len(m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { + return + } + m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() +} + +//GetMsgUniqueKey only use by system +func (m *MessageExtImpl) GetMsgUniqueKey() string { + if m.Properties != nil { + originMessageId := m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] + if len(originMessageId) > 0 { + return originMessageId + } + } + return m.MsgId +} + +//only use by system +func (m *MessageImpl) SetOriginMessageId(messageId string) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId +} + +//only use by system +func (m *MessageImpl) SetRetryTopic(retryTopic string) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic +} + +//only use by system +func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) +} + +//only use by system +func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { + reConsumeTime = 0 + if m.Properties != nil { + reConsumeTimeStr := m.Properties[constant.PROPERTY_RECONSUME_TIME] + if len(reConsumeTimeStr) > 0 { + reConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) + } + } + return +} + +//only use by system +func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { + if m.Properties == nil { + m.Properties = make(map[string]string) + } + m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) +} + +//only use by system +func (m *MessageImpl) GetMaxReconsumeTimes() (maxConsumeTime int) { + maxConsumeTime = 0 + if m.Properties != nil { + reConsumeTimeStr := m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] + if len(reConsumeTimeStr) > 0 { + maxConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) + } + } + return +} diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index 68dcf714c..394f8c9d5 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -18,17 +18,17 @@ limitations under the License. package model import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/emirpasic/gods/maps/treemap" "github.com/golang/glog" "sync" "time" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ProcessQueue struct { - msgTreeMap *treemap.Map // int | MessageExt + msgTreeMap *treemap.Map // int | MessageExtImpl msgCount int lockTreeMap sync.RWMutex locked bool @@ -106,7 +106,7 @@ func (p *ProcessQueue) DeleteExpireMsg(queueOffset int) { } } -func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmqm.MessageExt) { +func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *message.MessageExtImpl) { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() key, value := p.msgTreeMap.Min() @@ -115,7 +115,7 @@ func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *rocketmq } offset = key.(int) - message := value.(rocketmqm.MessageExt) + message := value.(message.MessageExtImpl) messagePoint = &message return } @@ -139,7 +139,7 @@ func (p *ProcessQueue) GetMaxSpan() int { return maxOffset - minOffset } -func (p *ProcessQueue) RemoveMessage(msgs []rocketmqm.MessageExt) (offset int64) { +func (p *ProcessQueue) RemoveMessage(msgs []message.MessageExtImpl) (offset int64) { now := time.Now() offset = -1 defer p.lockTreeMap.Unlock() @@ -160,7 +160,7 @@ func (p *ProcessQueue) RemoveMessage(msgs []rocketmqm.MessageExt) (offset int64) return } -func (p *ProcessQueue) PutMessage(msgs []rocketmqm.MessageExt) (dispatchToConsume bool) { +func (p *ProcessQueue) PutMessage(msgs []message.MessageExtImpl) (dispatchToConsume bool) { dispatchToConsume = false msgsLen := len(msgs) if msgsLen == 0 { diff --git a/rocketmq-go/model/pull_result.go b/rocketmq-go/model/pull_result.go index e9ec9fd11..095460753 100644 --- a/rocketmq-go/model/pull_result.go +++ b/rocketmq-go/model/pull_result.go @@ -37,10 +37,10 @@ package model // nextBeginOffset int64 // minOffset int64 // maxOffset int64 -// msgFoundList []*message.MessageExt +// msgFoundList []*message.MessageExtImpl //} // -//func NewPullResult(ps PullStatus, next, min, max int64, list []*message.MessageExt) *PullResult { +//func NewPullResult(ps PullStatus, next, min, max int64, list []*message.MessageExtImpl) *PullResult { // return &PullResult{ // ps, // next, @@ -66,11 +66,11 @@ package model // return result.minOffset //} // -//func (result *PullResult) MsgFoundList() []*message.MessageExt { +//func (result *PullResult) MsgFoundList() []*message.MessageExtImpl { // return result.msgFoundList //} // -//func (result *PullResult) SetMsgFoundList(list []*message.MessageExt) { +//func (result *PullResult) SetMsgFoundList(list []*message.MessageExtImpl) { // result.msgFoundList = list //} // diff --git a/rocketmq-go/model/query_result.go b/rocketmq-go/model/query_result.go index be47f83af..c1a105ab2 100644 --- a/rocketmq-go/model/query_result.go +++ b/rocketmq-go/model/query_result.go @@ -25,10 +25,10 @@ package model // //type QueryResult struct { // indexLastUpdateTimestamp int64 -// messageList []*message.MessageExt +// messageList []*message.MessageExtImpl //} // -//func NewQueryResult(timestamp int64, list []*message.MessageExt) *QueryResult { +//func NewQueryResult(timestamp int64, list []*message.MessageExtImpl) *QueryResult { // return &QueryResult{ // indexLastUpdateTimestamp: timestamp, // messageList: list, @@ -39,7 +39,7 @@ package model // return qr.indexLastUpdateTimestamp //} // -//func (qr *QueryResult) MessageList() []*message.MessageExt { //TODO: address? +//func (qr *QueryResult) MessageList() []*message.MessageExtImpl { //TODO: address? // return qr.messageList //} // diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index 5ea8a2572..6cc5b6cf8 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -43,11 +43,11 @@ func main() { var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) fmt.Println(tag) - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, message := range messageList { - if string(message.Body) != testMessageBody { - panic("message.Body is wrong message.Body=" + string(message.Body) + " testMessageBody=" + testMessageBody + " tag=" + message.Tag()) + if string(message.body) != testMessageBody { + panic("message.body is wrong message.body=" + string(message.body) + " testMessageBody=" + testMessageBody + " tag=" + message.Tag()) } if consumeTime < 2 { consumeTime++ @@ -63,7 +63,7 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(testMessageBody)} + var message = &message.MessageImpl{Topic: testTopic, body: []byte(testMessageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index c0dade4e8..51832cd95 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -40,7 +40,7 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { endTime := time.Now() @@ -60,7 +60,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() <-time.After(time.Second * 30) // wait - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello world")} + var message = &message.MessageImpl{Topic: testTopic, body: []byte("hello world")} message.SetTag(tag) message.SetDelayTimeLevel(3) // cost 15 second result, err := producer.Send(message) diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index cb81145d5..5f4ab91a6 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -42,7 +42,7 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.Tag() == tag && messageBody == string(messageBody) { @@ -56,7 +56,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < messageCount; i++ { - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(messageBody)} + var message = &message.MessageImpl{Topic: testTopic, body: []byte(messageBody)} message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index 079c5c836..7856e53f6 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -45,13 +45,13 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "compress_message_test") - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.SysFlag&constant.CompressedFlag != constant.CompressedFlag { panic("message not be compressed") } - if string(msg.Body) != bigMessageBody { + if string(msg.body) != bigMessageBody { panic("message not be unCompressed") } glog.Info("Test compress and tag success") @@ -63,7 +63,7 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte(bigMessageBody)} + var message = &message.MessageImpl{Topic: testTopic, body: []byte(bigMessageBody)} message.SetTag("compress_message_test") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index 46a938747..0ca16e548 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -39,7 +39,7 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "tag0 || tag2||tag4") - consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.Tag() != "tag0" && msg.Tag() != "tag2" && msg.Tag() != "tag4" { @@ -55,7 +55,7 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < 5; i++ { - var message = &rocketmqm.MessageImpl{Topic: testTopic, Body: []byte("hello world")} + var message = &message.MessageImpl{Topic: testTopic, body: []byte("hello world")} message.SetTag("tag" + util.IntToString(i)) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) From 77e9083f0c5259987d21f081b7d0d65b8ef435a6 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Fri, 18 Aug 2017 00:11:24 +0800 Subject: [PATCH 40/88] make api easy to use --- rocketmq-go/api/model/message_ext.go | 9 ++++----- rocketmq-go/api/rocketmq_producer.go | 4 ++-- rocketmq-go/example/simple_consumer.go | 8 ++++---- rocketmq-go/example/simple_producer.go | 2 +- rocketmq-go/example/simple_producer_consumer.go | 2 +- rocketmq-go/kernel/consume_message_service.go | 2 +- rocketmq-go/kernel/producer_service.go | 3 +-- .../kernel/producer_service_for_send_back.go | 2 +- rocketmq-go/kernel/rebalance.go | 2 +- rocketmq-go/manage/mq_client_manage.go | 2 +- rocketmq-go/manage/mq_producer.go | 2 +- rocketmq-go/manage/mq_push_consumer.go | 2 +- rocketmq-go/manage/pull_message_controller.go | 4 ++-- rocketmq-go/model/message/message_ext_impl.go | 14 ++++++-------- rocketmq-go/model/message/message_impl.go | 4 +--- rocketmq-go/model/process_queue.go | 2 +- 16 files changed, 29 insertions(+), 35 deletions(-) diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 5acb418ba..95b0caaf1 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,13 +1,12 @@ package rocketmqm + type MessageExt interface { //get message topic - Topic2()(tag string) + Topic2() (tag string) //get message tag Tag2() (tag string) // get body - Body2()([]byte) - MsgId2()(string) + Body2() []byte + MsgId2() string } - - diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index bd9cbdab3..b34262265 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -24,8 +24,8 @@ import ( ) type MQProducer interface { - Send(message rocketmqm.Message ) (sendResult *model.SendResult, err error) - SendWithTimeout(message rocketmqm.Message , timeout int64) (sendResult *model.SendResult, err error) + Send(message rocketmqm.Message) (sendResult *model.SendResult, err error) + SendWithTimeout(message rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) } func NewDefaultMQProducer(producerGroup string) (r MQProducer) { diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 1dc8c28b8..9bfaf8e8a 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -18,10 +18,10 @@ limitations under the License. package main import ( - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - //"github.com/golang/glog" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +//"github.com/golang/glog" +//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) func main() { diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index fd6cf7e0e..58690f781 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -39,7 +39,7 @@ func main() { rocketMQClientInstance.Start() //start send test message - var message =rocketmqm.NewMessage() + var message = rocketmqm.NewMessage() message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index ae7c76463..89fac3d6f 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -57,7 +57,7 @@ func main() { //start send test message for { - var message =rocketmqm.NewMessage() + var message = rocketmqm.NewMessage() message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index fb69317df..a105a8362 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -21,9 +21,9 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ConsumeMessageService interface { diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 7e489cce8..6e10cf12a 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -23,10 +23,10 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ProducerService interface { @@ -192,7 +192,6 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.Mes return } - func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, messageQueue model.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 465b51745..1bb11b12b 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -23,9 +23,9 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type SendMessageBackProducerService interface { diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 2458f17f6..f61547151 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -21,11 +21,11 @@ import ( "encoding/json" "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate_message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate_message" "github.com/golang/glog" "sort" "strings" diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index eafe62ba3..ee194eb5c 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -22,11 +22,11 @@ import ( "errors" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" "github.com/golang/glog" "strings" diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/manage/mq_producer.go index 03b72c535..f951e9682 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/manage/mq_producer.go @@ -19,9 +19,9 @@ package manage import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index dfd1944f3..c65602f8c 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -19,8 +19,8 @@ package manage import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strings" diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index 6a2c74403..ff0919b95 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -21,16 +21,16 @@ import ( "bytes" "encoding/binary" "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strconv" "time" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type PullMessageController struct { diff --git a/rocketmq-go/model/message/message_ext_impl.go b/rocketmq-go/model/message/message_ext_impl.go index b4f65064d..f17f396d8 100644 --- a/rocketmq-go/model/message/message_ext_impl.go +++ b/rocketmq-go/model/message/message_ext_impl.go @@ -23,9 +23,6 @@ import ( "math" ) - - - type MessageExtImpl struct { *MessageImpl QueueId int32 @@ -46,22 +43,23 @@ type MessageExtImpl struct { } //get message topic -func (m *MessageExtImpl)Topic2()(topic string){ +func (m *MessageExtImpl) Topic2() (topic string) { topic = m.Topic return } + //get message tag -func (m *MessageExtImpl)Tag2() (tag string){ +func (m *MessageExtImpl) Tag2() (tag string) { tag = m.Tag() return } // get body -func (m *MessageExtImpl)Body2()(body []byte){ - body = m.Body +func (m *MessageExtImpl) Body2() (body []byte) { + body = m.Body return } -func (m *MessageExtImpl)MsgId2()(msgId string){ +func (m *MessageExtImpl) MsgId2() (msgId string) { msgId = m.MsgId return } diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go index e65b6fd13..93f63b405 100644 --- a/rocketmq-go/model/message/message_impl.go +++ b/rocketmq-go/model/message/message_impl.go @@ -23,8 +23,6 @@ import ( "strings" ) - - type MessageImpl struct { Topic string Flag int @@ -32,7 +30,7 @@ type MessageImpl struct { Body []byte } -func NewMessageImpl()(message *MessageImpl) { +func NewMessageImpl() (message *MessageImpl) { message = &MessageImpl{} return } diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index 394f8c9d5..a13f00a8c 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -19,12 +19,12 @@ package model import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/emirpasic/gods/maps/treemap" "github.com/golang/glog" "sync" "time" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) type ProcessQueue struct { From c3cd3d42ecc77a77ca317e322250805d0c7d6610 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 20 Aug 2017 18:44:57 +0800 Subject: [PATCH 41/88] refactor --- rocketmq-go/api/model/message.go | 11 ++ rocketmq-go/api/model/message_ext.go | 10 +- rocketmq-go/api/model/message_listener.go | 18 ++ rocketmq-go/api/model/mq_client_config.go | 5 +- rocketmq-go/api/model/mq_consumer_config.go | 43 ++--- rocketmq-go/api/model/mq_producer_config.go | 7 +- rocketmq-go/api/rocketmq_clent_instance.go | 7 +- rocketmq-go/api/rocketmq_consumer.go | 11 ++ rocketmq-go/api/rocketmq_producer.go | 11 ++ rocketmq-go/example/map_util_a.go | 19 ++ rocketmq-go/example/simple_consumer.go | 29 +-- .../example/simple_producer_consumer.go | 3 +- rocketmq-go/kernel/consume_message_service.go | 8 +- rocketmq-go/kernel/mq_client.go | 75 +++++--- rocketmq-go/kernel/offset_store.go | 4 + rocketmq-go/kernel/producer_service.go | 38 ++-- .../kernel/producer_service_for_send_back.go | 20 +- rocketmq-go/kernel/rebalance.go | 1 + .../manage/clean_expire_msg_controller.go | 12 +- rocketmq-go/manage/mq_client_manage.go | 100 +++++----- rocketmq-go/manage/mq_push_consumer.go | 2 +- rocketmq-go/manage/pull_message_controller.go | 20 +- rocketmq-go/manage/rebalance_controller.go | 6 +- rocketmq-go/manage/tasks.go | 6 +- rocketmq-go/model/client_config.go | 171 ------------------ rocketmq-go/model/message/message_ext_impl.go | 59 +++--- rocketmq-go/model/message/message_impl.go | 120 +++++++----- rocketmq-go/model/process_queue.go | 2 +- rocketmq-go/model/pull_request.go | 10 +- rocketmq-go/model/pull_result.go | 80 -------- rocketmq-go/remoting/communication_mode.go | 3 + rocketmq-go/remoting/custom_header.go | 1 + rocketmq-go/remoting/json_serializable.go | 4 +- .../remoting/json_serializable_test.go | 6 +- rocketmq-go/remoting/remoting_client.go | 16 +- rocketmq-go/remoting/remoting_command.go | 4 +- rocketmq-go/remoting/rocketmq_serializable.go | 16 +- rocketmq-go/remoting/serializable.go | 14 +- .../concurrent_map_test.go} | 16 +- rocketmq-go/util/map_util.go | 29 +++ 40 files changed, 445 insertions(+), 572 deletions(-) create mode 100644 rocketmq-go/example/map_util_a.go delete mode 100644 rocketmq-go/model/client_config.go delete mode 100644 rocketmq-go/model/pull_result.go rename rocketmq-go/{api/model/consume_concurrently_result.go => util/concurrent_map_test.go} (75%) create mode 100644 rocketmq-go/util/map_util.go diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 74ab475e0..10bf0bb8c 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -2,21 +2,32 @@ package rocketmqm import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" +// rocketmq message +// topic: the topic of this message +// tag: the topic of this message, one topic may have no tag or different tag +// key: key makes this message easy to search by console (https://github.com/apache/incubator-rocketmq-externals/rocketmq-console) +// body: the message's user content +// see MessageImpl type Message interface { + // + Topic() (topic string) //set message topic SetTopic(tag string) + //set message tag SetTag(tag string) //get message tag Tag() (tag string) //set message key SetKeys(keys []string) + // set delay time level SetDelayTimeLevel(delayTimeLevel int) // set body SetBody([]byte) } +//create a message instance func NewMessage() (msg Message) { msg = message.NewMessageImpl() return diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 95b0caaf1..36c2ac78b 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,12 +1,14 @@ package rocketmqm +// see MessageExtImpl type MessageExt interface { //get message topic - Topic2() (tag string) + Topic() (tag string) //get message tag - Tag2() (tag string) + Tag() (tag string) // get body - Body2() []byte - MsgId2() string + Body() []byte + // get messageId + MsgId() string } diff --git a/rocketmq-go/api/model/message_listener.go b/rocketmq-go/api/model/message_listener.go index c365f442b..c0436926c 100644 --- a/rocketmq-go/api/model/message_listener.go +++ b/rocketmq-go/api/model/message_listener.go @@ -17,4 +17,22 @@ limitations under the License. package rocketmqm +type ConsumeStatus int + +const ( + //consume success + CONSUME_SUCCESS ConsumeStatus = iota + //consume fail, consume again later + RECONSUME_LATER +) + +// message consume result +type ConsumeConcurrentlyResult struct { + // consume status + ConsumeConcurrentlyStatus ConsumeStatus + //success ack index, from 0 to n-1 + AckIndex int +} + +//mqConsumer.RegisterMessageListener type MessageListener func(msgs []MessageExt) ConsumeConcurrentlyResult diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 4b5813ee7..93a910960 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -22,13 +22,16 @@ package rocketmqm type SerializeType byte const ( + //see json_serializable.go JSON_SERIALIZE SerializeType = iota + //see rocketmq_serializable.go ROCKETMQ_SERIALIZE ) type MqClientConfig struct { // NameServerAddress split by ; - NameServerAddress string + NameServerAddress string + //this client use which serialize type ClientSerializeType SerializeType } diff --git a/rocketmq-go/api/model/mq_consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go index db17e1ee6..fc88b12ee 100644 --- a/rocketmq-go/api/model/mq_consumer_config.go +++ b/rocketmq-go/api/model/mq_consumer_config.go @@ -19,14 +19,10 @@ package rocketmqm import "time" -/** - * Delay some time when exception occur - */ +//Delay some time when exception occur const PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION int64 = 3000 -/** - * Flow control interval - */ +//Flow control interval const PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL int64 = 50 //consume from where @@ -34,6 +30,7 @@ type ConsumeFromWhere int //first consume from the last offset const ( + //first consume from the last offset CONSUME_FROM_LAST_OFFSET ConsumeFromWhere = iota //first consume from the first offset @@ -44,34 +41,26 @@ const ( ) type MqConsumerConfig struct { - /** - * consume from where - */ + + //consume from where ConsumeFromWhere ConsumeFromWhere - /** - * Concurrently max span offset.it has no effect on sequential consumption - */ + + //Concurrently max span offset.it has no effect on sequential consumption ConsumeConcurrentlyMaxSpan int // = 2000; - /** - * Flow control threshold - */ + + //Flow control threshold PullThresholdForQueue int //= 1000; - /** - * MessageImpl pull Interval - */ + + //Message pull Interval PullInterval int64 //= 0; - /** - * Batch consumption size - */ + + //Batch consumption size ConsumeMessageBatchMaxSize int //= 1; - /** - * Batch pull size - */ + + //Batch pull size PullBatchSize int //= 32; - /** - * Whether update subscription relationship when every pull - */ + //Whether update subscription relationship when every pull PostSubscriptionWhenPull bool //= false; //get subExpression /** diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index 735436391..b1b9e3ad1 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -18,8 +18,11 @@ limitations under the License. package rocketmqm type MqProducerConfig struct { - SendMsgTimeout int64 - CompressMsgBodyOverHowMuch int + // SendMsgTimeout for this producer + SendMsgTimeout int64 + // CompressMsgBodyOverHowMuch + CompressMsgBodyOverHowMuch int + // ZipCompressLevel ZipCompressLevel int RetryTimesWhenSendFailed int RetryTimesWhenSendAsyncFailed int diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index a75451d60..22c65f70c 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -23,8 +23,11 @@ import ( ) type MQClientInstance interface { + //Register rocketmq producer to this client instance RegisterProducer(producer MQProducer) + //Register rocketmq consumer to this client instance RegisterConsumer(consumer MQConsumer) + // start this client instance. (register should before start) Start() } @@ -42,11 +45,11 @@ func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmqm. } func (r *ClientInstanceImpl) RegisterProducer(producer MQProducer) { - r.rocketMqManager.RegistProducer(producer.(*manage.DefaultMQProducer)) + r.rocketMqManager.RegisterProducer(producer.(*manage.DefaultMQProducer)) } func (r *ClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { - r.rocketMqManager.RegistConsumer(consumer.(*manage.DefaultMQPushConsumer)) + r.rocketMqManager.RegisterConsumer(consumer.(*manage.DefaultMQPushConsumer)) } func (r *ClientInstanceImpl) Start() { r.rocketMqManager.Start() diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 986e6e55b..8b039f5fe 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -23,14 +23,25 @@ import ( ) type MQConsumer interface { + // register custom's message listener to this consumer RegisterMessageListener(listener rocketmqm.MessageListener) + + // this consumer subscribe which topic, filter tags with subExpression + // subExpression is split by | + // for example. + // consume topic "TestTopic1",consume all message tag + // mqConsumer.Subscribe("TestTopic1","*") + // consume topic "TestTopic2",consume message with tag1 or tag2 + // mqConsumer.Subscribe("TestTopic2","tag1|tag2") Subscribe(topic string, subExpression string) } +// Concurrently(no order) CLUSTERING mq consumer with default config func NewDefaultMQPushConsumer(producerGroup string) (r MQConsumer) { return NewDefaultMQPushConsumerWithCustomConfig(producerGroup, rocketmqm.NewRocketMqConsumerConfig()) } +// Concurrently(no order) CLUSTERING mq consumer with custom config func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (r MQConsumer) { return manage.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index b34262265..b90f9fa07 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -23,15 +23,26 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) +type mqProducerType int + +const ( + //DEFAULT_MQ_PRODUCER + default_mq_producer mqProducerType = iota +) + type MQProducer interface { + //send message,default timeout is 3000 Send(message rocketmqm.Message) (sendResult *model.SendResult, err error) + //send message with custom timeout SendWithTimeout(message rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) } +//mq producer with default config func NewDefaultMQProducer(producerGroup string) (r MQProducer) { return NewDefaultMQProducerWithCustomConfig(producerGroup, rocketmqm.NewProducerConfig()) } +//mq producer with custom config func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (r MQProducer) { return manage.NewDefaultMQProducer(producerGroup, producerConfig) } diff --git a/rocketmq-go/example/map_util_a.go b/rocketmq-go/example/map_util_a.go new file mode 100644 index 000000000..6b5e59a3a --- /dev/null +++ b/rocketmq-go/example/map_util_a.go @@ -0,0 +1,19 @@ +package main + +import ( + "encoding/json" + "fmt" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" +) + +func main() { + config := rocketmqm.NewRocketMqConsumerConfig() + print(structs.Map(config)) + print(util.Struct2Map(config)) +} +func print(m map[string]interface{}) { + bb, _ := json.Marshal(m) + fmt.Println(string(bb)) +} diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 9bfaf8e8a..18a8933b2 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -25,32 +25,5 @@ import ( ) func main() { - //var ( - // nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) - // testTopic = "GoLangRocketMQ" - // testConsumerGroup = "TestConsumerGroup" - //) - //// init rocketMQClientInstance - //rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) - // - //// 1.init rocketMQConsumer - //// 2.subscribe topic and register our function to message listener - //// 3.register it - //var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) - //consumer.Subscribe(testTopic, "*") - //consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { - // successIndex := -1 - // for index, msg := range messageList { - // glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) - // // call your function - // successIndex = index - // } - // return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} - //}) - //rocketMQClientInstance.RegisterConsumer(consumer) - // - //// start rocketMQ client instance - //rocketMQClientInstance.Start() - // - //select {} + } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 89fac3d6f..0e1a803e0 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -44,7 +44,7 @@ func main() { consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId2(), string(msg.Body2())) + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId(), string(msg.Body())) // call your function successIndex = index } @@ -60,6 +60,7 @@ func main() { var message = rocketmqm.NewMessage() message.SetTopic(testTopic) message.SetBody([]byte("hello World")) + glog.Info("here") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index a105a8362..a530b6723 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -37,7 +37,7 @@ type ConsumeMessageService interface { type ConsumeMessageConcurrentlyServiceImpl struct { consumerGroup string messageListener rocketmqm.MessageListener - sendMessageBackProducerService SendMessageBackProducerService //for send retry MessageImpl + sendMessageBackProducerService sendMessageBackProducerService //for send retry MessageImpl offsetStore OffsetStore consumerConfig *rocketmqm.MqConsumerConfig } @@ -149,10 +149,10 @@ func transformMessageToConsume(consumerGroup string, msgs []message.MessageExtIm for _, msg := range msgs { //reset retry topic name - if msg.MessageImpl.Topic == retryTopicName { - retryTopic := msg.Properties[constant.PROPERTY_RETRY_TOPIC] + if msg.MessageImpl.Topic() == retryTopicName { + retryTopic := msg.PropertiesKeyValue(constant.PROPERTY_RETRY_TOPIC) if len(retryTopic) > 0 { - msg.MessageImpl.Topic = retryTopic + msg.MessageImpl.SetTopic(retryTopic) } } //set consume start time diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 6ed480677..69c414090 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -40,49 +40,66 @@ import ( //3.subscribeInfo //4.heartbeat type RocketMqClient interface { + //get mqClient's clientId ip@pid GetClientId() (clientId string) + //get remoting client in mqClient GetRemotingClient() (remotingClient *remoting.DefaultRemotingClient) + //get topic subscribe info GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) + //GetPublishTopicList GetPublishTopicList() []string + //FetchMasterBrokerAddress FetchMasterBrokerAddress(brokerName string) (masterAddress string) + //EnqueuePullMessageRequest EnqueuePullMessageRequest(pullRequest *model.PullRequest) + //DequeuePullMessageRequest DequeuePullMessageRequest() (pullRequest *model.PullRequest) + //FindBrokerAddressInSubscribe FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) + //TryToFindTopicPublishInfo TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) + //FindBrokerAddrByTopic FindBrokerAddrByTopic(topic string) (addr string, ok bool) + //UpdateTopicRouteInfoFromNameServer UpdateTopicRouteInfoFromNameServer(topic string) (err error) + //UpdateTopicRouteInfoFromNameServerUseDefaultTopic UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) + //SendHeartbeatToAllBroker SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) + //ClearExpireResponse ClearExpireResponse() + //GetMaxOffset GetMaxOffset(mq *model.MessageQueue) int64 + //SearchOffset SearchOffset(mq *model.MessageQueue, time time.Time) int64 } var DEFAULT_TIMEOUT int64 = 6000 type MqClientImpl struct { - ClientId string + clientId string remotingClient *remoting.DefaultRemotingClient - TopicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData - BrokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address - TopicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this - TopicSubscribeInfoTable util.ConcurrentMap //map[string][]*model.MessageQueue //topic | MessageQueue - PullRequestQueue chan *model.PullRequest + topicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData + brokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address + topicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this + topicSubscribeInfoTable util.ConcurrentMap //map[string][]*model.MessageQueue //topic | MessageQueue + pullRequestQueue chan *model.PullRequest } +// create a MqClientInit instance func MqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} - mqClientImpl.ClientId = buildMqClientImplId() - mqClientImpl.TopicRouteTable = util.New() // make(map[string]*model.TopicRouteData) - mqClientImpl.BrokerAddrTable = util.New() //make(map[string]map[int]string) + mqClientImpl.clientId = buildMqClientImplId() + mqClientImpl.topicRouteTable = util.New() // make(map[string]*model.TopicRouteData) + mqClientImpl.brokerAddrTable = util.New() //make(map[string]map[int]string) mqClientImpl.remotingClient = remoting.RemotingClientInit(clientConfig, clientRequestProcessor) - mqClientImpl.TopicPublishInfoTable = util.New() //make(map[string]*model.TopicPublishInfo) - mqClientImpl.TopicSubscribeInfoTable = util.New() //make(map[string][]*model.MessageQueue) - mqClientImpl.PullRequestQueue = make(chan *model.PullRequest, 1024) + mqClientImpl.topicPublishInfoTable = util.New() //make(map[string]*model.TopicPublishInfo) + mqClientImpl.topicSubscribeInfoTable = util.New() //make(map[string][]*model.MessageQueue) + mqClientImpl.pullRequestQueue = make(chan *model.PullRequest, 1024) return } func (m *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) { - value, ok := m.TopicSubscribeInfoTable.Get(topic) + value, ok := m.topicSubscribeInfoTable.Get(topic) if ok { messageQueueList = value.([]*model.MessageQueue) } @@ -123,11 +140,11 @@ func (m *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) int6 return queryOffsetResponseHeader.Offset } func (m *MqClientImpl) GetClientId() string { - return m.ClientId + return m.clientId } func (m *MqClientImpl) GetPublishTopicList() []string { var publishTopicList []string - for _, topic := range m.TopicPublishInfoTable.Keys() { + for _, topic := range m.topicPublishInfoTable.Keys() { publishTopicList = append(publishTopicList, topic) } return publishTopicList @@ -137,10 +154,10 @@ func (m *MqClientImpl) GetRemotingClient() *remoting.DefaultRemotingClient { } func (m *MqClientImpl) EnqueuePullMessageRequest(pullRequest *model.PullRequest) { - m.PullRequestQueue <- pullRequest + m.pullRequestQueue <- pullRequest } func (m *MqClientImpl) DequeuePullMessageRequest() (pullRequest *model.PullRequest) { - pullRequest = <-m.PullRequestQueue + pullRequest = <-m.pullRequestQueue return } @@ -149,25 +166,25 @@ func (m *MqClientImpl) ClearExpireResponse() { } func (m *MqClientImpl) FetchMasterBrokerAddress(brokerName string) (masterAddress string) { - value, ok := m.BrokerAddrTable.Get(brokerName) + value, ok := m.brokerAddrTable.Get(brokerName) if ok { masterAddress = value.(map[string]string)["0"] } return } func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) { - value, ok := m.TopicPublishInfoTable.Get(topic) + value, ok := m.topicPublishInfoTable.Get(topic) if ok { topicPublicInfo = value.(*model.TopicPublishInfo) } if topicPublicInfo == nil || !topicPublicInfo.JudgeTopicPublishInfoOk() { - m.TopicPublishInfoTable.Set(topic, &model.TopicPublishInfo{HaveTopicRouterInfo: false}) + m.topicPublishInfoTable.Set(topic, &model.TopicPublishInfo{HaveTopicRouterInfo: false}) err = m.UpdateTopicRouteInfoFromNameServer(topic) if err != nil { glog.Warning(err) // if updateRouteInfo error, maybe we can use the defaultTopic } - value, ok := m.TopicPublishInfoTable.Get(topic) + value, ok := m.topicPublishInfoTable.Get(topic) if ok { topicPublicInfo = value.(*model.TopicPublishInfo) } @@ -178,7 +195,7 @@ func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo //try to use the defaultTopic err = m.UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic) - defaultValue, defaultValueOk := m.TopicPublishInfoTable.Get(topic) + defaultValue, defaultValueOk := m.topicPublishInfoTable.Get(topic) if defaultValueOk { topicPublicInfo = defaultValue.(*model.TopicPublishInfo) } @@ -216,7 +233,7 @@ func (m MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMilli func (m MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { slave = false found = false - value, ok := m.BrokerAddrTable.Get(brokerName) + value, ok := m.brokerAddrTable.Get(brokerName) if !ok { return } @@ -280,21 +297,21 @@ func (m MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData *mo } //update brokerAddrTable for _, brokerData := range topicRouteData.BrokerDatas { - m.BrokerAddrTable.Set(brokerData.BrokerName, brokerData.BrokerAddrs) + m.brokerAddrTable.Set(brokerData.BrokerName, brokerData.BrokerAddrs) } //update pubInfo for each topicPublishInfo := model.BuildTopicPublishInfoFromTopicRoteData(topic, topicRouteData) - m.TopicPublishInfoTable.Set(topic, topicPublishInfo) + m.topicPublishInfoTable.Set(topic, topicPublishInfo) mqList := model.BuildTopicSubscribeInfoFromRoteData(topic, topicRouteData) - m.TopicSubscribeInfoTable.Set(topic, mqList) - m.TopicRouteTable.Set(topic, topicRouteData) + m.topicSubscribeInfoTable.Set(topic, mqList) + m.topicRouteTable.Set(topic, topicRouteData) return } func (m MqClientImpl) FindBrokerAddrByTopic(topic string) (addr string, ok bool) { - value, findValue := m.TopicRouteTable.Get(topic) + value, findValue := m.topicRouteTable.Get(topic) if !findValue { return "", false } @@ -334,7 +351,7 @@ func (m MqClientImpl) sendHeartBeat(addr string, remotingCommand *remoting.Remot } func (m MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) { - for _, brokerTable := range m.BrokerAddrTable.Items() { + for _, brokerTable := range m.brokerAddrTable.Items() { for brokerId, addr := range brokerTable.(map[string]string) { if len(addr) == 0 || brokerId != "0" { continue diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index 665d933fd..02b7c43fd 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -34,9 +34,13 @@ const ( ) type OffsetStore interface { + //update local offsetTable's offset UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) + //read offset,from memory or broker ReadOffset(mq *model.MessageQueue, readType int) int64 + //update broker's offset Persist(mq *model.MessageQueue) + //remove local offsetTable's offset RemoveOffset(mq *model.MessageQueue) } type RemoteOffsetStore struct { diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 6e10cf12a..dcb32a90c 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -63,12 +63,12 @@ func (d *DefaultProducerService) SendDefaultImpl(message *message.MessageImpl, c if err != nil { return } - topicPublishInfo, err = d.mqClient.TryToFindTopicPublishInfo(message.Topic) + topicPublishInfo, err = d.mqClient.TryToFindTopicPublishInfo(message.Topic()) if err != nil { return } if topicPublishInfo.JudgeTopicPublishInfoOk() == false { - err = errors.New("topicPublishInfo is error,topic=" + message.Topic) + err = errors.New("topicPublishInfo is error,topic=" + message.Topic()) return } glog.V(2).Info("op=look topicPublishInfo", topicPublishInfo) @@ -78,7 +78,7 @@ func (d *DefaultProducerService) SendDefaultImpl(message *message.MessageImpl, c } func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { - remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body) + remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body()) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) if err != nil { @@ -119,11 +119,11 @@ func processSendResponse(brokerName string, message *message.MessageImpl, respon if response.ExtFields != nil { responseHeader.FromMap(response.ExtFields) //change map[string]interface{} into CustomerHeader struct } - sendResult.SetMsgID(message.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) + sendResult.SetMsgID(message.PropertiesKeyValue(constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX)) sendResult.SetOffsetMsgID(responseHeader.MsgId) sendResult.SetQueueOffset(responseHeader.QueueOffset) sendResult.SetTransactionID(responseHeader.TransactionId) - messageQueue := model.MessageQueue{Topic: message.Topic, BrokerName: brokerName, + messageQueue := model.MessageQueue{Topic: message.Topic(), BrokerName: brokerName, QueueId: responseHeader.QueueId} sendResult.SetMessageQueue(messageQueue) var regionId = responseHeader.MsgRegion @@ -139,29 +139,29 @@ func (d *DefaultProducerService) checkMessage(message *message.MessageImpl) (err err = errors.New("message is nil") return } - if len(message.Topic) == 0 { + if len(message.Topic()) == 0 { err = errors.New("topic is empty") return } - if message.Topic == constant.DEFAULT_TOPIC { - err = errors.New("the topic[" + message.Topic + "] is conflict with default topic.") + if message.Topic() == constant.DEFAULT_TOPIC { + err = errors.New("the topic[" + message.Topic() + "] is conflict with default topic.") return } - if len(message.Topic) > constant.MAX_MESSAGE_TOPIC_SIZE { + if len(message.Topic()) > constant.MAX_MESSAGE_TOPIC_SIZE { err = errors.New("the specified topic is longer than topic max length 255.") return } - if !util.MatchString(message.Topic, `^[%|a-zA-Z0-9_-]+$`) { - err = errors.New("the specified topic[" + message.Topic + "] contains illegal characters") + if !util.MatchString(message.Topic(), `^[%|a-zA-Z0-9_-]+$`) { + err = errors.New("the specified topic[" + message.Topic() + "] contains illegal characters") return } - if len(message.Body) == 0 { + if len(message.Body()) == 0 { err = errors.New("messageBody is empty") return } - if len(message.Body) > d.producerConfig.MaxMessageSize { + if len(message.Body()) > d.producerConfig.MaxMessageSize { err = errors.New("messageBody is large than " + util.IntToString(d.producerConfig.MaxMessageSize)) return } @@ -214,14 +214,14 @@ func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, mes message.GeneratorMsgUniqueKey() sendMessageHeader := &header.SendMessageRequestHeader{ ProducerGroup: d.producerGroup, - Topic: message.Topic, + Topic: message.Topic(), DefaultTopic: constant.DEFAULT_TOPIC, DefaultTopicQueueNums: 4, QueueId: messageQueue.QueueId, SysFlag: sysFlag, BornTimestamp: util.CurrentTimeMillisInt64(), - Flag: message.Flag, - Properties: util.MessageProperties2String(message.Properties), + Flag: message.Flag(), + Properties: util.MessageProperties2String(message.Properties()), UnitMode: false, ReconsumeTimes: message.GetReconsumeTimes(), @@ -232,11 +232,13 @@ func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, mes } func (d *DefaultProducerService) tryToCompressMessage(message *message.MessageImpl) (compressedFlag int, err error) { - if len(message.Body) < d.producerConfig.CompressMsgBodyOverHowMuch { + if len(message.Body()) < d.producerConfig.CompressMsgBodyOverHowMuch { compressedFlag = 0 return } compressedFlag = int(constant.CompressedFlag) - message.Body, err = util.CompressWithLevel(message.Body, d.producerConfig.ZipCompressLevel) + var compressBody []byte + compressBody, err = util.CompressWithLevel(message.Body(), d.producerConfig.ZipCompressLevel) + message.SetBody(compressBody) return } diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 1bb11b12b..bc0f5c25a 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -28,7 +28,7 @@ import ( "github.com/golang/glog" ) -type SendMessageBackProducerService interface { +type sendMessageBackProducerService interface { SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) } @@ -42,7 +42,7 @@ type SendMessageBackProducerServiceImpl struct { // send to original broker,if fail send a new retry message func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { - glog.V(2).Info("op=look_send_message_back", messageExt.MsgId, messageExt.Properties, string(messageExt.Body)) + glog.V(2).Info("op=look_send_message_back", messageExt.MsgId(), messageExt.Properties(), string(messageExt.Body())) err = s.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) if err == nil { return @@ -55,17 +55,17 @@ func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *message func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *message.MessageExtImpl) error { retryMessage := &message.MessageImpl{} originMessageId := messageExt.GetOriginMessageId() - retryMessage.Properties = messageExt.Properties + retryMessage.SetProperties(messageExt.Properties()) retryMessage.SetOriginMessageId(originMessageId) - retryMessage.Flag = messageExt.Flag - retryMessage.Topic = constant.RETRY_GROUP_TOPIC_PREFIX + s.consumerGroup - retryMessage.Body = messageExt.Body - retryMessage.SetRetryTopic(messageExt.Topic) + retryMessage.SetFlag(messageExt.Flag()) + retryMessage.SetTopic(constant.RETRY_GROUP_TOPIC_PREFIX + s.consumerGroup) + retryMessage.SetBody(messageExt.Body()) + retryMessage.SetRetryTopic(messageExt.Topic()) retryMessage.SetReconsumeTime(messageExt.GetReconsumeTimes() + 1) retryMessage.SetMaxReconsumeTimes(s.consumerConfig.MaxReconsumeTimes) retryMessage.SetDelayTimeLevel(3 + messageExt.GetReconsumeTimes()) pp, _ := json.Marshal(retryMessage) - glog.Info("look retryMessage ", string(pp), string(messageExt.Body)) + glog.Info("look retryMessage ", string(pp), string(messageExt.Body())) sendResult, err := s.defaultProducerService.SendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", s.defaultProducerService.producerConfig.SendMsgTimeout) if err != nil { glog.Error(err) @@ -95,8 +95,8 @@ func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName Offset: messageExt.CommitLogOffset, Group: s.consumerGroup, DelayLevel: 0, //MessageImpl consume retry strategy
-1,no retry,put into DLQ directly
0,broker control retry frequency
>0,client control retry frequency - OriginMsgId: messageExt.MsgId, - OriginTopic: messageExt.Topic, + OriginMsgId: messageExt.MsgId(), + OriginTopic: messageExt.Topic(), UnitMode: false, MaxReconsumeTimes: int32(s.consumerConfig.MaxReconsumeTimes), } diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index f61547151..fdfc6ced2 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -48,6 +48,7 @@ type Rebalance struct { consumerConfig *rocketmqm.MqConsumerConfig } +//when invoke GET_CONSUMER_RUNNING_INFO, GetMqTableInfo will return ProcessQueueInfo func (r *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() diff --git a/rocketmq-go/manage/clean_expire_msg_controller.go b/rocketmq-go/manage/clean_expire_msg_controller.go index 3a2f9eef7..ff67268e3 100644 --- a/rocketmq-go/manage/clean_expire_msg_controller.go +++ b/rocketmq-go/manage/clean_expire_msg_controller.go @@ -22,20 +22,20 @@ import ( "time" ) -type CleanExpireMsgController struct { +type cleanExpireMsgController struct { mqClient kernel.RocketMqClient - clientFactory *ClientFactory + clientFactory *clientFactory } -func NewCleanExpireMsgController(mqClient kernel.RocketMqClient, clientFactory *ClientFactory) *CleanExpireMsgController { - return &CleanExpireMsgController{ +func newCleanExpireMsgController(mqClient kernel.RocketMqClient, clientFactory *clientFactory) *cleanExpireMsgController { + return &cleanExpireMsgController{ mqClient: mqClient, clientFactory: clientFactory, } } -func (self *CleanExpireMsgController) Start() { - for _, consumer := range self.clientFactory.ConsumerTable { +func (self *cleanExpireMsgController) start() { + for _, consumer := range self.clientFactory.consumerTable { go func() { cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) //cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index ee194eb5c..e291900ec 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -27,23 +27,24 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strings" "sync" "time" ) +//type MqClientManager interface { +// +//} + type MqClientManager struct { rocketMqManagerLock sync.Mutex BootTimestamp int64 - clientFactory *ClientFactory - NamesrvLock sync.Mutex - HeartBeatLock sync.Mutex + clientFactory *clientFactory mqClient kernel.RocketMqClient - ServiceState int pullMessageController *PullMessageController - cleanExpireMsgController *CleanExpireMsgController + cleanExpireMsgController *cleanExpireMsgController rebalanceControllr *RebalanceController defaultProducerService *kernel.DefaultProducerService } @@ -52,14 +53,39 @@ func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManage rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() rocketMqManager.clientFactory = ClientFactoryInit() - rocketMqManager.mqClient = kernel.MqClientInit(clientConfig, rocketMqManager.InitClientRequestProcessor()) // todo todo todo + rocketMqManager.mqClient = kernel.MqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) - rocketMqManager.cleanExpireMsgController = NewCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) + rocketMqManager.cleanExpireMsgController = newCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) return } -func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { +func (m *MqClientManager) Start() { + //d.sendHeartbeatToAllBrokerWithLock()//we should send heartbeat first todo check + m.StartAllScheduledTask() +} + +func (m *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { + producer.producerService = kernel.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) + m.clientFactory.producerTable[producer.producerGroup] = producer + return +} + +func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { + if m.defaultProducerService == nil { + m.defaultProducerService = kernel.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) + } + consumer.mqClient = m.mqClient + consumer.offsetStore = kernel.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) + m.clientFactory.consumerTable[consumer.consumerGroup] = consumer + consumer.rebalance = kernel.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) + + fmt.Println(consumer.consumeMessageService) + + consumer.consumeMessageService.Init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) + return +} +func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { clientRequestProcessor = func(cmd *remoting.RemotingCommand) (response *remoting.RemotingCommand) { switch cmd.Code { case remoting.CHECK_TRANSACTION_STATE: @@ -95,8 +121,8 @@ func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor r getConsumerRunningInfoRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct consumerRunningInfo := model.ConsumerRunningInfo{} consumerRunningInfo.Properties = map[string]string{} - defaultMQPushConsumer := m.clientFactory.ConsumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] - consumerConfigMap := structs.Map(defaultMQPushConsumer.ConsumerConfig) + defaultMQPushConsumer := m.clientFactory.consumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] + consumerConfigMap := util.Struct2Map(defaultMQPushConsumer.ConsumerConfig) for key, value := range consumerConfigMap { consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) } @@ -122,7 +148,7 @@ func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor r consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) messageExt := &DecodeMessage(cmd.Body)[0] glog.V(2).Info("op=look", messageExt) - defaultMQPushConsumer := m.clientFactory.ConsumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] + defaultMQPushConsumer := m.clientFactory.consumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] consumeResult, err := defaultMQPushConsumer.consumeMessageService.ConsumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) if err != nil { return @@ -141,54 +167,30 @@ func (m *MqClientManager) InitClientRequestProcessor() (clientRequestProcessor r } return } -func (m *MqClientManager) RegistProducer(producer *DefaultMQProducer) { - producer.producerService = kernel.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) - m.clientFactory.ProducerTable[producer.producerGroup] = producer - return -} func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { - consumer := m.clientFactory.ConsumerTable[group] + consumer := m.clientFactory.consumerTable[group] if consumer == nil { glog.Error("resetConsumerOffset because consumer not online,group=", group) return } consumer.resetOffset(offsetTable) } -func (m *MqClientManager) RegistConsumer(consumer *DefaultMQPushConsumer) { - if m.defaultProducerService == nil { - m.defaultProducerService = kernel.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) - } - consumer.mqClient = m.mqClient - consumer.offsetStore = kernel.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) - m.clientFactory.ConsumerTable[consumer.consumerGroup] = consumer - consumer.rebalance = kernel.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) - - fmt.Println(consumer.consumeMessageService) - - consumer.consumeMessageService.Init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) - return -} - -func (m *MqClientManager) Start() { - //d.SendHeartbeatToAllBrokerWithLock()//we should send heartbeat first todo check - m.StartAllScheduledTask() -} -type ClientFactory struct { - ProducerTable map[string]*DefaultMQProducer //group|RocketMQProducer - ConsumerTable map[string]*DefaultMQPushConsumer //group|Consumer +type clientFactory struct { + producerTable map[string]*DefaultMQProducer //group|RocketMQProducer + consumerTable map[string]*DefaultMQPushConsumer //group|Consumer } -func ClientFactoryInit() (clientFactory *ClientFactory) { - clientFactory = &ClientFactory{} - clientFactory.ProducerTable = make(map[string]*DefaultMQProducer) - clientFactory.ConsumerTable = make(map[string]*DefaultMQPushConsumer) +func ClientFactoryInit() (clientFactoryInstance *clientFactory) { + clientFactoryInstance = &clientFactory{} + clientFactoryInstance.producerTable = make(map[string]*DefaultMQProducer) + clientFactoryInstance.consumerTable = make(map[string]*DefaultMQPushConsumer) return } //heart beat -func (m MqClientManager) SendHeartbeatToAllBrokerWithLock() error { +func (m MqClientManager) sendHeartbeatToAllBrokerWithLock() error { heartbeatData := m.prepareHeartbeatData() if len(heartbeatData.ConsumerDataSet) == 0 { return errors.New("send heartbeat error") @@ -198,9 +200,9 @@ func (m MqClientManager) SendHeartbeatToAllBrokerWithLock() error { } //routeInfo -func (m MqClientManager) UpdateTopicRouteInfoFromNameServer() { +func (m MqClientManager) updateTopicRouteInfoFromNameServer() { var topicSet []string - for _, consumer := range m.clientFactory.ConsumerTable { + for _, consumer := range m.clientFactory.consumerTable { for key, _ := range consumer.subscription { topicSet = append(topicSet, key) } @@ -217,7 +219,7 @@ func (m MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData.ClientId = m.mqClient.GetClientId() heartbeatData.ConsumerDataSet = make([]*model.ConsumerData, 0) heartbeatData.ProducerDataSet = make([]*model.ProducerData, 0) - for group, consumer := range m.clientFactory.ConsumerTable { + for group, consumer := range m.clientFactory.consumerTable { consumerData := new(model.ConsumerData) consumerData.GroupName = group consumerData.ConsumeType = consumer.consumeType @@ -227,7 +229,7 @@ func (m MqClientManager) prepareHeartbeatData() *model.HeartbeatData { consumerData.UnitMode = consumer.unitMode heartbeatData.ConsumerDataSet = append(heartbeatData.ConsumerDataSet, consumerData) } - for group := range m.clientFactory.ProducerTable { + for group := range m.clientFactory.producerTable { producerData := new(model.ProducerData) producerData.GroupName = group heartbeatData.ProducerDataSet = append(heartbeatData.ProducerDataSet, producerData) diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index c65602f8c..7b6383840 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -135,7 +135,7 @@ func (d *DefaultMQPushConsumer) CleanExpireMsg() { if nowTime-consumeStartTime < maxDiffTime { break } - glog.Info("look now we send expire message back", message.Topic, message.MsgId) + glog.Info("look now we send expire message back", message.Topic(), message.MsgId()) err := d.consumeMessageService.SendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) if err != nil { glog.Error("op=send_expire_message_back_error", err) diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index ff0919b95..d50a6aa2c 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -35,10 +35,10 @@ import ( type PullMessageController struct { mqClient kernel.RocketMqClient - clientFactory *ClientFactory + clientFactory *clientFactory } -func NewPullMessageController(mqClient kernel.RocketMqClient, clientFactory *ClientFactory) *PullMessageController { +func NewPullMessageController(mqClient kernel.RocketMqClient, clientFactory *clientFactory) *PullMessageController { return &PullMessageController{ mqClient: mqClient, clientFactory: clientFactory, @@ -74,7 +74,7 @@ func (p *PullMessageController) pullMessageLater(pullRequest *model.PullRequest, } func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { - defaultMQPullConsumer := p.clientFactory.ConsumerTable[pullRequest.ConsumerGroup] + defaultMQPullConsumer := p.clientFactory.consumerTable[pullRequest.ConsumerGroup] if pullRequest.ProcessQueue.IsDropped() { return } @@ -288,7 +288,7 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { return nil } - msg.Topic = string(topic) + msg.SetTopic(string(topic)) msg.QueueId = queueId msg.SysFlag = sysFlag msg.QueueOffset = queueOffset @@ -296,17 +296,17 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { msg.StoreSize = storeSize msg.BornTimestamp = bornTimeStamp msg.ReconsumeTimes = reconsumeTimes - msg.Flag = int(flag) + msg.SetFlag(int(flag)) msg.CommitLogOffset = physicOffset msg.StoreTimestamp = storeTimestamp msg.PreparedTransactionOffset = preparedTransactionOffset - msg.Body = body - msg.Properties = propertiesmap + msg.SetBody(body) + msg.SetProperties(propertiesmap) // < 3.5.8 use messageOffsetId // >= 3.5.8 use clientUniqMsgId - msg.MsgId = msg.GetMsgUniqueKey() - if len(msg.MsgId) == 0 { - msg.MsgId = util.GeneratorMessageOffsetId(storeHost, storePort, msg.CommitLogOffset) + msg.SetMsgId(msg.GetMsgUniqueKey()) + if len(msg.MsgId()) == 0 { + msg.SetMsgId(util.GeneratorMessageOffsetId(storeHost, storePort, msg.CommitLogOffset)) } msgs = append(msgs, msg) } diff --git a/rocketmq-go/manage/rebalance_controller.go b/rocketmq-go/manage/rebalance_controller.go index b1f9d864f..c1e82a9a6 100644 --- a/rocketmq-go/manage/rebalance_controller.go +++ b/rocketmq-go/manage/rebalance_controller.go @@ -18,17 +18,17 @@ limitations under the License. package manage type RebalanceController struct { - clientFactory *ClientFactory + clientFactory *clientFactory } -func NewRebalanceController(clientFactory *ClientFactory) *RebalanceController { +func NewRebalanceController(clientFactory *clientFactory) *RebalanceController { return &RebalanceController{ clientFactory: clientFactory, } } func (self *RebalanceController) doRebalance() { - for _, consumer := range self.clientFactory.ConsumerTable { + for _, consumer := range self.clientFactory.consumerTable { consumer.rebalance.DoRebalance() } } diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/manage/tasks.go index f5b6f43a3..9be3decd4 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/manage/tasks.go @@ -31,7 +31,7 @@ func (m MqClientManager) StartAllScheduledTask() { updateTopicRouteTimer := time.NewTimer(5 * time.Second) for { <-updateTopicRouteTimer.C - m.UpdateTopicRouteInfoFromNameServer() + m.updateTopicRouteInfoFromNameServer() updateTopicRouteTimer.Reset(5 * time.Second) } }() @@ -40,7 +40,7 @@ func (m MqClientManager) StartAllScheduledTask() { heartbeatTimer := time.NewTimer(10 * time.Second) for { <-heartbeatTimer.C - m.SendHeartbeatToAllBrokerWithLock() + m.sendHeartbeatToAllBrokerWithLock() heartbeatTimer.Reset(5 * time.Second) } }() @@ -65,5 +65,5 @@ func (m MqClientManager) StartAllScheduledTask() { m.pullMessageController.Start() //cleanExpireMsg - m.cleanExpireMsgController.Start() + m.cleanExpireMsgController.start() } diff --git a/rocketmq-go/model/client_config.go b/rocketmq-go/model/client_config.go deleted file mode 100644 index ca58dfb45..000000000 --- a/rocketmq-go/model/client_config.go +++ /dev/null @@ -1,171 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -// -//import ( -// "bytes" -// "time" -//) -// -//// client common config -//type ClientConfig struct { -// nameServerAddress string // only this is in use -// -// clientIP string -// instanceName string -// clientCallbackExecutorThreads int // TODO: clientCallbackExecutorThreads -// // Pulling topic information interval from the named server -// pullNameServerInterval time.Duration // default 30 -// // Heartbeat interval in microseconds with message broker -// heartbeatBrokerInterval time.Duration // default 30 -// // Offset persistent interval for consumer -// persistConsumerOffsetInterval time.Duration // default 5 -// unitMode bool -// unitName string -// vipChannelEnabled bool -//} -// -//func NewClientConfig() *ClientConfig { -// return &ClientConfig{ -// unitMode: false, -// pullNameServerInterval: time.Second * 30, -// heartbeatBrokerInterval: time.Second * 30, -// persistConsumerOffsetInterval: time.Second * 30, -// } -//} -// -//func (config *ClientConfig) BuildMQClientId() string { -// var buffer bytes.Buffer -// buffer.WriteString(config.clientIP) -// buffer.WriteString("@") -// buffer.WriteString(config.instanceName) -// if config.unitName != "" { -// buffer.WriteString("@") -// buffer.WriteString(config.unitName) -// } -// return buffer.String() -//} -// -//func (config *ClientConfig) ChangeInstanceNameToPID() { -// // TODO -//} -// -//func (config *ClientConfig) ResetClientConfig(cfg *ClientConfig) { -// // TODO -//} -// -//func (config *ClientConfig) CloneClientConfig() *ClientConfig { -// return &ClientConfig{ -// nameServerAddress: config.nameServerAddress, -// clientIP: config.clientIP, -// instanceName: config.instanceName, -// clientCallbackExecutorThreads: config.clientCallbackExecutorThreads, -// pullNameServerInterval: config.pullNameServerInterval, -// heartbeatBrokerInterval: config.heartbeatBrokerInterval, -// persistConsumerOffsetInterval: config.persistConsumerOffsetInterval, -// unitMode: config.unitMode, -// unitName: config.unitName, -// vipChannelEnabled: config.vipChannelEnabled, -// } -//} -// -//func (config *ClientConfig) ClientIP() string { -// return config.clientIP -//} -// -//func (config *ClientConfig) SetClientIP(s string) { -// config.clientIP = s -//} -// -//func (config *ClientConfig) InstanceName() string { -// return config.instanceName -//} -// -//func (config *ClientConfig) SetInstanceName(s string) { -// config.instanceName = s -//} -// -//func (config *ClientConfig) NameServerAddress() string { -// return config.nameServerAddress -//} -// -//func (config *ClientConfig) SetNameServerAddress(s string) { -// config.nameServerAddress = s -//} -// -//func (config *ClientConfig) ClientCallbackExecutorThreads() int { -// return config.clientCallbackExecutorThreads -//} -// -//func (config *ClientConfig) SetClientCallbackExecutorThreads(threads int) { -// config.clientCallbackExecutorThreads = threads -//} -// -//func (config *ClientConfig) PullNameServerInteval() time.Duration { -// return config.pullNameServerInterval -//} -// -//func (config *ClientConfig) SetPullNameServerInteval(interval time.Duration) { -// config.pullNameServerInterval = interval -//} -// -//func (config *ClientConfig) HeartbeatBrokerInterval() time.Duration { -// return config.heartbeatBrokerInterval -//} -// -//func (config *ClientConfig) SetHeartbeatBrokerInterval(interval time.Duration) { -// config.heartbeatBrokerInterval = interval -//} -// -//func (config *ClientConfig) PersistConsumerOffsetInterval() time.Duration { -// return config.persistConsumerOffsetInterval -//} -// -//func (config *ClientConfig) SetPersistConsumerOffsetInterval(interval time.Duration) { -// config.persistConsumerOffsetInterval = interval -//} -// -//func (config *ClientConfig) UnitName() string { -// return config.unitName -//} -// -//func (config *ClientConfig) SetUnitName(name string) { -// config.unitName = name -//} -// -//func (config *ClientConfig) UnitMode() bool { -// return config.unitMode -//} -// -//func (config *ClientConfig) SetUnitMode(mode bool) { -// config.unitMode = mode -//} -// -//func (config *ClientConfig) VipChannelEnabled() bool { -// return config.vipChannelEnabled -//} -// -//func (config *ClientConfig) SetVipChannelEnabled(enable bool) { -// config.vipChannelEnabled = enable -//} -// -//func (config *ClientConfig) String() string { -// //TODO -// return "" -//} diff --git a/rocketmq-go/model/message/message_ext_impl.go b/rocketmq-go/model/message/message_ext_impl.go index f17f396d8..05f5ce2b6 100644 --- a/rocketmq-go/model/message/message_ext_impl.go +++ b/rocketmq-go/model/message/message_ext_impl.go @@ -25,53 +25,40 @@ import ( type MessageExtImpl struct { *MessageImpl - QueueId int32 - StoreSize int32 - QueueOffset int64 - SysFlag int32 - BornTimestamp int64 - BornHost string - StoreTimestamp int64 - StoreHost string - MsgId string - CommitLogOffset int64 - BodyCRC int32 - ReconsumeTimes int32 - PreparedTransactionOffset int64 - + msgId string + QueueId int32 + StoreSize int32 + QueueOffset int64 + SysFlag int32 + BornTimestamp int64 + BornHost string + StoreTimestamp int64 + StoreHost string + CommitLogOffset int64 + BodyCRC int32 + ReconsumeTimes int32 + PreparedTransactionOffset int64 propertyConsumeStartTimestamp string } -//get message topic -func (m *MessageExtImpl) Topic2() (topic string) { - topic = m.Topic - return -} - -//get message tag -func (m *MessageExtImpl) Tag2() (tag string) { - tag = m.Tag() +func (m *MessageExtImpl) MsgId() (msgId string) { + msgId = m.msgId return } -// get body -func (m *MessageExtImpl) Body2() (body []byte) { - body = m.Body - return -} -func (m *MessageExtImpl) MsgId2() (msgId string) { - msgId = m.MsgId +func (m *MessageExtImpl) SetMsgId(msgId string) { + m.msgId = msgId return } func (m *MessageExtImpl) GetOriginMessageId() string { - if m.Properties != nil { - originMessageId := m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] + if m.properties != nil { + originMessageId := m.properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] if len(originMessageId) > 0 { return originMessageId } } - return m.MsgId + return m.msgId } func (m *MessageExtImpl) GetConsumeStartTime() int64 { @@ -82,11 +69,11 @@ func (m *MessageExtImpl) GetConsumeStartTime() int64 { } func (m *MessageExtImpl) SetConsumeStartTime() { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } nowTime := util.CurrentTimeMillisStr() - m.Properties[constant.PROPERTY_KEYS] = nowTime + m.properties[constant.PROPERTY_KEYS] = nowTime m.propertyConsumeStartTimestamp = nowTime return } diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go index 93f63b405..d007b635c 100644 --- a/rocketmq-go/model/message/message_impl.go +++ b/rocketmq-go/model/message/message_impl.go @@ -24,115 +24,147 @@ import ( ) type MessageImpl struct { - Topic string - Flag int - Properties map[string]string - Body []byte + topic string + flag int + properties map[string]string + body []byte } func NewMessageImpl() (message *MessageImpl) { message = &MessageImpl{} return } +func (m *MessageImpl) Properties() (properties map[string]string) { + properties = m.properties + return +} +func (m *MessageImpl) SetProperties(properties map[string]string) { + m.properties = properties + return +} +func (m *MessageImpl) PropertiesKeyValue(key string) (value string) { + value = m.properties[key] + return +} + +func (m *MessageImpl) Body() (body []byte) { + body = m.body + return + +} +func (m *MessageImpl) Topic() (topic string) { + topic = m.topic + return + +} +func (m *MessageImpl) SetFlag(flag int) { + m.flag = flag + return +} +func (m *MessageImpl) Flag() (flag int) { + flag = m.flag + return + +} func (m *MessageImpl) SetTopic(topic string) { - m.Topic = topic + m.topic = topic } func (m *MessageImpl) SetBody(body []byte) { - m.Body = body + m.body = body } //set message tag func (m *MessageImpl) SetTag(tag string) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_TAGS] = tag + m.properties[constant.PROPERTY_TAGS] = tag } -//get message tag from Properties +//get message tag from properties func (m *MessageImpl) Tag() (tag string) { - if m.Properties != nil { - tag = m.Properties[constant.PROPERTY_TAGS] + if m.properties != nil { + tag = m.properties[constant.PROPERTY_TAGS] } return } //set message key func (m *MessageImpl) SetKeys(keys []string) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") + m.properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") } //SetDelayTimeLevel func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) + m.properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) } ////SetWaitStoreMsgOK //func (m *MessageImpl) SetWaitStoreMsgOK(waitStoreMsgOK bool) { -// if m.Properties == nil { -// m.Properties = make(map[string]string) +// if m.properties == nil { +// m.properties = make(map[string]string) // } -// m.Properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) +// m.properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) //} //GeneratorMsgUniqueKey only use by system func (m *MessageImpl) GeneratorMsgUniqueKey() { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - if len(m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { + if len(m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { return } - m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() + m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() } //GetMsgUniqueKey only use by system func (m *MessageExtImpl) GetMsgUniqueKey() string { - if m.Properties != nil { - originMessageId := m.Properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] + if m.properties != nil { + originMessageId := m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] if len(originMessageId) > 0 { return originMessageId } } - return m.MsgId + return m.msgId } //only use by system func (m *MessageImpl) SetOriginMessageId(messageId string) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId + m.properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId } //only use by system func (m *MessageImpl) SetRetryTopic(retryTopic string) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic + m.properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic } //only use by system func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) + m.properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) } //only use by system func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { reConsumeTime = 0 - if m.Properties != nil { - reConsumeTimeStr := m.Properties[constant.PROPERTY_RECONSUME_TIME] + if m.properties != nil { + reConsumeTimeStr := m.properties[constant.PROPERTY_RECONSUME_TIME] if len(reConsumeTimeStr) > 0 { reConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) } @@ -142,17 +174,17 @@ func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { //only use by system func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { - if m.Properties == nil { - m.Properties = make(map[string]string) + if m.properties == nil { + m.properties = make(map[string]string) } - m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) + m.properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) } //only use by system func (m *MessageImpl) GetMaxReconsumeTimes() (maxConsumeTime int) { maxConsumeTime = 0 - if m.Properties != nil { - reConsumeTimeStr := m.Properties[constant.PROPERTY_MAX_RECONSUME_TIMES] + if m.properties != nil { + reConsumeTimeStr := m.properties[constant.PROPERTY_MAX_RECONSUME_TIMES] if len(reConsumeTimeStr) > 0 { maxConsumeTime = util.StrToIntWithDefaultValue(reConsumeTimeStr, 0) } diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index a13f00a8c..676dcabf7 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -181,7 +181,7 @@ func (p *ProcessQueue) PutMessage(msgs []message.MessageExtImpl) (dispatchToCons p.consuming = true } lastMsg := msgs[msgsLen-1] - remoteMaxOffset := util.StrToInt64WithDefaultValue(lastMsg.Properties[constant.PROPERTY_MAX_OFFSET], -1) + remoteMaxOffset := util.StrToInt64WithDefaultValue(lastMsg.PropertiesKeyValue(constant.PROPERTY_MAX_OFFSET), -1) if remoteMaxOffset > 0 { accTotal := remoteMaxOffset - lastMsg.QueueOffset if accTotal > 0 { diff --git a/rocketmq-go/model/pull_request.go b/rocketmq-go/model/pull_request.go index 1ae0e35f3..77c857311 100644 --- a/rocketmq-go/model/pull_request.go +++ b/rocketmq-go/model/pull_request.go @@ -18,8 +18,12 @@ limitations under the License. package model type PullRequest struct { + //consumer group ConsumerGroup string - MessageQueue *MessageQueue - ProcessQueue *ProcessQueue - NextOffset int64 + //which message queue + MessageQueue *MessageQueue + //process queue info + ProcessQueue *ProcessQueue + //next offset + NextOffset int64 } diff --git a/rocketmq-go/model/pull_result.go b/rocketmq-go/model/pull_result.go deleted file mode 100644 index 095460753..000000000 --- a/rocketmq-go/model/pull_result.go +++ /dev/null @@ -1,80 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -// -//import ( -// "fmt" -// "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -//) -// -//type PullStatus int -// -//const ( -// Found PullStatus = iota -// NoNewMsg -// NoMatchedMsg -// OffsetIllegal -//) -// -//type PullResult struct { -// pullStatus PullStatus -// nextBeginOffset int64 -// minOffset int64 -// maxOffset int64 -// msgFoundList []*message.MessageExtImpl -//} -// -//func NewPullResult(ps PullStatus, next, min, max int64, list []*message.MessageExtImpl) *PullResult { -// return &PullResult{ -// ps, -// next, -// min, -// max, -// list, -// } -//} -// -//func (result *PullResult) PullStatus() PullStatus { -// return result.pullStatus -//} -// -//func (result *PullResult) NextBeginOffset() int64 { -// return result.nextBeginOffset -//} -// -//func (result *PullResult) MaxOffset() int64 { -// return result.maxOffset -//} -// -//func (result *PullResult) MinOffset() int64 { -// return result.minOffset -//} -// -//func (result *PullResult) MsgFoundList() []*message.MessageExtImpl { -// return result.msgFoundList -//} -// -//func (result *PullResult) SetMsgFoundList(list []*message.MessageExtImpl) { -// result.msgFoundList = list -//} -// -//func (result *PullResult) String() string { -// return fmt.Sprintf("PullResult [pullStatus=%s, nextBeginOffset=%s, minOffset=%s, maxOffset=%s, msgFoundList=%s]", -// result.pullStatus, result.nextBeginOffset, result.minOffset, result.maxOffset, len(result.msgFoundList)) -//} diff --git a/rocketmq-go/remoting/communication_mode.go b/rocketmq-go/remoting/communication_mode.go index bc084a57a..fc9863e50 100644 --- a/rocketmq-go/remoting/communication_mode.go +++ b/rocketmq-go/remoting/communication_mode.go @@ -20,7 +20,10 @@ package remoting type CommunicationMode int const ( + //Sync invoke Sync CommunicationMode = iota + //ASync invoke Async + //OneWay invoke OneWay ) diff --git a/rocketmq-go/remoting/custom_header.go b/rocketmq-go/remoting/custom_header.go index c85935cc7..2d4fde0d7 100644 --- a/rocketmq-go/remoting/custom_header.go +++ b/rocketmq-go/remoting/custom_header.go @@ -18,6 +18,7 @@ limitations under the License. package remoting type CustomerHeader interface { + //convert map[string]interface to struct FromMap(headerMap map[string]interface{}) //ToMap()(headerMap map[string]interface{}) } diff --git a/rocketmq-go/remoting/json_serializable.go b/rocketmq-go/remoting/json_serializable.go index a32e0ffd2..d568108e2 100644 --- a/rocketmq-go/remoting/json_serializable.go +++ b/rocketmq-go/remoting/json_serializable.go @@ -24,14 +24,14 @@ import ( type JsonSerializer struct { } -func (j *JsonSerializer) EncodeHeaderData(command *RemotingCommand) []byte { +func (j *JsonSerializer) encodeHeaderData(command *RemotingCommand) []byte { buf, err := json.Marshal(command) if err != nil { return nil } return buf } -func (j *JsonSerializer) DecodeRemoteCommand(header, body []byte) *RemotingCommand { +func (j *JsonSerializer) decodeRemoteCommand(header, body []byte) *RemotingCommand { cmd := &RemotingCommand{} cmd.ExtFields = make(map[string]interface{}) err := json.Unmarshal(header, cmd) diff --git a/rocketmq-go/remoting/json_serializable_test.go b/rocketmq-go/remoting/json_serializable_test.go index ab8e5a3fe..f9415e0ce 100644 --- a/rocketmq-go/remoting/json_serializable_test.go +++ b/rocketmq-go/remoting/json_serializable_test.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package remoting_test +package remoting import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" @@ -40,7 +40,7 @@ func TestEncodeHeaderData(t *testing.T) { } jsonSerializer := remoting.JsonSerializer{} - resultJson := jsonSerializer.EncodeHeaderData(command) + resultJson := jsonSerializer.encodeHeaderData(command) if testJson != string(resultJson) { t.Errorf("resultJson is not equals testJson resultJson=%s ", resultJson) } @@ -49,7 +49,7 @@ func TestEncodeHeaderData(t *testing.T) { func TestDecodeRemoteCommand(t *testing.T) { jsonSerializer := remoting.JsonSerializer{} testByte := []byte(testJson) - remotingCommand := jsonSerializer.DecodeRemoteCommand(testByte, []byte{1, 2, 3, 4}) + remotingCommand := jsonSerializer.decodeRemoteCommand(testByte, []byte{1, 2, 3, 4}) if remotingCommand.Language != "GO" || remotingCommand.Remark != "remark" { t.Error("TestDecodeRemoteCommand fail reslutData") } else { diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 6e533d869..296fbc59b 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -33,8 +33,11 @@ import ( ) type RemotingClient interface { + //sync invoke remote InvokeSync(addr string, request *RemotingCommand, timeoutMillis int64) (remotingCommand *RemotingCommand, err error) + //async invoke remote InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error + //one way invoke remote InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error } type DefaultRemotingClient struct { @@ -54,6 +57,7 @@ type DefaultRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } +// create a RemotingClient instance func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { client = &DefaultRemotingClient{} client.connTable = map[string]net.Conn{} @@ -63,7 +67,7 @@ func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestPro client.namesrvAddrList = strings.Split(clientConfig.NameServerAddress, ";") client.namesrvAddrSelectedIndex = -1 client.clientRequestProcessor = clientRequestProcessor - client.serializerHandler = NewSerializerHandler(clientConfig.ClientSerializeType) + client.serializerHandler = newSerializerHandler(clientConfig.ClientSerializeType) return } @@ -77,7 +81,7 @@ func (drc *DefaultRemotingClient) InvokeSync(addr string, request *RemotingComma BeginTimestamp: time.Now().Unix(), Done: make(chan bool), } - header := drc.serializerHandler.EncodeHeader(request) + header := drc.serializerHandler.encodeHeader(request) body := request.Body drc.SetResponse(request.Opaque, response) err = drc.sendRequest(header, body, conn, addr) @@ -107,7 +111,7 @@ func (drc *DefaultRemotingClient) InvokeAsync(addr string, request *RemotingComm InvokeCallback: invokeCallback, } drc.SetResponse(request.Opaque, response) - header := drc.serializerHandler.EncodeHeader(request) + header := drc.serializerHandler.encodeHeader(request) body := request.Body err = drc.sendRequest(header, body, conn, addr) if err != nil { @@ -121,7 +125,7 @@ func (drc *DefaultRemotingClient) InvokeOneWay(addr string, request *RemotingCom if err != nil { return err } - header := drc.serializerHandler.EncodeHeader(request) + header := drc.serializerHandler.encodeHeader(request) body := request.Body err = drc.sendRequest(header, body, conn, addr) if err != nil { @@ -317,7 +321,7 @@ func (drc *DefaultRemotingClient) handlerReceiveLoop(conn net.Conn, addr string) } } func (drc *DefaultRemotingClient) handlerReceivedMessage(conn net.Conn, headerSerializableType byte, headBytes []byte, bodyBytes []byte) { - cmd := drc.serializerHandler.DecodeRemoteCommand(headerSerializableType, headBytes, bodyBytes) + cmd := drc.serializerHandler.decodeRemoteCommand(headerSerializableType, headBytes, bodyBytes) if cmd.IsResponseType() { drc.handlerResponse(cmd) return @@ -331,7 +335,7 @@ func (drc *DefaultRemotingClient) handlerRequest(conn net.Conn, cmd *RemotingCom } responseCommand.Opaque = cmd.Opaque responseCommand.MarkResponseType() - header := drc.serializerHandler.EncodeHeader(responseCommand) + header := drc.serializerHandler.encodeHeader(responseCommand) body := responseCommand.Body err := drc.sendRequest(header, body, conn, "") if err != nil { diff --git a/rocketmq-go/remoting/remoting_command.go b/rocketmq-go/remoting/remoting_command.go index 4df511d13..2b752c10f 100644 --- a/rocketmq-go/remoting/remoting_command.go +++ b/rocketmq-go/remoting/remoting_command.go @@ -19,7 +19,7 @@ package remoting import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "sync/atomic" ) @@ -55,7 +55,7 @@ func NewRemotingCommandWithBody(commandCode int16, customerHeader CustomerHeader remotingCommand.Language = constant.REMOTING_COMMAND_LANGUAGE remotingCommand.Version = constant.REMOTING_COMMAND_VERSION if customerHeader != nil { - remotingCommand.ExtFields = structs.Map(customerHeader) + remotingCommand.ExtFields = util.Struct2Map(customerHeader) } remotingCommand.Body = body return remotingCommand diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index 3c1ab38ca..bdd2d6020 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -34,7 +34,7 @@ const ( value_item ) -func (r *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { +func (r *RocketMqSerializer) encodeHeaderData(cmd *RemotingCommand) []byte { var ( remarkBytes []byte remarkBytesLen int @@ -68,7 +68,7 @@ func (r *RocketMqSerializer) EncodeHeaderData(cmd *RemotingCommand) []byte { return buf.Bytes() } -func (r *RocketMqSerializer) DecodeRemoteCommand(headerArray, body []byte) (cmd *RemotingCommand) { +func (r *RocketMqSerializer) decodeRemoteCommand(headerArray, body []byte) (cmd *RemotingCommand) { cmd = &RemotingCommand{} buf := bytes.NewBuffer(headerArray) // int code(~32767) @@ -128,16 +128,16 @@ func customHeaderDeserialize(extFiledDataBytes []byte) (extFiledMap map[string]i } func getItemFormExtFiledDataBytes(buff *bytes.Buffer, iType itemType) (item string) { if iType == key_item { - var len int16 - binary.Read(buff, binary.BigEndian, &len) - var data = make([]byte, len) + var length int16 + binary.Read(buff, binary.BigEndian, &length) + var data = make([]byte, length) binary.Read(buff, binary.BigEndian, &data) item = string(data) } if iType == value_item { - var len int32 - binary.Read(buff, binary.BigEndian, &len) - var data = make([]byte, len) + var length int32 + binary.Read(buff, binary.BigEndian, &length) + var data = make([]byte, length) binary.Read(buff, binary.BigEndian, &data) item = string(data) } diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index 4b57bc61e..0c819850d 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -30,14 +30,14 @@ type SerializerHandler struct { } type Serializer interface { - EncodeHeaderData(request *RemotingCommand) []byte - DecodeRemoteCommand(header, body []byte) *RemotingCommand + encodeHeaderData(request *RemotingCommand) []byte + decodeRemoteCommand(header, body []byte) *RemotingCommand } var JSON_SERIALIZER = &JsonSerializer{} var ROCKETMQ_SERIALIZER = &RocketMqSerializer{} -func NewSerializerHandler(serializeType rocketmqm.SerializeType) SerializerHandler { +func newSerializerHandler(serializeType rocketmqm.SerializeType) SerializerHandler { serializerHandler := SerializerHandler{serializeType: serializeType} switch serializeType { case rocketmqm.JSON_SERIALIZE: @@ -52,9 +52,9 @@ func NewSerializerHandler(serializeType rocketmqm.SerializeType) SerializerHandl } return serializerHandler } -func (s *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { +func (s *SerializerHandler) encodeHeader(request *RemotingCommand) []byte { length := 4 - headerData := s.serializer.EncodeHeaderData(request) + headerData := s.serializer.encodeHeaderData(request) length += len(headerData) if request.Body != nil { length += len(request.Body) @@ -66,7 +66,7 @@ func (s *SerializerHandler) EncodeHeader(request *RemotingCommand) []byte { return buf.Bytes() } -func (s *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { +func (s *SerializerHandler) decodeRemoteCommand(headerSerializableType byte, header, body []byte) *RemotingCommand { var serializer Serializer switch rocketmqm.SerializeType(headerSerializableType) { case rocketmqm.JSON_SERIALIZE: @@ -78,5 +78,5 @@ func (s *SerializerHandler) DecodeRemoteCommand(headerSerializableType byte, hea default: glog.Error("Unknow headerSerializableType", headerSerializableType) } - return serializer.DecodeRemoteCommand(header, body) + return serializer.decodeRemoteCommand(header, body) } diff --git a/rocketmq-go/api/model/consume_concurrently_result.go b/rocketmq-go/util/concurrent_map_test.go similarity index 75% rename from rocketmq-go/api/model/consume_concurrently_result.go rename to rocketmq-go/util/concurrent_map_test.go index 61a460b4f..8868a3d45 100644 --- a/rocketmq-go/api/model/consume_concurrently_result.go +++ b/rocketmq-go/util/concurrent_map_test.go @@ -15,18 +15,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package rocketmqm +package util -type ConsumeStatus int - -const ( - //consume success - CONSUME_SUCCESS ConsumeStatus = iota - //consume fail - RECONSUME_LATER +import ( + "testing" ) -type ConsumeConcurrentlyResult struct { - ConsumeConcurrentlyStatus ConsumeStatus - AckIndex int +func TestConcurrentMap_Count(t *testing.T) { + } diff --git a/rocketmq-go/util/map_util.go b/rocketmq-go/util/map_util.go new file mode 100644 index 000000000..86fcb1ac4 --- /dev/null +++ b/rocketmq-go/util/map_util.go @@ -0,0 +1,29 @@ +package util + +import ( + "reflect" + "strings" +) + +func Struct2Map(structBody interface{}) (resultMap map[string]interface{}) { + resultMap = make(map[string]interface{}) + value := reflect.ValueOf(structBody) + for value.Kind() == reflect.Ptr { + value = value.Elem() + } + if value.Kind() != reflect.Struct { + panic("input is not a struct") + } + valueType := value.Type() + for i := 0; i < valueType.NumField(); i++ { + field := valueType.Field(i) + if field.PkgPath != "" { + continue + } + name := field.Name + smallName := strings.Replace(name, string(name[0]), string(strings.ToLower(string(name[0]))), 1) + val := value.FieldByName(name).Interface() + resultMap[smallName] = val + } + return +} From cf3cd1a1857dc13a78052bdd086603d6c4b52e12 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 20 Aug 2017 23:47:27 +0800 Subject: [PATCH 42/88] change package --- rocketmq-go/docs/package.puml | 69 ++++--- rocketmq-go/example/simple_consumer.go | 36 +++- .../example/simple_producer_consumer.go | 1 - ..._message_directly_result_request_header.go | 11 +- .../consumer_send_msg_back_request_header.go | 0 .../header/get_consumer_list_by_group.go | 0 ...et_consumer_running_info_request_header.go | 0 .../header/get_max_offset_request_header.go | 0 .../header/get_max_offset_response_header.go | 0 .../header/get_route_info_request_header.go | 0 .../header/pull_message_request_header.go | 0 .../query_consumer_offset_request_header.go | 0 .../header/reset_offset_request_header.go | 0 .../header/search_offset_request_header.go | 0 .../header/send_message_request_header.go | 0 .../header/send_message_response_header.go | 0 .../update_consumer_offset_request_header.go | 0 rocketmq-go/kernel/mq_client.go | 3 +- rocketmq-go/kernel/offset_store.go | 2 +- rocketmq-go/kernel/producer_service.go | 2 +- .../kernel/producer_service_for_send_back.go | 2 +- rocketmq-go/kernel/rebalance.go | 3 +- rocketmq-go/manage/mq_client_manage.go | 4 +- rocketmq-go/manage/mq_push_consumer.go | 2 +- rocketmq-go/manage/pull_message_controller.go | 2 +- rocketmq-go/manage/tasks.go | 5 +- rocketmq-go/remoting/remoting_client.go | 4 +- rocketmq-go/remoting/remoting_command.go | 29 ++- rocketmq-go/util/concurrent_map.go | 186 +++--------------- 29 files changed, 148 insertions(+), 213 deletions(-) rename rocketmq-go/{model => kernel}/header/consume_message_directly_result_request_header.go (85%) rename rocketmq-go/{model => kernel}/header/consumer_send_msg_back_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/get_consumer_list_by_group.go (100%) rename rocketmq-go/{model => kernel}/header/get_consumer_running_info_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/get_max_offset_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/get_max_offset_response_header.go (100%) rename rocketmq-go/{model => kernel}/header/get_route_info_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/pull_message_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/query_consumer_offset_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/reset_offset_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/search_offset_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/send_message_request_header.go (100%) rename rocketmq-go/{model => kernel}/header/send_message_response_header.go (100%) rename rocketmq-go/{model => kernel}/header/update_consumer_offset_request_header.go (100%) diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index cc688728c..aca94bc62 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -1,28 +1,47 @@ @startuml +interface api.MQProducer{ +} +interface api.MQConsumer{ + +} +interface api.MQClientInstance{ -class rocketmq_go.MqClientManager{ -serviceState + +} +class api.ClientInstanceImpl{ } -class rocketmq_go.PullMessageController{ +namespace api{ + MQClientInstance o-- manage.MqClientManager + MQProducer o-- manage.DefaultMQProducer + MQConsumer o-- manage.DefaultMQPushConsumer + MQConsumer o-- manage.DefaultMQPullConsumer } -class service.AllocateMessageQueueStrategy{ + +class manage.MqClientManager{ +kernelState } -class rocketmq_go.ClientFactory{ +class manage.PullMessageController{ + +} +class kernel.AllocateMessageQueueStrategy{ +} +class manage.ClientFactory{ mqConsumerTable mqProducerTable } -class service.MqConsumer{ -serviceState +class manage.DefaultMQPushConsumer{ +} +class manage.DefaultMQPullConsumer{ + to be done } -class service.MqProducer{ -serviceState +class manage.DefaultMQProducer{ } - class service.MqClient{ + class kernel.MqClient{ } - class service.MessageListener{ + class kernel.MessageListener{ } class remoting.RemotingClient{ invokeSync @@ -51,21 +70,16 @@ serviceState } -namespace service{ +namespace kernel{ + + +manage.PullMessageController *-- manage.ClientFactory:contains -rocketmq_go.PullMessageController *-- rocketmq_go.ClientFactory:contains -rocketmq_go.ClientFactory *-- MqProducer:contains -rocketmq_go.ClientFactory *-- MqConsumer:contains -MqProducer *-- service.MqClient :contains -MqConsumer *-- service.PullAPIWrapper : contains -MqConsumer *-- service.OffsetStore : contains -MqConsumer *-- service.Rebalance : contains -MqConsumer *-- service.ConsumeMessageService : contains } -namespace service{ +namespace kernel{ MqClientAPI o-- remoting.RemotingClient:contains OffsetStore o-- MqClient PullAPIWrapper o-- MqClient @@ -74,7 +88,7 @@ namespace service{ Rebalance o-- MqClient Rebalance o-- OffsetStore Rebalance o-- AllocateMessageQueueStrategy - ConsumeMessageService o-- SendMessageBackProducerService + ConsumeMessageService o-- SendMessageBackProducerkernel ConsumeMessageService o-- OffsetStore ConsumeMessageService o-- MessageListener } @@ -88,9 +102,18 @@ namespace remoting { } -namespace rocketmq_go{ +namespace manage{ MqClientManager o-- PullMessageController MqClientManager o-- ClientFactory + manage.ClientFactory *-- DefaultMQPushConsumer:contains + manage.ClientFactory *-- DefaultMQPullConsumer:contains + DefaultMQPushConsumer *-- kernel.PullAPIWrapper : contains + DefaultMQPushConsumer *-- kernel.OffsetStore : contains + DefaultMQPushConsumer *-- kernel.Rebalance : contains + DefaultMQPushConsumer *-- kernel.ConsumeMessageService : contains + manage.ClientFactory *-- DefaultMQProducer:contains + + DefaultMQProducer *-- kernel.MqClient :contains } diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 18a8933b2..1f6a608cd 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -18,12 +18,40 @@ limitations under the License. package main import ( -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" -//"github.com/golang/glog" -//"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + //"github.com/golang/glog" + //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + "github.com/golang/glog" ) func main() { + //ch := make(chan string, 10) + //for i := 0; i < 11; i++ { + // ch<-"2345" + //} + mmp := util.New() + go func() { + i := 1 + for true { + i *= 3 + mmp.Set(util.IntToString(i), "2345") + } + }() + go func() { + i := 1 + for true { + i *= 4 + mmp.Set(util.IntToString(i), "2345") + } + }() + + go func() { + for true { + glog.Info(len(mmp.Keys())) + } + }() + select {} } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 0e1a803e0..9de07e0dc 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -60,7 +60,6 @@ func main() { var message = rocketmqm.NewMessage() message.SetTopic(testTopic) message.SetBody([]byte("hello World")) - glog.Info("here") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) } diff --git a/rocketmq-go/model/header/consume_message_directly_result_request_header.go b/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go similarity index 85% rename from rocketmq-go/model/header/consume_message_directly_result_request_header.go rename to rocketmq-go/kernel/header/consume_message_directly_result_request_header.go index 9b99039dd..834d49e24 100644 --- a/rocketmq-go/model/header/consume_message_directly_result_request_header.go +++ b/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go @@ -17,11 +17,16 @@ limitations under the License. package header +//ConsumeMessageDirectlyResultRequestHeader type ConsumeMessageDirectlyResultRequestHeader struct { + //ConsumerGroup ConsumerGroup string `json:"consumerGroup"` - ClientId string `json:"clientId"` - MsgId string `json:"msgId"` - BrokerName string `json:"brokerName"` + //ClientId + ClientId string `json:"clientId"` + //MsgId + MsgId string `json:"msgId"` + //BrokerName + BrokerName string `json:"brokerName"` } func (c *ConsumeMessageDirectlyResultRequestHeader) FromMap(headerMap map[string]interface{}) { diff --git a/rocketmq-go/model/header/consumer_send_msg_back_request_header.go b/rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go similarity index 100% rename from rocketmq-go/model/header/consumer_send_msg_back_request_header.go rename to rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go diff --git a/rocketmq-go/model/header/get_consumer_list_by_group.go b/rocketmq-go/kernel/header/get_consumer_list_by_group.go similarity index 100% rename from rocketmq-go/model/header/get_consumer_list_by_group.go rename to rocketmq-go/kernel/header/get_consumer_list_by_group.go diff --git a/rocketmq-go/model/header/get_consumer_running_info_request_header.go b/rocketmq-go/kernel/header/get_consumer_running_info_request_header.go similarity index 100% rename from rocketmq-go/model/header/get_consumer_running_info_request_header.go rename to rocketmq-go/kernel/header/get_consumer_running_info_request_header.go diff --git a/rocketmq-go/model/header/get_max_offset_request_header.go b/rocketmq-go/kernel/header/get_max_offset_request_header.go similarity index 100% rename from rocketmq-go/model/header/get_max_offset_request_header.go rename to rocketmq-go/kernel/header/get_max_offset_request_header.go diff --git a/rocketmq-go/model/header/get_max_offset_response_header.go b/rocketmq-go/kernel/header/get_max_offset_response_header.go similarity index 100% rename from rocketmq-go/model/header/get_max_offset_response_header.go rename to rocketmq-go/kernel/header/get_max_offset_response_header.go diff --git a/rocketmq-go/model/header/get_route_info_request_header.go b/rocketmq-go/kernel/header/get_route_info_request_header.go similarity index 100% rename from rocketmq-go/model/header/get_route_info_request_header.go rename to rocketmq-go/kernel/header/get_route_info_request_header.go diff --git a/rocketmq-go/model/header/pull_message_request_header.go b/rocketmq-go/kernel/header/pull_message_request_header.go similarity index 100% rename from rocketmq-go/model/header/pull_message_request_header.go rename to rocketmq-go/kernel/header/pull_message_request_header.go diff --git a/rocketmq-go/model/header/query_consumer_offset_request_header.go b/rocketmq-go/kernel/header/query_consumer_offset_request_header.go similarity index 100% rename from rocketmq-go/model/header/query_consumer_offset_request_header.go rename to rocketmq-go/kernel/header/query_consumer_offset_request_header.go diff --git a/rocketmq-go/model/header/reset_offset_request_header.go b/rocketmq-go/kernel/header/reset_offset_request_header.go similarity index 100% rename from rocketmq-go/model/header/reset_offset_request_header.go rename to rocketmq-go/kernel/header/reset_offset_request_header.go diff --git a/rocketmq-go/model/header/search_offset_request_header.go b/rocketmq-go/kernel/header/search_offset_request_header.go similarity index 100% rename from rocketmq-go/model/header/search_offset_request_header.go rename to rocketmq-go/kernel/header/search_offset_request_header.go diff --git a/rocketmq-go/model/header/send_message_request_header.go b/rocketmq-go/kernel/header/send_message_request_header.go similarity index 100% rename from rocketmq-go/model/header/send_message_request_header.go rename to rocketmq-go/kernel/header/send_message_request_header.go diff --git a/rocketmq-go/model/header/send_message_response_header.go b/rocketmq-go/kernel/header/send_message_response_header.go similarity index 100% rename from rocketmq-go/model/header/send_message_response_header.go rename to rocketmq-go/kernel/header/send_message_response_header.go diff --git a/rocketmq-go/model/header/update_consumer_offset_request_header.go b/rocketmq-go/kernel/header/update_consumer_offset_request_header.go similarity index 100% rename from rocketmq-go/model/header/update_consumer_offset_request_header.go rename to rocketmq-go/kernel/header/update_consumer_offset_request_header.go diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 69c414090..3f7694501 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -22,9 +22,9 @@ import ( "errors" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" @@ -215,6 +215,7 @@ func (m MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMilli } if response.Code == remoting.SUCCESS { topicRouteData := new(model.TopicRouteData) + glog.Info(string(response.Body) + "todo todo") bodyjson := strings.Replace(string(response.Body), ",0:", ",\"0\":", -1) bodyjson = strings.Replace(bodyjson, ",1:", ",\"1\":", -1) // fastJson key is string todo todo bodyjson = strings.Replace(bodyjson, "{0:", "{\"0\":", -1) diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index 02b7c43fd..e2a8f00c6 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -19,8 +19,8 @@ package kernel import ( "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" "strconv" diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index dcb32a90c..6940fe86f 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -20,9 +20,9 @@ package kernel import ( "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index bc0f5c25a..2176175ea 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -21,8 +21,8 @@ import ( "encoding/json" "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index fdfc6ced2..27ce78a1c 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -22,9 +22,9 @@ import ( "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate_message" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" "sort" @@ -295,6 +295,7 @@ func (r *Rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, } if response.Code == remoting.SUCCESS { getConsumerListByGroupResponseBody := new(header.GetConsumerListByGroupResponseBody) + glog.Info("string(response.Body)" + string(response.Body) + "todo todo") // todo check bodyjson := strings.Replace(string(response.Body), "0:", "\"0\":", -1) bodyjson = strings.Replace(bodyjson, "1:", "\"1\":", -1) err := json.Unmarshal([]byte(bodyjson), getConsumerListByGroupResponseBody) diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/manage/mq_client_manage.go index e291900ec..a1846aaed 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/manage/mq_client_manage.go @@ -23,9 +23,9 @@ import ( "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" @@ -62,7 +62,7 @@ func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManage } func (m *MqClientManager) Start() { //d.sendHeartbeatToAllBrokerWithLock()//we should send heartbeat first todo check - m.StartAllScheduledTask() + m.startAllScheduledTask() } func (m *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/manage/mq_push_consumer.go index 7b6383840..5b0301a5a 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/manage/mq_push_consumer.go @@ -96,7 +96,7 @@ func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]i if processQueue == nil || offset < 0 { continue } - glog.Info("now we UpdateOffset", messageQueue, offset) + glog.V(2).Info("now we UpdateOffset", messageQueue, offset) d.offsetStore.UpdateOffset(&messageQueue, offset, false) d.rebalance.RemoveProcessQueue(&messageQueue) } diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/manage/pull_message_controller.go index d50a6aa2c..8f0622d54 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/manage/pull_message_controller.go @@ -22,9 +22,9 @@ import ( "encoding/binary" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/manage/tasks.go index 9be3decd4..475dc2170 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/manage/tasks.go @@ -22,10 +22,7 @@ import ( "time" ) -type TaskManager struct { -} - -func (m MqClientManager) StartAllScheduledTask() { +func (m MqClientManager) startAllScheduledTask() { rand.Seed(time.Now().UnixNano()) go func() { updateTopicRouteTimer := time.NewTimer(5 * time.Second) diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 296fbc59b..8de886aab 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -322,7 +322,7 @@ func (drc *DefaultRemotingClient) handlerReceiveLoop(conn net.Conn, addr string) } func (drc *DefaultRemotingClient) handlerReceivedMessage(conn net.Conn, headerSerializableType byte, headBytes []byte, bodyBytes []byte) { cmd := drc.serializerHandler.decodeRemoteCommand(headerSerializableType, headBytes, bodyBytes) - if cmd.IsResponseType() { + if cmd.isResponseType() { drc.handlerResponse(cmd) return } @@ -334,7 +334,7 @@ func (drc *DefaultRemotingClient) handlerRequest(conn net.Conn, cmd *RemotingCom return } responseCommand.Opaque = cmd.Opaque - responseCommand.MarkResponseType() + responseCommand.markResponseType() header := drc.serializerHandler.encodeHeader(responseCommand) body := responseCommand.Body err := drc.sendRequest(header, body, conn, "") diff --git a/rocketmq-go/remoting/remoting_command.go b/rocketmq-go/remoting/remoting_command.go index 2b752c10f..eccf3b396 100644 --- a/rocketmq-go/remoting/remoting_command.go +++ b/rocketmq-go/remoting/remoting_command.go @@ -31,15 +31,26 @@ var RPC_ONEWAY int = 1 // 0, RPC //var RESPONSE_TYPE int= 1 << RPC_TYPE var RESPONSE_TYPE int = 1 +//rocketmq remoting command +// both request and response use it type RemotingCommand struct { - Code int16 `json:"code"` - Language string `json:"language"` //int 8 - Version int16 `json:"version"` - Opaque int32 `json:"opaque"` - Flag int `json:"flag"` - Remark string `json:"remark"` + //request: request_code.go + //response: response_code.go + Code int16 `json:"code"` + //this client's language. see config.go + Language string `json:"language"` //int 8 + //this client's version. see config.go + Version int16 `json:"version"` + //the client's Opaque,it is auto increase + Opaque int32 `json:"opaque"` + //this request's flag + Flag int `json:"flag"` + //remark, for example error message + Remark string `json:"remark"` + //this request's param ExtFields map[string]interface{} `json:"extFields"` //java's ExtFields and customHeader is use this key word - Body []byte `json:"body,omitempty"` + //response content + Body []byte `json:"body,omitempty"` } func NewRemotingCommand(commandCode int16, customerHeader CustomerHeader) *RemotingCommand { @@ -61,9 +72,9 @@ func NewRemotingCommandWithBody(commandCode int16, customerHeader CustomerHeader return remotingCommand } -func (r *RemotingCommand) IsResponseType() bool { +func (r *RemotingCommand) isResponseType() bool { return r.Flag&(RESPONSE_TYPE) == RESPONSE_TYPE } -func (r *RemotingCommand) MarkResponseType() { +func (r *RemotingCommand) markResponseType() { r.Flag = (r.Flag | RESPONSE_TYPE) } diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go index 0b70e918c..d0af20f82 100644 --- a/rocketmq-go/util/concurrent_map.go +++ b/rocketmq-go/util/concurrent_map.go @@ -18,91 +18,43 @@ limitations under the License. package util import ( - "encoding/json" "sync" ) -var SHARD_COUNT = 33 +var MAP_ITEM_COUNT = 32 -// A "thread" safe map of type string:Anything. -// To avoid lock bottlenecks this map is dived to several (SHARD_COUNT) map shards. -type ConcurrentMap []*concurrentMapShared +//ConcurrentMap +type ConcurrentMap []*concurrentMapItem -// A "thread" safe string to anything map. -type concurrentMapShared struct { - items map[string]interface{} - sync.RWMutex // Read Write mutex, guards access to internal map. +type concurrentMapItem struct { + item map[string]interface{} + sync.RWMutex } -// Creates a new concurrent map. +//create a ConcurrentMap instance func New() ConcurrentMap { - m := make(ConcurrentMap, SHARD_COUNT) - for i := 0; i < SHARD_COUNT; i++ { - m[i] = &concurrentMapShared{items: make(map[string]interface{})} + m := make(ConcurrentMap, MAP_ITEM_COUNT) + for i := 0; i < MAP_ITEM_COUNT; i++ { + m[i] = &concurrentMapItem{item: make(map[string]interface{})} } return m } -// Returns shard under given key -func (m ConcurrentMap) GetShard(key string) *concurrentMapShared { - return m[uint(fnv32(key))%uint(SHARD_COUNT)] +func (m ConcurrentMap) GetMapItem(key string) *concurrentMapItem { + return m[uint(fnv32(key))%uint(MAP_ITEM_COUNT)] } -func (m ConcurrentMap) MSet(data map[string]interface{}) { - for key, value := range data { - shard := m.GetShard(key) - shard.Lock() - shard.items[key] = value - shard.Unlock() - } -} - -// Sets the given value under the specified key. func (m *ConcurrentMap) Set(key string, value interface{}) { - // Get map shard. - shard := m.GetShard(key) - shard.Lock() - shard.items[key] = value - shard.Unlock() -} - -// Callback to return new element to be inserted into the map -// It is called while lock is held, therefore it MUST NOT -// try to access other keys in same map, as it can lead to deadlock since -// Go sync.RWLock is not reentrant -type UpsertCb func(exist bool, valueInMap interface{}, newValue interface{}) interface{} - -// Insert or Update - updates existing element or inserts a new one using UpsertCb -func (m *ConcurrentMap) Upsert(key string, value interface{}, cb UpsertCb) (res interface{}) { - shard := m.GetShard(key) + shard := m.GetMapItem(key) shard.Lock() - v, ok := shard.items[key] - res = cb(ok, v, value) - shard.items[key] = res + shard.item[key] = value shard.Unlock() - return res } -// Sets the given value under the specified key if no value was associated with it. -func (m *ConcurrentMap) SetIfAbsent(key string, value interface{}) bool { - // Get map shard. - shard := m.GetShard(key) - shard.Lock() - _, ok := shard.items[key] - if !ok { - shard.items[key] = value - } - shard.Unlock() - return !ok -} - -// Retrieves an element from map under given key. func (m ConcurrentMap) Get(key string) (interface{}, bool) { - // Get shard - shard := m.GetShard(key) + shard := m.GetMapItem(key) shard.RLock() - // Get item from shard. - val, ok := shard.items[key] + val, ok := shard.item[key] shard.RUnlock() return val, ok } @@ -110,49 +62,22 @@ func (m ConcurrentMap) Get(key string) (interface{}, bool) { // Returns the number of elements within the map. func (m ConcurrentMap) Count() int { count := 0 - for i := 0; i < SHARD_COUNT; i++ { + for i := 0; i < MAP_ITEM_COUNT; i++ { shard := m[i] shard.RLock() - count += len(shard.items) + count += len(shard.item) shard.RUnlock() } return count } -// Looks up an item under specified key -func (m *ConcurrentMap) Has(key string) bool { - // Get shard - shard := m.GetShard(key) - shard.RLock() - // See if element is within shard. - _, ok := shard.items[key] - shard.RUnlock() - return ok -} - // Removes an element from the map. func (m *ConcurrentMap) Remove(key string) { // Try to get shard. - shard := m.GetShard(key) - shard.Lock() - delete(shard.items, key) - shard.Unlock() -} - -// Removes an element from the map and returns it -func (m *ConcurrentMap) Pop(key string) (v interface{}, exists bool) { - // Try to get shard. - shard := m.GetShard(key) + shard := m.GetMapItem(key) shard.Lock() - v, exists = shard.items[key] - delete(shard.items, key) + delete(shard.item, key) shard.Unlock() - return v, exists -} - -// Checks if map is empty. -func (m *ConcurrentMap) IsEmpty() bool { - return m.Count() == 0 } // Used by the Iter & IterBuffered functions to wrap two variables together over a channel, @@ -161,42 +86,18 @@ type Tuple struct { Val interface{} } -// Returns an iterator which could be used in a for range loop. -func (m ConcurrentMap) Iter() <-chan Tuple { - ch := make(chan Tuple) - go func() { - wg := sync.WaitGroup{} - wg.Add(SHARD_COUNT) - // Foreach shard. - for _, shard := range m { - go func(shard *concurrentMapShared) { - // Foreach key, value pair. - shard.RLock() - for key, val := range shard.items { - ch <- Tuple{key, val} - } - shard.RUnlock() - wg.Done() - }(shard) - } - wg.Wait() - close(ch) - }() - return ch -} - // Returns a buffered iterator which could be used in a for range loop. func (m ConcurrentMap) IterBuffered() <-chan Tuple { ch := make(chan Tuple, m.Count()) go func() { wg := sync.WaitGroup{} - wg.Add(SHARD_COUNT) + wg.Add(MAP_ITEM_COUNT) // Foreach shard. for _, shard := range m { - go func(shard *concurrentMapShared) { + go func(shard *concurrentMapItem) { // Foreach key, value pair. shard.RLock() - for key, val := range shard.items { + for key, val := range shard.item { ch <- Tuple{key, val} } shard.RUnlock() @@ -209,11 +110,11 @@ func (m ConcurrentMap) IterBuffered() <-chan Tuple { return ch } -// Returns all items as map[string]interface{} +// Returns all item as map[string]interface{} func (m ConcurrentMap) Items() map[string]interface{} { tmp := make(map[string]interface{}) - // Insert items to temporary map. + // Insert item to temporary map. for item := range m.IterBuffered() { tmp[item.Key] = item.Val } @@ -221,25 +122,6 @@ func (m ConcurrentMap) Items() map[string]interface{} { return tmp } -// Iterator callback,called for every key,value found in -// maps. RLock is held for all calls for a given shard -// therefore callback sess consistent view of a shard, -// but not across the shards -type IterCb func(key string, v interface{}) - -// Callback based iterator, cheapest way to read -// all elements in a map. -func (m *ConcurrentMap) IterCb(fn IterCb) { - for idx := range *m { - shard := (*m)[idx] - shard.RLock() - for key, value := range shard.items { - fn(key, value) - } - shard.RUnlock() - } -} - // Return all keys as []string func (m ConcurrentMap) Keys() []string { count := m.Count() @@ -247,12 +129,12 @@ func (m ConcurrentMap) Keys() []string { go func() { // Foreach shard. wg := sync.WaitGroup{} - wg.Add(SHARD_COUNT) + wg.Add(MAP_ITEM_COUNT) for _, shard := range m { - go func(shard *concurrentMapShared) { + go func(shard *concurrentMapItem) { // Foreach key, value pair. shard.RLock() - for key := range shard.items { + for key := range shard.item { ch <- key } shard.RUnlock() @@ -270,18 +152,6 @@ func (m ConcurrentMap) Keys() []string { return keys } -//Reviles ConcurrentMap "private" variables to json marshal. -func (m ConcurrentMap) MarshalJSON() ([]byte, error) { - // Create a temporary map, which will hold all item spread across shards. - tmp := make(map[string]interface{}) - - // Insert items to temporary map. - for item := range m.IterBuffered() { - tmp[item.Key] = item.Val - } - return json.Marshal(tmp) -} - func fnv32(key string) uint32 { hash := uint32(2166136261) const prime32 = uint32(16777619) From 4b14070361e8794897354854b0f4a548627b4044 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:08:53 +0800 Subject: [PATCH 43/88] manage->kernel --- rocketmq-go/api/model/message.go | 2 +- rocketmq-go/api/rocketmq_clent_instance.go | 10 +- rocketmq-go/api/rocketmq_consumer.go | 4 +- rocketmq-go/api/rocketmq_producer.go | 4 +- rocketmq-go/docs/package.puml | 28 +- .../clean_expire_msg_controller.go | 11 +- rocketmq-go/kernel/consume_message_service.go | 32 +- .../{manage => kernel}/mq_client_manage.go | 24 +- rocketmq-go/kernel/mq_fault_strategy.go | 2 +- rocketmq-go/{manage => kernel}/mq_producer.go | 6 +- .../{manage => kernel}/mq_push_consumer.go | 32 +- rocketmq-go/kernel/producer_service.go | 2 +- .../kernel/producer_service_for_send_back.go | 10 +- .../pull_message_controller.go | 15 +- rocketmq-go/kernel/rebalance.go | 58 +- .../rebalance_controller.go | 4 +- rocketmq-go/{manage => kernel}/tasks.go | 2 +- rocketmq-go/util/structs/field.go | 158 ----- rocketmq-go/util/structs/structs.go | 597 ------------------ rocketmq-go/util/structs/tags.go | 49 -- 20 files changed, 122 insertions(+), 928 deletions(-) rename rocketmq-go/{manage => kernel}/clean_expire_msg_controller.go (81%) rename rocketmq-go/{manage => kernel}/mq_client_manage.go (90%) rename rocketmq-go/{manage => kernel}/mq_producer.go (94%) rename rocketmq-go/{manage => kernel}/mq_push_consumer.go (83%) rename rocketmq-go/{manage => kernel}/pull_message_controller.go (95%) rename rocketmq-go/{manage => kernel}/rebalance_controller.go (95%) rename rocketmq-go/{manage => kernel}/tasks.go (99%) delete mode 100644 rocketmq-go/util/structs/field.go delete mode 100644 rocketmq-go/util/structs/structs.go delete mode 100644 rocketmq-go/util/structs/tags.go diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 10bf0bb8c..5e72d9cd2 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -27,7 +27,7 @@ type Message interface { SetBody([]byte) } -//create a message instance +/*create a message instance*/ func NewMessage() (msg Message) { msg = message.NewMessageImpl() return diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 22c65f70c..883882deb 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -19,7 +19,7 @@ package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" ) type MQClientInstance interface { @@ -32,7 +32,7 @@ type MQClientInstance interface { } type ClientInstanceImpl struct { - rocketMqManager *manage.MqClientManager + rocketMqManager *kernel.MqClientManager } func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance MQClientInstance) { @@ -40,16 +40,16 @@ func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstanc return InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) } func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmqm.MqClientConfig) (rocketMQClientInstance MQClientInstance) { - rocketMQClientInstance = &ClientInstanceImpl{rocketMqManager: manage.MqClientManagerInit(mqClientConfig)} + rocketMQClientInstance = &ClientInstanceImpl{rocketMqManager: kernel.MqClientManagerInit(mqClientConfig)} return } func (r *ClientInstanceImpl) RegisterProducer(producer MQProducer) { - r.rocketMqManager.RegisterProducer(producer.(*manage.DefaultMQProducer)) + r.rocketMqManager.RegisterProducer(producer.(*kernel.DefaultMQProducer)) } func (r *ClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { - r.rocketMqManager.RegisterConsumer(consumer.(*manage.DefaultMQPushConsumer)) + r.rocketMqManager.RegisterConsumer(consumer.(*kernel.DefaultMQPushConsumer)) } func (r *ClientInstanceImpl) Start() { r.rocketMqManager.Start() diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index 8b039f5fe..e11246623 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -19,7 +19,7 @@ package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" ) type MQConsumer interface { @@ -43,5 +43,5 @@ func NewDefaultMQPushConsumer(producerGroup string) (r MQConsumer) { // Concurrently(no order) CLUSTERING mq consumer with custom config func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (r MQConsumer) { - return manage.NewDefaultMQPushConsumer(producerGroup, consumerConfig) + return kernel.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index b90f9fa07..fde8e9879 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -19,7 +19,7 @@ package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/manage" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) @@ -44,5 +44,5 @@ func NewDefaultMQProducer(producerGroup string) (r MQProducer) { //mq producer with custom config func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (r MQProducer) { - return manage.NewDefaultMQProducer(producerGroup, producerConfig) + return kernel.NewDefaultMQProducer(producerGroup, producerConfig) } diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index aca94bc62..a0214d146 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -13,30 +13,30 @@ class api.ClientInstanceImpl{ } namespace api{ - MQClientInstance o-- manage.MqClientManager - MQProducer o-- manage.DefaultMQProducer - MQConsumer o-- manage.DefaultMQPushConsumer - MQConsumer o-- manage.DefaultMQPullConsumer + MQClientInstance o-- kernel.MqClientManager + MQProducer o-- kernel.DefaultMQProducer + MQConsumer o-- kernel.DefaultMQPushConsumer + MQConsumer o-- kernel.DefaultMQPullConsumer } -class manage.MqClientManager{ +class kernel.MqClientManager{ kernelState } -class manage.PullMessageController{ +class kernel.PullMessageController{ } class kernel.AllocateMessageQueueStrategy{ } -class manage.ClientFactory{ +class kernel.ClientFactory{ mqConsumerTable mqProducerTable } -class manage.DefaultMQPushConsumer{ +class kernel.DefaultMQPushConsumer{ } -class manage.DefaultMQPullConsumer{ +class kernel.DefaultMQPullConsumer{ to be done } -class manage.DefaultMQProducer{ +class kernel.DefaultMQProducer{ } class kernel.MqClient{ @@ -73,7 +73,7 @@ class manage.DefaultMQProducer{ namespace kernel{ -manage.PullMessageController *-- manage.ClientFactory:contains +kernel.PullMessageController *-- kernel.ClientFactory:contains @@ -105,13 +105,13 @@ namespace remoting { namespace manage{ MqClientManager o-- PullMessageController MqClientManager o-- ClientFactory - manage.ClientFactory *-- DefaultMQPushConsumer:contains - manage.ClientFactory *-- DefaultMQPullConsumer:contains + kernel.ClientFactory *-- DefaultMQPushConsumer:contains + kernel.ClientFactory *-- DefaultMQPullConsumer:contains DefaultMQPushConsumer *-- kernel.PullAPIWrapper : contains DefaultMQPushConsumer *-- kernel.OffsetStore : contains DefaultMQPushConsumer *-- kernel.Rebalance : contains DefaultMQPushConsumer *-- kernel.ConsumeMessageService : contains - manage.ClientFactory *-- DefaultMQProducer:contains + kernel.ClientFactory *-- DefaultMQProducer:contains DefaultMQProducer *-- kernel.MqClient :contains } diff --git a/rocketmq-go/manage/clean_expire_msg_controller.go b/rocketmq-go/kernel/clean_expire_msg_controller.go similarity index 81% rename from rocketmq-go/manage/clean_expire_msg_controller.go rename to rocketmq-go/kernel/clean_expire_msg_controller.go index ff67268e3..1d4139ac3 100644 --- a/rocketmq-go/manage/clean_expire_msg_controller.go +++ b/rocketmq-go/kernel/clean_expire_msg_controller.go @@ -15,27 +15,26 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "time" ) type cleanExpireMsgController struct { - mqClient kernel.RocketMqClient + mqClient RocketMqClient clientFactory *clientFactory } -func newCleanExpireMsgController(mqClient kernel.RocketMqClient, clientFactory *clientFactory) *cleanExpireMsgController { +func newCleanExpireMsgController(mqClient RocketMqClient, clientFactory *clientFactory) *cleanExpireMsgController { return &cleanExpireMsgController{ mqClient: mqClient, clientFactory: clientFactory, } } -func (self *cleanExpireMsgController) start() { - for _, consumer := range self.clientFactory.consumerTable { +func (c *cleanExpireMsgController) start() { + for _, consumer := range c.clientFactory.consumerTable { go func() { cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) //cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index a530b6723..27a68b4c5 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -26,15 +26,15 @@ import ( "github.com/golang/glog" ) -type ConsumeMessageService interface { - Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) - SubmitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, +type consumeMessageService interface { + init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) + submitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) - SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) - ConsumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) + sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) + consumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) } -type ConsumeMessageConcurrentlyServiceImpl struct { +type consumeMessageConcurrentlyServiceImpl struct { consumerGroup string messageListener rocketmqm.MessageListener sendMessageBackProducerService sendMessageBackProducerService //for send retry MessageImpl @@ -42,19 +42,19 @@ type ConsumeMessageConcurrentlyServiceImpl struct { consumerConfig *rocketmqm.MqConsumerConfig } -func NewConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageListener) (consumeService ConsumeMessageService) { - consumeService = &ConsumeMessageConcurrentlyServiceImpl{messageListener: messageListener, sendMessageBackProducerService: &SendMessageBackProducerServiceImpl{}} +func NewConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageListener) (consumeService consumeMessageService) { + consumeService = &consumeMessageConcurrentlyServiceImpl{messageListener: messageListener, sendMessageBackProducerService: &sendMessageBackProducerServiceImpl{}} return } -func (c *ConsumeMessageConcurrentlyServiceImpl) Init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { +func (c *consumeMessageConcurrentlyServiceImpl) init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { c.consumerGroup = consumerGroup c.offsetStore = offsetStore c.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) c.consumerConfig = consumerConfig } -func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (c *consumeMessageConcurrentlyServiceImpl) submitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i @@ -73,7 +73,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) SubmitConsumeRequest(msgs []mess } return } -func (c *ConsumeMessageConcurrentlyServiceImpl) convert2ConsumeType(msgs []message.MessageExtImpl) (ret []rocketmqm.MessageExt) { +func (c *consumeMessageConcurrentlyServiceImpl) convert2ConsumeType(msgs []message.MessageExtImpl) (ret []rocketmqm.MessageExt) { msgLen := len(msgs) ret = make([]rocketmqm.MessageExt, msgLen) @@ -83,12 +83,12 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) convert2ConsumeType(msgs []messa return } -func (c *ConsumeMessageConcurrentlyServiceImpl) SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { +func (c *consumeMessageConcurrentlyServiceImpl) sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { err = c.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) return } -func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { +func (c *consumeMessageConcurrentlyServiceImpl) consumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) { start := util.CurrentTimeMillisInt64() consumeResult := c.messageListener(c.convert2ConsumeType([]message.MessageExtImpl{*messageExt})) consumeMessageDirectlyResult.AutoCommit = true @@ -102,7 +102,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) ConsumeMessageDirectly(messageEx return } -func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []message.MessageExtImpl, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (c *consumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []message.MessageExtImpl, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return @@ -126,7 +126,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock successMessages = msgs[:ackIndex+1] } for i := ackIndex + 1; i < len(msgs); i++ { - err := c.SendMessageBack(&msgs[i], 0, messageQueue.BrokerName) + err := c.sendMessageBack(&msgs[i], 0, messageQueue.BrokerName) if err != nil { msgs[i].ReconsumeTimes = msgs[i].ReconsumeTimes + 1 failedMessages = append(failedMessages, msgs[i]) @@ -135,7 +135,7 @@ func (c *ConsumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } } if len(failedMessages) > 0 { - c.SubmitConsumeRequest(failedMessages, processQueue, messageQueue, true) + c.submitConsumeRequest(failedMessages, processQueue, messageQueue, true) } commitOffset := processQueue.RemoveMessage(successMessages) if commitOffset > 0 && !processQueue.IsDropped() { diff --git a/rocketmq-go/manage/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go similarity index 90% rename from rocketmq-go/manage/mq_client_manage.go rename to rocketmq-go/kernel/mq_client_manage.go index a1846aaed..8f197eff5 100644 --- a/rocketmq-go/manage/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -15,14 +15,14 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( "encoding/json" "errors" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" @@ -42,18 +42,18 @@ type MqClientManager struct { rocketMqManagerLock sync.Mutex BootTimestamp int64 clientFactory *clientFactory - mqClient kernel.RocketMqClient + mqClient RocketMqClient pullMessageController *PullMessageController cleanExpireMsgController *cleanExpireMsgController rebalanceControllr *RebalanceController - defaultProducerService *kernel.DefaultProducerService + defaultProducerService *DefaultProducerService } func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManager *MqClientManager) { rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() rocketMqManager.clientFactory = ClientFactoryInit() - rocketMqManager.mqClient = kernel.MqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo + rocketMqManager.mqClient = MqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.cleanExpireMsgController = newCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) @@ -66,23 +66,23 @@ func (m *MqClientManager) Start() { } func (m *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { - producer.producerService = kernel.NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) + producer.producerService = NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) m.clientFactory.producerTable[producer.producerGroup] = producer return } func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { if m.defaultProducerService == nil { - m.defaultProducerService = kernel.NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) + m.defaultProducerService = NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) } consumer.mqClient = m.mqClient - consumer.offsetStore = kernel.RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) + consumer.offsetStore = RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) m.clientFactory.consumerTable[consumer.consumerGroup] = consumer - consumer.rebalance = kernel.NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) + consumer.rebalance = NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) fmt.Println(consumer.consumeMessageService) - consumer.consumeMessageService.Init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) + consumer.consumeMessageService.init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) return } func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor remoting.ClientRequestProcessor) { @@ -128,7 +128,7 @@ func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor r } consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.GetRemotingClient().GetNamesrvAddrList(), ";") - consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.GetMqTableInfo() + consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.getMqTableInfo() glog.V(2).Info("op=look consumerRunningInfo", consumerRunningInfo) jsonByte, err := consumerRunningInfo.Encode() @@ -149,7 +149,7 @@ func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor r messageExt := &DecodeMessage(cmd.Body)[0] glog.V(2).Info("op=look", messageExt) defaultMQPushConsumer := m.clientFactory.consumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] - consumeResult, err := defaultMQPushConsumer.consumeMessageService.ConsumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) + consumeResult, err := defaultMQPushConsumer.consumeMessageService.consumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) if err != nil { return } diff --git a/rocketmq-go/kernel/mq_fault_strategy.go b/rocketmq-go/kernel/mq_fault_strategy.go index abc12da3d..b052f3c53 100644 --- a/rocketmq-go/kernel/mq_fault_strategy.go +++ b/rocketmq-go/kernel/mq_fault_strategy.go @@ -22,7 +22,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) -type MQFaultStrategy struct { +type mqFaultStrategy struct { } //if first select : random one diff --git a/rocketmq-go/manage/mq_producer.go b/rocketmq-go/kernel/mq_producer.go similarity index 94% rename from rocketmq-go/manage/mq_producer.go rename to rocketmq-go/kernel/mq_producer.go index f951e9682..0c7acf133 100644 --- a/rocketmq-go/manage/mq_producer.go +++ b/rocketmq-go/kernel/mq_producer.go @@ -15,11 +15,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" @@ -29,7 +29,7 @@ type DefaultMQProducer struct { producerGroup string ProducerConfig *rocketmqm.MqProducerConfig - producerService kernel.ProducerService + producerService ProducerService } func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (rocketMQProducer *DefaultMQProducer) { diff --git a/rocketmq-go/manage/mq_push_consumer.go b/rocketmq-go/kernel/mq_push_consumer.go similarity index 83% rename from rocketmq-go/manage/mq_push_consumer.go rename to rocketmq-go/kernel/mq_push_consumer.go index 5b0301a5a..c2879655f 100644 --- a/rocketmq-go/manage/mq_push_consumer.go +++ b/rocketmq-go/kernel/mq_push_consumer.go @@ -15,11 +15,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" @@ -34,11 +34,11 @@ type DefaultMQPushConsumer struct { unitMode bool subscription map[string]string //topic|subExpression subscriptionTag map[string][]string // we use it filter again - offsetStore kernel.OffsetStore - mqClient kernel.RocketMqClient - rebalance *kernel.Rebalance + offsetStore OffsetStore + mqClient RocketMqClient + rebalance *rebalance pause bool - consumeMessageService kernel.ConsumeMessageService + consumeMessageService consumeMessageService ConsumerConfig *rocketmqm.MqConsumerConfig } @@ -74,38 +74,38 @@ func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { } func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener rocketmqm.MessageListener) { - d.consumeMessageService = kernel.NewConsumeMessageConcurrentlyServiceImpl(messageListener) + d.consumeMessageService = NewConsumeMessageConcurrentlyServiceImpl(messageListener) } func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { d.pause = true - glog.V(2).Info("now we ClearProcessQueue 0 ", offsetTable) + glog.V(2).Info("now we clearProcessQueue 0 ", offsetTable) - d.rebalance.ClearProcessQueue(offsetTable) - glog.V(2).Info("now we ClearProcessQueue", offsetTable) + d.rebalance.clearProcessQueue(offsetTable) + glog.V(2).Info("now we clearProcessQueue", offsetTable) go func() { waitTime := time.NewTimer(10 * time.Second) <-waitTime.C defer func() { d.pause = false - d.rebalance.DoRebalance() + d.rebalance.doRebalance() }() for messageQueue, offset := range offsetTable { - processQueue := d.rebalance.GetProcessQueue(messageQueue) + processQueue := d.rebalance.getProcessQueue(messageQueue) if processQueue == nil || offset < 0 { continue } glog.V(2).Info("now we UpdateOffset", messageQueue, offset) d.offsetStore.UpdateOffset(&messageQueue, offset, false) - d.rebalance.RemoveProcessQueue(&messageQueue) + d.rebalance.removeProcessQueue(&messageQueue) } }() } func (d *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { subscriptions := make([]*model.SubscriptionData, 0) - for _, subscription := range d.rebalance.SubscriptionInner { + for _, subscription := range d.rebalance.subscriptionInner { subscriptions = append(subscriptions, subscription) } return subscriptions @@ -113,7 +113,7 @@ func (d *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { func (d *DefaultMQPushConsumer) CleanExpireMsg() { nowTime := util.CurrentTimeMillisInt64() //will cause nowTime - consumeStartTime <0 ,but no matter - messageQueueList, processQueueList := d.rebalance.GetProcessQueueList() + messageQueueList, processQueueList := d.rebalance.getProcessQueueList() for messageQueueIndex, processQueue := range processQueueList { loop := processQueue.GetMsgCount() if loop > 16 { @@ -136,7 +136,7 @@ func (d *DefaultMQPushConsumer) CleanExpireMsg() { break } glog.Info("look now we send expire message back", message.Topic(), message.MsgId()) - err := d.consumeMessageService.SendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) + err := d.consumeMessageService.sendMessageBack(message, 3, messageQueueList[messageQueueIndex].BrokerName) if err != nil { glog.Error("op=send_expire_message_back_error", err) continue diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 6940fe86f..c6612b434 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -38,7 +38,7 @@ type DefaultProducerService struct { producerGroup string producerConfig *rocketmqm.MqProducerConfig mqClient RocketMqClient - mqFaultStrategy MQFaultStrategy + mqFaultStrategy mqFaultStrategy } func NewDefaultProducerService(producerGroup string, producerConfig *rocketmqm.MqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 2176175ea..d5ff43f01 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -33,7 +33,7 @@ type sendMessageBackProducerService interface { InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) } -type SendMessageBackProducerServiceImpl struct { +type sendMessageBackProducerServiceImpl struct { mqClient RocketMqClient defaultProducerService *DefaultProducerService // one namesvr only one consumerGroup string @@ -41,7 +41,7 @@ type SendMessageBackProducerServiceImpl struct { } // send to original broker,if fail send a new retry message -func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { +func (s *sendMessageBackProducerServiceImpl) sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { glog.V(2).Info("op=look_send_message_back", messageExt.MsgId(), messageExt.Properties(), string(messageExt.Body())) err = s.consumerSendMessageBack(brokerName, messageExt, delayLayLevel) if err == nil { @@ -52,7 +52,7 @@ func (s *SendMessageBackProducerServiceImpl) SendMessageBack(messageExt *message return } -func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *message.MessageExtImpl) error { +func (s *sendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *message.MessageExtImpl) error { retryMessage := &message.MessageImpl{} originMessageId := messageExt.GetOriginMessageId() retryMessage.SetProperties(messageExt.Properties()) @@ -77,14 +77,14 @@ func (s *SendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *me } -func (s *SendMessageBackProducerServiceImpl) InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { +func (s *sendMessageBackProducerServiceImpl) initSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { s.mqClient = mqClient s.consumerGroup = consumerGroup s.defaultProducerService = defaultProducerService s.consumerConfig = consumerConfig } -func (s *SendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *message.MessageExtImpl, delayLayLevel int) (err error) { +func (s *sendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName string, messageExt *message.MessageExtImpl, delayLayLevel int) (err error) { if len(brokerName) == 0 { err = errors.New("broker can't be empty") glog.Error(err) diff --git a/rocketmq-go/manage/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go similarity index 95% rename from rocketmq-go/manage/pull_message_controller.go rename to rocketmq-go/kernel/pull_message_controller.go index 8f0622d54..b429c33f5 100644 --- a/rocketmq-go/manage/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -15,13 +15,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( "bytes" "encoding/binary" "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" @@ -34,11 +33,11 @@ import ( ) type PullMessageController struct { - mqClient kernel.RocketMqClient + mqClient RocketMqClient clientFactory *clientFactory } -func NewPullMessageController(mqClient kernel.RocketMqClient, clientFactory *clientFactory) *PullMessageController { +func NewPullMessageController(mqClient RocketMqClient, clientFactory *clientFactory) *PullMessageController { return &PullMessageController{ mqClient: mqClient, clientFactory: clientFactory, @@ -83,9 +82,9 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { p.pullMessageLater(pullRequest, delayPullTime) return } - commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, kernel.READ_FROM_MEMORY) + commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, READ_FROM_MEMORY) - subscriptionData, ok := defaultMQPullConsumer.rebalance.SubscriptionInner[pullRequest.MessageQueue.Topic] + subscriptionData, ok := defaultMQPullConsumer.rebalance.subscriptionInner[pullRequest.MessageQueue.Topic] if !ok { p.pullMessageLater(pullRequest, defaultMQPullConsumer.ConsumerConfig.PullTimeDelayMillsWhenException) return @@ -140,7 +139,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { } // pullRequest.ProcessQueue.PutMessage(msgs) - defaultMQPullConsumer.consumeMessageService.SubmitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) + defaultMQPullConsumer.consumeMessageService.submitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) } else { var err error // change the offset , use nextBeginOffset pullResult := responseCommand.ExtFields @@ -170,7 +169,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { executeTaskLater := time.NewTimer(10 * time.Second) <-executeTaskLater.C defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, false) - defaultMQPullConsumer.rebalance.RemoveProcessQueue(pullRequest.MessageQueue) + defaultMQPullConsumer.rebalance.removeProcessQueue(pullRequest.MessageQueue) }() } else { glog.Errorf("illegal response code. pull message error,code=%d,request=%v OFFSET_ILLEGAL", responseCommand.Code, requestHeader) diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 27ce78a1c..84a25c025 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -33,11 +33,11 @@ import ( "time" ) -type Rebalance struct { +type rebalance struct { groupName string messageModel string topicSubscribeInfoTableLock sync.RWMutex - SubscriptionInner map[string]*model.SubscriptionData + subscriptionInner map[string]*model.SubscriptionData subscriptionInnerLock sync.RWMutex mqClient RocketMqClient allocateMessageQueueStrategy service_allocate_message.AllocateMessageQueueStrategy @@ -48,8 +48,8 @@ type Rebalance struct { consumerConfig *rocketmqm.MqConsumerConfig } -//when invoke GET_CONSUMER_RUNNING_INFO, GetMqTableInfo will return ProcessQueueInfo -func (r *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { +//when invoke GET_CONSUMER_RUNNING_INFO, getMqTableInfo will return ProcessQueueInfo +func (r *rebalance) getMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() mqTable := map[model.MessageQueue]model.ProcessQueueInfo{} @@ -59,13 +59,13 @@ func (r *Rebalance) GetMqTableInfo() map[model.MessageQueue]model.ProcessQueueIn return mqTable } -func (r *Rebalance) GetProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { +func (r *rebalance) getProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() return r.processQueueTable[messageQueue] } -func (r *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int64) { +func (r *rebalance) clearProcessQueue(offsetTable map[model.MessageQueue]int64) { defer r.processQueueTableLock.Unlock() r.processQueueTableLock.Lock() for mq, _ := range offsetTable { @@ -78,7 +78,7 @@ func (r *Rebalance) ClearProcessQueue(offsetTable map[model.MessageQueue]int64) } -func (r *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { +func (r *rebalance) getProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() for messageQueue, processQueue := range r.processQueueTable { @@ -89,19 +89,19 @@ func (r *Rebalance) GetProcessQueueList() (messageQueueList []model.MessageQueue } //removeUnnecessaryMessageQueue you should drop it first -func (r *Rebalance) RemoveProcessQueue(messageQueue *model.MessageQueue) { +func (r *rebalance) removeProcessQueue(messageQueue *model.MessageQueue) { r.offsetStore.Persist(messageQueue) r.offsetStore.RemoveOffset(messageQueue) r.removeMessageQueueFromMap(*messageQueue) } -func (r *Rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { +func (r *rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { defer r.processQueueTableLock.Unlock() r.processQueueTableLock.Lock() delete(r.processQueueTable, messageQueue) } -func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmqm.MqConsumerConfig) *Rebalance { +func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmqm.MqConsumerConfig) *rebalance { subscriptionInner := make(map[string]*model.SubscriptionData) for topic, subExpression := range subscription { subData := &model.SubscriptionData{ @@ -118,11 +118,11 @@ func NewRebalance(groupName string, subscription map[string]string, mqClient Roc SubString: "*", SubVersion: time.Now().Unix(), } - return &Rebalance{ + return &rebalance{ groupName: groupName, mqClient: mqClient, offsetStore: offsetStore, - SubscriptionInner: subscriptionInner, + subscriptionInner: subscriptionInner, allocateMessageQueueStrategy: service_allocate_message.GetAllocateMessageQueueStrategyByConfig("default"), messageModel: "CLUSTERING", processQueueTable: make(map[model.MessageQueue]*model.ProcessQueue), @@ -130,30 +130,30 @@ func NewRebalance(groupName string, subscription map[string]string, mqClient Roc } } -func (r *Rebalance) DoRebalance() { +func (r *rebalance) doRebalance() { r.mutex.Lock() defer r.mutex.Unlock() - for topic, _ := range r.SubscriptionInner { + for topic, _ := range r.subscriptionInner { r.rebalanceByTopic(topic) } } -type ConsumerIdSorter []string +type consumerIdSorter []string -func (self ConsumerIdSorter) Len() int { - return len(self) +func (c consumerIdSorter) Len() int { + return len(c) } -func (self ConsumerIdSorter) Swap(i, j int) { - self[i], self[j] = self[j], self[i] +func (c consumerIdSorter) Swap(i, j int) { + c[i], c[j] = c[j], c[i] } -func (self ConsumerIdSorter) Less(i, j int) bool { - if self[i] < self[j] { +func (c consumerIdSorter) Less(i, j int) bool { + if c[i] < c[j] { return true } return false } -func (r *Rebalance) rebalanceByTopic(topic string) error { +func (r *rebalance) rebalanceByTopic(topic string) error { var cidAll []string cidAll, err := r.findConsumerIdList(topic, r.groupName) if err != nil { @@ -165,7 +165,7 @@ func (r *Rebalance) rebalanceByTopic(topic string) error { r.topicSubscribeInfoTableLock.RUnlock() if len(mqs) > 0 && len(cidAll) > 0 { var messageQueues model.MessageQueues = mqs - var consumerIdSorter ConsumerIdSorter = cidAll + var consumerIdSorter consumerIdSorter = cidAll sort.Sort(messageQueues) sort.Sort(consumerIdSorter) @@ -182,14 +182,14 @@ func (r *Rebalance) rebalanceByTopic(topic string) error { return nil } -func (r *Rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() r.removeTheQueueDontBelongHere(topic, mqSet) r.putTheQueueToProcessQueueTable(topic, mqSet) } -func (r *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { // there is n^2 todo improve for key, value := range r.processQueueTable { if topic != key.Topic { @@ -209,7 +209,7 @@ func (r *Rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.Mes } } -func (r *Rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { for index, mq := range mqSet { _, ok := r.processQueueTable[mq] if !ok { @@ -224,7 +224,7 @@ func (r *Rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.M } } -func (r *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { +func (r *rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { var result int64 = -1 lastOffset := r.offsetStore.ReadOffset(mq, READ_FROM_STORE) switch r.consumerConfig.ConsumeFromWhere { @@ -264,7 +264,7 @@ func (r *Rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { return result } -func (r *Rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { +func (r *rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { brokerAddr, ok := r.mqClient.FindBrokerAddrByTopic(topic) if !ok { err := r.mqClient.UpdateTopicRouteInfoFromNameServer(topic) @@ -282,7 +282,7 @@ func (r *Rebalance) findConsumerIdList(topic string, groupName string) ([]string } -func (r *Rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, timeoutMillis int64) ([]string, error) { +func (r *rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, timeoutMillis int64) ([]string, error) { requestHeader := new(header.GetConsumerListByGroupRequestHeader) requestHeader.ConsumerGroup = consumerGroup diff --git a/rocketmq-go/manage/rebalance_controller.go b/rocketmq-go/kernel/rebalance_controller.go similarity index 95% rename from rocketmq-go/manage/rebalance_controller.go rename to rocketmq-go/kernel/rebalance_controller.go index c1e82a9a6..171585238 100644 --- a/rocketmq-go/manage/rebalance_controller.go +++ b/rocketmq-go/kernel/rebalance_controller.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel type RebalanceController struct { clientFactory *clientFactory @@ -29,6 +29,6 @@ func NewRebalanceController(clientFactory *clientFactory) *RebalanceController { func (self *RebalanceController) doRebalance() { for _, consumer := range self.clientFactory.consumerTable { - consumer.rebalance.DoRebalance() + consumer.rebalance.doRebalance() } } diff --git a/rocketmq-go/manage/tasks.go b/rocketmq-go/kernel/tasks.go similarity index 99% rename from rocketmq-go/manage/tasks.go rename to rocketmq-go/kernel/tasks.go index 475dc2170..0c39c1f5c 100644 --- a/rocketmq-go/manage/tasks.go +++ b/rocketmq-go/kernel/tasks.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package manage +package kernel import ( "math/rand" diff --git a/rocketmq-go/util/structs/field.go b/rocketmq-go/util/structs/field.go deleted file mode 100644 index 97721208b..000000000 --- a/rocketmq-go/util/structs/field.go +++ /dev/null @@ -1,158 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package structs - -import ( - "errors" - "fmt" - "reflect" -) - -var ( - errNotExported = errors.New("field is not exported") - errNotSettable = errors.New("field is not settable") -) - -// Field represents a single struct field that encapsulates high level -// functions around the field. -type Field struct { - value reflect.Value - field reflect.StructField - defaultTag string -} - -// Tag returns the value associated with key in the tag string. If there is no -// such key in the tag, Tag returns the empty string. -func (f *Field) Tag(key string) string { - return f.field.Tag.Get(key) -} - -// Value returns the underlying value of the field. It panics if the field -// is not exported. -func (f *Field) Value() interface{} { - return f.value.Interface() -} - -// IsEmbedded returns true if the given field is an anonymous field (embedded) -func (f *Field) IsEmbedded() bool { - return f.field.Anonymous -} - -// IsExported returns true if the given field is exported. -func (f *Field) IsExported() bool { - return f.field.PkgPath == "" -} - -// IsZero returns true if the given field is not initialized (has a zero value). -// It panics if the field is not exported. -func (f *Field) IsZero() bool { - zero := reflect.Zero(f.value.Type()).Interface() - current := f.Value() - - return reflect.DeepEqual(current, zero) -} - -// Name returns the name of the given field -func (f *Field) Name() string { - return f.field.Name -} - -// Kind returns the fields kind, such as "string", "map", "bool", etc .. -func (f *Field) Kind() reflect.Kind { - return f.value.Kind() -} - -// Set sets the field to given value v. It returns an error if the field is not -// settable (not addressable or not exported) or if the given value's type -// doesn't match the fields type. -func (f *Field) Set(val interface{}) error { - // we can't set unexported fields, so be sure this field is exported - if !f.IsExported() { - return errNotExported - } - - // do we get here? not sure... - if !f.value.CanSet() { - return errNotSettable - } - - given := reflect.ValueOf(val) - - if f.value.Kind() != given.Kind() { - return fmt.Errorf("wrong kind. got: %s want: %s", given.Kind(), f.value.Kind()) - } - - f.value.Set(given) - return nil -} - -// Zero sets the field to its zero value. It returns an error if the field is not -// settable (not addressable or not exported). -func (f *Field) Zero() error { - zero := reflect.Zero(f.value.Type()).Interface() - return f.Set(zero) -} - -// Fields returns a slice of Fields. This is particular handy to get the fields -// of a nested struct . A struct tag with the content of "-" ignores the -// checking of that particular field. Example: -// -// // Field is ignored by this package. -// Field *http.Request `structs:"-"` -// -// It panics if field is not exported or if field's kind is not struct -func (f *Field) Fields() []*Field { - return getFields(f.value, f.defaultTag) -} - -// Field returns the field from a nested struct. It panics if the nested struct -// is not exported or if the field was not found. -func (f *Field) Field(name string) *Field { - field, ok := f.FieldOk(name) - if !ok { - panic("field not found") - } - - return field -} - -// FieldOk returns the field from a nested struct. The boolean returns whether -// the field was found (true) or not (false). -func (f *Field) FieldOk(name string) (*Field, bool) { - value := &f.value - // value must be settable so we need to make sure it holds the address of the - // variable and not a copy, so we can pass the pointer to strctVal instead of a - // copy (which is not assigned to any variable, hence not settable). - // see "https://blog.golang.org/laws-of-reflection#TOC_8." - if f.value.Kind() != reflect.Ptr { - a := f.value.Addr() - value = &a - } - v := strctVal(value.Interface()) - t := v.Type() - - field, ok := t.FieldByName(name) - if !ok { - return nil, false - } - - return &Field{ - field: field, - value: v.FieldByName(name), - }, true -} diff --git a/rocketmq-go/util/structs/structs.go b/rocketmq-go/util/structs/structs.go deleted file mode 100644 index 5cce7c86b..000000000 --- a/rocketmq-go/util/structs/structs.go +++ /dev/null @@ -1,597 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package structs - -import ( - "fmt" - - "reflect" - "strings" -) - -var ( - // DefaultTagName is the default tag name for struct fields which provides - // a more granular to tweak certain structs. Lookup the necessary functions - // for more info. - DefaultTagName = "structs" // struct's field default tag name -) - -// Struct encapsulates a struct type to provide several high level functions -// around the struct. -type Struct struct { - raw interface{} - value reflect.Value - TagName string -} - -// New returns a new *Struct with the struct s. It panics if the s's kind is -// not struct. -func New(s interface{}) *Struct { - return &Struct{ - raw: s, - value: strctVal(s), - TagName: DefaultTagName, - } -} - -// Map converts the given struct to a map[string]interface{}, where the keys -// of the map are the field names and the values of the map the associated -// values of the fields. The default key string is the struct field name but -// can be changed in the struct field's tag value. The "structs" key in the -// struct's field tag value is the key name. Example: -// -// // Field appears in map as key "myName". -// Name string `structs:"myName"` -// -// A tag value with the content of "-" ignores that particular field. Example: -// -// // Field is ignored by this package. -// Field bool `structs:"-"` -// -// A tag value with the content of "string" uses the stringer to get the value. Example: -// -// // The value will be output of Animal's String() func. -// // Map will panic if Animal does not implement String(). -// Field *Animal `structs:"field,string"` -// -// A tag value with the option of "flatten" used in a struct field is to flatten its fields -// in the output map. Example: -// -// // The FieldStruct's fields will be flattened into the output map. -// FieldStruct time.Time `structs:",flatten"` -// -// A tag value with the option of "omitnested" stops iterating further if the type -// is a struct. Example: -// -// // Field is not processed further by this package. -// Field time.Time `structs:"myName,omitnested"` -// Field *http.Request `structs:",omitnested"` -// -// A tag value with the option of "omitempty" ignores that particular field if -// the field value is empty. Example: -// -// // Field appears in map as key "myName", but the field is -// // skipped if empty. -// Field string `structs:"myName,omitempty"` -// -// // Field appears in map as key "Field" (the default), but -// // the field is skipped if empty. -// Field string `structs:",omitempty"` -// -// Note that only exported fields of a struct can be accessed, non exported -// fields will be neglected. -func (s *Struct) Map() map[string]interface{} { - out := make(map[string]interface{}) - s.FillMap(out) - return out -} - -// FillMap is the same as Map. Instead of returning the output, it fills the -// given map. -func (s *Struct) FillMap(out map[string]interface{}) { - if out == nil { - return - } - - fields := s.structFields() - - for _, field := range fields { - name := field.Name - smallName := strings.Replace(name, string(name[0]), string(strings.ToLower(string(name[0]))), 1) //todo - val := s.value.FieldByName(name) - isSubStruct := false - var finalVal interface{} - - tagName, tagOpts := parseTag(field.Tag.Get(s.TagName)) - if tagName != "" { - name = tagName - } - - // if the value is a zero value and the field is marked as omitempty do - // not include - if tagOpts.Has("omitempty") { - zero := reflect.Zero(val.Type()).Interface() - current := val.Interface() - - if reflect.DeepEqual(current, zero) { - continue - } - } - - if !tagOpts.Has("omitnested") { - finalVal = s.nested(val) - - v := reflect.ValueOf(val.Interface()) - if v.Kind() == reflect.Ptr { - v = v.Elem() - } - - switch v.Kind() { - case reflect.Map, reflect.Struct: - isSubStruct = true - } - } else { - finalVal = val.Interface() - } - - if tagOpts.Has("string") { - s, ok := val.Interface().(fmt.Stringer) - if ok { - out[smallName] = s.String() - } - continue - } - - if isSubStruct && (tagOpts.Has("flatten")) { - for k := range finalVal.(map[string]interface{}) { - out[k] = finalVal.(map[string]interface{})[k] - } - } else { - out[smallName] = finalVal - } - } -} - -// Values converts the given s struct's field values to a []interface{}. A -// struct tag with the content of "-" ignores the that particular field. -// Example: -// -// // Field is ignored by this package. -// Field int `structs:"-"` -// -// A value with the option of "omitnested" stops iterating further if the type -// is a struct. Example: -// -// // Fields is not processed further by this package. -// Field time.Time `structs:",omitnested"` -// Field *http.Request `structs:",omitnested"` -// -// A tag value with the option of "omitempty" ignores that particular field and -// is not added to the values if the field value is empty. Example: -// -// // Field is skipped if empty -// Field string `structs:",omitempty"` -// -// Note that only exported fields of a struct can be accessed, non exported -// fields will be neglected. -func (s *Struct) Values() []interface{} { - fields := s.structFields() - - var t []interface{} - - for _, field := range fields { - val := s.value.FieldByName(field.Name) - - _, tagOpts := parseTag(field.Tag.Get(s.TagName)) - - // if the value is a zero value and the field is marked as omitempty do - // not include - if tagOpts.Has("omitempty") { - zero := reflect.Zero(val.Type()).Interface() - current := val.Interface() - - if reflect.DeepEqual(current, zero) { - continue - } - } - - if tagOpts.Has("string") { - s, ok := val.Interface().(fmt.Stringer) - if ok { - t = append(t, s.String()) - } - continue - } - - if IsStruct(val.Interface()) && !tagOpts.Has("omitnested") { - // look out for embedded structs, and convert them to a - // []interface{} to be added to the final values slice - for _, embeddedVal := range Values(val.Interface()) { - t = append(t, embeddedVal) - } - } else { - t = append(t, val.Interface()) - } - } - - return t -} - -// Fields returns a slice of Fields. A struct tag with the content of "-" -// ignores the checking of that particular field. Example: -// -// // Field is ignored by this package. -// Field bool `structs:"-"` -// -// It panics if s's kind is not struct. -func (s *Struct) Fields() []*Field { - return getFields(s.value, s.TagName) -} - -// Names returns a slice of field names. A struct tag with the content of "-" -// ignores the checking of that particular field. Example: -// -// // Field is ignored by this package. -// Field bool `structs:"-"` -// -// It panics if s's kind is not struct. -func (s *Struct) Names() []string { - fields := getFields(s.value, s.TagName) - - names := make([]string, len(fields)) - - for i, field := range fields { - names[i] = field.Name() - } - - return names -} - -func getFields(v reflect.Value, tagName string) []*Field { - if v.Kind() == reflect.Ptr { - v = v.Elem() - } - - t := v.Type() - - var fields []*Field - - for i := 0; i < t.NumField(); i++ { - field := t.Field(i) - - if tag := field.Tag.Get(tagName); tag == "-" { - continue - } - - f := &Field{ - field: field, - value: v.FieldByName(field.Name), - } - - fields = append(fields, f) - - } - - return fields -} - -// Field returns a new Field struct that provides several high level functions -// around a single struct field entity. It panics if the field is not found. -func (s *Struct) Field(name string) *Field { - f, ok := s.FieldOk(name) - if !ok { - panic("field not found") - } - - return f -} - -// FieldOk returns a new Field struct that provides several high level functions -// around a single struct field entity. The boolean returns true if the field -// was found. -func (s *Struct) FieldOk(name string) (*Field, bool) { - t := s.value.Type() - - field, ok := t.FieldByName(name) - if !ok { - return nil, false - } - - return &Field{ - field: field, - value: s.value.FieldByName(name), - defaultTag: s.TagName, - }, true -} - -// IsZero returns true if all fields in a struct is a zero value (not -// initialized) A struct tag with the content of "-" ignores the checking of -// that particular field. Example: -// -// // Field is ignored by this package. -// Field bool `structs:"-"` -// -// A value with the option of "omitnested" stops iterating further if the type -// is a struct. Example: -// -// // Field is not processed further by this package. -// Field time.Time `structs:"myName,omitnested"` -// Field *http.Request `structs:",omitnested"` -// -// Note that only exported fields of a struct can be accessed, non exported -// fields will be neglected. It panics if s's kind is not struct. -func (s *Struct) IsZero() bool { - fields := s.structFields() - - for _, field := range fields { - val := s.value.FieldByName(field.Name) - - _, tagOpts := parseTag(field.Tag.Get(s.TagName)) - - if IsStruct(val.Interface()) && !tagOpts.Has("omitnested") { - ok := IsZero(val.Interface()) - if !ok { - return false - } - - continue - } - - // zero value of the given field, such as "" for string, 0 for int - zero := reflect.Zero(val.Type()).Interface() - - // current value of the given field - current := val.Interface() - - if !reflect.DeepEqual(current, zero) { - return false - } - } - - return true -} - -// HasZero returns true if a field in a struct is not initialized (zero value). -// A struct tag with the content of "-" ignores the checking of that particular -// field. Example: -// -// // Field is ignored by this package. -// Field bool `structs:"-"` -// -// A value with the option of "omitnested" stops iterating further if the type -// is a struct. Example: -// -// // Field is not processed further by this package. -// Field time.Time `structs:"myName,omitnested"` -// Field *http.Request `structs:",omitnested"` -// -// Note that only exported fields of a struct can be accessed, non exported -// fields will be neglected. It panics if s's kind is not struct. -func (s *Struct) HasZero() bool { - fields := s.structFields() - - for _, field := range fields { - val := s.value.FieldByName(field.Name) - - _, tagOpts := parseTag(field.Tag.Get(s.TagName)) - - if IsStruct(val.Interface()) && !tagOpts.Has("omitnested") { - ok := HasZero(val.Interface()) - if ok { - return true - } - - continue - } - - // zero value of the given field, such as "" for string, 0 for int - zero := reflect.Zero(val.Type()).Interface() - - // current value of the given field - current := val.Interface() - - if reflect.DeepEqual(current, zero) { - return true - } - } - - return false -} - -// Name returns the structs's type name within its package. For more info refer -// to Name() function. -func (s *Struct) Name() string { - return s.value.Type().Name() -} - -// structFields returns the exported struct fields for a given s struct. This -// is a convenient helper method to avoid duplicate code in some of the -// functions. -func (s *Struct) structFields() []reflect.StructField { - t := s.value.Type() - - var f []reflect.StructField - - for i := 0; i < t.NumField(); i++ { - field := t.Field(i) - // we can't access the value of unexported fields - if field.PkgPath != "" { - continue - } - - // don't check if it's omitted - if tag := field.Tag.Get(s.TagName); tag == "-" { - continue - } - - f = append(f, field) - } - - return f -} - -func strctVal(s interface{}) reflect.Value { - v := reflect.ValueOf(s) - - // if pointer get the underlying element≤ - for v.Kind() == reflect.Ptr { - v = v.Elem() - } - - if v.Kind() != reflect.Struct { - panic("not struct") - } - - return v -} - -// Map converts the given struct to a map[string]interface{}. For more info -// refer to Struct types Map() method. It panics if s's kind is not struct. -func Map(s interface{}) map[string]interface{} { - return New(s).Map() -} - -// FillMap is the same as Map. Instead of returning the output, it fills the -// given map. -func FillMap(s interface{}, out map[string]interface{}) { - New(s).FillMap(out) -} - -// Values converts the given struct to a []interface{}. For more info refer to -// Struct types Values() method. It panics if s's kind is not struct. -func Values(s interface{}) []interface{} { - return New(s).Values() -} - -// Fields returns a slice of *Field. For more info refer to Struct types -// Fields() method. It panics if s's kind is not struct. -func Fields(s interface{}) []*Field { - return New(s).Fields() -} - -// Names returns a slice of field names. For more info refer to Struct types -// Names() method. It panics if s's kind is not struct. -func Names(s interface{}) []string { - return New(s).Names() -} - -// IsZero returns true if all fields is equal to a zero value. For more info -// refer to Struct types IsZero() method. It panics if s's kind is not struct. -func IsZero(s interface{}) bool { - return New(s).IsZero() -} - -// HasZero returns true if any field is equal to a zero value. For more info -// refer to Struct types HasZero() method. It panics if s's kind is not struct. -func HasZero(s interface{}) bool { - return New(s).HasZero() -} - -// IsStruct returns true if the given variable is a struct or a pointer to -// struct. -func IsStruct(s interface{}) bool { - v := reflect.ValueOf(s) - if v.Kind() == reflect.Ptr { - v = v.Elem() - } - - // uninitialized zero value of a struct - if v.Kind() == reflect.Invalid { - return false - } - - return v.Kind() == reflect.Struct -} - -// Name returns the structs's type name within its package. It returns an -// empty string for unnamed types. It panics if s's kind is not struct. -func Name(s interface{}) string { - return New(s).Name() -} - -// nested retrieves recursively all types for the given value and returns the -// nested value. -func (s *Struct) nested(val reflect.Value) interface{} { - var finalVal interface{} - - v := reflect.ValueOf(val.Interface()) - if v.Kind() == reflect.Ptr { - v = v.Elem() - } - - switch v.Kind() { - case reflect.Struct: - n := New(val.Interface()) - n.TagName = s.TagName - m := n.Map() - - // do not add the converted value if there are no exported fields, ie: - // time.Time - if len(m) == 0 { - finalVal = val.Interface() - } else { - finalVal = m - } - case reflect.Map: - v := val.Type().Elem() - if v.Kind() == reflect.Ptr { - v = v.Elem() - } - - // only iterate over struct types, ie: map[string]StructType, - // map[string][]StructType, - if v.Kind() == reflect.Struct || - (v.Kind() == reflect.Slice && v.Elem().Kind() == reflect.Struct) { - m := make(map[string]interface{}, val.Len()) - for _, k := range val.MapKeys() { - m[k.String()] = s.nested(val.MapIndex(k)) - } - finalVal = m - break - } - - // TODO(arslan): should this be optional? - finalVal = val.Interface() - case reflect.Slice, reflect.Array: - if val.Type().Kind() == reflect.Interface { - finalVal = val.Interface() - break - } - - // TODO(arslan): should this be optional? - // do not iterate of non struct types, just pass the value. Ie: []int, - // []string, co... We only iterate further if it's a struct. - // i.e []foo or []*foo - if val.Type().Elem().Kind() != reflect.Struct && - !(val.Type().Elem().Kind() == reflect.Ptr && - val.Type().Elem().Elem().Kind() == reflect.Struct) { - finalVal = val.Interface() - break - } - - slices := make([]interface{}, val.Len(), val.Len()) - for x := 0; x < val.Len(); x++ { - slices[x] = s.nested(val.Index(x)) - } - finalVal = slices - default: - finalVal = val.Interface() - } - - return finalVal -} diff --git a/rocketmq-go/util/structs/tags.go b/rocketmq-go/util/structs/tags.go deleted file mode 100644 index 2dcf66b32..000000000 --- a/rocketmq-go/util/structs/tags.go +++ /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. -*/ - -package structs - -import "strings" - -// tagOptions contains a slice of tag options -type tagOptions []string - -// Has returns true if the given optiton is available in tagOptions -func (t tagOptions) Has(opt string) bool { - for _, tagOpt := range t { - if tagOpt == opt { - return true - } - } - - return false -} - -// parseTag splits a struct field's tag into its name and a list of options -// which comes after a name. A tag is in the form of: "name,option1,option2". -// The name can be neglectected. -func parseTag(tag string) (string, tagOptions) { - // tag is one of followings: - // "" - // "name" - // "name,opt" - // "name,opt,opt2" - // ",opt" - - res := strings.Split(tag, ",") - return res[0], res[1:] -} From 4f2827765b0d3f215d25e5b53399de36dab8cf5b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:10:03 +0800 Subject: [PATCH 44/88] manage->kernel --- rocketmq-go/kernel/consume_message_service.go | 4 ++-- rocketmq-go/kernel/producer_service_for_send_back.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 27a68b4c5..4843612cc 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -50,7 +50,7 @@ func NewConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageL func (c *consumeMessageConcurrentlyServiceImpl) init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) { c.consumerGroup = consumerGroup c.offsetStore = offsetStore - c.sendMessageBackProducerService.InitSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) + c.sendMessageBackProducerService.initSendMessageBackProducerService(consumerGroup, mqClient, defaultProducerService, consumerConfig) c.consumerConfig = consumerConfig } @@ -84,7 +84,7 @@ func (c *consumeMessageConcurrentlyServiceImpl) convert2ConsumeType(msgs []messa } func (c *consumeMessageConcurrentlyServiceImpl) sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) { - err = c.sendMessageBackProducerService.SendMessageBack(messageExt, 0, brokerName) + err = c.sendMessageBackProducerService.sendMessageBack(messageExt, 0, brokerName) return } diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index d5ff43f01..55ba7ee63 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -29,8 +29,8 @@ import ( ) type sendMessageBackProducerService interface { - SendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) - InitSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) + sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) + initSendMessageBackProducerService(consumerGroup string, mqClient RocketMqClient, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) } type sendMessageBackProducerServiceImpl struct { From 45526629f9b8d15251921cd2dc823e6038b281d6 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:13:46 +0800 Subject: [PATCH 45/88] change rebalance.go --- rocketmq-go/kernel/mq_client_manage.go | 2 +- rocketmq-go/kernel/rebalance.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 8f197eff5..02e4bfbeb 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -78,7 +78,7 @@ func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { consumer.mqClient = m.mqClient consumer.offsetStore = RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) m.clientFactory.consumerTable[consumer.consumerGroup] = consumer - consumer.rebalance = NewRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) + consumer.rebalance = newRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) fmt.Println(consumer.consumeMessageService) diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 84a25c025..62082b529 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -68,7 +68,7 @@ func (r *rebalance) getProcessQueue(messageQueue model.MessageQueue) *model.Proc func (r *rebalance) clearProcessQueue(offsetTable map[model.MessageQueue]int64) { defer r.processQueueTableLock.Unlock() r.processQueueTableLock.Lock() - for mq, _ := range offsetTable { + for mq := range offsetTable { processQueue, ok := r.processQueueTable[mq] if !ok { continue @@ -101,7 +101,7 @@ func (r *rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { } -func NewRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmqm.MqConsumerConfig) *rebalance { +func newRebalance(groupName string, subscription map[string]string, mqClient RocketMqClient, offsetStore OffsetStore, consumerConfig *rocketmqm.MqConsumerConfig) *rebalance { subscriptionInner := make(map[string]*model.SubscriptionData) for topic, subExpression := range subscription { subData := &model.SubscriptionData{ @@ -133,7 +133,7 @@ func NewRebalance(groupName string, subscription map[string]string, mqClient Roc func (r *rebalance) doRebalance() { r.mutex.Lock() defer r.mutex.Unlock() - for topic, _ := range r.subscriptionInner { + for topic := range r.subscriptionInner { r.rebalanceByTopic(topic) } } From d3ece643b02ffb61a2c0aaffbb13f11baac3cdf2 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:19:54 +0800 Subject: [PATCH 46/88] change rebalance.go --- rocketmq-go/model/request_code.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-go/model/request_code.go b/rocketmq-go/model/request_code.go index c70fa1e25..2bb28a4b3 100644 --- a/rocketmq-go/model/request_code.go +++ b/rocketmq-go/model/request_code.go @@ -18,7 +18,7 @@ limitations under the License. package model const ( - // send message + //SendMsg send message SendMsg = 10 // subscribe message PullMsg = 11 From fee140e2848a29c5c5c614554d4d93eac1ea4fcb Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:21:48 +0800 Subject: [PATCH 47/88] remove request_code and response_code --- rocketmq-go/model/request_code.go | 181 ----------------------------- rocketmq-go/model/response_code.go | 84 ------------- 2 files changed, 265 deletions(-) delete mode 100644 rocketmq-go/model/request_code.go delete mode 100644 rocketmq-go/model/response_code.go diff --git a/rocketmq-go/model/request_code.go b/rocketmq-go/model/request_code.go deleted file mode 100644 index 2bb28a4b3..000000000 --- a/rocketmq-go/model/request_code.go +++ /dev/null @@ -1,181 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -const ( - //SendMsg send message - SendMsg = 10 - // subscribe message - PullMsg = 11 - // query message - QueryMessage = 12 - // queryOffset - QueryBrokerOffset = 13 - // query Consumer Offset - QueryConsumerOffset = 14 - // update Consumer Offset - UpdateConsumerOffset = 15 - // update or increase a topic - UpdateAndCreateTopic = 17 - // get all config of topic (Slave and Namesrv query the config to master) - GetAllTopicConfig = 21 - // get all config (Slave and Namesrv query the config to master) - GetTopicConfigList = 22 - // get all name list of topic - GetTopicNameList = 23 - // update config - UpdateBrokerConfig = 25 - // get config - GetBrokerConfig = 26 - // trigger delete files - TriggerDeleteFILES = 27 - // get runtime information - GetBrokerRuntimeInfo = 28 - // search offset by timestamp - SearchOffsetByTimeStamp = 29 - // query max offset of queue - GetMaxOffset = 30 - // query min offset of queue - GetMinOffset = 31 - // query earliest message store time - GetEarliestMsgStoreTime = 32 - // query message by id - ViewMsgById = 33 - // client send heartbeat to broker, and register self - HeartBeat = 34 - // unregister client - UnregisterClient = 35 - // consumer send message back to broker when can't process message - ConsumerSendMsgBack = 36 - // Commit or Rollback transaction - EndTransaction = 37 - // get ConsumerId list by GroupName - GetConsumerListByGroup = 38 - // ckeck transaction state from producer - CheckTransactionState = 39 - // broker notify consumer ids changed - NotifyConsumerIdsChanged = 40 - // Consumer lock queue to master - LockBatchMq = 41 - // Consumer unlock queue to master - UNLockBatchMq = 42 - // get all consumer offset - GetAllConsumerOffset = 43 - // get all delay offset - GetAllDelayOffset = 45 - // put kv config to Namesrv - PutKVConfig = 100 - // get kv config to Namesrv - GetKVConfig = 101 - // delete kv config to Namesrv - DeleteKVConfig = 102 - // register a broker to Namesrv. As data is persistent, - // the broker will overwrite if old config existed. - RegisterBroker = 103 - // register a broker - UnregisterBroker = 104 - // get broker name, queue numbers by topic. - GetRouteinfoByTopic = 105 - // get all registered broker to namesrv info - GetBrokerClusterInfo = 106 - UpdateAndCreateSubscriptionGroup = 200 - GetAllSubscriptionGroupConfig = 201 - GetTopicStatsInfo = 202 - GetConsumerConnList = 203 - GetProducerConnList = 204 - WipeWritePermOfBroker = 205 - - // get all topic list from namesrv - GetAllTopicListFromNamesrv = 206 - // delete subscription group from broker - DeleteSubscriptionGroup = 207 - // get consume stats from broker - GetConsumeStats = 208 - // Suspend Consumer - SuspendConsumer = 209 - // Resume Consumer - ResumeConsumer = 210 - // reset Consumer Offset - ResetConsumerOffsetInConsumer = 211 - // reset Consumer Offset - ResetConsumerOffsetInBroker = 212 - // query which consumer groups consume the msg - WhoConsumeMessage = 214 - - // namesrv delete topic config from broker - DeleteTopicInBroker = 215 - // namesrv delete topic config from namesrv - DeleteTopicInNamesrv = 216 - // namesrv get server ip info by project - GetKvConfigByValue = 217 - // Namesrv delete all server ip by project group - DeleteKvConfigByValue = 218 - // get all KV list by namespace - GetKvlistByNamespace = 219 - - // reset offset - ResetConsumerClientOffset = 220 - // get consumer status from client - GetConsumerStatusFromClient = 221 - // invoke broker to reset offset - InvokeBrokerToResetOffset = 222 - // invoke broker to get consumer status - InvokeBrokerToGetConsumerStatus = 223 - - // query which consumer consume msg - QueryTopicConsumeByWho = 300 - - // get topics by cluster - GetTopicsByCluster = 224 - - // register filter server to broker - RegisterFilterServer = 301 - // register class to filter server - RegisterMsgFilterClass = 302 - // get time span by topic and group - QueryConsumeTimeSpan = 303 - // get all system topics from namesrv - GetSysTopicListFromNS = 304 - // get all system topics from broker - GetSysTopicListFromBroker = 305 - - // clean expired consume queue - CleanExpiredConsumequeue = 306 - - // query consumer memory data by broker - GetConsumerRunningInfo = 307 - - // TODO: query correction offset(transfer component?) - QueryCorrectionOffset = 308 - - // Send msg to one consumer by broker, The msg will immediately consume, - // and return result to broker, broker return result to caller - ConsumeMsgDirectly = 309 - - // send msg with optimized network datagram - SendMsgV2 = 310 - - // get unit topic list - GetUnitTopicList = 311 - GetHasUnitSubTopicList = 312 - GetHasUnitSubUnunitTopicList = 313 - CloneGroupOffset = 314 - - // query all status that broker count - ViewBrokerStatsData = 315 -) diff --git a/rocketmq-go/model/response_code.go b/rocketmq-go/model/response_code.go deleted file mode 100644 index 03a0fb2ac..000000000 --- a/rocketmq-go/model/response_code.go +++ /dev/null @@ -1,84 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -const ( - // success - Success = 0 - // happened unknow error exception - SystemError = 1 - // system busy - SystemBusy = 2 - // unsupport request code - RequestCodeNotSupported = 3 - // transaction failed, because of add db failed - TransactionFailed = 4 - // Broker flush disk timeout - //FlushDiskTimeout = 10 - // Broker slave unavailable, just for sync double write - SlaveNotAvailable = 11 - // Broker write slave timeout, just for sync double write - //FlushSlaveTimeout = 12 - // Broker illegal message - MessageIllegal = 13 - // Broker, Namesrv not available,maybe service is closing or incorrect permission - ServiceNotAvailable = 14 - // Broker, Namesrv unsupport version - VersionNOtSupported = 15 - // Broker, Namesrv no permission for operation with send/receive or other - NoPermission = 16 - // Broker, topic not exist - TopicNotExist = 17 - // Broker, topic already exist - TopicExistAlready = 18 - // Broker message not found when pull - PullNotFound = 19 - // Broker retry immediately, maybe msg was filtered or incorrect notification TODO confirm annotation - PullRetryImmediately = 20 - // Broker pull offset moved, because of too big or to small TODO confirm annotation - PullOffsetMoved = 21 - // Broker query not found - QueryNotFound = 22 - // Broker parse subscription failed - SubscriptionParseFailed = 23 - // Broker subscription relationship not existed - SubscriptionNotExist = 24 - // Broker subscription relationship not latest - SubscriptionNotLatest = 25 - // Broker subscription group not exist - SubscriptionGroupNotExist = 26 - // Producer transaction should commit - TransactionShouldCommit = 200 - // Producer transaction should rollback - TransactionShouldRollback = 201 - // Producer transaction status unknow - TransactionStatusUnknow = 202 - // Producer ProducerGroup transaction error - TransactionStatusGroupWrong = 203 - // unit message,need set buyerId - NoBuyerID = 204 - - // unit message,not current unit msg - NotInCurrentUnit = 205 - - // Consumer not online - ConsumerNotOnline = 206 - - // Consumer consume msg timeout - ConsumeMsgTimeout = 207 -) From 9c4f2fe9042a85a30e0de4980ee21b31946a1a36 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:24:36 +0800 Subject: [PATCH 48/88] send_message_request_header --- rocketmq-go/kernel/header/send_message_request_header.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rocketmq-go/kernel/header/send_message_request_header.go b/rocketmq-go/kernel/header/send_message_request_header.go index 41d4b3e87..212c6a2c3 100644 --- a/rocketmq-go/kernel/header/send_message_request_header.go +++ b/rocketmq-go/kernel/header/send_message_request_header.go @@ -16,7 +16,7 @@ limitations under the License. */ package header - +//SendMessageRequestHeader<-CustomerHeader type SendMessageRequestHeader struct { ProducerGroup string `json:"producerGroup"` Topic string `json:"topic"` @@ -31,7 +31,7 @@ type SendMessageRequestHeader struct { UnitMode bool `json:"unitMode"` MaxReconsumeTimes int `json:"maxReconsumeTimes"` } - +//FromMap to SendMessageRequestHeader func (s *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return } From c33d39da4e4d1259a747ea56f872b4bb95e52112 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:25:53 +0800 Subject: [PATCH 49/88] send_message_request_header --- rocketmq-go/kernel/header/send_message_request_header.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-go/kernel/header/send_message_request_header.go b/rocketmq-go/kernel/header/send_message_request_header.go index 212c6a2c3..82c078302 100644 --- a/rocketmq-go/kernel/header/send_message_request_header.go +++ b/rocketmq-go/kernel/header/send_message_request_header.go @@ -16,7 +16,7 @@ limitations under the License. */ package header -//SendMessageRequestHeader<-CustomerHeader +//SendMessageRequestHeader <- CustomerHeader type SendMessageRequestHeader struct { ProducerGroup string `json:"producerGroup"` Topic string `json:"topic"` From 739f442da20f6386f4122323e09a16c21c0b9e4a Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:35:56 +0800 Subject: [PATCH 50/88] range variable consumer captured by func literal --- rocketmq-go/kernel/clean_expire_msg_controller.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/rocketmq-go/kernel/clean_expire_msg_controller.go b/rocketmq-go/kernel/clean_expire_msg_controller.go index 1d4139ac3..d1551267f 100644 --- a/rocketmq-go/kernel/clean_expire_msg_controller.go +++ b/rocketmq-go/kernel/clean_expire_msg_controller.go @@ -35,15 +35,15 @@ func newCleanExpireMsgController(mqClient RocketMqClient, clientFactory *clientF func (c *cleanExpireMsgController) start() { for _, consumer := range c.clientFactory.consumerTable { - go func() { - cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) + go func(currentConsumer *DefaultMQPushConsumer) { + cleanExpireMsgTimer := time.NewTimer(time.Duration(currentConsumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) //cleanExpireMsgTimer := time.NewTimer(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) for { <-cleanExpireMsgTimer.C - consumer.CleanExpireMsg() - cleanExpireMsgTimer.Reset(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) + currentConsumer.CleanExpireMsg() + cleanExpireMsgTimer.Reset(time.Duration(currentConsumer.ConsumerConfig.ConsumeTimeout) * 1000 * 60 * time.Millisecond) //cleanExpireMsgTimer.Reset(time.Duration(consumer.ConsumerConfig.ConsumeTimeout) * time.Millisecond) } - }() + }(consumer) } } From f073f8af02ccc87c6cb20a68b238751575c1df5f Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:45:41 +0800 Subject: [PATCH 51/88] go_vet --- .../kernel/header/send_message_request_header.go | 2 ++ .../header/update_consumer_offset_request_header.go | 2 ++ rocketmq-go/kernel/mq_client_manage.go | 9 ++++----- rocketmq-go/kernel/offset_store.go | 3 ++- rocketmq-go/kernel/tasks.go | 2 +- rocketmq-go/model/send_result.go | 12 ++++-------- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/rocketmq-go/kernel/header/send_message_request_header.go b/rocketmq-go/kernel/header/send_message_request_header.go index 82c078302..ca47f099d 100644 --- a/rocketmq-go/kernel/header/send_message_request_header.go +++ b/rocketmq-go/kernel/header/send_message_request_header.go @@ -16,6 +16,7 @@ limitations under the License. */ package header + //SendMessageRequestHeader <- CustomerHeader type SendMessageRequestHeader struct { ProducerGroup string `json:"producerGroup"` @@ -31,6 +32,7 @@ type SendMessageRequestHeader struct { UnitMode bool `json:"unitMode"` MaxReconsumeTimes int `json:"maxReconsumeTimes"` } + //FromMap to SendMessageRequestHeader func (s *SendMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return diff --git a/rocketmq-go/kernel/header/update_consumer_offset_request_header.go b/rocketmq-go/kernel/header/update_consumer_offset_request_header.go index 287240cbe..572861d43 100644 --- a/rocketmq-go/kernel/header/update_consumer_offset_request_header.go +++ b/rocketmq-go/kernel/header/update_consumer_offset_request_header.go @@ -19,6 +19,7 @@ package header import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" +//UpdateConsumerOffsetRequestHeader of CustomHeader type UpdateConsumerOffsetRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` Topic string `json:"topic"` @@ -26,6 +27,7 @@ type UpdateConsumerOffsetRequestHeader struct { CommitOffset int64 `json:"commitOffset"` } +//FromMap to UpdateConsumerOffsetRequestHeader func (u *UpdateConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { u.ConsumerGroup = headerMap["consumerGroup"].(string) u.QueueId = util.StrToInt32WithDefaultValue(util.ReadString(headerMap["queueId"]), 0) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 02e4bfbeb..b5a570828 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -30,7 +30,6 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" "strings" - "sync" "time" ) @@ -39,7 +38,7 @@ import ( //} type MqClientManager struct { - rocketMqManagerLock sync.Mutex + //rocketMqManagerLock sync.Mutex BootTimestamp int64 clientFactory *clientFactory mqClient RocketMqClient @@ -190,7 +189,7 @@ func ClientFactoryInit() (clientFactoryInstance *clientFactory) { } //heart beat -func (m MqClientManager) sendHeartbeatToAllBrokerWithLock() error { +func (m *MqClientManager) sendHeartbeatToAllBrokerWithLock() error { heartbeatData := m.prepareHeartbeatData() if len(heartbeatData.ConsumerDataSet) == 0 { return errors.New("send heartbeat error") @@ -200,7 +199,7 @@ func (m MqClientManager) sendHeartbeatToAllBrokerWithLock() error { } //routeInfo -func (m MqClientManager) updateTopicRouteInfoFromNameServer() { +func (m *MqClientManager) updateTopicRouteInfoFromNameServer() { var topicSet []string for _, consumer := range m.clientFactory.consumerTable { for key, _ := range consumer.subscription { @@ -214,7 +213,7 @@ func (m MqClientManager) updateTopicRouteInfoFromNameServer() { } } -func (m MqClientManager) prepareHeartbeatData() *model.HeartbeatData { +func (m *MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData := new(model.HeartbeatData) heartbeatData.ClientId = m.mqClient.GetClientId() heartbeatData.ConsumerDataSet = make([]*model.ConsumerData, 0) diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index e2a8f00c6..1d401c237 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -47,7 +47,7 @@ type RemoteOffsetStore struct { groupName string mqClient RocketMqClient offsetTable map[model.MessageQueue]int64 - offsetTableLock sync.RWMutex + offsetTableLock *sync.RWMutex } func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStore { @@ -55,6 +55,7 @@ func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStor offsetStore.groupName = groupName offsetStore.mqClient = mqClient offsetStore.offsetTable = make(map[model.MessageQueue]int64) + offsetStore.offsetTableLock = new(sync.RWMutex) return offsetStore } func (r *RemoteOffsetStore) RemoveOffset(mq *model.MessageQueue) { diff --git a/rocketmq-go/kernel/tasks.go b/rocketmq-go/kernel/tasks.go index 0c39c1f5c..b10c8104c 100644 --- a/rocketmq-go/kernel/tasks.go +++ b/rocketmq-go/kernel/tasks.go @@ -22,7 +22,7 @@ import ( "time" ) -func (m MqClientManager) startAllScheduledTask() { +func (m *MqClientManager) startAllScheduledTask() { rand.Seed(time.Now().UnixNano()) go func() { updateTopicRouteTimer := time.NewTimer(5 * time.Second) diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go index 2cccd0c8e..d08a0f702 100644 --- a/rocketmq-go/model/send_result.go +++ b/rocketmq-go/model/send_result.go @@ -17,10 +17,6 @@ limitations under the License. package model -import ( - "fmt" -) - type SendStatus int const ( @@ -119,7 +115,7 @@ func (result *SendResult) SetOffsetMsgID(s string) { result.offsetMsgID = s } -func (result *SendResult) String() string { - return fmt.Sprintf("SendResult [sendStatus=%s, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", - result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) -} +//func (result *SendResult) String() string { +// return fmt.Sprintf("SendResult [sendStatus=%s, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", +// result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) +//} From 31d5274511b44743e17a36beed5a602a4a80ce0d Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:51:23 +0800 Subject: [PATCH 52/88] gofmt -s --- rocketmq-go/kernel/header/search_offset_request_header.go | 3 +-- rocketmq-go/kernel/mq_client_manage.go | 8 ++------ 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/rocketmq-go/kernel/header/search_offset_request_header.go b/rocketmq-go/kernel/header/search_offset_request_header.go index 6226f5ff9..b6931f4a9 100644 --- a/rocketmq-go/kernel/header/search_offset_request_header.go +++ b/rocketmq-go/kernel/header/search_offset_request_header.go @@ -17,7 +17,6 @@ limitations under the License. package header -import () import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" type SearchOffsetRequestHeader struct { @@ -31,4 +30,4 @@ func (s *SearchOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { s.Topic = headerMap["queueId"].(string) s.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) return -} +} \ No newline at end of file diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index b5a570828..a84426008 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -33,10 +33,6 @@ import ( "time" ) -//type MqClientManager interface { -// -//} - type MqClientManager struct { //rocketMqManagerLock sync.Mutex BootTimestamp int64 @@ -202,7 +198,7 @@ func (m *MqClientManager) sendHeartbeatToAllBrokerWithLock() error { func (m *MqClientManager) updateTopicRouteInfoFromNameServer() { var topicSet []string for _, consumer := range m.clientFactory.consumerTable { - for key, _ := range consumer.subscription { + for key := range consumer.subscription { topicSet = append(topicSet, key) } } @@ -234,4 +230,4 @@ func (m *MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData.ProducerDataSet = append(heartbeatData.ProducerDataSet, producerData) } return heartbeatData -} +} \ No newline at end of file From 916b85bc6d943e0d46f01f660132a16c5fe7a8fc Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:54:46 +0800 Subject: [PATCH 53/88] gofmt -s --- rocketmq-go/kernel/header/search_offset_request_header.go | 6 ++++-- rocketmq-go/kernel/mq_client_manage.go | 3 +-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/rocketmq-go/kernel/header/search_offset_request_header.go b/rocketmq-go/kernel/header/search_offset_request_header.go index b6931f4a9..a2bdc75ee 100644 --- a/rocketmq-go/kernel/header/search_offset_request_header.go +++ b/rocketmq-go/kernel/header/search_offset_request_header.go @@ -17,7 +17,9 @@ limitations under the License. package header -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" +) type SearchOffsetRequestHeader struct { Topic string `json:"topic"` @@ -30,4 +32,4 @@ func (s *SearchOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { s.Topic = headerMap["queueId"].(string) s.Timestamp = util.StrToInt64WithDefaultValue(headerMap["timestamp"].(string), -1) return -} \ No newline at end of file +} diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index a84426008..5a6a14829 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -22,7 +22,6 @@ import ( "errors" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" @@ -230,4 +229,4 @@ func (m *MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData.ProducerDataSet = append(heartbeatData.ProducerDataSet, producerData) } return heartbeatData -} \ No newline at end of file +} From ea381852a27805f813685f74e554a7f53110eb02 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 00:58:14 +0800 Subject: [PATCH 54/88] update design --- rocketmq-go/docs/package.puml | 25 ++++++++++--------------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index a0214d146..9a87ea5eb 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -8,8 +8,6 @@ interface api.MQConsumer{ interface api.MQClientInstance{ -} -class api.ClientInstanceImpl{ } namespace api{ @@ -74,7 +72,17 @@ namespace kernel{ kernel.PullMessageController *-- kernel.ClientFactory:contains + MqClientManager o-- PullMessageController + MqClientManager o-- ClientFactory + kernel.ClientFactory *-- DefaultMQPushConsumer:contains + kernel.ClientFactory *-- DefaultMQPullConsumer:contains + DefaultMQPushConsumer *-- kernel.PullAPIWrapper : contains + DefaultMQPushConsumer *-- kernel.OffsetStore : contains + DefaultMQPushConsumer *-- kernel.Rebalance : contains + DefaultMQPushConsumer *-- kernel.ConsumeMessageService : contains + kernel.ClientFactory *-- DefaultMQProducer:contains + DefaultMQProducer *-- kernel.MqClient :contains } @@ -102,19 +110,6 @@ namespace remoting { } -namespace manage{ - MqClientManager o-- PullMessageController - MqClientManager o-- ClientFactory - kernel.ClientFactory *-- DefaultMQPushConsumer:contains - kernel.ClientFactory *-- DefaultMQPullConsumer:contains - DefaultMQPushConsumer *-- kernel.PullAPIWrapper : contains - DefaultMQPushConsumer *-- kernel.OffsetStore : contains - DefaultMQPushConsumer *-- kernel.Rebalance : contains - DefaultMQPushConsumer *-- kernel.ConsumeMessageService : contains - kernel.ClientFactory *-- DefaultMQProducer:contains - - DefaultMQProducer *-- kernel.MqClient :contains -} note top of remoting.RemotingClient :(sync|aysc|oneWay) From 509f04d14b78530f7160be60c8ed124d3aa9baa2 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Mon, 21 Aug 2017 01:05:58 +0800 Subject: [PATCH 55/88] update example --- rocketmq-go/example/map_util_a.go | 19 -------------- rocketmq-go/example/simple_consumer.go | 35 -------------------------- 2 files changed, 54 deletions(-) delete mode 100644 rocketmq-go/example/map_util_a.go diff --git a/rocketmq-go/example/map_util_a.go b/rocketmq-go/example/map_util_a.go deleted file mode 100644 index 6b5e59a3a..000000000 --- a/rocketmq-go/example/map_util_a.go +++ /dev/null @@ -1,19 +0,0 @@ -package main - -import ( - "encoding/json" - "fmt" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util/structs" -) - -func main() { - config := rocketmqm.NewRocketMqConsumerConfig() - print(structs.Map(config)) - print(util.Struct2Map(config)) -} -func print(m map[string]interface{}) { - bb, _ := json.Marshal(m) - fmt.Println(string(bb)) -} diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index 1f6a608cd..a5ebdc40e 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -17,41 +17,6 @@ limitations under the License. package main -import ( - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - //"github.com/golang/glog" - //"github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" - "github.com/golang/glog" -) - func main() { - //ch := make(chan string, 10) - //for i := 0; i < 11; i++ { - // ch<-"2345" - //} - mmp := util.New() - go func() { - i := 1 - for true { - i *= 3 - mmp.Set(util.IntToString(i), "2345") - } - }() - go func() { - i := 1 - for true { - i *= 4 - mmp.Set(util.IntToString(i), "2345") - } - }() - - go func() { - for true { - glog.Info(len(mmp.Keys())) - } - }() - select {} } From 3b6b82c363354937fe5ca92219fcb1c64c7c40d2 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 21:46:41 +0800 Subject: [PATCH 56/88] cyclomatic complexity 16 of function (*MqClientManager).initClientRequestProcessor() --- rocketmq-go/kernel/mq_client_manage.go | 126 ++++++++++++++----------- 1 file changed, 69 insertions(+), 57 deletions(-) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 5a6a14829..023857aec 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -90,70 +90,17 @@ func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor r m.rebalanceControllr.doRebalance() break case remoting.RESET_CONSUMER_CLIENT_OFFSET: // struct json key supported - glog.V(2).Info("receive_request_code RESET_CONSUMER_CLIENT_OFFSET") - glog.V(2).Info("op=look cmd body", string(cmd.Body)) - var resetOffsetRequestHeader = &header.ResetOffsetRequestHeader{} - if cmd.ExtFields != nil { - resetOffsetRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct - glog.V(2).Info("op=look ResetOffsetRequestHeader", resetOffsetRequestHeader) - resetOffsetBody := &model.ResetOffsetBody{} - err := resetOffsetBody.Decode(cmd.Body) - if err != nil { - return - } - glog.V(2).Info("op=look resetOffsetBody xxxxx", resetOffsetBody) - m.resetConsumerOffset(resetOffsetRequestHeader.Topic, resetOffsetRequestHeader.Group, resetOffsetBody.OffsetTable) - } + m.resetConsumerClientOffset(cmd) break case remoting.GET_CONSUMER_STATUS_FROM_CLIENT: // useless we can use GET_CONSUMER_RUNNING_INFO instead glog.V(2).Info("receive_request_code GET_CONSUMER_STATUS_FROM_CLIENT") break case remoting.GET_CONSUMER_RUNNING_INFO: - glog.V(2).Info("receive_request_code GET_CONSUMER_RUNNING_INFO") - var getConsumerRunningInfoRequestHeader = &header.GetConsumerRunningInfoRequestHeader{} - if cmd.ExtFields != nil { - getConsumerRunningInfoRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct - consumerRunningInfo := model.ConsumerRunningInfo{} - consumerRunningInfo.Properties = map[string]string{} - defaultMQPushConsumer := m.clientFactory.consumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] - consumerConfigMap := util.Struct2Map(defaultMQPushConsumer.ConsumerConfig) - for key, value := range consumerConfigMap { - consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) - } - - consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.GetRemotingClient().GetNamesrvAddrList(), ";") - consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.getMqTableInfo() - - glog.V(2).Info("op=look consumerRunningInfo", consumerRunningInfo) - jsonByte, err := consumerRunningInfo.Encode() - glog.V(2).Info("op=enCode jsonByte", string(jsonByte)) - if err != nil { - glog.Error(err) - return - } - response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) - } - + response = m.getConsumerRunningInfo(cmd) break case remoting.CONSUME_MESSAGE_DIRECTLY: - glog.V(2).Info("receive_request_code CONSUME_MESSAGE_DIRECTLY") - var consumeMessageDirectlyResultRequestHeader = &header.ConsumeMessageDirectlyResultRequestHeader{} - if cmd.ExtFields != nil { - consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) - messageExt := &DecodeMessage(cmd.Body)[0] - glog.V(2).Info("op=look", messageExt) - defaultMQPushConsumer := m.clientFactory.consumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] - consumeResult, err := defaultMQPushConsumer.consumeMessageService.consumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) - if err != nil { - return - } - jsonByte, err := json.Marshal(consumeResult) - if err != nil { - glog.Error(err) - return - } - response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) - } + response = m.consumeMessageDirectly(cmd) + break default: glog.Error("illeage requestCode ", cmd.Code) } @@ -162,6 +109,71 @@ func (m *MqClientManager) initClientRequestProcessor() (clientRequestProcessor r return } +func (m *MqClientManager) consumeMessageDirectly(cmd *remoting.RemotingCommand) (response *remoting.RemotingCommand) { + glog.V(2).Info("receive_request_code CONSUME_MESSAGE_DIRECTLY") + var consumeMessageDirectlyResultRequestHeader = &header.ConsumeMessageDirectlyResultRequestHeader{} + if cmd.ExtFields != nil { + consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) + messageExt := &DecodeMessage(cmd.Body)[0] + glog.V(2).Info("op=look", messageExt) + defaultMQPushConsumer := m.clientFactory.consumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] + consumeResult, err := defaultMQPushConsumer.consumeMessageService.consumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) + if err != nil { + return + } + jsonByte, err := json.Marshal(consumeResult) + if err != nil { + glog.Error(err) + return + } + response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) + } + return +} + +func (m *MqClientManager) getConsumerRunningInfo(cmd *remoting.RemotingCommand) (response *remoting.RemotingCommand) { + glog.V(2).Info("receive_request_code GET_CONSUMER_RUNNING_INFO") + var getConsumerRunningInfoRequestHeader = &header.GetConsumerRunningInfoRequestHeader{} + if cmd.ExtFields != nil { + getConsumerRunningInfoRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct + consumerRunningInfo := model.ConsumerRunningInfo{} + consumerRunningInfo.Properties = map[string]string{} + defaultMQPushConsumer := m.clientFactory.consumerTable[getConsumerRunningInfoRequestHeader.ConsumerGroup] + consumerConfigMap := util.Struct2Map(defaultMQPushConsumer.ConsumerConfig) + for key, value := range consumerConfigMap { + consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) + } + + consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.GetRemotingClient().GetNamesrvAddrList(), ";") + consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.getMqTableInfo() + + glog.V(2).Info("op=look consumerRunningInfo", consumerRunningInfo) + jsonByte, err := consumerRunningInfo.Encode() + glog.V(2).Info("op=enCode jsonByte", string(jsonByte)) + if err != nil { + glog.Error(err) + return + } + response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) + } +} +func (m *MqClientManager) resetConsumerClientOffset(cmd *remoting.RemotingCommand) { + glog.V(2).Info("receive_request_code RESET_CONSUMER_CLIENT_OFFSET") + glog.V(2).Info("op=look cmd body", string(cmd.Body)) + var resetOffsetRequestHeader = &header.ResetOffsetRequestHeader{} + if cmd.ExtFields != nil { + resetOffsetRequestHeader.FromMap(cmd.ExtFields) //change map[string]interface{} into CustomerHeader struct + glog.V(2).Info("op=look ResetOffsetRequestHeader", resetOffsetRequestHeader) + resetOffsetBody := &model.ResetOffsetBody{} + err := resetOffsetBody.Decode(cmd.Body) + if err != nil { + return + } + glog.V(2).Info("op=look resetOffsetBody xxxxx", resetOffsetBody) + m.resetConsumerOffset(resetOffsetRequestHeader.Topic, resetOffsetRequestHeader.Group, resetOffsetBody.OffsetTable) + } +} + func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { consumer := m.clientFactory.consumerTable[group] if consumer == nil { From 3dda6738b57740e6fd7cdaded296daaeb1440a6b Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 22:07:00 +0800 Subject: [PATCH 57/88] fix cyclomatic complexity 16 of function GetKvStringMap() is high (> 15) (gocyclo) --- rocketmq-go/util/json_util.go | 103 ++++++++++++++++++++-------------- 1 file changed, 62 insertions(+), 41 deletions(-) diff --git a/rocketmq-go/util/json_util.go b/rocketmq-go/util/json_util.go index 5fa6716c5..b7b013497 100644 --- a/rocketmq-go/util/json_util.go +++ b/rocketmq-go/util/json_util.go @@ -45,70 +45,91 @@ type Token struct { ////{"offsetTable":{{"brokerName":"broker-b","queueId":122222,"topic":"GoLang"}:9420,{"brokerName":"broker-b","queueId":2,"topic":"GoLang"}:9184,{"brokerName":"broker-b","queueId":1,"topic":"GoLang"}:9260,{"brokerName":"broker-b","queueId":3,"topic":"GoLang"}:9139}} +type parseInfo struct { + startObjCount int + // 0 begin 1 key 2 value + readType int + nowKey string + nowValue string +} + func GetKvStringMap(str string) (kvMap map[string]string, err error) { var tokenList []Token tokenList, err = parseTokenList(str) kvMap = map[string]string{} - startObjCount := 0 - readType := 0 // 0 begin 1 key 2 value - nowKey := "" - nowValue := "" + currentParseInfo := &parseInfo{ + startObjCount: 0, + readType: 0, + nowKey: "", + nowValue: "", + } for i := 0; i < len(tokenList); i++ { - nowToken := tokenList[i] + nowToken := &tokenList[i] if nowToken.tokenType == START_OBJ { - startObjCount++ + currentParseInfo.startObjCount++ } if nowToken.tokenType == END_OBJ { - startObjCount-- + currentParseInfo.startObjCount-- } - if readType == 0 { + if currentParseInfo.readType == 0 { if nowToken.tokenType != START_OBJ { err = errors.New("json not start with {") return } - readType = 1 - } else if readType == 1 { - if nowToken.tokenType == COLON { //: split k and v - if startObjCount == 1 { - readType = 2 - continue - } - } - if nowToken.tokenType == STRING { - nowKey = nowKey + "\"" + nowToken.tokenValue + "\"" - } else { - nowKey = nowKey + nowToken.tokenValue - } - } else if readType == 2 { - if nowToken.tokenType == COMMA { // , split kv pair - if startObjCount == 1 { - kvMap[nowKey] = nowValue - nowKey = "" - nowValue = "" - readType = 1 - continue - } - } - if nowToken.tokenType == STRING { - nowValue = nowValue + "\"" + nowToken.tokenValue + "\"" - - } else { - if startObjCount > 0 { //use less end } - nowValue = nowValue + nowToken.tokenValue - } + currentParseInfo.readType = 1 + } else if currentParseInfo.readType == 1 { + currentParseInfo = parseKey(currentParseInfo, nowToken) + } else if currentParseInfo.readType == 2 { + var k, v string + currentParseInfo, k, v = parseValue(currentParseInfo, nowToken) + if len(k) > 0 { + kvMap[k] = v } - } else { err = errors.New("this is a bug") return } } - if len(nowKey) > 0 { + if len(currentParseInfo.nowKey) > 0 { + kvMap[currentParseInfo.nowKey] = currentParseInfo.nowValue + } + return +} +func parseValue(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo, key, value string) { + if nowToken.tokenType == COMMA { // , split kv pair + if info.startObjCount == 1 { + key = info.nowKey + value = info.nowValue + info.nowKey = "" + info.nowValue = "" + info.readType = 1 + return + } + } + if nowToken.tokenType == STRING { + info.nowValue = info.nowValue + "\"" + nowToken.tokenValue + "\"" - kvMap[nowKey] = nowValue + } else { + if info.startObjCount > 0 { //use less end } + info.nowValue = info.nowValue + nowToken.tokenValue + } } return } +func parseKey(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo) { + if nowToken.tokenType == COLON { //: split k and v + if info.startObjCount == 1 { + info.readType = 2 + return + } + } + if nowToken.tokenType == STRING { + info.nowKey = info.nowKey + "\"" + nowToken.tokenValue + "\"" + } else { + info.nowKey = info.nowKey + nowToken.tokenValue + } + return info +} func parseTokenList(str string) (tokenList []Token, err error) { From ea3f7158fb38ee2ff4a9745853b8737c4606c2e8 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 22:12:40 +0800 Subject: [PATCH 58/88] fix gocyclo problem --- rocketmq-go/kernel/mq_client_manage.go | 1 + rocketmq-go/kernel/producer_service.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 023857aec..e51b8c49a 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -156,6 +156,7 @@ func (m *MqClientManager) getConsumerRunningInfo(cmd *remoting.RemotingCommand) } response = remoting.NewRemotingCommandWithBody(remoting.SUCCESS, nil, jsonByte) } + return } func (m *MqClientManager) resetConsumerClientOffset(cmd *remoting.RemotingCommand) { glog.V(2).Info("receive_request_code RESET_CONSUMER_CLIENT_OFFSET") diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index c6612b434..7d70548a0 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -112,7 +112,7 @@ func processSendResponse(brokerName string, message *message.MessageImpl, respon break } default: - err = errors.New("response.Code error code") + err = errors.New("response.Code error_code=" + util.IntToString(int(response.Code))) return } var responseHeader = &header.SendMessageResponseHeader{} From 48db05399476cc5c7b232d74284d329413902f0f Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 22:42:53 +0800 Subject: [PATCH 59/88] pullMessage is very complicated, todo check --- rocketmq-go/kernel/pull_message_controller.go | 106 ++++++++++-------- 1 file changed, 59 insertions(+), 47 deletions(-) diff --git a/rocketmq-go/kernel/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go index b429c33f5..4475bb270 100644 --- a/rocketmq-go/kernel/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -117,42 +117,31 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { responseCommand := responseFuture.ResponseCommand if responseCommand.Code == remoting.SUCCESS && len(responseCommand.Body) > 0 { var err error - pullResult := responseCommand.ExtFields - if ok { - if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { - if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { - nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) - if err != nil { - glog.Error(err) - return - } - } - } + nextBeginOffset, err = parseNextBeginOffset(responseCommand) + if err != nil { + return } + //} msgs := DecodeMessage(responseFuture.ResponseCommand.Body) - msgs = FilterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) - if len(msgs) == 0 { - if pullRequest.ProcessQueue.GetMsgCount() == 0 { - defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) - } - } - // + p.updateOffsetIfNeed(msgs, pullRequest, defaultMQPullConsumer, nextBeginOffset) pullRequest.ProcessQueue.PutMessage(msgs) defaultMQPullConsumer.consumeMessageService.submitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) } else { - var err error // change the offset , use nextBeginOffset - pullResult := responseCommand.ExtFields - if ok { - if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { - if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { - nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) - if err != nil { - glog.Error(err) - } - } - } - } + //var err error // change the offset , use nextBeginOffset + //pullResult := responseCommand.ExtFields + //if ok { + // if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { + // if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { + // nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) + // if err != nil { + // glog.Error(err) + // } + // } + // } + nextBeginOffset, _ = parseNextBeginOffset(responseCommand) + + //} if responseCommand.Code == remoting.PULL_NOT_FOUND || responseCommand.Code == remoting.PULL_RETRY_IMMEDIATELY { //NO_NEW_MSG //NO_MATCHED_MSG if pullRequest.ProcessQueue.GetMsgCount() == 0 { @@ -180,28 +169,51 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { } else { glog.Error("responseFuture is nil") } + p.enqueueNextPullRequest(defaultMQPullConsumer, pullRequest, nextBeginOffset) - if pullRequest.ProcessQueue.IsDropped() { - return + } + glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) + p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) +} +func (p *PullMessageController) updateOffsetIfNeed(msgs []message.MessageExtImpl, pullRequest *model.PullRequest, defaultMQPullConsumer *DefaultMQPushConsumer, nextBeginOffset int64) { + if len(msgs) == 0 { + if pullRequest.ProcessQueue.GetMsgCount() == 0 { + defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) } - nextPullRequest := &model.PullRequest{ - ConsumerGroup: pullRequest.ConsumerGroup, - NextOffset: nextBeginOffset, - MessageQueue: pullRequest.MessageQueue, - ProcessQueue: pullRequest.ProcessQueue, + } +} +func parseNextBeginOffset(responseCommand *remoting.RemotingCommand) (nextBeginOffset int64, err error) { + pullResult := responseCommand.ExtFields + if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { + if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { + nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) + if err != nil { + glog.Error(err) + return + } } - if defaultMQPullConsumer.ConsumerConfig.PullInterval > 0 { - go func() { - nextPullTime := time.NewTimer(time.Duration(defaultMQPullConsumer.ConsumerConfig.PullInterval) * time.Millisecond) - <-nextPullTime.C - p.mqClient.EnqueuePullMessageRequest(nextPullRequest) - }() - } else { + } + return +} +func (p *PullMessageController) enqueueNextPullRequest(defaultMQPullConsumer *DefaultMQPushConsumer, pullRequest *model.PullRequest, nextBeginOffset int64) { + if pullRequest.ProcessQueue.IsDropped() { + return + } + nextPullRequest := &model.PullRequest{ + ConsumerGroup: pullRequest.ConsumerGroup, + NextOffset: nextBeginOffset, + MessageQueue: pullRequest.MessageQueue, + ProcessQueue: pullRequest.ProcessQueue, + } + if defaultMQPullConsumer.ConsumerConfig.PullInterval > 0 { + go func() { + nextPullTime := time.NewTimer(time.Duration(defaultMQPullConsumer.ConsumerConfig.PullInterval) * time.Millisecond) + <-nextPullTime.C p.mqClient.EnqueuePullMessageRequest(nextPullRequest) - } + }() + } else { + p.mqClient.EnqueuePullMessageRequest(nextPullRequest) } - glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) - p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } func FilterMessageAgainByTags(msgExts []message.MessageExtImpl, subscriptionTagList []string) (result []message.MessageExtImpl) { result = msgExts From 2d4dffad8720cdd3b0b9b638ea7bb4b6869b7495 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 22:51:31 +0800 Subject: [PATCH 60/88] pullMessage is very complicated, todo check --- rocketmq-go/kernel/pull_message_controller.go | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/rocketmq-go/kernel/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go index 4475bb270..c0ea2436c 100644 --- a/rocketmq-go/kernel/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -116,15 +116,15 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { if responseFuture != nil { responseCommand := responseFuture.ResponseCommand if responseCommand.Code == remoting.SUCCESS && len(responseCommand.Body) > 0 { - var err error - nextBeginOffset, err = parseNextBeginOffset(responseCommand) - if err != nil { - return - } + nextBeginOffset = parseNextBeginOffset(responseCommand) //} msgs := DecodeMessage(responseFuture.ResponseCommand.Body) msgs = FilterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) - p.updateOffsetIfNeed(msgs, pullRequest, defaultMQPullConsumer, nextBeginOffset) + if len(msgs) == 0 { + if pullRequest.ProcessQueue.GetMsgCount() == 0 { + defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) + } + } pullRequest.ProcessQueue.PutMessage(msgs) defaultMQPullConsumer.consumeMessageService.submitConsumeRequest(msgs, pullRequest.ProcessQueue, pullRequest.MessageQueue, true) } else { @@ -139,7 +139,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { // } // } // } - nextBeginOffset, _ = parseNextBeginOffset(responseCommand) + nextBeginOffset = parseNextBeginOffset(responseCommand) //} if responseCommand.Code == remoting.PULL_NOT_FOUND || responseCommand.Code == remoting.PULL_RETRY_IMMEDIATELY { @@ -175,19 +175,22 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { glog.V(2).Infof("requestHeader look offset %s %s %s %s", requestHeader.QueueOffset, requestHeader.Topic, requestHeader.QueueId, requestHeader.CommitOffset) p.consumerPullMessageAsync(pullRequest.MessageQueue.BrokerName, requestHeader, pullCallback) } -func (p *PullMessageController) updateOffsetIfNeed(msgs []message.MessageExtImpl, pullRequest *model.PullRequest, defaultMQPullConsumer *DefaultMQPushConsumer, nextBeginOffset int64) { - if len(msgs) == 0 { - if pullRequest.ProcessQueue.GetMsgCount() == 0 { - defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) - } - } -} -func parseNextBeginOffset(responseCommand *remoting.RemotingCommand) (nextBeginOffset int64, err error) { + +//func (p *PullMessageController) updateOffsetIfNeed(msgs []message.MessageExtImpl, pullRequest *model.PullRequest, defaultMQPullConsumer *DefaultMQPushConsumer, nextBeginOffset int64) { +// if len(msgs) == 0 { +// if pullRequest.ProcessQueue.GetMsgCount() == 0 { +// defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) +// } +// } +//} +func parseNextBeginOffset(responseCommand *remoting.RemotingCommand) (nextBeginOffset int64) { + var err error pullResult := responseCommand.ExtFields if nextBeginOffsetInter, ok := pullResult["nextBeginOffset"]; ok { if nextBeginOffsetStr, ok := nextBeginOffsetInter.(string); ok { nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) if err != nil { + panic(err) glog.Error(err) return } From d55f2b44dbd87f9be6a04d0b8afcf8d68881e439 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 22 Aug 2017 22:52:56 +0800 Subject: [PATCH 61/88] go_vet pull_message_controller.go --- rocketmq-go/kernel/pull_message_controller.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/rocketmq-go/kernel/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go index c0ea2436c..cc088d35e 100644 --- a/rocketmq-go/kernel/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -191,8 +191,6 @@ func parseNextBeginOffset(responseCommand *remoting.RemotingCommand) (nextBeginO nextBeginOffset, err = strconv.ParseInt(nextBeginOffsetStr, 10, 64) if err != nil { panic(err) - glog.Error(err) - return } } } From ce6602c8f81272731c88288b04c02bfdc2ed4cb4 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 23 Aug 2017 21:42:11 +0800 Subject: [PATCH 62/88] exported type Message should be of the form Message --- rocketmq-go/api/model/message.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 5e72d9cd2..12b060c17 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -2,14 +2,14 @@ package rocketmqm import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -// rocketmq message +// Message rocketmq message // topic: the topic of this message // tag: the topic of this message, one topic may have no tag or different tag // key: key makes this message easy to search by console (https://github.com/apache/incubator-rocketmq-externals/rocketmq-console) // body: the message's user content // see MessageImpl type Message interface { - // + //Topic get topic Topic() (topic string) //set message topic SetTopic(tag string) @@ -27,7 +27,7 @@ type Message interface { SetBody([]byte) } -/*create a message instance*/ +// NewMessage create a message instance func NewMessage() (msg Message) { msg = message.NewMessageImpl() return From a0abe7ad547071282e3860a0806daf3e1f33a7cc Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 23 Aug 2017 21:50:04 +0800 Subject: [PATCH 63/88] comment on api --- rocketmq-go/api/model/message_ext.go | 2 +- rocketmq-go/api/model/message_listener.go | 9 +++++---- rocketmq-go/api/model/mq_client_config.go | 8 +++++--- rocketmq-go/api/model/mq_consumer_config.go | 14 ++++++++------ rocketmq-go/api/model/mq_producer_config.go | 2 ++ rocketmq-go/api/rocketmq_clent_instance.go | 9 +++++++++ rocketmq-go/api/rocketmq_consumer.go | 5 +++-- rocketmq-go/api/rocketmq_producer.go | 10 +++------- 8 files changed, 36 insertions(+), 23 deletions(-) diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 36c2ac78b..1862444a1 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,6 +1,6 @@ package rocketmqm -// see MessageExtImpl +//MessageExt see MessageExtImpl type MessageExt interface { //get message topic Topic() (tag string) diff --git a/rocketmq-go/api/model/message_listener.go b/rocketmq-go/api/model/message_listener.go index c0436926c..18d5cf21d 100644 --- a/rocketmq-go/api/model/message_listener.go +++ b/rocketmq-go/api/model/message_listener.go @@ -17,16 +17,17 @@ limitations under the License. package rocketmqm +// ConsumeStatus message consume status type ConsumeStatus int const ( - //consume success + //CONSUME_SUCCESS consume success CONSUME_SUCCESS ConsumeStatus = iota - //consume fail, consume again later + //RECONSUME_LATER consume fail, consume again later RECONSUME_LATER ) -// message consume result +// ConsumeConcurrentlyResult message consume result type ConsumeConcurrentlyResult struct { // consume status ConsumeConcurrentlyStatus ConsumeStatus @@ -34,5 +35,5 @@ type ConsumeConcurrentlyResult struct { AckIndex int } -//mqConsumer.RegisterMessageListener +// MessageListener mqConsumer.RegisterMessageListener type MessageListener func(msgs []MessageExt) ConsumeConcurrentlyResult diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 93a910960..254bef00d 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -18,16 +18,17 @@ limitations under the License. package rocketmqm //-------SerializeType------- -// default serialize type is JSON_SERIALIZE, but ROCKETMQ_SERIALIZE(need version >= ?) is faster +//SerializeType default serialize type is JSON_SERIALIZE, but ROCKETMQ_SERIALIZE(need version >= ?) is faster type SerializeType byte const ( - //see json_serializable.go + //JSON_SERIALIZE see json_serializable.go JSON_SERIALIZE SerializeType = iota - //see rocketmq_serializable.go + //ROCKETMQ_SERIALIZE see rocketmq_serializable.go ROCKETMQ_SERIALIZE ) +//MqClientConfig MqClientConfig type MqClientConfig struct { // NameServerAddress split by ; NameServerAddress string @@ -35,6 +36,7 @@ type MqClientConfig struct { ClientSerializeType SerializeType } +//NewMqClientConfig create a MqClientConfig instance func NewMqClientConfig(nameServerAddress string) (mqClientConfig *MqClientConfig) { mqClientConfig = &MqClientConfig{ NameServerAddress: nameServerAddress, diff --git a/rocketmq-go/api/model/mq_consumer_config.go b/rocketmq-go/api/model/mq_consumer_config.go index fc88b12ee..fdb386c01 100644 --- a/rocketmq-go/api/model/mq_consumer_config.go +++ b/rocketmq-go/api/model/mq_consumer_config.go @@ -19,27 +19,28 @@ package rocketmqm import "time" -//Delay some time when exception occur +//PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION Delay some time when exception occur const PULL_TIME_DELAY_MILLS_WHEN_EXCEPTION int64 = 3000 -//Flow control interval +//PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL Flow control interval const PULL_TIME_DELAY_MILLS_WHEN_FLOW_CONTROL int64 = 50 -//consume from where +//ConsumeFromWhere consume from where type ConsumeFromWhere int //first consume from the last offset const ( - //first consume from the last offset + //CONSUME_FROM_LAST_OFFSET first consume from the last offset CONSUME_FROM_LAST_OFFSET ConsumeFromWhere = iota - //first consume from the first offset + //CONSUME_FROM_FIRST_OFFSET first consume from the first offset CONSUME_FROM_FIRST_OFFSET - //first consume from the time + //CONSUME_FROM_TIMESTAMP first consume from the time CONSUME_FROM_TIMESTAMP ) +//MqConsumerConfig MqConsumerConfig type MqConsumerConfig struct { //consume from where @@ -96,6 +97,7 @@ type MqConsumerConfig struct { ConsumeTimestamp time.Time //when use CONSUME_FROM_TIMESTAMP } +//NewRocketMqConsumerConfig create a MqConsumerConfig instance func NewRocketMqConsumerConfig() (consumerConfig *MqConsumerConfig) { consumerConfig = &MqConsumerConfig{ ConsumeFromWhere: CONSUME_FROM_LAST_OFFSET, diff --git a/rocketmq-go/api/model/mq_producer_config.go b/rocketmq-go/api/model/mq_producer_config.go index b1b9e3ad1..1e8067e65 100644 --- a/rocketmq-go/api/model/mq_producer_config.go +++ b/rocketmq-go/api/model/mq_producer_config.go @@ -17,6 +17,7 @@ limitations under the License. package rocketmqm +//MqProducerConfig MqProducerConfig type MqProducerConfig struct { // SendMsgTimeout for this producer SendMsgTimeout int64 @@ -33,6 +34,7 @@ type MqProducerConfig struct { NotAvailableDuration []int64 } +//NewProducerConfig create a MqProducerConfig instance func NewProducerConfig() (producerConfig *MqProducerConfig) { producerConfig = &MqProducerConfig{ SendMsgTimeout: 3000, diff --git a/rocketmq-go/api/rocketmq_clent_instance.go b/rocketmq-go/api/rocketmq_clent_instance.go index 883882deb..f3eecbda2 100644 --- a/rocketmq-go/api/rocketmq_clent_instance.go +++ b/rocketmq-go/api/rocketmq_clent_instance.go @@ -22,6 +22,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" ) +//MQClientInstance MQClientInstance type MQClientInstance interface { //Register rocketmq producer to this client instance RegisterProducer(producer MQProducer) @@ -31,26 +32,34 @@ type MQClientInstance interface { Start() } +//ClientInstanceImpl MQClientInstance's implement type ClientInstanceImpl struct { rocketMqManager *kernel.MqClientManager } +//InitRocketMQClientInstance create a MQClientInstance instance func InitRocketMQClientInstance(nameServerAddress string) (rocketMQClientInstance MQClientInstance) { mqClientConfig := rocketmqm.NewMqClientConfig(nameServerAddress) return InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig) } + +//InitRocketMQClientInstanceWithCustomClientConfig create a MQClientInstance instance with custom client config func InitRocketMQClientInstanceWithCustomClientConfig(mqClientConfig *rocketmqm.MqClientConfig) (rocketMQClientInstance MQClientInstance) { rocketMQClientInstance = &ClientInstanceImpl{rocketMqManager: kernel.MqClientManagerInit(mqClientConfig)} return } +//RegisterProducer register producer to this client instance func (r *ClientInstanceImpl) RegisterProducer(producer MQProducer) { r.rocketMqManager.RegisterProducer(producer.(*kernel.DefaultMQProducer)) } +//RegisterConsumer register consumer to this client instance func (r *ClientInstanceImpl) RegisterConsumer(consumer MQConsumer) { r.rocketMqManager.RegisterConsumer(consumer.(*kernel.DefaultMQPushConsumer)) } + +//Start start this client instance. (register should before start) func (r *ClientInstanceImpl) Start() { r.rocketMqManager.Start() } diff --git a/rocketmq-go/api/rocketmq_consumer.go b/rocketmq-go/api/rocketmq_consumer.go index e11246623..ef59b4693 100644 --- a/rocketmq-go/api/rocketmq_consumer.go +++ b/rocketmq-go/api/rocketmq_consumer.go @@ -22,6 +22,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" ) +//MQConsumer rocketmq consumer type MQConsumer interface { // register custom's message listener to this consumer RegisterMessageListener(listener rocketmqm.MessageListener) @@ -36,12 +37,12 @@ type MQConsumer interface { Subscribe(topic string, subExpression string) } -// Concurrently(no order) CLUSTERING mq consumer with default config +//NewDefaultMQPushConsumer Concurrently(no order) CLUSTERING mq consumer with default config func NewDefaultMQPushConsumer(producerGroup string) (r MQConsumer) { return NewDefaultMQPushConsumerWithCustomConfig(producerGroup, rocketmqm.NewRocketMqConsumerConfig()) } -// Concurrently(no order) CLUSTERING mq consumer with custom config +//NewDefaultMQPushConsumerWithCustomConfig Concurrently(no order) CLUSTERING mq consumer with custom config func NewDefaultMQPushConsumerWithCustomConfig(producerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (r MQConsumer) { return kernel.NewDefaultMQPushConsumer(producerGroup, consumerConfig) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index fde8e9879..86e082932 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -25,11 +25,7 @@ import ( type mqProducerType int -const ( - //DEFAULT_MQ_PRODUCER - default_mq_producer mqProducerType = iota -) - +//MQProducer rocketmq producer type MQProducer interface { //send message,default timeout is 3000 Send(message rocketmqm.Message) (sendResult *model.SendResult, err error) @@ -37,12 +33,12 @@ type MQProducer interface { SendWithTimeout(message rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) } -//mq producer with default config +//NewDefaultMQProducer mq producer with default config func NewDefaultMQProducer(producerGroup string) (r MQProducer) { return NewDefaultMQProducerWithCustomConfig(producerGroup, rocketmqm.NewProducerConfig()) } -//mq producer with custom config +//NewDefaultMQProducerWithCustomConfig mq producer with custom config func NewDefaultMQProducerWithCustomConfig(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (r MQProducer) { return kernel.NewDefaultMQProducer(producerGroup, producerConfig) } From dd2ecfcc779d9ce9ba8991fff3acaf5162b380de Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 23 Aug 2017 22:01:55 +0800 Subject: [PATCH 64/88] allocate and header --- rocketmq-go/api/model/mq_client_config.go | 1 - rocketmq-go/api/rocketmq_producer.go | 2 -- .../allocate_message_averagely.go | 4 +++- .../allocate_message_averagely_by_circle.go | 4 +++- .../allocate_message_by_config.go | 4 +++- .../allocate_message_by_machine_room.go | 4 +++- .../allocate_message_queue_strategy.go | 4 +++- .../consume_message_directly_result_request_header.go | 3 ++- .../kernel/header/consumer_send_msg_back_request_header.go | 2 ++ rocketmq-go/kernel/header/get_consumer_list_by_group.go | 4 ++++ .../header/get_consumer_running_info_request_header.go | 2 ++ rocketmq-go/kernel/header/get_max_offset_request_header.go | 2 ++ rocketmq-go/kernel/header/get_max_offset_response_header.go | 2 ++ rocketmq-go/kernel/header/get_route_info_request_header.go | 2 ++ rocketmq-go/kernel/header/pull_message_request_header.go | 2 ++ .../kernel/header/query_consumer_offset_request_header.go | 2 ++ rocketmq-go/kernel/header/reset_offset_request_header.go | 2 ++ rocketmq-go/kernel/header/search_offset_request_header.go | 2 ++ rocketmq-go/kernel/header/send_message_request_header.go | 2 +- rocketmq-go/kernel/header/send_message_response_header.go | 2 ++ .../kernel/header/update_consumer_offset_request_header.go | 4 ++-- rocketmq-go/kernel/rebalance.go | 6 +++--- rocketmq-go/remoting/custom_header.go | 1 + 23 files changed, 48 insertions(+), 15 deletions(-) rename rocketmq-go/kernel/{allocate_message => allocate}/allocate_message_averagely.go (95%) rename rocketmq-go/kernel/{allocate_message => allocate}/allocate_message_averagely_by_circle.go (94%) rename rocketmq-go/kernel/{allocate_message => allocate}/allocate_message_by_config.go (91%) rename rocketmq-go/kernel/{allocate_message => allocate}/allocate_message_by_machine_room.go (95%) rename rocketmq-go/kernel/{allocate_message => allocate}/allocate_message_queue_strategy.go (90%) diff --git a/rocketmq-go/api/model/mq_client_config.go b/rocketmq-go/api/model/mq_client_config.go index 254bef00d..9af574691 100644 --- a/rocketmq-go/api/model/mq_client_config.go +++ b/rocketmq-go/api/model/mq_client_config.go @@ -17,7 +17,6 @@ limitations under the License. package rocketmqm -//-------SerializeType------- //SerializeType default serialize type is JSON_SERIALIZE, but ROCKETMQ_SERIALIZE(need version >= ?) is faster type SerializeType byte diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 86e082932..47de28b30 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -23,8 +23,6 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) -type mqProducerType int - //MQProducer rocketmq producer type MQProducer interface { //send message,default timeout is 3000 diff --git a/rocketmq-go/kernel/allocate_message/allocate_message_averagely.go b/rocketmq-go/kernel/allocate/allocate_message_averagely.go similarity index 95% rename from rocketmq-go/kernel/allocate_message/allocate_message_averagely.go rename to rocketmq-go/kernel/allocate/allocate_message_averagely.go index 572666701..c476c27a7 100644 --- a/rocketmq-go/kernel/allocate_message/allocate_message_averagely.go +++ b/rocketmq-go/kernel/allocate/allocate_message_averagely.go @@ -15,15 +15,17 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service_allocate_message +package allocate import ( "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) +//AllocateMessageQueueAveragely AllocateMessageQueueAveragely type AllocateMessageQueueAveragely struct{} +//Allocate message queue func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { diff --git a/rocketmq-go/kernel/allocate_message/allocate_message_averagely_by_circle.go b/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go similarity index 94% rename from rocketmq-go/kernel/allocate_message/allocate_message_averagely_by_circle.go rename to rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go index 0dacb7a8e..7626b0d9c 100644 --- a/rocketmq-go/kernel/allocate_message/allocate_message_averagely_by_circle.go +++ b/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go @@ -15,15 +15,17 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service_allocate_message +package allocate import ( "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) +//AllocateMessageQueueAveragelyByCircle AllocateMessageQueueAveragelyByCircle type AllocateMessageQueueAveragelyByCircle struct{} +//Allocate message queue func (a *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") diff --git a/rocketmq-go/kernel/allocate_message/allocate_message_by_config.go b/rocketmq-go/kernel/allocate/allocate_message_by_config.go similarity index 91% rename from rocketmq-go/kernel/allocate_message/allocate_message_by_config.go rename to rocketmq-go/kernel/allocate/allocate_message_by_config.go index 934a4fb9a..6bebbd598 100644 --- a/rocketmq-go/kernel/allocate_message/allocate_message_by_config.go +++ b/rocketmq-go/kernel/allocate/allocate_message_by_config.go @@ -15,14 +15,16 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service_allocate_message +package allocate import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +//AllocateMessageQueueByConfig AllocateMessageQueueByConfig type AllocateMessageQueueByConfig struct { messageQueueList []model.MessageQueue } +//Allocate message queue func (a *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { return a.messageQueueList, nil } diff --git a/rocketmq-go/kernel/allocate_message/allocate_message_by_machine_room.go b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go similarity index 95% rename from rocketmq-go/kernel/allocate_message/allocate_message_by_machine_room.go rename to rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go index b470ba6d9..2af120106 100644 --- a/rocketmq-go/kernel/allocate_message/allocate_message_by_machine_room.go +++ b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go @@ -15,16 +15,18 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service_allocate_message +package allocate import ( "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) +//AllocateMessageQueueByMachineRoom AllocateMessageQueueByMachineRoom type AllocateMessageQueueByMachineRoom struct { } +//Allocate message queue func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") diff --git a/rocketmq-go/kernel/allocate_message/allocate_message_queue_strategy.go b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go similarity index 90% rename from rocketmq-go/kernel/allocate_message/allocate_message_queue_strategy.go rename to rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go index 9c9444440..4dabb5af4 100644 --- a/rocketmq-go/kernel/allocate_message/allocate_message_queue_strategy.go +++ b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go @@ -15,14 +15,16 @@ See the License for the specific language governing permissions and limitations under the License. */ -package service_allocate_message +package allocate import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +//Allocate message queue type AllocateMessageQueueStrategy interface { Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) } +//GetAllocateMessageQueueStrategyByConfig get allocate message queue strategy by config func GetAllocateMessageQueueStrategyByConfig(allocateMessageQueueStrategy string) AllocateMessageQueueStrategy { return new(AllocateMessageQueueAveragely) } diff --git a/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go b/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go index 834d49e24..989d826cd 100644 --- a/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go +++ b/rocketmq-go/kernel/header/consume_message_directly_result_request_header.go @@ -17,7 +17,7 @@ limitations under the License. package header -//ConsumeMessageDirectlyResultRequestHeader +//ConsumeMessageDirectlyResultRequestHeader of CustomerHeader type ConsumeMessageDirectlyResultRequestHeader struct { //ConsumerGroup ConsumerGroup string `json:"consumerGroup"` @@ -29,6 +29,7 @@ type ConsumeMessageDirectlyResultRequestHeader struct { BrokerName string `json:"brokerName"` } +//FromMap convert map[string]interface to struct func (c *ConsumeMessageDirectlyResultRequestHeader) FromMap(headerMap map[string]interface{}) { c.ConsumerGroup = headerMap["consumerGroup"].(string) c.ClientId = headerMap["clientId"].(string) diff --git a/rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go b/rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go index 5ac822278..07c57c97b 100644 --- a/rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go +++ b/rocketmq-go/kernel/header/consumer_send_msg_back_request_header.go @@ -17,6 +17,7 @@ limitations under the License. package header +//ConsumerSendMsgBackRequestHeader of CustomerHeader type ConsumerSendMsgBackRequestHeader struct { Offset int64 Group string @@ -27,6 +28,7 @@ type ConsumerSendMsgBackRequestHeader struct { MaxReconsumeTimes int32 } +//FromMap convert map[string]interface to struct func (c *ConsumerSendMsgBackRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/get_consumer_list_by_group.go b/rocketmq-go/kernel/header/get_consumer_list_by_group.go index 69aa394ed..ef72edd2a 100644 --- a/rocketmq-go/kernel/header/get_consumer_list_by_group.go +++ b/rocketmq-go/kernel/header/get_consumer_list_by_group.go @@ -17,18 +17,22 @@ limitations under the License. package header +//GetConsumerListByGroupRequestHeader of CustomerHeader type GetConsumerListByGroupRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` } +//FromMap convert map[string]interface to struct func (g *GetConsumerListByGroupRequestHeader) FromMap(headerMap map[string]interface{}) { return } +//GetConsumerListByGroupResponseBody of CustomerHeader type GetConsumerListByGroupResponseBody struct { ConsumerIdList []string } +//FromMap convert map[string]interface to struct func (g *GetConsumerListByGroupResponseBody) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/get_consumer_running_info_request_header.go b/rocketmq-go/kernel/header/get_consumer_running_info_request_header.go index e9e02a661..5e04e277d 100644 --- a/rocketmq-go/kernel/header/get_consumer_running_info_request_header.go +++ b/rocketmq-go/kernel/header/get_consumer_running_info_request_header.go @@ -17,12 +17,14 @@ limitations under the License. package header +//GetConsumerRunningInfoRequestHeader of CustomerHeader type GetConsumerRunningInfoRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` ClientId string `json:"clientId"` JstackEnable bool `json:"jstackEnable"` } +//FromMap convert map[string]interface to struct func (g *GetConsumerRunningInfoRequestHeader) FromMap(headerMap map[string]interface{}) { g.ConsumerGroup = headerMap["consumerGroup"].(string) g.ClientId = headerMap["clientId"].(string) diff --git a/rocketmq-go/kernel/header/get_max_offset_request_header.go b/rocketmq-go/kernel/header/get_max_offset_request_header.go index 2393fe35f..df09f7ab5 100644 --- a/rocketmq-go/kernel/header/get_max_offset_request_header.go +++ b/rocketmq-go/kernel/header/get_max_offset_request_header.go @@ -17,11 +17,13 @@ limitations under the License. package header +//GetMaxOffsetRequestHeader of CustomerHeader type GetMaxOffsetRequestHeader struct { Topic string `json:"topic"` QueueId int32 `json:"queueId"` } +//FromMap convert map[string]interface to struct func (g *GetMaxOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/get_max_offset_response_header.go b/rocketmq-go/kernel/header/get_max_offset_response_header.go index 5e695d840..d50a1a65b 100644 --- a/rocketmq-go/kernel/header/get_max_offset_response_header.go +++ b/rocketmq-go/kernel/header/get_max_offset_response_header.go @@ -19,10 +19,12 @@ package header import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" +//QueryOffsetResponseHeader of CustomerHeader type QueryOffsetResponseHeader struct { Offset int64 `json:"offset"` } +//FromMap convert map[string]interface to struct func (q *QueryOffsetResponseHeader) FromMap(headerMap map[string]interface{}) { q.Offset = util.StrToInt64WithDefaultValue(headerMap["offset"].(string), -1) return diff --git a/rocketmq-go/kernel/header/get_route_info_request_header.go b/rocketmq-go/kernel/header/get_route_info_request_header.go index 91eefc9ef..6a8391b8c 100644 --- a/rocketmq-go/kernel/header/get_route_info_request_header.go +++ b/rocketmq-go/kernel/header/get_route_info_request_header.go @@ -17,10 +17,12 @@ limitations under the License. package header +//GetRouteInfoRequestHeader of CustomerHeader type GetRouteInfoRequestHeader struct { Topic string `json:"topic"` } +//FromMap convert map[string]interface to struct func (g *GetRouteInfoRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/pull_message_request_header.go b/rocketmq-go/kernel/header/pull_message_request_header.go index 73ae576a6..a26b2db2d 100644 --- a/rocketmq-go/kernel/header/pull_message_request_header.go +++ b/rocketmq-go/kernel/header/pull_message_request_header.go @@ -17,6 +17,7 @@ limitations under the License. package header +//PullMessageRequestHeader of CustomerHeader type PullMessageRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` Topic string `json:"topic"` @@ -30,6 +31,7 @@ type PullMessageRequestHeader struct { SubVersion int64 `json:"subVersion"` } +//FromMap convert map[string]interface to struct func (p *PullMessageRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/query_consumer_offset_request_header.go b/rocketmq-go/kernel/header/query_consumer_offset_request_header.go index 86eed07d6..75c456f80 100644 --- a/rocketmq-go/kernel/header/query_consumer_offset_request_header.go +++ b/rocketmq-go/kernel/header/query_consumer_offset_request_header.go @@ -17,12 +17,14 @@ limitations under the License. package header +//QueryConsumerOffsetRequestHeader of CustomerHeader type QueryConsumerOffsetRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` Topic string `json:"topic"` QueueId int32 `json:"queueId"` } +//FromMap convert map[string]interface to struct func (q *QueryConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/reset_offset_request_header.go b/rocketmq-go/kernel/header/reset_offset_request_header.go index 934ef28e3..0cb234628 100644 --- a/rocketmq-go/kernel/header/reset_offset_request_header.go +++ b/rocketmq-go/kernel/header/reset_offset_request_header.go @@ -22,6 +22,7 @@ import ( "strconv" ) +//ResetOffsetRequestHeader of CustomerHeader type ResetOffsetRequestHeader struct { Topic string `json:"topic"` Group string `json:"group"` @@ -29,6 +30,7 @@ type ResetOffsetRequestHeader struct { IsForce bool `json:"isForce"` } +//FromMap convert map[string]interface to struct func (r *ResetOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { r.Group = headerMap["group"].(string) r.Topic = headerMap["topic"].(string) diff --git a/rocketmq-go/kernel/header/search_offset_request_header.go b/rocketmq-go/kernel/header/search_offset_request_header.go index a2bdc75ee..7170f0ab1 100644 --- a/rocketmq-go/kernel/header/search_offset_request_header.go +++ b/rocketmq-go/kernel/header/search_offset_request_header.go @@ -21,12 +21,14 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" ) +//SearchOffsetRequestHeader of CustomerHeader type SearchOffsetRequestHeader struct { Topic string `json:"topic"` QueueId int32 `json:"queueId"` Timestamp int64 `json:"timestamp"` } +//FromMap convert map[string]interface to struct func (s *SearchOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { s.Topic = headerMap["topic"].(string) s.Topic = headerMap["queueId"].(string) diff --git a/rocketmq-go/kernel/header/send_message_request_header.go b/rocketmq-go/kernel/header/send_message_request_header.go index ca47f099d..9109e1f76 100644 --- a/rocketmq-go/kernel/header/send_message_request_header.go +++ b/rocketmq-go/kernel/header/send_message_request_header.go @@ -17,7 +17,7 @@ limitations under the License. package header -//SendMessageRequestHeader <- CustomerHeader +//SendMessageRequestHeader of CustomerHeader type SendMessageRequestHeader struct { ProducerGroup string `json:"producerGroup"` Topic string `json:"topic"` diff --git a/rocketmq-go/kernel/header/send_message_response_header.go b/rocketmq-go/kernel/header/send_message_response_header.go index c2bc50d3c..a46820ac0 100644 --- a/rocketmq-go/kernel/header/send_message_response_header.go +++ b/rocketmq-go/kernel/header/send_message_response_header.go @@ -17,6 +17,7 @@ limitations under the License. package header +//SendMessageResponseHeader of CustomerHeader type SendMessageResponseHeader struct { MsgId string QueueId int32 @@ -25,6 +26,7 @@ type SendMessageResponseHeader struct { MsgRegion string } +//FromMap convert map[string]interface to struct func (header *SendMessageResponseHeader) FromMap(headerMap map[string]interface{}) { return } diff --git a/rocketmq-go/kernel/header/update_consumer_offset_request_header.go b/rocketmq-go/kernel/header/update_consumer_offset_request_header.go index 572861d43..b7b5d4440 100644 --- a/rocketmq-go/kernel/header/update_consumer_offset_request_header.go +++ b/rocketmq-go/kernel/header/update_consumer_offset_request_header.go @@ -19,7 +19,7 @@ package header import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" -//UpdateConsumerOffsetRequestHeader of CustomHeader +//UpdateConsumerOffsetRequestHeader of CustomerHeader type UpdateConsumerOffsetRequestHeader struct { ConsumerGroup string `json:"consumerGroup"` Topic string `json:"topic"` @@ -27,7 +27,7 @@ type UpdateConsumerOffsetRequestHeader struct { CommitOffset int64 `json:"commitOffset"` } -//FromMap to UpdateConsumerOffsetRequestHeader +//FromMap convert map[string]interface to struct func (u *UpdateConsumerOffsetRequestHeader) FromMap(headerMap map[string]interface{}) { u.ConsumerGroup = headerMap["consumerGroup"].(string) u.QueueId = util.StrToInt32WithDefaultValue(util.ReadString(headerMap["queueId"]), 0) diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 62082b529..42a1f5f90 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -21,7 +21,7 @@ import ( "encoding/json" "errors" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate_message" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/allocate" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" @@ -40,7 +40,7 @@ type rebalance struct { subscriptionInner map[string]*model.SubscriptionData subscriptionInnerLock sync.RWMutex mqClient RocketMqClient - allocateMessageQueueStrategy service_allocate_message.AllocateMessageQueueStrategy + allocateMessageQueueStrategy allocate.AllocateMessageQueueStrategy processQueueTable map[model.MessageQueue]*model.ProcessQueue // both subscribe topic and retry group processQueueTableLock sync.RWMutex mutex sync.Mutex @@ -123,7 +123,7 @@ func newRebalance(groupName string, subscription map[string]string, mqClient Roc mqClient: mqClient, offsetStore: offsetStore, subscriptionInner: subscriptionInner, - allocateMessageQueueStrategy: service_allocate_message.GetAllocateMessageQueueStrategyByConfig("default"), + allocateMessageQueueStrategy: allocate.GetAllocateMessageQueueStrategyByConfig("default"), messageModel: "CLUSTERING", processQueueTable: make(map[model.MessageQueue]*model.ProcessQueue), consumerConfig: consumerConfig, diff --git a/rocketmq-go/remoting/custom_header.go b/rocketmq-go/remoting/custom_header.go index 2d4fde0d7..8683aedce 100644 --- a/rocketmq-go/remoting/custom_header.go +++ b/rocketmq-go/remoting/custom_header.go @@ -17,6 +17,7 @@ limitations under the License. package remoting +//CustomerHeader see kernel.header type CustomerHeader interface { //convert map[string]interface to struct FromMap(headerMap map[string]interface{}) From 0d384227b8aeb0865518e8c6d28028ce60755658 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Wed, 23 Aug 2017 22:42:26 +0800 Subject: [PATCH 65/88] comment on model --- rocketmq-go/model/constant/config.go | 5 ++ rocketmq-go/model/constant/message_const.go | 4 +- .../model/constant/message_sys_flag.go | 13 ++--- rocketmq-go/model/constant/mix_all.go | 1 + rocketmq-go/model/constant/perm.go | 19 +++++-- rocketmq-go/model/constant/pull_sys_flag.go | 10 ++-- .../model/consume_message_directly_result.go | 15 ++++-- rocketmq-go/model/consumer_running_info.go | 2 + rocketmq-go/model/heart_beat.go | 5 ++ rocketmq-go/model/message/message_ext_impl.go | 6 +++ rocketmq-go/model/message/message_impl.go | 37 ++++++++++---- rocketmq-go/model/message_queue.go | 6 +++ rocketmq-go/model/process_queue.go | 15 ++++++ rocketmq-go/model/process_queue_info.go | 1 + rocketmq-go/model/pull_request.go | 1 + rocketmq-go/model/query_result.go | 49 ------------------- rocketmq-go/model/reset_offset_body.go | 2 + rocketmq-go/model/send_result.go | 26 +++------- rocketmq-go/model/subscription_data.go | 1 + rocketmq-go/model/topic_publish_info.go | 7 +++ rocketmq-go/model/topic_route_data.go | 5 ++ rocketmq-go/remoting/communication_mode.go | 7 +-- rocketmq-go/remoting/custom_header.go | 2 +- rocketmq-go/remoting/json_serializable.go | 1 + rocketmq-go/remoting/remoting_client.go | 47 +++++++++++------- rocketmq-go/remoting/remoting_command.go | 9 ++-- rocketmq-go/remoting/request_processor.go | 2 + rocketmq-go/remoting/response_future.go | 3 ++ rocketmq-go/remoting/rocketmq_serializable.go | 1 + rocketmq-go/remoting/rpchook.go | 3 ++ rocketmq-go/remoting/serializable.go | 5 ++ rocketmq-go/util/compress_util.go | 1 + 32 files changed, 187 insertions(+), 124 deletions(-) delete mode 100644 rocketmq-go/model/query_result.go diff --git a/rocketmq-go/model/constant/config.go b/rocketmq-go/model/constant/config.go index 1aba15407..4ba2285a0 100644 --- a/rocketmq-go/model/constant/config.go +++ b/rocketmq-go/model/constant/config.go @@ -17,6 +17,11 @@ limitations under the License. package constant +//REMOTING_COMMAND_FLAG 0, REQUEST_COMMAND var REMOTING_COMMAND_FLAG = 0 + +//REMOTING_COMMAND_LANGUAGE org.apache.rocketmq.remoting.protocol.LanguageCode var REMOTING_COMMAND_LANGUAGE = "OTHER" + +//REMOTING_COMMAND_VERSION org.apache.rocketmq.common.MQVersion.Version.ordinal() var REMOTING_COMMAND_VERSION int16 = 213 diff --git a/rocketmq-go/model/constant/message_const.go b/rocketmq-go/model/constant/message_const.go index 14ec59651..4a67bcc1a 100644 --- a/rocketmq-go/model/constant/message_const.go +++ b/rocketmq-go/model/constant/message_const.go @@ -18,7 +18,9 @@ limitations under the License. package constant const ( - PROPERTY_KEYS = "KEYS" + //PROPERTY_KEYS property keys in MessageImpl.properties + PROPERTY_KEYS = "KEYS" + //PROPERTY_TAGS message tags PROPERTY_TAGS = "TAGS" PROPERTY_WAIT_STORE_MSG_OK = "WAIT" PROPERTY_DELAY_TIME_LEVEL = "DELAY" diff --git a/rocketmq-go/model/constant/message_sys_flag.go b/rocketmq-go/model/constant/message_sys_flag.go index f9bd5bff8..ac6297b99 100644 --- a/rocketmq-go/model/constant/message_sys_flag.go +++ b/rocketmq-go/model/constant/message_sys_flag.go @@ -18,10 +18,11 @@ limitations under the License. package constant const ( - CompressedFlag int32 = (0x1 << 0) - MultiTagsFlag int32 = (0x1 << 1) - TransactionNotType int32 = (0x0 << 2) - TransactionPreparedType int32 = (0x1 << 2) - TransactionCommitType int32 = (0x2 << 2) - TransactionRollbackType int32 = (0x3 << 2) + //CompressedFlag message compressed flag + CompressedFlag int32 = (0x1 << 0) + //MultiTagsFlag int32 = (0x1 << 1) + //TransactionNotType int32 = (0x0 << 2) + //TransactionPreparedType int32 = (0x1 << 2) + //TransactionCommitType int32 = (0x2 << 2) + //TransactionRollbackType int32 = (0x3 << 2) ) diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index d0f9c3fad..97829e1f7 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -18,6 +18,7 @@ limitations under the License. package constant const ( + //ROCKETMQ_HOME_ENV rocketmq home env ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME" ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir" NAMESRV_ADDR_ENV = "NAMESRV_ADDR" diff --git a/rocketmq-go/model/constant/perm.go b/rocketmq-go/model/constant/perm.go index 6bcc034ae..aa7835a3e 100644 --- a/rocketmq-go/model/constant/perm.go +++ b/rocketmq-go/model/constant/perm.go @@ -18,17 +18,26 @@ limitations under the License. package constant const ( + //PERM_PRIORITY PERM_PRIORITY PERM_PRIORITY = 0x1 << 3 - PERM_READ = 0x1 << 2 - PERM_WRITE = 0x1 << 1 - PERM_INHERIT = 0x1 << 0 + + //PERM_READ this queue can read + PERM_READ = 0x1 << 2 + //PERM_WRITE this queue can write + PERM_WRITE = 0x1 << 1 + + //PERM_INHERIT PERM_INHERIT + PERM_INHERIT = 0x1 << 0 ) +//WriteAble this queue can write func WriteAble(perm int32) (ret bool) { - ret = ((perm & PERM_WRITE) == PERM_WRITE) + ret = (perm & PERM_WRITE) == PERM_WRITE return } + +//ReadAble this queue can read func ReadAble(perm int32) (ret bool) { - ret = ((perm & PERM_READ) == PERM_READ) + ret = (perm & PERM_READ) == PERM_READ return } diff --git a/rocketmq-go/model/constant/pull_sys_flag.go b/rocketmq-go/model/constant/pull_sys_flag.go index f06b39fbc..d6f1e3f13 100644 --- a/rocketmq-go/model/constant/pull_sys_flag.go +++ b/rocketmq-go/model/constant/pull_sys_flag.go @@ -18,8 +18,12 @@ limitations under the License. package constant const ( + //FLAG_COMMIT_OFFSET FLAG_COMMIT_OFFSET FLAG_COMMIT_OFFSET int32 = 0x1 << 0 - FLAG_SUSPEND int32 = 0x1 << 1 - FLAG_SUBSCRIPTION int32 = 0x1 << 2 - FLAG_CLASS_FILTER int32 = 0x1 << 3 + //FLAG_SUSPEND FLAG_SUSPEND + FLAG_SUSPEND int32 = 0x1 << 1 + //FLAG_SUBSCRIPTION FLAG_SUBSCRIPTION + FLAG_SUBSCRIPTION int32 = 0x1 << 2 + //FLAG_CLASS_FILTER FLAG_CLASS_FILTER + FLAG_CLASS_FILTER int32 = 0x1 << 3 ) diff --git a/rocketmq-go/model/consume_message_directly_result.go b/rocketmq-go/model/consume_message_directly_result.go index 92cf1a8a2..52500b858 100644 --- a/rocketmq-go/model/consume_message_directly_result.go +++ b/rocketmq-go/model/consume_message_directly_result.go @@ -17,17 +17,22 @@ limitations under the License. package model +//ConsumeResultType message consume result type type ConsumeResultType string const ( - CR_SUCCESS ConsumeResultType = "CR_SUCCESS" - CR_LATER ConsumeResultType = "CR_LATER" - CR_ROLLBACK ConsumeResultType = "CR_ROLLBACK" - CR_COMMIT ConsumeResultType = "CR_COMMIT" + //CR_SUCCESS consume success + CR_SUCCESS ConsumeResultType = "CR_SUCCESS" + + //CR_THROW_EXCEPTION consume error CR_THROW_EXCEPTION ConsumeResultType = "CR_THROW_EXCEPTION" - CR_RETURN_NULL ConsumeResultType = "CR_RETURN_NULL" + //CR_RETURN_NULL ConsumeResultType = "CR_RETURN_NULL" + //CR_LATER ConsumeResultType = "CR_LATER" + //CR_ROLLBACK ConsumeResultType = "CR_ROLLBACK" + //CR_COMMIT ConsumeResultType = "CR_COMMIT" ) +//ConsumeMessageDirectlyResult consume message directly's result type ConsumeMessageDirectlyResult struct { Order bool `json:"order"` AutoCommit bool `json:"autoCommit"` diff --git a/rocketmq-go/model/consumer_running_info.go b/rocketmq-go/model/consumer_running_info.go index e327275c6..a36c6aaa7 100644 --- a/rocketmq-go/model/consumer_running_info.go +++ b/rocketmq-go/model/consumer_running_info.go @@ -19,11 +19,13 @@ package model import "encoding/json" +//ConsumerRunningInfo this client's consumer running info type ConsumerRunningInfo struct { Properties map[string]string `json:"properties"` MqTable map[MessageQueue]ProcessQueueInfo `json:"mqTable"` } +//Encode ConsumerRunningInfo to byte array func (c *ConsumerRunningInfo) Encode() (jsonByte []byte, err error) { mqTableJsonStr := "{" first := true diff --git a/rocketmq-go/model/heart_beat.go b/rocketmq-go/model/heart_beat.go index 87b49eca8..17e3e277f 100644 --- a/rocketmq-go/model/heart_beat.go +++ b/rocketmq-go/model/heart_beat.go @@ -19,6 +19,7 @@ package model import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +//ConsumerData this client's ConsumerData type ConsumerData struct { GroupName string ConsumeType string @@ -27,9 +28,13 @@ type ConsumerData struct { SubscriptionDataSet []*SubscriptionData UnitMode bool } + +//ProducerData producer data type ProducerData struct { GroupName string } + +//HeartbeatData heartbeat data type HeartbeatData struct { ClientId string ConsumerDataSet []*ConsumerData diff --git a/rocketmq-go/model/message/message_ext_impl.go b/rocketmq-go/model/message/message_ext_impl.go index 05f5ce2b6..9b19e1a2c 100644 --- a/rocketmq-go/model/message/message_ext_impl.go +++ b/rocketmq-go/model/message/message_ext_impl.go @@ -23,6 +23,7 @@ import ( "math" ) +//MessageExtImpl the implement of MessageExt type MessageExtImpl struct { *MessageImpl msgId string @@ -41,16 +42,19 @@ type MessageExtImpl struct { propertyConsumeStartTimestamp string } +//MsgId get MessageId func (m *MessageExtImpl) MsgId() (msgId string) { msgId = m.msgId return } +//SetMsgId SetMsgId func (m *MessageExtImpl) SetMsgId(msgId string) { m.msgId = msgId return } +//GetOriginMessageId GetOriginMessageId func (m *MessageExtImpl) GetOriginMessageId() string { if m.properties != nil { originMessageId := m.properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] @@ -61,6 +65,7 @@ func (m *MessageExtImpl) GetOriginMessageId() string { return m.msgId } +//GetConsumeStartTime GetConsumeStartTime func (m *MessageExtImpl) GetConsumeStartTime() int64 { if len(m.propertyConsumeStartTimestamp) > 0 { return util.StrToInt64WithDefaultValue(m.propertyConsumeStartTimestamp, -1) @@ -68,6 +73,7 @@ func (m *MessageExtImpl) GetConsumeStartTime() int64 { return math.MaxInt64 } +//SetConsumeStartTime SetConsumeStartTime func (m *MessageExtImpl) SetConsumeStartTime() { if m.properties == nil { m.properties = make(map[string]string) diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go index d007b635c..aeefc0404 100644 --- a/rocketmq-go/model/message/message_impl.go +++ b/rocketmq-go/model/message/message_impl.go @@ -23,6 +23,7 @@ import ( "strings" ) +//MessageImpl the implement of Message type MessageImpl struct { topic string flag int @@ -30,52 +31,68 @@ type MessageImpl struct { body []byte } +//NewMessageImpl create a Message instance func NewMessageImpl() (message *MessageImpl) { message = &MessageImpl{} return } + +//Properties get message's Properties func (m *MessageImpl) Properties() (properties map[string]string) { properties = m.properties return } + +//SetProperties set message's Properties func (m *MessageImpl) SetProperties(properties map[string]string) { m.properties = properties return } + +//PropertiesKeyValue get properties[key]'s value func (m *MessageImpl) PropertiesKeyValue(key string) (value string) { value = m.properties[key] return } +//Body get message body func (m *MessageImpl) Body() (body []byte) { body = m.body return } + +//Topic get message Topic func (m *MessageImpl) Topic() (topic string) { topic = m.topic return } + +//SetFlag set message flag func (m *MessageImpl) SetFlag(flag int) { m.flag = flag return } + +//Flag get message flag func (m *MessageImpl) Flag() (flag int) { flag = m.flag return } +//SetTopic set topic func (m *MessageImpl) SetTopic(topic string) { m.topic = topic } +//SetBody set body func (m *MessageImpl) SetBody(body []byte) { m.body = body } -//set message tag +//SetTag set message tag func (m *MessageImpl) SetTag(tag string) { if m.properties == nil { m.properties = make(map[string]string) @@ -83,7 +100,7 @@ func (m *MessageImpl) SetTag(tag string) { m.properties[constant.PROPERTY_TAGS] = tag } -//get message tag from properties +//Tag get message tag from properties func (m *MessageImpl) Tag() (tag string) { if m.properties != nil { tag = m.properties[constant.PROPERTY_TAGS] @@ -91,7 +108,7 @@ func (m *MessageImpl) Tag() (tag string) { return } -//set message key +//SetKeys set message key func (m *MessageImpl) SetKeys(keys []string) { if m.properties == nil { m.properties = make(map[string]string) @@ -99,7 +116,7 @@ func (m *MessageImpl) SetKeys(keys []string) { m.properties[constant.PROPERTY_KEYS] = strings.Join(keys, " ") } -//SetDelayTimeLevel +//SetDelayTimeLevel set message delay time level func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { if m.properties == nil { m.properties = make(map[string]string) @@ -136,7 +153,7 @@ func (m *MessageExtImpl) GetMsgUniqueKey() string { return m.msgId } -//only use by system +//SetOriginMessageId only use by system func (m *MessageImpl) SetOriginMessageId(messageId string) { if m.properties == nil { m.properties = make(map[string]string) @@ -144,7 +161,7 @@ func (m *MessageImpl) SetOriginMessageId(messageId string) { m.properties[constant.PROPERTY_ORIGIN_MESSAGE_ID] = messageId } -//only use by system +//SetRetryTopic only use by system func (m *MessageImpl) SetRetryTopic(retryTopic string) { if m.properties == nil { m.properties = make(map[string]string) @@ -152,7 +169,7 @@ func (m *MessageImpl) SetRetryTopic(retryTopic string) { m.properties[constant.PROPERTY_RETRY_TOPIC] = retryTopic } -//only use by system +//SetReconsumeTime only use by system func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { if m.properties == nil { m.properties = make(map[string]string) @@ -160,7 +177,7 @@ func (m *MessageImpl) SetReconsumeTime(reConsumeTime int) { m.properties[constant.PROPERTY_RECONSUME_TIME] = util.IntToString(reConsumeTime) } -//only use by system +//GetReconsumeTimes only use by system func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { reConsumeTime = 0 if m.properties != nil { @@ -172,7 +189,7 @@ func (m *MessageImpl) GetReconsumeTimes() (reConsumeTime int) { return } -//only use by system +//SetMaxReconsumeTimes only use by system func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { if m.properties == nil { m.properties = make(map[string]string) @@ -180,7 +197,7 @@ func (m *MessageImpl) SetMaxReconsumeTimes(maxConsumeTime int) { m.properties[constant.PROPERTY_MAX_RECONSUME_TIMES] = util.IntToString(maxConsumeTime) } -//only use by system +//GetMaxReconsumeTimes only use by system func (m *MessageImpl) GetMaxReconsumeTimes() (maxConsumeTime int) { maxConsumeTime = 0 if m.properties != nil { diff --git a/rocketmq-go/model/message_queue.go b/rocketmq-go/model/message_queue.go index 5f6219663..4b03a4536 100644 --- a/rocketmq-go/model/message_queue.go +++ b/rocketmq-go/model/message_queue.go @@ -17,6 +17,7 @@ limitations under the License. package model +//MessageQueue message queue type MessageQueue struct { Topic string `json:"topic"` BrokerName string `json:"brokerName"` @@ -31,8 +32,10 @@ func (m *MessageQueue) clone() *MessageQueue { return no } +//MessageQueues queue array type MessageQueues []*MessageQueue +//Less compare queue func (m MessageQueues) Less(i, j int) bool { imq := m[i] jmq := m[j] @@ -56,14 +59,17 @@ func (m MessageQueues) Less(i, j int) bool { } } +//Swap swap queue func (m MessageQueues) Swap(i, j int) { m[i], m[j] = m[j], m[i] } +//Len messageQueues's length func (m MessageQueues) Len() int { return len(m) } +//Equals judge messageQueue is the same func (m MessageQueue) Equals(messageQueue *MessageQueue) bool { if m.QueueId != messageQueue.QueueId { return false diff --git a/rocketmq-go/model/process_queue.go b/rocketmq-go/model/process_queue.go index 676dcabf7..aaf5849b8 100644 --- a/rocketmq-go/model/process_queue.go +++ b/rocketmq-go/model/process_queue.go @@ -27,6 +27,7 @@ import ( "time" ) +//ProcessQueue message process queue type ProcessQueue struct { msgTreeMap *treemap.Map // int | MessageExtImpl msgCount int @@ -44,6 +45,7 @@ type ProcessQueue struct { msgTreeMapToBeConsume *treemap.Map } +//NewProcessQueue create a ProcessQueue func NewProcessQueue() (processQueue *ProcessQueue) { processQueue = new(ProcessQueue) processQueue.dropped = false @@ -52,12 +54,15 @@ func NewProcessQueue() (processQueue *ProcessQueue) { return } + +//GetMsgCount get message count func (p *ProcessQueue) GetMsgCount() int { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() return p.msgCount } +//Clear clear func (p *ProcessQueue) Clear() { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() @@ -68,6 +73,7 @@ func (p *ProcessQueue) Clear() { } +//ChangeToProcessQueueInfo changeToProcessQueueInfo func (p *ProcessQueue) ChangeToProcessQueueInfo() (processQueueInfo ProcessQueueInfo) { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() @@ -91,6 +97,7 @@ func (p *ProcessQueue) ChangeToProcessQueueInfo() (processQueueInfo ProcessQueue return } +//DeleteExpireMsg deleteExpireMsg func (p *ProcessQueue) DeleteExpireMsg(queueOffset int) { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() @@ -106,6 +113,7 @@ func (p *ProcessQueue) DeleteExpireMsg(queueOffset int) { } } +//GetMinMessageInTree getMinMessageInTree func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *message.MessageExtImpl) { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() @@ -120,12 +128,17 @@ func (p *ProcessQueue) GetMinMessageInTree() (offset int, messagePoint *message. return } +//SetDrop set this queue is dropped func (p *ProcessQueue) SetDrop(drop bool) { p.dropped = drop } + +//IsDropped judge whether this queue is dropped func (p *ProcessQueue) IsDropped() bool { return p.dropped } + +//GetMaxSpan getMaxSpan func (p *ProcessQueue) GetMaxSpan() int { defer p.lockTreeMap.Unlock() p.lockTreeMap.Lock() @@ -139,6 +152,7 @@ func (p *ProcessQueue) GetMaxSpan() int { return maxOffset - minOffset } +//RemoveMessage from this process queue func (p *ProcessQueue) RemoveMessage(msgs []message.MessageExtImpl) (offset int64) { now := time.Now() offset = -1 @@ -160,6 +174,7 @@ func (p *ProcessQueue) RemoveMessage(msgs []message.MessageExtImpl) (offset int6 return } +//PutMessage put message into this process queue func (p *ProcessQueue) PutMessage(msgs []message.MessageExtImpl) (dispatchToConsume bool) { dispatchToConsume = false msgsLen := len(msgs) diff --git a/rocketmq-go/model/process_queue_info.go b/rocketmq-go/model/process_queue_info.go index 6b49616fb..bf90134f9 100644 --- a/rocketmq-go/model/process_queue_info.go +++ b/rocketmq-go/model/process_queue_info.go @@ -17,6 +17,7 @@ limitations under the License. package model +//ProcessQueueInfo processQueueInfo type ProcessQueueInfo struct { CommitOffset int64 `json:"commitOffset"` diff --git a/rocketmq-go/model/pull_request.go b/rocketmq-go/model/pull_request.go index 77c857311..7638eb4ea 100644 --- a/rocketmq-go/model/pull_request.go +++ b/rocketmq-go/model/pull_request.go @@ -17,6 +17,7 @@ limitations under the License. package model +//PullRequest pullRequest type PullRequest struct { //consumer group ConsumerGroup string diff --git a/rocketmq-go/model/query_result.go b/rocketmq-go/model/query_result.go deleted file mode 100644 index c1a105ab2..000000000 --- a/rocketmq-go/model/query_result.go +++ /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. -*/ - -package model - -// -//import ( -// "fmt" -// "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" -//) -// -//type QueryResult struct { -// indexLastUpdateTimestamp int64 -// messageList []*message.MessageExtImpl -//} -// -//func NewQueryResult(timestamp int64, list []*message.MessageExtImpl) *QueryResult { -// return &QueryResult{ -// indexLastUpdateTimestamp: timestamp, -// messageList: list, -// } -//} -// -//func (qr *QueryResult) IndexLastUpdateTimestamp() int64 { -// return qr.indexLastUpdateTimestamp -//} -// -//func (qr *QueryResult) MessageList() []*message.MessageExtImpl { //TODO: address? -// return qr.messageList -//} -// -//func (qr *QueryResult) String() string { -// return fmt.Sprintf("QueryResult [indexLastUpdateTimestamp=%s, messageList=%s]", -// qr.indexLastUpdateTimestamp, qr.messageList) -//} diff --git a/rocketmq-go/model/reset_offset_body.go b/rocketmq-go/model/reset_offset_body.go index 274148f59..5b1fe6937 100644 --- a/rocketmq-go/model/reset_offset_body.go +++ b/rocketmq-go/model/reset_offset_body.go @@ -23,10 +23,12 @@ import ( "github.com/golang/glog" ) +//ResetOffsetBody resetOffsetBody type ResetOffsetBody struct { OffsetTable map[MessageQueue]int64 `json:"offsetTable"` } +//Decode decode byte array to ResetOffsetBody func (r *ResetOffsetBody) Decode(data []byte) (err error) { r.OffsetTable = map[MessageQueue]int64{} var kvMap map[string]string diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go index d08a0f702..a5c7d5865 100644 --- a/rocketmq-go/model/send_result.go +++ b/rocketmq-go/model/send_result.go @@ -17,15 +17,21 @@ limitations under the License. package model +//SendStatus message send result type SendStatus int const ( + //SendOK message send success SendOK SendStatus = iota + //FlushDiskTimeout FlushDiskTimeout FlushDiskTimeout + //FlushSlaveTimeout FlushSlaveTimeout FlushSlaveTimeout + //SlaveNotAvaliable SlaveNotAvaliable SlaveNotAvaliable ) +//SendResult SendResult type SendResult struct { sendStatus SendStatus msgID string @@ -37,24 +43,6 @@ type SendResult struct { traceOn bool } -func NewSendResult(status SendStatus, msgID, offsetID string, queue MessageQueue, queueOffset int64) *SendResult { - return &SendResult{ - sendStatus: status, - msgID: msgID, - offsetMsgID: offsetID, - messageQueue: queue, - queueOffset: queueOffset, - } -} - -func EncoderSendResultToJson(obj interface{}) string { - return "" // TODO -} - -func DecoderSendResultFromJson(json string) *SendResult { - return nil // TODO -} - func (result *SendResult) TraceOn() bool { return result.traceOn } @@ -107,10 +95,12 @@ func (result *SendResult) SetTransactionID(s string) { result.transactionID = s } +//OffsetMsgID OffsetMsgID func (result *SendResult) OffsetMsgID() string { return result.offsetMsgID } +//SetOffsetMsgID SetOffsetMsgID func (result *SendResult) SetOffsetMsgID(s string) { result.offsetMsgID = s } diff --git a/rocketmq-go/model/subscription_data.go b/rocketmq-go/model/subscription_data.go index 90a135a44..134559aec 100644 --- a/rocketmq-go/model/subscription_data.go +++ b/rocketmq-go/model/subscription_data.go @@ -17,6 +17,7 @@ limitations under the License. package model +//SubscriptionData message subscription data type SubscriptionData struct { Topic string SubString string diff --git a/rocketmq-go/model/topic_publish_info.go b/rocketmq-go/model/topic_publish_info.go index bccaf2a3d..cc21c41df 100644 --- a/rocketmq-go/model/topic_publish_info.go +++ b/rocketmq-go/model/topic_publish_info.go @@ -22,6 +22,7 @@ import ( "sync/atomic" ) +//TopicPublishInfo TopicPublishInfo type TopicPublishInfo struct { OrderTopic bool HaveTopicRouterInfo bool @@ -30,10 +31,13 @@ type TopicPublishInfo struct { topicQueueIndex int32 } +//JudgeTopicPublishInfoOk JudgeTopicPublishInfoOk func (t *TopicPublishInfo) JudgeTopicPublishInfoOk() (bIsTopicOk bool) { bIsTopicOk = (len(t.MessageQueueList) > 0) return } + +//FetchQueueIndex FetchQueueIndex func (t *TopicPublishInfo) FetchQueueIndex() (index int) { qLen := len(t.MessageQueueList) if qLen > 0 { @@ -43,6 +47,8 @@ func (t *TopicPublishInfo) FetchQueueIndex() (index int) { } return } + +//BuildTopicSubscribeInfoFromRoteData BuildTopicSubscribeInfoFromRoteData func BuildTopicSubscribeInfoFromRoteData(topic string, topicRouteData *TopicRouteData) (mqList []*MessageQueue) { mqList = make([]*MessageQueue, 0) for _, queueData := range topicRouteData.QueueDatas { @@ -62,6 +68,7 @@ func BuildTopicSubscribeInfoFromRoteData(topic string, topicRouteData *TopicRout return } +//BuildTopicPublishInfoFromTopicRoteData BuildTopicPublishInfoFromTopicRoteData func BuildTopicPublishInfoFromTopicRoteData(topic string, topicRouteData *TopicRouteData) (topicPublishInfo *TopicPublishInfo) { topicPublishInfo = &TopicPublishInfo{ TopicRouteDataInstance: topicRouteData, diff --git a/rocketmq-go/model/topic_route_data.go b/rocketmq-go/model/topic_route_data.go index 0241411a2..eabd50e30 100644 --- a/rocketmq-go/model/topic_route_data.go +++ b/rocketmq-go/model/topic_route_data.go @@ -21,11 +21,14 @@ import ( "sync" ) +//TopicRouteData TopicRouteData type TopicRouteData struct { OrderTopicConf string QueueDatas []*QueueData BrokerDatas []*BrokerData } + +//QueueData QueueData type QueueData struct { BrokerName string ReadQueueNums int32 @@ -33,6 +36,8 @@ type QueueData struct { Perm int32 TopicSynFlag int32 } + +//BrokerData BrokerData type BrokerData struct { BrokerName string BrokerAddrs map[string]string diff --git a/rocketmq-go/remoting/communication_mode.go b/rocketmq-go/remoting/communication_mode.go index fc9863e50..71afc6b14 100644 --- a/rocketmq-go/remoting/communication_mode.go +++ b/rocketmq-go/remoting/communication_mode.go @@ -17,13 +17,14 @@ limitations under the License. package remoting +//CommunicationMode Sync|Async|OneWay type CommunicationMode int const ( - //Sync invoke + //Sync Sync invoke Sync CommunicationMode = iota - //ASync invoke + //Async ASync invoke Async - //OneWay invoke + //OneWay OneWay invoke OneWay ) diff --git a/rocketmq-go/remoting/custom_header.go b/rocketmq-go/remoting/custom_header.go index 8683aedce..537875cf5 100644 --- a/rocketmq-go/remoting/custom_header.go +++ b/rocketmq-go/remoting/custom_header.go @@ -19,7 +19,7 @@ package remoting //CustomerHeader see kernel.header type CustomerHeader interface { - //convert map[string]interface to struct + //FromMap convert map[string]interface to struct FromMap(headerMap map[string]interface{}) //ToMap()(headerMap map[string]interface{}) } diff --git a/rocketmq-go/remoting/json_serializable.go b/rocketmq-go/remoting/json_serializable.go index d568108e2..f826c2284 100644 --- a/rocketmq-go/remoting/json_serializable.go +++ b/rocketmq-go/remoting/json_serializable.go @@ -21,6 +21,7 @@ import ( "encoding/json" ) +//JsonSerializer of Serializer type JsonSerializer struct { } diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 8de886aab..38af41db8 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -32,14 +32,17 @@ import ( "time" ) +//RemotingClient mq remoting client type RemotingClient interface { - //sync invoke remote + //InvokeSync sync invoke remote InvokeSync(addr string, request *RemotingCommand, timeoutMillis int64) (remotingCommand *RemotingCommand, err error) - //async invoke remote + //InvokeAsync async invoke remote InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error - //one way invoke remote + //InvokeOneWay one way invoke remote InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error } + +//DefaultRemotingClient of RemotingClient type DefaultRemotingClient struct { clientId string clientConfig *rocketmqm.MqClientConfig @@ -57,7 +60,7 @@ type DefaultRemotingClient struct { serializerHandler SerializerHandler //rocketmq encode decode } -// create a RemotingClient instance +//RemotingClientInit create a RemotingClient instance func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { client = &DefaultRemotingClient{} client.connTable = map[string]net.Conn{} @@ -71,9 +74,10 @@ func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestPro return } +//InvokeSync sync invoke remote func (drc *DefaultRemotingClient) InvokeSync(addr string, request *RemotingCommand, timeoutMillis int64) (remotingCommand *RemotingCommand, err error) { var conn net.Conn - conn, err = drc.GetOrCreateConn(addr) + conn, err = drc.getOrCreateConn(addr) response := &ResponseFuture{ SendRequestOK: false, Opaque: request.Opaque, @@ -83,7 +87,7 @@ func (drc *DefaultRemotingClient) InvokeSync(addr string, request *RemotingComma } header := drc.serializerHandler.encodeHeader(request) body := request.Body - drc.SetResponse(request.Opaque, response) + drc.setResponse(request.Opaque, response) err = drc.sendRequest(header, body, conn, addr) if err != nil { glog.Error(err) @@ -98,8 +102,10 @@ func (drc *DefaultRemotingClient) InvokeSync(addr string, request *RemotingComma return } } + +//InvokeAsync async invoke remote func (drc *DefaultRemotingClient) InvokeAsync(addr string, request *RemotingCommand, timeoutMillis int64, invokeCallback InvokeCallback) error { - conn, err := drc.GetOrCreateConn(addr) + conn, err := drc.getOrCreateConn(addr) if err != nil { return err } @@ -110,7 +116,7 @@ func (drc *DefaultRemotingClient) InvokeAsync(addr string, request *RemotingComm BeginTimestamp: time.Now().Unix(), InvokeCallback: invokeCallback, } - drc.SetResponse(request.Opaque, response) + drc.setResponse(request.Opaque, response) header := drc.serializerHandler.encodeHeader(request) body := request.Body err = drc.sendRequest(header, body, conn, addr) @@ -120,8 +126,10 @@ func (drc *DefaultRemotingClient) InvokeAsync(addr string, request *RemotingComm } return err } + +//InvokeOneWay one way invoke remote func (drc *DefaultRemotingClient) InvokeOneWay(addr string, request *RemotingCommand, timeoutMillis int64) error { - conn, err := drc.GetOrCreateConn(addr) + conn, err := drc.getOrCreateConn(addr) if err != nil { return err } @@ -151,11 +159,13 @@ func (drc *DefaultRemotingClient) sendRequest(header, body []byte, conn net.Conn } return nil } + +//GetNamesrvAddrList GetNamesrvAddrList func (drc *DefaultRemotingClient) GetNamesrvAddrList() []string { return drc.namesrvAddrList } -func (drc *DefaultRemotingClient) SetResponse(index int32, response *ResponseFuture) { +func (drc *DefaultRemotingClient) setResponse(index int32, response *ResponseFuture) { drc.responseTable.Set(strconv.Itoa(int(index)), response) } func (drc *DefaultRemotingClient) getResponse(index int32) (response *ResponseFuture, err error) { @@ -170,25 +180,25 @@ func (drc *DefaultRemotingClient) getResponse(index int32) (response *ResponseFu func (drc *DefaultRemotingClient) removeResponse(index int32) { drc.responseTable.Remove(strconv.Itoa(int(index))) } -func (drc *DefaultRemotingClient) GetOrCreateConn(address string) (conn net.Conn, err error) { +func (drc *DefaultRemotingClient) getOrCreateConn(address string) (conn net.Conn, err error) { if len(address) == 0 { conn, err = drc.getNamesvrConn() return } - conn = drc.GetConn(address) + conn = drc.getConn(address) if conn != nil { return } - conn, err = drc.CreateConn(address) + conn, err = drc.createConn(address) return } -func (drc *DefaultRemotingClient) GetConn(address string) (conn net.Conn) { +func (drc *DefaultRemotingClient) getConn(address string) (conn net.Conn) { drc.connTableLock.RLock() conn = drc.connTable[address] drc.connTableLock.RUnlock() return } -func (drc *DefaultRemotingClient) CreateConn(address string) (conn net.Conn, err error) { +func (drc *DefaultRemotingClient) createConn(address string) (conn net.Conn, err error) { defer drc.connTableLock.Unlock() drc.connTableLock.Lock() conn = drc.connTable[address] @@ -205,7 +215,7 @@ func (drc *DefaultRemotingClient) getNamesvrConn() (conn net.Conn, err error) { address := drc.namesrvAddrSelectedAddr drc.namesvrLockRW.RUnlock() if len(address) != 0 { - conn = drc.GetConn(address) + conn = drc.getConn(address) if conn != nil { return } @@ -216,7 +226,7 @@ func (drc *DefaultRemotingClient) getNamesvrConn() (conn net.Conn, err error) { //already connected by another write lock owner address = drc.namesrvAddrSelectedAddr if len(address) != 0 { - conn = drc.GetConn(address) + conn = drc.getConn(address) if conn != nil { return } @@ -232,7 +242,7 @@ func (drc *DefaultRemotingClient) getNamesvrConn() (conn net.Conn, err error) { if len(selectAddress) == 0 { continue } - conn, err = drc.CreateConn(selectAddress) + conn, err = drc.createConn(selectAddress) if err == nil { drc.namesrvAddrSelectedAddr = selectAddress drc.namesrvAddrSelectedIndex = selectedIndex @@ -358,6 +368,7 @@ func (drc *DefaultRemotingClient) handlerResponse(cmd *RemotingCommand) { } } +//ClearExpireResponse clear expire response which is not consumed after 30 seconds func (drc *DefaultRemotingClient) ClearExpireResponse() { for seq, responseObj := range drc.responseTable.Items() { response := responseObj.(*ResponseFuture) diff --git a/rocketmq-go/remoting/remoting_command.go b/rocketmq-go/remoting/remoting_command.go index eccf3b396..71e4def6b 100644 --- a/rocketmq-go/remoting/remoting_command.go +++ b/rocketmq-go/remoting/remoting_command.go @@ -25,13 +25,10 @@ import ( var opaque int32 -var RPC_TYPE int = 0 // 0, REQUEST_COMMAND -var RPC_ONEWAY int = 1 // 0, RPC - -//var RESPONSE_TYPE int= 1 << RPC_TYPE +//RESPONSE_TYPE RESPONSE_TYPE int= 1 << RPC_TYPE var RESPONSE_TYPE int = 1 -//rocketmq remoting command +//RemotingCommand rocketmq remoting command // both request and response use it type RemotingCommand struct { //request: request_code.go @@ -53,10 +50,12 @@ type RemotingCommand struct { Body []byte `json:"body,omitempty"` } +//NewRemotingCommand NewRemotingCommand func NewRemotingCommand(commandCode int16, customerHeader CustomerHeader) *RemotingCommand { return NewRemotingCommandWithBody(commandCode, customerHeader, nil) } +//NewRemotingCommandWithBody NewRemotingCommandWithBody func NewRemotingCommandWithBody(commandCode int16, customerHeader CustomerHeader, body []byte) *RemotingCommand { remotingCommand := new(RemotingCommand) remotingCommand.Code = commandCode diff --git a/rocketmq-go/remoting/request_processor.go b/rocketmq-go/remoting/request_processor.go index 1c71b0970..f5b8a9564 100644 --- a/rocketmq-go/remoting/request_processor.go +++ b/rocketmq-go/remoting/request_processor.go @@ -17,6 +17,8 @@ limitations under the License. package remoting +//ClientRequestProcessor ClientRequestProcessor when invoke this client +// see initClientRequestProcessor type ClientRequestProcessor func(remotingCommand *RemotingCommand) (responseCommand *RemotingCommand) //CHECK_TRANSACTION_STATE diff --git a/rocketmq-go/remoting/response_future.go b/rocketmq-go/remoting/response_future.go index a1c3b8945..869daa55e 100644 --- a/rocketmq-go/remoting/response_future.go +++ b/rocketmq-go/remoting/response_future.go @@ -17,6 +17,7 @@ limitations under the License. package remoting +//ResponseFuture ResponseFuture type ResponseFuture struct { ResponseCommand *RemotingCommand SendRequestOK bool @@ -27,4 +28,6 @@ type ResponseFuture struct { BeginTimestamp int64 Done chan bool } + +//InvokeCallback InvokeCallback type InvokeCallback func(responseFuture *ResponseFuture) diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index bdd2d6020..ee2d7e4d5 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -24,6 +24,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" ) +//RocketMqSerializer RocketMqSerializer type RocketMqSerializer struct { } diff --git a/rocketmq-go/remoting/rpchook.go b/rocketmq-go/remoting/rpchook.go index 92d7cf122..894f25c68 100644 --- a/rocketmq-go/remoting/rpchook.go +++ b/rocketmq-go/remoting/rpchook.go @@ -17,7 +17,10 @@ limitations under the License. package remoting +//RPCHook RPCHook type RPCHook interface { + //DoBeforeRequest DoBeforeRequest DoBeforeRequest(string, *RemotingCommand) + //DoBeforeResponse DoBeforeResponse DoBeforeResponse(string, *RemotingCommand) } diff --git a/rocketmq-go/remoting/serializable.go b/rocketmq-go/remoting/serializable.go index 0c819850d..4f9ac628f 100644 --- a/rocketmq-go/remoting/serializable.go +++ b/rocketmq-go/remoting/serializable.go @@ -24,17 +24,22 @@ import ( "github.com/golang/glog" ) +//SerializerHandler rocketmq client SerializerHandler type SerializerHandler struct { serializeType rocketmqm.SerializeType serializer Serializer //which serializer this client use, depend on constant.USE_HEADER_SERIALIZE_TYPE } +//Serializer rocketmq Serializer type Serializer interface { encodeHeaderData(request *RemotingCommand) []byte decodeRemoteCommand(header, body []byte) *RemotingCommand } +//JSON_SERIALIZER json serializer var JSON_SERIALIZER = &JsonSerializer{} + +//ROCKETMQ_SERIALIZER rocketmq serializer var ROCKETMQ_SERIALIZER = &RocketMqSerializer{} func newSerializerHandler(serializeType rocketmqm.SerializeType) SerializerHandler { diff --git a/rocketmq-go/util/compress_util.go b/rocketmq-go/util/compress_util.go index 06eb6ed94..913088bd2 100644 --- a/rocketmq-go/util/compress_util.go +++ b/rocketmq-go/util/compress_util.go @@ -24,6 +24,7 @@ import ( "io/ioutil" ) +//UnCompress un compress byte array func UnCompress(body []byte) (unCompressBody []byte, err error) { b := bytes.NewReader(body) z, err := zlib.NewReader(b) From 6306f14939a777c2cd6341c06969d8a185566109 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Fri, 25 Aug 2017 22:16:07 +0800 Subject: [PATCH 66/88] comment on kernel --- rocketmq-go/kernel/consume_message_service.go | 2 +- rocketmq-go/kernel/mq_client_manage.go | 8 +++--- rocketmq-go/kernel/mq_producer.go | 4 +-- rocketmq-go/kernel/mq_push_consumer.go | 11 ++++++-- rocketmq-go/kernel/offset_store.go | 28 +++++++++++-------- rocketmq-go/kernel/producer_service.go | 14 ++++++---- .../kernel/producer_service_for_send_back.go | 2 +- rocketmq-go/kernel/pull_message_controller.go | 12 ++++---- rocketmq-go/kernel/rebalance.go | 6 ++-- rocketmq-go/kernel/rebalance_controller.go | 11 ++++---- .../message}/message_client_id_generator.go | 10 ++++--- .../message_client_id_generator_test.go | 6 ++-- rocketmq-go/model/message/message_impl.go | 2 +- rocketmq-go/remoting/remoting_client.go | 6 ++-- 14 files changed, 70 insertions(+), 52 deletions(-) rename rocketmq-go/{util => model/message}/message_client_id_generator.go (92%) rename rocketmq-go/{util => model/message}/message_client_id_generator_test.go (93%) diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 4843612cc..6bc440731 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -139,7 +139,7 @@ func (c *consumeMessageConcurrentlyServiceImpl) processConsumeResult(result rock } commitOffset := processQueue.RemoveMessage(successMessages) if commitOffset > 0 && !processQueue.IsDropped() { - c.offsetStore.UpdateOffset(messageQueue, commitOffset, true) + c.offsetStore.updateOffset(messageQueue, commitOffset, true) } } diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index e51b8c49a..0aa97c823 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -39,7 +39,7 @@ type MqClientManager struct { mqClient RocketMqClient pullMessageController *PullMessageController cleanExpireMsgController *cleanExpireMsgController - rebalanceControllr *RebalanceController + rebalanceControllr *rebalanceController defaultProducerService *DefaultProducerService } @@ -50,7 +50,7 @@ func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManage rocketMqManager.mqClient = MqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.cleanExpireMsgController = newCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) - rocketMqManager.rebalanceControllr = NewRebalanceController(rocketMqManager.clientFactory) + rocketMqManager.rebalanceControllr = newRebalanceController(rocketMqManager.clientFactory) return } @@ -60,14 +60,14 @@ func (m *MqClientManager) Start() { } func (m *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { - producer.producerService = NewDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) + producer.producerService = newDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) m.clientFactory.producerTable[producer.producerGroup] = producer return } func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { if m.defaultProducerService == nil { - m.defaultProducerService = NewDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) + m.defaultProducerService = newDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) } consumer.mqClient = m.mqClient consumer.offsetStore = RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) diff --git a/rocketmq-go/kernel/mq_producer.go b/rocketmq-go/kernel/mq_producer.go index 0c7acf133..2848e9231 100644 --- a/rocketmq-go/kernel/mq_producer.go +++ b/rocketmq-go/kernel/mq_producer.go @@ -41,10 +41,10 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProd } func (d *DefaultMQProducer) Send(msg rocketmqm.Message) (sendResult *model.SendResult, err error) { - sendResult, err = d.producerService.SendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", d.ProducerConfig.SendMsgTimeout) + sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", d.ProducerConfig.SendMsgTimeout) return } func (d *DefaultMQProducer) SendWithTimeout(msg rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { - sendResult, err = d.producerService.SendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", timeout) + sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", timeout) return } diff --git a/rocketmq-go/kernel/mq_push_consumer.go b/rocketmq-go/kernel/mq_push_consumer.go index c2879655f..c3f950916 100644 --- a/rocketmq-go/kernel/mq_push_consumer.go +++ b/rocketmq-go/kernel/mq_push_consumer.go @@ -27,6 +27,7 @@ import ( "time" ) +//DefaultMQPushConsumer no order/cluster type DefaultMQPushConsumer struct { consumerGroup string consumeType string @@ -42,6 +43,7 @@ type DefaultMQPushConsumer struct { ConsumerConfig *rocketmqm.MqConsumerConfig } +//NewDefaultMQPushConsumer create a DefaultMQPushConsumer instance func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmqm.MqConsumerConfig) (defaultMQPushConsumer *DefaultMQPushConsumer) { defaultMQPushConsumer = &DefaultMQPushConsumer{ consumerGroup: consumerGroup, @@ -54,6 +56,8 @@ func NewDefaultMQPushConsumer(consumerGroup string, consumerConfig *rocketmqm.Mq return } + +//Subscribe subscribe topic, filter by subExpression func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { d.subscription[topic] = subExpression if len(subExpression) == 0 || subExpression == "*" { @@ -73,6 +77,7 @@ func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { } } +//RegisterMessageListener register message listener to this consumer func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener rocketmqm.MessageListener) { d.consumeMessageService = NewConsumeMessageConcurrentlyServiceImpl(messageListener) } @@ -96,13 +101,14 @@ func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]i if processQueue == nil || offset < 0 { continue } - glog.V(2).Info("now we UpdateOffset", messageQueue, offset) - d.offsetStore.UpdateOffset(&messageQueue, offset, false) + glog.V(2).Info("now we updateOffset", messageQueue, offset) + d.offsetStore.updateOffset(&messageQueue, offset, false) d.rebalance.removeProcessQueue(&messageQueue) } }() } +//Subscriptions get this consumer's subscription data func (d *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { subscriptions := make([]*model.SubscriptionData, 0) for _, subscription := range d.rebalance.subscriptionInner { @@ -111,6 +117,7 @@ func (d *DefaultMQPushConsumer) Subscriptions() []*model.SubscriptionData { return subscriptions } +//CleanExpireMsg cleanExpireMsg func (d *DefaultMQPushConsumer) CleanExpireMsg() { nowTime := util.CurrentTimeMillisInt64() //will cause nowTime - consumeStartTime <0 ,but no matter messageQueueList, processQueueList := d.rebalance.getProcessQueueList() diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index 1d401c237..ef24b14d7 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -28,21 +28,27 @@ import ( ) const ( + //MEMORY_FIRST_THEN_STORE memory first then store MEMORY_FIRST_THEN_STORE = 0 - READ_FROM_MEMORY = 1 - READ_FROM_STORE = 2 + //READ_FROM_MEMORY READ_FROM_MEMORY + READ_FROM_MEMORY = 1 + //READ_FROM_STORE READ_FROM_STORE + READ_FROM_STORE = 2 ) +//OffsetStore OffsetStore type OffsetStore interface { //update local offsetTable's offset - UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) + updateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) //read offset,from memory or broker - ReadOffset(mq *model.MessageQueue, readType int) int64 + readOffset(mq *model.MessageQueue, readType int) int64 //update broker's offset - Persist(mq *model.MessageQueue) + persist(mq *model.MessageQueue) //remove local offsetTable's offset - RemoveOffset(mq *model.MessageQueue) + removeOffset(mq *model.MessageQueue) } + +//RemoteOffsetStore offset store on remote type RemoteOffsetStore struct { groupName string mqClient RocketMqClient @@ -58,13 +64,13 @@ func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStor offsetStore.offsetTableLock = new(sync.RWMutex) return offsetStore } -func (r *RemoteOffsetStore) RemoveOffset(mq *model.MessageQueue) { +func (r *RemoteOffsetStore) removeOffset(mq *model.MessageQueue) { defer r.offsetTableLock.Unlock() r.offsetTableLock.Lock() delete(r.offsetTable, *mq) } -func (r *RemoteOffsetStore) Persist(mq *model.MessageQueue) { +func (r *RemoteOffsetStore) persist(mq *model.MessageQueue) { brokerAddr := r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { r.mqClient.TryToFindTopicPublishInfo(mq.Topic) @@ -78,7 +84,7 @@ func (r *RemoteOffsetStore) Persist(mq *model.MessageQueue) { r.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) } -func (r *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) int64 { +func (r *RemoteOffsetStore) readOffset(mq *model.MessageQueue, readType int) int64 { switch readType { case MEMORY_FIRST_THEN_STORE: @@ -98,7 +104,7 @@ func (r *RemoteOffsetStore) ReadOffset(mq *model.MessageQueue, readType int) int return -1 } glog.V(2).Info("READ_FROM_STORE", offset) - r.UpdateOffset(mq, offset, false) + r.updateOffset(mq, offset, false) return offset } @@ -151,7 +157,7 @@ func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *heade return -1, errors.New("query offset error") } -func (r *RemoteOffsetStore) UpdateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { +func (r *RemoteOffsetStore) updateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { defer r.offsetTableLock.Unlock() r.offsetTableLock.Lock() if mq != nil { diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 7d70548a0..c9e938bbe 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -29,11 +29,13 @@ import ( "github.com/golang/glog" ) +//ProducerService producerService, for send message type ProducerService interface { - CheckConfig() (err error) - SendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + checkConfig() (err error) + sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) } +//ProducerService ProducerService's implement type DefaultProducerService struct { producerGroup string producerConfig *rocketmqm.MqProducerConfig @@ -41,21 +43,21 @@ type DefaultProducerService struct { mqFaultStrategy mqFaultStrategy } -func NewDefaultProducerService(producerGroup string, producerConfig *rocketmqm.MqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { +func newDefaultProducerService(producerGroup string, producerConfig *rocketmqm.MqProducerConfig, mqClient RocketMqClient) (defaultProducerService *DefaultProducerService) { defaultProducerService = &DefaultProducerService{ mqClient: mqClient, producerGroup: producerGroup, producerConfig: producerConfig, } - defaultProducerService.CheckConfig() + defaultProducerService.checkConfig() return } -func (d *DefaultProducerService) CheckConfig() (err error) { +func (d *DefaultProducerService) checkConfig() (err error) { // todo check if not pass panic return } -func (d *DefaultProducerService) SendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 55ba7ee63..185d4c9a0 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -66,7 +66,7 @@ func (s *sendMessageBackProducerServiceImpl) sendRetryMessageBack(messageExt *me retryMessage.SetDelayTimeLevel(3 + messageExt.GetReconsumeTimes()) pp, _ := json.Marshal(retryMessage) glog.Info("look retryMessage ", string(pp), string(messageExt.Body())) - sendResult, err := s.defaultProducerService.SendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", s.defaultProducerService.producerConfig.SendMsgTimeout) + sendResult, err := s.defaultProducerService.sendDefaultImpl(retryMessage, constant.COMMUNICATIONMODE_SYNC, "", s.defaultProducerService.producerConfig.SendMsgTimeout) if err != nil { glog.Error(err) return err diff --git a/rocketmq-go/kernel/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go index cc088d35e..26066be30 100644 --- a/rocketmq-go/kernel/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -82,7 +82,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { p.pullMessageLater(pullRequest, delayPullTime) return } - commitOffsetValue := defaultMQPullConsumer.offsetStore.ReadOffset(pullRequest.MessageQueue, READ_FROM_MEMORY) + commitOffsetValue := defaultMQPullConsumer.offsetStore.readOffset(pullRequest.MessageQueue, READ_FROM_MEMORY) subscriptionData, ok := defaultMQPullConsumer.rebalance.subscriptionInner[pullRequest.MessageQueue.Topic] if !ok { @@ -122,7 +122,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { msgs = FilterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) if len(msgs) == 0 { if pullRequest.ProcessQueue.GetMsgCount() == 0 { - defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) + defaultMQPullConsumer.offsetStore.updateOffset(pullRequest.MessageQueue, nextBeginOffset, true) } } pullRequest.ProcessQueue.PutMessage(msgs) @@ -145,7 +145,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { if responseCommand.Code == remoting.PULL_NOT_FOUND || responseCommand.Code == remoting.PULL_RETRY_IMMEDIATELY { //NO_NEW_MSG //NO_MATCHED_MSG if pullRequest.ProcessQueue.GetMsgCount() == 0 { - defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) + defaultMQPullConsumer.offsetStore.updateOffset(pullRequest.MessageQueue, nextBeginOffset, true) } //update offset increase only //failedPullRequest, _ := json.Marshal(pullRequest) @@ -157,7 +157,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { go func() { executeTaskLater := time.NewTimer(10 * time.Second) <-executeTaskLater.C - defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, false) + defaultMQPullConsumer.offsetStore.updateOffset(pullRequest.MessageQueue, nextBeginOffset, false) defaultMQPullConsumer.rebalance.removeProcessQueue(pullRequest.MessageQueue) }() } else { @@ -179,7 +179,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { //func (p *PullMessageController) updateOffsetIfNeed(msgs []message.MessageExtImpl, pullRequest *model.PullRequest, defaultMQPullConsumer *DefaultMQPushConsumer, nextBeginOffset int64) { // if len(msgs) == 0 { // if pullRequest.ProcessQueue.GetMsgCount() == 0 { -// defaultMQPullConsumer.offsetStore.UpdateOffset(pullRequest.MessageQueue, nextBeginOffset, true) +// defaultMQPullConsumer.OffsetStore.updateOffset(pullRequest.MessageQueue, nextBeginOffset, true) // } // } //} @@ -318,7 +318,7 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { // >= 3.5.8 use clientUniqMsgId msg.SetMsgId(msg.GetMsgUniqueKey()) if len(msg.MsgId()) == 0 { - msg.SetMsgId(util.GeneratorMessageOffsetId(storeHost, storePort, msg.CommitLogOffset)) + msg.SetMsgId(message.GeneratorMessageOffsetId(storeHost, storePort, msg.CommitLogOffset)) } msgs = append(msgs, msg) } diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 42a1f5f90..0be2464e7 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -90,8 +90,8 @@ func (r *rebalance) getProcessQueueList() (messageQueueList []model.MessageQueue //removeUnnecessaryMessageQueue you should drop it first func (r *rebalance) removeProcessQueue(messageQueue *model.MessageQueue) { - r.offsetStore.Persist(messageQueue) - r.offsetStore.RemoveOffset(messageQueue) + r.offsetStore.persist(messageQueue) + r.offsetStore.removeOffset(messageQueue) r.removeMessageQueueFromMap(*messageQueue) } func (r *rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { @@ -226,7 +226,7 @@ func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.M } func (r *rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { var result int64 = -1 - lastOffset := r.offsetStore.ReadOffset(mq, READ_FROM_STORE) + lastOffset := r.offsetStore.readOffset(mq, READ_FROM_STORE) switch r.consumerConfig.ConsumeFromWhere { case rocketmqm.CONSUME_FROM_LAST_OFFSET: if lastOffset >= 0 { diff --git a/rocketmq-go/kernel/rebalance_controller.go b/rocketmq-go/kernel/rebalance_controller.go index 171585238..093f1a0e2 100644 --- a/rocketmq-go/kernel/rebalance_controller.go +++ b/rocketmq-go/kernel/rebalance_controller.go @@ -17,18 +17,19 @@ limitations under the License. package kernel -type RebalanceController struct { +//rebalanceController rebalanceController +type rebalanceController struct { clientFactory *clientFactory } -func NewRebalanceController(clientFactory *clientFactory) *RebalanceController { - return &RebalanceController{ +func newRebalanceController(clientFactory *clientFactory) *rebalanceController { + return &rebalanceController{ clientFactory: clientFactory, } } -func (self *RebalanceController) doRebalance() { - for _, consumer := range self.clientFactory.consumerTable { +func (r *rebalanceController) doRebalance() { + for _, consumer := range r.clientFactory.consumerTable { consumer.rebalance.doRebalance() } } diff --git a/rocketmq-go/util/message_client_id_generator.go b/rocketmq-go/model/message/message_client_id_generator.go similarity index 92% rename from rocketmq-go/util/message_client_id_generator.go rename to rocketmq-go/model/message/message_client_id_generator.go index 895cdf68f..1dc4b860a 100644 --- a/rocketmq-go/util/message_client_id_generator.go +++ b/rocketmq-go/model/message/message_client_id_generator.go @@ -15,11 +15,12 @@ See the License for the specific language governing permissions and limitations under the License. */ -package util +package message import ( "bytes" "encoding/binary" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "os" "strconv" "strings" @@ -28,7 +29,7 @@ import ( ) var ( - counter int16 = 0 + counter int16 startTime int64 nextStartTime int64 idPrefix string @@ -42,7 +43,7 @@ var ( //2 bytes for counter, //4 bytes for timediff, //(time.Now().UnixNano() - startTime) / 1000000) divide 1000000 because use time millis -func GeneratorMessageClientId() (uniqMessageId string) { +func generatorMessageClientId() (uniqMessageId string) { defer lock.Unlock() lock.Lock() if len(idPrefix) == 0 { @@ -59,6 +60,7 @@ func GeneratorMessageClientId() (uniqMessageId string) { return } +//GeneratorMessageOffsetId generator message offsetId func GeneratorMessageOffsetId(storeHost []byte, port int32, commitOffset int64) (messageOffsetId string) { var buf = bytes.NewBuffer([]byte{}) binary.Write(buf, binary.BigEndian, storeHost) @@ -75,7 +77,7 @@ func generatorMessageClientIdPrefix() (messageClientIdPrefix string) { pid int16 classloaderId int32 = -1 // golang don't have this ) - ip4Bytes = GetIp4Bytes() + ip4Bytes = util.GetIp4Bytes() pid = int16(os.Getpid()) var buf = bytes.NewBuffer([]byte{}) binary.Write(buf, binary.BigEndian, ip4Bytes) diff --git a/rocketmq-go/util/message_client_id_generator_test.go b/rocketmq-go/model/message/message_client_id_generator_test.go similarity index 93% rename from rocketmq-go/util/message_client_id_generator_test.go rename to rocketmq-go/model/message/message_client_id_generator_test.go index 3e16c8644..f30143ed8 100644 --- a/rocketmq-go/util/message_client_id_generator_test.go +++ b/rocketmq-go/model/message/message_client_id_generator_test.go @@ -15,18 +15,18 @@ See the License for the specific language governing permissions and limitations under the License. */ -package util +package message import ( "testing" ) func TestGeneratorMessageClientId(t *testing.T) { - id1 := GeneratorMessageClientId() + id1 := generatorMessageClientId() if id1 == "" { t.Errorf("TestGeneratorMessageClientId failed") } - id2 := GeneratorMessageClientId() + id2 := generatorMessageClientId() if id2 == "" || id1 == id2 { t.Errorf("TestGeneratorMessageClientId failed : create same clientId") } diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go index aeefc0404..ed19533c1 100644 --- a/rocketmq-go/model/message/message_impl.go +++ b/rocketmq-go/model/message/message_impl.go @@ -139,7 +139,7 @@ func (m *MessageImpl) GeneratorMsgUniqueKey() { if len(m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX]) > 0 { return } - m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = util.GeneratorMessageClientId() + m.properties[constant.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX] = generatorMessageClientId() } //GetMsgUniqueKey only use by system diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 38af41db8..08806ce64 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -153,7 +153,7 @@ func (drc *DefaultRemotingClient) sendRequest(header, body []byte, conn net.Conn if err != nil { glog.Error(err) if len(addr) > 0 { - drc.ReleaseConn(addr, conn) + drc.releaseConn(addr, conn) } return err } @@ -261,7 +261,7 @@ func (drc *DefaultRemotingClient) createAndHandleTcpConn(address string) (conn n go drc.handlerReceiveLoop(conn, address) //handler连接 处理这个连接返回的结果 return } -func (drc *DefaultRemotingClient) ReleaseConn(addr string, conn net.Conn) { +func (drc *DefaultRemotingClient) releaseConn(addr string, conn net.Conn) { defer drc.connTableLock.Unlock() conn.Close() drc.connTableLock.Lock() @@ -272,7 +272,7 @@ func (drc *DefaultRemotingClient) handlerReceiveLoop(conn net.Conn, addr string) defer func() { //when for is break releaseConn glog.Error(err, addr) - drc.ReleaseConn(addr, conn) + drc.releaseConn(addr, conn) }() b := make([]byte, 1024) var length, headerLength, bodyLength int32 From 528bda1acd42584963ca8ed03ef965adb561d96e Mon Sep 17 00:00:00 2001 From: StyleTang Date: Fri, 25 Aug 2017 22:33:25 +0800 Subject: [PATCH 67/88] comment on util --- rocketmq-go/kernel/mq_client_manage.go | 2 +- rocketmq-go/kernel/offset_store.go | 6 ++-- rocketmq-go/model/message_queue.go | 7 +---- rocketmq-go/util/compress_util.go | 3 ++ rocketmq-go/util/ip.go | 2 ++ rocketmq-go/util/json_util.go | 43 +++++++++++++------------- rocketmq-go/util/map_util.go | 1 + rocketmq-go/util/message_properties.go | 5 ++- rocketmq-go/util/regex_util.go | 4 +-- rocketmq-go/util/string_util.go | 18 ++++++++--- rocketmq-go/util/time_util.go | 3 ++ 11 files changed, 55 insertions(+), 39 deletions(-) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 0aa97c823..138915e73 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -70,7 +70,7 @@ func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { m.defaultProducerService = newDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) } consumer.mqClient = m.mqClient - consumer.offsetStore = RemoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) + consumer.offsetStore = remoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) m.clientFactory.consumerTable[consumer.consumerGroup] = consumer consumer.rebalance = newRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index ef24b14d7..1edc3354f 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -56,7 +56,7 @@ type RemoteOffsetStore struct { offsetTableLock *sync.RWMutex } -func RemoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStore { +func remoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStore { offsetStore := new(RemoteOffsetStore) offsetStore.groupName = groupName offsetStore.mqClient = mqClient @@ -94,9 +94,9 @@ func (r *RemoteOffsetStore) readOffset(mq *model.MessageQueue, readType int) int r.offsetTableLock.RUnlock() if ok { return offset - } else { - return -1 } + return -1 + case READ_FROM_STORE: offset, err := r.fetchConsumeOffsetFromBroker(mq) if err != nil { diff --git a/rocketmq-go/model/message_queue.go b/rocketmq-go/model/message_queue.go index 4b03a4536..c8429e44f 100644 --- a/rocketmq-go/model/message_queue.go +++ b/rocketmq-go/model/message_queue.go @@ -51,12 +51,7 @@ func (m MessageQueues) Less(i, j int) bool { } else if imq.BrokerName < jmq.BrokerName { return false } - - if imq.QueueId < jmq.QueueId { - return true - } else { - return false - } + return imq.QueueId < jmq.QueueId } //Swap swap queue diff --git a/rocketmq-go/util/compress_util.go b/rocketmq-go/util/compress_util.go index 913088bd2..30df33c59 100644 --- a/rocketmq-go/util/compress_util.go +++ b/rocketmq-go/util/compress_util.go @@ -36,6 +36,8 @@ func UnCompress(body []byte) (unCompressBody []byte, err error) { z.Close() return unCompressBody, nil } + +//Compress compress byte array func Compress(body []byte) (compressBody []byte, err error) { var in bytes.Buffer w := zlib.NewWriter(&in) @@ -45,6 +47,7 @@ func Compress(body []byte) (compressBody []byte, err error) { return } +//CompressWithLevel compress byte array with level func CompressWithLevel(body []byte, level int) (compressBody []byte, err error) { var ( in bytes.Buffer diff --git a/rocketmq-go/util/ip.go b/rocketmq-go/util/ip.go index 874b532b6..6599842f5 100644 --- a/rocketmq-go/util/ip.go +++ b/rocketmq-go/util/ip.go @@ -22,12 +22,14 @@ import ( "strings" ) +//GetIp4Bytes get ip4 byte array func GetIp4Bytes() (ret []byte) { ip := getIp() ret = ip[len(ip)-4:] return } +//GetLocalIp4 get local ip4 func GetLocalIp4() (ip4 string) { ip := getIp() if ip.To4() != nil { diff --git a/rocketmq-go/util/json_util.go b/rocketmq-go/util/json_util.go index b7b013497..c052223d8 100644 --- a/rocketmq-go/util/json_util.go +++ b/rocketmq-go/util/json_util.go @@ -21,25 +21,26 @@ import ( "errors" ) -const ( - STRING = "STRING" - NUMBER = "NUMBER" - - START_OBJ = "START_OBJ" //{ - END_OBJ = "END_OBJ" //} - COMMA = "COMMA" //, - COLON = "COLON" //: +//TOKEN_TYPE json string's token type +type TOKEN_TYPE byte - //// may be next version impl it - //BOOL - //NULL - //START_ARRAY //[ - //END_ARRAY //] - //EOF +const ( + //STRING string + STRING TOKEN_TYPE = iota + //NUMBER number + NUMBER + //START_OBJ start object + START_OBJ + //END_OBJ end object + END_OBJ + //COMMA comma + COMMA + //COLON colon + COLON ) -type Token struct { - tokenType string +type token struct { + tokenType TOKEN_TYPE tokenValue string } @@ -54,7 +55,7 @@ type parseInfo struct { } func GetKvStringMap(str string) (kvMap map[string]string, err error) { - var tokenList []Token + var tokenList []token tokenList, err = parseTokenList(str) kvMap = map[string]string{} currentParseInfo := &parseInfo{ @@ -95,7 +96,7 @@ func GetKvStringMap(str string) (kvMap map[string]string, err error) { } return } -func parseValue(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo, key, value string) { +func parseValue(info *parseInfo, nowToken *token) (parsedInfo *parseInfo, key, value string) { if nowToken.tokenType == COMMA { // , split kv pair if info.startObjCount == 1 { key = info.nowKey @@ -116,7 +117,7 @@ func parseValue(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo, key, v } return } -func parseKey(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo) { +func parseKey(info *parseInfo, nowToken *token) (parsedInfo *parseInfo) { if nowToken.tokenType == COLON { //: split k and v if info.startObjCount == 1 { info.readType = 2 @@ -131,11 +132,11 @@ func parseKey(info *parseInfo, nowToken *Token) (parsedInfo *parseInfo) { return info } -func parseTokenList(str string) (tokenList []Token, err error) { +func parseTokenList(str string) (tokenList []token, err error) { for i := 0; i < len(str); i++ { c := str[i] - token := Token{} + token := token{} switch c { case '{': token.tokenType = START_OBJ diff --git a/rocketmq-go/util/map_util.go b/rocketmq-go/util/map_util.go index 86fcb1ac4..eea263107 100644 --- a/rocketmq-go/util/map_util.go +++ b/rocketmq-go/util/map_util.go @@ -5,6 +5,7 @@ import ( "strings" ) +// func Struct2Map(structBody interface{}) (resultMap map[string]interface{}) { resultMap = make(map[string]interface{}) value := reflect.ValueOf(structBody) diff --git a/rocketmq-go/util/message_properties.go b/rocketmq-go/util/message_properties.go index 30a40318b..56babb432 100644 --- a/rocketmq-go/util/message_properties.go +++ b/rocketmq-go/util/message_properties.go @@ -21,11 +21,13 @@ import ( "strings" ) -//char 1 and 2 from java code +//NAME_VALUE_SEPARATOR char 1 and 2 from java code var NAME_VALUE_SEPARATOR = string(rune(1)) +//PROPERTY_SEPARATOR property separator var PROPERTY_SEPARATOR = string(rune(2)) +//MessageProperties2String convert message properties to string func MessageProperties2String(propertiesMap map[string]string) (ret string) { for key, value := range propertiesMap { ret = ret + key + NAME_VALUE_SEPARATOR + value + PROPERTY_SEPARATOR @@ -33,6 +35,7 @@ func MessageProperties2String(propertiesMap map[string]string) (ret string) { return } +//String2MessageProperties convert string properties to map func String2MessageProperties(properties string) (ret map[string]string) { ret = make(map[string]string) for _, nameValueStr := range strings.Split(properties, PROPERTY_SEPARATOR) { diff --git a/rocketmq-go/util/regex_util.go b/rocketmq-go/util/regex_util.go index 1527e4042..1352415bd 100644 --- a/rocketmq-go/util/regex_util.go +++ b/rocketmq-go/util/regex_util.go @@ -21,9 +21,7 @@ import ( "regexp" ) -//var regexpMap map[string]*regexp.Regexp -//var rwMutex sync.RWMutex - +//MatchString reports whether the Regexp matches the string func MatchString(value, pattern string) bool { re, err := regexp.Compile(pattern) if err != nil { diff --git a/rocketmq-go/util/string_util.go b/rocketmq-go/util/string_util.go index 3318cdcc3..07a96d99c 100644 --- a/rocketmq-go/util/string_util.go +++ b/rocketmq-go/util/string_util.go @@ -19,6 +19,7 @@ package util import "strconv" +//StrToIntWithDefaultValue convert string to int func StrToIntWithDefaultValue(str string, defaultValue int) (result int) { ret, err := strconv.Atoi(str) if err != nil { @@ -28,6 +29,8 @@ func StrToIntWithDefaultValue(str string, defaultValue int) (result int) { } return } + +//ReadString read string if the interface{} is string func ReadString(obj interface{}) (ret string) { if obj == nil { ret = "" @@ -37,32 +40,35 @@ func ReadString(obj interface{}) (ret string) { return } +//IntToString convert int to string func IntToString(intValue int) (ret string) { ret = strconv.Itoa(intValue) return } -func StrToInt(str string) (result int, err error) { - result, err = strconv.Atoi(str) - return -} +//StrToInt32 convert string to int32 func StrToInt32(str string) (result int32, err error) { var ret int64 ret, err = strconv.ParseInt(str, 10, 32) result = int32(ret) return } + +//StrToInt16 convert string to int16 func StrToInt16(str string) (result int16, err error) { var ret int64 ret, err = strconv.ParseInt(str, 10, 16) result = int16(ret) return } + +//StrToInt64 convert string to int64 func StrToInt64(str string) (result int64, err error) { result, err = strconv.ParseInt(str, 10, 64) return } +//StrToInt32WithDefaultValue convert string to int32,if error return defaultValue func StrToInt32WithDefaultValue(str string, defaultValue int32) (result int32) { ret, err := StrToInt32(str) if err != nil { @@ -72,6 +78,8 @@ func StrToInt32WithDefaultValue(str string, defaultValue int32) (result int32) { } return } + +//StrToInt16WithDefaultValue convert string to int16,if error return defaultValue func StrToInt16WithDefaultValue(str string, defaultValue int16) (result int16) { ret, err := StrToInt16(str) if err != nil { @@ -81,6 +89,8 @@ func StrToInt16WithDefaultValue(str string, defaultValue int16) (result int16) { } return } + +//StrToInt64WithDefaultValue convert string to int64,if error return defaultValue func StrToInt64WithDefaultValue(str string, defaultValue int64) (result int64) { ret, err := StrToInt64(str) if err != nil { diff --git a/rocketmq-go/util/time_util.go b/rocketmq-go/util/time_util.go index 80f962d23..352728c9b 100644 --- a/rocketmq-go/util/time_util.go +++ b/rocketmq-go/util/time_util.go @@ -22,10 +22,13 @@ import ( "time" ) +//CurrentTimeMillisInt64 get current time millis func CurrentTimeMillisInt64() (ret int64) { ret = time.Now().UnixNano() / 1000000 return } + +//CurrentTimeMillisStr get current time millis string func CurrentTimeMillisStr() (ret string) { ret = strconv.FormatInt(CurrentTimeMillisInt64(), 10) return From 1904e27f2a4daf0f0de6261b2fd82e9c25fcd504 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 16:19:37 +0800 Subject: [PATCH 68/88] send result mq brokerName --- rocketmq-go/kernel/producer_service.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index c9e938bbe..6fd0a6245 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -79,7 +79,7 @@ func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, c return } -func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerName, brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body()) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -87,7 +87,8 @@ func (d *DefaultProducerService) producerSendMessageRequest(brokerAddr string, s glog.Error(err) return } - sendResult, err = processSendResponse(brokerAddr, message, response) + + sendResult, err = processSendResponse(brokerName, message, response) return } func processSendResponse(brokerName string, message *message.MessageImpl, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { @@ -229,7 +230,7 @@ func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, mes ReconsumeTimes: message.GetReconsumeTimes(), MaxReconsumeTimes: message.GetMaxReconsumeTimes(), } - sendResult, err = d.producerSendMessageRequest(brokerAddr, sendMessageHeader, message, timeout) + sendResult, err = d.producerSendMessageRequest(messageQueue.BrokerName,brokerAddr, sendMessageHeader, message, timeout) return } From 4e6f2a88a0f888aee23f2c51dc1d8b2e5813d77d Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 16:25:36 +0800 Subject: [PATCH 69/88] change api --- rocketmq-go/{ => api}/model/message_queue.go | 2 +- rocketmq-go/api/model/send_result.go | 114 ++++++++++++++++++ .../allocate/allocate_message_averagely.go | 6 +- .../allocate_message_averagely_by_circle.go | 6 +- .../allocate/allocate_message_by_config.go | 8 +- .../allocate_message_by_machine_room.go | 6 +- .../allocate_message_queue_strategy.go | 4 +- rocketmq-go/kernel/consume_message_service.go | 6 +- rocketmq-go/kernel/mq_client.go | 18 +-- rocketmq-go/kernel/mq_client_manage.go | 2 +- rocketmq-go/kernel/mq_fault_strategy.go | 3 +- rocketmq-go/kernel/mq_push_consumer.go | 2 +- rocketmq-go/kernel/offset_store.go | 24 ++-- rocketmq-go/kernel/producer_service.go | 8 +- rocketmq-go/kernel/rebalance.go | 28 ++--- rocketmq-go/model/consumer_running_info.go | 9 +- rocketmq-go/model/pull_request.go | 4 +- rocketmq-go/model/reset_offset_body.go | 7 +- rocketmq-go/model/send_result.go | 8 +- rocketmq-go/model/topic_publish_info.go | 13 +- 20 files changed, 202 insertions(+), 76 deletions(-) rename rocketmq-go/{ => api}/model/message_queue.go (99%) create mode 100644 rocketmq-go/api/model/send_result.go diff --git a/rocketmq-go/model/message_queue.go b/rocketmq-go/api/model/message_queue.go similarity index 99% rename from rocketmq-go/model/message_queue.go rename to rocketmq-go/api/model/message_queue.go index c8429e44f..d1858060d 100644 --- a/rocketmq-go/model/message_queue.go +++ b/rocketmq-go/api/model/message_queue.go @@ -15,7 +15,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package model +package rocketmqm //MessageQueue message queue type MessageQueue struct { diff --git a/rocketmq-go/api/model/send_result.go b/rocketmq-go/api/model/send_result.go new file mode 100644 index 000000000..71e710481 --- /dev/null +++ b/rocketmq-go/api/model/send_result.go @@ -0,0 +1,114 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package rocketmqm + +import ( + "fmt" +) + +//SendStatus message send result +type SendStatus int + +const ( + //SendOK message send success + SendOK SendStatus = iota + //FlushDiskTimeout FlushDiskTimeout + FlushDiskTimeout + //FlushSlaveTimeout FlushSlaveTimeout + FlushSlaveTimeout + //SlaveNotAvaliable SlaveNotAvaliable + SlaveNotAvaliable +) + +//SendResult SendResult +type SendResult struct { + sendStatus SendStatus + msgID string + messageQueue MessageQueue + queueOffset int64 + transactionID string + offsetMsgID string + regionID string + traceOn bool +} + +func (result *SendResult) TraceOn() bool { + return result.traceOn +} + +func (result *SendResult) SetTraceOn(b bool) { + result.traceOn = b +} + +func (result *SendResult) SetRegionID(s string) { + result.regionID = s +} + +func (result *SendResult) MsgID() string { + return result.msgID +} + +func (result *SendResult) SetMsgID(s string) { + result.msgID = s +} + +func (result *SendResult) SendStatus() SendStatus { + return result.sendStatus +} + +func (result *SendResult) SetSendStatus(status SendStatus) { + result.sendStatus = status +} + +func (result *SendResult) MessageQueue() MessageQueue { + return result.messageQueue +} + +func (result *SendResult) SetMessageQueue(queue MessageQueue) { + result.messageQueue = queue +} + +func (result *SendResult) QueueOffset() int64 { + return result.queueOffset +} + +func (result *SendResult) SetQueueOffset(offset int64) { + result.queueOffset = offset +} + +func (result *SendResult) TransactionID() string { + return result.transactionID +} + +func (result *SendResult) SetTransactionID(s string) { + result.transactionID = s +} + +//OffsetMsgID get +func (result *SendResult) OffsetMsgID() string { + return result.offsetMsgID +} + +func (result *SendResult) SetOffsetMsgID(s string) { + result.offsetMsgID = s +} + +func (result *SendResult) String() string { + return fmt.Sprintf("SendResult [sendStatus=%s, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", + result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) +} diff --git a/rocketmq-go/kernel/allocate/allocate_message_averagely.go b/rocketmq-go/kernel/allocate/allocate_message_averagely.go index c476c27a7..588d8f9ee 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_averagely.go +++ b/rocketmq-go/kernel/allocate/allocate_message_averagely.go @@ -19,14 +19,14 @@ package allocate import ( "errors" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) //AllocateMessageQueueAveragely AllocateMessageQueueAveragely type AllocateMessageQueueAveragely struct{} //Allocate message queue -func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") @@ -40,7 +40,7 @@ func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCI return nil, errors.New("cidAll is nil or cidAll empty") } - result := make([]model.MessageQueue, 0) + result := make([]rocketmqm.MessageQueue, 0) for i, cid := range cidAll { if cid == currentCID { mqLen := len(mqAll) diff --git a/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go b/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go index 7626b0d9c..369d59f1c 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go +++ b/rocketmq-go/kernel/allocate/allocate_message_averagely_by_circle.go @@ -19,14 +19,14 @@ package allocate import ( "errors" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) //AllocateMessageQueueAveragelyByCircle AllocateMessageQueueAveragelyByCircle type AllocateMessageQueueAveragelyByCircle struct{} //Allocate message queue -func (a *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") } @@ -39,7 +39,7 @@ func (a *AllocateMessageQueueAveragelyByCircle) Allocate(consumerGroup string, c return nil, errors.New("cidAll is nil or cidAll empty") } - result := make([]model.MessageQueue, 0) + result := make([]rocketmqm.MessageQueue, 0) for i, cid := range cidAll { if cid == currentCID { mqLen := len(mqAll) diff --git a/rocketmq-go/kernel/allocate/allocate_message_by_config.go b/rocketmq-go/kernel/allocate/allocate_message_by_config.go index 6bebbd598..d392e677d 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_by_config.go +++ b/rocketmq-go/kernel/allocate/allocate_message_by_config.go @@ -17,14 +17,16 @@ limitations under the License. package allocate -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +) //AllocateMessageQueueByConfig AllocateMessageQueueByConfig type AllocateMessageQueueByConfig struct { - messageQueueList []model.MessageQueue + messageQueueList []rocketmqm.MessageQueue } //Allocate message queue -func (a *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueByConfig) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { return a.messageQueueList, nil } diff --git a/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go index 2af120106..8ccc4c6b7 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go +++ b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go @@ -19,7 +19,7 @@ package allocate import ( "errors" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) //AllocateMessageQueueByMachineRoom AllocateMessageQueueByMachineRoom @@ -27,7 +27,7 @@ type AllocateMessageQueueByMachineRoom struct { } //Allocate message queue -func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) { +func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") } @@ -40,7 +40,7 @@ func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, curre return nil, errors.New("cidAll is nil or cidAll empty") } - result := make([]model.MessageQueue, 0) + result := make([]rocketmqm.MessageQueue, 0) for i, cid := range cidAll { if cid == currentCID { mqLen := len(mqAll) diff --git a/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go index 4dabb5af4..d5123aa53 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go +++ b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go @@ -17,11 +17,11 @@ limitations under the License. package allocate -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" //Allocate message queue type AllocateMessageQueueStrategy interface { - Allocate(consumerGroup string, currentCID string, mqAll []*model.MessageQueue, cidAll []string) ([]model.MessageQueue, error) + Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) } //GetAllocateMessageQueueStrategyByConfig get allocate message queue strategy by config diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 6bc440731..4f9703b9c 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -29,7 +29,7 @@ import ( type consumeMessageService interface { init(consumerGroup string, mqClient RocketMqClient, offsetStore OffsetStore, defaultProducerService *DefaultProducerService, consumerConfig *rocketmqm.MqConsumerConfig) submitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, - messageQueue *model.MessageQueue, dispathToConsume bool) + messageQueue *rocketmqm.MessageQueue, dispathToConsume bool) sendMessageBack(messageExt *message.MessageExtImpl, delayLayLevel int, brokerName string) (err error) consumeMessageDirectly(messageExt *message.MessageExtImpl, brokerName string) (consumeMessageDirectlyResult model.ConsumeMessageDirectlyResult, err error) } @@ -54,7 +54,7 @@ func (c *consumeMessageConcurrentlyServiceImpl) init(consumerGroup string, mqCli c.consumerConfig = consumerConfig } -func (c *consumeMessageConcurrentlyServiceImpl) submitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *model.MessageQueue, dispathToConsume bool) { +func (c *consumeMessageConcurrentlyServiceImpl) submitConsumeRequest(msgs []message.MessageExtImpl, processQueue *model.ProcessQueue, messageQueue *rocketmqm.MessageQueue, dispathToConsume bool) { msgsLen := len(msgs) for i := 0; i < msgsLen; { begin := i @@ -102,7 +102,7 @@ func (c *consumeMessageConcurrentlyServiceImpl) consumeMessageDirectly(messageEx return } -func (c *consumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []message.MessageExtImpl, messageQueue *model.MessageQueue, processQueue *model.ProcessQueue) { +func (c *consumeMessageConcurrentlyServiceImpl) processConsumeResult(result rocketmqm.ConsumeConcurrentlyResult, msgs []message.MessageExtImpl, messageQueue *rocketmqm.MessageQueue, processQueue *model.ProcessQueue) { if processQueue.IsDropped() { glog.Warning("processQueue is dropped without process consume result. ", msgs) return diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 3f7694501..fb37237e8 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -45,7 +45,7 @@ type RocketMqClient interface { //get remoting client in mqClient GetRemotingClient() (remotingClient *remoting.DefaultRemotingClient) //get topic subscribe info - GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) + GetTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) //GetPublishTopicList GetPublishTopicList() []string //FetchMasterBrokerAddress @@ -69,9 +69,9 @@ type RocketMqClient interface { //ClearExpireResponse ClearExpireResponse() //GetMaxOffset - GetMaxOffset(mq *model.MessageQueue) int64 + GetMaxOffset(mq *rocketmqm.MessageQueue) int64 //SearchOffset - SearchOffset(mq *model.MessageQueue, time time.Time) int64 + SearchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 } var DEFAULT_TIMEOUT int64 = 6000 @@ -82,7 +82,7 @@ type MqClientImpl struct { topicRouteTable util.ConcurrentMap // map[string]*model.TopicRouteData //topic | topicRoteData brokerAddrTable util.ConcurrentMap //map[string]map[int]string //brokerName | map[brokerId]address topicPublishInfoTable util.ConcurrentMap //map[string]*model.TopicPublishInfo //topic | TopicPublishInfo //all use this - topicSubscribeInfoTable util.ConcurrentMap //map[string][]*model.MessageQueue //topic | MessageQueue + topicSubscribeInfoTable util.ConcurrentMap //map[string][]*rocketmqm.MessageQueue //topic | MessageQueue pullRequestQueue chan *model.PullRequest } @@ -94,18 +94,18 @@ func MqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor mqClientImpl.brokerAddrTable = util.New() //make(map[string]map[int]string) mqClientImpl.remotingClient = remoting.RemotingClientInit(clientConfig, clientRequestProcessor) mqClientImpl.topicPublishInfoTable = util.New() //make(map[string]*model.TopicPublishInfo) - mqClientImpl.topicSubscribeInfoTable = util.New() //make(map[string][]*model.MessageQueue) + mqClientImpl.topicSubscribeInfoTable = util.New() //make(map[string][]*rocketmqm.MessageQueue) mqClientImpl.pullRequestQueue = make(chan *model.PullRequest, 1024) return } -func (m *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*model.MessageQueue) { +func (m *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) { value, ok := m.topicSubscribeInfoTable.Get(topic) if ok { - messageQueueList = value.([]*model.MessageQueue) + messageQueueList = value.([]*rocketmqm.MessageQueue) } return } -func (m *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { +func (m *MqClientImpl) GetMaxOffset(mq *rocketmqm.MessageQueue) int64 { brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { m.TryToFindTopicPublishInfo(mq.Topic) @@ -121,7 +121,7 @@ func (m *MqClientImpl) GetMaxOffset(mq *model.MessageQueue) int64 { queryOffsetResponseHeader.FromMap(response.ExtFields) return queryOffsetResponseHeader.Offset } -func (m *MqClientImpl) SearchOffset(mq *model.MessageQueue, time time.Time) int64 { +func (m *MqClientImpl) SearchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 { brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { m.TryToFindTopicPublishInfo(mq.Topic) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 138915e73..91a393078 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -175,7 +175,7 @@ func (m *MqClientManager) resetConsumerClientOffset(cmd *remoting.RemotingComman } } -func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[model.MessageQueue]int64) { +func (m *MqClientManager) resetConsumerOffset(topic, group string, offsetTable map[rocketmqm.MessageQueue]int64) { consumer := m.clientFactory.consumerTable[group] if consumer == nil { glog.Error("resetConsumerOffset because consumer not online,group=", group) diff --git a/rocketmq-go/kernel/mq_fault_strategy.go b/rocketmq-go/kernel/mq_fault_strategy.go index b052f3c53..bca20d630 100644 --- a/rocketmq-go/kernel/mq_fault_strategy.go +++ b/rocketmq-go/kernel/mq_fault_strategy.go @@ -19,6 +19,7 @@ package kernel import ( "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) @@ -27,7 +28,7 @@ type mqFaultStrategy struct { //if first select : random one //if has error broker before ,skip the err broker -func selectOneMessageQueue(topicPublishInfo *model.TopicPublishInfo, lastFailedBroker string) (mqQueue model.MessageQueue, err error) { +func selectOneMessageQueue(topicPublishInfo *model.TopicPublishInfo, lastFailedBroker string) (mqQueue rocketmqm.MessageQueue, err error) { queueIndex := topicPublishInfo.FetchQueueIndex() queues := topicPublishInfo.MessageQueueList if len(lastFailedBroker) == 0 { diff --git a/rocketmq-go/kernel/mq_push_consumer.go b/rocketmq-go/kernel/mq_push_consumer.go index c3f950916..92630b84b 100644 --- a/rocketmq-go/kernel/mq_push_consumer.go +++ b/rocketmq-go/kernel/mq_push_consumer.go @@ -82,7 +82,7 @@ func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener rocketmq d.consumeMessageService = NewConsumeMessageConcurrentlyServiceImpl(messageListener) } -func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[model.MessageQueue]int64) { +func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[rocketmqm.MessageQueue]int64) { d.pause = true glog.V(2).Info("now we clearProcessQueue 0 ", offsetTable) diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index 1edc3354f..efede3bce 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -19,8 +19,8 @@ package kernel import ( "errors" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/remoting" "github.com/golang/glog" "strconv" @@ -39,20 +39,20 @@ const ( //OffsetStore OffsetStore type OffsetStore interface { //update local offsetTable's offset - updateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) + updateOffset(mq *rocketmqm.MessageQueue, offset int64, increaseOnly bool) //read offset,from memory or broker - readOffset(mq *model.MessageQueue, readType int) int64 + readOffset(mq *rocketmqm.MessageQueue, readType int) int64 //update broker's offset - persist(mq *model.MessageQueue) + persist(mq *rocketmqm.MessageQueue) //remove local offsetTable's offset - removeOffset(mq *model.MessageQueue) + removeOffset(mq *rocketmqm.MessageQueue) } //RemoteOffsetStore offset store on remote type RemoteOffsetStore struct { groupName string mqClient RocketMqClient - offsetTable map[model.MessageQueue]int64 + offsetTable map[rocketmqm.MessageQueue]int64 offsetTableLock *sync.RWMutex } @@ -60,17 +60,17 @@ func remoteOffsetStoreInit(groupName string, mqClient RocketMqClient) OffsetStor offsetStore := new(RemoteOffsetStore) offsetStore.groupName = groupName offsetStore.mqClient = mqClient - offsetStore.offsetTable = make(map[model.MessageQueue]int64) + offsetStore.offsetTable = make(map[rocketmqm.MessageQueue]int64) offsetStore.offsetTableLock = new(sync.RWMutex) return offsetStore } -func (r *RemoteOffsetStore) removeOffset(mq *model.MessageQueue) { +func (r *RemoteOffsetStore) removeOffset(mq *rocketmqm.MessageQueue) { defer r.offsetTableLock.Unlock() r.offsetTableLock.Lock() delete(r.offsetTable, *mq) } -func (r *RemoteOffsetStore) persist(mq *model.MessageQueue) { +func (r *RemoteOffsetStore) persist(mq *rocketmqm.MessageQueue) { brokerAddr := r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { r.mqClient.TryToFindTopicPublishInfo(mq.Topic) @@ -84,7 +84,7 @@ func (r *RemoteOffsetStore) persist(mq *model.MessageQueue) { r.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) } -func (r *RemoteOffsetStore) readOffset(mq *model.MessageQueue, readType int) int64 { +func (r *RemoteOffsetStore) readOffset(mq *rocketmqm.MessageQueue, readType int) int64 { switch readType { case MEMORY_FIRST_THEN_STORE: @@ -112,7 +112,7 @@ func (r *RemoteOffsetStore) readOffset(mq *model.MessageQueue, readType int) int } -func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *model.MessageQueue) (int64, error) { +func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *rocketmqm.MessageQueue) (int64, error) { brokerAddr, _, found := r.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) if !found { @@ -157,7 +157,7 @@ func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *heade return -1, errors.New("query offset error") } -func (r *RemoteOffsetStore) updateOffset(mq *model.MessageQueue, offset int64, increaseOnly bool) { +func (r *RemoteOffsetStore) updateOffset(mq *rocketmqm.MessageQueue, offset int64, increaseOnly bool) { defer r.offsetTableLock.Unlock() r.offsetTableLock.Lock() if mq != nil { diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 6fd0a6245..3caccf281 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -126,7 +126,7 @@ func processSendResponse(brokerName string, message *message.MessageImpl, respon sendResult.SetOffsetMsgID(responseHeader.MsgId) sendResult.SetQueueOffset(responseHeader.QueueOffset) sendResult.SetTransactionID(responseHeader.TransactionId) - messageQueue := model.MessageQueue{Topic: message.Topic(), BrokerName: brokerName, + messageQueue := rocketmqm.MessageQueue{Topic: message.Topic(), BrokerName: brokerName, QueueId: responseHeader.QueueId} sendResult.SetMessageQueue(messageQueue) var regionId = responseHeader.MsgRegion @@ -174,7 +174,7 @@ func (d *DefaultProducerService) checkMessage(message *message.MessageImpl) (err func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { var ( sendTotalTime int - messageQueue model.MessageQueue + messageQueue rocketmqm.MessageQueue ) sendTotalTime = 1 @@ -195,7 +195,7 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.Mes return } -func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, messageQueue model.MessageQueue, +func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, messageQueue rocketmqm.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { @@ -230,7 +230,7 @@ func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, mes ReconsumeTimes: message.GetReconsumeTimes(), MaxReconsumeTimes: message.GetMaxReconsumeTimes(), } - sendResult, err = d.producerSendMessageRequest(messageQueue.BrokerName,brokerAddr, sendMessageHeader, message, timeout) + sendResult, err = d.producerSendMessageRequest(messageQueue.BrokerName, brokerAddr, sendMessageHeader, message, timeout) return } diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index 0be2464e7..c135717bf 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -41,7 +41,7 @@ type rebalance struct { subscriptionInnerLock sync.RWMutex mqClient RocketMqClient allocateMessageQueueStrategy allocate.AllocateMessageQueueStrategy - processQueueTable map[model.MessageQueue]*model.ProcessQueue // both subscribe topic and retry group + processQueueTable map[rocketmqm.MessageQueue]*model.ProcessQueue // both subscribe topic and retry group processQueueTableLock sync.RWMutex mutex sync.Mutex offsetStore OffsetStore @@ -49,23 +49,23 @@ type rebalance struct { } //when invoke GET_CONSUMER_RUNNING_INFO, getMqTableInfo will return ProcessQueueInfo -func (r *rebalance) getMqTableInfo() map[model.MessageQueue]model.ProcessQueueInfo { +func (r *rebalance) getMqTableInfo() map[rocketmqm.MessageQueue]model.ProcessQueueInfo { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() - mqTable := map[model.MessageQueue]model.ProcessQueueInfo{} + mqTable := map[rocketmqm.MessageQueue]model.ProcessQueueInfo{} for messageQueue, processQueue := range r.processQueueTable { mqTable[messageQueue] = processQueue.ChangeToProcessQueueInfo() } return mqTable } -func (r *rebalance) getProcessQueue(messageQueue model.MessageQueue) *model.ProcessQueue { +func (r *rebalance) getProcessQueue(messageQueue rocketmqm.MessageQueue) *model.ProcessQueue { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() return r.processQueueTable[messageQueue] } -func (r *rebalance) clearProcessQueue(offsetTable map[model.MessageQueue]int64) { +func (r *rebalance) clearProcessQueue(offsetTable map[rocketmqm.MessageQueue]int64) { defer r.processQueueTableLock.Unlock() r.processQueueTableLock.Lock() for mq := range offsetTable { @@ -78,7 +78,7 @@ func (r *rebalance) clearProcessQueue(offsetTable map[model.MessageQueue]int64) } -func (r *rebalance) getProcessQueueList() (messageQueueList []model.MessageQueue, processQueueList []*model.ProcessQueue) { +func (r *rebalance) getProcessQueueList() (messageQueueList []rocketmqm.MessageQueue, processQueueList []*model.ProcessQueue) { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() for messageQueue, processQueue := range r.processQueueTable { @@ -89,12 +89,12 @@ func (r *rebalance) getProcessQueueList() (messageQueueList []model.MessageQueue } //removeUnnecessaryMessageQueue you should drop it first -func (r *rebalance) removeProcessQueue(messageQueue *model.MessageQueue) { +func (r *rebalance) removeProcessQueue(messageQueue *rocketmqm.MessageQueue) { r.offsetStore.persist(messageQueue) r.offsetStore.removeOffset(messageQueue) r.removeMessageQueueFromMap(*messageQueue) } -func (r *rebalance) removeMessageQueueFromMap(messageQueue model.MessageQueue) { +func (r *rebalance) removeMessageQueueFromMap(messageQueue rocketmqm.MessageQueue) { defer r.processQueueTableLock.Unlock() r.processQueueTableLock.Lock() delete(r.processQueueTable, messageQueue) @@ -125,7 +125,7 @@ func newRebalance(groupName string, subscription map[string]string, mqClient Roc subscriptionInner: subscriptionInner, allocateMessageQueueStrategy: allocate.GetAllocateMessageQueueStrategyByConfig("default"), messageModel: "CLUSTERING", - processQueueTable: make(map[model.MessageQueue]*model.ProcessQueue), + processQueueTable: make(map[rocketmqm.MessageQueue]*model.ProcessQueue), consumerConfig: consumerConfig, } } @@ -164,7 +164,7 @@ func (r *rebalance) rebalanceByTopic(topic string) error { mqs := r.mqClient.GetTopicSubscribeInfo(topic) r.topicSubscribeInfoTableLock.RUnlock() if len(mqs) > 0 && len(cidAll) > 0 { - var messageQueues model.MessageQueues = mqs + var messageQueues rocketmqm.MessageQueues = mqs var consumerIdSorter consumerIdSorter = cidAll sort.Sort(messageQueues) @@ -182,14 +182,14 @@ func (r *rebalance) rebalanceByTopic(topic string) error { return nil } -func (r *rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) updateProcessQueueTableInRebalance(topic string, mqSet []rocketmqm.MessageQueue) { defer r.processQueueTableLock.RUnlock() r.processQueueTableLock.RLock() r.removeTheQueueDontBelongHere(topic, mqSet) r.putTheQueueToProcessQueueTable(topic, mqSet) } -func (r *rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) removeTheQueueDontBelongHere(topic string, mqSet []rocketmqm.MessageQueue) { // there is n^2 todo improve for key, value := range r.processQueueTable { if topic != key.Topic { @@ -209,7 +209,7 @@ func (r *rebalance) removeTheQueueDontBelongHere(topic string, mqSet []model.Mes } } -func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.MessageQueue) { +func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []rocketmqm.MessageQueue) { for index, mq := range mqSet { _, ok := r.processQueueTable[mq] if !ok { @@ -224,7 +224,7 @@ func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []model.M } } -func (r *rebalance) computePullFromWhere(mq *model.MessageQueue) int64 { +func (r *rebalance) computePullFromWhere(mq *rocketmqm.MessageQueue) int64 { var result int64 = -1 lastOffset := r.offsetStore.readOffset(mq, READ_FROM_STORE) switch r.consumerConfig.ConsumeFromWhere { diff --git a/rocketmq-go/model/consumer_running_info.go b/rocketmq-go/model/consumer_running_info.go index a36c6aaa7..dc79d0fc4 100644 --- a/rocketmq-go/model/consumer_running_info.go +++ b/rocketmq-go/model/consumer_running_info.go @@ -17,12 +17,15 @@ limitations under the License. package model -import "encoding/json" +import ( + "encoding/json" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" +) //ConsumerRunningInfo this client's consumer running info type ConsumerRunningInfo struct { - Properties map[string]string `json:"properties"` - MqTable map[MessageQueue]ProcessQueueInfo `json:"mqTable"` + Properties map[string]string `json:"properties"` + MqTable map[rocketmqm.MessageQueue]ProcessQueueInfo `json:"mqTable"` } //Encode ConsumerRunningInfo to byte array diff --git a/rocketmq-go/model/pull_request.go b/rocketmq-go/model/pull_request.go index 7638eb4ea..a5292808f 100644 --- a/rocketmq-go/model/pull_request.go +++ b/rocketmq-go/model/pull_request.go @@ -17,12 +17,14 @@ limitations under the License. package model +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + //PullRequest pullRequest type PullRequest struct { //consumer group ConsumerGroup string //which message queue - MessageQueue *MessageQueue + MessageQueue *rocketmqm.MessageQueue //process queue info ProcessQueue *ProcessQueue //next offset diff --git a/rocketmq-go/model/reset_offset_body.go b/rocketmq-go/model/reset_offset_body.go index 5b1fe6937..1747b2b01 100644 --- a/rocketmq-go/model/reset_offset_body.go +++ b/rocketmq-go/model/reset_offset_body.go @@ -19,18 +19,19 @@ package model import ( "encoding/json" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" "github.com/golang/glog" ) //ResetOffsetBody resetOffsetBody type ResetOffsetBody struct { - OffsetTable map[MessageQueue]int64 `json:"offsetTable"` + OffsetTable map[rocketmqm.MessageQueue]int64 `json:"offsetTable"` } //Decode decode byte array to ResetOffsetBody func (r *ResetOffsetBody) Decode(data []byte) (err error) { - r.OffsetTable = map[MessageQueue]int64{} + r.OffsetTable = map[rocketmqm.MessageQueue]int64{} var kvMap map[string]string kvMap, err = util.GetKvStringMap(string(data)) if err != nil { @@ -42,7 +43,7 @@ func (r *ResetOffsetBody) Decode(data []byte) (err error) { return } for k, v := range kvMap { - messageQueue := &MessageQueue{} + messageQueue := &rocketmqm.MessageQueue{} var offset int64 err = json.Unmarshal([]byte(k), messageQueue) if err != nil { diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go index a5c7d5865..dfeedde58 100644 --- a/rocketmq-go/model/send_result.go +++ b/rocketmq-go/model/send_result.go @@ -17,6 +17,8 @@ limitations under the License. package model +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + //SendStatus message send result type SendStatus int @@ -35,7 +37,7 @@ const ( type SendResult struct { sendStatus SendStatus msgID string - messageQueue MessageQueue + messageQueue rocketmqm.MessageQueue queueOffset int64 transactionID string offsetMsgID string @@ -71,11 +73,11 @@ func (result *SendResult) SetSendStatus(status SendStatus) { result.sendStatus = status } -func (result *SendResult) MessageQueue() MessageQueue { +func (result *SendResult) MessageQueue() rocketmqm.MessageQueue { return result.messageQueue } -func (result *SendResult) SetMessageQueue(queue MessageQueue) { +func (result *SendResult) SetMessageQueue(queue rocketmqm.MessageQueue) { result.messageQueue = queue } diff --git a/rocketmq-go/model/topic_publish_info.go b/rocketmq-go/model/topic_publish_info.go index cc21c41df..6d833c95d 100644 --- a/rocketmq-go/model/topic_publish_info.go +++ b/rocketmq-go/model/topic_publish_info.go @@ -18,6 +18,7 @@ limitations under the License. package model import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "sync/atomic" ) @@ -26,7 +27,7 @@ import ( type TopicPublishInfo struct { OrderTopic bool HaveTopicRouterInfo bool - MessageQueueList []MessageQueue + MessageQueueList []rocketmqm.MessageQueue TopicRouteDataInstance *TopicRouteData topicQueueIndex int32 } @@ -49,15 +50,15 @@ func (t *TopicPublishInfo) FetchQueueIndex() (index int) { } //BuildTopicSubscribeInfoFromRoteData BuildTopicSubscribeInfoFromRoteData -func BuildTopicSubscribeInfoFromRoteData(topic string, topicRouteData *TopicRouteData) (mqList []*MessageQueue) { - mqList = make([]*MessageQueue, 0) +func BuildTopicSubscribeInfoFromRoteData(topic string, topicRouteData *TopicRouteData) (mqList []*rocketmqm.MessageQueue) { + mqList = make([]*rocketmqm.MessageQueue, 0) for _, queueData := range topicRouteData.QueueDatas { if !constant.ReadAble(queueData.Perm) { continue } var i int32 for i = 0; i < queueData.ReadQueueNums; i++ { - mq := &MessageQueue{ + mq := &rocketmqm.MessageQueue{ Topic: topic, BrokerName: queueData.BrokerName, QueueId: i, @@ -73,7 +74,7 @@ func BuildTopicPublishInfoFromTopicRoteData(topic string, topicRouteData *TopicR topicPublishInfo = &TopicPublishInfo{ TopicRouteDataInstance: topicRouteData, OrderTopic: false, - MessageQueueList: []MessageQueue{}} + MessageQueueList: []rocketmqm.MessageQueue{}} for _, queueData := range topicRouteData.QueueDatas { if !constant.WriteAble(queueData.Perm) { continue @@ -85,7 +86,7 @@ func BuildTopicPublishInfoFromTopicRoteData(topic string, topicRouteData *TopicR } var i int32 for i = 0; i < queueData.WriteQueueNums; i++ { - messageQueue := MessageQueue{Topic: topic, BrokerName: queueData.BrokerName, QueueId: i} + messageQueue := rocketmqm.MessageQueue{Topic: topic, BrokerName: queueData.BrokerName, QueueId: i} topicPublishInfo.MessageQueueList = append(topicPublishInfo.MessageQueueList, messageQueue) topicPublishInfo.HaveTopicRouterInfo = true } From f2d450b672f755ac4a0f2c59c9467205324ff6b7 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 16:30:45 +0800 Subject: [PATCH 70/88] change api --- rocketmq-go/api/model/message_queue.go | 40 --------- rocketmq-go/api/model/send_result.go | 2 +- rocketmq-go/api/rocketmq_producer.go | 5 +- rocketmq-go/kernel/mq_producer.go | 5 +- rocketmq-go/kernel/producer_service.go | 22 ++--- rocketmq-go/kernel/rebalance.go | 2 +- rocketmq-go/model/message_queues.go | 35 ++++++++ rocketmq-go/model/send_result.go | 113 ------------------------- 8 files changed, 52 insertions(+), 172 deletions(-) create mode 100644 rocketmq-go/model/message_queues.go delete mode 100644 rocketmq-go/model/send_result.go diff --git a/rocketmq-go/api/model/message_queue.go b/rocketmq-go/api/model/message_queue.go index d1858060d..9b944dc05 100644 --- a/rocketmq-go/api/model/message_queue.go +++ b/rocketmq-go/api/model/message_queue.go @@ -24,46 +24,6 @@ type MessageQueue struct { QueueId int32 `json:"queueId"` } -func (m *MessageQueue) clone() *MessageQueue { - no := new(MessageQueue) - no.Topic = m.Topic - no.QueueId = m.QueueId - no.BrokerName = m.BrokerName - return no -} - -//MessageQueues queue array -type MessageQueues []*MessageQueue - -//Less compare queue -func (m MessageQueues) Less(i, j int) bool { - imq := m[i] - jmq := m[j] - - if imq.Topic < jmq.Topic { - return true - } else if imq.Topic < jmq.Topic { - return false - } - - if imq.BrokerName < jmq.BrokerName { - return true - } else if imq.BrokerName < jmq.BrokerName { - return false - } - return imq.QueueId < jmq.QueueId -} - -//Swap swap queue -func (m MessageQueues) Swap(i, j int) { - m[i], m[j] = m[j], m[i] -} - -//Len messageQueues's length -func (m MessageQueues) Len() int { - return len(m) -} - //Equals judge messageQueue is the same func (m MessageQueue) Equals(messageQueue *MessageQueue) bool { if m.QueueId != messageQueue.QueueId { diff --git a/rocketmq-go/api/model/send_result.go b/rocketmq-go/api/model/send_result.go index 71e710481..753ec59ab 100644 --- a/rocketmq-go/api/model/send_result.go +++ b/rocketmq-go/api/model/send_result.go @@ -109,6 +109,6 @@ func (result *SendResult) SetOffsetMsgID(s string) { } func (result *SendResult) String() string { - return fmt.Sprintf("SendResult [sendStatus=%s, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", + return fmt.Sprintf("SendResult [sendStatus=%d, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) } diff --git a/rocketmq-go/api/rocketmq_producer.go b/rocketmq-go/api/rocketmq_producer.go index 47de28b30..b95c703e0 100644 --- a/rocketmq-go/api/rocketmq_producer.go +++ b/rocketmq-go/api/rocketmq_producer.go @@ -20,15 +20,14 @@ package rocketmq import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" ) //MQProducer rocketmq producer type MQProducer interface { //send message,default timeout is 3000 - Send(message rocketmqm.Message) (sendResult *model.SendResult, err error) + Send(message rocketmqm.Message) (sendResult *rocketmqm.SendResult, err error) //send message with custom timeout - SendWithTimeout(message rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) + SendWithTimeout(message rocketmqm.Message, timeout int64) (sendResult *rocketmqm.SendResult, err error) } //NewDefaultMQProducer mq producer with default config diff --git a/rocketmq-go/kernel/mq_producer.go b/rocketmq-go/kernel/mq_producer.go index 2848e9231..185642392 100644 --- a/rocketmq-go/kernel/mq_producer.go +++ b/rocketmq-go/kernel/mq_producer.go @@ -20,7 +20,6 @@ package kernel import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) @@ -40,11 +39,11 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProd return } -func (d *DefaultMQProducer) Send(msg rocketmqm.Message) (sendResult *model.SendResult, err error) { +func (d *DefaultMQProducer) Send(msg rocketmqm.Message) (sendResult *rocketmqm.SendResult, err error) { sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", d.ProducerConfig.SendMsgTimeout) return } -func (d *DefaultMQProducer) SendWithTimeout(msg rocketmqm.Message, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultMQProducer) SendWithTimeout(msg rocketmqm.Message, timeout int64) (sendResult *rocketmqm.SendResult, err error) { sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", timeout) return } diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 3caccf281..5711732b7 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -32,7 +32,7 @@ import ( //ProducerService producerService, for send message type ProducerService interface { checkConfig() (err error) - sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) + sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *rocketmqm.SendResult, err error) } //ProducerService ProducerService's implement @@ -57,7 +57,7 @@ func (d *DefaultProducerService) checkConfig() (err error) { return } -func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *rocketmqm.SendResult, err error) { var ( topicPublishInfo *model.TopicPublishInfo ) @@ -79,7 +79,7 @@ func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, c return } -func (d *DefaultProducerService) producerSendMessageRequest(brokerName, brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) producerSendMessageRequest(brokerName, brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *rocketmqm.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body()) var response *remoting.RemotingCommand response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) @@ -91,27 +91,27 @@ func (d *DefaultProducerService) producerSendMessageRequest(brokerName, brokerAd sendResult, err = processSendResponse(brokerName, message, response) return } -func processSendResponse(brokerName string, message *message.MessageImpl, response *remoting.RemotingCommand) (sendResult *model.SendResult, err error) { - sendResult = &model.SendResult{} +func processSendResponse(brokerName string, message *message.MessageImpl, response *remoting.RemotingCommand) (sendResult *rocketmqm.SendResult, err error) { + sendResult = &rocketmqm.SendResult{} switch response.Code { case remoting.FLUSH_DISK_TIMEOUT: { - sendResult.SetSendStatus(model.FlushDiskTimeout) + sendResult.SetSendStatus(rocketmqm.FlushDiskTimeout) break } case remoting.FLUSH_SLAVE_TIMEOUT: { - sendResult.SetSendStatus(model.FlushSlaveTimeout) + sendResult.SetSendStatus(rocketmqm.FlushSlaveTimeout) break } case remoting.SLAVE_NOT_AVAILABLE: { - sendResult.SetSendStatus(model.SlaveNotAvaliable) + sendResult.SetSendStatus(rocketmqm.SlaveNotAvaliable) break } case remoting.SUCCESS: { - sendResult.SetSendStatus(model.SendOK) + sendResult.SetSendStatus(rocketmqm.SendOK) break } default: @@ -171,7 +171,7 @@ func (d *DefaultProducerService) checkMessage(message *message.MessageImpl) (err return } -func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *model.SendResult, err error) { +func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.MessageImpl, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, timeout int64) (sendResult *rocketmqm.SendResult, err error) { var ( sendTotalTime int messageQueue rocketmqm.MessageQueue @@ -198,7 +198,7 @@ func (d *DefaultProducerService) sendMsgUseTopicPublishInfo(message *message.Mes func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, messageQueue rocketmqm.MessageQueue, communicationMode string, sendCallback string, topicPublishInfo *model.TopicPublishInfo, - timeout int64) (sendResult *model.SendResult, err error) { + timeout int64) (sendResult *rocketmqm.SendResult, err error) { var ( brokerAddr string sysFlag int diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index c135717bf..b2f0faaba 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -164,7 +164,7 @@ func (r *rebalance) rebalanceByTopic(topic string) error { mqs := r.mqClient.GetTopicSubscribeInfo(topic) r.topicSubscribeInfoTableLock.RUnlock() if len(mqs) > 0 && len(cidAll) > 0 { - var messageQueues rocketmqm.MessageQueues = mqs + var messageQueues model.MessageQueues = mqs var consumerIdSorter consumerIdSorter = cidAll sort.Sort(messageQueues) diff --git a/rocketmq-go/model/message_queues.go b/rocketmq-go/model/message_queues.go new file mode 100644 index 000000000..790d0b745 --- /dev/null +++ b/rocketmq-go/model/message_queues.go @@ -0,0 +1,35 @@ +package model + +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + +//MessageQueues queue array +type MessageQueues []*rocketmqm.MessageQueue + +//Less compare queue +func (m MessageQueues) Less(i, j int) bool { + imq := m[i] + jmq := m[j] + + if imq.Topic < jmq.Topic { + return true + } else if imq.Topic < jmq.Topic { + return false + } + + if imq.BrokerName < jmq.BrokerName { + return true + } else if imq.BrokerName < jmq.BrokerName { + return false + } + return imq.QueueId < jmq.QueueId +} + +//Swap swap queue +func (m MessageQueues) Swap(i, j int) { + m[i], m[j] = m[j], m[i] +} + +//Len messageQueues's length +func (m MessageQueues) Len() int { + return len(m) +} diff --git a/rocketmq-go/model/send_result.go b/rocketmq-go/model/send_result.go deleted file mode 100644 index dfeedde58..000000000 --- a/rocketmq-go/model/send_result.go +++ /dev/null @@ -1,113 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package model - -import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - -//SendStatus message send result -type SendStatus int - -const ( - //SendOK message send success - SendOK SendStatus = iota - //FlushDiskTimeout FlushDiskTimeout - FlushDiskTimeout - //FlushSlaveTimeout FlushSlaveTimeout - FlushSlaveTimeout - //SlaveNotAvaliable SlaveNotAvaliable - SlaveNotAvaliable -) - -//SendResult SendResult -type SendResult struct { - sendStatus SendStatus - msgID string - messageQueue rocketmqm.MessageQueue - queueOffset int64 - transactionID string - offsetMsgID string - regionID string - traceOn bool -} - -func (result *SendResult) TraceOn() bool { - return result.traceOn -} - -func (result *SendResult) SetTraceOn(b bool) { - result.traceOn = b -} - -func (result *SendResult) SetRegionID(s string) { - result.regionID = s -} - -func (result *SendResult) MsgID() string { - return result.msgID -} - -func (result *SendResult) SetMsgID(s string) { - result.msgID = s -} - -func (result *SendResult) SendStatus() SendStatus { - return result.sendStatus -} - -func (result *SendResult) SetSendStatus(status SendStatus) { - result.sendStatus = status -} - -func (result *SendResult) MessageQueue() rocketmqm.MessageQueue { - return result.messageQueue -} - -func (result *SendResult) SetMessageQueue(queue rocketmqm.MessageQueue) { - result.messageQueue = queue -} - -func (result *SendResult) QueueOffset() int64 { - return result.queueOffset -} - -func (result *SendResult) SetQueueOffset(offset int64) { - result.queueOffset = offset -} - -func (result *SendResult) TransactionID() string { - return result.transactionID -} - -func (result *SendResult) SetTransactionID(s string) { - result.transactionID = s -} - -//OffsetMsgID OffsetMsgID -func (result *SendResult) OffsetMsgID() string { - return result.offsetMsgID -} - -//SetOffsetMsgID SetOffsetMsgID -func (result *SendResult) SetOffsetMsgID(s string) { - result.offsetMsgID = s -} - -//func (result *SendResult) String() string { -// return fmt.Sprintf("SendResult [sendStatus=%s, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", -// result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) -//} From 43db1c1595a003f36f23d56acdf900b6eea48d79 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 16:51:18 +0800 Subject: [PATCH 71/88] rocketmq send message result --- rocketmq-go/api/model/send_result.go | 20 +++++++++++-- .../example/simple_producer_consumer.go | 2 +- .../header/send_message_response_header.go | 28 ++++++++++++++++++- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/rocketmq-go/api/model/send_result.go b/rocketmq-go/api/model/send_result.go index 753ec59ab..d3d77fbc1 100644 --- a/rocketmq-go/api/model/send_result.go +++ b/rocketmq-go/api/model/send_result.go @@ -35,7 +35,7 @@ const ( SlaveNotAvaliable ) -//SendResult SendResult +//SendResult rocketmq send result type SendResult struct { sendStatus SendStatus msgID string @@ -47,67 +47,81 @@ type SendResult struct { traceOn bool } +//TraceOn TraceOn func (result *SendResult) TraceOn() bool { return result.traceOn } +//SetTraceOn SetTraceOn func (result *SendResult) SetTraceOn(b bool) { result.traceOn = b } +//SetRegionID SetRegionID func (result *SendResult) SetRegionID(s string) { result.regionID = s } +//MsgID get rocketmq message id func (result *SendResult) MsgID() string { return result.msgID } +//SetMsgID set rocketmq message id func (result *SendResult) SetMsgID(s string) { result.msgID = s } +//SendStatus SendStatus func (result *SendResult) SendStatus() SendStatus { return result.sendStatus } +//SetSendStatus SetSendStatus func (result *SendResult) SetSendStatus(status SendStatus) { result.sendStatus = status } +//MessageQueue this message send to which message queue func (result *SendResult) MessageQueue() MessageQueue { return result.messageQueue } +//SetMessageQueue SetMessageQueue func (result *SendResult) SetMessageQueue(queue MessageQueue) { result.messageQueue = queue } +//QueueOffset this message in this message queue's offset func (result *SendResult) QueueOffset() int64 { return result.queueOffset } +//SetQueueOffset SetQueueOffset func (result *SendResult) SetQueueOffset(offset int64) { result.queueOffset = offset } +//TransactionID no use,because not support transaction message func (result *SendResult) TransactionID() string { return result.transactionID } +//SetTransactionID no use,because not support transaction message func (result *SendResult) SetTransactionID(s string) { result.transactionID = s } -//OffsetMsgID get +//OffsetMsgID OffsetMsgID func (result *SendResult) OffsetMsgID() string { return result.offsetMsgID } +//SetOffsetMsgID SetOffsetMsgID func (result *SendResult) SetOffsetMsgID(s string) { result.offsetMsgID = s } - +//SendResult send message result to string(detail result) func (result *SendResult) String() string { return fmt.Sprintf("SendResult [sendStatus=%d, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 9de07e0dc..052d7c6c9 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -61,6 +61,6 @@ func main() { message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult result=[%s] err=[%s]", result.String(), err) } } diff --git a/rocketmq-go/kernel/header/send_message_response_header.go b/rocketmq-go/kernel/header/send_message_response_header.go index a46820ac0..32fd1eb01 100644 --- a/rocketmq-go/kernel/header/send_message_response_header.go +++ b/rocketmq-go/kernel/header/send_message_response_header.go @@ -17,6 +17,8 @@ limitations under the License. package header +import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/util" + //SendMessageResponseHeader of CustomerHeader type SendMessageResponseHeader struct { MsgId string @@ -27,6 +29,30 @@ type SendMessageResponseHeader struct { } //FromMap convert map[string]interface to struct -func (header *SendMessageResponseHeader) FromMap(headerMap map[string]interface{}) { +func (s *SendMessageResponseHeader) FromMap(headerMap map[string]interface{}) { + s.MsgId = headerMap["msgId"].(string) + s.QueueId = util.StrToInt32WithDefaultValue(headerMap["queueId"].(string), -1) + s.QueueOffset = util.StrToInt64WithDefaultValue(headerMap["queueOffset"].(string), -1) + transactionId := headerMap["transactionId"] + if transactionId != nil { + s.TransactionId = headerMap["transactionId"].(string) + } + msgRegion := headerMap["MSG_REGION"] + if msgRegion != nil { + s.MsgRegion = headerMap["MSG_REGION"].(string) + } + return } + +//for example + +/** +{ + "MSG_REGION": "DefaultRegion", + "TRACE_ON": "true", + "msgId": "C0A8000200002A9F0000000039FA93B5", + "queueId": "3", + "queueOffset": "1254671" +} +*/ From 80665f361be50d0dfffa7144e6e56baac4fe0057 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 23:26:41 +0800 Subject: [PATCH 72/88] golint --- rocketmq-go/api/model/send_result.go | 5 +- .../allocate/allocate_message_averagely.go | 6 +- .../allocate_message_by_machine_room.go | 8 +- .../allocate_message_queue_strategy.go | 4 +- rocketmq-go/kernel/consume_message_service.go | 2 +- rocketmq-go/kernel/mq_client.go | 122 ++++++++---------- rocketmq-go/kernel/mq_client_manage.go | 26 ++-- rocketmq-go/kernel/mq_producer.go | 6 + rocketmq-go/kernel/mq_push_consumer.go | 2 +- rocketmq-go/kernel/offset_store.go | 14 +- rocketmq-go/kernel/producer_service.go | 8 +- .../kernel/producer_service_for_send_back.go | 4 +- rocketmq-go/kernel/pull_message_controller.go | 31 ++--- rocketmq-go/kernel/rebalance.go | 18 +-- rocketmq-go/kernel/tasks.go | 4 +- rocketmq-go/model/message/message_impl.go | 7 - rocketmq-go/remoting/rocketmq_serializable.go | 12 +- rocketmq-go/util/json_util.go | 1 + rocketmq-go/util/map_util.go | 2 +- 19 files changed, 141 insertions(+), 141 deletions(-) diff --git a/rocketmq-go/api/model/send_result.go b/rocketmq-go/api/model/send_result.go index d3d77fbc1..a817f575e 100644 --- a/rocketmq-go/api/model/send_result.go +++ b/rocketmq-go/api/model/send_result.go @@ -121,8 +121,9 @@ func (result *SendResult) OffsetMsgID() string { func (result *SendResult) SetOffsetMsgID(s string) { result.offsetMsgID = s } + //SendResult send message result to string(detail result) func (result *SendResult) String() string { - return fmt.Sprintf("SendResult [sendStatus=%d, msgId=%s, offsetMsgId=%s, messageQueue=%s, queueOffset=%s]", - result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue, result.queueOffset) + return fmt.Sprintf("SendResult [sendStatus=%d, msgId=%s, offsetMsgId=%s, messageQueue.BrokerName=%s, messageQueue.QueueId=%d, queueOffset=%d]", + result.sendStatus, result.msgID, result.offsetMsgID, result.messageQueue.BrokerName, result.messageQueue.QueueId, result.queueOffset) } diff --git a/rocketmq-go/kernel/allocate/allocate_message_averagely.go b/rocketmq-go/kernel/allocate/allocate_message_averagely.go index 588d8f9ee..48cc050d4 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_averagely.go +++ b/rocketmq-go/kernel/allocate/allocate_message_averagely.go @@ -22,11 +22,11 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) -//AllocateMessageQueueAveragely AllocateMessageQueueAveragely -type AllocateMessageQueueAveragely struct{} +//allocateMessageQueueAveragely allocateMessageQueueAveragely +type allocateMessageQueueAveragely struct{} //Allocate message queue -func (a *AllocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { +func (a *allocateMessageQueueAveragely) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") diff --git a/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go index 8ccc4c6b7..cae3cd8c7 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go +++ b/rocketmq-go/kernel/allocate/allocate_message_by_machine_room.go @@ -22,12 +22,12 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" ) -//AllocateMessageQueueByMachineRoom AllocateMessageQueueByMachineRoom -type AllocateMessageQueueByMachineRoom struct { +//allocateMessageQueueByMachineRoom allocateMessageQueueByMachineRoom +type allocateMessageQueueByMachineRoom struct { } -//Allocate message queue -func (a *AllocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { +//Allocate Allocate message queue by machine room +func (a *allocateMessageQueueByMachineRoom) Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) { if currentCID == "" { return nil, errors.New("currentCID is empty") } diff --git a/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go index d5123aa53..10bb6a796 100644 --- a/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go +++ b/rocketmq-go/kernel/allocate/allocate_message_queue_strategy.go @@ -19,12 +19,12 @@ package allocate import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" -//Allocate message queue +//AllocateMessageQueueStrategy Allocate message queue type AllocateMessageQueueStrategy interface { Allocate(consumerGroup string, currentCID string, mqAll []*rocketmqm.MessageQueue, cidAll []string) ([]rocketmqm.MessageQueue, error) } //GetAllocateMessageQueueStrategyByConfig get allocate message queue strategy by config func GetAllocateMessageQueueStrategyByConfig(allocateMessageQueueStrategy string) AllocateMessageQueueStrategy { - return new(AllocateMessageQueueAveragely) + return new(allocateMessageQueueAveragely) } diff --git a/rocketmq-go/kernel/consume_message_service.go b/rocketmq-go/kernel/consume_message_service.go index 4f9703b9c..f34991ebd 100644 --- a/rocketmq-go/kernel/consume_message_service.go +++ b/rocketmq-go/kernel/consume_message_service.go @@ -42,7 +42,7 @@ type consumeMessageConcurrentlyServiceImpl struct { consumerConfig *rocketmqm.MqConsumerConfig } -func NewConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageListener) (consumeService consumeMessageService) { +func newConsumeMessageConcurrentlyServiceImpl(messageListener rocketmqm.MessageListener) (consumeService consumeMessageService) { consumeService = &consumeMessageConcurrentlyServiceImpl{messageListener: messageListener, sendMessageBackProducerService: &sendMessageBackProducerServiceImpl{}} return } diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index fb37237e8..02c49c56e 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -34,48 +34,38 @@ import ( "time" ) -//this struct is for something common,for example +//RocketMqClient this struct is for something common,for example //1.brokerInfo //2.routerInfo //3.subscribeInfo //4.heartbeat type RocketMqClient interface { //get mqClient's clientId ip@pid - GetClientId() (clientId string) + getClientId() (clientId string) //get remoting client in mqClient - GetRemotingClient() (remotingClient *remoting.DefaultRemotingClient) + getRemotingClient() (remotingClient *remoting.DefaultRemotingClient) //get topic subscribe info - GetTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) - //GetPublishTopicList - GetPublishTopicList() []string - //FetchMasterBrokerAddress - FetchMasterBrokerAddress(brokerName string) (masterAddress string) - //EnqueuePullMessageRequest - EnqueuePullMessageRequest(pullRequest *model.PullRequest) - //DequeuePullMessageRequest - DequeuePullMessageRequest() (pullRequest *model.PullRequest) - //FindBrokerAddressInSubscribe - FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) - //TryToFindTopicPublishInfo - TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) - //FindBrokerAddrByTopic - FindBrokerAddrByTopic(topic string) (addr string, ok bool) - //UpdateTopicRouteInfoFromNameServer - UpdateTopicRouteInfoFromNameServer(topic string) (err error) - //UpdateTopicRouteInfoFromNameServerUseDefaultTopic - UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) - //SendHeartbeatToAllBroker - SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) - //ClearExpireResponse - ClearExpireResponse() - //GetMaxOffset - GetMaxOffset(mq *rocketmqm.MessageQueue) int64 - //SearchOffset - SearchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 + getTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) + //getPublishTopicList + getPublishTopicList() []string + fetchMasterBrokerAddress(brokerName string) (masterAddress string) + enqueuePullMessageRequest(pullRequest *model.PullRequest) + dequeuePullMessageRequest() (pullRequest *model.PullRequest) + findBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) + tryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) + findBrokerAddrByTopic(topic string) (addr string, ok bool) + updateTopicRouteInfoFromNameServer(topic string) (err error) + updateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) + sendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) + clearExpireResponse() + getMaxOffset(mq *rocketmqm.MessageQueue) int64 + searchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 } -var DEFAULT_TIMEOUT int64 = 6000 +//DEFAULT_TIMEOUT rocketmq client's default timeout +var DEFAULT_TIMEOUT int64 = 3000 +//MqClientImpl RocketMqClient type MqClientImpl struct { clientId string remotingClient *remoting.DefaultRemotingClient @@ -86,8 +76,8 @@ type MqClientImpl struct { pullRequestQueue chan *model.PullRequest } -// create a MqClientInit instance -func MqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { +//mqClientInit create a mqClientInit instance +func mqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} mqClientImpl.clientId = buildMqClientImplId() mqClientImpl.topicRouteTable = util.New() // make(map[string]*model.TopicRouteData) @@ -98,18 +88,20 @@ func MqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor mqClientImpl.pullRequestQueue = make(chan *model.PullRequest, 1024) return } -func (m *MqClientImpl) GetTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) { + +//getTopicSubscribeInfo +func (m *MqClientImpl) getTopicSubscribeInfo(topic string) (messageQueueList []*rocketmqm.MessageQueue) { value, ok := m.topicSubscribeInfoTable.Get(topic) if ok { messageQueueList = value.([]*rocketmqm.MessageQueue) } return } -func (m *MqClientImpl) GetMaxOffset(mq *rocketmqm.MessageQueue) int64 { - brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) +func (m *MqClientImpl) getMaxOffset(mq *rocketmqm.MessageQueue) int64 { + brokerAddr := m.fetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - m.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = m.FetchMasterBrokerAddress(mq.BrokerName) + m.tryToFindTopicPublishInfo(mq.Topic) + brokerAddr = m.fetchMasterBrokerAddress(mq.BrokerName) } getMaxOffsetRequestHeader := &header.GetMaxOffsetRequestHeader{Topic: mq.Topic, QueueId: mq.QueueId} remotingCmd := remoting.NewRemotingCommand(remoting.GET_MAX_OFFSET, getMaxOffsetRequestHeader) @@ -121,11 +113,11 @@ func (m *MqClientImpl) GetMaxOffset(mq *rocketmqm.MessageQueue) int64 { queryOffsetResponseHeader.FromMap(response.ExtFields) return queryOffsetResponseHeader.Offset } -func (m *MqClientImpl) SearchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 { - brokerAddr := m.FetchMasterBrokerAddress(mq.BrokerName) +func (m *MqClientImpl) searchOffset(mq *rocketmqm.MessageQueue, time time.Time) int64 { + brokerAddr := m.fetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - m.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = m.FetchMasterBrokerAddress(mq.BrokerName) + m.tryToFindTopicPublishInfo(mq.Topic) + brokerAddr = m.fetchMasterBrokerAddress(mq.BrokerName) } timeStamp := util.CurrentTimeMillisInt64() searchOffsetRequestHeader := &header.SearchOffsetRequestHeader{Topic: mq.Topic, QueueId: mq.QueueId, Timestamp: timeStamp} @@ -139,40 +131,40 @@ func (m *MqClientImpl) SearchOffset(mq *rocketmqm.MessageQueue, time time.Time) queryOffsetResponseHeader.FromMap(response.ExtFields) return queryOffsetResponseHeader.Offset } -func (m *MqClientImpl) GetClientId() string { +func (m *MqClientImpl) getClientId() string { return m.clientId } -func (m *MqClientImpl) GetPublishTopicList() []string { +func (m *MqClientImpl) getPublishTopicList() []string { var publishTopicList []string for _, topic := range m.topicPublishInfoTable.Keys() { publishTopicList = append(publishTopicList, topic) } return publishTopicList } -func (m *MqClientImpl) GetRemotingClient() *remoting.DefaultRemotingClient { +func (m *MqClientImpl) getRemotingClient() *remoting.DefaultRemotingClient { return m.remotingClient } -func (m *MqClientImpl) EnqueuePullMessageRequest(pullRequest *model.PullRequest) { +func (m *MqClientImpl) enqueuePullMessageRequest(pullRequest *model.PullRequest) { m.pullRequestQueue <- pullRequest } -func (m *MqClientImpl) DequeuePullMessageRequest() (pullRequest *model.PullRequest) { +func (m *MqClientImpl) dequeuePullMessageRequest() (pullRequest *model.PullRequest) { pullRequest = <-m.pullRequestQueue return } -func (m *MqClientImpl) ClearExpireResponse() { +func (m *MqClientImpl) clearExpireResponse() { m.remotingClient.ClearExpireResponse() } -func (m *MqClientImpl) FetchMasterBrokerAddress(brokerName string) (masterAddress string) { +func (m *MqClientImpl) fetchMasterBrokerAddress(brokerName string) (masterAddress string) { value, ok := m.brokerAddrTable.Get(brokerName) if ok { masterAddress = value.(map[string]string)["0"] } return } -func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) { +func (m *MqClientImpl) tryToFindTopicPublishInfo(topic string) (topicPublicInfo *model.TopicPublishInfo, err error) { value, ok := m.topicPublishInfoTable.Get(topic) if ok { topicPublicInfo = value.(*model.TopicPublishInfo) @@ -180,7 +172,7 @@ func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo if topicPublicInfo == nil || !topicPublicInfo.JudgeTopicPublishInfoOk() { m.topicPublishInfoTable.Set(topic, &model.TopicPublishInfo{HaveTopicRouterInfo: false}) - err = m.UpdateTopicRouteInfoFromNameServer(topic) + err = m.updateTopicRouteInfoFromNameServer(topic) if err != nil { glog.Warning(err) // if updateRouteInfo error, maybe we can use the defaultTopic } @@ -193,7 +185,7 @@ func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo return } //try to use the defaultTopic - err = m.UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic) + err = m.updateTopicRouteInfoFromNameServerUseDefaultTopic(topic) defaultValue, defaultValueOk := m.topicPublishInfoTable.Get(topic) if defaultValueOk { @@ -203,7 +195,7 @@ func (m *MqClientImpl) TryToFindTopicPublishInfo(topic string) (topicPublicInfo return } -func (m MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMillis int64) (*model.TopicRouteData, error) { +func (m MqClientImpl) getTopicRouteInfoFromNameServer(topic string, timeoutMillis int64) (*model.TopicRouteData, error) { requestHeader := &header.GetRouteInfoRequestHeader{ Topic: topic, } @@ -231,7 +223,7 @@ func (m MqClientImpl) GetTopicRouteInfoFromNameServer(topic string, timeoutMilli } } -func (m MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { +func (m MqClientImpl) findBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { slave = false found = false value, ok := m.brokerAddrTable.Get(brokerName) @@ -254,24 +246,24 @@ func (m MqClientImpl) FindBrokerAddressInSubscribe(brokerName string, brokerId i return } -func (m MqClientImpl) UpdateTopicRouteInfoFromNameServer(topic string) (err error) { +func (m MqClientImpl) updateTopicRouteInfoFromNameServer(topic string) (err error) { var ( topicRouteData *model.TopicRouteData ) //namesvr lock - topicRouteData, err = m.GetTopicRouteInfoFromNameServer(topic, 1000*3) + topicRouteData, err = m.getTopicRouteInfoFromNameServer(topic, DEFAULT_TIMEOUT) if err != nil { return } m.updateTopicRouteInfoLocal(topic, topicRouteData) return } -func (m MqClientImpl) UpdateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) { +func (m MqClientImpl) updateTopicRouteInfoFromNameServerUseDefaultTopic(topic string) (err error) { var ( topicRouteData *model.TopicRouteData ) //namesvr lock - topicRouteData, err = m.GetTopicRouteInfoFromNameServer(constant.DEFAULT_TOPIC, 1000*3) + topicRouteData, err = m.getTopicRouteInfoFromNameServer(constant.DEFAULT_TOPIC, DEFAULT_TIMEOUT) if err != nil { return } @@ -292,10 +284,10 @@ func (m MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData *mo return } // topicRouteData judgeTopicRouteData need update - needUpdate := true - if !needUpdate { - return - } + //needUpdate := true + //if !needUpdate { + // return + //} //update brokerAddrTable for _, brokerData := range topicRouteData.BrokerDatas { m.brokerAddrTable.Set(brokerData.BrokerName, brokerData.BrokerAddrs) @@ -311,7 +303,7 @@ func (m MqClientImpl) updateTopicRouteInfoLocal(topic string, topicRouteData *mo return } -func (m MqClientImpl) FindBrokerAddrByTopic(topic string) (addr string, ok bool) { +func (m MqClientImpl) findBrokerAddrByTopic(topic string) (addr string, ok bool) { value, findValue := m.topicRouteTable.Get(topic) if !findValue { return "", false @@ -351,7 +343,7 @@ func (m MqClientImpl) sendHeartBeat(addr string, remotingCommand *remoting.Remot return err } -func (m MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) { +func (m MqClientImpl) sendHeartbeatToAllBroker(heartBeatData *model.HeartbeatData) (err error) { for _, brokerTable := range m.brokerAddrTable.Items() { for brokerId, addr := range brokerTable.(map[string]string) { if len(addr) == 0 || brokerId != "0" { @@ -365,7 +357,7 @@ func (m MqClientImpl) SendHeartbeatToAllBroker(heartBeatData *model.HeartbeatDat glog.V(2).Info("send heartbeat to broker look data[", string(data)+"]") remotingCommand := remoting.NewRemotingCommandWithBody(remoting.HEART_BEAT, nil, data) glog.V(2).Info("send heartbeat to broker[", addr+"]") - m.sendHeartBeat(addr, remotingCommand, 3000) + m.sendHeartBeat(addr, remotingCommand, DEFAULT_TIMEOUT) } } diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 91a393078..3d57a16c9 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -32,6 +32,7 @@ import ( "time" ) +//MqClientManager MqClientManager type MqClientManager struct { //rocketMqManagerLock sync.Mutex BootTimestamp int64 @@ -43,28 +44,33 @@ type MqClientManager struct { defaultProducerService *DefaultProducerService } +//MqClientManagerInit create a MqClientManager instance func MqClientManagerInit(clientConfig *rocketmqm.MqClientConfig) (rocketMqManager *MqClientManager) { rocketMqManager = &MqClientManager{} rocketMqManager.BootTimestamp = time.Now().Unix() - rocketMqManager.clientFactory = ClientFactoryInit() - rocketMqManager.mqClient = MqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo - rocketMqManager.pullMessageController = NewPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) + rocketMqManager.clientFactory = clientFactoryInit() + rocketMqManager.mqClient = mqClientInit(clientConfig, rocketMqManager.initClientRequestProcessor()) // todo todo todo + rocketMqManager.pullMessageController = newPullMessageController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.cleanExpireMsgController = newCleanExpireMsgController(rocketMqManager.mqClient, rocketMqManager.clientFactory) rocketMqManager.rebalanceControllr = newRebalanceController(rocketMqManager.clientFactory) return } + +//Start start MqClientManager func (m *MqClientManager) Start() { //d.sendHeartbeatToAllBrokerWithLock()//we should send heartbeat first todo check m.startAllScheduledTask() } +//RegisterProducer register producer to this MqClientManager func (m *MqClientManager) RegisterProducer(producer *DefaultMQProducer) { producer.producerService = newDefaultProducerService(producer.producerGroup, producer.ProducerConfig, m.mqClient) m.clientFactory.producerTable[producer.producerGroup] = producer return } +//RegisterConsumer register consumer to this MqClientManager func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { if m.defaultProducerService == nil { m.defaultProducerService = newDefaultProducerService(constant.CLIENT_INNER_PRODUCER_GROUP, rocketmqm.NewProducerConfig(), m.mqClient) @@ -114,7 +120,7 @@ func (m *MqClientManager) consumeMessageDirectly(cmd *remoting.RemotingCommand) var consumeMessageDirectlyResultRequestHeader = &header.ConsumeMessageDirectlyResultRequestHeader{} if cmd.ExtFields != nil { consumeMessageDirectlyResultRequestHeader.FromMap(cmd.ExtFields) - messageExt := &DecodeMessage(cmd.Body)[0] + messageExt := &decodeMessage(cmd.Body)[0] glog.V(2).Info("op=look", messageExt) defaultMQPushConsumer := m.clientFactory.consumerTable[consumeMessageDirectlyResultRequestHeader.ConsumerGroup] consumeResult, err := defaultMQPushConsumer.consumeMessageService.consumeMessageDirectly(messageExt, consumeMessageDirectlyResultRequestHeader.BrokerName) @@ -144,7 +150,7 @@ func (m *MqClientManager) getConsumerRunningInfo(cmd *remoting.RemotingCommand) consumerRunningInfo.Properties[key] = fmt.Sprintf("%v", value) } - consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.GetRemotingClient().GetNamesrvAddrList(), ";") + consumerRunningInfo.Properties["PROP_NAMESERVER_ADDR"] = strings.Join(defaultMQPushConsumer.mqClient.getRemotingClient().GetNamesrvAddrList(), ";") consumerRunningInfo.MqTable = defaultMQPushConsumer.rebalance.getMqTableInfo() glog.V(2).Info("op=look consumerRunningInfo", consumerRunningInfo) @@ -189,7 +195,7 @@ type clientFactory struct { consumerTable map[string]*DefaultMQPushConsumer //group|Consumer } -func ClientFactoryInit() (clientFactoryInstance *clientFactory) { +func clientFactoryInit() (clientFactoryInstance *clientFactory) { clientFactoryInstance = &clientFactory{} clientFactoryInstance.producerTable = make(map[string]*DefaultMQProducer) clientFactoryInstance.consumerTable = make(map[string]*DefaultMQPushConsumer) @@ -202,7 +208,7 @@ func (m *MqClientManager) sendHeartbeatToAllBrokerWithLock() error { if len(heartbeatData.ConsumerDataSet) == 0 { return errors.New("send heartbeat error") } - m.mqClient.SendHeartbeatToAllBroker(heartbeatData) + m.mqClient.sendHeartbeatToAllBroker(heartbeatData) return nil } @@ -214,16 +220,16 @@ func (m *MqClientManager) updateTopicRouteInfoFromNameServer() { topicSet = append(topicSet, key) } } - topicSet = append(topicSet, m.mqClient.GetPublishTopicList()...) + topicSet = append(topicSet, m.mqClient.getPublishTopicList()...) for _, topic := range topicSet { - m.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + m.mqClient.updateTopicRouteInfoFromNameServer(topic) } } func (m *MqClientManager) prepareHeartbeatData() *model.HeartbeatData { heartbeatData := new(model.HeartbeatData) - heartbeatData.ClientId = m.mqClient.GetClientId() + heartbeatData.ClientId = m.mqClient.getClientId() heartbeatData.ConsumerDataSet = make([]*model.ConsumerData, 0) heartbeatData.ProducerDataSet = make([]*model.ProducerData, 0) for group, consumer := range m.clientFactory.consumerTable { diff --git a/rocketmq-go/kernel/mq_producer.go b/rocketmq-go/kernel/mq_producer.go index 185642392..dafa712cf 100644 --- a/rocketmq-go/kernel/mq_producer.go +++ b/rocketmq-go/kernel/mq_producer.go @@ -24,6 +24,7 @@ import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" ) +//DefaultMQProducer use DefaultMQProducer send message type DefaultMQProducer struct { producerGroup string ProducerConfig *rocketmqm.MqProducerConfig @@ -31,6 +32,8 @@ type DefaultMQProducer struct { producerService ProducerService } +//NewDefaultMQProducer create a DefaultMQProducer instance +// see rocketmq_producer.go func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProducerConfig) (rocketMQProducer *DefaultMQProducer) { rocketMQProducer = &DefaultMQProducer{ producerGroup: producerGroup, @@ -39,10 +42,13 @@ func NewDefaultMQProducer(producerGroup string, producerConfig *rocketmqm.MqProd return } +//Send see rocketmq_producer.go func (d *DefaultMQProducer) Send(msg rocketmqm.Message) (sendResult *rocketmqm.SendResult, err error) { sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", d.ProducerConfig.SendMsgTimeout) return } + +//SendWithTimeout see rocketmq_producer.go func (d *DefaultMQProducer) SendWithTimeout(msg rocketmqm.Message, timeout int64) (sendResult *rocketmqm.SendResult, err error) { sendResult, err = d.producerService.sendDefaultImpl(msg.(*message.MessageImpl), constant.COMMUNICATIONMODE_SYNC, "", timeout) return diff --git a/rocketmq-go/kernel/mq_push_consumer.go b/rocketmq-go/kernel/mq_push_consumer.go index 92630b84b..bc2f1ad45 100644 --- a/rocketmq-go/kernel/mq_push_consumer.go +++ b/rocketmq-go/kernel/mq_push_consumer.go @@ -79,7 +79,7 @@ func (d *DefaultMQPushConsumer) Subscribe(topic string, subExpression string) { //RegisterMessageListener register message listener to this consumer func (d *DefaultMQPushConsumer) RegisterMessageListener(messageListener rocketmqm.MessageListener) { - d.consumeMessageService = NewConsumeMessageConcurrentlyServiceImpl(messageListener) + d.consumeMessageService = newConsumeMessageConcurrentlyServiceImpl(messageListener) } func (d *DefaultMQPushConsumer) resetOffset(offsetTable map[rocketmqm.MessageQueue]int64) { diff --git a/rocketmq-go/kernel/offset_store.go b/rocketmq-go/kernel/offset_store.go index efede3bce..67e1375f4 100644 --- a/rocketmq-go/kernel/offset_store.go +++ b/rocketmq-go/kernel/offset_store.go @@ -71,17 +71,17 @@ func (r *RemoteOffsetStore) removeOffset(mq *rocketmqm.MessageQueue) { } func (r *RemoteOffsetStore) persist(mq *rocketmqm.MessageQueue) { - brokerAddr := r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) + brokerAddr := r.mqClient.fetchMasterBrokerAddress(mq.BrokerName) if len(brokerAddr) == 0 { - r.mqClient.TryToFindTopicPublishInfo(mq.Topic) - brokerAddr = r.mqClient.FetchMasterBrokerAddress(mq.BrokerName) + r.mqClient.tryToFindTopicPublishInfo(mq.Topic) + brokerAddr = r.mqClient.fetchMasterBrokerAddress(mq.BrokerName) } r.offsetTableLock.RLock() offset := r.offsetTable[*mq] r.offsetTableLock.RUnlock() updateConsumerOffsetRequestHeader := &header.UpdateConsumerOffsetRequestHeader{ConsumerGroup: r.groupName, Topic: mq.Topic, QueueId: mq.QueueId, CommitOffset: offset} requestCommand := remoting.NewRemotingCommand(remoting.UPDATE_CONSUMER_OFFSET, updateConsumerOffsetRequestHeader) - r.mqClient.GetRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) + r.mqClient.getRemotingClient().InvokeOneWay(brokerAddr, requestCommand, 1000*5) } func (r *RemoteOffsetStore) readOffset(mq *rocketmqm.MessageQueue, readType int) int64 { @@ -113,10 +113,10 @@ func (r *RemoteOffsetStore) readOffset(mq *rocketmqm.MessageQueue, readType int) } func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *rocketmqm.MessageQueue) (int64, error) { - brokerAddr, _, found := r.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) + brokerAddr, _, found := r.mqClient.findBrokerAddressInSubscribe(mq.BrokerName, 0, false) if !found { - brokerAddr, _, found = r.mqClient.FindBrokerAddressInSubscribe(mq.BrokerName, 0, false) + brokerAddr, _, found = r.mqClient.findBrokerAddressInSubscribe(mq.BrokerName, 0, false) } if found { @@ -132,7 +132,7 @@ func (r *RemoteOffsetStore) fetchConsumeOffsetFromBroker(mq *rocketmqm.MessageQu func (r RemoteOffsetStore) queryConsumerOffset(addr string, requestHeader *header.QueryConsumerOffsetRequestHeader, timeoutMillis int64) (int64, error) { remotingCommand := remoting.NewRemotingCommand(remoting.QUERY_CONSUMER_OFFSET, requestHeader) - response, err := r.mqClient.GetRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) + response, err := r.mqClient.getRemotingClient().InvokeSync(addr, remotingCommand, timeoutMillis) if err != nil { glog.Error(err) return -1, err diff --git a/rocketmq-go/kernel/producer_service.go b/rocketmq-go/kernel/producer_service.go index 5711732b7..8ad91f05d 100644 --- a/rocketmq-go/kernel/producer_service.go +++ b/rocketmq-go/kernel/producer_service.go @@ -35,7 +35,7 @@ type ProducerService interface { sendDefaultImpl(message *message.MessageImpl, communicationMode string, sendCallback string, timeout int64) (sendResult *rocketmqm.SendResult, err error) } -//ProducerService ProducerService's implement +//DefaultProducerService ProducerService's implement type DefaultProducerService struct { producerGroup string producerConfig *rocketmqm.MqProducerConfig @@ -65,7 +65,7 @@ func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, c if err != nil { return } - topicPublishInfo, err = d.mqClient.TryToFindTopicPublishInfo(message.Topic()) + topicPublishInfo, err = d.mqClient.tryToFindTopicPublishInfo(message.Topic()) if err != nil { return } @@ -82,7 +82,7 @@ func (d *DefaultProducerService) sendDefaultImpl(message *message.MessageImpl, c func (d *DefaultProducerService) producerSendMessageRequest(brokerName, brokerAddr string, sendMessageHeader remoting.CustomerHeader, message *message.MessageImpl, timeout int64) (sendResult *rocketmqm.SendResult, err error) { remotingCommand := remoting.NewRemotingCommandWithBody(remoting.SEND_MESSAGE, sendMessageHeader, message.Body()) var response *remoting.RemotingCommand - response, err = d.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) + response, err = d.mqClient.getRemotingClient().InvokeSync(brokerAddr, remotingCommand, timeout) if err != nil { glog.Error(err) return @@ -209,7 +209,7 @@ func (d *DefaultProducerService) doSendMessage(message *message.MessageImpl, mes return } sysFlag = sysFlag | compressMessageFlag - brokerAddr = d.mqClient.FetchMasterBrokerAddress(messageQueue.BrokerName) + brokerAddr = d.mqClient.fetchMasterBrokerAddress(messageQueue.BrokerName) if len(brokerAddr) == 0 { err = errors.New("The broker[" + messageQueue.BrokerName + "] not exist") return diff --git a/rocketmq-go/kernel/producer_service_for_send_back.go b/rocketmq-go/kernel/producer_service_for_send_back.go index 185d4c9a0..f3dafba03 100644 --- a/rocketmq-go/kernel/producer_service_for_send_back.go +++ b/rocketmq-go/kernel/producer_service_for_send_back.go @@ -90,7 +90,7 @@ func (s *sendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName glog.Error(err) return } - brokerAddr := s.mqClient.FetchMasterBrokerAddress(brokerName) + brokerAddr := s.mqClient.fetchMasterBrokerAddress(brokerName) sendMsgBackHeader := &header.ConsumerSendMsgBackRequestHeader{ Offset: messageExt.CommitLogOffset, Group: s.consumerGroup, @@ -101,7 +101,7 @@ func (s *sendMessageBackProducerServiceImpl) consumerSendMessageBack(brokerName MaxReconsumeTimes: int32(s.consumerConfig.MaxReconsumeTimes), } remotingCommand := remoting.NewRemotingCommand(remoting.CONSUMER_SEND_MSG_BACK, sendMsgBackHeader) - response, invokeErr := s.mqClient.GetRemotingClient().InvokeSync(brokerAddr, remotingCommand, 5000) + response, invokeErr := s.mqClient.getRemotingClient().InvokeSync(brokerAddr, remotingCommand, 5000) if invokeErr != nil { err = invokeErr return diff --git a/rocketmq-go/kernel/pull_message_controller.go b/rocketmq-go/kernel/pull_message_controller.go index 26066be30..06188428f 100644 --- a/rocketmq-go/kernel/pull_message_controller.go +++ b/rocketmq-go/kernel/pull_message_controller.go @@ -32,22 +32,23 @@ import ( "time" ) +//PullMessageController put pull message logic here type PullMessageController struct { mqClient RocketMqClient clientFactory *clientFactory } -func NewPullMessageController(mqClient RocketMqClient, clientFactory *clientFactory) *PullMessageController { +func newPullMessageController(mqClient RocketMqClient, clientFactory *clientFactory) *PullMessageController { return &PullMessageController{ mqClient: mqClient, clientFactory: clientFactory, } } -func (p *PullMessageController) Start() { +func (p *PullMessageController) start() { go func() { for { - pullRequest := p.mqClient.DequeuePullMessageRequest() + pullRequest := p.mqClient.dequeuePullMessageRequest() p.pullMessage(pullRequest) } }() @@ -90,7 +91,7 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { return } - var sysFlag int32 = 0 + var sysFlag int32 if commitOffsetValue > 0 { sysFlag |= constant.FLAG_COMMIT_OFFSET } @@ -118,8 +119,8 @@ func (p *PullMessageController) pullMessage(pullRequest *model.PullRequest) { if responseCommand.Code == remoting.SUCCESS && len(responseCommand.Body) > 0 { nextBeginOffset = parseNextBeginOffset(responseCommand) //} - msgs := DecodeMessage(responseFuture.ResponseCommand.Body) - msgs = FilterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) + msgs := decodeMessage(responseFuture.ResponseCommand.Body) + msgs = filterMessageAgainByTags(msgs, defaultMQPullConsumer.subscriptionTag[pullRequest.MessageQueue.Topic]) if len(msgs) == 0 { if pullRequest.ProcessQueue.GetMsgCount() == 0 { defaultMQPullConsumer.offsetStore.updateOffset(pullRequest.MessageQueue, nextBeginOffset, true) @@ -210,13 +211,13 @@ func (p *PullMessageController) enqueueNextPullRequest(defaultMQPullConsumer *De go func() { nextPullTime := time.NewTimer(time.Duration(defaultMQPullConsumer.ConsumerConfig.PullInterval) * time.Millisecond) <-nextPullTime.C - p.mqClient.EnqueuePullMessageRequest(nextPullRequest) + p.mqClient.enqueuePullMessageRequest(nextPullRequest) }() } else { - p.mqClient.EnqueuePullMessageRequest(nextPullRequest) + p.mqClient.enqueuePullMessageRequest(nextPullRequest) } } -func FilterMessageAgainByTags(msgExts []message.MessageExtImpl, subscriptionTagList []string) (result []message.MessageExtImpl) { +func filterMessageAgainByTags(msgExts []message.MessageExtImpl, subscriptionTagList []string) (result []message.MessageExtImpl) { result = msgExts if len(subscriptionTagList) == 0 { return @@ -234,14 +235,14 @@ func FilterMessageAgainByTags(msgExts []message.MessageExtImpl, subscriptionTagL } func (p *PullMessageController) consumerPullMessageAsync(brokerName string, requestHeader remoting.CustomerHeader, invokeCallback remoting.InvokeCallback) { - brokerAddr, _, found := p.mqClient.FindBrokerAddressInSubscribe(brokerName, 0, false) + brokerAddr, _, found := p.mqClient.findBrokerAddressInSubscribe(brokerName, 0, false) if found { remotingCommand := remoting.NewRemotingCommand(remoting.PULL_MESSAGE, requestHeader) - p.mqClient.GetRemotingClient().InvokeAsync(brokerAddr, remotingCommand, 1000, invokeCallback) + p.mqClient.getRemotingClient().InvokeAsync(brokerAddr, remotingCommand, 1000, invokeCallback) } } -func DecodeMessage(data []byte) []message.MessageExtImpl { +func decodeMessage(data []byte) []message.MessageExtImpl { buf := bytes.NewBuffer(data) var storeSize, magicCode, bodyCRC, queueId, flag, sysFlag, reconsumeTimes, bodyLength, bornPort, storePort int32 var queueOffset, physicOffset, preparedTransactionOffset, bornTimeStamp, storeTimestamp int64 @@ -249,7 +250,7 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { var topic, body, properties, bornHost, storeHost []byte var propertiesLength int16 - var propertiesmap = make(map[string]string) + var propertiesMap = make(map[string]string) msgs := []message.MessageExtImpl{} for buf.Len() > 0 { @@ -292,7 +293,7 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { if propertiesLength > 0 { properties = make([]byte, propertiesLength) binary.Read(buf, binary.BigEndian, &properties) - propertiesmap = util.String2MessageProperties(string(properties)) + propertiesMap = util.String2MessageProperties(string(properties)) } if magicCode != -626843481 { @@ -313,7 +314,7 @@ func DecodeMessage(data []byte) []message.MessageExtImpl { msg.StoreTimestamp = storeTimestamp msg.PreparedTransactionOffset = preparedTransactionOffset msg.SetBody(body) - msg.SetProperties(propertiesmap) + msg.SetProperties(propertiesMap) // < 3.5.8 use messageOffsetId // >= 3.5.8 use clientUniqMsgId msg.SetMsgId(msg.GetMsgUniqueKey()) diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index b2f0faaba..e6333ee86 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -161,7 +161,7 @@ func (r *rebalance) rebalanceByTopic(topic string) error { return err } r.topicSubscribeInfoTableLock.RLock() - mqs := r.mqClient.GetTopicSubscribeInfo(topic) + mqs := r.mqClient.getTopicSubscribeInfo(topic) r.topicSubscribeInfoTableLock.RUnlock() if len(mqs) > 0 && len(cidAll) > 0 { var messageQueues model.MessageQueues = mqs @@ -170,7 +170,7 @@ func (r *rebalance) rebalanceByTopic(topic string) error { sort.Sort(messageQueues) sort.Sort(consumerIdSorter) } - allocateResult, err := r.allocateMessageQueueStrategy.Allocate(r.groupName, r.mqClient.GetClientId(), mqs, cidAll) + allocateResult, err := r.allocateMessageQueueStrategy.Allocate(r.groupName, r.mqClient.getClientId(), mqs, cidAll) if err != nil { glog.Error(err) @@ -219,7 +219,7 @@ func (r *rebalance) putTheQueueToProcessQueueTable(topic string, mqSet []rocketm pullRequest.NextOffset = r.computePullFromWhere(&mq) pullRequest.ProcessQueue = model.NewProcessQueue() r.processQueueTable[mq] = pullRequest.ProcessQueue - r.mqClient.EnqueuePullMessageRequest(pullRequest) + r.mqClient.enqueuePullMessageRequest(pullRequest) } } @@ -235,7 +235,7 @@ func (r *rebalance) computePullFromWhere(mq *rocketmqm.MessageQueue) int64 { if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { result = 0 } else { - result = r.mqClient.GetMaxOffset(mq) + result = r.mqClient.getMaxOffset(mq) } } break @@ -253,7 +253,7 @@ func (r *rebalance) computePullFromWhere(mq *rocketmqm.MessageQueue) int64 { if strings.HasPrefix(mq.Topic, constant.RETRY_GROUP_TOPIC_PREFIX) { result = 0 } else { - result = r.mqClient.SearchOffset(mq, r.consumerConfig.ConsumeTimestamp) + result = r.mqClient.searchOffset(mq, r.consumerConfig.ConsumeTimestamp) } } break @@ -265,13 +265,13 @@ func (r *rebalance) computePullFromWhere(mq *rocketmqm.MessageQueue) int64 { } func (r *rebalance) findConsumerIdList(topic string, groupName string) ([]string, error) { - brokerAddr, ok := r.mqClient.FindBrokerAddrByTopic(topic) + brokerAddr, ok := r.mqClient.findBrokerAddrByTopic(topic) if !ok { - err := r.mqClient.UpdateTopicRouteInfoFromNameServer(topic) + err := r.mqClient.updateTopicRouteInfoFromNameServer(topic) if err != nil { glog.Error(err) } - brokerAddr, ok = r.mqClient.FindBrokerAddrByTopic(topic) + brokerAddr, ok = r.mqClient.findBrokerAddrByTopic(topic) } if ok { @@ -288,7 +288,7 @@ func (r *rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, request := remoting.NewRemotingCommand(remoting.GET_CONSUMER_LIST_BY_GROUP, requestHeader) - response, err := r.mqClient.GetRemotingClient().InvokeSync(addr, request, timeoutMillis) + response, err := r.mqClient.getRemotingClient().InvokeSync(addr, request, timeoutMillis) if err != nil { glog.Error(err) return nil, err diff --git a/rocketmq-go/kernel/tasks.go b/rocketmq-go/kernel/tasks.go index b10c8104c..d07bbc2ae 100644 --- a/rocketmq-go/kernel/tasks.go +++ b/rocketmq-go/kernel/tasks.go @@ -55,11 +55,11 @@ func (m *MqClientManager) startAllScheduledTask() { timeoutTimer := time.NewTimer(3 * time.Second) for { <-timeoutTimer.C - m.mqClient.ClearExpireResponse() + m.mqClient.clearExpireResponse() timeoutTimer.Reset(time.Second) } }() - m.pullMessageController.Start() + m.pullMessageController.start() //cleanExpireMsg m.cleanExpireMsgController.start() diff --git a/rocketmq-go/model/message/message_impl.go b/rocketmq-go/model/message/message_impl.go index ed19533c1..ad6fea050 100644 --- a/rocketmq-go/model/message/message_impl.go +++ b/rocketmq-go/model/message/message_impl.go @@ -124,13 +124,6 @@ func (m *MessageImpl) SetDelayTimeLevel(delayTimeLevel int) { m.properties[constant.PROPERTY_DELAY_TIME_LEVEL] = util.IntToString(delayTimeLevel) } -////SetWaitStoreMsgOK -//func (m *MessageImpl) SetWaitStoreMsgOK(waitStoreMsgOK bool) { -// if m.properties == nil { -// m.properties = make(map[string]string) -// } -// m.properties[constant.PROPERTY_WAIT_STORE_MSG_OK] = strconv.FormatBool(waitStoreMsgOK) -//} //GeneratorMsgUniqueKey only use by system func (m *MessageImpl) GeneratorMsgUniqueKey() { if m.properties == nil { diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index ee2d7e4d5..e90d0d567 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -31,8 +31,8 @@ type RocketMqSerializer struct { type itemType int8 const ( - key_item itemType = iota - value_item + keyItem itemType = iota + valueItem ) func (r *RocketMqSerializer) encodeHeaderData(cmd *RemotingCommand) []byte { @@ -121,21 +121,21 @@ func customHeaderDeserialize(extFiledDataBytes []byte) (extFiledMap map[string]i extFiledMap = make(map[string]interface{}) buf := bytes.NewBuffer(extFiledDataBytes) for buf.Len() > 0 { - var key = getItemFormExtFiledDataBytes(buf, key_item) - var value = getItemFormExtFiledDataBytes(buf, value_item) + var key = getItemFormExtFiledDataBytes(buf, keyItem) + var value = getItemFormExtFiledDataBytes(buf, valueItem) extFiledMap[key] = value } return } func getItemFormExtFiledDataBytes(buff *bytes.Buffer, iType itemType) (item string) { - if iType == key_item { + if iType == keyItem { var length int16 binary.Read(buff, binary.BigEndian, &length) var data = make([]byte, length) binary.Read(buff, binary.BigEndian, &data) item = string(data) } - if iType == value_item { + if iType == valueItem { var length int32 binary.Read(buff, binary.BigEndian, &length) var data = make([]byte, length) diff --git a/rocketmq-go/util/json_util.go b/rocketmq-go/util/json_util.go index c052223d8..97253b0b6 100644 --- a/rocketmq-go/util/json_util.go +++ b/rocketmq-go/util/json_util.go @@ -54,6 +54,7 @@ type parseInfo struct { nowValue string } +//GetKvStringMap convert json string to map[string]string func GetKvStringMap(str string) (kvMap map[string]string, err error) { var tokenList []token tokenList, err = parseTokenList(str) diff --git a/rocketmq-go/util/map_util.go b/rocketmq-go/util/map_util.go index eea263107..c0d9181ff 100644 --- a/rocketmq-go/util/map_util.go +++ b/rocketmq-go/util/map_util.go @@ -5,7 +5,7 @@ import ( "strings" ) -// +//Struct2Map convert interface{} to map[string]interface{} func Struct2Map(structBody interface{}) (resultMap map[string]interface{}) { resultMap = make(map[string]interface{}) value := reflect.ValueOf(structBody) From e31c606b7d2e69068e680971a627a690da862796 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 23:30:34 +0800 Subject: [PATCH 73/88] replace errors.New(fmt.Sprintf(...)) with fmt.Errorf(...) (golint) --- rocketmq-go/example/simple_producer_consumer.go | 1 + rocketmq-go/kernel/mq_client.go | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 052d7c6c9..b38060efe 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -18,6 +18,7 @@ limitations under the License. package main import ( + "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/golang/glog" diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 02c49c56e..758e59ca0 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -19,7 +19,6 @@ package kernel import ( "encoding/json" - "errors" "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/kernel/header" @@ -218,9 +217,8 @@ func (m MqClientImpl) getTopicRouteInfoFromNameServer(topic string, timeoutMilli return nil, err } return topicRouteData, nil - } else { - return nil, errors.New(fmt.Sprintf("get topicRouteInfo from nameServer error[code:%d,topic:%s]", response.Code, topic)) } + return nil, fmt.Errorf("get topicRouteInfo from nameServer error[code:%d,topic:%s]", response.Code, topic) } func (m MqClientImpl) findBrokerAddressInSubscribe(brokerName string, brokerId int, onlyThisBroker bool) (brokerAddr string, slave bool, found bool) { From 6eae38049c28a524f0e4044d9cba69597e843001 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sat, 26 Aug 2017 23:33:02 +0800 Subject: [PATCH 74/88] golint --- rocketmq-go/example/simple_producer_consumer.go | 1 - 1 file changed, 1 deletion(-) diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index b38060efe..052d7c6c9 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -18,7 +18,6 @@ limitations under the License. package main import ( - "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/golang/glog" From 36ccf5bc0aa0cc7098d58da6b6b04e2ec2895fcf Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 13:00:34 +0800 Subject: [PATCH 75/88] golint --- rocketmq-go/kernel/mq_client.go | 8 +- rocketmq-go/remoting/remoting_client.go | 2 +- rocketmq-go/util/concurrent_map.go | 287 ++++++++++++------------ rocketmq-go/util/concurrentmap.go | 162 +++++++++++++ 4 files changed, 310 insertions(+), 149 deletions(-) create mode 100644 rocketmq-go/util/concurrentmap.go diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 758e59ca0..47c2ec47b 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -79,11 +79,11 @@ type MqClientImpl struct { func mqClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor remoting.ClientRequestProcessor) (mqClientImpl *MqClientImpl) { mqClientImpl = &MqClientImpl{} mqClientImpl.clientId = buildMqClientImplId() - mqClientImpl.topicRouteTable = util.New() // make(map[string]*model.TopicRouteData) - mqClientImpl.brokerAddrTable = util.New() //make(map[string]map[int]string) + mqClientImpl.topicRouteTable = util.NewConcurrentMap() // make(map[string]*model.TopicRouteData) + mqClientImpl.brokerAddrTable = util.NewConcurrentMap() //make(map[string]map[int]string) mqClientImpl.remotingClient = remoting.RemotingClientInit(clientConfig, clientRequestProcessor) - mqClientImpl.topicPublishInfoTable = util.New() //make(map[string]*model.TopicPublishInfo) - mqClientImpl.topicSubscribeInfoTable = util.New() //make(map[string][]*rocketmqm.MessageQueue) + mqClientImpl.topicPublishInfoTable = util.NewConcurrentMap() //make(map[string]*model.TopicPublishInfo) + mqClientImpl.topicSubscribeInfoTable = util.NewConcurrentMap() //make(map[string][]*rocketmqm.MessageQueue) mqClientImpl.pullRequestQueue = make(chan *model.PullRequest, 1024) return } diff --git a/rocketmq-go/remoting/remoting_client.go b/rocketmq-go/remoting/remoting_client.go index 08806ce64..3cf0e0442 100644 --- a/rocketmq-go/remoting/remoting_client.go +++ b/rocketmq-go/remoting/remoting_client.go @@ -64,7 +64,7 @@ type DefaultRemotingClient struct { func RemotingClientInit(clientConfig *rocketmqm.MqClientConfig, clientRequestProcessor ClientRequestProcessor) (client *DefaultRemotingClient) { client = &DefaultRemotingClient{} client.connTable = map[string]net.Conn{} - client.responseTable = util.New() + client.responseTable = util.NewConcurrentMap() client.clientConfig = clientConfig client.namesrvAddrList = strings.Split(clientConfig.NameServerAddress, ";") diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go index d0af20f82..f94c33788 100644 --- a/rocketmq-go/util/concurrent_map.go +++ b/rocketmq-go/util/concurrent_map.go @@ -17,147 +17,146 @@ limitations under the License. package util -import ( - "sync" -) - -var MAP_ITEM_COUNT = 32 - -//ConcurrentMap -type ConcurrentMap []*concurrentMapItem - -type concurrentMapItem struct { - item map[string]interface{} - sync.RWMutex -} - -//create a ConcurrentMap instance -func New() ConcurrentMap { - m := make(ConcurrentMap, MAP_ITEM_COUNT) - for i := 0; i < MAP_ITEM_COUNT; i++ { - m[i] = &concurrentMapItem{item: make(map[string]interface{})} - } - return m -} - -func (m ConcurrentMap) GetMapItem(key string) *concurrentMapItem { - return m[uint(fnv32(key))%uint(MAP_ITEM_COUNT)] -} - -func (m *ConcurrentMap) Set(key string, value interface{}) { - shard := m.GetMapItem(key) - shard.Lock() - shard.item[key] = value - shard.Unlock() -} - -func (m ConcurrentMap) Get(key string) (interface{}, bool) { - shard := m.GetMapItem(key) - shard.RLock() - val, ok := shard.item[key] - shard.RUnlock() - return val, ok -} - -// Returns the number of elements within the map. -func (m ConcurrentMap) Count() int { - count := 0 - for i := 0; i < MAP_ITEM_COUNT; i++ { - shard := m[i] - shard.RLock() - count += len(shard.item) - shard.RUnlock() - } - return count -} - -// Removes an element from the map. -func (m *ConcurrentMap) Remove(key string) { - // Try to get shard. - shard := m.GetMapItem(key) - shard.Lock() - delete(shard.item, key) - shard.Unlock() -} - -// Used by the Iter & IterBuffered functions to wrap two variables together over a channel, -type Tuple struct { - Key string - Val interface{} -} - -// Returns a buffered iterator which could be used in a for range loop. -func (m ConcurrentMap) IterBuffered() <-chan Tuple { - ch := make(chan Tuple, m.Count()) - go func() { - wg := sync.WaitGroup{} - wg.Add(MAP_ITEM_COUNT) - // Foreach shard. - for _, shard := range m { - go func(shard *concurrentMapItem) { - // Foreach key, value pair. - shard.RLock() - for key, val := range shard.item { - ch <- Tuple{key, val} - } - shard.RUnlock() - wg.Done() - }(shard) - } - wg.Wait() - close(ch) - }() - return ch -} - -// Returns all item as map[string]interface{} -func (m ConcurrentMap) Items() map[string]interface{} { - tmp := make(map[string]interface{}) - - // Insert item to temporary map. - for item := range m.IterBuffered() { - tmp[item.Key] = item.Val - } - - return tmp -} - -// Return all keys as []string -func (m ConcurrentMap) Keys() []string { - count := m.Count() - ch := make(chan string, count) - go func() { - // Foreach shard. - wg := sync.WaitGroup{} - wg.Add(MAP_ITEM_COUNT) - for _, shard := range m { - go func(shard *concurrentMapItem) { - // Foreach key, value pair. - shard.RLock() - for key := range shard.item { - ch <- key - } - shard.RUnlock() - wg.Done() - }(shard) - } - wg.Wait() - close(ch) - }() - - keys := make([]string, 0, count) - for k := range ch { - keys = append(keys, k) - } - return keys -} - -func fnv32(key string) uint32 { - hash := uint32(2166136261) - const prime32 = uint32(16777619) - for i := 0; i < len(key); i++ { - hash *= prime32 - hash ^= uint32(key[i]) - } - return hash -} +//import ( +// "sync" +// "hash/fnv" +//) +// +// +// +////default_map_segment_count +//var default_map_segment_count = 33 +// +////ConcurrentMap +//type ConcurrentMap []*concurrentMapSegment +// +//type concurrentMapSegment struct { +// item map[string]interface{} +// sync.RWMutex +//} +// +////NewConcurrentMap create a ConcurrentMap instance +//func NewConcurrentMap() ConcurrentMap { +// m := make(ConcurrentMap, default_map_segment_count) +// for i := 0; i < default_map_segment_count; i++ { +// m[i] = &concurrentMapSegment{item: make(map[string]interface{})} +// } +// return m +//} +// +//func (m ConcurrentMap) getMapSegment(key string) *concurrentMapSegment { +// return m[segmentIndex(key)] +//} +// +//func (m *ConcurrentMap) Set(key string, value interface{}) { +// shard := m.getMapSegment(key) +// shard.Lock() +// shard.item[key] = value +// shard.Unlock() +//} +// +//func (m ConcurrentMap) Get(key string) (interface{}, bool) { +// shard := m.getMapSegment(key) +// shard.RLock() +// val, ok := shard.item[key] +// shard.RUnlock() +// return val, ok +//} +// +//// Returns the number of elements within the map. +//func (m ConcurrentMap) Count() int { +// count := 0 +// for i := 0; i < default_map_segment_count; i++ { +// shard := m[i] +// shard.RLock() +// count += len(shard.item) +// shard.RUnlock() +// } +// return count +//} +// +//// Removes an element from the map. +//func (m *ConcurrentMap) Remove(key string) { +// // Try to get shard. +// shard := m.getMapSegment(key) +// shard.Lock() +// delete(shard.item, key) +// shard.Unlock() +//} +// +//// Used by the Iter & allMapEntry functions to wrap two variables together over a channel, +//type MapEntry struct { +// Key string +// Value interface{} +//} +// +//// Returns a buffered allMapEntry which could be used in a for range loop. +//func (m ConcurrentMap) allMapEntry() <-chan MapEntry { +// ch := make(chan MapEntry, m.Count()) +// go func() { +// wg := sync.WaitGroup{} +// wg.Add(default_map_segment_count) +// // Foreach shard. +// for _, shard := range m { +// go func(shard *concurrentMapSegment) { +// // Foreach key, value pair. +// shard.RLock() +// for key, val := range shard.item { +// ch <- MapEntry{key, val} +// } +// shard.RUnlock() +// wg.Done() +// }(shard) +// } +// wg.Wait() +// close(ch) +// }() +// return ch +//} +// +//// Returns all item as map[string]interface{} +//func (m ConcurrentMap) Items() map[string]interface{} { +// tmp := make(map[string]interface{}) +// +// // Insert item to temporary map. +// for item := range m.allMapEntry() { +// tmp[item.Key] = item.Value +// } +// +// return tmp +//} +// +//// Return all keys as []string +//func (m ConcurrentMap) Keys() []string { +// count := m.Count() +// ch := make(chan string, count) +// go func() { +// // Foreach shard. +// wg := sync.WaitGroup{} +// wg.Add(default_map_segment_count) +// for _, shard := range m { +// go func(shard *concurrentMapSegment) { +// // Foreach key, value pair. +// shard.RLock() +// for key := range shard.item { +// ch <- key +// } +// shard.RUnlock() +// wg.Done() +// }(shard) +// } +// wg.Wait() +// close(ch) +// }() +// +// keys := make([]string, 0, count) +// for k := range ch { +// keys = append(keys, k) +// } +// return keys +//} +//func segmentIndex(key string) uint { +// h := fnv.New32a() +// h.Write([]byte(key)) +// return uint(h.Sum32()) % uint(default_map_segment_count) +//} diff --git a/rocketmq-go/util/concurrentmap.go b/rocketmq-go/util/concurrentmap.go new file mode 100644 index 000000000..577aafbdd --- /dev/null +++ b/rocketmq-go/util/concurrentmap.go @@ -0,0 +1,162 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package util + +import ( + "hash/fnv" + "sync" +) + +type ConcurrentMap interface { + Get(key string) (interface{}, bool) + Set(key string, value interface{}) + Count() int + Remove(key string) + Items() map[string]interface{} + Keys() []string + Values() []interface{} +} + +//default_map_segment_count +var default_map_segment_count = 33 + +//concurrentMapImpl +type concurrentMapImpl struct { + segments []*concurrentMapSegment + segmentCount int +} + +type concurrentMapSegment struct { + item map[string]interface{} + sync.RWMutex +} + +//NewConcurrentMap create a concurrentMap instance with default segments count +func NewConcurrentMap() (concurrentMap ConcurrentMap) { + return NewConcurrentMapWithSegmentCount(default_map_segment_count) +} + +//NewConcurrentMapWithSegmentCount create a concurrentMap instance with segments count +func NewConcurrentMapWithSegmentCount(segmentCount int) (concurrentMap ConcurrentMap) { + concurrentMapSegments := make([]*concurrentMapSegment, segmentCount) + for i := 0; i < segmentCount; i++ { + concurrentMapSegments[i] = &concurrentMapSegment{item: make(map[string]interface{})} + } + concurrentMap = &concurrentMapImpl{segments: concurrentMapSegments, segmentCount: segmentCount} + return +} + +//Get get map[key] +func (m concurrentMapImpl) Get(key string) (interface{}, bool) { + segment := m.getMapSegment(key) + segment.RLock() + val, ok := segment.item[key] + segment.RUnlock() + return val, ok +} + +func (m concurrentMapImpl) getMapSegment(key string) *concurrentMapSegment { + h := fnv.New32a() + h.Write([]byte(key)) + segmentIndex := uint(h.Sum32() % uint32(m.segmentCount)) + return m.segments[segmentIndex] +} + +//Set set map[key] = value +func (m *concurrentMapImpl) Set(key string, value interface{}) { + segment := m.getMapSegment(key) + segment.Lock() + segment.item[key] = value + segment.Unlock() +} + +//Count count the number of items in this map. +func (m concurrentMapImpl) Count() int { + count := 0 + for i := 0; i < m.segmentCount; i++ { + segment := m.segments[i] + segment.RLock() + count += len(segment.item) + segment.RUnlock() + } + return count +} + +//Remove remove item by key +func (m *concurrentMapImpl) Remove(key string) { + segment := m.getMapSegment(key) + segment.Lock() + delete(segment.item, key) + segment.Unlock() +} + +//Items put all item into one map +func (m concurrentMapImpl) Items() map[string]interface{} { + tmp := make(map[string]interface{}) + for item := range m.allMapEntry() { + tmp[item.Key] = item.Value + } + return tmp +} + +//Keys all keys in this concurrent map +func (m concurrentMapImpl) Keys() []string { + allMapEntry := m.allMapEntry() + keys := make([]string, 0, len(allMapEntry)) + for entry := range allMapEntry { + keys = append(keys, entry.Key) + } + return keys +} + +//Values all values in this concurrent map +func (m concurrentMapImpl) Values() []interface{} { + allMapEntry := m.allMapEntry() + values := make([]interface{}, 0, len(allMapEntry)) + for entry := range allMapEntry { + values = append(values, entry.Value) + } + return values +} + +//MapEntry map entry,has key and value +type MapEntry struct { + Key string + Value interface{} +} + +func (m concurrentMapImpl) allMapEntry() <-chan MapEntry { + ch := make(chan MapEntry, m.Count()) + go func() { + wg := sync.WaitGroup{} + wg.Add(m.segmentCount) + for _, segment := range m.segments { + go func(segment *concurrentMapSegment) { + segment.RLock() + for key, val := range segment.item { + ch <- MapEntry{key, val} + } + segment.RUnlock() + wg.Done() + }(segment) + } + wg.Wait() + close(ch) + }() + return ch +} From 6da2037b63158cec59a626943f336aa396961c68 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 16:43:15 +0800 Subject: [PATCH 76/88] golint --- rocketmq-go/model/constant/message_const.go | 61 +++++--- rocketmq-go/model/constant/mix_all.go | 49 +++--- rocketmq-go/remoting/request_code.go | 120 ++++----------- rocketmq-go/remoting/response_code.go | 49 ++---- rocketmq-go/util/concurrent_map.go | 162 -------------------- rocketmq-go/util/concurrent_map_test.go | 26 ---- rocketmq-go/util/concurrentmap.go | 8 +- 7 files changed, 113 insertions(+), 362 deletions(-) delete mode 100644 rocketmq-go/util/concurrent_map.go delete mode 100644 rocketmq-go/util/concurrent_map_test.go diff --git a/rocketmq-go/model/constant/message_const.go b/rocketmq-go/model/constant/message_const.go index 4a67bcc1a..73c270cb1 100644 --- a/rocketmq-go/model/constant/message_const.go +++ b/rocketmq-go/model/constant/message_const.go @@ -21,28 +21,49 @@ const ( //PROPERTY_KEYS property keys in MessageImpl.properties PROPERTY_KEYS = "KEYS" //PROPERTY_TAGS message tags - PROPERTY_TAGS = "TAGS" - PROPERTY_WAIT_STORE_MSG_OK = "WAIT" - PROPERTY_DELAY_TIME_LEVEL = "DELAY" - PROPERTY_RETRY_TOPIC = "RETRY_TOPIC" - PROPERTY_REAL_TOPIC = "REAL_TOPIC" - PROPERTY_REAL_QUEUE_ID = "REAL_QID" - PROPERTY_TRANSACTION_PREPARED = "TRAN_MSG" - PROPERTY_PRODUCER_GROUP = "PGROUP" - PROPERTY_MIN_OFFSET = "MIN_OFFSET" - PROPERTY_MAX_OFFSET = "MAX_OFFSET" - PROPERTY_BUYER_ID = "BUYER_ID" - PROPERTY_ORIGIN_MESSAGE_ID = "ORIGIN_MESSAGE_ID" - PROPERTY_TRANSFER_FLAG = "TRANSFER_FLAG" - PROPERTY_CORRECTION_FLAG = "CORRECTION_FLAG" - PROPERTY_MQ2_FLAG = "MQ2_FLAG" - PROPERTY_RECONSUME_TIME = "RECONSUME_TIME" + PROPERTY_TAGS = "TAGS" + //PROPERTY_WAIT_STORE_MSG_OK PROPERTY_WAIT_STORE_MSG_OK + PROPERTY_WAIT_STORE_MSG_OK = "WAIT" + //PROPERTY_DELAY_TIME_LEVEL PROPERTY_DELAY_TIME_LEVEL + PROPERTY_DELAY_TIME_LEVEL = "DELAY" + //PROPERTY_RETRY_TOPIC PROPERTY_RETRY_TOPIC + PROPERTY_RETRY_TOPIC = "RETRY_TOPIC" + //PROPERTY_REAL_TOPIC PROPERTY_REAL_TOPIC + PROPERTY_REAL_TOPIC = "REAL_TOPIC" + //PROPERTY_REAL_QUEUE_ID PROPERTY_REAL_QUEUE_ID + PROPERTY_REAL_QUEUE_ID = "REAL_QID" + //PROPERTY_TRANSACTION_PREPARED PROPERTY_TRANSACTION_PREPARED + PROPERTY_TRANSACTION_PREPARED = "TRAN_MSG" + //PROPERTY_PRODUCER_GROUP PROPERTY_PRODUCER_GROUP + PROPERTY_PRODUCER_GROUP = "PGROUP" + //PROPERTY_MIN_OFFSET PROPERTY_MIN_OFFSET + PROPERTY_MIN_OFFSET = "MIN_OFFSET" + //PROPERTY_MAX_OFFSET PROPERTY_MAX_OFFSET + PROPERTY_MAX_OFFSET = "MAX_OFFSET" + //PROPERTY_BUYER_ID PROPERTY_BUYER_ID + PROPERTY_BUYER_ID = "BUYER_ID" + //PROPERTY_ORIGIN_MESSAGE_ID PROPERTY_ORIGIN_MESSAGE_ID + PROPERTY_ORIGIN_MESSAGE_ID = "ORIGIN_MESSAGE_ID" + //PROPERTY_TRANSFER_FLAG PROPERTY_TRANSFER_FLAG + PROPERTY_TRANSFER_FLAG = "TRANSFER_FLAG" + //PROPERTY_CORRECTION_FLAG PROPERTY_CORRECTION_FLAG + PROPERTY_CORRECTION_FLAG = "CORRECTION_FLAG" + //PROPERTY_MQ2_FLAG PROPERTY_MQ2_FLAG + PROPERTY_MQ2_FLAG = "MQ2_FLAG" + //PROPERTY_RECONSUME_TIME PROPERTY_RECONSUME_TIME + PROPERTY_RECONSUME_TIME = "RECONSUME_TIME" + //PROPERTY_MSG_REGION PROPERTY_MSG_REGION PROPERTY_MSG_REGION = "MSG_REGION" PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY" - PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES" - PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME" + //PROPERTY_MAX_RECONSUME_TIMES PROPERTY_MAX_RECONSUME_TIMES + PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES" + //PROPERTY_CONSUME_START_TIMESTAMP PROPERTY_CONSUME_START_TIMESTAMP + PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME" - COMMUNICATIONMODE_SYNC = "SYNC" - COMMUNICATIONMODE_ASYNC = "ASYNC" + //COMMUNICATIONMODE_SYNC COMMUNICATIONMODE_SYNC + COMMUNICATIONMODE_SYNC = "SYNC" + //COMMUNICATIONMODE_ASYNC COMMUNICATIONMODE_ASYNC + COMMUNICATIONMODE_ASYNC = "ASYNC" + //COMMUNICATIONMODE_ONEWAY COMMUNICATIONMODE_ONEWAY COMMUNICATIONMODE_ONEWAY = "ONEWAY" ) diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index 97829e1f7..6c1858734 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -18,40 +18,29 @@ limitations under the License. package constant const ( - //ROCKETMQ_HOME_ENV rocketmq home env - ROCKETMQ_HOME_ENV = "ROCKETMQ_HOME" - ROCKETMQ_HOME_PROPERTY = "rocketmq.home.dir" - NAMESRV_ADDR_ENV = "NAMESRV_ADDR" - NAMESRV_ADDR_PROPERTY = "rocketmq.namesrv.addr" - MESSAGE_COMPRESS_LEVEL = "rocketmq.message.compressLevel" - DEFAULT_TOPIC = "TBW102" - BENCHMARK_TOPIC = "BenchmarkTest" - DEFAULT_PRODUCER_GROUP = "DEFAULT_PRODUCER" - DEFAULT_CONSUMER_GROUP = "DEFAULT_CONSUMER" - TOOLS_CONSUMER_GROUP = "TOOLS_CONSUMER" - FILTERSRV_CONSUMER_GROUP = "FILTERSRV_CONSUMER" - MONITOR_CONSUMER_GROUP = "__MONITOR_CONSUMER" + //MESSAGE_COMPRESS_LEVEL MESSAGE_COMPRESS_LEVEL + MESSAGE_COMPRESS_LEVEL = "rocketmq.message.compressLevel" + //DEFAULT_TOPIC DEFAULT_TOPIC + DEFAULT_TOPIC = "TBW102" + //CLIENT_INNER_PRODUCER_GROUP CLIENT_INNER_PRODUCER_GROUP CLIENT_INNER_PRODUCER_GROUP = "CLIENT_INNER_PRODUCER" - SELF_TEST_PRODUCER_GROUP = "SELF_TEST_P_GROUP" - SELF_TEST_CONSUMER_GROUP = "SELF_TEST_C_GROUP" - SELF_TEST_TOPIC = "SELF_TEST_TOPIC" - OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT" - ONS_HTTP_PROXY_GROUP = "CID_ONS-HTTP-PROXY" - CID_ONSAPI_PERMISSION_GROUP = "CID_ONSAPI_PERMISSION" - CID_ONSAPI_OWNER_GROUP = "CID_ONSAPI_OWNER" - CID_ONSAPI_PULL_GROUP = "CID_ONSAPI_PULL" - CID_RMQ_SYS_PREFIX = "CID_RMQ_SYS_" + //MASTER_ID MASTER_ID MASTER_ID int64 = 0 + //RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX CURRENT_JVM_PID - + //RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX = "%RETRY%" - - DLQ_GROUP_TOPIC_PREFIX = "%DLQ%" - SYSTEM_TOPIC_PREFIX = "rmq_sys_" - UNIQUE_MSG_QUERY_FLAG = "_UNIQUE_KEY_QUERY" - MAX_MESSAGE_BODY_SIZE int = 4 * 1024 * 1024 //4m - MAX_MESSAGE_TOPIC_SIZE int = 255 //255char - + //DLQ_GROUP_TOPIC_PREFIX DLQ_GROUP_TOPIC_PREFIX + DLQ_GROUP_TOPIC_PREFIX = "%DLQ%" + //SYSTEM_TOPIC_PREFIX SYSTEM_TOPIC_PREFIX + SYSTEM_TOPIC_PREFIX = "rmq_sys_" + //UNIQUE_MSG_QUERY_FLAG UNIQUE_MSG_QUERY_FLAG + UNIQUE_MSG_QUERY_FLAG = "_UNIQUE_KEY_QUERY" + //MAX_MESSAGE_BODY_SIZE MAX_MESSAGE_BODY_SIZE + MAX_MESSAGE_BODY_SIZE int = 4 * 1024 * 1024 //4m + //MAX_MESSAGE_TOPIC_SIZE MAX_MESSAGE_TOPIC_SIZE + MAX_MESSAGE_TOPIC_SIZE int = 255 //255char + //DEFAULT_TOPIC_QUEUE_NUMS DEFAULT_TOPIC_QUEUE_NUMS DEFAULT_TOPIC_QUEUE_NUMS int32 = 4 ) diff --git a/rocketmq-go/remoting/request_code.go b/rocketmq-go/remoting/request_code.go index b6ded0cd9..69806dac5 100644 --- a/rocketmq-go/remoting/request_code.go +++ b/rocketmq-go/remoting/request_code.go @@ -18,95 +18,39 @@ limitations under the License. package remoting const ( - SEND_MESSAGE = 10 - PULL_MESSAGE = 11 - QUERY_MESSAGE = 12 - QUERY_BROKER_OFFSET = 13 - QUERY_CONSUMER_OFFSET = 14 - UPDATE_CONSUMER_OFFSET = 15 - UPDATE_AND_CREATE_TOPIC = 17 - GET_ALL_TOPIC_CONFIG = 21 - GET_TOPIC_CONFIG_LIST = 22 - GET_TOPIC_NAME_LIST = 23 - UPDATE_BROKER_CONFIG = 25 - GET_BROKER_CONFIG = 26 - TRIGGER_DELETE_FILES = 27 - GET_BROKER_RUNTIME_INFO = 28 - SEARCH_OFFSET_BY_TIMESTAMP = 29 - GET_MAX_OFFSET = 30 - GET_MIN_OFFSET = 31 - GET_EARLIEST_MSG_STORETIME = 32 - VIEW_MESSAGE_BY_ID = 33 - HEART_BEAT = 34 - UNREGISTER_CLIENT = 35 - CONSUMER_SEND_MSG_BACK = 36 - END_TRANSACTION = 37 - GET_CONSUMER_LIST_BY_GROUP = 38 - CHECK_TRANSACTION_STATE = 39 - NOTIFY_CONSUMER_IDS_CHANGED = 40 - LOCK_BATCH_MQ = 41 - UNLOCK_BATCH_MQ = 42 - GET_ALL_CONSUMER_OFFSET = 43 - GET_ALL_DELAY_OFFSET = 45 - PUT_KV_CONFIG = 100 - GET_KV_CONFIG = 101 - DELETE_KV_CONFIG = 102 - REGISTER_BROKER = 103 - UNREGISTER_BROKER = 104 - GET_ROUTEINTO_BY_TOPIC = 105 - GET_BROKER_CLUSTER_INFO = 106 - UPDATE_AND_CREATE_SUBSCRIPTIONGROUP = 200 - GET_ALL_SUBSCRIPTIONGROUP_CONFIG = 201 - GET_TOPIC_STATS_INFO = 202 - GET_CONSUMER_CONNECTION_LIST = 203 - GET_PRODUCER_CONNECTION_LIST = 204 - WIPE_WRITE_PERM_OF_BROKER = 205 - - GET_ALL_TOPIC_LIST_FROM_NAMESERVER = 206 - DELETE_SUBSCRIPTIONGROUP = 207 - GET_CONSUME_STATS = 208 - SUSPEND_CONSUMER = 209 - RESUME_CONSUMER = 210 - RESET_CONSUMER_OFFSET_IN_CONSUMER = 211 - RESET_CONSUMER_OFFSET_IN_BROKER = 212 - ADJUST_CONSUMER_THREAD_POOL = 213 - WHO_CONSUME_THE_MESSAGE = 214 - - DELETE_TOPIC_IN_BROKER = 215 - DELETE_TOPIC_IN_NAMESRV = 216 - GET_KV_CONFIG_BY_VALUE = 217 - DELETE_KV_CONFIG_BY_VALUE = 218 - GET_KVLIST_BY_NAMESPACE = 219 - - RESET_CONSUMER_CLIENT_OFFSET = 220 - GET_CONSUMER_STATUS_FROM_CLIENT = 221 - INVOKE_BROKER_TO_RESET_OFFSET = 222 - INVOKE_BROKER_TO_GET_CONSUMER_STATUS = 223 - - QUERY_TOPIC_CONSUME_BY_WHO = 300 - - GET_TOPICS_BY_CLUSTER = 224 - - REGISTER_FILTER_SERVER = 301 - REGISTER_MESSAGE_FILTER_CLASS = 302 - QUERY_CONSUME_TIME_SPAN = 303 - GET_SYSTEM_TOPIC_LIST_FROM_NS = 304 - GET_SYSTEM_TOPIC_LIST_FROM_BROKER = 305 - - CLEAN_EXPIRED_CONSUMEQUEUE = 306 - + //SEND_MESSAGE SEND_MESSAGE + SEND_MESSAGE = 10 + //PULL_MESSAGE PULL_MESSAGE + PULL_MESSAGE = 11 + //QUERY_CONSUMER_OFFSET QUERY_CONSUMER_OFFSET + QUERY_CONSUMER_OFFSET = 14 + //UPDATE_CONSUMER_OFFSET UPDATE_CONSUMER_OFFSET + UPDATE_CONSUMER_OFFSET = 15 + //SEARCH_OFFSET_BY_TIMESTAMP SEARCH_OFFSET_BY_TIMESTAMP + SEARCH_OFFSET_BY_TIMESTAMP = 29 + //GET_MAX_OFFSET GET_MAX_OFFSET + GET_MAX_OFFSET = 30 + //HEART_BEAT HEART_BEAT + HEART_BEAT = 34 + //CONSUMER_SEND_MSG_BACK CONSUMER_SEND_MSG_BACK + CONSUMER_SEND_MSG_BACK = 36 + //GET_CONSUMER_LIST_BY_GROUP GET_CONSUMER_LIST_BY_GROUP + GET_CONSUMER_LIST_BY_GROUP = 38 + //CHECK_TRANSACTION_STATE CHECK_TRANSACTION_STATE + CHECK_TRANSACTION_STATE = 39 + //NOTIFY_CONSUMER_IDS_CHANGED NOTIFY_CONSUMER_IDS_CHANGED + NOTIFY_CONSUMER_IDS_CHANGED = 40 + //GET_ROUTEINTO_BY_TOPIC GET_ROUTEINTO_BY_TOPIC + GET_ROUTEINTO_BY_TOPIC = 105 + + //RESET_CONSUMER_CLIENT_OFFSET RESET_CONSUMER_CLIENT_OFFSET + RESET_CONSUMER_CLIENT_OFFSET = 220 + //GET_CONSUMER_STATUS_FROM_CLIENT GET_CONSUMER_STATUS_FROM_CLIENT + GET_CONSUMER_STATUS_FROM_CLIENT = 221 + + //GET_CONSUMER_RUNNING_INFO GET_CONSUMER_RUNNING_INFO GET_CONSUMER_RUNNING_INFO = 307 - QUERY_CORRECTION_OFFSET = 308 - + //CONSUME_MESSAGE_DIRECTLY CONSUME_MESSAGE_DIRECTLY CONSUME_MESSAGE_DIRECTLY = 309 - - SEND_MESSAGE_V2 = 310 - - GET_UNIT_TOPIC_LIST = 311 - GET_HAS_UNIT_SUB_TOPIC_LIST = 312 - GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST = 313 - CLONE_GROUP_OFFSET = 314 - - VIEW_BROKER_STATS_DATA = 315 ) diff --git a/rocketmq-go/remoting/response_code.go b/rocketmq-go/remoting/response_code.go index 2b5682768..3f1c46259 100644 --- a/rocketmq-go/remoting/response_code.go +++ b/rocketmq-go/remoting/response_code.go @@ -18,37 +18,20 @@ limitations under the License. package remoting const ( - SUCCESS = 0 - SYSTEM_ERROR = 1 - SYSTEM_BUSY = 2 - REQUEST_CODE_NOT_SUPPORTED = 3 - TRANSACTION_FAILED = 4 - FLUSH_DISK_TIMEOUT = 10 - SLAVE_NOT_AVAILABLE = 11 - FLUSH_SLAVE_TIMEOUT = 12 - MESSAGE_ILLEGAL = 13 - SERVICE_NOT_AVAILABLE = 14 - VERSION_NOT_SUPPORTED = 15 - NO_PERMISSION = 16 - TOPIC_NOT_EXIST = 17 - TOPIC_EXIST_ALREADY = 18 - PULL_NOT_FOUND = 19 - PULL_RETRY_IMMEDIATELY = 20 - PULL_OFFSET_MOVED = 21 - QUERY_NOT_FOUND = 22 - SUBSCRIPTION_PARSE_FAILED = 23 - SUBSCRIPTION_NOT_EXIST = 24 - SUBSCRIPTION_NOT_LATEST = 25 - SUBSCRIPTION_GROUP_NOT_EXIST = 26 - TRANSACTION_SHOULD_COMMIT = 200 - TRANSACTION_SHOULD_ROLLBACK = 201 - TRANSACTION_STATE_UNKNOW = 202 - TRANSACTION_STATE_GROUP_WRONG = 203 - NO_BUYER_ID = 204 - - NOT_IN_CURRENT_UNIT = 205 - - CONSUMER_NOT_ONLINE = 206 - - CONSUME_MSG_TIMEOUT = 207 + //SUCCESS SUCCESS + SUCCESS = 0 + //FLUSH_DISK_TIMEOUT FLUSH_DISK_TIMEOUT + FLUSH_DISK_TIMEOUT = 10 + //SLAVE_NOT_AVAILABLE SLAVE_NOT_AVAILABLE + SLAVE_NOT_AVAILABLE = 11 + //FLUSH_SLAVE_TIMEOUT FLUSH_SLAVE_TIMEOUT + FLUSH_SLAVE_TIMEOUT = 12 + //PULL_NOT_FOUND PULL_NOT_FOUND + PULL_NOT_FOUND = 19 + //PULL_RETRY_IMMEDIATELY PULL_RETRY_IMMEDIATELY + PULL_RETRY_IMMEDIATELY = 20 + //PULL_OFFSET_MOVED PULL_OFFSET_MOVED + PULL_OFFSET_MOVED = 21 + //QUERY_NOT_FOUND QUERY_NOT_FOUND + QUERY_NOT_FOUND = 22 ) diff --git a/rocketmq-go/util/concurrent_map.go b/rocketmq-go/util/concurrent_map.go deleted file mode 100644 index f94c33788..000000000 --- a/rocketmq-go/util/concurrent_map.go +++ /dev/null @@ -1,162 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package util - -//import ( -// "sync" -// "hash/fnv" -//) -// -// -// -////default_map_segment_count -//var default_map_segment_count = 33 -// -////ConcurrentMap -//type ConcurrentMap []*concurrentMapSegment -// -//type concurrentMapSegment struct { -// item map[string]interface{} -// sync.RWMutex -//} -// -////NewConcurrentMap create a ConcurrentMap instance -//func NewConcurrentMap() ConcurrentMap { -// m := make(ConcurrentMap, default_map_segment_count) -// for i := 0; i < default_map_segment_count; i++ { -// m[i] = &concurrentMapSegment{item: make(map[string]interface{})} -// } -// return m -//} -// -//func (m ConcurrentMap) getMapSegment(key string) *concurrentMapSegment { -// return m[segmentIndex(key)] -//} -// -//func (m *ConcurrentMap) Set(key string, value interface{}) { -// shard := m.getMapSegment(key) -// shard.Lock() -// shard.item[key] = value -// shard.Unlock() -//} -// -//func (m ConcurrentMap) Get(key string) (interface{}, bool) { -// shard := m.getMapSegment(key) -// shard.RLock() -// val, ok := shard.item[key] -// shard.RUnlock() -// return val, ok -//} -// -//// Returns the number of elements within the map. -//func (m ConcurrentMap) Count() int { -// count := 0 -// for i := 0; i < default_map_segment_count; i++ { -// shard := m[i] -// shard.RLock() -// count += len(shard.item) -// shard.RUnlock() -// } -// return count -//} -// -//// Removes an element from the map. -//func (m *ConcurrentMap) Remove(key string) { -// // Try to get shard. -// shard := m.getMapSegment(key) -// shard.Lock() -// delete(shard.item, key) -// shard.Unlock() -//} -// -//// Used by the Iter & allMapEntry functions to wrap two variables together over a channel, -//type MapEntry struct { -// Key string -// Value interface{} -//} -// -//// Returns a buffered allMapEntry which could be used in a for range loop. -//func (m ConcurrentMap) allMapEntry() <-chan MapEntry { -// ch := make(chan MapEntry, m.Count()) -// go func() { -// wg := sync.WaitGroup{} -// wg.Add(default_map_segment_count) -// // Foreach shard. -// for _, shard := range m { -// go func(shard *concurrentMapSegment) { -// // Foreach key, value pair. -// shard.RLock() -// for key, val := range shard.item { -// ch <- MapEntry{key, val} -// } -// shard.RUnlock() -// wg.Done() -// }(shard) -// } -// wg.Wait() -// close(ch) -// }() -// return ch -//} -// -//// Returns all item as map[string]interface{} -//func (m ConcurrentMap) Items() map[string]interface{} { -// tmp := make(map[string]interface{}) -// -// // Insert item to temporary map. -// for item := range m.allMapEntry() { -// tmp[item.Key] = item.Value -// } -// -// return tmp -//} -// -//// Return all keys as []string -//func (m ConcurrentMap) Keys() []string { -// count := m.Count() -// ch := make(chan string, count) -// go func() { -// // Foreach shard. -// wg := sync.WaitGroup{} -// wg.Add(default_map_segment_count) -// for _, shard := range m { -// go func(shard *concurrentMapSegment) { -// // Foreach key, value pair. -// shard.RLock() -// for key := range shard.item { -// ch <- key -// } -// shard.RUnlock() -// wg.Done() -// }(shard) -// } -// wg.Wait() -// close(ch) -// }() -// -// keys := make([]string, 0, count) -// for k := range ch { -// keys = append(keys, k) -// } -// return keys -//} -//func segmentIndex(key string) uint { -// h := fnv.New32a() -// h.Write([]byte(key)) -// return uint(h.Sum32()) % uint(default_map_segment_count) -//} diff --git a/rocketmq-go/util/concurrent_map_test.go b/rocketmq-go/util/concurrent_map_test.go deleted file mode 100644 index 8868a3d45..000000000 --- a/rocketmq-go/util/concurrent_map_test.go +++ /dev/null @@ -1,26 +0,0 @@ -/* -Licensed to the Apache Software Foundation (ASF) under one or more -contributor license agreements. See the NOTICE file distributed with -this work for additional information regarding copyright ownership. -The ASF licenses this file to You under the Apache License, Version 2.0 -(the "License"); you may not use this file except in compliance with -the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package util - -import ( - "testing" -) - -func TestConcurrentMap_Count(t *testing.T) { - -} diff --git a/rocketmq-go/util/concurrentmap.go b/rocketmq-go/util/concurrentmap.go index 577aafbdd..b2f38407d 100644 --- a/rocketmq-go/util/concurrentmap.go +++ b/rocketmq-go/util/concurrentmap.go @@ -22,6 +22,7 @@ import ( "sync" ) +//ConcurrentMap concurrent map type ConcurrentMap interface { Get(key string) (interface{}, bool) Set(key string, value interface{}) @@ -32,8 +33,9 @@ type ConcurrentMap interface { Values() []interface{} } -//default_map_segment_count -var default_map_segment_count = 33 +//defaultMapSegmentCount default is 33,because map's default is 32 +//you can use NewConcurrentMapWithSegmentCount to change the segment count(it depend on your map size) +var defaultMapSegmentCount = 33 //concurrentMapImpl type concurrentMapImpl struct { @@ -48,7 +50,7 @@ type concurrentMapSegment struct { //NewConcurrentMap create a concurrentMap instance with default segments count func NewConcurrentMap() (concurrentMap ConcurrentMap) { - return NewConcurrentMapWithSegmentCount(default_map_segment_count) + return NewConcurrentMapWithSegmentCount(defaultMapSegmentCount) } //NewConcurrentMapWithSegmentCount create a concurrentMap instance with segments count From efc87c653e5a2cb2c019209eb4ed0583eaff6025 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 17:35:27 +0800 Subject: [PATCH 77/88] go test --- rocketmq-go/kernel/rebalance.go | 1 - rocketmq-go/model/constant/message_const.go | 3 +- rocketmq-go/model/constant/mix_all.go | 2 +- .../test/test_consume_message_retry.go | 12 +- .../test_delay_message_send_and_receive.go | 10 +- ...etmq_serialize_message_send_and_receive.go | 6 +- .../test_send_and_receive_compress_message.go | 15 +-- .../test/test_send_and_receive_tag_message.go | 6 +- rocketmq-go/util/concurrentmap_test.go | 106 ++++++++++++++++++ rocketmq-go/util/json_util.go | 10 +- 10 files changed, 142 insertions(+), 29 deletions(-) create mode 100644 rocketmq-go/util/concurrentmap_test.go diff --git a/rocketmq-go/kernel/rebalance.go b/rocketmq-go/kernel/rebalance.go index e6333ee86..fefd5b338 100644 --- a/rocketmq-go/kernel/rebalance.go +++ b/rocketmq-go/kernel/rebalance.go @@ -295,7 +295,6 @@ func (r *rebalance) getConsumerIdListByGroup(addr string, consumerGroup string, } if response.Code == remoting.SUCCESS { getConsumerListByGroupResponseBody := new(header.GetConsumerListByGroupResponseBody) - glog.Info("string(response.Body)" + string(response.Body) + "todo todo") // todo check bodyjson := strings.Replace(string(response.Body), "0:", "\"0\":", -1) bodyjson = strings.Replace(bodyjson, "1:", "\"1\":", -1) err := json.Unmarshal([]byte(bodyjson), getConsumerListByGroupResponseBody) diff --git a/rocketmq-go/model/constant/message_const.go b/rocketmq-go/model/constant/message_const.go index 73c270cb1..cff8f0870 100644 --- a/rocketmq-go/model/constant/message_const.go +++ b/rocketmq-go/model/constant/message_const.go @@ -53,7 +53,8 @@ const ( //PROPERTY_RECONSUME_TIME PROPERTY_RECONSUME_TIME PROPERTY_RECONSUME_TIME = "RECONSUME_TIME" //PROPERTY_MSG_REGION PROPERTY_MSG_REGION - PROPERTY_MSG_REGION = "MSG_REGION" + PROPERTY_MSG_REGION = "MSG_REGION" + //PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY" //PROPERTY_MAX_RECONSUME_TIMES PROPERTY_MAX_RECONSUME_TIMES PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES" diff --git a/rocketmq-go/model/constant/mix_all.go b/rocketmq-go/model/constant/mix_all.go index 6c1858734..3286fe2e6 100644 --- a/rocketmq-go/model/constant/mix_all.go +++ b/rocketmq-go/model/constant/mix_all.go @@ -27,7 +27,7 @@ const ( //MASTER_ID MASTER_ID MASTER_ID int64 = 0 - //RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX + //CURRENT_JVM_PID CURRENT_JVM_PID CURRENT_JVM_PID //RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX RETRY_GROUP_TOPIC_PREFIX = "%RETRY%" diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index 6cc5b6cf8..7c1a8f629 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -18,7 +18,6 @@ limitations under the License. package main import ( - "fmt" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" "github.com/golang/glog" @@ -42,12 +41,11 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - fmt.Println(tag) - consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, message := range messageList { - if string(message.body) != testMessageBody { - panic("message.body is wrong message.body=" + string(message.body) + " testMessageBody=" + testMessageBody + " tag=" + message.Tag()) + if string(message.Body()) != testMessageBody { + panic("message.body is wrong message.body=" + string(message.Body()) + " testMessageBody=" + testMessageBody + " tag=" + message.Tag()) } if consumeTime < 2 { consumeTime++ @@ -63,7 +61,9 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &message.MessageImpl{Topic: testTopic, body: []byte(testMessageBody)} + var message = rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte(testMessageBody)) message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index 51832cd95..e57ffbbad 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -40,12 +40,12 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { endTime := time.Now() - if msg.MsgId != messageId { - panic("messageId is wrong " + msg.MsgId) + if msg.MsgId() != messageId { + panic("messageId is wrong " + msg.MsgId()) } costSeconds := endTime.Unix() - startTime.Unix() if costSeconds < 14 || costSeconds > 16 { @@ -60,7 +60,9 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() <-time.After(time.Second * 30) // wait - var message = &message.MessageImpl{Topic: testTopic, body: []byte("hello world")} + var message = rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte("hello world")) message.SetTag(tag) message.SetDelayTimeLevel(3) // cost 15 second result, err := producer.Send(message) diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index 5f4ab91a6..78ab8d312 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -42,7 +42,7 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, tag) - consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.Tag() == tag && messageBody == string(messageBody) { @@ -56,7 +56,9 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < messageCount; i++ { - var message = &message.MessageImpl{Topic: testTopic, body: []byte(messageBody)} + var message = rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte(messageBody)) message.SetTag(tag) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index 7856e53f6..e80c2b870 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -20,7 +20,6 @@ package main import ( "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" - "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/constant" "github.com/golang/glog" "time" ) @@ -45,13 +44,13 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "compress_message_test") - consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { - if msg.SysFlag&constant.CompressedFlag != constant.CompressedFlag { - panic("message not be compressed") - } - if string(msg.body) != bigMessageBody { + //if msg.SysFlag&constant.CompressedFlag != constant.CompressedFlag { + // panic("message not be compressed") + //} + if string(msg.Body()) != bigMessageBody { panic("message not be unCompressed") } glog.Info("Test compress and tag success") @@ -63,7 +62,9 @@ func main() { }) rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() - var message = &message.MessageImpl{Topic: testTopic, body: []byte(bigMessageBody)} + var message = rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte(bigMessageBody)) message.SetTag("compress_message_test") result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index 0ca16e548..b1155b31f 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -39,7 +39,7 @@ func main() { rocketMQClientInstance.RegisterProducer(producer) var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) consumer.Subscribe(testTopic, "tag0 || tag2||tag4") - consumer.RegisterMessageListener(func(messageList []message.MessageExtImpl) rocketmqm.ConsumeConcurrentlyResult { + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { successIndex := -1 for index, msg := range messageList { if msg.Tag() != "tag0" && msg.Tag() != "tag2" && msg.Tag() != "tag4" { @@ -55,7 +55,9 @@ func main() { rocketMQClientInstance.RegisterConsumer(consumer) rocketMQClientInstance.Start() for i := 0; i < 5; i++ { - var message = &message.MessageImpl{Topic: testTopic, body: []byte("hello world")} + var message = rocketmqm.NewMessage() + message.SetTopic(testTopic) + message.SetBody([]byte("Hello world")) message.SetTag("tag" + util.IntToString(i)) result, err := producer.Send(message) glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) diff --git a/rocketmq-go/util/concurrentmap_test.go b/rocketmq-go/util/concurrentmap_test.go new file mode 100644 index 000000000..337cb479b --- /dev/null +++ b/rocketmq-go/util/concurrentmap_test.go @@ -0,0 +1,106 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package util + +import ( + "testing" +) + +func TestGetAndSet(t *testing.T) { + existKey := "123" + existValue := 1 + notExistKey := "321" + concurrentMap := NewConcurrentMap() + concurrentMap.Set(existKey, existValue) + value, isHave := concurrentMap.Get(existKey) + if !isHave { + t.Errorf("can't read existKey") + } + if value != existValue { + t.Errorf("get value fail") + } + value2, isHave2 := concurrentMap.Get(notExistKey) + if isHave2 { + t.Errorf("read notExistKey") + } + if value2 != nil { + t.Errorf("read notExistKey error") + } +} +func TesCount(t *testing.T) { + count := 123 + concurrentMap := NewConcurrentMap() + for i := 0; i < count; i++ { + concurrentMap.Set(IntToString(i), "") + } + if concurrentMap.Count() != count { + t.Errorf("Count error") + } +} +func TesRemove(t *testing.T) { + count := 123 + removeKey := "111" + concurrentMap := NewConcurrentMap() + for i := 0; i < count; i++ { + concurrentMap.Set(IntToString(i), "") + } + concurrentMap.Remove(removeKey) + if concurrentMap.Count() != count-1 { + t.Errorf("remove error") + } + _, isHave := concurrentMap.Get(removeKey) + if isHave { + t.Errorf("remove error") + + } +} +func TesItems(t *testing.T) { + count := 123 + concurrentMap := NewConcurrentMap() + for i := 0; i < count; i++ { + concurrentMap.Set(IntToString(i), "") + } + itemMap := concurrentMap.Items() + if len(itemMap) != count { + t.Errorf("Items error") + } + +} +func TesKeys(t *testing.T) { + count := 123 + concurrentMap := NewConcurrentMap() + for i := 0; i < count; i++ { + concurrentMap.Set(IntToString(i), "") + } + keys := concurrentMap.Keys() + if len(keys) != count { + t.Errorf("keys error") + } + +} +func TesValues(t *testing.T) { + count := 123 + concurrentMap := NewConcurrentMap() + for i := 0; i < count; i++ { + concurrentMap.Set(IntToString(i), "") + } + values := concurrentMap.Values() + if len(values) != count { + t.Errorf("values error") + } +} diff --git a/rocketmq-go/util/json_util.go b/rocketmq-go/util/json_util.go index 97253b0b6..60fb71b17 100644 --- a/rocketmq-go/util/json_util.go +++ b/rocketmq-go/util/json_util.go @@ -80,10 +80,10 @@ func GetKvStringMap(str string) (kvMap map[string]string, err error) { } currentParseInfo.readType = 1 } else if currentParseInfo.readType == 1 { - currentParseInfo = parseKey(currentParseInfo, nowToken) + parseKey(currentParseInfo, nowToken) } else if currentParseInfo.readType == 2 { var k, v string - currentParseInfo, k, v = parseValue(currentParseInfo, nowToken) + k, v = parseValue(currentParseInfo, nowToken) if len(k) > 0 { kvMap[k] = v } @@ -97,7 +97,7 @@ func GetKvStringMap(str string) (kvMap map[string]string, err error) { } return } -func parseValue(info *parseInfo, nowToken *token) (parsedInfo *parseInfo, key, value string) { +func parseValue(info *parseInfo, nowToken *token) (key, value string) { if nowToken.tokenType == COMMA { // , split kv pair if info.startObjCount == 1 { key = info.nowKey @@ -118,7 +118,7 @@ func parseValue(info *parseInfo, nowToken *token) (parsedInfo *parseInfo, key, v } return } -func parseKey(info *parseInfo, nowToken *token) (parsedInfo *parseInfo) { +func parseKey(info *parseInfo, nowToken *token) { if nowToken.tokenType == COLON { //: split k and v if info.startObjCount == 1 { info.readType = 2 @@ -130,7 +130,7 @@ func parseKey(info *parseInfo, nowToken *token) (parsedInfo *parseInfo) { } else { info.nowKey = info.nowKey + nowToken.tokenValue } - return info + return } func parseTokenList(str string) (tokenList []token, err error) { From 82d26055a56e672a68c6f991fc2da3b8f4b3505e Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 18:17:38 +0800 Subject: [PATCH 78/88] rocketmq api --- rocketmq-go/api/model/message.go | 31 +++++++++++++++++++++------- rocketmq-go/api/model/message_ext.go | 25 ++++++++++++++++++---- rocketmq-go/model/message_queues.go | 17 +++++++++++++++ 3 files changed, 62 insertions(+), 11 deletions(-) diff --git a/rocketmq-go/api/model/message.go b/rocketmq-go/api/model/message.go index 12b060c17..a2b51c851 100644 --- a/rocketmq-go/api/model/message.go +++ b/rocketmq-go/api/model/message.go @@ -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. +*/ + package rocketmqm import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message" @@ -9,21 +26,21 @@ import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/model/message // body: the message's user content // see MessageImpl type Message interface { - //Topic get topic + //Topic Topic get topic Topic() (topic string) - //set message topic + //SetTopic set message topic SetTopic(tag string) - //set message tag + //SetTag set message tag SetTag(tag string) - //get message tag + //Tag get message tag Tag() (tag string) - //set message key + //SetKeys set message key SetKeys(keys []string) - // set delay time level + //SetDelayTimeLevel set delay time level SetDelayTimeLevel(delayTimeLevel int) - // set body + //SetBody set body SetBody([]byte) } diff --git a/rocketmq-go/api/model/message_ext.go b/rocketmq-go/api/model/message_ext.go index 1862444a1..de19bdb97 100644 --- a/rocketmq-go/api/model/message_ext.go +++ b/rocketmq-go/api/model/message_ext.go @@ -1,14 +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. +*/ + package rocketmqm //MessageExt see MessageExtImpl type MessageExt interface { - //get message topic + //Topic get message topic Topic() (tag string) - //get message tag + //Tag get message tag Tag() (tag string) - // get body + //Body get body Body() []byte - // get messageId + //MsgId get messageId MsgId() string } diff --git a/rocketmq-go/model/message_queues.go b/rocketmq-go/model/message_queues.go index 790d0b745..9b44f0d42 100644 --- a/rocketmq-go/model/message_queues.go +++ b/rocketmq-go/model/message_queues.go @@ -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. +*/ + package model import "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" From 5218ab4f1004578fb0d6eeaf86b72151a785db39 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 18:36:21 +0800 Subject: [PATCH 79/88] rocketmq console test --- .../console/service/impl/ConsumerServiceImplTest.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ConsumerServiceImplTest.java b/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ConsumerServiceImplTest.java index 897939122..a0e55cbe1 100644 --- a/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ConsumerServiceImplTest.java +++ b/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ConsumerServiceImplTest.java @@ -19,6 +19,7 @@ import com.google.common.base.Function; import com.google.common.collect.Lists; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -64,6 +65,16 @@ public void tearDown() throws Exception { public void queryGroupList() throws Exception { List consumeInfoList = consumerService.queryGroupList(); Assert.assertTrue(CollectionUtils.isNotEmpty(consumeInfoList)); + GroupConsumeInfo consumeInfo1 = new GroupConsumeInfo(); + consumeInfo1.setCount(1); + consumeInfo1.setDiffTotal(1); + GroupConsumeInfo consumeInfo2 = new GroupConsumeInfo(); + consumeInfo2.setCount(2); + consumeInfo2.setDiffTotal(2); + List consumeInfoListTestCompare = Lists.newArrayList(consumeInfo1,consumeInfo2); + Collections.sort(consumeInfoListTestCompare); + Assert.assertEquals(consumeInfoListTestCompare.get(0),consumeInfo2); + } @Test From bbbbf922bcc948bcc9e68ae86e4ea65a60188f70 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 18:46:34 +0800 Subject: [PATCH 80/88] test examineProducerConnectionInfo exception --- .../service/impl/ProducerServiceImplTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ProducerServiceImplTest.java b/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ProducerServiceImplTest.java index 1baa43317..48a394508 100644 --- a/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ProducerServiceImplTest.java +++ b/rocketmq-console/src/test/java/org/apache/rocketmq/console/service/impl/ProducerServiceImplTest.java @@ -25,11 +25,16 @@ import org.apache.rocketmq.console.service.ProducerService; import org.apache.rocketmq.console.testbase.RocketMQConsoleTestBase; import org.apache.rocketmq.console.testbase.TestConstant; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.tools.admin.MQAdminExt; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.Mockito; import org.springframework.boot.test.context.SpringBootTest; import org.springframework.test.annotation.DirtiesContext; import org.springframework.test.context.junit4.SpringRunner; @@ -68,4 +73,17 @@ public void getProducerConnection() throws Exception { } + + @Mock + private MQAdminExt mqAdminExt; + + @InjectMocks + private ProducerService producerServiceMock = new ProducerServiceImpl(); + + @Test(expected = RuntimeException.class) + public void getProducerConnectionException() throws Exception { + Mockito.when(mqAdminExt.examineProducerConnectionInfo(Mockito.anyString(),Mockito.anyString())).thenThrow(new RemotingException("test-RemotingException")); + producerServiceMock.getProducerConnection(TEST_PRODUCER_GROUP,TEST_CONSOLE_TOPIC); + } + } \ No newline at end of file From b6af3f81ec7cf6251d9d307bd17bf401113d80a9 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Sun, 27 Aug 2017 21:14:18 +0800 Subject: [PATCH 81/88] License --- LICENSE | 201 -------------------------------------------------------- 1 file changed, 201 deletions(-) delete mode 100644 LICENSE diff --git a/LICENSE b/LICENSE deleted file mode 100644 index 7f77f44e7..000000000 --- a/LICENSE +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (properties) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. \ No newline at end of file From c5aab325e3fcb1f0be62db000601fd15ac311818 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 29 Aug 2017 21:26:53 +0800 Subject: [PATCH 82/88] go rocketmq api --- LICENSE | 201 ++++++++++++++++++ rocketmq-go/example/simple_consumer.go | 31 +++ rocketmq-go/example/simple_producer.go | 2 +- rocketmq-go/kernel/mq_client.go | 1 - rocketmq-go/kernel/mq_client_manage.go | 3 - rocketmq-go/remoting/rocketmq_serializable.go | 1 - 6 files changed, 233 insertions(+), 6 deletions(-) create mode 100644 LICENSE diff --git a/LICENSE b/LICENSE new file mode 100644 index 000000000..7f77f44e7 --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (properties) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. \ No newline at end of file diff --git a/rocketmq-go/example/simple_consumer.go b/rocketmq-go/example/simple_consumer.go index a5ebdc40e..2dcec45ab 100644 --- a/rocketmq-go/example/simple_consumer.go +++ b/rocketmq-go/example/simple_consumer.go @@ -17,6 +17,37 @@ limitations under the License. package main +import ( + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api" + "github.com/apache/incubator-rocketmq-externals/rocketmq-go/api/model" + "github.com/golang/glog" +) + func main() { + var ( + nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) + testTopic = "GoLangRocketMQ" + testConsumerGroup = "TestConsumerGroup" + ) + // init rocketMQClientInstance + rocketMQClientInstance := rocketmq.InitRocketMQClientInstance(nameServerAddress) + // 1.init rocketMQConsumer + // 2.subscribe topic and register our function to message listener + // 3.register it + var consumer = rocketmq.NewDefaultMQPushConsumer(testConsumerGroup) + consumer.Subscribe(testTopic, "*") + consumer.RegisterMessageListener(func(messageList []rocketmqm.MessageExt) rocketmqm.ConsumeConcurrentlyResult { + successIndex := -1 + for index, msg := range messageList { + glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId(), string(msg.Body())) + // call your function + successIndex = index + } + return rocketmqm.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmqm.CONSUME_SUCCESS, AckIndex: successIndex} + }) + rocketMQClientInstance.RegisterConsumer(consumer) + // start rocketMQ client instance + rocketMQClientInstance.Start() + select {} } diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index 58690f781..84f4b2dbd 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -43,5 +43,5 @@ func main() { message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult result=[%s] err=[%s]", result.String(), err) } diff --git a/rocketmq-go/kernel/mq_client.go b/rocketmq-go/kernel/mq_client.go index 47c2ec47b..07ac0ab17 100644 --- a/rocketmq-go/kernel/mq_client.go +++ b/rocketmq-go/kernel/mq_client.go @@ -206,7 +206,6 @@ func (m MqClientImpl) getTopicRouteInfoFromNameServer(topic string, timeoutMilli } if response.Code == remoting.SUCCESS { topicRouteData := new(model.TopicRouteData) - glog.Info(string(response.Body) + "todo todo") bodyjson := strings.Replace(string(response.Body), ",0:", ",\"0\":", -1) bodyjson = strings.Replace(bodyjson, ",1:", ",\"1\":", -1) // fastJson key is string todo todo bodyjson = strings.Replace(bodyjson, "{0:", "{\"0\":", -1) diff --git a/rocketmq-go/kernel/mq_client_manage.go b/rocketmq-go/kernel/mq_client_manage.go index 3d57a16c9..f5cd9ef65 100644 --- a/rocketmq-go/kernel/mq_client_manage.go +++ b/rocketmq-go/kernel/mq_client_manage.go @@ -79,9 +79,6 @@ func (m *MqClientManager) RegisterConsumer(consumer *DefaultMQPushConsumer) { consumer.offsetStore = remoteOffsetStoreInit(consumer.consumerGroup, m.mqClient) m.clientFactory.consumerTable[consumer.consumerGroup] = consumer consumer.rebalance = newRebalance(consumer.consumerGroup, consumer.subscription, consumer.mqClient, consumer.offsetStore, consumer.ConsumerConfig) - - fmt.Println(consumer.consumeMessageService) - consumer.consumeMessageService.init(consumer.consumerGroup, m.mqClient, consumer.offsetStore, m.defaultProducerService, consumer.ConsumerConfig) return } diff --git a/rocketmq-go/remoting/rocketmq_serializable.go b/rocketmq-go/remoting/rocketmq_serializable.go index e90d0d567..b230efd12 100644 --- a/rocketmq-go/remoting/rocketmq_serializable.go +++ b/rocketmq-go/remoting/rocketmq_serializable.go @@ -65,7 +65,6 @@ func (r *RocketMqSerializer) encodeHeaderData(cmd *RemotingCommand) []byte { if extFieldsBytesLen > 0 { buf.Write(extFieldsBytes) } - fmt.Println(buf.Bytes()) return buf.Bytes() } From 3cde1de58b820abf371ceb8ab215292b10a897af Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 29 Aug 2017 21:47:00 +0800 Subject: [PATCH 83/88] docs --- rocketmq-go/README.md | 56 ++---------- rocketmq-go/docs/package.puml | 11 ++- rocketmq-go/docs/roadmap.md | 158 +++++++++++----------------------- 3 files changed, 63 insertions(+), 162 deletions(-) diff --git a/rocketmq-go/README.md b/rocketmq-go/README.md index 2310206dc..7e19b13e3 100644 --- a/rocketmq-go/README.md +++ b/rocketmq-go/README.md @@ -1,52 +1,18 @@ # RocketMQ Go SDK Millstone1 Detail Design[![Go Report Card](https://goreportcard.com/badge/github.com/StyleTang/incubator-rocketmq-externals)](https://goreportcard.com/report/github.com/StyleTang/incubator-rocketmq-externals) -## Example -``` -func main() { - var ( - nameServerAddress = "127.0.0.1:9876" //address split by ; (for example 192.168.1.1:9876;192.168.1.2:9876) - testTopic = "GoLangRocketMQ" - testProducerGroup = "TestProducerGroup" - testConsumerGroup = "TestConsumerGroup" - ) - // init rocketMQClientInstance - rocketMQClientInstance := rocketmq_api.InitRocketMQClientInstance(nameServerAddress) - // init rocketMQProducer and register it - var producer = rocketmq_api.NewDefaultMQProducer(testProducerGroup) - rocketMQClientInstance.RegisterProducer(producer) - - // 1.init rocketMQConsumer - // 2.subscribe topic and register our function to message listener - // 3.register it - var consumer = rocketmq_api.NewDefaultMQPushConsumer(testConsumerGroup) - consumer.Subscribe(testTopic, "*") - consumer.RegisterMessageListener(func(messageList []rocketmq_api_model.MessageExt) rocketmq_api_model.ConsumeConcurrentlyResult { - successIndex := -1 - for index, msg := range messageList { - glog.Infof("test receiveMessage messageId=[%s] messageBody=[%s]", msg.MsgId, string(msg.Body)) - // call your function - successIndex = index - } - return rocketmq_api_model.ConsumeConcurrentlyResult{ConsumeConcurrentlyStatus: rocketmq_api_model.CONSUME_SUCCESS, AckIndex: successIndex} - }) - rocketMQClientInstance.RegisterConsumer(consumer) - - // start rocketMQ client instance - rocketMQClientInstance.Start() - - //start send test message - for { - var message = &rocketmq_api_model.Message{Topic: testTopic, Body: []byte("hello World")} - result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) - } - -``` +## How to Use +examples: +* rocketmq-go/example/producer_consumer.go +* rocketmq-go/example/simple_consumer.go +* rocketmq-go/example/simple_producer_consumer.go # Go RocketMQ Client's Arch ![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/svg/ZLNDRk8m4BxdANokL1w06qNzOTK4Dg7g4RdEGACcDjWE4jjotvNsaVfEVOKTsp7vPIa7bF5yy-MRySo4vLGf8WLx0rtiLXin2dVJF0EkGyhf1kHxe43kCmQ9fXg2Oy1w4OiopqDG2k5JmRKKUMxYJjYAO3J9Sy6GfCB-BA54LeadcyFx8BDJSaUP5X8XnGxvLLc0NDAN7D1UI96MpDlT5uOd_9aiQg0dwW39CI0zJYiAynjmkimUCwM16p83gJ0I2g4plXd5rOFd8OHaV2_U83bmLbiJrJBd79xf0UtJI_k4eYWeJmqZAvKMnGFG52IQ4dObA3qLALXBRR4kuCm1XKuPrcwTRQm-JWj8v7wIfeODuSO_MpXrIbFE84A8KpVC5Zi9M6U6HHBIyXej3B8zU8K4zLS94_qAf03zAjAHmzxVBbVJUPGyXRVnAbbEba_9wYwUXwlfu-msMWw0ugSecaNtgrbqDtVkqSYH7Ur_Hsa2CgDvzWla0-gmJKpSxuSIlFRwqzYsQhZhU8v149YAgIrbu7i3-oMuE3Jawlhw_02C6G8f5Zmu2x44TJT_Fy8FIXrHirX8j_z94-cZMqYJuMGnUvPqkqNNc5mAcA_N2dI2gk0Rw1XUQ6uwxnlOwh2gT-9Ect6u4OQkpDocJnXJ8S9Sp_0Sr-KmGSrKX2kmHLbdfp0z_wd4JGy6N0hsaBeoFUgsgw5omFk_TMtXmrKreGdmj3g-eUpDDr85kH3SLNKUsLctxKlqzqJCDRRlYjZcwApDFig0cgkmgb6VQNZ33S6lR74wMoRej5zZLjijM7sIRUw3--CikZNoWLrGGkEA5LiYrkTQceo_Fd-Rhz07FIZs8UmNwElpps0i6NFGNvz_bSyEdt4aE31pwjUx__Jy0m00) +# Go RocketMQ Client's Roadmap + [Go RocketMQ Client's Roadmap](https://github.com/StyleTang/incubator-rocketmq-externals/blob/master/rocketmq-go/docs/roadmap.md) + # How Go RocketMQ Client Works @@ -126,9 +92,3 @@ when message cost too many time,we will drop this message(send message back) -# Go RocketMQ Client's Roadmap - [Go RocketMQ Client's Roadmap](https://github.com/StyleTang/incubator-rocketmq-externals/blob/master/rocketmq-go/docs/roadmap.md) - -# Go RocketMQ Client's Check List - -## todo \ No newline at end of file diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index 9a87ea5eb..7af865ea8 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -14,7 +14,9 @@ namespace api{ MQClientInstance o-- kernel.MqClientManager MQProducer o-- kernel.DefaultMQProducer MQConsumer o-- kernel.DefaultMQPushConsumer - MQConsumer o-- kernel.DefaultMQPullConsumer + MQConsumer o-- kernel.DefaultMQPullConsumer + MQConsumer o-- kernel.MessageListener + } class kernel.MqClientManager{ @@ -76,7 +78,7 @@ kernel.PullMessageController *-- kernel.ClientFactory:contains MqClientManager o-- ClientFactory kernel.ClientFactory *-- DefaultMQPushConsumer:contains kernel.ClientFactory *-- DefaultMQPullConsumer:contains - DefaultMQPushConsumer *-- kernel.PullAPIWrapper : contains + DefaultMQPushConsumer *-- kernel.MqClient : contains DefaultMQPushConsumer *-- kernel.OffsetStore : contains DefaultMQPushConsumer *-- kernel.Rebalance : contains DefaultMQPushConsumer *-- kernel.ConsumeMessageService : contains @@ -88,11 +90,8 @@ kernel.PullMessageController *-- kernel.ClientFactory:contains } namespace kernel{ - MqClientAPI o-- remoting.RemotingClient:contains + MqClient o-- remoting.RemotingClient:contains OffsetStore o-- MqClient - PullAPIWrapper o-- MqClient - MqClient o-- MqClientAPI - PullAPIWrapper o-- MqClientAPI Rebalance o-- MqClient Rebalance o-- OffsetStore Rebalance o-- AllocateMessageQueueStrategy diff --git a/rocketmq-go/docs/roadmap.md b/rocketmq-go/docs/roadmap.md index 0db903326..3f00e4461 100644 --- a/rocketmq-go/docs/roadmap.md +++ b/rocketmq-go/docs/roadmap.md @@ -1,78 +1,22 @@ # RoadMap-Milestone1 - -## Consumer -- [ ] ConsumerType - - [ ] PushConsumer -- [ ] MessageListener - - [ ] Concurrently -- [ ] MessageModel - - [ ] CLUSTERING -- [ ] OffsetStore - - [ ] RemoteBrokerOffsetStore -- [ ] RebalanceService -- [ ] PullMessageService -- [ ] ConsumeMessageService -- [ ] AllocateMessageQueueStrategy - - [ ] AllocateMessageQueueAveragely -- [ ] Other - - [ ] Config - - [ ] ZIP - - [ ] ConsumeFromWhere - - [ ] CONSUME_FROM_LAST_OFFSET - - [ ] CONSUME_FROM_FIRST_OFFSET - - [ ] CONSUME_FROM_TIMESTAMP - - [ ] Retry(sendMessageBack) - - [ ] TimeOut(clearExpiredMessage) - - [ ] ACK(partSuccess) - - [ ] FlowControl(messageCanNotConsume) -## Manager -- [ ] Controller - - [ ] PullMessageController -- [ ] Task - - [ ] Heartbeat - - [ ] UpdateTopicRouteInfoFromNameServer - - [ ] PersistAllConsumerOffset - - [ ] ClearExpiredMessage(form consumer consumeMessageService) -- [ ] ClientRemotingProcessor - - [ ] CHECK_TRANSACTION_STATE - - [ ] NOTIFY_CONSUMER_IDS_CHANGED - - [ ] RESET_CONSUMER_CLIENT_OFFSET - - [ ] GET_CONSUMER_STATUS_FROM_CLIENT - - [ ] GET_CONSUMER_RUNNING_INFO - - [ ] CONSUME_MESSAGE_DIRECTLY - -## Remoting -- [x] MqClientRequest - - [x] InvokeSync - - [x] InvokeAsync - - [x] InvokeOneWay -- [x] Serialize - - [x] JSON - - [x] ROCKETMQ -- [x] NamesrvAddrChoosed(HA) - - -# RoadMap-ALL - ## Producer -- [ ] ProducerType - - [ ] DefaultProducer - - [ ] TransactionProducer +- [x] ProducerType + - [x] DefaultProducer - [ ] API - [ ] Send - - [ ] Sync + - [x] Sync - [ ] Async - [ ] OneWay - [ ] Other - - [ ] DelayMessage - - [ ] Config - - [ ] MessageId Generate - - [ ] CompressMsg - - [ ] TimeOut - - [ ] LoadBalance + - [x] DelayMessage + - [x] Config + - [x] MessageId Generate + - [x] CompressMsg + - [x] TimeOut + - [x] LoadBalance - [ ] DefaultTopic - [ ] VipChannel - - [ ] Retry + - [x] Retry - [ ] SendMessageHook - [ ] CheckRequestQueue - [ ] CheckForbiddenHookList @@ -82,69 +26,67 @@ ## Consumer - [ ] ConsumerType - - [ ] PushConsumer + - [x] PushConsumer - [ ] PullConsumer - [ ] MessageListener - - [ ] Concurrently + - [x] Concurrently - [ ] Orderly - [ ] MessageModel - - [ ] CLUSTERING + - [x] CLUSTERING - [ ] BROADCASTING - [ ] OffsetStore - - [ ] RemoteBrokerOffsetStore - - [ ] many actions + - [x] RemoteBrokerOffsetStore - [ ] LocalFileOffsetStore -- [ ] RebalanceService -- [ ] PullMessageService -- [ ] ConsumeMessageService +- [x] RebalanceService +- [x] PullMessageService +- [x] ConsumeMessageService - [ ] AllocateMessageQueueStrategy - - [ ] AllocateMessageQueueAveragely + - [x] AllocateMessageQueueAveragely - [ ] AllocateMessageQueueAveragelyByCircle - [ ] AllocateMessageQueueByConfig - [ ] AllocateMessageQueueByMachineRoom - [ ] Other - - [ ] Config - - [ ] ZIP + - [x] Config + - [x] ZIP - [ ] AllocateMessageQueueStrategy - - [ ] ConsumeFromWhere - - [ ] CONSUME_FROM_LAST_OFFSET - - [ ] CONSUME_FROM_FIRST_OFFSET - - [ ] CONSUME_FROM_TIMESTAMP - - [ ] Retry(sendMessageBack) - - [ ] TimeOut(clearExpiredMessage) - - [ ] ACK(partSuccess) - - [ ] FlowControl(messageCanNotConsume) + - [x] ConsumeFromWhere + - [x] CONSUME_FROM_LAST_OFFSET + - [x] CONSUME_FROM_FIRST_OFFSET + - [x] CONSUME_FROM_TIMESTAMP + - [x] Retry(sendMessageBack) + - [x] TimeOut(clearExpiredMessage) + - [x] ACK(partSuccess) + - [x] FlowControl(messageCanNotConsume) - [ ] ConsumeMessageHook - [ ] filterMessageHookList -## Manager -- [ ] Controller - - [ ] RebalanceController - - [ ] PullMessageController +## Kernel +- [x] Controller + - [x] RebalanceController + - [x] PullMessageController - [ ] Task - - [ ] PollNameServer - - [ ] Heartbeat - - [ ] UpdateTopicRouteInfoFromNameServer + - [x] Heartbeat + - [x] UpdateTopicRouteInfoFromNameServer - [ ] CleanOfflineBroker - - [ ] PersistAllConsumerOffset - - [ ] ClearExpiredMessage(form consumer consumeMessageService) + - [x] PersistAllConsumerOffset + - [x] ClearExpiredMessage(form consumer consumeMessageService) - [ ] UploadFilterClassSource(FromHeartBeat/But Golang Not Easy To do this(Java Source)) -- [ ] ClientRemotingProcessor +- [x] ClientRemotingProcessor - [ ] CHECK_TRANSACTION_STATE - - [ ] NOTIFY_CONSUMER_IDS_CHANGED - - [ ] RESET_CONSUMER_CLIENT_OFFSET - - [ ] GET_CONSUMER_STATUS_FROM_CLIENT - - [ ] GET_CONSUMER_RUNNING_INFO - - [ ] CONSUME_MESSAGE_DIRECTLY + - [x] NOTIFY_CONSUMER_IDS_CHANGED + - [x] RESET_CONSUMER_CLIENT_OFFSET + - [x] GET_CONSUMER_STATUS_FROM_CLIENT + - [x] GET_CONSUMER_RUNNING_INFO + - [x] CONSUME_MESSAGE_DIRECTLY ## Remoting -- [ ] MqClientRequest - - [ ] InvokeSync - - [ ] InvokeAsync - - [ ] InvokeOneWay -- [ ] Serialize - - [ ] JSON - - [ ] ROCKETMQ -- [ ] NamesrvAddrChoosed(HA) +- [x] MqClientRequest + - [x] InvokeSync + - [x] InvokeAsync + - [x] InvokeOneWay +- [x] Serialize + - [x] JSON + - [x] ROCKETMQ +- [x] NamesrvAddrChoosed(HA) - [ ] Other - [ ] VIPChannel - [ ] RPCHook From 1708f8c782657e18c26756b530ed11826f457fb0 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Tue, 29 Aug 2017 21:55:44 +0800 Subject: [PATCH 84/88] docs --- rocketmq-go/README.md | 2 +- rocketmq-go/docs/package.puml | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/rocketmq-go/README.md b/rocketmq-go/README.md index 7e19b13e3..febc30932 100644 --- a/rocketmq-go/README.md +++ b/rocketmq-go/README.md @@ -8,7 +8,7 @@ examples: # Go RocketMQ Client's Arch -![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/svg/ZLNDRk8m4BxdANokL1w06qNzOTK4Dg7g4RdEGACcDjWE4jjotvNsaVfEVOKTsp7vPIa7bF5yy-MRySo4vLGf8WLx0rtiLXin2dVJF0EkGyhf1kHxe43kCmQ9fXg2Oy1w4OiopqDG2k5JmRKKUMxYJjYAO3J9Sy6GfCB-BA54LeadcyFx8BDJSaUP5X8XnGxvLLc0NDAN7D1UI96MpDlT5uOd_9aiQg0dwW39CI0zJYiAynjmkimUCwM16p83gJ0I2g4plXd5rOFd8OHaV2_U83bmLbiJrJBd79xf0UtJI_k4eYWeJmqZAvKMnGFG52IQ4dObA3qLALXBRR4kuCm1XKuPrcwTRQm-JWj8v7wIfeODuSO_MpXrIbFE84A8KpVC5Zi9M6U6HHBIyXej3B8zU8K4zLS94_qAf03zAjAHmzxVBbVJUPGyXRVnAbbEba_9wYwUXwlfu-msMWw0ugSecaNtgrbqDtVkqSYH7Ur_Hsa2CgDvzWla0-gmJKpSxuSIlFRwqzYsQhZhU8v149YAgIrbu7i3-oMuE3Jawlhw_02C6G8f5Zmu2x44TJT_Fy8FIXrHirX8j_z94-cZMqYJuMGnUvPqkqNNc5mAcA_N2dI2gk0Rw1XUQ6uwxnlOwh2gT-9Ect6u4OQkpDocJnXJ8S9Sp_0Sr-KmGSrKX2kmHLbdfp0z_wd4JGy6N0hsaBeoFUgsgw5omFk_TMtXmrKreGdmj3g-eUpDDr85kH3SLNKUsLctxKlqzqJCDRRlYjZcwApDFig0cgkmgb6VQNZ33S6lR74wMoRej5zZLjijM7sIRUw3--CikZNoWLrGGkEA5LiYrkTQceo_Fd-Rhz07FIZs8UmNwElpps0i6NFGNvz_bSyEdt4aE31pwjUx__Jy0m00) +![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/svg/bLNDRk8m4Bxp53vNAW_0ZMNxGzM4Dg7Q8tATWKOuDiGE4ajxtvNsaVfEVOMT-8UGO6YR0s8ytum_VvuPvq93EQSC25tb_UZnePHPnQ3SzNfllRpZ74cXgkAKa-SWz5WeJGK3X13x8RGdQ05gvR0xu-a647bvIPPG2k3zQ6rz4HLq0QN3Usg7o9ymfnNNZTFdTap3s4gzUC3Ny9plyNWQngbI9xZkUdOzrLG3WblOEb44IY4Gy-bISkx4RC66d4k6uGxwM445gIvnPR7TTS5szslAj2pHMwmzqobzve2sLyRQdUYW9hipg4O9UurQacSWcHHdKkzBYhprI52ykbgEVFRqzxdI87oqWvHGI9sBHJznVtmQP9IBZLn2kXMiiOQgPKu4_A9RjFTx7YUKHK55LcTabTAok0MQcN8fOLs1qYEPWR7K2WK1QmOOMcOesZnV67vrBwJbPfXbflYOxO87oyxsGemGoowSBR4KK5Y68abt5OXYa6_0CoJ8tnsOw1SW1UWNcVKjzkPgEXjDulGfkafcwNGuVKfdryaacetknrVnD01Adk9u7D_Cnl7rnBhxVPlipVnscAPGvfJdlw6yHHxSrmj-TqgAnkjN4ycMeADrnuFjTZ0zR6bWjVJETHIvQ9gurHu3XXYAAjkkRtUkQVrWlfFeLdA32krgw0GRhStdAx5D-xLZW-aFTV3qoO1yBt0odolGAPA1RyScy4ov6UxVZNHhxYgm9ZRvSPOl2EP6JoDMPtl_f0ILQeBQRvYlmiEoCFLmOYorh-fGq3h8Pp3EHh5FN0TP0jvpBuY1X_MjGrCGcLl-GTdIAsJF_4bibzHfP6Uuj_NrCf9QuhQKnDnKPqWVeoxspMa7hvFeE5DudeRW5qQRpaopbO7SfSOPArT4payL0HbylF-fdw5NkbNiLPedwEFzRp2M30JeZ_T_oc-8_x681B1wrSzt__Ry1m00) # Go RocketMQ Client's Roadmap [Go RocketMQ Client's Roadmap](https://github.com/StyleTang/incubator-rocketmq-externals/blob/master/rocketmq-go/docs/roadmap.md) diff --git a/rocketmq-go/docs/package.puml b/rocketmq-go/docs/package.puml index 7af865ea8..ebeacdf66 100644 --- a/rocketmq-go/docs/package.puml +++ b/rocketmq-go/docs/package.puml @@ -15,8 +15,6 @@ namespace api{ MQProducer o-- kernel.DefaultMQProducer MQConsumer o-- kernel.DefaultMQPushConsumer MQConsumer o-- kernel.DefaultMQPullConsumer - MQConsumer o-- kernel.MessageListener - } class kernel.MqClientManager{ From daa4462df6872a51283c2caa4ae8853489439f99 Mon Sep 17 00:00:00 2001 From: StyleTang Date: Thu, 7 Sep 2017 23:16:44 +0800 Subject: [PATCH 85/88] update readme's svg --- rocketmq-go/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/rocketmq-go/README.md b/rocketmq-go/README.md index febc30932..f85a8e9d9 100644 --- a/rocketmq-go/README.md +++ b/rocketmq-go/README.md @@ -8,7 +8,7 @@ examples: # Go RocketMQ Client's Arch -![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/svg/bLNDRk8m4Bxp53vNAW_0ZMNxGzM4Dg7Q8tATWKOuDiGE4ajxtvNsaVfEVOMT-8UGO6YR0s8ytum_VvuPvq93EQSC25tb_UZnePHPnQ3SzNfllRpZ74cXgkAKa-SWz5WeJGK3X13x8RGdQ05gvR0xu-a647bvIPPG2k3zQ6rz4HLq0QN3Usg7o9ymfnNNZTFdTap3s4gzUC3Ny9plyNWQngbI9xZkUdOzrLG3WblOEb44IY4Gy-bISkx4RC66d4k6uGxwM445gIvnPR7TTS5szslAj2pHMwmzqobzve2sLyRQdUYW9hipg4O9UurQacSWcHHdKkzBYhprI52ykbgEVFRqzxdI87oqWvHGI9sBHJznVtmQP9IBZLn2kXMiiOQgPKu4_A9RjFTx7YUKHK55LcTabTAok0MQcN8fOLs1qYEPWR7K2WK1QmOOMcOesZnV67vrBwJbPfXbflYOxO87oyxsGemGoowSBR4KK5Y68abt5OXYa6_0CoJ8tnsOw1SW1UWNcVKjzkPgEXjDulGfkafcwNGuVKfdryaacetknrVnD01Adk9u7D_Cnl7rnBhxVPlipVnscAPGvfJdlw6yHHxSrmj-TqgAnkjN4ycMeADrnuFjTZ0zR6bWjVJETHIvQ9gurHu3XXYAAjkkRtUkQVrWlfFeLdA32krgw0GRhStdAx5D-xLZW-aFTV3qoO1yBt0odolGAPA1RyScy4ov6UxVZNHhxYgm9ZRvSPOl2EP6JoDMPtl_f0ILQeBQRvYlmiEoCFLmOYorh-fGq3h8Pp3EHh5FN0TP0jvpBuY1X_MjGrCGcLl-GTdIAsJF_4bibzHfP6Uuj_NrCf9QuhQKnDnKPqWVeoxspMa7hvFeE5DudeRW5qQRpaopbO7SfSOPArT4payL0HbylF-fdw5NkbNiLPedwEFzRp2M30JeZ_T_oc-8_x681B1wrSzt__Ry1m00) +![Go RocketMQ Client's Arch](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/package.puml) # Go RocketMQ Client's Roadmap [Go RocketMQ Client's Roadmap](https://github.com/StyleTang/incubator-rocketmq-externals/blob/master/rocketmq-go/docs/roadmap.md) @@ -47,7 +47,7 @@ examples: put them into local memory(BrokerAddrTable/TopicPublishInfoTable/TopicSubscribeInfoTable/TopicRouteTable) -![update-topic-routeInfo-timing-diagram](http://www.plantuml.com/plantuml/proxy?src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/go-client-detail-design/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml) +![update-topic-routeInfo-timing-diagram](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/update-topic-routeInfo-timing-diagram.puml) ### heartbeat: @@ -57,7 +57,7 @@ examples: (only broker know the distribution of the consumers we can rebalance) -![heartbeat-timing-diagram](http://www.plantuml.com/plantuml/proxy?src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/go-client-detail-design/rocketmq-go/docs/heartbeat-timing-diagram.puml) +![heartbeat-timing-diagram](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/heartbeat-timing-diagram.puml) ### rebalance @@ -69,7 +69,7 @@ examples: enqueue pull message request (chan *model.PullRequest) - ![rebalance-timing-diagram](http://www.plantuml.com/plantuml/svg/XL7DQiCm3BxdANJSO7s170gZi55OsTOMTdPi9J4uaclBbBtz72TBCoZiPblVdpuViL5EaKROR8-_vxhb0AXq3yBUQh04fzH47QmNorGjmCtsSDavYoHrQydic68QCEpDcutoKCXFNU3a7-zoNb7E8sOMRt1FBK-qFuHdvrWhmGF6g3hyJ9Zm926_TD-rceTmjT93le6UOu0kDZ260KMc32yZEM_KyjhXjdho9ejz1DRPh3YTLUDoiWLIAIUmk2eWlCwg3Jgc3gItSGcnTdblsuXo4WvOQnvyoaR9kPV0mrUF0QiLO5LJG6M0o-XkZKYJlSzQC4mTGS3y6AL25n76pyb99nYn_9lqreT1XtdDWlIhLYeaymC0) + ![rebalance-timing-diagram](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/rebalance-timing-diagram.puml) ### pullMessage @@ -80,11 +80,11 @@ examples: #### pullMessage - ![pull-message-timing-diagram](http://www.plantuml.com/plantuml/svg/dPHHZzem3CVV-odynYO1Uwyc9AvNRfCet5hG7b7Qkb2HDbpY7BTlFoSK1bgPnDxYsFdPVyT9YhcGeYqGHfFql0v5HQX1Ntn7X2qI24Z4uMk2neWj_h1eSVYgLS6sDoP1UaM3LojbYcyM3KMg9QsaP6W8aDkwPDRXZnz4Mx9DGEfwsrE3Viu_4XntjKIGIXs0n1w1TWWrOGEg-elkdAtVxMJrfnjDdhJQemvB1KOrIBkwtOB85TTSINM4uXJwfJbHeDJgC1wFeQv0xOV2_6eBdmNE0PLM3UGU6fpOBAatTrW8FfUBOZ_cx4wC1saqLb8W9C5ikLuytokSryOssCdBKB_NVCF6varDdQyxTO_GNnL-O6495_X1Lm51RxhHP5uRmXPrmgrJPVYf2uizH6cMHyNETT7jUf5TepxpjCZoxEcmhWpE2xupj-ZWrhodNoDPtLuI6X9aJJ1mtOoMYsoTn9ji7KLnbWM3EvBwmS40fK58upDcFbt5AU-svRtUD6-HhB6bq72Grru9dk3oCYlkxeSyIOPgrkkSG_TOwfQVIsEsJ-oU-HF1GwNspG1KV1ctpCUW6XlrVhf1OmltDrnak4VklX7dOpm_nyeW3UsX58IT5VZkBPQRHVnpasGl3ytavN0oNKNVukV_12ndionURRxFv_7BTFuWe2r_0m00) + ![pull-message-timing-diagram](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/pull-message-timing-diagram.puml) #### consumeMessage(submitConsumeRequest) - ![consume-message-activity](http://www.plantuml.com/plantuml/svg/VL5FQy8m5B_tKxoR4PdWjKvaAnu4Ey7Eqv0HsnSDQvCkBztcjzzBxT3si1m2UVdzJSXBvwEuGY9vmeqc3mlmJXfIrbNfTVng4skegN0UnGu38htXqQ0JT_pnFD8A1EEc7IlpqZS4YmMCakrBjazNxza-ILPPDbgEmP_HWBWWZIE0MEOVQrFW3ti4XQVkE8-m90HXx13rCECxKsWLnTJaEUVeih4XL7IYjnjw0hC3Lm3DLt_32VE_pxfaSGsFBMDQeZdvyxAr8XP_Pl0EgIb3zJ3eBC9Sj1xwmBK1j19z_B3VTGsJJcLTCwqd5VhU24fomk9VVFi6lBTbeZYtLQNzLYbgFynXPRymw_cAGGpMuMH7fdKVjyFF1icBldk0DNKXFLxLrsPZSrccxE0kujwNaUHj_Gi0) + ![consume-message-activity](http://www.plantuml.com/plantuml/proxy?fmt=svg&src=https://raw.githubusercontent.com/StyleTang/incubator-rocketmq-externals/master/rocketmq-go/docs/consume-message-activity.puml) ### cleanExpireMsg (Non-major) From ab37b78c3fe9abb0838149361e4290fc7aa734af Mon Sep 17 00:00:00 2001 From: lilylee Date: Thu, 25 Jan 2018 15:47:25 +0800 Subject: [PATCH 86/88] format err %s --> %v --- rocketmq-go/example/simple_producer.go | 2 +- rocketmq-go/example/simple_producer_consumer.go | 2 +- rocketmq-go/test/test_consume_message_retry.go | 2 +- rocketmq-go/test/test_delay_message_send_and_receive.go | 2 +- .../test/test_rocketmq_serialize_message_send_and_receive.go | 2 +- rocketmq-go/test/test_send_and_receive_compress_message.go | 2 +- rocketmq-go/test/test_send_and_receive_tag_message.go | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/rocketmq-go/example/simple_producer.go b/rocketmq-go/example/simple_producer.go index 84f4b2dbd..10180bd07 100644 --- a/rocketmq-go/example/simple_producer.go +++ b/rocketmq-go/example/simple_producer.go @@ -43,5 +43,5 @@ func main() { message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) - glog.Infof("test sendMessageResult result=[%s] err=[%s]", result.String(), err) + glog.Infof("test sendMessageResult result=[%s] err=[%v]", result.String(), err) } diff --git a/rocketmq-go/example/simple_producer_consumer.go b/rocketmq-go/example/simple_producer_consumer.go index 052d7c6c9..fa2fcff37 100644 --- a/rocketmq-go/example/simple_producer_consumer.go +++ b/rocketmq-go/example/simple_producer_consumer.go @@ -61,6 +61,6 @@ func main() { message.SetTopic(testTopic) message.SetBody([]byte("hello World")) result, err := producer.Send(message) - glog.Infof("test sendMessageResult result=[%s] err=[%s]", result.String(), err) + glog.Infof("test sendMessageResult result=[%s] err=[%v]", result.String(), err) } } diff --git a/rocketmq-go/test/test_consume_message_retry.go b/rocketmq-go/test/test_consume_message_retry.go index 7c1a8f629..938fe5753 100644 --- a/rocketmq-go/test/test_consume_message_retry.go +++ b/rocketmq-go/test/test_consume_message_retry.go @@ -66,7 +66,7 @@ func main() { message.SetBody([]byte(testMessageBody)) message.SetTag(tag) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult messageId=[%s] err=[%v]", result.MsgID(), err) for i := 0; i < 3; i++ { select { case <-chResult: diff --git a/rocketmq-go/test/test_delay_message_send_and_receive.go b/rocketmq-go/test/test_delay_message_send_and_receive.go index e57ffbbad..4800e9d06 100644 --- a/rocketmq-go/test/test_delay_message_send_and_receive.go +++ b/rocketmq-go/test/test_delay_message_send_and_receive.go @@ -68,7 +68,7 @@ func main() { result, err := producer.Send(message) startTime = time.Now() messageId = result.MsgID() - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult messageId=[%s] err=[%v]", result.MsgID(), err) select { case <-chResult: case <-time.After(time.Second * 30): diff --git a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go index 78ab8d312..d70e049cc 100644 --- a/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go +++ b/rocketmq-go/test/test_rocketmq_serialize_message_send_and_receive.go @@ -61,7 +61,7 @@ func main() { message.SetBody([]byte(messageBody)) message.SetTag(tag) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult messageId=[%s] err=[%v]", result.MsgID(), err) } for i := 0; i < messageCount; i++ { select { diff --git a/rocketmq-go/test/test_send_and_receive_compress_message.go b/rocketmq-go/test/test_send_and_receive_compress_message.go index e80c2b870..57b92a2e6 100644 --- a/rocketmq-go/test/test_send_and_receive_compress_message.go +++ b/rocketmq-go/test/test_send_and_receive_compress_message.go @@ -67,7 +67,7 @@ func main() { message.SetBody([]byte(bigMessageBody)) message.SetTag("compress_message_test") result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult messageId=[%s] err=[%v]", result.MsgID(), err) select { case <-chResult: case <-time.After(time.Second * 30): diff --git a/rocketmq-go/test/test_send_and_receive_tag_message.go b/rocketmq-go/test/test_send_and_receive_tag_message.go index b1155b31f..6125701bd 100644 --- a/rocketmq-go/test/test_send_and_receive_tag_message.go +++ b/rocketmq-go/test/test_send_and_receive_tag_message.go @@ -60,7 +60,7 @@ func main() { message.SetBody([]byte("Hello world")) message.SetTag("tag" + util.IntToString(i)) result, err := producer.Send(message) - glog.Infof("test sendMessageResult messageId=[%s] err=[%s]", result.MsgID(), err) + glog.Infof("test sendMessageResult messageId=[%s] err=[%v]", result.MsgID(), err) } for i := 0; i < 3; i++ { select { From fbaef55aa7c4d83ea16e512f10b3e4c5bfd8cdc8 Mon Sep 17 00:00:00 2001 From: lilylee Date: Thu, 25 Jan 2018 15:48:46 +0800 Subject: [PATCH 87/88] isInternetIpv4 add 172. ... --- rocketmq-go/util/ip.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-go/util/ip.go b/rocketmq-go/util/ip.go index 6599842f5..cbfb3fc98 100644 --- a/rocketmq-go/util/ip.go +++ b/rocketmq-go/util/ip.go @@ -69,7 +69,7 @@ func getIp() (ip net.IP) { return } func isIntranetIpv4(ip string) bool { - if strings.HasPrefix(ip, "192.168.") || strings.HasPrefix(ip, "169.254.") { + if strings.HasPrefix(ip, "192.168.") || strings.HasPrefix(ip, "169.254.") || strings.HasPrefix(ip, "172.") { return true } return false From 66bd9b86746c0263014ad86cb63490522801a793 Mon Sep 17 00:00:00 2001 From: lily-lee Date: Wed, 2 May 2018 15:40:30 +0800 Subject: [PATCH 88/88] update: util/isIntranetIpv4, add "10." --- rocketmq-go/util/ip.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rocketmq-go/util/ip.go b/rocketmq-go/util/ip.go index cbfb3fc98..83e849980 100644 --- a/rocketmq-go/util/ip.go +++ b/rocketmq-go/util/ip.go @@ -69,7 +69,7 @@ func getIp() (ip net.IP) { return } func isIntranetIpv4(ip string) bool { - if strings.HasPrefix(ip, "192.168.") || strings.HasPrefix(ip, "169.254.") || strings.HasPrefix(ip, "172.") { + if strings.HasPrefix(ip, "192.168.") || strings.HasPrefix(ip, "169.254.") || strings.HasPrefix(ip, "172.") || strings.HasPrefix(ip, "10.") { return true } return false