From 2ed6bc560ce4f03062756519347ec929a5dfb564 Mon Sep 17 00:00:00 2001 From: matthew Date: Mon, 27 Jan 2025 11:27:41 +0800 Subject: [PATCH] Add new command to check async task status in broker. --- .../broker/AdminAsyncTaskManager.java | 66 + .../processor/AdminBrokerProcessor.java | 38 + .../rocketmq/client/impl/MQClientAPIImpl.java | 2205 ++++++++--------- .../org/apache/rocketmq/common/AsyncTask.java | 81 + .../apache/rocketmq/common/TaskStatus.java | 46 + .../remoting/protocol/RequestCode.java | 1 + .../CheckAsyncTaskStatusRequestHeader.java | 46 + .../CheckAsyncTaskStatusResponseHeader.java | 69 + .../tools/admin/DefaultMQAdminExt.java | 7 + .../tools/admin/DefaultMQAdminExtImpl.java | 7 + .../rocketmq/tools/admin/MQAdminExt.java | 4 + .../stats/CheckAsyncTaskStatusSubCommand.java | 120 + 12 files changed, 1577 insertions(+), 1113 deletions(-) create mode 100644 broker/src/main/java/org/apache/rocketmq/broker/AdminAsyncTaskManager.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/AsyncTask.java create mode 100644 common/src/main/java/org/apache/rocketmq/common/TaskStatus.java create mode 100644 remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusRequestHeader.java create mode 100644 remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusResponseHeader.java create mode 100644 tools/src/main/java/org/apache/rocketmq/tools/command/stats/CheckAsyncTaskStatusSubCommand.java diff --git a/broker/src/main/java/org/apache/rocketmq/broker/AdminAsyncTaskManager.java b/broker/src/main/java/org/apache/rocketmq/broker/AdminAsyncTaskManager.java new file mode 100644 index 00000000000..859311fe9f1 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/AdminAsyncTaskManager.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.broker; + +import org.apache.rocketmq.common.AsyncTask; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +public class AdminAsyncTaskManager { + + private static final Map ASYNC_TASK_MAP = new ConcurrentHashMap<>(); + + private static final Map> TASK_NAME_TO_IDS_MAP = new ConcurrentHashMap<>(); + + public static String createTask(String taskName) { + String taskId = UUID.randomUUID().toString(); + ASYNC_TASK_MAP.put(taskId, new AsyncTask(taskName, taskId)); + TASK_NAME_TO_IDS_MAP.computeIfAbsent(taskName, k -> new ArrayList<>()).add(taskId); + return taskId; + } + + public static List getTaskIdsByName(String taskName) { + return TASK_NAME_TO_IDS_MAP.getOrDefault(taskName, Collections.emptyList()); + } + + public static AsyncTask getTaskStatus(String taskId) { + return ASYNC_TASK_MAP.get(taskId); + } + + public static void updateTaskStatus(String taskId, int status, String result) { + AsyncTask task = ASYNC_TASK_MAP.get(taskId); + if (task != null) { + task.setStatus(status); + task.setResult(result); + } + } + + public static void removeTask(String taskId) { + AsyncTask task = ASYNC_TASK_MAP.remove(taskId); + if (task != null) { + TASK_NAME_TO_IDS_MAP.computeIfPresent(task.getTaskName(), (k, v) -> { + v.remove(taskId); + return v.isEmpty() ? null : v; + }); + } + } +} \ No newline at end of file diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java index 6fb7584aa9b..b227729f676 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/AdminBrokerProcessor.java @@ -55,6 +55,7 @@ import org.apache.rocketmq.auth.authorization.exception.AuthorizationException; import org.apache.rocketmq.auth.authorization.model.Acl; import org.apache.rocketmq.auth.authorization.model.Resource; +import org.apache.rocketmq.broker.AdminAsyncTaskManager; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.auth.converter.AclConverter; import org.apache.rocketmq.broker.auth.converter.UserConverter; @@ -68,6 +69,7 @@ import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin; import org.apache.rocketmq.broker.subscription.SubscriptionGroupManager; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageUtil; +import org.apache.rocketmq.common.AsyncTask; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; @@ -77,6 +79,7 @@ import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.PlainAccessConfig; +import org.apache.rocketmq.common.TaskStatus; import org.apache.rocketmq.common.TopicAttributes; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.UnlockCallback; @@ -145,6 +148,8 @@ import org.apache.rocketmq.remoting.protocol.body.TopicList; import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody; import org.apache.rocketmq.remoting.protocol.body.UserInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusResponseHeader; import org.apache.rocketmq.remoting.protocol.header.CheckRocksdbCqWriteProgressRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CloneGroupOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; @@ -408,6 +413,8 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, return this.listAcl(ctx, request); case RequestCode.POP_ROLLBACK: return this.transferPopToFsStore(ctx, request); + case RequestCode.CHECK_ASYNC_TASK_STATUS: + return this.checkAsyncTaskStatus(ctx, request); default: return getUnknownCmdResponse(ctx, request); } @@ -480,11 +487,14 @@ private RemotingCommand updateAndGetGroupForbidden(ChannelHandlerContext ctx, Re private RemotingCommand checkRocksdbCqWriteProgress(ChannelHandlerContext ctx, RemotingCommand request) { CheckRocksdbCqWriteResult result = new CheckRocksdbCqWriteResult(); result.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_IN_PROGRESS.getValue()); + String taskId = AdminAsyncTaskManager.createTask("checkRocksdbCqWriteProgress"); Runnable runnable = () -> { try { CheckRocksdbCqWriteResult checkResult = doCheckRocksdbCqWriteProgress(ctx, request); + AdminAsyncTaskManager.updateTaskStatus(taskId, TaskStatus.SUCCESS.getValue(), JSON.toJSONString(checkResult)); LOGGER.info("checkRocksdbCqWriteProgress result: {}", JSON.toJSONString(checkResult)); } catch (Exception e) { + AdminAsyncTaskManager.updateTaskStatus(taskId, TaskStatus.ERROR.getValue(), e.getMessage()); LOGGER.error("checkRocksdbCqWriteProgress error", e); } }; @@ -3545,4 +3555,32 @@ private RemotingCommand transferPopToFsStore(ChannelHandlerContext ctx, Remoting } return response; } + + private RemotingCommand checkAsyncTaskStatus(ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException { + final CheckAsyncTaskStatusRequestHeader requestHeader = request.decodeCommandCustomHeader(CheckAsyncTaskStatusRequestHeader.class); + try { + List taskIds = AdminAsyncTaskManager.getTaskIdsByName(requestHeader.getTaskName()); + if (taskIds == null || taskIds.isEmpty()) { + throw new RemotingCommandException("taskId not found"); + } + + List result = new ArrayList<>(); + for (String taskId : taskIds) { + AsyncTask taskStatus = AdminAsyncTaskManager.getTaskStatus(taskId); + result.add(taskStatus); + + if (taskStatus.getStatus() == TaskStatus.SUCCESS.getValue()) { + AdminAsyncTaskManager.removeTask(taskId); + } + } + + RemotingCommand response = RemotingCommand.createResponseCommand(CheckAsyncTaskStatusResponseHeader.class); + response.setCode(ResponseCode.SUCCESS); + response.setBody(JSON.toJSONBytes(result)); + return response; + } catch (Exception e) { + LOGGER.error("checkAsyncTaskStatus error", e); + return RemotingCommand.createResponseCommand(ResponseCode.SYSTEM_ERROR, e.getMessage()); + } + } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 7d4b51cfc5f..4edf63904d9 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java @@ -55,6 +55,7 @@ import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.client.producer.SendStatus; import org.apache.rocketmq.client.rpchook.NamespaceRpcHook; +import org.apache.rocketmq.common.AsyncTask; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; import org.apache.rocketmq.common.MQVersion; @@ -149,6 +150,7 @@ import org.apache.rocketmq.remoting.protocol.header.AddBrokerRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeResponseHeader; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CheckRocksdbCqWriteProgressRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CloneGroupOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ConsumeMessageDirectlyResultRequestHeader; @@ -253,35 +255,40 @@ import static org.apache.rocketmq.remoting.protocol.RemotingSysResponseCode.SUCCESS; public class MQClientAPIImpl implements NameServerUpdateCallback, StartAndShutdown { + private final static Logger log = LoggerFactory.getLogger(MQClientAPIImpl.class); - private static boolean sendSmartMsg = - Boolean.parseBoolean(System.getProperty("org.apache.rocketmq.client.sendSmartMsg", "true")); - + + private static boolean sendSmartMsg = Boolean.parseBoolean( + System.getProperty("org.apache.rocketmq.client.sendSmartMsg", "true")); + static { System.setProperty(RemotingCommand.REMOTING_VERSION_KEY, Integer.toString(MQVersion.CURRENT_VERSION)); } - + private final RemotingClient remotingClient; + private final TopAddressing topAddressing; + private final ClientRemotingProcessor clientRemotingProcessor; + private String nameSrvAddr = null; + private ClientConfig clientConfig; - + public MQClientAPIImpl(final NettyClientConfig nettyClientConfig, - final ClientRemotingProcessor clientRemotingProcessor, - RPCHook rpcHook, final ClientConfig clientConfig) { + final ClientRemotingProcessor clientRemotingProcessor, RPCHook rpcHook, final ClientConfig clientConfig) { this(nettyClientConfig, clientRemotingProcessor, rpcHook, clientConfig, null); } - + public MQClientAPIImpl(final NettyClientConfig nettyClientConfig, - final ClientRemotingProcessor clientRemotingProcessor, - RPCHook rpcHook, final ClientConfig clientConfig, final ChannelEventListener channelEventListener) { + final ClientRemotingProcessor clientRemotingProcessor, RPCHook rpcHook, final ClientConfig clientConfig, + final ChannelEventListener channelEventListener) { this.clientConfig = clientConfig; topAddressing = new DefaultTopAddressing(MixAll.getWSAddr(), clientConfig.getUnitName()); topAddressing.registerChangeCallBack(this); this.remotingClient = new NettyRemotingClient(nettyClientConfig, channelEventListener); this.clientRemotingProcessor = clientRemotingProcessor; - + this.remotingClient.registerRPCHook(new NamespaceRpcHook(clientConfig)); // Inject stream rpc hook first to make reserve field signature if (clientConfig.isEnableStreamRequestType()) { @@ -290,28 +297,33 @@ public MQClientAPIImpl(final NettyClientConfig nettyClientConfig, this.remotingClient.registerRPCHook(rpcHook); this.remotingClient.registerRPCHook(new DynamicalExtFieldRPCHook()); this.remotingClient.registerProcessor(RequestCode.CHECK_TRANSACTION_STATE, this.clientRemotingProcessor, null); - - this.remotingClient.registerProcessor(RequestCode.NOTIFY_CONSUMER_IDS_CHANGED, this.clientRemotingProcessor, null); - - this.remotingClient.registerProcessor(RequestCode.RESET_CONSUMER_CLIENT_OFFSET, this.clientRemotingProcessor, null); - - this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_STATUS_FROM_CLIENT, this.clientRemotingProcessor, null); - - this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_RUNNING_INFO, this.clientRemotingProcessor, null); - + + this.remotingClient.registerProcessor(RequestCode.NOTIFY_CONSUMER_IDS_CHANGED, this.clientRemotingProcessor, + null); + + this.remotingClient.registerProcessor(RequestCode.RESET_CONSUMER_CLIENT_OFFSET, this.clientRemotingProcessor, + null); + + this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_STATUS_FROM_CLIENT, this.clientRemotingProcessor, + null); + + this.remotingClient.registerProcessor(RequestCode.GET_CONSUMER_RUNNING_INFO, this.clientRemotingProcessor, + null); + this.remotingClient.registerProcessor(RequestCode.CONSUME_MESSAGE_DIRECTLY, this.clientRemotingProcessor, null); - - this.remotingClient.registerProcessor(RequestCode.PUSH_REPLY_MESSAGE_TO_CLIENT, this.clientRemotingProcessor, null); + + this.remotingClient.registerProcessor(RequestCode.PUSH_REPLY_MESSAGE_TO_CLIENT, this.clientRemotingProcessor, + null); } - + public List getNameServerAddressList() { return this.remotingClient.getNameServerAddressList(); } - + public RemotingClient getRemotingClient() { return remotingClient; } - + public String fetchNameServerAddr() { try { String addrs = this.topAddressing.fetchNSAddr(); @@ -328,7 +340,7 @@ public String fetchNameServerAddr() { } return nameSrvAddr; } - + @Override public String onNameServerAddressChange(String namesrvAddress) { if (namesrvAddress != null) { @@ -341,58 +353,60 @@ public String onNameServerAddressChange(String namesrvAddress) { } return nameSrvAddr; } - + public void updateNameServerAddressList(final String addrs) { String[] addrArray = addrs.split(";"); List list = Arrays.asList(addrArray); this.remotingClient.updateNameServerAddressList(list); } - + public void start() { this.remotingClient.start(); } - + public void shutdown() { this.remotingClient.shutdown(); } - + public Set queryAssignment(final String addr, final String topic, - final String consumerGroup, final String clientId, final String strategyName, - final MessageModel messageModel, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + final String consumerGroup, final String clientId, final String strategyName, + final MessageModel messageModel, final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { QueryAssignmentRequestBody requestBody = new QueryAssignmentRequestBody(); requestBody.setTopic(topic); requestBody.setConsumerGroup(consumerGroup); requestBody.setClientId(clientId); requestBody.setMessageModel(messageModel); requestBody.setStrategyName(strategyName); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_ASSIGNMENT, null); request.setBody(requestBody.encode()); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { - QueryAssignmentResponseBody queryAssignmentResponseBody = QueryAssignmentResponseBody.decode(response.getBody(), QueryAssignmentResponseBody.class); + QueryAssignmentResponseBody queryAssignmentResponseBody = QueryAssignmentResponseBody.decode( + response.getBody(), QueryAssignmentResponseBody.class); return queryAssignmentResponseBody.getMessageQueueAssignments(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public void createSubscriptionGroup(final String addr, final SubscriptionGroupConfig config, - final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP, null); - + final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP, + null); + byte[] body = RemotingSerializable.encode(config); request.setBody(body); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -401,32 +415,33 @@ public void createSubscriptionGroup(final String addr, final SubscriptionGroupCo default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); - + } - + public void createSubscriptionGroupList(final String address, final List configs, - final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP_LIST, null); + final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { + RemotingCommand request = RemotingCommand.createRequestCommand( + RequestCode.UPDATE_AND_CREATE_SUBSCRIPTIONGROUP_LIST, null); SubscriptionGroupList requestBody = new SubscriptionGroupList(configs); request.setBody(requestBody.encode()); - + RemotingCommand response = this.remotingClient.invokeSync( - MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), address), request, timeoutMillis); + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), address), request, timeoutMillis); assert response != null; if (response.getCode() == ResponseCode.SUCCESS) { return; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void createTopic(final String addr, final String defaultTopic, final TopicConfig topicConfig, - final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { Validators.checkTopicConfig(topicConfig); - + CreateTopicRequestHeader requestHeader = new CreateTopicRequestHeader(); requestHeader.setTopic(topicConfig.getTopicName()); requestHeader.setDefaultTopic(defaultTopic); @@ -437,11 +452,12 @@ public void createTopic(final String addr, final String defaultTopic, final Topi requestHeader.setTopicSysFlag(topicConfig.getTopicSysFlag()); requestHeader.setOrder(topicConfig.isOrder()); requestHeader.setAttributes(AttributeParser.parseToString(topicConfig.getAttributes())); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_TOPIC, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_TOPIC, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -450,31 +466,31 @@ public void createTopic(final String addr, final String defaultTopic, final Topi default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void createTopicList(final String address, final List topicConfigList, final long timeoutMillis) - throws InterruptedException, RemotingException, MQClientException { + throws InterruptedException, RemotingException, MQClientException { CreateTopicListRequestHeader requestHeader = new CreateTopicListRequestHeader(); CreateTopicListRequestBody requestBody = new CreateTopicListRequestBody(topicConfigList); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_TOPIC_LIST, requestHeader); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_TOPIC_LIST, + requestHeader); request.setBody(requestBody.encode()); - + RemotingCommand response = this.remotingClient.invokeSync( - MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), address), request, timeoutMillis); + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), address), request, timeoutMillis); assert response != null; if (response.getCode() == ResponseCode.SUCCESS) { return; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void createPlainAccessConfig(final String addr, final PlainAccessConfig plainAccessConfig, - final long timeoutMillis) - throws RemotingException, InterruptedException, MQClientException { + final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { CreateAccessConfigRequestHeader requestHeader = new CreateAccessConfigRequestHeader(); requestHeader.setAccessKey(plainAccessConfig.getAccessKey()); requestHeader.setSecretKey(plainAccessConfig.getSecretKey()); @@ -484,11 +500,12 @@ public void createPlainAccessConfig(final String addr, final PlainAccessConfig p requestHeader.setWhiteRemoteAddress(plainAccessConfig.getWhiteRemoteAddress()); requestHeader.setTopicPerms(UtilAll.join(plainAccessConfig.getTopicPerms(), ",")); requestHeader.setGroupPerms(UtilAll.join(plainAccessConfig.getGroupPerms(), ",")); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_ACL_CONFIG, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_ACL_CONFIG, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -497,19 +514,19 @@ public void createPlainAccessConfig(final String addr, final PlainAccessConfig p default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void deleteAccessConfig(final String addr, final String accessKey, final long timeoutMillis) - throws RemotingException, InterruptedException, MQClientException { + throws RemotingException, InterruptedException, MQClientException { DeleteAccessConfigRequestHeader requestHeader = new DeleteAccessConfigRequestHeader(); requestHeader.setAccessKey(accessKey); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_ACL_CONFIG, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -518,21 +535,22 @@ public void deleteAccessConfig(final String addr, final String accessKey, final default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void updateGlobalWhiteAddrsConfig(final String addr, final String globalWhiteAddrs, String aclFileFullPath, - final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { UpdateGlobalWhiteAddrsConfigRequestHeader requestHeader = new UpdateGlobalWhiteAddrsConfigRequestHeader(); requestHeader.setGlobalWhiteAddrs(globalWhiteAddrs); requestHeader.setAclFileFullPath(aclFileFullPath); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_GLOBAL_WHITE_ADDRS_CONFIG, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_GLOBAL_WHITE_ADDRS_CONFIG, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -541,31 +559,34 @@ public void updateGlobalWhiteAddrsConfig(final String addr, final String globalW default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr, - final long timeoutMillis) throws RemotingCommandException, InterruptedException, RemotingTimeoutException, - RemotingSendRequestException, RemotingConnectException, MQBrokerException { + + public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr, final long timeoutMillis) + throws RemotingCommandException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_ACL_INFO, null); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - GetBrokerAclConfigResponseHeader responseHeader = - (GetBrokerAclConfigResponseHeader) response.decodeCommandCustomHeader(GetBrokerAclConfigResponseHeader.class); - + GetBrokerAclConfigResponseHeader responseHeader = (GetBrokerAclConfigResponseHeader) response.decodeCommandCustomHeader( + GetBrokerAclConfigResponseHeader.class); + ClusterAclVersionInfo clusterAclVersionInfo = new ClusterAclVersionInfo(); clusterAclVersionInfo.setClusterName(responseHeader.getClusterName()); clusterAclVersionInfo.setBrokerName(responseHeader.getBrokerName()); clusterAclVersionInfo.setBrokerAddr(responseHeader.getBrokerAddr()); - clusterAclVersionInfo.setAclConfigDataVersion(DataVersion.fromJson(responseHeader.getVersion(), DataVersion.class)); - HashMap dataVersionMap = JSON.parseObject(responseHeader.getAllAclFileVersion(), HashMap.class); + clusterAclVersionInfo.setAclConfigDataVersion( + DataVersion.fromJson(responseHeader.getVersion(), DataVersion.class)); + HashMap dataVersionMap = JSON.parseObject(responseHeader.getAllAclFileVersion(), + HashMap.class); Map allAclConfigDataVersion = new HashMap<>(dataVersionMap.size(), 1); for (Map.Entry entry : dataVersionMap.entrySet()) { - allAclConfigDataVersion.put(entry.getKey(), DataVersion.fromJson(JSON.toJSONString(entry.getValue()), DataVersion.class)); + allAclConfigDataVersion.put(entry.getKey(), + DataVersion.fromJson(JSON.toJSONString(entry.getValue()), DataVersion.class)); } clusterAclVersionInfo.setAllAclConfigDataVersion(allAclConfigDataVersion); return clusterAclVersionInfo; @@ -573,59 +594,50 @@ public ClusterAclVersionInfo getBrokerClusterAclInfo(final String addr, default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); - - } - - public SendResult sendMessage( - final String addr, - final String brokerName, - final Message msg, - final SendMessageRequestHeader requestHeader, - final long timeoutMillis, - final CommunicationMode communicationMode, - final SendMessageContext context, - final DefaultMQProducerImpl producer - ) throws RemotingException, MQBrokerException, InterruptedException { - return sendMessage(addr, brokerName, msg, requestHeader, timeoutMillis, communicationMode, null, null, null, 0, context, producer); - } - - public SendResult sendMessage( - final String addr, - final String brokerName, - final Message msg, - final SendMessageRequestHeader requestHeader, - final long timeoutMillis, - final CommunicationMode communicationMode, - final SendCallback sendCallback, - final TopicPublishInfo topicPublishInfo, - final MQClientInstance instance, - final int retryTimesWhenSendFailed, - final SendMessageContext context, - final DefaultMQProducerImpl producer - ) throws RemotingException, MQBrokerException, InterruptedException { + + } + + public SendResult sendMessage(final String addr, final String brokerName, final Message msg, + final SendMessageRequestHeader requestHeader, final long timeoutMillis, + final CommunicationMode communicationMode, final SendMessageContext context, + final DefaultMQProducerImpl producer) throws RemotingException, MQBrokerException, InterruptedException { + return sendMessage(addr, brokerName, msg, requestHeader, timeoutMillis, communicationMode, null, null, null, 0, + context, producer); + } + + public SendResult sendMessage(final String addr, final String brokerName, final Message msg, + final SendMessageRequestHeader requestHeader, final long timeoutMillis, + final CommunicationMode communicationMode, final SendCallback sendCallback, + final TopicPublishInfo topicPublishInfo, final MQClientInstance instance, + final int retryTimesWhenSendFailed, final SendMessageContext context, final DefaultMQProducerImpl producer) + throws RemotingException, MQBrokerException, InterruptedException { long beginStartTime = System.currentTimeMillis(); RemotingCommand request = null; String msgType = msg.getProperty(MessageConst.PROPERTY_MESSAGE_TYPE); boolean isReply = msgType != null && msgType.equals(MixAll.REPLY_MESSAGE_FLAG); if (isReply) { if (sendSmartMsg) { - SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader); + SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2( + requestHeader); request = RemotingCommand.createRequestCommand(RequestCode.SEND_REPLY_MESSAGE_V2, requestHeaderV2); } else { request = RemotingCommand.createRequestCommand(RequestCode.SEND_REPLY_MESSAGE, requestHeader); } } else { if (sendSmartMsg || msg instanceof MessageBatch) { - SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2(requestHeader); - request = RemotingCommand.createRequestCommand(msg instanceof MessageBatch ? RequestCode.SEND_BATCH_MESSAGE : RequestCode.SEND_MESSAGE_V2, requestHeaderV2); + SendMessageRequestHeaderV2 requestHeaderV2 = SendMessageRequestHeaderV2.createSendMessageRequestHeaderV2( + requestHeader); + request = RemotingCommand.createRequestCommand( + msg instanceof MessageBatch ? RequestCode.SEND_BATCH_MESSAGE : RequestCode.SEND_MESSAGE_V2, + requestHeaderV2); } else { request = RemotingCommand.createRequestCommand(RequestCode.SEND_MESSAGE, requestHeader); } } request.setBody(msg.getBody()); - + switch (communicationMode) { case ONEWAY: this.remotingClient.invokeOneway(addr, request, timeoutMillis); @@ -636,8 +648,8 @@ public SendResult sendMessage( if (timeoutMillis < costTimeAsync) { throw new RemotingTooMuchRequestException("sendMessage call timeout"); } - this.sendMessageAsync(addr, brokerName, msg, timeoutMillis - costTimeAsync, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, context, producer); + this.sendMessageAsync(addr, brokerName, msg, timeoutMillis - costTimeAsync, request, sendCallback, + topicPublishInfo, instance, retryTimesWhenSendFailed, times, context, producer); return null; case SYNC: long costTimeSync = System.currentTimeMillis() - beginStartTime; @@ -649,107 +661,97 @@ public SendResult sendMessage( assert false; break; } - + return null; } - - private SendResult sendMessageSync( - final String addr, - final String brokerName, - final Message msg, - final long timeoutMillis, - final RemotingCommand request - ) throws RemotingException, MQBrokerException, InterruptedException { + + private SendResult sendMessageSync(final String addr, final String brokerName, final Message msg, + final long timeoutMillis, final RemotingCommand request) + throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; return this.processSendResponse(brokerName, msg, response, addr); } - + void execRpcHooksAfterRequest(ResponseFuture responseFuture) { if (this.remotingClient instanceof NettyRemotingClient) { NettyRemotingClient remotingClient = (NettyRemotingClient) this.remotingClient; RemotingCommand response = responseFuture.getResponseCommand(); - remotingClient.doAfterRpcHooks(RemotingHelper.parseChannelRemoteAddr(responseFuture.getChannel()), responseFuture.getRequestCommand(), response); + remotingClient.doAfterRpcHooks(RemotingHelper.parseChannelRemoteAddr(responseFuture.getChannel()), + responseFuture.getRequestCommand(), response); } } - - private void sendMessageAsync( - final String addr, - final String brokerName, - final Message msg, - final long timeoutMillis, - final RemotingCommand request, - final SendCallback sendCallback, - final TopicPublishInfo topicPublishInfo, - final MQClientInstance instance, - final int retryTimesWhenSendFailed, - final AtomicInteger times, - final SendMessageContext context, - final DefaultMQProducerImpl producer - ) { + + private void sendMessageAsync(final String addr, final String brokerName, final Message msg, + final long timeoutMillis, final RemotingCommand request, final SendCallback sendCallback, + final TopicPublishInfo topicPublishInfo, final MQClientInstance instance, + final int retryTimesWhenSendFailed, final AtomicInteger times, final SendMessageContext context, + final DefaultMQProducerImpl producer) { final long beginStartTime = System.currentTimeMillis(); try { this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { - + } - + @Override public void operationSucceed(RemotingCommand response) { long cost = System.currentTimeMillis() - beginStartTime; if (null == sendCallback) { try { - SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); + SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, + addr); if (context != null && sendResult != null) { context.setSendResult(sendResult); context.getProducer().executeSendMessageHookAfter(context); } } catch (Throwable e) { } - + producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, false, true); return; } - + try { - SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, addr); + SendResult sendResult = MQClientAPIImpl.this.processSendResponse(brokerName, msg, response, + addr); assert sendResult != null; if (context != null) { context.setSendResult(sendResult); context.getProducer().executeSendMessageHookAfter(context); } - + try { sendCallback.onSuccess(sendResult); } catch (Throwable e) { } - + producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, false, true); } catch (Exception e) { producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, true, true); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, e, context, false, producer); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, + instance, retryTimesWhenSendFailed, times, e, context, false, producer); } } - + @Override public void operationFail(Throwable throwable) { producer.updateFaultItem(brokerName, System.currentTimeMillis() - beginStartTime, true, true); long cost = System.currentTimeMillis() - beginStartTime; if (throwable instanceof RemotingSendRequestException) { MQClientException ex = new MQClientException("send request failed", throwable); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, + instance, retryTimesWhenSendFailed, times, ex, context, true, producer); } else if (throwable instanceof RemotingTimeoutException) { MQClientException ex = new MQClientException("wait response timeout, cost=" + cost, throwable); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, + instance, retryTimesWhenSendFailed, times, ex, context, true, producer); } else { MQClientException ex = new MQClientException("unknown reason", throwable); boolean needRetry = !(throwable instanceof RemotingTooMuchRequestException); - onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, needRetry, producer); + onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, + instance, retryTimesWhenSendFailed, times, ex, context, needRetry, producer); } } }); @@ -757,57 +759,44 @@ public void operationFail(Throwable throwable) { long cost = System.currentTimeMillis() - beginStartTime; producer.updateFaultItem(brokerName, cost, true, false); onExceptionImpl(brokerName, msg, timeoutMillis - cost, request, sendCallback, topicPublishInfo, instance, - retryTimesWhenSendFailed, times, ex, context, true, producer); + retryTimesWhenSendFailed, times, ex, context, true, producer); } } - - private void onExceptionImpl(final String brokerName, - final Message msg, - final long timeoutMillis, - final RemotingCommand request, - final SendCallback sendCallback, - final TopicPublishInfo topicPublishInfo, - final MQClientInstance instance, - final int timesTotal, - final AtomicInteger curTimes, - final Exception e, - final SendMessageContext context, - final boolean needRetry, - final DefaultMQProducerImpl producer - ) { + + private void onExceptionImpl(final String brokerName, final Message msg, final long timeoutMillis, + final RemotingCommand request, final SendCallback sendCallback, final TopicPublishInfo topicPublishInfo, + final MQClientInstance instance, final int timesTotal, final AtomicInteger curTimes, final Exception e, + final SendMessageContext context, final boolean needRetry, final DefaultMQProducerImpl producer) { int tmp = curTimes.incrementAndGet(); if (needRetry && tmp <= timesTotal && timeoutMillis > 0) { String retryBrokerName = brokerName;//by default, it will send to the same broker - if (topicPublishInfo != null) { //select one message queue accordingly, in order to determine which broker to send + if (topicPublishInfo + != null) { //select one message queue accordingly, in order to determine which broker to send MessageQueue mqChosen = producer.selectOneMessageQueue(topicPublishInfo, brokerName, false); retryBrokerName = instance.getBrokerNameFromMessageQueue(mqChosen); } String addr = instance.findBrokerAddressInPublish(retryBrokerName); - log.warn("async send msg by retry {} times. topic={}, brokerAddr={}, brokerName={}", tmp, msg.getTopic(), addr, - retryBrokerName, e); + log.warn("async send msg by retry {} times. topic={}, brokerAddr={}, brokerName={}", tmp, msg.getTopic(), + addr, retryBrokerName, e); request.setOpaque(RemotingCommand.createNewRequestId()); - sendMessageAsync(addr, retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, instance, - timesTotal, curTimes, context, producer); + sendMessageAsync(addr, retryBrokerName, msg, timeoutMillis, request, sendCallback, topicPublishInfo, + instance, timesTotal, curTimes, context, producer); } else { - + if (context != null) { context.setException(e); context.getProducer().executeSendMessageHookAfter(context); } - + try { sendCallback.onException(e); } catch (Exception ignored) { } } } - - protected SendResult processSendResponse( - final String brokerName, - final Message msg, - final RemotingCommand response, - final String addr - ) throws MQBrokerException, RemotingCommandException { + + protected SendResult processSendResponse(final String brokerName, final Message msg, final RemotingCommand response, + final String addr) throws MQBrokerException, RemotingCommandException { SendStatus sendStatus; switch (response.getCode()) { case ResponseCode.FLUSH_DISK_TIMEOUT: { @@ -830,18 +819,18 @@ protected SendResult processSendResponse( throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } } - - SendMessageResponseHeader responseHeader = - (SendMessageResponseHeader) response.decodeCommandCustomHeader(SendMessageResponseHeader.class); - + + SendMessageResponseHeader responseHeader = (SendMessageResponseHeader) response.decodeCommandCustomHeader( + SendMessageResponseHeader.class); + //If namespace not null , reset Topic without namespace. String topic = msg.getTopic(); if (StringUtils.isNotEmpty(this.clientConfig.getNamespace())) { topic = NamespaceUtil.withoutNamespace(topic, this.clientConfig.getNamespace()); } - + MessageQueue messageQueue = new MessageQueue(topic, brokerName, responseHeader.getQueueId()); - + String uniqMsgId = MessageClientIDSetter.getUniqID(msg); if (msg instanceof MessageBatch && responseHeader.getBatchUniqId() == null) { // This means it is not an inner batch @@ -851,9 +840,8 @@ protected SendResult processSendResponse( } uniqMsgId = sb.toString(); } - SendResult sendResult = new SendResult(sendStatus, - uniqMsgId, - responseHeader.getMsgId(), messageQueue, responseHeader.getQueueOffset()); + SendResult sendResult = new SendResult(sendStatus, uniqMsgId, responseHeader.getMsgId(), messageQueue, + responseHeader.getQueueOffset()); sendResult.setTransactionId(responseHeader.getTransactionId()); sendResult.setRecallHandle(responseHeader.getRecallHandle()); String regionId = response.getExtFields().get(MessageConst.PROPERTY_MSG_REGION); @@ -865,21 +853,17 @@ protected SendResult processSendResponse( sendResult.setTraceOn(!Boolean.FALSE.toString().equals(traceOn)); return sendResult; } - - public PullResult pullMessage( - final String addr, - final PullMessageRequestHeader requestHeader, - final long timeoutMillis, - final CommunicationMode communicationMode, - final PullCallback pullCallback - ) throws RemotingException, MQBrokerException, InterruptedException { + + public PullResult pullMessage(final String addr, final PullMessageRequestHeader requestHeader, + final long timeoutMillis, final CommunicationMode communicationMode, final PullCallback pullCallback) + throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand request; if (PullSysFlag.hasLitePullFlag(requestHeader.getSysFlag())) { request = RemotingCommand.createRequestCommand(RequestCode.LITE_PULL_MESSAGE, requestHeader); } else { request = RemotingCommand.createRequestCommand(RequestCode.PULL_MESSAGE, requestHeader); } - + switch (communicationMode) { case ONEWAY: assert false; @@ -893,54 +877,46 @@ public PullResult pullMessage( assert false; break; } - + return null; } - - public void popMessageAsync( - final String brokerName, final String addr, final PopMessageRequestHeader requestHeader, - final long timeoutMillis, final PopCallback popCallback - ) throws RemotingException, InterruptedException { + + public void popMessageAsync(final String brokerName, final String addr, final PopMessageRequestHeader requestHeader, + final long timeoutMillis, final PopCallback popCallback) throws RemotingException, InterruptedException { final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.POP_MESSAGE, requestHeader); this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { - + } - + @Override public void operationSucceed(RemotingCommand response) { try { - PopResult popResult = MQClientAPIImpl.this.processPopResponse(brokerName, response, requestHeader.getTopic(), requestHeader); + PopResult popResult = MQClientAPIImpl.this.processPopResponse(brokerName, response, + requestHeader.getTopic(), requestHeader); popCallback.onSuccess(popResult); } catch (Exception e) { popCallback.onException(e); } } + @Override public void operationFail(Throwable throwable) { popCallback.onException(throwable); } }); } - - public void ackMessageAsync( - final String addr, - final long timeOut, - final AckCallback ackCallback, - final AckMessageRequestHeader requestHeader - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void ackMessageAsync(final String addr, final long timeOut, final AckCallback ackCallback, + final AckMessageRequestHeader requestHeader) + throws RemotingException, MQBrokerException, InterruptedException { ackMessageAsync(addr, timeOut, ackCallback, requestHeader, null); } - - public void batchAckMessageAsync( - final String addr, - final long timeOut, - final AckCallback ackCallback, - final String topic, - final String consumerGroup, - final List extraInfoList - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void batchAckMessageAsync(final String addr, final long timeOut, final AckCallback ackCallback, + final String topic, final String consumerGroup, final List extraInfoList) + throws RemotingException, MQBrokerException, InterruptedException { String brokerName = null; Map batchAckMap = new HashMap<>(); for (String extraInfo : extraInfoList) { @@ -948,10 +924,10 @@ public void batchAckMessageAsync( if (brokerName == null) { brokerName = ExtraInfoUtil.getBrokerName(extraInfoData); } - String mergeKey = ExtraInfoUtil.getRetry(extraInfoData) + "@" + - ExtraInfoUtil.getQueueId(extraInfoData) + "@" + - ExtraInfoUtil.getCkQueueOffset(extraInfoData) + "@" + - ExtraInfoUtil.getPopTime(extraInfoData); + String mergeKey = + ExtraInfoUtil.getRetry(extraInfoData) + "@" + ExtraInfoUtil.getQueueId(extraInfoData) + "@" + + ExtraInfoUtil.getCkQueueOffset(extraInfoData) + "@" + ExtraInfoUtil.getPopTime( + extraInfoData); BatchAck bAck = batchAckMap.computeIfAbsent(mergeKey, k -> { BatchAck newBatchAck = new BatchAck(); newBatchAck.setConsumerGroup(consumerGroup); @@ -965,31 +941,25 @@ public void batchAckMessageAsync( newBatchAck.setBitSet(new BitSet()); return newBatchAck; }); - bAck.getBitSet().set((int) (ExtraInfoUtil.getQueueOffset(extraInfoData) - ExtraInfoUtil.getCkQueueOffset(extraInfoData))); + bAck.getBitSet().set((int) (ExtraInfoUtil.getQueueOffset(extraInfoData) - ExtraInfoUtil.getCkQueueOffset( + extraInfoData))); } - + BatchAckMessageRequestBody requestBody = new BatchAckMessageRequestBody(); requestBody.setBrokerName(brokerName); requestBody.setAcks(new ArrayList<>(batchAckMap.values())); batchAckMessageAsync(addr, timeOut, ackCallback, requestBody); } - - public void batchAckMessageAsync( - final String addr, - final long timeOut, - final AckCallback ackCallback, - final BatchAckMessageRequestBody requestBody - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void batchAckMessageAsync(final String addr, final long timeOut, final AckCallback ackCallback, + final BatchAckMessageRequestBody requestBody) + throws RemotingException, MQBrokerException, InterruptedException { ackMessageAsync(addr, timeOut, ackCallback, null, requestBody); } - - protected void ackMessageAsync( - final String addr, - final long timeOut, - final AckCallback ackCallback, - final AckMessageRequestHeader requestHeader, - final BatchAckMessageRequestBody requestBody - ) throws RemotingException, MQBrokerException, InterruptedException { + + protected void ackMessageAsync(final String addr, final long timeOut, final AckCallback ackCallback, + final AckMessageRequestHeader requestHeader, final BatchAckMessageRequestBody requestBody) + throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand request; if (requestHeader != null) { request = RemotingCommand.createRequestCommand(RequestCode.ACK_MESSAGE, requestHeader); @@ -1002,9 +972,9 @@ protected void ackMessageAsync( this.remotingClient.invokeAsync(addr, request, timeOut, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { - + } - + @Override public void operationSucceed(RemotingCommand response) { AckResult ackResult = new AckResult(); @@ -1015,40 +985,41 @@ public void operationSucceed(RemotingCommand response) { } ackCallback.onSuccess(ackResult); } - + @Override public void operationFail(Throwable throwable) { ackCallback.onException(throwable); } }); } - + public void changeInvisibleTimeAsync(// - final String brokerName, - final String addr, // - final ChangeInvisibleTimeRequestHeader requestHeader,// - final long timeoutMillis, - final AckCallback ackCallback - ) throws RemotingException, MQBrokerException, InterruptedException { - final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHANGE_MESSAGE_INVISIBLETIME, requestHeader); + final String brokerName, final String addr, // + final ChangeInvisibleTimeRequestHeader requestHeader,// + final long timeoutMillis, final AckCallback ackCallback) + throws RemotingException, MQBrokerException, InterruptedException { + final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHANGE_MESSAGE_INVISIBLETIME, + requestHeader); this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { - + } - + @Override public void operationSucceed(RemotingCommand response) { try { - ChangeInvisibleTimeResponseHeader responseHeader = (ChangeInvisibleTimeResponseHeader) response.decodeCommandCustomHeader(ChangeInvisibleTimeResponseHeader.class); + ChangeInvisibleTimeResponseHeader responseHeader = (ChangeInvisibleTimeResponseHeader) response.decodeCommandCustomHeader( + ChangeInvisibleTimeResponseHeader.class); AckResult ackResult = new AckResult(); if (ResponseCode.SUCCESS == response.getCode()) { ackResult.setStatus(AckStatus.OK); ackResult.setPopTime(responseHeader.getPopTime()); - ackResult.setExtraInfo(ExtraInfoUtil - .buildExtraInfo(requestHeader.getOffset(), responseHeader.getPopTime(), responseHeader.getInvisibleTime(), - responseHeader.getReviveQid(), requestHeader.getTopic(), brokerName, requestHeader.getQueueId()) + MessageConst.KEY_SEPARATOR - + requestHeader.getOffset()); + ackResult.setExtraInfo( + ExtraInfoUtil.buildExtraInfo(requestHeader.getOffset(), responseHeader.getPopTime(), + responseHeader.getInvisibleTime(), responseHeader.getReviveQid(), + requestHeader.getTopic(), brokerName, requestHeader.getQueueId()) + + MessageConst.KEY_SEPARATOR + requestHeader.getOffset()); } else { ackResult.setStatus(AckStatus.NO_EXIST); } @@ -1057,26 +1028,22 @@ public void operationSucceed(RemotingCommand response) { ackCallback.onException(e); } } - + @Override public void operationFail(Throwable throwable) { ackCallback.onException(throwable); } }); } - - private void pullMessageAsync( - final String addr, - final RemotingCommand request, - final long timeoutMillis, - final PullCallback pullCallback - ) throws RemotingException, InterruptedException { + + private void pullMessageAsync(final String addr, final RemotingCommand request, final long timeoutMillis, + final PullCallback pullCallback) throws RemotingException, InterruptedException { this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { - + } - + @Override public void operationSucceed(RemotingCommand response) { try { @@ -1086,27 +1053,23 @@ public void operationSucceed(RemotingCommand response) { pullCallback.onException(e); } } - + @Override public void operationFail(Throwable throwable) { pullCallback.onException(throwable); } }); } - - private PullResult pullMessageSync( - final String addr, - final RemotingCommand request, - final long timeoutMillis - ) throws RemotingException, InterruptedException, MQBrokerException { + + private PullResult pullMessageSync(final String addr, final RemotingCommand request, final long timeoutMillis) + throws RemotingException, InterruptedException, MQBrokerException { RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; return this.processPullResponse(response, addr); } - - private PullResult processPullResponse( - final RemotingCommand response, - final String addr) throws MQBrokerException, RemotingCommandException { + + private PullResult processPullResponse(final RemotingCommand response, final String addr) + throws MQBrokerException, RemotingCommandException { PullStatus pullStatus = PullStatus.NO_NEW_MSG; switch (response.getCode()) { case ResponseCode.SUCCESS: @@ -1121,31 +1084,29 @@ private PullResult processPullResponse( case ResponseCode.PULL_OFFSET_MOVED: pullStatus = PullStatus.OFFSET_ILLEGAL; break; - + default: throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - PullMessageResponseHeader responseHeader = - (PullMessageResponseHeader) response.decodeCommandCustomHeader(PullMessageResponseHeader.class); - + + PullMessageResponseHeader responseHeader = (PullMessageResponseHeader) response.decodeCommandCustomHeader( + PullMessageResponseHeader.class); + return new PullResultExt(pullStatus, responseHeader.getNextBeginOffset(), responseHeader.getMinOffset(), - responseHeader.getMaxOffset(), null, responseHeader.getSuggestWhichBrokerId(), response.getBody(), responseHeader.getOffsetDelta()); + responseHeader.getMaxOffset(), null, responseHeader.getSuggestWhichBrokerId(), response.getBody(), + responseHeader.getOffsetDelta()); } - + private PopResult processPopResponse(final String brokerName, final RemotingCommand response, String topic, - CommandCustomHeader requestHeader) throws MQBrokerException, RemotingCommandException { + CommandCustomHeader requestHeader) throws MQBrokerException, RemotingCommandException { PopStatus popStatus = PopStatus.NO_NEW_MSG; List msgFoundList = null; switch (response.getCode()) { case ResponseCode.SUCCESS: popStatus = PopStatus.FOUND; ByteBuffer byteBuffer = ByteBuffer.wrap(response.getBody()); - msgFoundList = MessageDecoder.decodesBatch( - byteBuffer, - clientConfig.isDecodeReadBody(), - clientConfig.isDecodeDecompressBody(), - true); + msgFoundList = MessageDecoder.decodesBatch(byteBuffer, clientConfig.isDecodeReadBody(), + clientConfig.isDecodeDecompressBody(), true); break; case ResponseCode.POLLING_FULL: popStatus = PopStatus.POLLING_FULL; @@ -1159,9 +1120,10 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm default: throw new MQBrokerException(response.getCode(), response.getRemark()); } - + PopResult popResult = new PopResult(popStatus, msgFoundList); - PopMessageResponseHeader responseHeader = (PopMessageResponseHeader) response.decodeCommandCustomHeader(PopMessageResponseHeader.class); + PopMessageResponseHeader responseHeader = (PopMessageResponseHeader) response.decodeCommandCustomHeader( + PopMessageResponseHeader.class); popResult.setRestNum(responseHeader.getRestNum()); if (popStatus != PopStatus.FOUND) { return popResult; @@ -1177,8 +1139,8 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm msgOffsetInfo = ExtraInfoUtil.parseMsgOffsetInfo(responseHeader.getMsgOffsetInfo()); orderCountInfo = ExtraInfoUtil.parseOrderCountInfo(responseHeader.getOrderCountInfo()); } - Map/*msg queueOffset*/> sortMap - = buildQueueOffsetSortedMap(topic, msgFoundList); + Map/*msg queueOffset*/> sortMap = buildQueueOffsetSortedMap(topic, + msgFoundList); Map map = new HashMap<>(5); for (MessageExt messageExt : msgFoundList) { if (requestHeader instanceof PopMessageRequestHeader) { @@ -1187,11 +1149,14 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm // find pop ck offset String key = messageExt.getTopic() + messageExt.getQueueId(); if (!map.containsKey(messageExt.getTopic() + messageExt.getQueueId())) { - map.put(key, ExtraInfoUtil.buildExtraInfo(messageExt.getQueueOffset(), responseHeader.getPopTime(), responseHeader.getInvisibleTime(), responseHeader.getReviveQid(), - messageExt.getTopic(), brokerName, messageExt.getQueueId())); - + map.put(key, + ExtraInfoUtil.buildExtraInfo(messageExt.getQueueOffset(), responseHeader.getPopTime(), + responseHeader.getInvisibleTime(), responseHeader.getReviveQid(), + messageExt.getTopic(), brokerName, messageExt.getQueueId())); + } - messageExt.getProperties().put(MessageConst.PROPERTY_POP_CK, map.get(key) + MessageConst.KEY_SEPARATOR + messageExt.getQueueOffset()); + messageExt.getProperties().put(MessageConst.PROPERTY_POP_CK, + map.get(key) + MessageConst.KEY_SEPARATOR + messageExt.getQueueOffset()); } else { if (messageExt.getProperty(MessageConst.PROPERTY_POP_CK) == null) { final String queueIdKey; @@ -1199,12 +1164,13 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm final int index; final Long msgQueueOffset; if (MixAll.isLmq(topic) && messageExt.getReconsumeTimes() == 0 && StringUtils.isNotEmpty( - messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH))) { + messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH))) { // process LMQ String[] queues = messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH) - .split(MixAll.LMQ_DISPATCH_SEPARATOR); - String[] queueOffsets = messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET) - .split(MixAll.LMQ_DISPATCH_SEPARATOR); + .split(MixAll.LMQ_DISPATCH_SEPARATOR); + String[] queueOffsets = messageExt.getProperty( + MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET) + .split(MixAll.LMQ_DISPATCH_SEPARATOR); long offset = Long.parseLong(queueOffsets[ArrayUtils.indexOf(queues, topic)]); // LMQ topic has only 1 queue, which queue id is 0 queueIdKey = ExtraInfoUtil.getStartOffsetInfoMapKey(topic, MixAll.LMQ_QUEUE_ID); @@ -1213,24 +1179,28 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm msgQueueOffset = msgOffsetInfo.get(queueIdKey).get(index); if (msgQueueOffset != offset) { log.warn("Queue offset[{}] of msg is strange, not equal to the stored in msg, {}", - msgQueueOffset, messageExt); + msgQueueOffset, messageExt); } messageExt.getProperties().put(MessageConst.PROPERTY_POP_CK, - ExtraInfoUtil.buildExtraInfo(startOffsetInfo.get(queueIdKey), responseHeader.getPopTime(), responseHeader.getInvisibleTime(), - responseHeader.getReviveQid(), topic, brokerName, 0, msgQueueOffset) - ); + ExtraInfoUtil.buildExtraInfo(startOffsetInfo.get(queueIdKey), + responseHeader.getPopTime(), responseHeader.getInvisibleTime(), + responseHeader.getReviveQid(), topic, brokerName, 0, msgQueueOffset)); } else { - queueIdKey = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(), messageExt.getQueueId()); - queueOffsetKey = ExtraInfoUtil.getQueueOffsetMapKey(messageExt.getTopic(), messageExt.getQueueId(), messageExt.getQueueOffset()); + queueIdKey = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(), + messageExt.getQueueId()); + queueOffsetKey = ExtraInfoUtil.getQueueOffsetMapKey(messageExt.getTopic(), + messageExt.getQueueId(), messageExt.getQueueOffset()); index = sortMap.get(queueIdKey).indexOf(messageExt.getQueueOffset()); msgQueueOffset = msgOffsetInfo.get(queueIdKey).get(index); if (msgQueueOffset != messageExt.getQueueOffset()) { - log.warn("Queue offset[{}] of msg is strange, not equal to the stored in msg, {}", msgQueueOffset, messageExt); + log.warn("Queue offset[{}] of msg is strange, not equal to the stored in msg, {}", + msgQueueOffset, messageExt); } messageExt.getProperties().put(MessageConst.PROPERTY_POP_CK, - ExtraInfoUtil.buildExtraInfo(startOffsetInfo.get(queueIdKey), responseHeader.getPopTime(), responseHeader.getInvisibleTime(), - responseHeader.getReviveQid(), messageExt.getTopic(), brokerName, messageExt.getQueueId(), msgQueueOffset) - ); + ExtraInfoUtil.buildExtraInfo(startOffsetInfo.get(queueIdKey), + responseHeader.getPopTime(), responseHeader.getInvisibleTime(), + responseHeader.getReviveQid(), messageExt.getTopic(), brokerName, + messageExt.getQueueId(), msgQueueOffset)); } if (((PopMessageRequestHeader) requestHeader).isOrder() && orderCountInfo != null) { Integer count = orderCountInfo.get(queueOffsetKey); @@ -1243,19 +1213,19 @@ private PopResult processPopResponse(final String brokerName, final RemotingComm } } } - messageExt.getProperties().computeIfAbsent( - MessageConst.PROPERTY_FIRST_POP_TIME, k -> String.valueOf(responseHeader.getPopTime())); + messageExt.getProperties().computeIfAbsent(MessageConst.PROPERTY_FIRST_POP_TIME, + k -> String.valueOf(responseHeader.getPopTime())); } messageExt.setBrokerName(brokerName); messageExt.setTopic(NamespaceUtil.withoutNamespace(topic, this.clientConfig.getNamespace())); } return popResult; } - + /** * Build queue offset sorted map * - * @param topic pop consumer topic + * @param topic pop consumer topic * @param msgFoundList popped message list * @return sorted map, key is topicMark@queueId, value is sorted msg queueOffset list */ @@ -1263,13 +1233,13 @@ private static Map> buildQueueOffsetSortedMap(String topic, L Map/*msg queueOffset*/> sortMap = new HashMap<>(16); for (MessageExt messageExt : msgFoundList) { final String key; - if (MixAll.isLmq(topic) && messageExt.getReconsumeTimes() == 0 - && StringUtils.isNotEmpty(messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH))) { + if (MixAll.isLmq(topic) && messageExt.getReconsumeTimes() == 0 && StringUtils.isNotEmpty( + messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH))) { // process LMQ String[] queues = messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_DISPATCH) - .split(MixAll.LMQ_DISPATCH_SEPARATOR); + .split(MixAll.LMQ_DISPATCH_SEPARATOR); String[] queueOffsets = messageExt.getProperty(MessageConst.PROPERTY_INNER_MULTI_QUEUE_OFFSET) - .split(MixAll.LMQ_DISPATCH_SEPARATOR); + .split(MixAll.LMQ_DISPATCH_SEPARATOR); // LMQ topic has only 1 queue, which queue id is 0 key = ExtraInfoUtil.getStartOffsetInfoMapKey(topic, MixAll.LMQ_QUEUE_ID); sortMap.putIfAbsent(key, new ArrayList<>(4)); @@ -1279,7 +1249,7 @@ private static Map> buildQueueOffsetSortedMap(String topic, L // Value of POP_CK is used to determine whether it is a pop retry, // cause topic could be rewritten by broker. key = ExtraInfoUtil.getStartOffsetInfoMapKey(messageExt.getTopic(), - messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getQueueId()); + messageExt.getProperty(MessageConst.PROPERTY_POP_CK), messageExt.getQueueId()); if (!sortMap.containsKey(key)) { sortMap.put(key, new ArrayList<>(4)); } @@ -1287,16 +1257,16 @@ private static Map> buildQueueOffsetSortedMap(String topic, L } return sortMap; } - + public MessageExt viewMessage(final String addr, final String topic, final long phyoffset, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + throws RemotingException, MQBrokerException, InterruptedException { ViewMessageRequestHeader requestHeader = new ViewMessageRequestHeader(); requestHeader.setTopic(topic); requestHeader.setOffset(phyoffset); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.VIEW_MESSAGE_BY_ID, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -1304,192 +1274,191 @@ public MessageExt viewMessage(final String addr, final String topic, final long MessageExt messageExt = MessageDecoder.clientDecode(byteBuffer, true); //If namespace not null , reset Topic without namespace. if (StringUtils.isNotEmpty(this.clientConfig.getNamespace())) { - messageExt.setTopic(NamespaceUtil.withoutNamespace(messageExt.getTopic(), this.clientConfig.getNamespace())); + messageExt.setTopic( + NamespaceUtil.withoutNamespace(messageExt.getTopic(), this.clientConfig.getNamespace())); } return messageExt; } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + @Deprecated public long searchOffset(final String addr, final String topic, final int queueId, final long timestamp, - final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { SearchOffsetRequestHeader requestHeader = new SearchOffsetRequestHeader(); requestHeader.setTopic(topic); requestHeader.setQueueId(queueId); requestHeader.setTimestamp(timestamp); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - SearchOffsetResponseHeader responseHeader = - (SearchOffsetResponseHeader) response.decodeCommandCustomHeader(SearchOffsetResponseHeader.class); + SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.decodeCommandCustomHeader( + SearchOffsetResponseHeader.class); return responseHeader.getOffset(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public long searchOffset(final String addr, final MessageQueue messageQueue, final long timestamp, - final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { // default return lower boundary offset when there are more than one offsets. return searchOffset(addr, messageQueue, timestamp, BoundaryType.LOWER, timeoutMillis); } - + public long searchOffset(final String addr, final MessageQueue messageQueue, final long timestamp, - final BoundaryType boundaryType, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + final BoundaryType boundaryType, final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { SearchOffsetRequestHeader requestHeader = new SearchOffsetRequestHeader(); requestHeader.setTopic(messageQueue.getTopic()); requestHeader.setQueueId(messageQueue.getQueueId()); requestHeader.setBrokerName(messageQueue.getBrokerName()); requestHeader.setTimestamp(timestamp); requestHeader.setBoundaryType(boundaryType); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SEARCH_OFFSET_BY_TIMESTAMP, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - SearchOffsetResponseHeader responseHeader = - (SearchOffsetResponseHeader) response.decodeCommandCustomHeader(SearchOffsetResponseHeader.class); + SearchOffsetResponseHeader responseHeader = (SearchOffsetResponseHeader) response.decodeCommandCustomHeader( + SearchOffsetResponseHeader.class); return responseHeader.getOffset(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public long getMaxOffset(final String addr, final MessageQueue messageQueue, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + throws RemotingException, MQBrokerException, InterruptedException { GetMaxOffsetRequestHeader requestHeader = new GetMaxOffsetRequestHeader(); requestHeader.setTopic(messageQueue.getTopic()); requestHeader.setQueueId(messageQueue.getQueueId()); requestHeader.setBrokerName(messageQueue.getBrokerName()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MAX_OFFSET, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - GetMaxOffsetResponseHeader responseHeader = - (GetMaxOffsetResponseHeader) response.decodeCommandCustomHeader(GetMaxOffsetResponseHeader.class); - + GetMaxOffsetResponseHeader responseHeader = (GetMaxOffsetResponseHeader) response.decodeCommandCustomHeader( + GetMaxOffsetResponseHeader.class); + return responseHeader.getOffset(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public List getConsumerIdListByGroup( - final String addr, - final String consumerGroup, - final long timeoutMillis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, - MQBrokerException, InterruptedException { + + public List getConsumerIdListByGroup(final String addr, final String consumerGroup, + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { GetConsumerListByGroupRequestHeader requestHeader = new GetConsumerListByGroupRequestHeader(); requestHeader.setConsumerGroup(consumerGroup); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_LIST_BY_GROUP, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_LIST_BY_GROUP, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { if (response.getBody() != null) { - GetConsumerListByGroupResponseBody body = - GetConsumerListByGroupResponseBody.decode(response.getBody(), GetConsumerListByGroupResponseBody.class); + GetConsumerListByGroupResponseBody body = GetConsumerListByGroupResponseBody.decode( + response.getBody(), GetConsumerListByGroupResponseBody.class); return body.getConsumerIdList(); } } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public long getMinOffset(final String addr, final MessageQueue messageQueue, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + throws RemotingException, MQBrokerException, InterruptedException { GetMinOffsetRequestHeader requestHeader = new GetMinOffsetRequestHeader(); requestHeader.setTopic(messageQueue.getTopic()); requestHeader.setQueueId(messageQueue.getQueueId()); requestHeader.setBrokerName(messageQueue.getBrokerName()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_MIN_OFFSET, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - GetMinOffsetResponseHeader responseHeader = - (GetMinOffsetResponseHeader) response.decodeCommandCustomHeader(GetMinOffsetResponseHeader.class); - + GetMinOffsetResponseHeader responseHeader = (GetMinOffsetResponseHeader) response.decodeCommandCustomHeader( + GetMinOffsetResponseHeader.class); + return responseHeader.getOffset(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public long getEarliestMsgStoretime(final String addr, final MessageQueue mq, final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException { + throws RemotingException, MQBrokerException, InterruptedException { GetEarliestMsgStoretimeRequestHeader requestHeader = new GetEarliestMsgStoretimeRequestHeader(); requestHeader.setTopic(mq.getTopic()); requestHeader.setQueueId(mq.getQueueId()); requestHeader.setBrokerName(mq.getBrokerName()); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_EARLIEST_MSG_STORETIME, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_EARLIEST_MSG_STORETIME, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - GetEarliestMsgStoretimeResponseHeader responseHeader = - (GetEarliestMsgStoretimeResponseHeader) response.decodeCommandCustomHeader(GetEarliestMsgStoretimeResponseHeader.class); - + GetEarliestMsgStoretimeResponseHeader responseHeader = (GetEarliestMsgStoretimeResponseHeader) response.decodeCommandCustomHeader( + GetEarliestMsgStoretimeResponseHeader.class); + return responseHeader.getTimestamp(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public long queryConsumerOffset( - final String addr, - final QueryConsumerOffsetRequestHeader requestHeader, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUMER_OFFSET, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + public long queryConsumerOffset(final String addr, final QueryConsumerOffsetRequestHeader requestHeader, + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUMER_OFFSET, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - QueryConsumerOffsetResponseHeader responseHeader = - (QueryConsumerOffsetResponseHeader) response.decodeCommandCustomHeader(QueryConsumerOffsetResponseHeader.class); + QueryConsumerOffsetResponseHeader responseHeader = (QueryConsumerOffsetResponseHeader) response.decodeCommandCustomHeader( + QueryConsumerOffsetResponseHeader.class); return responseHeader.getOffset(); } case ResponseCode.QUERY_NOT_FOUND: { @@ -1498,19 +1467,17 @@ public long queryConsumerOffset( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void updateConsumerOffset( - final String addr, - final UpdateConsumerOffsetRequestHeader requestHeader, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + public void updateConsumerOffset(final String addr, final UpdateConsumerOffsetRequestHeader requestHeader, + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -1519,27 +1486,24 @@ public void updateConsumerOffset( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void updateConsumerOffsetOneway( - final String addr, - final UpdateConsumerOffsetRequestHeader requestHeader, - final long timeoutMillis - ) throws RemotingConnectException, RemotingTooMuchRequestException, RemotingTimeoutException, RemotingSendRequestException, - InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET, requestHeader); - - this.remotingClient.invokeOneway(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); - } - - public int sendHeartbeat( - final String addr, - final HeartbeatData heartbeatData, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, new HeartbeatRequestHeader()); + + public void updateConsumerOffsetOneway(final String addr, final UpdateConsumerOffsetRequestHeader requestHeader, + final long timeoutMillis) + throws RemotingConnectException, RemotingTooMuchRequestException, RemotingTimeoutException, RemotingSendRequestException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONSUMER_OFFSET, + requestHeader); + + this.remotingClient.invokeOneway(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), + request, timeoutMillis); + } + + public int sendHeartbeat(final String addr, final HeartbeatData heartbeatData, final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, + new HeartbeatRequestHeader()); request.setLanguage(clientConfig.getLanguage()); request.setBody(heartbeatData.encode()); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); @@ -1551,16 +1515,14 @@ public int sendHeartbeat( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public HeartbeatV2Result sendHeartbeatV2( - final String addr, - final HeartbeatData heartbeatData, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, new HeartbeatRequestHeader()); + + public HeartbeatV2Result sendHeartbeatV2(final String addr, final HeartbeatData heartbeatData, + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, + new HeartbeatRequestHeader()); request.setLanguage(clientConfig.getLanguage()); request.setBody(heartbeatData.encode()); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); @@ -1568,30 +1530,28 @@ public HeartbeatV2Result sendHeartbeatV2( switch (response.getCode()) { case ResponseCode.SUCCESS: { if (response.getExtFields() != null) { - return new HeartbeatV2Result(response.getVersion(), Boolean.parseBoolean(response.getExtFields().get(MixAll.IS_SUB_CHANGE)), Boolean.parseBoolean(response.getExtFields().get(MixAll.IS_SUPPORT_HEART_BEAT_V2))); + return new HeartbeatV2Result(response.getVersion(), + Boolean.parseBoolean(response.getExtFields().get(MixAll.IS_SUB_CHANGE)), + Boolean.parseBoolean(response.getExtFields().get(MixAll.IS_SUPPORT_HEART_BEAT_V2))); } return new HeartbeatV2Result(response.getVersion(), false, false); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void unregisterClient( - final String addr, - final String clientID, - final String producerGroup, - final String consumerGroup, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void unregisterClient(final String addr, final String clientID, final String producerGroup, + final String consumerGroup, final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { final UnregisterClientRequestHeader requestHeader = new UnregisterClientRequestHeader(); requestHeader.setClientID(clientID); requestHeader.setProducerGroup(producerGroup); requestHeader.setConsumerGroup(consumerGroup); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UNREGISTER_CLIENT, requestHeader); - + RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -1601,56 +1561,44 @@ public void unregisterClient( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void endTransactionOneway( - final String addr, - final EndTransactionRequestHeader requestHeader, - final String remark, - final long timeoutMillis - ) throws RemotingException, InterruptedException { + + public void endTransactionOneway(final String addr, final EndTransactionRequestHeader requestHeader, + final String remark, final long timeoutMillis) throws RemotingException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.END_TRANSACTION, requestHeader); - + request.setRemark(remark); this.remotingClient.invokeOneway(addr, request, timeoutMillis); } - - public void queryMessage( - final String addr, - final QueryMessageRequestHeader requestHeader, - final long timeoutMillis, - final InvokeCallback invokeCallback, - final Boolean isUniqueKey - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void queryMessage(final String addr, final QueryMessageRequestHeader requestHeader, final long timeoutMillis, + final InvokeCallback invokeCallback, final Boolean isUniqueKey) + throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_MESSAGE, requestHeader); request.addExtField(MixAll.UNIQUE_MSG_QUERY_FLAG, isUniqueKey.toString()); - this.remotingClient.invokeAsync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis, - invokeCallback); + this.remotingClient.invokeAsync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, + timeoutMillis, invokeCallback); } - + public boolean registerClient(final String addr, final HeartbeatData heartbeat, final long timeoutMillis) - throws RemotingException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, new HeartbeatRequestHeader()); - + throws RemotingException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.HEART_BEAT, + new HeartbeatRequestHeader()); + request.setBody(heartbeat.encode()); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); return response.getCode() == ResponseCode.SUCCESS; } - - public void consumerSendMessageBack( - final String addr, - final String brokerName, - final MessageExt msg, - final String consumerGroup, - final int delayLevel, - final long timeoutMillis, - final int maxConsumeRetryTimes - ) throws RemotingException, MQBrokerException, InterruptedException { + + public void consumerSendMessageBack(final String addr, final String brokerName, final MessageExt msg, + final String consumerGroup, final int delayLevel, final long timeoutMillis, final int maxConsumeRetryTimes) + throws RemotingException, MQBrokerException, InterruptedException { ConsumerSendMsgBackRequestHeader requestHeader = new ConsumerSendMsgBackRequestHeader(); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONSUMER_SEND_MSG_BACK, requestHeader); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONSUMER_SEND_MSG_BACK, + requestHeader); + requestHeader.setGroup(consumerGroup); requestHeader.setOriginTopic(msg.getTopic()); requestHeader.setOffset(msg.getCommitLogOffset()); @@ -1658,9 +1606,9 @@ public void consumerSendMessageBack( requestHeader.setOriginMsgId(msg.getMsgId()); requestHeader.setMaxReconsumeTimes(maxConsumeRetryTimes); requestHeader.setBrokerName(brokerName); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -1669,47 +1617,44 @@ public void consumerSendMessageBack( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public Set lockBatchMQ( - final String addr, - final LockBatchRequestBody requestBody, - final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, new LockBatchMqRequestHeader()); - + + public Set lockBatchMQ(final String addr, final LockBatchRequestBody requestBody, + final long timeoutMillis) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.LOCK_BATCH_MQ, + new LockBatchMqRequestHeader()); + request.setBody(requestBody.encode()); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { - LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(), LockBatchResponseBody.class); + LockBatchResponseBody responseBody = LockBatchResponseBody.decode(response.getBody(), + LockBatchResponseBody.class); Set messageQueues = responseBody.getLockOKMQSet(); return messageQueues; } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void unlockBatchMQ( - final String addr, - final UnlockBatchRequestBody requestBody, - final long timeoutMillis, - final boolean oneway - ) throws RemotingException, MQBrokerException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UNLOCK_BATCH_MQ, new UnlockBatchMqRequestHeader()); - + + public void unlockBatchMQ(final String addr, final UnlockBatchRequestBody requestBody, final long timeoutMillis, + final boolean oneway) throws RemotingException, MQBrokerException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UNLOCK_BATCH_MQ, + new UnlockBatchMqRequestHeader()); + request.setBody(requestBody.encode()); - + if (oneway) { this.remotingClient.invokeOneway(addr, request, timeoutMillis); } else { - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return; @@ -1717,21 +1662,20 @@ public void unlockBatchMQ( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } } - - public TopicStatsTable getTopicStatsInfo(final String addr, final String topic, - final long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + + public TopicStatsTable getTopicStatsInfo(final String addr, final String topic, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { GetTopicStatsInfoRequestHeader requestHeader = new GetTopicStatsInfoRequestHeader(); requestHeader.setTopic(topic); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_STATS_INFO, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { TopicStatsTable topicStatsTable = TopicStatsTable.decode(response.getBody(), TopicStatsTable.class); @@ -1740,28 +1684,26 @@ public TopicStatsTable getTopicStatsInfo(final String addr, final String topic, default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public ConsumeStats getConsumeStats(final String addr, final String consumerGroup, final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, - MQBrokerException { + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { return getConsumeStats(addr, consumerGroup, null, timeoutMillis); } - + public ConsumeStats getConsumeStats(final String addr, final String consumerGroup, final String topic, - final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, - MQBrokerException { + final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { GetConsumeStatsRequestHeader requestHeader = new GetConsumeStatsRequestHeader(); requestHeader.setConsumerGroup(consumerGroup); requestHeader.setTopic(topic); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUME_STATS, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { ConsumeStats consumeStats = ConsumeStats.decode(response.getBody(), ConsumeStats.class); @@ -1770,21 +1712,21 @@ public ConsumeStats getConsumeStats(final String addr, final String consumerGrou default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public ProducerConnection getProducerConnectionList(final String addr, final String producerGroup, - final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { GetProducerConnectionListRequestHeader requestHeader = new GetProducerConnectionListRequestHeader(); requestHeader.setProducerGroup(producerGroup); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PRODUCER_CONNECTION_LIST, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_PRODUCER_CONNECTION_LIST, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return ProducerConnection.decode(response.getBody(), ProducerConnection.class); @@ -1792,19 +1734,19 @@ public ProducerConnection getProducerConnectionList(final String addr, final Str default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public ProducerTableInfo getAllProducerInfo(final String addr, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { GetAllProducerInfoRequestHeader requestHeader = new GetAllProducerInfoRequestHeader(); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_PRODUCER_INFO, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_PRODUCER_INFO, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return ProducerTableInfo.decode(response.getBody(), ProducerTableInfo.class); @@ -1812,21 +1754,21 @@ public ProducerTableInfo getAllProducerInfo(final String addr, final long timeou default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public ConsumerConnection getConsumerConnectionList(final String addr, final String consumerGroup, - final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { GetConsumerConnectionListRequestHeader requestHeader = new GetConsumerConnectionListRequestHeader(); requestHeader.setConsumerGroup(consumerGroup); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_CONNECTION_LIST, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_CONNECTION_LIST, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return ConsumerConnection.decode(response.getBody(), ConsumerConnection.class); @@ -1834,17 +1776,17 @@ public ConsumerConnection getConsumerConnectionList(final String addr, final Str default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public KVTable getBrokerRuntimeInfo(final String addr, final long timeoutMillis) throws RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { - + + public KVTable getBrokerRuntimeInfo(final String addr, final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_RUNTIME_INFO, null); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return KVTable.decode(response.getBody(), KVTable.class); @@ -1852,63 +1794,62 @@ public KVTable getBrokerRuntimeInfo(final String addr, final long timeoutMillis) default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public void addBroker(final String addr, final String brokerConfigPath, final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { AddBrokerRequestHeader requestHeader = new AddBrokerRequestHeader(); requestHeader.setConfigPath(brokerConfigPath); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.ADD_BROKER, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; - + switch (response.getCode()) { case ResponseCode.SUCCESS: return; default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public void removeBroker(final String addr, String clusterName, String brokerName, long brokerId, - final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { RemoveBrokerRequestHeader requestHeader = new RemoveBrokerRequestHeader(); requestHeader.setBrokerClusterName(clusterName); requestHeader.setBrokerName(brokerName); requestHeader.setBrokerId(brokerId); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.REMOVE_BROKER, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; - + switch (response.getCode()) { case ResponseCode.SUCCESS: return; default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public void updateBrokerConfig(final String addr, final Properties properties, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException, MQClientException, UnsupportedEncodingException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, MQClientException, UnsupportedEncodingException { Validators.checkBrokerConfig(properties); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_BROKER_CONFIG, null); - + String str = MixAll.properties2String(properties); if (str != null && str.length() > 0) { request.setBody(str.getBytes(MixAll.DEFAULT_CHARSET)); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return; @@ -1916,16 +1857,15 @@ public void updateBrokerConfig(final String addr, final Properties properties, f default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } } - + public Properties getBrokerConfig(final String addr, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException, UnsupportedEncodingException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CONFIG, null); - + RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -1935,18 +1875,20 @@ public Properties getBrokerConfig(final String addr, final long timeoutMillis) default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void updateColdDataFlowCtrGroupConfig(final String addr, final Properties properties, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_COLD_DATA_FLOW_CTR_CONFIG, null); + + public void updateColdDataFlowCtrGroupConfig(final String addr, final Properties properties, + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_COLD_DATA_FLOW_CTR_CONFIG, + null); String str = MixAll.properties2String(properties); if (str != null && str.length() > 0) { request.setBody(str.getBytes(MixAll.DEFAULT_CHARSET)); RemotingCommand response = this.remotingClient.invokeSync( - MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return; @@ -1957,14 +1899,16 @@ public void updateColdDataFlowCtrGroupConfig(final String addr, final Properties throw new MQBrokerException(response.getCode(), response.getRemark()); } } - - public void removeColdDataFlowCtrGroupConfig(final String addr, final String consumerGroup, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.REMOVE_COLD_DATA_FLOW_CTR_CONFIG, null); + + public void removeColdDataFlowCtrGroupConfig(final String addr, final String consumerGroup, + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.REMOVE_COLD_DATA_FLOW_CTR_CONFIG, + null); if (consumerGroup != null && consumerGroup.length() > 0) { request.setBody(consumerGroup.getBytes(MixAll.DEFAULT_CHARSET)); RemotingCommand response = this.remotingClient.invokeSync( - MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return; @@ -1975,9 +1919,9 @@ public void removeColdDataFlowCtrGroupConfig(final String addr, final String con throw new MQBrokerException(response.getCode(), response.getRemark()); } } - + public String getColdDataFlowCtrInfo(final String addr, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, UnsupportedEncodingException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_COLD_DATA_FLOW_CTR_INFO, null); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; @@ -1993,9 +1937,9 @@ public String getColdDataFlowCtrInfo(final String addr, final long timeoutMillis } throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public String setCommitLogReadAheadMode(final String addr, final String mode, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SET_COMMITLOG_READ_MODE, null); HashMap extFields = new HashMap<>(); extFields.put(FIleReadaheadMode.READ_AHEAD_MODE, mode); @@ -2014,12 +1958,11 @@ public String setCommitLogReadAheadMode(final String addr, final String mode, fi } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public ClusterInfo getBrokerClusterInfo( - final long timeoutMillis) throws InterruptedException, RemotingTimeoutException, - RemotingSendRequestException, RemotingConnectException, MQBrokerException { + + public ClusterInfo getBrokerClusterInfo(final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CLUSTER_INFO, null); - + RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2029,27 +1972,29 @@ public ClusterInfo getBrokerClusterInfo( default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public TopicRouteData getDefaultTopicRouteInfoFromNameServer(final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { - + throws RemotingException, MQClientException, InterruptedException { + return getTopicRouteInfoFromNameServer(TopicValidator.AUTO_CREATE_TOPIC_KEY_TOPIC, timeoutMillis, false); } - + public TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { return getTopicRouteInfoFromNameServer(topic, timeoutMillis, true); } - + public TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final long timeoutMillis, - boolean allowTopicNotExist) throws MQClientException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { + boolean allowTopicNotExist) + throws MQClientException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException { GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader(); requestHeader.setTopic(topic); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, requestHeader); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2057,7 +2002,7 @@ public TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final if (allowTopicNotExist) { log.warn("get Topic [{}] RouteInfoFromNameServer is not exist value", topic); } - + break; } case ResponseCode.SUCCESS: { @@ -2069,13 +2014,14 @@ public TopicRouteData getTopicRouteInfoFromNameServer(final String topic, final default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public TopicList getTopicListFromNameServer(final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_TOPIC_LIST_FROM_NAMESERVER, null); + throws RemotingException, MQClientException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_TOPIC_LIST_FROM_NAMESERVER, + null); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2088,46 +2034,46 @@ public TopicList getTopicListFromNameServer(final long timeoutMillis) default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public int wipeWritePermOfBroker(final String namesrvAddr, String brokerName, - final long timeoutMillis) throws RemotingCommandException, - RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { + + public int wipeWritePermOfBroker(final String namesrvAddr, String brokerName, final long timeoutMillis) + throws RemotingCommandException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { WipeWritePermOfBrokerRequestHeader requestHeader = new WipeWritePermOfBrokerRequestHeader(); requestHeader.setBrokerName(brokerName); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.WIPE_WRITE_PERM_OF_BROKER, requestHeader); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.WIPE_WRITE_PERM_OF_BROKER, + requestHeader); RemotingCommand response = this.remotingClient.invokeSync(namesrvAddr, request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - WipeWritePermOfBrokerResponseHeader responseHeader = - (WipeWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader(WipeWritePermOfBrokerResponseHeader.class); + WipeWritePermOfBrokerResponseHeader responseHeader = (WipeWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader( + WipeWritePermOfBrokerResponseHeader.class); return responseHeader.getWipeTopicCount(); } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public int addWritePermOfBroker(final String nameSrvAddr, String brokerName, final long timeoutMillis) - throws RemotingCommandException, - RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { + throws RemotingCommandException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQClientException { AddWritePermOfBrokerRequestHeader requestHeader = new AddWritePermOfBrokerRequestHeader(); requestHeader.setBrokerName(brokerName); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.ADD_WRITE_PERM_OF_BROKER, requestHeader); - + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.ADD_WRITE_PERM_OF_BROKER, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync(nameSrvAddr, request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - AddWritePermOfBrokerResponseHeader responseHeader = - (AddWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader(AddWritePermOfBrokerResponseHeader.class); + AddWritePermOfBrokerResponseHeader responseHeader = (AddWritePermOfBrokerResponseHeader) response.decodeCommandCustomHeader( + AddWritePermOfBrokerResponseHeader.class); return responseHeader.getAddTopicCount(); } default: @@ -2135,14 +2081,15 @@ public int addWritePermOfBroker(final String nameSrvAddr, String brokerName, fin } throw new MQClientException(response.getCode(), response.getRemark()); } - + public void deleteTopicInBroker(final String addr, final String topic, final long timeoutMillis) - throws RemotingException, InterruptedException, MQClientException { + throws RemotingException, InterruptedException, MQClientException { DeleteTopicRequestHeader requestHeader = new DeleteTopicRequestHeader(); requestHeader.setTopic(topic); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_BROKER, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_BROKER, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2151,15 +2098,16 @@ public void deleteTopicInBroker(final String addr, final String topic, final lon default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void deleteTopicInNameServer(final String addr, final String topic, final long timeoutMillis) - throws RemotingException, InterruptedException, MQClientException { + throws RemotingException, InterruptedException, MQClientException { DeleteTopicFromNamesrvRequestHeader requestHeader = new DeleteTopicFromNamesrvRequestHeader(); requestHeader.setTopic(topic); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV, requestHeader); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV, + requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2169,17 +2117,18 @@ public void deleteTopicInNameServer(final String addr, final String topic, final default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void deleteTopicInNameServer(final String addr, final String clusterName, final String topic, - final long timeoutMillis) - throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException, MQClientException { DeleteTopicFromNamesrvRequestHeader requestHeader = new DeleteTopicFromNamesrvRequestHeader(); requestHeader.setTopic(topic); requestHeader.setClusterName(clusterName); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV, requestHeader); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_TOPIC_IN_NAMESRV, + requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2189,20 +2138,20 @@ public void deleteTopicInNameServer(final String addr, final String clusterName, default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void deleteSubscriptionGroup(final String addr, final String groupName, final boolean removeOffset, - final long timeoutMillis) - throws RemotingException, InterruptedException, MQClientException { + final long timeoutMillis) throws RemotingException, InterruptedException, MQClientException { DeleteSubscriptionGroupRequestHeader requestHeader = new DeleteSubscriptionGroupRequestHeader(); requestHeader.setGroupName(groupName); requestHeader.setCleanOffset(removeOffset); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_SUBSCRIPTIONGROUP, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2211,41 +2160,41 @@ public void deleteSubscriptionGroup(final String addr, final String groupName, f default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public String getKVConfigValue(final String namespace, final String key, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { GetKVConfigRequestHeader requestHeader = new GetKVConfigRequestHeader(); requestHeader.setNamespace(namespace); requestHeader.setKey(key); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KV_CONFIG, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - GetKVConfigResponseHeader responseHeader = - (GetKVConfigResponseHeader) response.decodeCommandCustomHeader(GetKVConfigResponseHeader.class); + GetKVConfigResponseHeader responseHeader = (GetKVConfigResponseHeader) response.decodeCommandCustomHeader( + GetKVConfigResponseHeader.class); return responseHeader.getValue(); } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void putKVConfigValue(final String namespace, final String key, final String value, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { PutKVConfigRequestHeader requestHeader = new PutKVConfigRequestHeader(); requestHeader.setNamespace(namespace); requestHeader.setKey(key); requestHeader.setValue(value); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.PUT_KV_CONFIG, requestHeader); - + List nameServerAddressList = this.remotingClient.getNameServerAddressList(); if (nameServerAddressList != null) { RemotingCommand errResponse = null; @@ -2260,21 +2209,21 @@ public void putKVConfigValue(final String namespace, final String key, final Str errResponse = response; } } - + if (errResponse != null) { throw new MQClientException(errResponse.getCode(), errResponse.getRemark()); } } } - + public void deleteKVConfigValue(final String namespace, final String key, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { DeleteKVConfigRequestHeader requestHeader = new DeleteKVConfigRequestHeader(); requestHeader.setNamespace(namespace); requestHeader.setKey(key); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_KV_CONFIG, requestHeader); - + List nameServerAddressList = this.remotingClient.getNameServerAddressList(); if (nameServerAddressList != null) { RemotingCommand errResponse = null; @@ -2294,13 +2243,14 @@ public void deleteKVConfigValue(final String namespace, final String key, final } } } - + public KVTable getKVListByNamespace(final String namespace, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { GetKVListByNamespaceRequestHeader requestHeader = new GetKVListByNamespaceRequestHeader(); requestHeader.setNamespace(namespace); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KVLIST_BY_NAMESPACE, requestHeader); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_KVLIST_BY_NAMESPACE, + requestHeader); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2310,20 +2260,20 @@ public KVTable getKVListByNamespace(final String namespace, final long timeoutMi default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public Map invokeBrokerToResetOffset(final String addr, final String topic, final String group, - final long timestamp, final boolean isForce, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + final long timestamp, final boolean isForce, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { return invokeBrokerToResetOffset(addr, topic, group, timestamp, isForce, timeoutMillis, false); } - + public Map invokeBrokerToResetOffset(final String addr, final String topic, final String group, - final long timestamp, int queueId, Long offset, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { - + final long timestamp, int queueId, Long offset, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { + ResetOffsetRequestHeader requestHeader = new ResetOffsetRequestHeader(); requestHeader.setTopic(topic); requestHeader.setGroup(group); @@ -2331,10 +2281,10 @@ public Map invokeBrokerToResetOffset(final String addr, fina requestHeader.setTimestamp(timestamp); requestHeader.setOffset(offset); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.INVOKE_BROKER_TO_RESET_OFFSET, - requestHeader); - + requestHeader); + RemotingCommand response = remotingClient.invokeSync( - MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { if (null != response.getBody()) { @@ -2345,18 +2295,18 @@ public Map invokeBrokerToResetOffset(final String addr, fina case ResponseCode.TOPIC_NOT_EXIST: case ResponseCode.SUBSCRIPTION_NOT_EXIST: case ResponseCode.SYSTEM_ERROR: - log.warn("Invoke broker to reset offset error code={}, remark={}", - response.getCode(), response.getRemark()); + log.warn("Invoke broker to reset offset error code={}, remark={}", response.getCode(), + response.getRemark()); break; default: break; } throw new MQClientException(response.getCode(), response.getRemark()); } - + public Map invokeBrokerToResetOffset(final String addr, final String topic, final String group, - final long timestamp, final boolean isForce, final long timeoutMillis, boolean isC) - throws RemotingException, MQClientException, InterruptedException { + final long timestamp, final boolean isForce, final long timeoutMillis, boolean isC) + throws RemotingException, MQClientException, InterruptedException { ResetOffsetRequestHeader requestHeader = new ResetOffsetRequestHeader(); requestHeader.setTopic(topic); requestHeader.setGroup(group); @@ -2364,13 +2314,14 @@ public Map invokeBrokerToResetOffset(final String addr, fina requestHeader.setForce(isForce); // offset is -1 means offset is null requestHeader.setOffset(-1L); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.INVOKE_BROKER_TO_RESET_OFFSET, requestHeader); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.INVOKE_BROKER_TO_RESET_OFFSET, + requestHeader); if (isC) { request.setLanguage(LanguageCode.CPP); } - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2382,46 +2333,47 @@ public Map invokeBrokerToResetOffset(final String addr, fina default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public Map> invokeBrokerToGetConsumerStatus(final String addr, final String topic, - final String group, - final String clientAddr, - final long timeoutMillis) throws RemotingException, MQClientException, InterruptedException { + final String group, final String clientAddr, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { GetConsumerStatusRequestHeader requestHeader = new GetConsumerStatusRequestHeader(); requestHeader.setTopic(topic); requestHeader.setGroup(group); requestHeader.setClientAddr(clientAddr); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.INVOKE_BROKER_TO_GET_CONSUMER_STATUS, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.INVOKE_BROKER_TO_GET_CONSUMER_STATUS, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { if (response.getBody() != null) { - GetConsumerStatusBody body = GetConsumerStatusBody.decode(response.getBody(), GetConsumerStatusBody.class); + GetConsumerStatusBody body = GetConsumerStatusBody.decode(response.getBody(), + GetConsumerStatusBody.class); return body.getConsumerTable(); } } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public GroupList queryTopicConsumeByWho(final String addr, final String topic, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { QueryTopicConsumeByWhoRequestHeader requestHeader = new QueryTopicConsumeByWhoRequestHeader(); requestHeader.setTopic(topic); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_TOPIC_CONSUME_BY_WHO, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_TOPIC_CONSUME_BY_WHO, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { GroupList groupList = GroupList.decode(response.getBody(), GroupList.class); @@ -2430,19 +2382,19 @@ public GroupList queryTopicConsumeByWho(final String addr, final String topic, f default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public TopicList queryTopicsByConsumer(final String addr, final String group, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { QueryTopicsByConsumerRequestHeader requestHeader = new QueryTopicsByConsumerRequestHeader(); requestHeader.setGroup(group); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_TOPICS_BY_CONSUMER, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_TOPICS_BY_CONSUMER, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { TopicList topicList = TopicList.decode(response.getBody(), TopicList.class); @@ -2451,62 +2403,64 @@ public TopicList queryTopicsByConsumer(final String addr, final String group, fi default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public SubscriptionData querySubscriptionByConsumer(final String addr, final String group, final String topic, - final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { QuerySubscriptionByConsumerRequestHeader requestHeader = new QuerySubscriptionByConsumerRequestHeader(); requestHeader.setGroup(group); requestHeader.setTopic(topic); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_SUBSCRIPTION_BY_CONSUMER, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_SUBSCRIPTION_BY_CONSUMER, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { - QuerySubscriptionResponseBody subscriptionResponseBody = - QuerySubscriptionResponseBody.decode(response.getBody(), QuerySubscriptionResponseBody.class); + QuerySubscriptionResponseBody subscriptionResponseBody = QuerySubscriptionResponseBody.decode( + response.getBody(), QuerySubscriptionResponseBody.class); return subscriptionResponseBody.getSubscriptionData(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public List queryConsumeTimeSpan(final String addr, final String topic, final String group, - final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, - MQBrokerException { + final long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { QueryConsumeTimeSpanRequestHeader requestHeader = new QueryConsumeTimeSpanRequestHeader(); requestHeader.setTopic(topic); requestHeader.setGroup(group); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUME_TIME_SPAN, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUME_TIME_SPAN, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { - QueryConsumeTimeSpanBody consumeTimeSpanBody = GroupList.decode(response.getBody(), QueryConsumeTimeSpanBody.class); + QueryConsumeTimeSpanBody consumeTimeSpanBody = GroupList.decode(response.getBody(), + QueryConsumeTimeSpanBody.class); return consumeTimeSpanBody.getConsumeTimeSpanSet(); } default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public TopicList getTopicsByCluster(final String cluster, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { GetTopicsByClusterRequestHeader requestHeader = new GetTopicsByClusterRequestHeader(); requestHeader.setCluster(cluster); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPICS_BY_CLUSTER, requestHeader); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPICS_BY_CLUSTER, + requestHeader); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2520,12 +2474,12 @@ public TopicList getTopicsByCluster(final String cluster, final long timeoutMill default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public TopicList getSystemTopicList( - final long timeoutMillis) throws RemotingException, MQClientException, InterruptedException { + + public TopicList getSystemTopicList(final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SYSTEM_TOPIC_LIST_FROM_NS, null); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; @@ -2534,8 +2488,8 @@ public TopicList getSystemTopicList( byte[] body = response.getBody(); if (body != null) { TopicList topicList = TopicList.decode(response.getBody(), TopicList.class); - if (topicList.getTopicList() != null && !topicList.getTopicList().isEmpty() - && !UtilAll.isBlank(topicList.getBrokerAddr())) { + if (topicList.getTopicList() != null && !topicList.getTopicList().isEmpty() && !UtilAll.isBlank( + topicList.getBrokerAddr())) { TopicList tmp = getSystemTopicListFromBroker(topicList.getBrokerAddr(), timeoutMillis); if (tmp.getTopicList() != null && !tmp.getTopicList().isEmpty()) { topicList.getTopicList().addAll(tmp.getTopicList()); @@ -2547,15 +2501,16 @@ public TopicList getSystemTopicList( default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public TopicList getSystemTopicListFromBroker(final String addr, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SYSTEM_TOPIC_LIST_FROM_BROKER, null); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + throws RemotingException, MQClientException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SYSTEM_TOPIC_LIST_FROM_BROKER, + null); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2568,16 +2523,15 @@ public TopicList getSystemTopicListFromBroker(final String addr, final long time default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public boolean cleanExpiredConsumeQueue(final String addr, - long timeoutMillis) throws MQClientException, RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, InterruptedException { + + public boolean cleanExpiredConsumeQueue(final String addr, long timeoutMillis) + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CLEAN_EXPIRED_CONSUMEQUEUE, null); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return true; @@ -2585,15 +2539,15 @@ public boolean cleanExpiredConsumeQueue(final String addr, default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public boolean deleteExpiredCommitLog(final String addr, long timeoutMillis) throws MQClientException, - RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { + + public boolean deleteExpiredCommitLog(final String addr, long timeoutMillis) + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.DELETE_EXPIRED_COMMITLOG, null); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return true; @@ -2601,16 +2555,15 @@ public boolean deleteExpiredCommitLog(final String addr, long timeoutMillis) thr default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public boolean cleanUnusedTopicByAddr(final String addr, - long timeoutMillis) throws MQClientException, RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, InterruptedException { + + public boolean cleanUnusedTopicByAddr(final String addr, long timeoutMillis) + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CLEAN_UNUSED_TOPIC, null); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { return true; @@ -2618,22 +2571,23 @@ public boolean cleanUnusedTopicByAddr(final String addr, default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public ConsumerRunningInfo getConsumerRunningInfo(final String addr, String consumerGroup, String clientId, - boolean jstack, - final long timeoutMillis) throws RemotingException, MQClientException, InterruptedException { + boolean jstack, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { GetConsumerRunningInfoRequestHeader requestHeader = new GetConsumerRunningInfoRequestHeader(); requestHeader.setConsumerGroup(consumerGroup); requestHeader.setClientId(clientId); requestHeader.setJstackEnable(jstack); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_RUNNING_INFO, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONSUMER_RUNNING_INFO, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2646,46 +2600,44 @@ public ConsumerRunningInfo getConsumerRunningInfo(final String addr, String cons default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public ConsumeMessageDirectlyResult consumeMessageDirectly(final String addr, - String consumerGroup, - String clientId, - String topic, - String msgId, - final long timeoutMillis) throws RemotingException, MQClientException, InterruptedException { + + public ConsumeMessageDirectlyResult consumeMessageDirectly(final String addr, String consumerGroup, String clientId, + String topic, String msgId, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { ConsumeMessageDirectlyResultRequestHeader requestHeader = new ConsumeMessageDirectlyResultRequestHeader(); requestHeader.setTopic(topic); requestHeader.setConsumerGroup(consumerGroup); requestHeader.setClientId(clientId); requestHeader.setMsgId(msgId); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONSUME_MESSAGE_DIRECTLY, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONSUME_MESSAGE_DIRECTLY, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { byte[] body = response.getBody(); if (body != null) { - ConsumeMessageDirectlyResult info = ConsumeMessageDirectlyResult.decode(body, ConsumeMessageDirectlyResult.class); + ConsumeMessageDirectlyResult info = ConsumeMessageDirectlyResult.decode(body, + ConsumeMessageDirectlyResult.class); return info; } } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public Map queryCorrectionOffset(final String addr, final String topic, final String group, - Set filterGroup, - long timeoutMillis) throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, - InterruptedException { + Set filterGroup, long timeoutMillis) + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { QueryCorrectionOffsetHeader requestHeader = new QueryCorrectionOffsetHeader(); requestHeader.setCompareGroup(group); requestHeader.setTopic(topic); @@ -2698,26 +2650,28 @@ public Map queryCorrectionOffset(final String addr, final String } requestHeader.setFilterGroups(sb.toString()); } - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CORRECTION_OFFSET, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CORRECTION_OFFSET, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { if (response.getBody() != null) { - QueryCorrectionOffsetBody body = QueryCorrectionOffsetBody.decode(response.getBody(), QueryCorrectionOffsetBody.class); + QueryCorrectionOffsetBody body = QueryCorrectionOffsetBody.decode(response.getBody(), + QueryCorrectionOffsetBody.class); return body.getCorrectionOffsets(); } } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public TopicList getUnitTopicList(final boolean containRetry, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_UNIT_TOPIC_LIST, null); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; @@ -2735,19 +2689,19 @@ public TopicList getUnitTopicList(final boolean containRetry, final long timeout } } } - + return topicList; } } default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public TopicList getHasUnitSubTopicList(final boolean containRetry, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { + throws RemotingException, MQClientException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_HAS_UNIT_SUB_TOPIC_LIST, null); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; @@ -2771,13 +2725,14 @@ public TopicList getHasUnitSubTopicList(final boolean containRetry, final long t default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public TopicList getHasUnitSubUnUnitTopicList(final boolean containRetry, final long timeoutMillis) - throws RemotingException, MQClientException, InterruptedException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST, null); + throws RemotingException, MQClientException, InterruptedException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_HAS_UNIT_SUB_UNUNIT_TOPIC_LIST, + null); RemotingCommand response = this.remotingClient.invokeSync(null, request, timeoutMillis); assert response != null; switch (response.getCode()) { @@ -2800,21 +2755,21 @@ public TopicList getHasUnitSubUnUnitTopicList(final boolean containRetry, final default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public void cloneGroupOffset(final String addr, final String srcGroup, final String destGroup, final String topic, - final boolean isOffline, - final long timeoutMillis) throws RemotingException, MQClientException, InterruptedException { + final boolean isOffline, final long timeoutMillis) + throws RemotingException, MQClientException, InterruptedException { CloneGroupOffsetRequestHeader requestHeader = new CloneGroupOffsetRequestHeader(); requestHeader.setSrcGroup(srcGroup); requestHeader.setDestGroup(destGroup); requestHeader.setTopic(topic); requestHeader.setOffline(isOffline); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CLONE_GROUP_OFFSET, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2823,20 +2778,20 @@ public void cloneGroupOffset(final String addr, final String srcGroup, final Str default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - + public BrokerStatsData viewBrokerStatsData(String brokerAddr, String statsName, String statsKey, long timeoutMillis) - throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, - InterruptedException { + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { ViewBrokerStatsDataRequestHeader requestHeader = new ViewBrokerStatsDataRequestHeader(); requestHeader.setStatsName(statsName); requestHeader.setStatsKey(statsKey); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.VIEW_BROKER_STATS_DATA, requestHeader); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.VIEW_BROKER_STATS_DATA, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2848,24 +2803,23 @@ public BrokerStatsData viewBrokerStatsData(String brokerAddr, String statsName, default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public Set getClusterList(String topic, - long timeoutMillis) { + + public Set getClusterList(String topic, long timeoutMillis) { return Collections.EMPTY_SET; } - - public ConsumeStatsList fetchConsumeStatsInBroker(String brokerAddr, boolean isOrder, - long timeoutMillis) throws MQClientException, - RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { + + public ConsumeStatsList fetchConsumeStatsInBroker(String brokerAddr, boolean isOrder, long timeoutMillis) + throws MQClientException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException { GetConsumeStatsInBrokerHeader requestHeader = new GetConsumeStatsInBrokerHeader(); requestHeader.setIsOrder(isOrder); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CONSUME_STATS, requestHeader); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_CONSUME_STATS, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2877,16 +2831,16 @@ public ConsumeStatsList fetchConsumeStatsInBroker(String brokerAddr, boolean isO default: break; } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public SubscriptionGroupWrapper getAllSubscriptionGroup(final String brokerAddr, - long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_SUBSCRIPTIONGROUP_CONFIG, null); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + + public SubscriptionGroupWrapper getAllSubscriptionGroup(final String brokerAddr, long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_SUBSCRIPTIONGROUP_CONFIG, + null); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2897,15 +2851,15 @@ public SubscriptionGroupWrapper getAllSubscriptionGroup(final String brokerAddr, } throw new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr); } - - public SubscriptionGroupConfig getSubscriptionGroupConfig(final String brokerAddr, String group, - long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + + public SubscriptionGroupConfig getSubscriptionGroupConfig(final String brokerAddr, String group, long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { GetSubscriptionGroupConfigRequestHeader header = new GetSubscriptionGroupConfigRequestHeader(); header.setGroup(group); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SUBSCRIPTIONGROUP_CONFIG, header); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SUBSCRIPTIONGROUP_CONFIG, + header); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2916,14 +2870,13 @@ public SubscriptionGroupConfig getSubscriptionGroupConfig(final String brokerAdd } throw new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr); } - - public TopicConfigSerializeWrapper getAllTopicConfig(final String addr, - long timeoutMillis) throws RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public TopicConfigSerializeWrapper getAllTopicConfig(final String addr, long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_TOPIC_CONFIG, null); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -2932,26 +2885,26 @@ public TopicConfigSerializeWrapper getAllTopicConfig(final String addr, default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public void updateNameServerConfig(final Properties properties, final List nameServers, long timeoutMillis) - throws UnsupportedEncodingException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, - RemotingConnectException, MQClientException { + throws UnsupportedEncodingException, InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { String str = MixAll.properties2String(properties); if (str == null || str.length() < 1) { return; } - List invokeNameServers = (nameServers == null || nameServers.isEmpty()) ? - this.remotingClient.getNameServerAddressList() : nameServers; + List invokeNameServers = + (nameServers == null || nameServers.isEmpty()) ? this.remotingClient.getNameServerAddressList() + : nameServers; if (invokeNameServers == null || invokeNameServers.isEmpty()) { return; } - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_NAMESRV_CONFIG, null); request.setBody(str.getBytes(MixAll.DEFAULT_CHARSET)); - + RemotingCommand errResponse = null; for (String nameServer : invokeNameServers) { RemotingCommand response = this.remotingClient.invokeSync(nameServer, request, timeoutMillis); @@ -2964,70 +2917,71 @@ public void updateNameServerConfig(final Properties properties, final List getNameServerConfig(final List nameServers, long timeoutMillis) - throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, - MQClientException, UnsupportedEncodingException { - List invokeNameServers = (nameServers == null || nameServers.isEmpty()) ? - this.remotingClient.getNameServerAddressList() : nameServers; + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException, UnsupportedEncodingException { + List invokeNameServers = + (nameServers == null || nameServers.isEmpty()) ? this.remotingClient.getNameServerAddressList() + : nameServers; if (invokeNameServers == null || invokeNameServers.isEmpty()) { return null; } - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_NAMESRV_CONFIG, null); - + Map configMap = new HashMap<>(4); for (String nameServer : invokeNameServers) { RemotingCommand response = this.remotingClient.invokeSync(nameServer, request, timeoutMillis); - + assert response != null; - + if (ResponseCode.SUCCESS == response.getCode()) { - configMap.put(nameServer, MixAll.string2Properties(new String(response.getBody(), MixAll.DEFAULT_CHARSET))); + configMap.put(nameServer, + MixAll.string2Properties(new String(response.getBody(), MixAll.DEFAULT_CHARSET))); } else { throw new MQClientException(response.getCode(), response.getRemark()); } } return configMap; } - + public QueryConsumeQueueResponseBody queryConsumeQueue(final String brokerAddr, final String topic, - final int queueId, - final long index, final int count, final String consumerGroup, - final long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { - + final int queueId, final long index, final int count, final String consumerGroup, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { + QueryConsumeQueueRequestHeader requestHeader = new QueryConsumeQueueRequestHeader(); requestHeader.setTopic(topic); requestHeader.setQueueId(queueId); requestHeader.setIndex(index); requestHeader.setCount(count); requestHeader.setConsumerGroup(consumerGroup); - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUME_QUEUE, requestHeader); - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); - + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + assert response != null; - + if (ResponseCode.SUCCESS == response.getCode()) { return QueryConsumeQueueResponseBody.decode(response.getBody(), QueryConsumeQueueResponseBody.class); } - + throw new MQClientException(response.getCode(), response.getRemark()); } - - public CheckRocksdbCqWriteResult checkRocksdbCqWriteProgress(final String brokerAddr, final String topic, final long checkStoreTime, final long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { + + public CheckRocksdbCqWriteResult checkRocksdbCqWriteProgress(final String brokerAddr, final String topic, + final long checkStoreTime, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { CheckRocksdbCqWriteProgressRequestHeader header = new CheckRocksdbCqWriteProgressRequestHeader(); header.setTopic(topic); header.setCheckStoreTime(checkStoreTime); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_ROCKSDB_CQ_WRITE_PROGRESS, header); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_ROCKSDB_CQ_WRITE_PROGRESS, + header); RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis); assert response != null; if (ResponseCode.SUCCESS == response.getCode()) { @@ -3035,40 +2989,40 @@ public CheckRocksdbCqWriteResult checkRocksdbCqWriteProgress(final String broker } throw new MQClientException(response.getCode(), response.getRemark()); } - - public void checkClientInBroker(final String brokerAddr, final String consumerGroup, - final String clientId, final SubscriptionData subscriptionData, - final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, - RemotingConnectException, MQClientException { + + public void checkClientInBroker(final String brokerAddr, final String consumerGroup, final String clientId, + final SubscriptionData subscriptionData, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_CLIENT_CONFIG, null); - + CheckClientRequestBody requestBody = new CheckClientRequestBody(); requestBody.setClientId(clientId); requestBody.setGroup(consumerGroup); requestBody.setSubscriptionData(subscriptionData); - + request.setBody(requestBody.encode()); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); - + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + assert response != null; - + if (ResponseCode.SUCCESS != response.getCode()) { throw new MQClientException(response.getCode(), response.getRemark()); } } - - public boolean resumeCheckHalfMessage(final String addr, String topic, String msgId, - final long timeoutMillis) throws RemotingException, InterruptedException { + + public boolean resumeCheckHalfMessage(final String addr, String topic, String msgId, final long timeoutMillis) + throws RemotingException, InterruptedException { ResumeCheckHalfMessageRequestHeader requestHeader = new ResumeCheckHalfMessageRequestHeader(); requestHeader.setTopic(topic); requestHeader.setMsgId(msgId); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RESUME_CHECK_HALF_MESSAGE, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RESUME_CHECK_HALF_MESSAGE, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -3079,36 +3033,35 @@ public boolean resumeCheckHalfMessage(final String addr, String topic, String ms return false; } } - + public void setMessageRequestMode(final String brokerAddr, final String topic, final String consumerGroup, - final MessageRequestMode mode, final int popShareQueueNum, final long timeoutMillis) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, - RemotingConnectException, MQClientException { + final MessageRequestMode mode, final int popShareQueueNum, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SET_MESSAGE_REQUEST_MODE, null); - + SetMessageRequestModeRequestBody requestBody = new SetMessageRequestModeRequestBody(); requestBody.setTopic(topic); requestBody.setConsumerGroup(consumerGroup); requestBody.setMode(mode); requestBody.setPopShareQueueNum(popShareQueueNum); request.setBody(requestBody.encode()); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; if (ResponseCode.SUCCESS != response.getCode()) { throw new MQClientException(response.getCode(), response.getRemark()); } } - - public TopicConfigAndQueueMapping getTopicConfig(final String brokerAddr, String topic, - long timeoutMillis) throws InterruptedException, - RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + + public TopicConfigAndQueueMapping getTopicConfig(final String brokerAddr, String topic, long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { GetTopicConfigRequestHeader header = new GetTopicConfigRequestHeader(); header.setTopic(topic); header.setLo(true); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_CONFIG, header); - RemotingCommand response = this.remotingClient - .invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), brokerAddr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -3124,10 +3077,10 @@ public TopicConfigAndQueueMapping getTopicConfig(final String brokerAddr, String } throw new MQBrokerException(response.getCode(), response.getRemark()); } - + public void createStaticTopic(final String addr, final String defaultTopic, final TopicConfig topicConfig, - final TopicQueueMappingDetail topicQueueMappingDetail, boolean force, - final long timeoutMillis) throws RemotingException, InterruptedException, MQBrokerException { + final TopicQueueMappingDetail topicQueueMappingDetail, boolean force, final long timeoutMillis) + throws RemotingException, InterruptedException, MQBrokerException { CreateTopicRequestHeader requestHeader = new CreateTopicRequestHeader(); requestHeader.setTopic(topicConfig.getTopicName()); requestHeader.setDefaultTopic(defaultTopic); @@ -3138,11 +3091,12 @@ public void createStaticTopic(final String addr, final String defaultTopic, fina requestHeader.setTopicSysFlag(topicConfig.getTopicSysFlag()); requestHeader.setOrder(topicConfig.isOrder()); requestHeader.setForce(force); - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_STATIC_TOPIC, requestHeader); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_CREATE_STATIC_TOPIC, + requestHeader); request.setBody(topicQueueMappingDetail.encode()); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -3151,10 +3105,10 @@ public void createStaticTopic(final String addr, final String defaultTopic, fina default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - + /** * @param addr * @param requestHeader @@ -3166,11 +3120,13 @@ public void createStaticTopic(final String addr, final String defaultTopic, fina * @throws MQBrokerException */ public GroupForbidden updateAndGetGroupForbidden(String addr, UpdateGroupForbiddenRequestHeader requestHeader, - long timeoutMillis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_GET_GROUP_FORBIDDEN, requestHeader); - - RemotingCommand response = this.remotingClient.invokeSync(MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), - request, timeoutMillis); + long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_AND_GET_GROUP_FORBIDDEN, + requestHeader); + + RemotingCommand response = this.remotingClient.invokeSync( + MixAll.brokerVIPChannel(this.clientConfig.isVipChannelEnabled(), addr), request, timeoutMillis); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { @@ -3179,17 +3135,18 @@ public GroupForbidden updateAndGetGroupForbidden(String addr, UpdateGroupForbidd default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - + public void resetMasterFlushOffset(final String brokerAddr, final long masterFlushOffset) - throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQBrokerException { ResetMasterFlushOffsetHeader requestHeader = new ResetMasterFlushOffsetHeader(); requestHeader.setMasterFlushOffset(masterFlushOffset); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RESET_MASTER_FLUSH_OFFSET, requestHeader); - + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RESET_MASTER_FLUSH_OFFSET, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, 3000); assert response != null; switch (response.getCode()) { @@ -3201,14 +3158,13 @@ public void resetMasterFlushOffset(final String brokerAddr, final long masterFlu } throw new MQBrokerException(response.getCode(), response.getRemark(), brokerAddr); } - + public HARuntimeInfo getBrokerHAStatus(final String brokerAddr, final long timeoutMillis) - throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, - InterruptedException, MQBrokerException { + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_HA_STATUS, null); RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis); assert response != null; - + switch (response.getCode()) { case ResponseCode.SUCCESS: { return HARuntimeInfo.decode(response.getBody(), HARuntimeInfo.class); @@ -3216,13 +3172,14 @@ public HARuntimeInfo getBrokerHAStatus(final String brokerAddr, final long timeo default: break; } - + throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public GetMetaDataResponseHeader getControllerMetaData( - final String controllerAddress) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, RemotingCommandException, MQBrokerException { - final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_GET_METADATA_INFO, null); + + public GetMetaDataResponseHeader getControllerMetaData(final String controllerAddress) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, RemotingCommandException, MQBrokerException { + final RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_GET_METADATA_INFO, + null); final RemotingCommand response = this.remotingClient.invokeSync(controllerAddress, request, 3000); assert response != null; if (response.getCode() == SUCCESS) { @@ -3230,16 +3187,17 @@ public GetMetaDataResponseHeader getControllerMetaData( } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public BrokerReplicasInfo getInSyncStateData(final String controllerAddress, - final List brokers) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, RemotingCommandException { + + public BrokerReplicasInfo getInSyncStateData(final String controllerAddress, final List brokers) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException, RemotingCommandException { // Get controller leader address. final GetMetaDataResponseHeader controllerMetaData = getControllerMetaData(controllerAddress); assert controllerMetaData != null; assert controllerMetaData.getControllerLeaderAddress() != null; final String leaderAddress = controllerMetaData.getControllerLeaderAddress(); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_GET_SYNC_STATE_DATA, null); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_GET_SYNC_STATE_DATA, + null); final byte[] body = RemotingSerializable.encode(brokers); request.setBody(body); RemotingCommand response = this.remotingClient.invokeSync(leaderAddress, request, 3000); @@ -3253,9 +3211,9 @@ public BrokerReplicasInfo getInSyncStateData(final String controllerAddress, } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public EpochEntryCache getBrokerEpochCache( - String brokerAddr) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public EpochEntryCache getBrokerEpochCache(String brokerAddr) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_BROKER_EPOCH_CACHE, null); final RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, 3000); assert response != null; @@ -3268,44 +3226,44 @@ public EpochEntryCache getBrokerEpochCache( } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public Map getControllerConfig(final List controllerServers, - final long timeoutMillis) throws InterruptedException, RemotingTimeoutException, - RemotingSendRequestException, RemotingConnectException, MQClientException, UnsupportedEncodingException { - List invokeControllerServers = (controllerServers == null || controllerServers.isEmpty()) ? - this.remotingClient.getNameServerAddressList() : controllerServers; + + public Map getControllerConfig(final List controllerServers, final long timeoutMillis) + throws InterruptedException, RemotingTimeoutException, RemotingSendRequestException, RemotingConnectException, MQClientException, UnsupportedEncodingException { + List invokeControllerServers = (controllerServers == null || controllerServers.isEmpty()) + ? this.remotingClient.getNameServerAddressList() : controllerServers; if (invokeControllerServers == null || invokeControllerServers.isEmpty()) { return null; } - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_CONTROLLER_CONFIG, null); - + Map configMap = new HashMap<>(4); for (String controller : invokeControllerServers) { RemotingCommand response = this.remotingClient.invokeSync(controller, request, timeoutMillis); - + assert response != null; - + if (ResponseCode.SUCCESS == response.getCode()) { - configMap.put(controller, MixAll.string2Properties(new String(response.getBody(), MixAll.DEFAULT_CHARSET))); + configMap.put(controller, + MixAll.string2Properties(new String(response.getBody(), MixAll.DEFAULT_CHARSET))); } else { throw new MQClientException(response.getCode(), response.getRemark()); } } return configMap; } - + public void updateControllerConfig(final Properties properties, final List controllers, - final long timeoutMillis) throws InterruptedException, RemotingConnectException, UnsupportedEncodingException, - RemotingSendRequestException, RemotingTimeoutException, MQClientException { + final long timeoutMillis) + throws InterruptedException, RemotingConnectException, UnsupportedEncodingException, RemotingSendRequestException, RemotingTimeoutException, MQClientException { String str = MixAll.properties2String(properties); if (str.length() < 1 || controllers == null || controllers.isEmpty()) { return; } - + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.UPDATE_CONTROLLER_CONFIG, null); request.setBody(str.getBytes(MixAll.DEFAULT_CHARSET)); - + RemotingCommand errResponse = null; for (String controller : controllers) { RemotingCommand response = this.remotingClient.invokeSync(controller, request, timeoutMillis); @@ -3318,30 +3276,34 @@ public void updateControllerConfig(final Properties properties, final List electMaster(String controllerAddr, String clusterName, - String brokerName, - Long brokerId) throws MQBrokerException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, RemotingCommandException { - + String brokerName, Long brokerId) + throws MQBrokerException, RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, RemotingCommandException { + //get controller leader address final GetMetaDataResponseHeader controllerMetaData = this.getControllerMetaData(controllerAddr); assert controllerMetaData != null; assert controllerMetaData.getControllerLeaderAddress() != null; final String leaderAddress = controllerMetaData.getControllerLeaderAddress(); - ElectMasterRequestHeader electRequestHeader = ElectMasterRequestHeader.ofAdminTrigger(clusterName, brokerName, brokerId); - - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_ELECT_MASTER, electRequestHeader); + ElectMasterRequestHeader electRequestHeader = ElectMasterRequestHeader.ofAdminTrigger(clusterName, brokerName, + brokerId); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CONTROLLER_ELECT_MASTER, + electRequestHeader); final RemotingCommand response = this.remotingClient.invokeSync(leaderAddress, request, 3000); assert response != null; switch (response.getCode()) { case ResponseCode.SUCCESS: { - BrokerMemberGroup brokerMemberGroup = RemotingSerializable.decode(response.getBody(), BrokerMemberGroup.class); - ElectMasterResponseHeader responseHeader = (ElectMasterResponseHeader) response.decodeCommandCustomHeader(ElectMasterResponseHeader.class); + BrokerMemberGroup brokerMemberGroup = RemotingSerializable.decode(response.getBody(), + BrokerMemberGroup.class); + ElectMasterResponseHeader responseHeader = (ElectMasterResponseHeader) response.decodeCommandCustomHeader( + ElectMasterResponseHeader.class); return new Pair<>(responseHeader, brokerMemberGroup); } default: @@ -3349,20 +3311,21 @@ public Pair electMaster(String con } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void cleanControllerBrokerData(String controllerAddr, String clusterName, - String brokerName, String brokerControllerIdsToClean, boolean isCleanLivingBroker) - throws RemotingException, InterruptedException, MQBrokerException { - + + public void cleanControllerBrokerData(String controllerAddr, String clusterName, String brokerName, + String brokerControllerIdsToClean, boolean isCleanLivingBroker) + throws RemotingException, InterruptedException, MQBrokerException { + //get controller leader address final GetMetaDataResponseHeader controllerMetaData = this.getControllerMetaData(controllerAddr); assert controllerMetaData != null; assert controllerMetaData.getControllerLeaderAddress() != null; final String leaderAddress = controllerMetaData.getControllerLeaderAddress(); - - CleanControllerBrokerDataRequestHeader cleanHeader = new CleanControllerBrokerDataRequestHeader(clusterName, brokerName, brokerControllerIdsToClean, isCleanLivingBroker); + + CleanControllerBrokerDataRequestHeader cleanHeader = new CleanControllerBrokerDataRequestHeader(clusterName, + brokerName, brokerControllerIdsToClean, isCleanLivingBroker); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CLEAN_BROKER_DATA, cleanHeader); - + final RemotingCommand response = this.remotingClient.invokeSync(leaderAddress, request, 3000); assert response != null; switch (response.getCode()) { @@ -3374,8 +3337,9 @@ public void cleanControllerBrokerData(String controllerAddr, String clusterName, } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void createUser(String addr, UserInfo userInfo, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void createUser(String addr, UserInfo userInfo, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { CreateUserRequestHeader requestHeader = new CreateUserRequestHeader(userInfo.getUsername()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_CREATE_USER, requestHeader); request.setBody(RemotingSerializable.encode(userInfo)); @@ -3390,8 +3354,9 @@ public void createUser(String addr, UserInfo userInfo, long millis) throws Remot } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void updateUser(String addr, UserInfo userInfo, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void updateUser(String addr, UserInfo userInfo, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { UpdateUserRequestHeader requestHeader = new UpdateUserRequestHeader(userInfo.getUsername()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_UPDATE_USER, requestHeader); request.setBody(RemotingSerializable.encode(userInfo)); @@ -3406,8 +3371,9 @@ public void updateUser(String addr, UserInfo userInfo, long millis) throws Remot } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void deleteUser(String addr, String username, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void deleteUser(String addr, String username, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { DeleteUserRequestHeader requestHeader = new DeleteUserRequestHeader(username); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_DELETE_USER, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3421,8 +3387,9 @@ public void deleteUser(String addr, String username, long millis) throws Remotin } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public UserInfo getUser(String addr, String username, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public UserInfo getUser(String addr, String username, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { GetUserRequestHeader requestHeader = new GetUserRequestHeader(username); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_GET_USER, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3436,8 +3403,9 @@ public UserInfo getUser(String addr, String username, long millis) throws Remoti } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public List listUser(String addr, String filter, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public List listUser(String addr, String filter, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { ListUsersRequestHeader requestHeader = new ListUsersRequestHeader(filter); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_LIST_USER, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3451,8 +3419,9 @@ public List listUser(String addr, String filter, long millis) throws R } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void createAcl(String addr, AclInfo aclInfo, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void createAcl(String addr, AclInfo aclInfo, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { CreateAclRequestHeader requestHeader = new CreateAclRequestHeader(aclInfo.getSubject()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_CREATE_ACL, requestHeader); request.setBody(RemotingSerializable.encode(aclInfo)); @@ -3467,8 +3436,9 @@ public void createAcl(String addr, AclInfo aclInfo, long millis) throws Remoting } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void updateAcl(String addr, AclInfo aclInfo, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void updateAcl(String addr, AclInfo aclInfo, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { UpdateAclRequestHeader requestHeader = new UpdateAclRequestHeader(aclInfo.getSubject()); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_UPDATE_ACL, requestHeader); request.setBody(RemotingSerializable.encode(aclInfo)); @@ -3483,8 +3453,9 @@ public void updateAcl(String addr, AclInfo aclInfo, long millis) throws Remoting } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public void deleteAcl(String addr, String subject, String resource, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public void deleteAcl(String addr, String subject, String resource, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { DeleteAclRequestHeader requestHeader = new DeleteAclRequestHeader(subject, resource); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_DELETE_ACL, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3498,8 +3469,9 @@ public void deleteAcl(String addr, String subject, String resource, long millis) } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public AclInfo getAcl(String addr, String subject, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public AclInfo getAcl(String addr, String subject, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { GetAclRequestHeader requestHeader = new GetAclRequestHeader(subject); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_GET_ACL, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3513,8 +3485,9 @@ public AclInfo getAcl(String addr, String subject, long millis) throws RemotingC } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public List listAcl(String addr, String subjectFilter, String resourceFilter, long millis) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + + public List listAcl(String addr, String subjectFilter, String resourceFilter, long millis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { ListAclsRequestHeader requestHeader = new ListAclsRequestHeader(subjectFilter, resourceFilter); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.AUTH_LIST_ACL, requestHeader); RemotingCommand response = this.remotingClient.invokeSync(addr, request, millis); @@ -3528,19 +3501,16 @@ public List listAcl(String addr, String subjectFilter, String resourceF } throw new MQBrokerException(response.getCode(), response.getRemark()); } - - public String recallMessage( - final String addr, - RecallMessageRequestHeader requestHeader, - final long timeoutMillis - ) throws RemotingException, MQBrokerException, InterruptedException { + + public String recallMessage(final String addr, RecallMessageRequestHeader requestHeader, final long timeoutMillis) + throws RemotingException, MQBrokerException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RECALL_MESSAGE, requestHeader); - + RemotingCommand response = this.remotingClient.invokeSync(addr, request, timeoutMillis); switch (response.getCode()) { case ResponseCode.SUCCESS: { - RecallMessageResponseHeader responseHeader = - response.decodeCommandCustomHeader(RecallMessageResponseHeader.class); + RecallMessageResponseHeader responseHeader = response.decodeCommandCustomHeader( + RecallMessageResponseHeader.class); return responseHeader.getMsgId(); } default: @@ -3548,36 +3518,32 @@ public String recallMessage( } throw new MQBrokerException(response.getCode(), response.getRemark(), addr); } - - public void recallMessageAsync( - final String addr, - final RecallMessageRequestHeader requestHeader, - final long timeoutMillis, - final InvokeCallback invokeCallback - ) throws RemotingException, InterruptedException { + + public void recallMessageAsync(final String addr, final RecallMessageRequestHeader requestHeader, + final long timeoutMillis, final InvokeCallback invokeCallback) + throws RemotingException, InterruptedException { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.RECALL_MESSAGE, requestHeader); - + this.remotingClient.invokeAsync(addr, request, timeoutMillis, new InvokeCallback() { @Override public void operationComplete(ResponseFuture responseFuture) { } - + @Override public void operationSucceed(RemotingCommand response) { invokeCallback.operationSucceed(response); } - + @Override public void operationFail(Throwable throwable) { invokeCallback.operationFail(throwable); } }); } - - public void exportPopRecord(String brokerAddr, long timeout) throws RemotingConnectException, - RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { - RemotingCommand request = RemotingCommand.createRequestCommand( - RequestCode.POP_ROLLBACK, null); + + public void exportPopRecord(String brokerAddr, long timeout) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.POP_ROLLBACK, null); RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeout); assert response != null; if (response.getCode() == SUCCESS) { @@ -3585,4 +3551,17 @@ public void exportPopRecord(String brokerAddr, long timeout) throws RemotingConn } throw new MQBrokerException(response.getCode(), response.getRemark()); } -} + + public List checkAsyncTaskStatus(String brokerAddr, CheckAsyncTaskStatusRequestHeader requestHeader, + long timeoutMillis) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_ASYNC_TASK_STATUS, + requestHeader); + RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis); + assert response != null; + if (response.getCode() == SUCCESS) { + return RemotingSerializable.decodeList(response.getBody(), AsyncTask.class); + } + throw new MQBrokerException(response.getCode(), response.getRemark()); + } +} \ No newline at end of file diff --git a/common/src/main/java/org/apache/rocketmq/common/AsyncTask.java b/common/src/main/java/org/apache/rocketmq/common/AsyncTask.java new file mode 100644 index 00000000000..eca3d346c77 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/AsyncTask.java @@ -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 org.apache.rocketmq.common; + +import java.util.Date; + +public class AsyncTask { + + private String taskName; + + private String taskId; + + private int status; + + private Date createTime; + + private String result; + + public AsyncTask(String taskName, String taskId) { + this.taskName = taskName; + this.taskId = taskId; + this.status = TaskStatus.INIT.getValue(); + this.createTime = new Date(); + this.result = null; + } + + public String getTaskName() { + return taskName; + } + + public void setTaskName(String taskName) { + this.taskName = taskName; + } + + public int getStatus() { + return status; + } + + public void setStatus(int status) { + this.status = status; + } + + public String getResult() { + return result; + } + + public void setResult(String result) { + this.result = result; + } + + public Date getCreateTime() { + return createTime; + } + + public void setCreateTime(Date createTime) { + this.createTime = createTime; + } + + public String getTaskId() { + return taskId; + } + + public void setTaskId(String taskId) { + this.taskId = taskId; + } +} \ No newline at end of file diff --git a/common/src/main/java/org/apache/rocketmq/common/TaskStatus.java b/common/src/main/java/org/apache/rocketmq/common/TaskStatus.java new file mode 100644 index 00000000000..e2bbe2acee9 --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/TaskStatus.java @@ -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 org.apache.rocketmq.common; + +public enum TaskStatus { + + INIT(0, "Initialized"), + + IN_PROGRESS(1, "In Progress"), + + ERROR(2, "Error"), + + SUCCESS(3, "Success"); + + private final int value; + + private final String desc; + + TaskStatus(int value, String desc) { + this.value = value; + this.desc = desc; + } + + public int getValue() { + return value; + } + + public String getDesc() { + return desc; + } +} \ No newline at end of file diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java index 623f5748d5a..16f5ae1f747 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java @@ -219,6 +219,7 @@ public class RequestCode { public static final int GET_SUBSCRIPTIONGROUP_CONFIG = 352; public static final int UPDATE_AND_GET_GROUP_FORBIDDEN = 353; public static final int CHECK_ROCKSDB_CQ_WRITE_PROGRESS = 354; + public static final int CHECK_ASYNC_TASK_STATUS = 355; public static final int LITE_PULL_MESSAGE = 361; public static final int RECALL_MESSAGE = 370; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusRequestHeader.java new file mode 100644 index 00000000000..04446b234cd --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusRequestHeader.java @@ -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 org.apache.rocketmq.remoting.protocol.header; + +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.action.Action; +import org.apache.rocketmq.common.action.RocketMQAction; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.protocol.RequestCode; + +@RocketMQAction(value = RequestCode.CHECK_ASYNC_TASK_STATUS, action = Action.GET) +public class CheckAsyncTaskStatusRequestHeader implements CommandCustomHeader { + + private String taskName; + + @Override + public void checkFields() throws RemotingCommandException { + if (StringUtils.isBlank(taskName)) { + throw new RemotingCommandException("taskName cannot be null or blank"); + } + } + + public String getTaskName() { + return taskName; + } + + public void setTaskName(String taskId) { + this.taskName = taskId; + } +} \ No newline at end of file diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusResponseHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusResponseHeader.java new file mode 100644 index 00000000000..13f9b8852e9 --- /dev/null +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/CheckAsyncTaskStatusResponseHeader.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.remoting.protocol.header; + +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + + +public class CheckAsyncTaskStatusResponseHeader implements CommandCustomHeader { + + private String taskId; + + private String status; + + private int progress; + + private String result; + + @Override + public void checkFields() throws RemotingCommandException { + + } + + public String getTaskId() { + return taskId; + } + + public void setTaskId(String taskId) { + this.taskId = taskId; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + public int getProgress() { + return progress; + } + + public void setProgress(int progress) { + this.progress = progress; + } + + public String getResult() { + return result; + } + + public void setResult(String result) { + this.result = result; + } +} \ No newline at end of file diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index 4b97e14866a..8eb328386da 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java @@ -25,6 +25,7 @@ import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.AsyncTask; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; import org.apache.rocketmq.common.Pair; @@ -65,6 +66,7 @@ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.remoting.protocol.body.TopicList; import org.apache.rocketmq.remoting.protocol.body.UserInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader; import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; @@ -1010,4 +1012,9 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { defaultMQAdminExtImpl.exportPopRecords(brokerAddr, timeout); } + + public List checkAsyncTaskStatus(String brokerAddr, CheckAsyncTaskStatusRequestHeader requestHeader) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + return defaultMQAdminExtImpl.checkAsyncTaskStatus(brokerAddr, requestHeader); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java index 2523013af0d..6724cc78258 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImpl.java @@ -45,6 +45,7 @@ import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.client.impl.MQClientManager; import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.AsyncTask; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; import org.apache.rocketmq.common.KeyBuilder; @@ -103,6 +104,7 @@ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.remoting.protocol.body.TopicList; import org.apache.rocketmq.remoting.protocol.body.UserInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.UpdateGroupForbiddenRequestHeader; import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader; @@ -2091,4 +2093,9 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { this.mqClientInstance.getMQClientAPIImpl().exportPopRecord(brokerAddr, timeout); } + + public List checkAsyncTaskStatus(String brokerAddr, CheckAsyncTaskStatusRequestHeader requestHeader) + throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + return this.mqClientInstance.getMQClientAPIImpl().checkAsyncTaskStatus(brokerAddr, requestHeader, timeoutMillis); + } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java index 69a08218646..d8e9fba9d64 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/admin/MQAdminExt.java @@ -24,6 +24,7 @@ import org.apache.rocketmq.client.MQAdmin; import org.apache.rocketmq.client.exception.MQBrokerException; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.AsyncTask; import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.PlainAccessConfig; @@ -61,6 +62,7 @@ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.remoting.protocol.body.TopicList; import org.apache.rocketmq.remoting.protocol.body.UserInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; import org.apache.rocketmq.remoting.protocol.header.controller.ElectMasterResponseHeader; import org.apache.rocketmq.remoting.protocol.header.controller.GetMetaDataResponseHeader; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; @@ -529,4 +531,6 @@ String setCommitLogReadAheadMode(final String brokerAddr, String mode) void exportPopRecords(String brokerAddr, long timeout) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException; + + List checkAsyncTaskStatus(String brokerAddr, CheckAsyncTaskStatusRequestHeader requestHeader)throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException; } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/stats/CheckAsyncTaskStatusSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/CheckAsyncTaskStatusSubCommand.java new file mode 100644 index 00000000000..c648e861edc --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/stats/CheckAsyncTaskStatusSubCommand.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.rocketmq.tools.command.stats; + +import java.util.Set; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.AsyncTask; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.remoting.protocol.body.ClusterInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckAsyncTaskStatusRequestHeader; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.SubCommand; + +import java.util.List; +import java.util.Map; + +public class CheckAsyncTaskStatusSubCommand implements SubCommand { + + @Override + public String commandName() { + return "checkAsyncTaskStatus"; + } + + @Override + public String commandDesc() { + return "Check the status of an asynchronous task by task name."; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("c", "cluster", true, "Cluster name"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("n", "nameserverAddr", true, "NameServer address"); + opt.setRequired(true); + options.addOption(opt); + + opt = new Option("t", "taskName", true, "The name of the asynchronous task"); + opt.setRequired(true); + options.addOption(opt); + + return options; + } + + @Override + public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) { + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + defaultMQAdminExt.setNamesrvAddr(StringUtils.trim(commandLine.getOptionValue('n'))); + + String clusterName = commandLine.hasOption('c') ? commandLine.getOptionValue('c').trim() : ""; + String taskName = commandLine.hasOption('t') ? commandLine.getOptionValue('t').trim() : ""; + if (StringUtils.isBlank(taskName)) { + System.out.println("Task name cannot be empty. Please specify a task name with -t."); + return; + } + + try { + defaultMQAdminExt.start(); + + ClusterInfo clusterInfo = defaultMQAdminExt.examineBrokerClusterInfo(); + Map> clusterAddrTable = clusterInfo.getClusterAddrTable(); + Map brokerAddrTable = clusterInfo.getBrokerAddrTable(); + + if (clusterAddrTable.get(clusterName) == null) { + System.out.println("Cluster '" + clusterName + "' not found in cluster address table."); + return; + } + + for (Map.Entry entry : brokerAddrTable.entrySet()) { + String brokerName = entry.getKey(); + String brokerAddr = entry.getValue().getBrokerAddrs().get(0L); + + try { + CheckAsyncTaskStatusRequestHeader requestHeader = new CheckAsyncTaskStatusRequestHeader(); + requestHeader.setTaskName(taskName); + + List asyncTaskStatus = defaultMQAdminExt.checkAsyncTaskStatus(brokerAddr, requestHeader); + + if (asyncTaskStatus != null && !asyncTaskStatus.isEmpty()) { + for (AsyncTask taskStatus : asyncTaskStatus) { + System.out.printf("Found a task for task name '%s' on broker %s: Task ID: %s, Status: %s, Result: %s, CreateTime: %s%n", + taskName, brokerName, taskStatus.getTaskId(), taskStatus.getStatus(), + taskStatus.getResult(), taskStatus.getCreateTime().toString()); + } + } else { + System.out.printf("No tasks found for task name '%s' on broker %s.%n", taskName, brokerName); + } + } catch (Exception e) { + System.out.printf("Failed to query task status for task name '%s' on broker %s: %s%n", + taskName, brokerName, e.getMessage()); + } + } + } catch (Exception e) { + throw new RuntimeException("Failed to execute " + this.getClass().getSimpleName() + " command", e); + } finally { + defaultMQAdminExt.shutdown(); + } + } +}