You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by zh...@apache.org on 2023/03/08 07:47:30 UTC

[rocketmq] branch develop updated: [ISSUE #6207] Use static subscription to calculate consumer lag and latency

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

zhouxzhan pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/rocketmq.git


The following commit(s) were added to refs/heads/develop by this push:
     new 1986c6bdf [ISSUE #6207] Use static subscription to calculate consumer lag and latency
1986c6bdf is described below

commit 1986c6bdf9e724cd6a8c5f4f7f68f97e68257fc1
Author: Zhouxiang Zhan <zh...@apache.org>
AuthorDate: Wed Mar 8 15:47:20 2023 +0800

    [ISSUE #6207] Use static subscription to calculate consumer lag and latency
    
    * Fix retryTopicPerm in ConsumerLagCalculator
    
    * [ISSUE #6206] Use static subscription to calculate consumer lag and latency
    
    * Add version for SimpleSubscriptionData
    
    * Fix hashcode and equals for SubscriptionGroupConfig
    
    * remove brokerId in equal and hashcode method
    
    * Add getter setter for version
    
    * fix consumerGroupInfo and getSubscriptionDataSet npe
---
 .../broker/metrics/ConsumerLagCalculator.java      | 92 ++++++++++++++--------
 .../org/apache/rocketmq/common/BrokerConfig.java   | 10 +++
 .../subscription/SimpleSubscriptionData.java       | 91 +++++++++++++++++++++
 .../subscription/SubscriptionGroupConfig.java      | 47 +++++++----
 4 files changed, 192 insertions(+), 48 deletions(-)

diff --git a/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java b/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java
index 4b8767de5..f2abdba74 100644
--- a/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java
+++ b/broker/src/main/java/org/apache/rocketmq/broker/metrics/ConsumerLagCalculator.java
@@ -19,6 +19,7 @@ package org.apache.rocketmq.broker.metrics;
 import java.util.Map;
 import java.util.Set;
 import java.util.function.Consumer;
+import java.util.stream.Collectors;
 import org.apache.rocketmq.broker.BrokerController;
 import org.apache.rocketmq.broker.client.ConsumerGroupInfo;
 import org.apache.rocketmq.broker.client.ConsumerManager;
@@ -42,6 +43,7 @@ import org.apache.rocketmq.logging.org.slf4j.Logger;
 import org.apache.rocketmq.logging.org.slf4j.LoggerFactory;
 import org.apache.rocketmq.remoting.protocol.heartbeat.ConsumeType;
 import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData;
+import org.apache.rocketmq.remoting.protocol.subscription.SimpleSubscriptionData;
 import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig;
 import org.apache.rocketmq.store.DefaultMessageFilter;
 import org.apache.rocketmq.store.MessageStore;
@@ -124,30 +126,37 @@ public class ConsumerLagCalculator {
         }
     }
 
-    public static class CalculateSendToDLQResult extends BaseCalculateResult {
-        public long dlqMessageCount;
-
-        public CalculateSendToDLQResult(String group, String topic) {
-            super(group, topic, false);
-        }
-    }
-
     private void processAllGroup(Consumer<ProcessGroupInfo> consumer) {
-        for (Map.Entry<String, SubscriptionGroupConfig> subscriptionEntry : subscriptionGroupManager
-            .getSubscriptionGroupTable().entrySet()) {
-            String group = subscriptionEntry.getKey();
-            SubscriptionGroupConfig subscriptionGroupConfig = subscriptionEntry.getValue();
+        for (Map.Entry<String, SubscriptionGroupConfig> subscriptionEntry :
+            subscriptionGroupManager.getSubscriptionGroupTable().entrySet()) {
 
+            String group = subscriptionEntry.getKey();
             ConsumerGroupInfo consumerGroupInfo = consumerManager.getConsumerGroupInfo(group, true);
-            if (consumerGroupInfo == null) {
-                continue;
+            boolean isPop = false;
+            if (consumerGroupInfo != null) {
+                isPop = consumerGroupInfo.getConsumeType() == ConsumeType.CONSUME_POP;
             }
-            boolean isPop = consumerGroupInfo.getConsumeType() == ConsumeType.CONSUME_POP;
-            Set<String> topics = consumerGroupInfo.getSubscribeTopics();
+            Set<String> topics;
+            if (brokerConfig.isUseStaticSubscription()) {
+                SubscriptionGroupConfig subscriptionGroupConfig = subscriptionEntry.getValue();
+                if (subscriptionGroupConfig.getSubscriptionDataSet() == null ||
+                    subscriptionGroupConfig.getSubscriptionDataSet().isEmpty()) {
+                    continue;
+                }
+                topics = subscriptionGroupConfig.getSubscriptionDataSet()
+                    .stream()
+                    .map(SimpleSubscriptionData::getTopic)
+                    .collect(Collectors.toSet());
+            } else {
+                if (consumerGroupInfo == null) {
+                    continue;
+                }
+                topics = consumerGroupInfo.getSubscribeTopics();
+            }
+
             if (null == topics || topics.isEmpty()) {
                 continue;
             }
-
             for (String topic : topics) {
                 // skip retry topic
                 if (topic.startsWith(MixAll.RETRY_GROUP_TOPIC_PREFIX)) {
@@ -168,7 +177,7 @@ public class ConsumerLagCalculator {
                 if (isPop) {
                     String retryTopic = KeyBuilder.buildPopRetryTopic(topic, group);
                     TopicConfig retryTopicConfig = topicConfigManager.selectTopicConfig(retryTopic);
-                    int retryTopicPerm = topicConfig.getPerm() & brokerConfig.getBrokerPermission();
+                    int retryTopicPerm = retryTopicConfig.getPerm() & brokerConfig.getBrokerPermission();
                     if (PermName.isReadable(retryTopicPerm) || PermName.isWriteable(retryTopicPerm)) {
                         consumer.accept(new ProcessGroupInfo(group, topic, true, retryTopic));
                     } else {
@@ -403,23 +412,40 @@ public class ConsumerLagCalculator {
 
         if (brokerConfig.isEstimateAccumulation() && to > from) {
             SubscriptionData subscriptionData = null;
-            ConsumerGroupInfo consumerGroupInfo = consumerManager.getConsumerGroupInfo(group, true);
-            if (consumerGroupInfo != null) {
-                subscriptionData = consumerGroupInfo.findSubscriptionData(topic);
+            if (brokerConfig.isUseStaticSubscription()) {
+                SubscriptionGroupConfig subscriptionGroupConfig = subscriptionGroupManager.findSubscriptionGroupConfig(group);
+                if (subscriptionGroupConfig != null) {
+                    for (SimpleSubscriptionData simpleSubscriptionData : subscriptionGroupConfig.getSubscriptionDataSet()) {
+                        if (topic.equals(simpleSubscriptionData.getTopic())) {
+                            subscriptionData = new SubscriptionData();
+                            subscriptionData.setTopic(simpleSubscriptionData.getTopic());
+                            subscriptionData.setExpressionType(simpleSubscriptionData.getExpressionType());
+                            subscriptionData.setSubString(simpleSubscriptionData.getExpression());
+                            break;
+                        }
+                    }
+                }
+            } else {
+                ConsumerGroupInfo consumerGroupInfo = consumerManager.getConsumerGroupInfo(group, true);
+                if (consumerGroupInfo != null) {
+                    subscriptionData = consumerGroupInfo.findSubscriptionData(topic);
+                }
             }
-            if (null != subscriptionData &&
-                ExpressionType.TAG.equalsIgnoreCase(subscriptionData.getExpressionType()) &&
-                !SubscriptionData.SUB_ALL.equals(subscriptionData.getSubString())) {
-                count = messageStore.estimateMessageCount(topic, queueId, from, to,
-                    new DefaultMessageFilter(subscriptionData));
-            } else if (null != subscriptionData &&
-                ExpressionType.SQL92.equalsIgnoreCase(subscriptionData.getExpressionType())) {
-                ConsumerFilterData consumerFilterData = consumerFilterManager.get(topic, group);
-                count = messageStore.estimateMessageCount(topic, queueId, from, to,
-                    new ExpressionMessageFilter(subscriptionData,
-                        consumerFilterData,
-                        consumerFilterManager));
+
+            if (null != subscriptionData) {
+                if (ExpressionType.TAG.equalsIgnoreCase(subscriptionData.getExpressionType())
+                    && !SubscriptionData.SUB_ALL.equals(subscriptionData.getSubString())) {
+                    count = messageStore.estimateMessageCount(topic, queueId, from, to,
+                        new DefaultMessageFilter(subscriptionData));
+                } else if (ExpressionType.SQL92.equalsIgnoreCase(subscriptionData.getExpressionType())) {
+                    ConsumerFilterData consumerFilterData = consumerFilterManager.get(topic, group);
+                    count = messageStore.estimateMessageCount(topic, queueId, from, to,
+                        new ExpressionMessageFilter(subscriptionData,
+                            consumerFilterData,
+                            consumerFilterManager));
+                }
             }
+
         }
         return count < 0 ? 0 : count;
     }
diff --git a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
index 50874da08..63b32fc09 100644
--- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
+++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java
@@ -345,6 +345,8 @@ public class BrokerConfig extends BrokerIdentity {
      */
     private int brokerElectionPriority = Integer.MAX_VALUE;
 
+    private boolean useStaticSubscription = false;
+
     public enum MetricsExporterType {
         DISABLE(0),
         OTLP_GRPC(1),
@@ -1647,4 +1649,12 @@ public class BrokerConfig extends BrokerIdentity {
     public void setEstimateAccumulation(boolean estimateAccumulation) {
         this.estimateAccumulation = estimateAccumulation;
     }
+
+    public boolean isUseStaticSubscription() {
+        return useStaticSubscription;
+    }
+
+    public void setUseStaticSubscription(boolean useStaticSubscription) {
+        this.useStaticSubscription = useStaticSubscription;
+    }
 }
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SimpleSubscriptionData.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SimpleSubscriptionData.java
new file mode 100644
index 000000000..ec2b51e0b
--- /dev/null
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SimpleSubscriptionData.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.rocketmq.remoting.protocol.subscription;
+
+import com.google.common.base.MoreObjects;
+import java.util.Objects;
+
+public class SimpleSubscriptionData {
+    private String topic;
+    private String expressionType;
+    private String expression;
+    private long version;
+
+    public SimpleSubscriptionData(String topic, String expressionType, String expression, long version) {
+        this.topic = topic;
+        this.expressionType = expressionType;
+        this.expression = expression;
+        this.version = version;
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public void setTopic(String topic) {
+        this.topic = topic;
+    }
+
+    public String getExpressionType() {
+        return expressionType;
+    }
+
+    public void setExpressionType(String expressionType) {
+        this.expressionType = expressionType;
+    }
+
+    public String getExpression() {
+        return expression;
+    }
+
+    public void setExpression(String expression) {
+        this.expression = expression;
+    }
+
+    public long getVersion() {
+        return version;
+    }
+
+    public void setVersion(long version) {
+        this.version = version;
+    }
+
+    @Override public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        SimpleSubscriptionData that = (SimpleSubscriptionData) o;
+        return version == that.version && Objects.equals(topic, that.topic);
+    }
+
+    @Override public int hashCode() {
+        return Objects.hash(topic, version);
+    }
+
+    @Override public String toString() {
+        return MoreObjects.toStringHelper(this)
+            .add("topic", topic)
+            .add("expressionType", expressionType)
+            .add("expression", expression)
+            .add("version", version)
+            .toString();
+    }
+}
diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java
index fc798c77a..799c7492e 100644
--- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java
+++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/SubscriptionGroupConfig.java
@@ -17,6 +17,8 @@
 
 package org.apache.rocketmq.remoting.protocol.subscription;
 
+import com.google.common.base.MoreObjects;
+import java.util.Set;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.rocketmq.common.MixAll;
 
@@ -45,6 +47,8 @@ public class SubscriptionGroupConfig {
     // Only valid for push consumer
     private int consumeTimeoutMinute = 15;
 
+    private Set<SimpleSubscriptionData> subscriptionDataSet;
+
     public String getGroupName() {
         return groupName;
     }
@@ -149,6 +153,14 @@ public class SubscriptionGroupConfig {
         this.consumeTimeoutMinute = consumeTimeoutMinute;
     }
 
+    public Set<SimpleSubscriptionData> getSubscriptionDataSet() {
+        return subscriptionDataSet;
+    }
+
+    public void setSubscriptionDataSet(Set<SimpleSubscriptionData> subscriptionDataSet) {
+        this.subscriptionDataSet = subscriptionDataSet;
+    }
+
     @Override
     public int hashCode() {
         final int prime = 31;
@@ -163,6 +175,9 @@ public class SubscriptionGroupConfig {
         result = prime * result + retryQueueNums;
         result =
             prime * result + (int) (whichBrokerWhenConsumeSlowly ^ (whichBrokerWhenConsumeSlowly >>> 32));
+        result = prime * result + groupSysFlag;
+        result = prime * result + consumeTimeoutMinute;
+        result = prime * result + subscriptionDataSet.hashCode();
         return result;
     }
 
@@ -182,28 +197,30 @@ public class SubscriptionGroupConfig {
             .append(consumeBroadcastEnable, other.consumeBroadcastEnable)
             .append(retryQueueNums, other.retryQueueNums)
             .append(retryMaxTimes, other.retryMaxTimes)
-            .append(brokerId, other.brokerId)
             .append(whichBrokerWhenConsumeSlowly, other.whichBrokerWhenConsumeSlowly)
             .append(notifyConsumerIdsChangedEnable, other.notifyConsumerIdsChangedEnable)
             .append(groupSysFlag, other.groupSysFlag)
+            .append(consumeTimeoutMinute, other.consumeTimeoutMinute)
+            .append(subscriptionDataSet, other.subscriptionDataSet)
             .isEquals();
     }
 
     @Override
     public String toString() {
-        return "SubscriptionGroupConfig{" +
-            "groupName='" + groupName + '\'' +
-            ", consumeEnable=" + consumeEnable +
-            ", consumeFromMinEnable=" + consumeFromMinEnable +
-            ", consumeBroadcastEnable=" + consumeBroadcastEnable +
-            ", consumeMessageOrderly=" + consumeMessageOrderly +
-            ", retryQueueNums=" + retryQueueNums +
-            ", retryMaxTimes=" + retryMaxTimes +
-            ", groupRetryPolicy=" + groupRetryPolicy +
-            ", brokerId=" + brokerId +
-            ", whichBrokerWhenConsumeSlowly=" + whichBrokerWhenConsumeSlowly +
-            ", notifyConsumerIdsChangedEnable=" + notifyConsumerIdsChangedEnable +
-            ", groupSysFlag=" + groupSysFlag +
-            '}';
+        return MoreObjects.toStringHelper(this)
+            .add("groupName", groupName)
+            .add("consumeEnable", consumeEnable)
+            .add("consumeFromMinEnable", consumeFromMinEnable)
+            .add("consumeBroadcastEnable", consumeBroadcastEnable)
+            .add("consumeMessageOrderly", consumeMessageOrderly)
+            .add("retryQueueNums", retryQueueNums)
+            .add("retryMaxTimes", retryMaxTimes)
+            .add("groupRetryPolicy", groupRetryPolicy)
+            .add("whichBrokerWhenConsumeSlowly", whichBrokerWhenConsumeSlowly)
+            .add("notifyConsumerIdsChangedEnable", notifyConsumerIdsChangedEnable)
+            .add("groupSysFlag", groupSysFlag)
+            .add("consumeTimeoutMinute", consumeTimeoutMinute)
+            .add("subscriptionTopicSet", subscriptionDataSet)
+            .toString();
     }
 }