You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by sp...@apache.org on 2023/02/24 13:57:52 UTC
[iotdb] branch master updated: [IOTDB-5542] ConfigNode counter dashboard (#9129)
This is an automated email from the ASF dual-hosted git repository.
spricoder pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new 468c0fbd19 [IOTDB-5542] ConfigNode counter dashboard (#9129)
468c0fbd19 is described below
commit 468c0fbd199f6242467ebb4032239ac0111c8081
Author: YongzaoDan <33...@users.noreply.github.com>
AuthorDate: Fri Feb 24 21:57:46 2023 +0800
[IOTDB-5542] ConfigNode counter dashboard (#9129)
* Basic Prometheus
* add node_num
* basic dashboard
* internal
* dashboard
* rename
* Update DeleteDatabaseProcedure.java
* Update PartitionInfo.java
* Fix code smells
* Code smells
---------
Co-authored-by: Liu Xuxin <li...@outlook.com>
Co-authored-by: spricoder <sp...@qq.com>
---
.../iotdb/confignode/conf/ConfigNodeConstant.java | 11 +-
.../confignode/manager/ClusterSchemaManager.java | 13 +-
.../iotdb/confignode/manager/ConfigManager.java | 7 +-
.../apache/iotdb/confignode/manager/IManager.java | 1 +
.../confignode/manager/PermissionManager.java | 6 +-
.../iotdb/confignode/manager/ProcedureManager.java | 8 +-
.../iotdb/confignode/manager/SyncManager.java | 6 +-
.../iotdb/confignode/manager/TriggerManager.java | 2 +
.../manager/{ => consensus}/ConsensusManager.java | 42 +-
.../iotdb/confignode/manager/load/LoadManager.java | 35 +-
.../manager/load/LoadManagerMetrics.java | 326 --
.../iotdb/confignode/manager/node/NodeManager.java | 12 +-
.../iotdb/confignode/manager/node/NodeMetrics.java | 104 +
.../manager/partition/PartitionManager.java | 127 +-
.../manager/partition/PartitionMetrics.java | 353 ++
.../persistence/metric/PartitionInfoMetrics.java | 197 -
.../partition/DatabasePartitionTable.java | 36 +-
.../persistence/partition/PartitionInfo.java | 200 +-
.../persistence/schema/ClusterSchemaInfo.java | 100 +-
.../procedure/env/ConfigNodeProcedureEnv.java | 2 +-
.../procedure/env/DataNodeRemoveHandler.java | 13 +-
...Procedure.java => DeleteDatabaseProcedure.java} | 24 +-
.../procedure/store/ConfigProcedureStore.java | 2 +-
.../procedure/store/ProcedureFactory.java | 6 +-
.../iotdb/confignode/service/ConfigNode.java | 24 +-
.../thrift/ConfigNodeRPCServiceHandlerMetrics.java | 2 +-
.../thrift/ConfigNodeRPCServiceMetrics.java | 3 +-
.../thrift/ConfigNodeRPCServiceProcessor.java | 2 +-
.../request/ConfigPhysicalPlanSerDeTest.java | 16 +-
...eTest.java => DeleteDatabaseProcedureTest.java} | 11 +-
.../Apache-IoTDB-ConfigNode-Dashboard.json | 5186 +++++++++++++-------
.../apache/iotdb/commons/cluster/NodeStatus.java | 1 +
.../org/apache/iotdb/commons/cluster/NodeType.java | 3 +-
.../iotdb/commons/cluster/RegionRoleType.java | 1 +
.../iotdb/commons/service/metric/enums/Metric.java | 16 +-
35 files changed, 4295 insertions(+), 2603 deletions(-)
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConstant.java b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConstant.java
index 5ee02a2095..214db2c595 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConstant.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConstant.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.confignode.conf;
import java.io.File;
@@ -40,16 +41,6 @@ public class ConfigNodeConstant {
public static final int MIN_SUPPORTED_JDK_VERSION = 8;
- /** These variables are only used for cluster gauge metrics */
- public static final String METRIC_TAG_TOTAL = "total";
-
- public static final String METRIC_STATUS_REGISTER = "Registered";
- public static final String METRIC_STATUS_ONLINE = "Online";
- public static final String METRIC_STATUS_UNKNOWN = "Unknown";
-
- public static final String METRIC_CONFIG_NODE = "ConfigNode";
- public static final String METRIC_DATA_NODE = "DataNode";
-
public static final String REMOVE_CONFIGNODE_USAGE =
"Executed failed, check usage: <Node-id>/<internal_address>:<internal_port>";
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
index b2b45472a1..717a1ca88b 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ClusterSchemaManager.java
@@ -60,9 +60,11 @@ import org.apache.iotdb.confignode.consensus.response.template.AllTemplateSetInf
import org.apache.iotdb.confignode.consensus.response.template.TemplateInfoResp;
import org.apache.iotdb.confignode.consensus.response.template.TemplateSetInfoResp;
import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
import org.apache.iotdb.confignode.manager.observer.NodeStatisticsEvent;
import org.apache.iotdb.confignode.manager.partition.PartitionManager;
+import org.apache.iotdb.confignode.manager.partition.PartitionMetrics;
import org.apache.iotdb.confignode.persistence.schema.ClusterSchemaInfo;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseInfo;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
@@ -126,6 +128,7 @@ public class ClusterSchemaManager {
return RpcUtils.getStatus(
illegalPathException.getErrorCode(), illegalPathException.getMessage());
}
+
try {
clusterSchemaInfo.checkContainsStorageGroup(databaseSchemaPlan.getSchema().getName());
} catch (MetadataException metadataException) {
@@ -139,10 +142,14 @@ public class ClusterSchemaManager {
return result;
}
- // Cache StorageGroupSchema
+ // Cache DatabaseSchema
result = getConsensusManager().write(databaseSchemaPlan).getStatus();
- // Adjust the maximum RegionGroup number of each StorageGroup
+ // Bind Database metrics
+ PartitionMetrics.bindDatabasePartitionMetrics(
+ configManager, databaseSchemaPlan.getSchema().getName());
+
+ // Adjust the maximum RegionGroup number of each Database
adjustMaxRegionGroupNum();
return result;
@@ -322,7 +329,7 @@ public class ClusterSchemaManager {
// Get related DataNodes
Set<TDataNodeLocation> dataNodeLocations =
getPartitionManager()
- .getStorageGroupRelatedDataNodes(storageGroup, TConsensusGroupType.DataRegion);
+ .getDatabaseRelatedDataNodes(storageGroup, TConsensusGroupType.DataRegion);
for (TDataNodeLocation dataNodeLocation : dataNodeLocations) {
dataNodeLocationMap.putIfAbsent(dataNodeLocation.getDataNodeId(), dataNodeLocation);
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 44a3d26d38..84cb1b7550 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -36,6 +36,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.path.PathPatternTree;
+import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.commons.sync.pipe.PipeMessage;
import org.apache.iotdb.commons.utils.AuthUtils;
import org.apache.iotdb.commons.utils.PathUtils;
@@ -80,12 +81,15 @@ import org.apache.iotdb.confignode.consensus.response.partition.SchemaNodeManage
import org.apache.iotdb.confignode.consensus.response.partition.SchemaPartitionResp;
import org.apache.iotdb.confignode.consensus.response.template.TemplateSetInfoResp;
import org.apache.iotdb.confignode.consensus.statemachine.ConfigNodeRegionStateMachine;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.cq.CQManager;
import org.apache.iotdb.confignode.manager.load.LoadManager;
import org.apache.iotdb.confignode.manager.node.ClusterNodeStartUtils;
import org.apache.iotdb.confignode.manager.node.NodeManager;
+import org.apache.iotdb.confignode.manager.node.NodeMetrics;
import org.apache.iotdb.confignode.manager.node.heartbeat.NodeHeartbeatSample;
import org.apache.iotdb.confignode.manager.partition.PartitionManager;
+import org.apache.iotdb.confignode.manager.partition.PartitionMetrics;
import org.apache.iotdb.confignode.persistence.AuthorInfo;
import org.apache.iotdb.confignode.persistence.ProcedureInfo;
import org.apache.iotdb.confignode.persistence.TriggerInfo;
@@ -1380,7 +1384,8 @@ public class ConfigManager implements IManager {
@Override
public void addMetrics() {
- partitionManager.addMetrics();
+ MetricService.getInstance().addMetricSet(new NodeMetrics(getNodeManager()));
+ MetricService.getInstance().addMetricSet(new PartitionMetrics(this));
}
@Override
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
index b6c8fc4cc7..60c96b5a27 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java
@@ -43,6 +43,7 @@ import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetTTLPl
import org.apache.iotdb.confignode.consensus.request.write.storagegroup.SetTimePartitionIntervalPlan;
import org.apache.iotdb.confignode.consensus.request.write.sync.CreatePipeSinkPlan;
import org.apache.iotdb.confignode.consensus.request.write.sync.DropPipeSinkPlan;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.cq.CQManager;
import org.apache.iotdb.confignode.manager.load.LoadManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
index 3aee9c071d..098a67a88d 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java
@@ -26,22 +26,18 @@ import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType;
import org.apache.iotdb.confignode.consensus.request.auth.AuthorPlan;
import org.apache.iotdb.confignode.consensus.response.auth.PermissionInfoResp;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.persistence.AuthorInfo;
import org.apache.iotdb.confignode.rpc.thrift.TPermissionInfoResp;
import org.apache.iotdb.mpp.rpc.thrift.TInvalidatePermissionCacheReq;
import org.apache.iotdb.rpc.RpcUtils;
import org.apache.iotdb.rpc.TSStatusCode;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.util.List;
/** manager permission query and operation */
public class PermissionManager {
- private static final Logger logger = LoggerFactory.getLogger(PermissionManager.class);
-
private final ConfigManager configManager;
private final AuthorInfo authorInfo;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
index 71edebcfe9..5afe8b1343 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java
@@ -49,7 +49,7 @@ import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure;
-import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.UnsetTemplateProcedure;
import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
@@ -150,9 +150,9 @@ public class ProcedureManager {
public TSStatus deleteStorageGroups(ArrayList<TDatabaseSchema> deleteSgSchemaList) {
List<Long> procedureIds = new ArrayList<>();
for (TDatabaseSchema storageGroupSchema : deleteSgSchemaList) {
- DeleteStorageGroupProcedure deleteStorageGroupProcedure =
- new DeleteStorageGroupProcedure(storageGroupSchema);
- long procedureId = this.executor.submitProcedure(deleteStorageGroupProcedure);
+ DeleteDatabaseProcedure deleteDatabaseProcedure =
+ new DeleteDatabaseProcedure(storageGroupSchema);
+ long procedureId = this.executor.submitProcedure(deleteDatabaseProcedure);
procedureIds.add(procedureId);
}
List<TSStatus> procedureStatus = new ArrayList<>();
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/SyncManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/SyncManager.java
index 9e5645af7b..dc2c7bdd95 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/SyncManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/SyncManager.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.confignode.manager;
import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
@@ -43,6 +44,7 @@ import org.apache.iotdb.confignode.consensus.request.write.sync.SetPipeStatusPla
import org.apache.iotdb.confignode.consensus.request.write.sync.ShowPipePlan;
import org.apache.iotdb.confignode.consensus.response.pipe.PipeResp;
import org.apache.iotdb.confignode.consensus.response.pipe.PipeSinkResp;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
import org.apache.iotdb.confignode.manager.observer.NodeStatisticsEvent;
import org.apache.iotdb.confignode.persistence.sync.ClusterSyncInfo;
@@ -301,8 +303,4 @@ public class SyncManager {
private ConsensusManager getConsensusManager() {
return configManager.getConsensusManager();
}
-
- private ProcedureManager getProcedureManager() {
- return configManager.getProcedureManager();
- }
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java
index ae094acc3e..08c6f392a9 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java
@@ -37,6 +37,7 @@ import org.apache.iotdb.confignode.consensus.response.trigger.TransferringTrigge
import org.apache.iotdb.confignode.consensus.response.trigger.TriggerLocationResp;
import org.apache.iotdb.confignode.consensus.response.trigger.TriggerTableResp;
import org.apache.iotdb.confignode.consensus.response.udf.JarResp;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
import org.apache.iotdb.confignode.persistence.TriggerInfo;
import org.apache.iotdb.confignode.rpc.thrift.TCreateTriggerReq;
@@ -64,6 +65,7 @@ import java.util.Map;
import java.util.Optional;
public class TriggerManager {
+
private static final Logger LOGGER = LoggerFactory.getLogger(TriggerManager.class);
private final ConfigManager configManager;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
similarity index 93%
rename from confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
index aa914cf7c0..fddf8326b0 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConsensusManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/consensus/ConsensusManager.java
@@ -16,7 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.confignode.manager;
+
+package org.apache.iotdb.confignode.manager.consensus;
import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TEndPoint;
@@ -24,13 +25,13 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.consensus.ConfigNodeRegionId;
import org.apache.iotdb.commons.consensus.ConsensusGroupId;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
-import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.confignode.conf.ConfigNodeConfig;
import org.apache.iotdb.confignode.conf.ConfigNodeDescriptor;
import org.apache.iotdb.confignode.conf.SystemPropertiesUtils;
import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan;
import org.apache.iotdb.confignode.consensus.statemachine.ConfigNodeRegionStateMachine;
import org.apache.iotdb.confignode.exception.AddPeerException;
+import org.apache.iotdb.confignode.manager.IManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.consensus.IConsensus;
@@ -54,7 +55,7 @@ import java.util.concurrent.TimeUnit;
import static org.apache.iotdb.consensus.ConsensusFactory.SIMPLE_CONSENSUS;
-/** ConsensusManager maintains consensus class, request will redirect to consensus layer */
+/** ConsensusManager maintains consensus class, request will redirect to consensus layer. */
public class ConsensusManager {
private static final Logger LOGGER = LoggerFactory.getLogger(ConsensusManager.class);
@@ -76,7 +77,7 @@ public class ConsensusManager {
consensusImpl.stop();
}
- /** ConsensusLayer local implementation */
+ /** ConsensusLayer local implementation. */
private void setConsensusLayer(ConfigNodeRegionStateMachine stateMachine) throws IOException {
// There is only one ConfigNodeGroup
consensusGroupId = new ConfigNodeRegionId(CONF.getConfigNodeRegionId());
@@ -213,7 +214,7 @@ public class ConsensusManager {
}
/**
- * Create peer in new node to build consensus group
+ * Create peer in new node to build consensus group.
*
* @param configNodeLocations All registered ConfigNodes
*/
@@ -232,7 +233,7 @@ public class ConsensusManager {
}
/**
- * Add a new ConfigNode Peer into ConfigNodeRegion
+ * Add a new ConfigNode Peer into ConfigNodeRegion.
*
* @param configNodeLocation The new ConfigNode
* @throws AddPeerException When addPeer doesn't success
@@ -254,20 +255,20 @@ public class ConsensusManager {
}
/**
- * Remove a ConfigNode Peer out of ConfigNodeRegion
+ * Remove a ConfigNode Peer out of ConfigNodeRegion.
*
- * @param tConfigNodeLocation config node location
+ * @param configNodeLocation config node location
* @return True if successfully removePeer. False if another ConfigNode is being removed to the
* ConfigNodeRegion
*/
- public boolean removeConfigNodePeer(TConfigNodeLocation tConfigNodeLocation) {
+ public boolean removeConfigNodePeer(TConfigNodeLocation configNodeLocation) {
return consensusImpl
.removePeer(
consensusGroupId,
new Peer(
consensusGroupId,
- tConfigNodeLocation.getConfigNodeId(),
- tConfigNodeLocation.getConsensusEndPoint()))
+ configNodeLocation.getConfigNodeId(),
+ configNodeLocation.getConsensusEndPoint()))
.isSuccess();
}
@@ -306,13 +307,14 @@ public class ConsensusManager {
TimeUnit.MILLISECONDS.sleep(100);
} catch (InterruptedException e) {
LOGGER.warn("ConsensusManager getLeader been interrupted, ", e);
+ Thread.currentThread().interrupt();
}
}
return null;
}
/**
- * Confirm the current ConfigNode's leadership
+ * Confirm the current ConfigNode's leadership.
*
* @return SUCCESS_STATUS if the current ConfigNode is leader, NEED_REDIRECTION otherwise
*/
@@ -346,20 +348,4 @@ public class ConsensusManager {
private NodeManager getNodeManager() {
return configManager.getNodeManager();
}
-
- @TestOnly
- public void singleCopyMayWaitUntilLeaderReady() {
- long startTime = System.currentTimeMillis();
- long maxWaitTime = 1000L * 60; // milliseconds, which is 60s
- try {
- while (!consensusImpl.isLeader(consensusGroupId)) {
- TimeUnit.MILLISECONDS.sleep(100);
- long elapsed = System.currentTimeMillis() - startTime;
- if (elapsed > maxWaitTime) {
- return;
- }
- }
- } catch (InterruptedException ignored) {
- }
- }
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
index 33d9ed4398..a19c942d25 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManager.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
import org.apache.iotdb.commons.partition.DataPartitionTable;
import org.apache.iotdb.commons.partition.SchemaPartitionTable;
-import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.confignode.client.DataNodeRequestType;
import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool;
import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
@@ -39,8 +38,6 @@ import org.apache.iotdb.confignode.consensus.request.write.region.CreateRegionGr
import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
import org.apache.iotdb.confignode.exception.NoAvailableRegionGroupException;
import org.apache.iotdb.confignode.exception.NotEnoughDataNodeException;
-import org.apache.iotdb.confignode.manager.ClusterSchemaManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
import org.apache.iotdb.confignode.manager.IManager;
import org.apache.iotdb.confignode.manager.load.balancer.PartitionBalancer;
import org.apache.iotdb.confignode.manager.load.balancer.RegionBalancer;
@@ -68,6 +65,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/**
@@ -108,8 +106,6 @@ public class LoadManager {
eventBus.register(configManager.getClusterSchemaManager());
eventBus.register(configManager.getSyncManager());
-
- MetricService.getInstance().addMetricSet(new LoadManagerMetrics(configManager));
}
/**
@@ -151,10 +147,31 @@ public class LoadManager {
return partitionBalancer.allocateDataPartition(unassignedDataPartitionSlotsMap);
}
+ /** @return Map<RegionGroupId, DataNodeId where the leader resides> */
public Map<TConsensusGroupId, Integer> getLatestRegionLeaderMap() {
return routeBalancer.getLatestRegionLeaderMap();
}
+ /**
+ * Get the number of RegionGroup-leaders in the specified DataNode.
+ *
+ * @param dataNodeId The specified DataNode
+ * @param type SchemaRegion or DataRegion
+ * @return The number of RegionGroup-leaders
+ */
+ public int getRegionGroupLeaderCount(int dataNodeId, TConsensusGroupType type) {
+ AtomicInteger result = new AtomicInteger(0);
+ routeBalancer
+ .getLatestRegionLeaderMap()
+ .forEach(
+ ((consensusGroupId, leaderId) -> {
+ if (dataNodeId == leaderId && type.equals(consensusGroupId.getType())) {
+ result.getAndIncrement();
+ }
+ }));
+ return result.get();
+ }
+
/**
* Generate an optimal real-time read/write requests routing policy.
*
@@ -333,18 +350,10 @@ public class LoadManager {
return routeBalancer;
}
- private ConsensusManager getConsensusManager() {
- return configManager.getConsensusManager();
- }
-
private NodeManager getNodeManager() {
return configManager.getNodeManager();
}
- private ClusterSchemaManager getClusterSchemaManager() {
- return configManager.getClusterSchemaManager();
- }
-
private PartitionManager getPartitionManager() {
return configManager.getPartitionManager();
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManagerMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManagerMetrics.java
deleted file mode 100644
index 6198acc5ae..0000000000
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/LoadManagerMetrics.java
+++ /dev/null
@@ -1,326 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.confignode.manager.load;
-
-import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
-import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
-import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
-import org.apache.iotdb.commons.cluster.NodeStatus;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
-import org.apache.iotdb.commons.utils.NodeUrlUtils;
-import org.apache.iotdb.confignode.manager.IManager;
-import org.apache.iotdb.confignode.manager.node.NodeManager;
-import org.apache.iotdb.confignode.manager.partition.PartitionManager;
-import org.apache.iotdb.metrics.AbstractMetricService;
-import org.apache.iotdb.metrics.metricsets.IMetricSet;
-import org.apache.iotdb.metrics.utils.MetricLevel;
-import org.apache.iotdb.metrics.utils.MetricType;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentHashMap;
-
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_CONFIG_NODE;
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_DATA_NODE;
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_STATUS_ONLINE;
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_STATUS_REGISTER;
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_STATUS_UNKNOWN;
-import static org.apache.iotdb.confignode.conf.ConfigNodeConstant.METRIC_TAG_TOTAL;
-
-/** This class collates metrics about loadManager */
-public class LoadManagerMetrics implements IMetricSet {
-
- private final IManager configManager;
-
- public LoadManagerMetrics(IManager configManager) {
- this.configManager = configManager;
- }
-
- @Override
- public void bindTo(AbstractMetricService metricService) {
- metricService.createAutoGauge(
- Metric.CONFIG_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getRegisterConfigNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_REGISTER);
-
- metricService.createAutoGauge(
- Metric.DATA_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getRegisterDataNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_REGISTER);
-
- metricService.createAutoGauge(
- Metric.CONFIG_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getRunningConfigNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_ONLINE);
-
- metricService.createAutoGauge(
- Metric.DATA_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getRunningDataNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_ONLINE);
-
- metricService.createAutoGauge(
- Metric.CONFIG_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getUnknownConfigNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_UNKNOWN);
-
- metricService.createAutoGauge(
- Metric.DATA_NODE.toString(),
- MetricLevel.CORE,
- this,
- o -> getUnknownDataNodesNum(metricService),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_UNKNOWN);
- }
-
- @Override
- public void unbindFrom(AbstractMetricService metricService) {
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.CONFIG_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_REGISTER);
-
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.DATA_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_REGISTER);
-
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.CONFIG_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_ONLINE);
-
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.DATA_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_ONLINE);
-
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.CONFIG_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_UNKNOWN);
-
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.DATA_NODE.toString(),
- Tag.NAME.toString(),
- METRIC_TAG_TOTAL,
- Tag.STATUS.toString(),
- METRIC_STATUS_UNKNOWN);
-
- getNodeManager()
- .getRegisteredDataNodes()
- .forEach(
- dataNodeInfo -> {
- TDataNodeLocation dataNodeLocation = dataNodeInfo.getLocation();
- String name =
- NodeUrlUtils.convertTEndPointUrl(dataNodeLocation.getClientRpcEndPoint());
-
- metricService.remove(
- MetricType.GAUGE,
- Metric.CLUSTER_NODE_LEADER_COUNT.toString(),
- Tag.NAME.toString(),
- name);
- });
- }
-
- private NodeManager getNodeManager() {
- return configManager.getNodeManager();
- }
-
- private PartitionManager getPartitionManager() {
- return configManager.getPartitionManager();
- }
-
- private LoadManager getLoadManager() {
- return configManager.getLoadManager();
- }
-
- private int getRegisterConfigNodesNum(AbstractMetricService metricService) {
- return getNodeManager().getRegisteredConfigNodes().size();
- }
-
- private int getRegisterDataNodesNum(AbstractMetricService metricService) {
- List<TDataNodeConfiguration> dataNodeConfigurations = getNodeManager().getRegisteredDataNodes();
- Map<Integer, Integer> idToCountMap = new ConcurrentHashMap<>();
-
- getLoadManager()
- .getLatestRegionLeaderMap()
- .forEach((consensusGroupId, nodeId) -> idToCountMap.merge(nodeId, 1, Integer::sum));
- for (TDataNodeConfiguration dataNodeInfo : dataNodeConfigurations) {
- TDataNodeLocation dataNodeLocation = dataNodeInfo.getLocation();
- int dataNodeId = dataNodeLocation.getDataNodeId();
- String name = NodeUrlUtils.convertTEndPointUrl(dataNodeLocation.getClientRpcEndPoint());
- metricService
- .getOrCreateGauge(
- Metric.CLUSTER_NODE_LEADER_COUNT.toString(),
- MetricLevel.CORE,
- Tag.NAME.toString(),
- name)
- .set(idToCountMap.getOrDefault(dataNodeId, 0));
- }
- return dataNodeConfigurations.size();
- }
-
- private int getRunningConfigNodesNum(AbstractMetricService metricService) {
- List<TConfigNodeLocation> runningConfigNodes =
- getNodeManager().filterConfigNodeThroughStatus(NodeStatus.Running);
- if (runningConfigNodes == null) {
- return 0;
- }
- for (TConfigNodeLocation configNodeLocation : runningConfigNodes) {
- String name = NodeUrlUtils.convertTEndPointUrl(configNodeLocation.getInternalEndPoint());
-
- metricService
- .getOrCreateGauge(
- Metric.CLUSTER_NODE_STATUS.toString(),
- MetricLevel.CORE,
- Tag.NAME.toString(),
- name,
- Tag.TYPE.toString(),
- METRIC_CONFIG_NODE)
- .set(1);
- }
- return runningConfigNodes.size();
- }
-
- private int getRunningDataNodesNum(AbstractMetricService metricService) {
- List<TDataNodeConfiguration> runningDataNodes =
- getNodeManager().filterDataNodeThroughStatus(NodeStatus.Running);
- if (runningDataNodes == null) {
- return 0;
- }
- for (TDataNodeConfiguration dataNodeInfo : runningDataNodes) {
- TDataNodeLocation dataNodeLocation = dataNodeInfo.getLocation();
- String name = NodeUrlUtils.convertTEndPointUrl(dataNodeLocation.getClientRpcEndPoint());
-
- metricService
- .getOrCreateGauge(
- Metric.CLUSTER_NODE_STATUS.toString(),
- MetricLevel.CORE,
- Tag.NAME.toString(),
- name,
- Tag.TYPE.toString(),
- METRIC_DATA_NODE)
- .set(1);
- }
- return runningDataNodes.size();
- }
-
- private int getUnknownConfigNodesNum(AbstractMetricService metricService) {
- List<TConfigNodeLocation> unknownConfigNodes =
- getNodeManager().filterConfigNodeThroughStatus(NodeStatus.Unknown);
- if (unknownConfigNodes == null) {
- return 0;
- }
- for (TConfigNodeLocation configNodeLocation : unknownConfigNodes) {
- String name = NodeUrlUtils.convertTEndPointUrl(configNodeLocation.getInternalEndPoint());
-
- metricService
- .getOrCreateGauge(
- Metric.CLUSTER_NODE_STATUS.toString(),
- MetricLevel.CORE,
- Tag.NAME.toString(),
- name,
- Tag.TYPE.toString(),
- METRIC_CONFIG_NODE)
- .set(0);
- }
- return unknownConfigNodes.size();
- }
-
- private int getUnknownDataNodesNum(AbstractMetricService metricService) {
- List<TDataNodeConfiguration> unknownDataNodes =
- getNodeManager().filterDataNodeThroughStatus(NodeStatus.Unknown);
- if (unknownDataNodes == null) {
- return 0;
- }
- for (TDataNodeConfiguration dataNodeInfo : unknownDataNodes) {
- TDataNodeLocation dataNodeLocation = dataNodeInfo.getLocation();
- String name = NodeUrlUtils.convertTEndPointUrl(dataNodeLocation.getClientRpcEndPoint());
-
- metricService
- .getOrCreateGauge(
- Metric.CLUSTER_NODE_STATUS.toString(),
- MetricLevel.CORE,
- Tag.NAME.toString(),
- name,
- Tag.TYPE.toString(),
- METRIC_DATA_NODE)
- .set(0);
- }
- return unknownDataNodes.size();
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- LoadManagerMetrics that = (LoadManagerMetrics) o;
- return Objects.equals(configManager, that.configManager);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(configManager);
- }
-}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
index 2c929d4385..9812a2455a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
@@ -53,15 +53,16 @@ import org.apache.iotdb.confignode.consensus.response.datanode.DataNodeRegisterR
import org.apache.iotdb.confignode.consensus.response.datanode.DataNodeToStatusResp;
import org.apache.iotdb.confignode.manager.ClusterSchemaManager;
import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
import org.apache.iotdb.confignode.manager.IManager;
import org.apache.iotdb.confignode.manager.TriggerManager;
import org.apache.iotdb.confignode.manager.UDFManager;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.load.LoadManager;
import org.apache.iotdb.confignode.manager.node.heartbeat.BaseNodeCache;
import org.apache.iotdb.confignode.manager.node.heartbeat.ConfigNodeHeartbeatCache;
import org.apache.iotdb.confignode.manager.node.heartbeat.DataNodeHeartbeatCache;
import org.apache.iotdb.confignode.manager.partition.PartitionManager;
+import org.apache.iotdb.confignode.manager.partition.PartitionMetrics;
import org.apache.iotdb.confignode.persistence.node.NodeInfo;
import org.apache.iotdb.confignode.procedure.env.DataNodeRemoveHandler;
import org.apache.iotdb.confignode.rpc.thrift.TCQConfig;
@@ -256,15 +257,16 @@ public class NodeManager {
* success, and DATANODE_ALREADY_REGISTERED when the DataNode is already exist.
*/
public DataSet registerDataNode(RegisterDataNodePlan registerDataNodePlan) {
+ int dataNodeId = nodeInfo.generateNextNodeId();
DataNodeRegisterResp resp = new DataNodeRegisterResp();
// Register new DataNode
- registerDataNodePlan
- .getDataNodeConfiguration()
- .getLocation()
- .setDataNodeId(nodeInfo.generateNextNodeId());
+ registerDataNodePlan.getDataNodeConfiguration().getLocation().setDataNodeId(dataNodeId);
getConsensusManager().write(registerDataNodePlan);
+ // Bind DataNode metrics
+ PartitionMetrics.bindDataNodePartitionMetrics(configManager, dataNodeId);
+
// Adjust the maximum RegionGroup number of each StorageGroup
getClusterSchemaManager().adjustMaxRegionGroupNum();
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeMetrics.java
new file mode 100644
index 0000000000..73ad29498d
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeMetrics.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.confignode.manager.node;
+
+import org.apache.iotdb.commons.cluster.NodeStatus;
+import org.apache.iotdb.commons.cluster.NodeType;
+import org.apache.iotdb.commons.service.metric.enums.Metric;
+import org.apache.iotdb.commons.service.metric.enums.Tag;
+import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.metricsets.IMetricSet;
+import org.apache.iotdb.metrics.utils.MetricLevel;
+import org.apache.iotdb.metrics.utils.MetricType;
+
+import java.util.Objects;
+
+/** Monitoring cluster Nodes Metrics. */
+public class NodeMetrics implements IMetricSet {
+
+ private final NodeManager nodeManager;
+
+ public NodeMetrics(NodeManager nodeManager) {
+ this.nodeManager = nodeManager;
+ }
+
+ @Override
+ public void bindTo(AbstractMetricService metricService) {
+ for (NodeStatus status : NodeStatus.values()) {
+ metricService.createAutoGauge(
+ Metric.NODE_NUM.toString(),
+ MetricLevel.CORE,
+ nodeManager,
+ manager -> manager.filterConfigNodeThroughStatus(status).size(),
+ Tag.TYPE.toString(),
+ NodeType.ConfigNode.getNodeType(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+
+ metricService.createAutoGauge(
+ Metric.NODE_NUM.toString(),
+ MetricLevel.CORE,
+ nodeManager,
+ manager -> manager.filterDataNodeThroughStatus(status).size(),
+ Tag.TYPE.toString(),
+ NodeType.DataNode.getNodeType(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+ }
+ }
+
+ @Override
+ public void unbindFrom(AbstractMetricService metricService) {
+ for (NodeStatus status : NodeStatus.values()) {
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.NODE_NUM.toString(),
+ Tag.TYPE.toString(),
+ NodeType.ConfigNode.getNodeType(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.NODE_NUM.toString(),
+ Tag.TYPE.toString(),
+ NodeType.DataNode.getNodeType(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ NodeMetrics that = (NodeMetrics) o;
+ return nodeManager.equals(that.nodeManager);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(nodeManager);
+ }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
index 88f0b6fafc..3c29a9f796 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
@@ -32,7 +32,6 @@ import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
import org.apache.iotdb.commons.partition.DataPartitionTable;
import org.apache.iotdb.commons.partition.SchemaPartitionTable;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
-import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.confignode.client.DataNodeRequestType;
import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool;
import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
@@ -64,12 +63,11 @@ import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
import org.apache.iotdb.confignode.exception.NoAvailableRegionGroupException;
import org.apache.iotdb.confignode.exception.NotEnoughDataNodeException;
import org.apache.iotdb.confignode.manager.ClusterSchemaManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
import org.apache.iotdb.confignode.manager.IManager;
import org.apache.iotdb.confignode.manager.ProcedureManager;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.load.LoadManager;
import org.apache.iotdb.confignode.manager.partition.heartbeat.RegionGroupCache;
-import org.apache.iotdb.confignode.persistence.metric.PartitionInfoMetrics;
import org.apache.iotdb.confignode.persistence.partition.PartitionInfo;
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionCreateTask;
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionDeleteTask;
@@ -90,7 +88,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
@@ -103,6 +100,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/** The PartitionManager Manages cluster PartitionTable read and write requests. */
@@ -511,15 +509,15 @@ public class PartitionManager {
}
/**
- * Get the DataNodes who contain the specific StorageGroup's Schema or Data
+ * Get the DataNodes who contain the specified Database's Schema or Data
*
- * @param storageGroup The specific StorageGroup's name
+ * @param database The specific Database's name
* @param type SchemaRegion or DataRegion
* @return Set<TDataNodeLocation>, the related DataNodes
*/
- public Set<TDataNodeLocation> getStorageGroupRelatedDataNodes(
- String storageGroup, TConsensusGroupType type) {
- return partitionInfo.getStorageGroupRelatedDataNodes(storageGroup, type);
+ public Set<TDataNodeLocation> getDatabaseRelatedDataNodes(
+ String database, TConsensusGroupType type) {
+ return partitionInfo.getDatabaseRelatedDataNodes(database, type);
}
/**
@@ -556,26 +554,51 @@ public class PartitionManager {
/**
* Only leader use this interface.
*
- * @param storageGroup The specified StorageGroup
+ * @param database The specified Database
* @return All Regions' RegionReplicaSet of the specified StorageGroup
*/
- public List<TRegionReplicaSet> getAllReplicaSets(String storageGroup) {
- return partitionInfo.getAllReplicaSets(storageGroup);
+ public List<TRegionReplicaSet> getAllReplicaSets(String database) {
+ return partitionInfo.getAllReplicaSets(database);
}
/**
* Only leader use this interface.
*
- * <p>Get the number of RegionGroups currently owned by the specific StorageGroup
+ * <p>Get the number of Regions currently owned by the specified DataNode
*
- * @param storageGroup StorageGroupName
+ * @param dataNodeId The specified DataNode
+ * @param type SchemaRegion or DataRegion
+ * @return The number of Regions currently owned by the specified DataNode
+ */
+ public int getRegionCount(int dataNodeId, TConsensusGroupType type) {
+ return partitionInfo.getRegionCount(dataNodeId, type);
+ }
+
+ /**
+ * Only leader use this interface.
+ *
+ * <p>Get the number of RegionGroups currently owned by the specified Database
+ *
+ * @param database DatabaseName
* @param type SchemaRegion or DataRegion
- * @return Number of Regions currently owned by the specific StorageGroup
- * @throws DatabaseNotExistsException When the specific StorageGroup doesn't exist
+ * @return Number of Regions currently owned by the specified Database
+ * @throws DatabaseNotExistsException When the specified Database doesn't exist
*/
- public int getRegionGroupCount(String storageGroup, TConsensusGroupType type)
+ public int getRegionGroupCount(String database, TConsensusGroupType type)
throws DatabaseNotExistsException {
- return partitionInfo.getRegionGroupCount(storageGroup, type);
+ return partitionInfo.getRegionGroupCount(database, type);
+ }
+
+ /**
+ * Only leader use this interface.
+ *
+ * <p>Get the assigned SeriesPartitionSlots count in the specified Database
+ *
+ * @param database The specified Database
+ * @return The assigned SeriesPartitionSlots count
+ */
+ public int getAssignedSeriesPartitionSlotsCount(String database) {
+ return partitionInfo.getAssignedSeriesPartitionSlotsCount(database);
}
/**
@@ -606,25 +629,22 @@ public class PartitionManager {
throw new NoAvailableRegionGroupException(type);
}
- result.sort(new PartitionComparator());
+ result.sort(
+ (o1, o2) -> {
+ // Use the number of partitions as the first priority
+ if (o1.getLeft() < o2.getLeft()) {
+ return -1;
+ } else if (o1.getLeft() > o2.getLeft()) {
+ return 1;
+ } else {
+ // Use RegionGroup status as second priority, Running > Available > Discouraged
+ return getRegionGroupStatus(o1.getRight())
+ .compareTo(getRegionGroupStatus(o2.getRight()));
+ }
+ });
return result;
}
- class PartitionComparator implements Comparator<Pair<Long, TConsensusGroupId>> {
-
- @Override
- public int compare(Pair<Long, TConsensusGroupId> o1, Pair<Long, TConsensusGroupId> o2) {
- // Use partition number as first priority
- if (o1.getLeft() < o2.getLeft()) {
- return -1;
- } else if (o1.getLeft() > o2.getLeft()) {
- return 1;
- } else {
- // Use RegionGroup status as second priority, Running > Available > Discouraged
- return getRegionGroupStatus(o1.getRight()).compareTo(getRegionGroupStatus(o2.getRight()));
- }
- }
- }
/**
* Only leader use this interface
*
@@ -655,10 +675,6 @@ public class PartitionManager {
getConsensusManager().write(preDeleteDatabasePlan);
}
- public void addMetrics() {
- MetricService.getInstance().addMetricSet(new PartitionInfoMetrics(partitionInfo));
- }
-
/**
* Get TSeriesPartitionSlot
*
@@ -736,6 +752,7 @@ public class PartitionManager {
public GetSeriesSlotListResp getSeriesSlotList(GetSeriesSlotListPlan plan) {
return (GetSeriesSlotListResp) getConsensusManager().read(plan).getDataset();
}
+
/**
* get database for region
*
@@ -1021,7 +1038,35 @@ public class PartitionManager {
}
/**
- * Safely get RegionStatus
+ * Count the number of cluster Regions with specified RegionStatus
+ *
+ * @param type The specified RegionGroupType
+ * @param status The specified statues
+ * @return The number of cluster Regions with specified RegionStatus
+ */
+ public int countRegionWithSpecifiedStatus(TConsensusGroupType type, RegionStatus... status) {
+ AtomicInteger result = new AtomicInteger(0);
+ regionGroupCacheMap.forEach(
+ (regionGroupId, regionGroupCache) -> {
+ if (type.equals(regionGroupId.getType())) {
+ regionGroupCache
+ .getStatistics()
+ .getRegionStatisticsMap()
+ .values()
+ .forEach(
+ regionStatistics -> {
+ if (Arrays.stream(status)
+ .anyMatch(s -> s.equals(regionStatistics.getRegionStatus()))) {
+ result.getAndIncrement();
+ }
+ });
+ }
+ });
+ return result.get();
+ }
+
+ /**
+ * Safely get RegionStatus.
*
* @param consensusGroupId Specified RegionGroupId
* @param dataNodeId Specified RegionReplicaId
@@ -1034,7 +1079,7 @@ public class PartitionManager {
}
/**
- * Safely get RegionGroupStatus
+ * Safely get RegionGroupStatus.
*
* @param consensusGroupId Specified RegionGroupId
* @return Corresponding RegionGroupStatus if cache exists, Disabled otherwise
@@ -1045,7 +1090,7 @@ public class PartitionManager {
: RegionGroupStatus.Disabled;
}
- /** Initialize the regionGroupCacheMap when the ConfigNode-Leader is switched */
+ /** Initialize the regionGroupCacheMap when the ConfigNode-Leader is switched. */
public void initRegionGroupHeartbeatCache() {
regionGroupCacheMap.clear();
getAllReplicaSets()
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
new file mode 100644
index 0000000000..21909d6a5a
--- /dev/null
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionMetrics.java
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.iotdb.confignode.manager.partition;
+
+import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
+import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration;
+import org.apache.iotdb.commons.cluster.RegionStatus;
+import org.apache.iotdb.commons.service.metric.MetricService;
+import org.apache.iotdb.commons.service.metric.enums.Metric;
+import org.apache.iotdb.commons.service.metric.enums.Tag;
+import org.apache.iotdb.commons.utils.NodeUrlUtils;
+import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
+import org.apache.iotdb.confignode.manager.ClusterSchemaManager;
+import org.apache.iotdb.confignode.manager.IManager;
+import org.apache.iotdb.confignode.manager.load.LoadManager;
+import org.apache.iotdb.confignode.manager.node.NodeManager;
+import org.apache.iotdb.metrics.AbstractMetricService;
+import org.apache.iotdb.metrics.metricsets.IMetricSet;
+import org.apache.iotdb.metrics.utils.MetricLevel;
+import org.apache.iotdb.metrics.utils.MetricType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Objects;
+
+public class PartitionMetrics implements IMetricSet {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(PartitionMetrics.class);
+
+ private final IManager configManager;
+
+ public PartitionMetrics(IManager configManager) {
+ this.configManager = configManager;
+ }
+
+ @Override
+ public void bindTo(AbstractMetricService metricService) {
+ bindRegionPartitionMetrics(metricService);
+ bindDataNodePartitionMetrics();
+ bindDatabasePartitionMetrics(metricService);
+ }
+
+ @Override
+ public void unbindFrom(AbstractMetricService metricService) {
+ unbindRegionPartitionMetrics(metricService);
+ unbindDataNodePartitionMetrics();
+ unbindDatabasePartitionMetrics(metricService);
+ }
+
+ private void bindRegionPartitionMetrics(AbstractMetricService metricService) {
+ for (RegionStatus status : RegionStatus.values()) {
+ // Count the number of SchemaRegions
+ metricService.createAutoGauge(
+ Metric.REGION_NUM.toString(),
+ MetricLevel.CORE,
+ getPartitionManager(),
+ partitionManager ->
+ partitionManager.countRegionWithSpecifiedStatus(
+ TConsensusGroupType.SchemaRegion, status),
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+
+ // Count the number of DataRegions
+ metricService.createAutoGauge(
+ Metric.REGION_NUM.toString(),
+ MetricLevel.CORE,
+ getPartitionManager(),
+ partitionManager ->
+ partitionManager.countRegionWithSpecifiedStatus(
+ TConsensusGroupType.DataRegion, status),
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+ }
+ }
+
+ private void unbindRegionPartitionMetrics(AbstractMetricService metricService) {
+ for (RegionStatus status : RegionStatus.values()) {
+ // Remove the number of SchemaRegions
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_NUM.toString(),
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+
+ // Remove the number of DataRegions
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_NUM.toString(),
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString(),
+ Tag.STATUS.toString(),
+ status.getStatus());
+ }
+ }
+
+ public static void bindDataNodePartitionMetrics(IManager configManager, int dataNodeId) {
+ MetricService metricService = MetricService.getInstance();
+ NodeManager nodeManager = configManager.getNodeManager();
+ PartitionManager partitionManager = configManager.getPartitionManager();
+ LoadManager loadManager = configManager.getLoadManager();
+
+ String dataNodeName =
+ NodeUrlUtils.convertTEndPointUrl(
+ nodeManager.getRegisteredDataNode(dataNodeId).getLocation().getClientRpcEndPoint());
+
+ // Count the number of Regions in the specified DataNode
+ metricService.createAutoGauge(
+ Metric.REGION_NUM_IN_DATA_NODE.toString(),
+ MetricLevel.CORE,
+ partitionManager,
+ obj -> obj.getRegionCount(dataNodeId, TConsensusGroupType.SchemaRegion),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.createAutoGauge(
+ Metric.REGION_NUM_IN_DATA_NODE.toString(),
+ MetricLevel.CORE,
+ partitionManager,
+ obj -> obj.getRegionCount(dataNodeId, TConsensusGroupType.DataRegion),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+
+ // Count the number of RegionGroup-leaders in the specified DataNode
+ metricService.createAutoGauge(
+ Metric.REGION_GROUP_LEADER_NUM_IN_DATA_NODE.toString(),
+ MetricLevel.CORE,
+ loadManager,
+ obj -> obj.getRegionGroupLeaderCount(dataNodeId, TConsensusGroupType.SchemaRegion),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.createAutoGauge(
+ Metric.REGION_GROUP_LEADER_NUM_IN_DATA_NODE.toString(),
+ MetricLevel.CORE,
+ loadManager,
+ obj -> obj.getRegionGroupLeaderCount(dataNodeId, TConsensusGroupType.DataRegion),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+ }
+
+ private void bindDataNodePartitionMetrics() {
+ List<TDataNodeConfiguration> registerDataNodes = getNodeManager().getRegisteredDataNodes();
+ for (TDataNodeConfiguration dataNodeConfiguration : registerDataNodes) {
+ int dataNodeId = dataNodeConfiguration.getLocation().getDataNodeId();
+ bindDataNodePartitionMetrics(configManager, dataNodeId);
+ }
+ }
+
+ public static void unbindDataNodePartitionMetrics(String dataNodeName) {
+ MetricService metricService = MetricService.getInstance();
+
+ // Remove the number of Regions in the specified DataNode
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_NUM_IN_DATA_NODE.toString(),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_NUM_IN_DATA_NODE.toString(),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+
+ // Remove the number of RegionGroup-leaders in the specified DataNode
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_GROUP_LEADER_NUM_IN_DATA_NODE.toString(),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_GROUP_LEADER_NUM_IN_DATA_NODE.toString(),
+ Tag.NAME.toString(),
+ dataNodeName,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+ }
+
+ private void unbindDataNodePartitionMetrics() {
+ List<TDataNodeConfiguration> registerDataNodes = getNodeManager().getRegisteredDataNodes();
+ for (TDataNodeConfiguration dataNodeConfiguration : registerDataNodes) {
+ String dataNodeName =
+ NodeUrlUtils.convertTEndPointUrl(
+ dataNodeConfiguration.getLocation().getClientRpcEndPoint());
+ unbindDataNodePartitionMetrics(dataNodeName);
+ }
+ }
+
+ public static void bindDatabasePartitionMetrics(IManager configManager, String database) {
+ MetricService metricService = MetricService.getInstance();
+ PartitionManager partitionManager = configManager.getPartitionManager();
+
+ // Count the number of SeriesSlots in the specified Database
+ metricService.createAutoGauge(
+ Metric.SERIES_SLOT_NUM_IN_DATABASE.toString(),
+ MetricLevel.CORE,
+ partitionManager,
+ manager -> manager.getAssignedSeriesPartitionSlotsCount(database),
+ Tag.NAME.toString(),
+ database);
+
+ // Count the number of RegionGroups in the specified Database
+ metricService.createAutoGauge(
+ Metric.REGION_GROUP_NUM_IN_DATABASE.toString(),
+ MetricLevel.CORE,
+ partitionManager,
+ manager -> {
+ try {
+ return manager.getRegionGroupCount(database, TConsensusGroupType.SchemaRegion);
+ } catch (DatabaseNotExistsException e) {
+ LOGGER.warn("Error when counting SchemaRegionGroups in Database: {}", database, e);
+ return -1;
+ }
+ },
+ Tag.NAME.toString(),
+ database,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.createAutoGauge(
+ Metric.REGION_GROUP_NUM_IN_DATABASE.toString(),
+ MetricLevel.CORE,
+ partitionManager,
+ manager -> {
+ try {
+ return manager.getRegionGroupCount(database, TConsensusGroupType.DataRegion);
+ } catch (DatabaseNotExistsException e) {
+ LOGGER.warn("Error when counting DataRegionGroups in Database: {}", database, e);
+ return -1;
+ }
+ },
+ Tag.NAME.toString(),
+ database,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+ }
+
+ private void bindDatabasePartitionMetrics(AbstractMetricService metricService) {
+ // Count the number of Databases
+ metricService.createAutoGauge(
+ Metric.DATABASE_NUM.toString(),
+ MetricLevel.CORE,
+ getClusterSchemaManager(),
+ clusterSchemaManager -> clusterSchemaManager.getDatabaseNames().size());
+
+ List<String> databases = getClusterSchemaManager().getDatabaseNames();
+ for (String database : databases) {
+ bindDatabasePartitionMetrics(configManager, database);
+ }
+ }
+
+ public static void unbindDatabasePartitionMetrics(String database) {
+ MetricService metricService = MetricService.getInstance();
+
+ // Remove the number of SeriesSlots in the specified Database
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.SERIES_SLOT_NUM_IN_DATABASE.toString(),
+ Tag.NAME.toString(),
+ database);
+
+ // Remove number of RegionGroups in the specified Database
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_GROUP_NUM_IN_DATABASE.toString(),
+ Tag.NAME.toString(),
+ database,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.SchemaRegion.toString());
+ metricService.remove(
+ MetricType.AUTO_GAUGE,
+ Metric.REGION_GROUP_NUM_IN_DATABASE.toString(),
+ Tag.NAME.toString(),
+ database,
+ Tag.TYPE.toString(),
+ TConsensusGroupType.DataRegion.toString());
+ }
+
+ private void unbindDatabasePartitionMetrics(AbstractMetricService metricService) {
+ // Remove the number of Databases
+ metricService.remove(MetricType.AUTO_GAUGE, Metric.DATABASE_NUM.toString());
+
+ List<String> databases = getClusterSchemaManager().getDatabaseNames();
+ for (String database : databases) {
+ unbindDatabasePartitionMetrics(database);
+ }
+ }
+
+ private NodeManager getNodeManager() {
+ return configManager.getNodeManager();
+ }
+
+ private ClusterSchemaManager getClusterSchemaManager() {
+ return configManager.getClusterSchemaManager();
+ }
+
+ private PartitionManager getPartitionManager() {
+ return configManager.getPartitionManager();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ PartitionMetrics that = (PartitionMetrics) o;
+ return configManager.equals(that.configManager);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(configManager);
+ }
+}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/metric/PartitionInfoMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/metric/PartitionInfoMetrics.java
deleted file mode 100644
index 022f15be4e..0000000000
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/metric/PartitionInfoMetrics.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.iotdb.confignode.persistence.metric;
-
-import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
-import org.apache.iotdb.confignode.persistence.partition.DatabasePartitionTable;
-import org.apache.iotdb.confignode.persistence.partition.PartitionInfo;
-import org.apache.iotdb.metrics.AbstractMetricService;
-import org.apache.iotdb.metrics.metricsets.IMetricSet;
-import org.apache.iotdb.metrics.utils.MetricLevel;
-import org.apache.iotdb.metrics.utils.MetricType;
-
-import java.util.Objects;
-
-public class PartitionInfoMetrics implements IMetricSet {
- private PartitionInfo partitionInfo;
-
- public PartitionInfoMetrics(PartitionInfo partitionInfo) {
- this.partitionInfo = partitionInfo;
- }
-
- @Override
- public void bindTo(AbstractMetricService metricService) {
- metricService.createAutoGauge(
- Metric.QUANTITY.toString(),
- MetricLevel.CORE,
- partitionInfo,
- PartitionInfo::getStorageGroupPartitionTableSize,
- Tag.NAME.toString(),
- "database");
- metricService.createAutoGauge(
- Metric.REGION.toString(),
- MetricLevel.IMPORTANT,
- partitionInfo,
- o -> o.updateRegionGroupMetric(TConsensusGroupType.SchemaRegion),
- Tag.NAME.toString(),
- "total",
- Tag.TYPE.toString(),
- TConsensusGroupType.SchemaRegion.toString());
- metricService.createAutoGauge(
- Metric.REGION.toString(),
- MetricLevel.IMPORTANT,
- partitionInfo,
- o -> o.updateRegionGroupMetric(TConsensusGroupType.DataRegion),
- Tag.NAME.toString(),
- "total",
- Tag.TYPE.toString(),
- TConsensusGroupType.DataRegion.toString());
- }
-
- @Override
- public void unbindFrom(AbstractMetricService metricService) {
- metricService.remove(
- MetricType.AUTO_GAUGE, Metric.QUANTITY.toString(), Tag.NAME.toString(), "database");
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.REGION.toString(),
- Tag.NAME.toString(),
- "total",
- Tag.TYPE.toString(),
- TConsensusGroupType.SchemaRegion.toString());
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.REGION.toString(),
- Tag.NAME.toString(),
- "total",
- Tag.TYPE.toString(),
- TConsensusGroupType.DataRegion.toString());
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- PartitionInfoMetrics that = (PartitionInfoMetrics) o;
- return Objects.equals(partitionInfo, that.partitionInfo);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(partitionInfo);
- }
-
- public static class StorageGroupPartitionTableMetrics implements IMetricSet {
- private DatabasePartitionTable databasePartitionTable;
-
- public StorageGroupPartitionTableMetrics(DatabasePartitionTable databasePartitionTable) {
- this.databasePartitionTable = databasePartitionTable;
- }
-
- @Override
- public void bindTo(AbstractMetricService metricService) {
- metricService.createAutoGauge(
- Metric.REGION.toString(),
- MetricLevel.NORMAL,
- databasePartitionTable,
- o -> o.getRegionGroupCount(TConsensusGroupType.SchemaRegion),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- TConsensusGroupType.SchemaRegion.toString());
- metricService.createAutoGauge(
- Metric.REGION.toString(),
- MetricLevel.NORMAL,
- databasePartitionTable,
- o -> o.getRegionGroupCount(TConsensusGroupType.DataRegion),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- TConsensusGroupType.DataRegion.toString());
- // TODO slot will be updated in the future
- metricService.createAutoGauge(
- Metric.SLOT.toString(),
- MetricLevel.NORMAL,
- databasePartitionTable,
- DatabasePartitionTable::getSchemaPartitionMapSize,
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- "schemaSlotNumber");
- metricService.createAutoGauge(
- Metric.SLOT.toString(),
- MetricLevel.NORMAL,
- databasePartitionTable,
- DatabasePartitionTable::getDataPartitionMapSize,
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- "dataSlotNumber");
- }
-
- @Override
- public void unbindFrom(AbstractMetricService metricService) {
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.REGION.toString(),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- TConsensusGroupType.SchemaRegion.toString());
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.REGION.toString(),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- TConsensusGroupType.DataRegion.toString());
- // TODO slot will be updated in the future
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.SLOT.toString(),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- "schemaSlotNumber");
- metricService.remove(
- MetricType.AUTO_GAUGE,
- Metric.SLOT.toString(),
- Tag.NAME.toString(),
- databasePartitionTable.getDatabaseName(),
- Tag.TYPE.toString(),
- "dataSlotNumber");
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
- StorageGroupPartitionTableMetrics that = (StorageGroupPartitionTableMetrics) o;
- return Objects.equals(databasePartitionTable, that.databasePartitionTable);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(databasePartitionTable);
- }
- }
-}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
index c432edad3b..5866a092c3 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java
@@ -137,23 +137,35 @@ public class DatabasePartitionTable {
}
/**
- * Get all RegionGroups currently owned by this StorageGroup
+ * Only leader use this interface.
+ *
+ * <p>Get the number of Regions currently owned by the specified DataNode
*
+ * @param dataNodeId The specified DataNode
* @param type SchemaRegion or DataRegion
- * @return The regions currently owned by this StorageGroup
+ * @return The number of Regions currently owned by the specified DataNode
*/
- public Set<RegionGroup> getRegionGroups(TConsensusGroupType type) {
- Set<RegionGroup> regionGroups = new HashSet<>();
+ public int getRegionCount(int dataNodeId, TConsensusGroupType type) {
+ AtomicInteger result = new AtomicInteger(0);
regionGroupMap
.values()
.forEach(
regionGroup -> {
- if (regionGroup.getId().getType().equals(type)) {
- regionGroups.add(regionGroup);
+ if (type.equals(regionGroup.getId().getType())) {
+ regionGroup
+ .getReplicaSet()
+ .getDataNodeLocations()
+ .forEach(
+ dataNodeLocation -> {
+ if (dataNodeLocation.getDataNodeId() == dataNodeId) {
+ result.getAndIncrement();
+ }
+ });
}
});
- return regionGroups;
+ return result.get();
}
+
/**
* Get the number of RegionGroups currently owned by this StorageGroup
*
@@ -271,7 +283,7 @@ public class DatabasePartitionTable {
* @param type SchemaRegion or DataRegion
* @return Set<TDataNodeLocation>, the related DataNodes
*/
- public Set<TDataNodeLocation> getStorageGroupRelatedDataNodes(TConsensusGroupType type) {
+ public Set<TDataNodeLocation> getDatabaseRelatedDataNodes(TConsensusGroupType type) {
HashSet<TDataNodeLocation> result = new HashSet<>();
regionGroupMap.forEach(
(consensusGroupId, regionGroup) -> {
@@ -491,14 +503,6 @@ public class DatabasePartitionTable {
return databaseName;
}
- public int getDataPartitionMapSize() {
- return dataPartitionTable.getDataPartitionMap().size();
- }
-
- public int getSchemaPartitionMapSize() {
- return schemaPartitionTable.getSchemaPartitionMap().size();
- }
-
@Override
public boolean equals(Object o) {
if (this == o) {
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
index aee6efb9ec..02100ddc44 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java
@@ -28,9 +28,6 @@ import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.partition.DataPartitionTable;
import org.apache.iotdb.commons.partition.SchemaPartitionTable;
-import org.apache.iotdb.commons.service.metric.MetricService;
-import org.apache.iotdb.commons.service.metric.enums.Metric;
-import org.apache.iotdb.commons.service.metric.enums.Tag;
import org.apache.iotdb.commons.snapshot.SnapshotProcessor;
import org.apache.iotdb.confignode.consensus.request.read.partition.GetDataPartitionPlan;
import org.apache.iotdb.confignode.consensus.request.read.partition.GetSchemaPartitionPlan;
@@ -55,13 +52,11 @@ import org.apache.iotdb.confignode.consensus.response.partition.RegionInfoListRe
import org.apache.iotdb.confignode.consensus.response.partition.SchemaNodeManagementResp;
import org.apache.iotdb.confignode.consensus.response.partition.SchemaPartitionResp;
import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
-import org.apache.iotdb.confignode.persistence.metric.PartitionInfoMetrics;
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionMaintainTask;
import org.apache.iotdb.confignode.rpc.thrift.TRegionInfo;
import org.apache.iotdb.confignode.rpc.thrift.TShowRegionReq;
import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList;
import org.apache.iotdb.consensus.common.DataSet;
-import org.apache.iotdb.metrics.utils.MetricLevel;
import org.apache.iotdb.rpc.RpcUtils;
import org.apache.iotdb.rpc.TSStatusCode;
import org.apache.iotdb.tsfile.utils.Pair;
@@ -80,7 +75,6 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -106,7 +100,7 @@ public class PartitionInfo implements SnapshotProcessor {
// For allocating Regions
private final AtomicInteger nextRegionGroupId;
// Map<StorageGroupName, StorageGroupPartitionInfo>
- private final Map<String, DatabasePartitionTable> storageGroupPartitionTables;
+ private final Map<String, DatabasePartitionTable> databasePartitionTables;
/** For Region-Maintainer */
// For RegionReplicas' asynchronous management
@@ -116,7 +110,7 @@ public class PartitionInfo implements SnapshotProcessor {
public PartitionInfo() {
this.nextRegionGroupId = new AtomicInteger(-1);
- this.storageGroupPartitionTables = new ConcurrentHashMap<>();
+ this.databasePartitionTables = new ConcurrentHashMap<>();
this.regionMaintainTaskList = Collections.synchronizedList(new ArrayList<>());
}
@@ -138,10 +132,7 @@ public class PartitionInfo implements SnapshotProcessor {
public TSStatus createDatabase(DatabaseSchemaPlan plan) {
String storageGroupName = plan.getSchema().getName();
DatabasePartitionTable databasePartitionTable = new DatabasePartitionTable(storageGroupName);
- storageGroupPartitionTables.put(storageGroupName, databasePartitionTable);
- MetricService.getInstance()
- .addMetricSet(
- new PartitionInfoMetrics.StorageGroupPartitionTableMetrics(databasePartitionTable));
+ databasePartitionTables.put(storageGroupName, databasePartitionTable);
return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
}
@@ -158,7 +149,7 @@ public class PartitionInfo implements SnapshotProcessor {
plan.getRegionGroupMap()
.forEach(
(storageGroup, regionReplicaSets) -> {
- storageGroupPartitionTables.get(storageGroup).createRegionGroups(regionReplicaSets);
+ databasePartitionTables.get(storageGroup).createRegionGroups(regionReplicaSets);
regionReplicaSets.forEach(
regionReplicaSet ->
maxRegionId.set(
@@ -251,7 +242,7 @@ public class PartitionInfo implements SnapshotProcessor {
final PreDeleteDatabasePlan.PreDeleteType preDeleteType =
preDeleteDatabasePlan.getPreDeleteType();
final String storageGroup = preDeleteDatabasePlan.getStorageGroup();
- DatabasePartitionTable databasePartitionTable = storageGroupPartitionTables.get(storageGroup);
+ DatabasePartitionTable databasePartitionTable = databasePartitionTables.get(storageGroup);
if (databasePartitionTable == null) {
return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode());
}
@@ -275,7 +266,7 @@ public class PartitionInfo implements SnapshotProcessor {
*/
public void deleteDatabase(DeleteDatabasePlan plan) {
// Clean the StorageGroupTable cache
- storageGroupPartitionTables.remove(plan.getName());
+ databasePartitionTables.remove(plan.getName());
}
/**
@@ -291,7 +282,7 @@ public class PartitionInfo implements SnapshotProcessor {
if (plan.getPartitionSlotsMap().size() == 0) {
// Return all SchemaPartitions when the queried PartitionSlots are empty
- storageGroupPartitionTables.forEach(
+ databasePartitionTables.forEach(
(storageGroup, databasePartitionTable) -> {
if (!databasePartitionTable.isPredeleted()) {
schemaPartition.put(storageGroup, new SchemaPartitionTable());
@@ -313,7 +304,7 @@ public class PartitionInfo implements SnapshotProcessor {
if (isStorageGroupExisted(storageGroup)) {
schemaPartition.put(storageGroup, new SchemaPartitionTable());
- if (!storageGroupPartitionTables
+ if (!databasePartitionTables
.get(storageGroup)
.getSchemaPartition(partitionSlots, schemaPartition.get(storageGroup))) {
isAllPartitionsExist.set(false);
@@ -352,7 +343,7 @@ public class PartitionInfo implements SnapshotProcessor {
if (isStorageGroupExisted(storageGroup)) {
dataPartition.put(storageGroup, new DataPartitionTable());
- if (!storageGroupPartitionTables
+ if (!databasePartitionTables
.get(storageGroup)
.getDataPartition(partitionSlots, dataPartition.get(storageGroup))) {
isAllPartitionsExist.set(false);
@@ -387,8 +378,8 @@ public class PartitionInfo implements SnapshotProcessor {
TSeriesPartitionSlot seriesPartitionSlot,
TTimePartitionSlot timePartitionSlot,
long timePartitionInterval) {
- if (storageGroupPartitionTables.containsKey(storageGroup)) {
- return storageGroupPartitionTables
+ if (databasePartitionTables.containsKey(storageGroup)) {
+ return databasePartitionTables
.get(storageGroup)
.getPrecededDataPartition(seriesPartitionSlot, timePartitionSlot, timePartitionInterval);
} else {
@@ -397,8 +388,7 @@ public class PartitionInfo implements SnapshotProcessor {
}
private boolean isStorageGroupExisted(String storageGroup) {
- final DatabasePartitionTable databasePartitionTable =
- storageGroupPartitionTables.get(storageGroup);
+ final DatabasePartitionTable databasePartitionTable = databasePartitionTables.get(storageGroup);
return databasePartitionTable != null && !databasePartitionTable.isPredeleted();
}
@@ -413,7 +403,7 @@ public class PartitionInfo implements SnapshotProcessor {
.forEach(
(storageGroup, schemaPartitionTable) -> {
if (isStorageGroupExisted(storageGroup)) {
- storageGroupPartitionTables
+ databasePartitionTables
.get(storageGroup)
.createSchemaPartition(schemaPartitionTable);
}
@@ -433,9 +423,7 @@ public class PartitionInfo implements SnapshotProcessor {
.forEach(
(storageGroup, dataPartitionTable) -> {
if (isStorageGroupExisted(storageGroup)) {
- storageGroupPartitionTables
- .get(storageGroup)
- .createDataPartition(dataPartitionTable);
+ databasePartitionTables.get(storageGroup).createDataPartition(dataPartitionTable);
}
});
@@ -453,7 +441,7 @@ public class PartitionInfo implements SnapshotProcessor {
storageGroup -> {
schemaPartitionMap.put(storageGroup, new SchemaPartitionTable());
- storageGroupPartitionTables
+ databasePartitionTables
.get(storageGroup)
.getSchemaPartition(new ArrayList<>(), schemaPartitionMap.get(storageGroup));
@@ -472,14 +460,14 @@ public class PartitionInfo implements SnapshotProcessor {
public DataSet getRegionInfoList(GetRegionInfoListPlan regionsInfoPlan) {
RegionInfoListResp regionResp = new RegionInfoListResp();
List<TRegionInfo> regionInfoList = new Vector<>();
- if (storageGroupPartitionTables.isEmpty()) {
+ if (databasePartitionTables.isEmpty()) {
regionResp.setStatus(RpcUtils.getStatus(TSStatusCode.SUCCESS_STATUS));
regionResp.setRegionInfoList(new ArrayList<>());
return regionResp;
}
TShowRegionReq showRegionReq = regionsInfoPlan.getShowRegionReq();
final List<String> storageGroups = showRegionReq != null ? showRegionReq.getDatabases() : null;
- storageGroupPartitionTables.forEach(
+ databasePartitionTables.forEach(
(storageGroup, databasePartitionTable) -> {
if (storageGroups != null && !storageGroups.contains(storageGroup)) {
return;
@@ -507,7 +495,7 @@ public class PartitionInfo implements SnapshotProcessor {
TConsensusGroupId regionId = req.getRegionId();
TDataNodeLocation oldNode = req.getOldNode();
TDataNodeLocation newNode = req.getNewNode();
- storageGroupPartitionTables.values().stream()
+ databasePartitionTables.values().stream()
.filter(sgPartitionTable -> sgPartitionTable.containRegion(regionId))
.forEach(
sgPartitionTable -> sgPartitionTable.updateRegionLocation(regionId, oldNode, newNode));
@@ -523,7 +511,7 @@ public class PartitionInfo implements SnapshotProcessor {
*/
public String getRegionStorageGroup(TConsensusGroupId regionId) {
Optional<DatabasePartitionTable> sgPartitionTableOptional =
- storageGroupPartitionTables.values().stream()
+ databasePartitionTables.values().stream()
.filter(s -> s.containRegion(regionId))
.findFirst();
return sgPartitionTableOptional.map(DatabasePartitionTable::getDatabaseName).orElse(null);
@@ -549,7 +537,7 @@ public class PartitionInfo implements SnapshotProcessor {
if (isStorageGroupExisted(storageGroup)) {
result.put(
storageGroup,
- storageGroupPartitionTables
+ databasePartitionTables
.get(storageGroup)
.filterUnassignedSchemaPartitionSlots(partitionSlots));
}
@@ -574,7 +562,7 @@ public class PartitionInfo implements SnapshotProcessor {
if (isStorageGroupExisted(storageGroup)) {
result.put(
storageGroup,
- storageGroupPartitionTables
+ databasePartitionTables
.get(storageGroup)
.filterUnassignedDataPartitionSlots(partitionSlots));
}
@@ -590,7 +578,7 @@ public class PartitionInfo implements SnapshotProcessor {
*/
public List<TRegionReplicaSet> getAllReplicaSets() {
List<TRegionReplicaSet> result = new ArrayList<>();
- storageGroupPartitionTables
+ databasePartitionTables
.values()
.forEach(
databasePartitionTable -> result.addAll(databasePartitionTable.getAllReplicaSets()));
@@ -605,7 +593,7 @@ public class PartitionInfo implements SnapshotProcessor {
*/
public List<TRegionReplicaSet> getAllReplicaSets(TConsensusGroupType type) {
List<TRegionReplicaSet> result = new ArrayList<>();
- storageGroupPartitionTables
+ databasePartitionTables
.values()
.forEach(
databasePartitionTable ->
@@ -616,12 +604,12 @@ public class PartitionInfo implements SnapshotProcessor {
/**
* Only leader use this interface.
*
- * @param storageGroup The specified StorageGroup
- * @return All Regions' RegionReplicaSet of the specified StorageGroup
+ * @param database The specified Database
+ * @return All Regions' RegionReplicaSet of the specified Database
*/
- public List<TRegionReplicaSet> getAllReplicaSets(String storageGroup) {
- if (storageGroupPartitionTables.containsKey(storageGroup)) {
- return storageGroupPartitionTables.get(storageGroup).getAllReplicaSets();
+ public List<TRegionReplicaSet> getAllReplicaSets(String database) {
+ if (databasePartitionTables.containsKey(database)) {
+ return databasePartitionTables.get(database).getAllReplicaSets();
} else {
return new ArrayList<>();
}
@@ -630,36 +618,63 @@ public class PartitionInfo implements SnapshotProcessor {
/**
* Only leader use this interface.
*
- * <p>Get the number of RegionGroups currently owned by the specified StorageGroup
+ * <p>Get the number of Regions currently owned by the specified DataNode
*
- * @param storageGroup StorageGroupName
+ * @param dataNodeId The specified DataNode
+ * @param type SchemaRegion or DataRegion
+ * @return The number of Regions currently owned by the specified DataNode
+ */
+ public int getRegionCount(int dataNodeId, TConsensusGroupType type) {
+ AtomicInteger result = new AtomicInteger(0);
+ databasePartitionTables
+ .values()
+ .forEach(
+ databasePartitionTable ->
+ result.getAndAdd(databasePartitionTable.getRegionCount(dataNodeId, type)));
+ return result.get();
+ }
+
+ /**
+ * Only leader use this interface.
+ *
+ * <p>Get the number of RegionGroups currently owned by the specified Database
+ *
+ * @param database DatabaseName
* @param type SchemaRegion or DataRegion
* @return Number of Regions currently owned by the specific StorageGroup
* @throws DatabaseNotExistsException When the specific StorageGroup doesn't exist
*/
- public int getRegionGroupCount(String storageGroup, TConsensusGroupType type)
+ public int getRegionGroupCount(String database, TConsensusGroupType type)
throws DatabaseNotExistsException {
- if (!isStorageGroupExisted(storageGroup)) {
- throw new DatabaseNotExistsException(storageGroup);
+ if (!isStorageGroupExisted(database)) {
+ throw new DatabaseNotExistsException(database);
}
- return storageGroupPartitionTables.get(storageGroup).getRegionGroupCount(type);
+ return databasePartitionTables.get(database).getRegionGroupCount(type);
}
- public int getAssignedSeriesPartitionSlotsCount(String storageGroup) {
- return storageGroupPartitionTables.get(storageGroup).getAssignedSeriesPartitionSlotsCount();
+ /**
+ * Only leader use this interface.
+ *
+ * <p>Get the assigned SeriesPartitionSlots count in the specified Database
+ *
+ * @param database The specified Database
+ * @return The assigned SeriesPartitionSlots count
+ */
+ public int getAssignedSeriesPartitionSlotsCount(String database) {
+ return databasePartitionTables.get(database).getAssignedSeriesPartitionSlotsCount();
}
/**
* Get the DataNodes who contain the specific StorageGroup's Schema or Data
*
- * @param storageGroup The specific StorageGroup's name
+ * @param database The specific StorageGroup's name
* @param type SchemaRegion or DataRegion
* @return Set<TDataNodeLocation>, the related DataNodes
*/
- public Set<TDataNodeLocation> getStorageGroupRelatedDataNodes(
- String storageGroup, TConsensusGroupType type) {
- return storageGroupPartitionTables.get(storageGroup).getStorageGroupRelatedDataNodes(type);
+ public Set<TDataNodeLocation> getDatabaseRelatedDataNodes(
+ String database, TConsensusGroupType type) {
+ return databasePartitionTables.get(database).getDatabaseRelatedDataNodes(type);
}
/**
@@ -672,53 +687,7 @@ public class PartitionInfo implements SnapshotProcessor {
*/
public List<Pair<Long, TConsensusGroupId>> getRegionGroupSlotsCounter(
String storageGroup, TConsensusGroupType type) {
- return storageGroupPartitionTables.get(storageGroup).getRegionGroupSlotsCounter(type);
- }
-
- /**
- * Update RegionGroup-related metric
- *
- * @param type SchemaRegion or DataRegion
- * @return the number of SchemaRegion or DataRegion
- */
- public int updateRegionGroupMetric(TConsensusGroupType type) {
- Set<RegionGroup> regionGroups = new HashSet<>();
- for (Map.Entry<String, DatabasePartitionTable> entry : storageGroupPartitionTables.entrySet()) {
- regionGroups.addAll(entry.getValue().getRegionGroups(type));
- }
- int result = regionGroups.size();
- // datanode location -> region number
- Map<TDataNodeLocation, Integer> dataNodeLocationIntegerMap = new HashMap<>();
- for (RegionGroup regionGroup : regionGroups) {
- TRegionReplicaSet regionReplicaSet = regionGroup.getReplicaSet();
- List<TDataNodeLocation> dataNodeLocations = regionReplicaSet.getDataNodeLocations();
- for (TDataNodeLocation dataNodeLocation : dataNodeLocations) {
- if (!dataNodeLocationIntegerMap.containsKey(dataNodeLocation)) {
- dataNodeLocationIntegerMap.put(dataNodeLocation, 0);
- }
- dataNodeLocationIntegerMap.put(
- dataNodeLocation, dataNodeLocationIntegerMap.get(dataNodeLocation) + 1);
- }
- }
- for (Map.Entry<TDataNodeLocation, Integer> entry : dataNodeLocationIntegerMap.entrySet()) {
- TDataNodeLocation dataNodeLocation = entry.getKey();
- String name =
- "EndPoint("
- + dataNodeLocation.getClientRpcEndPoint().ip
- + ":"
- + dataNodeLocation.getClientRpcEndPoint().port
- + ")";
- MetricService.getInstance()
- .getOrCreateGauge(
- Metric.REGION.toString(),
- MetricLevel.IMPORTANT,
- Tag.NAME.toString(),
- name,
- Tag.TYPE.toString(),
- type.toString())
- .set(dataNodeLocationIntegerMap.get(dataNodeLocation));
- }
- return result;
+ return databasePartitionTables.get(storageGroup).getRegionGroupSlotsCounter(type);
}
@Override
@@ -744,9 +713,9 @@ public class PartitionInfo implements SnapshotProcessor {
ReadWriteIOUtils.write(nextRegionGroupId.get(), fileOutputStream);
// serialize StorageGroupPartitionTable
- ReadWriteIOUtils.write(storageGroupPartitionTables.size(), fileOutputStream);
+ ReadWriteIOUtils.write(databasePartitionTables.size(), fileOutputStream);
for (Map.Entry<String, DatabasePartitionTable> storageGroupPartitionTableEntry :
- storageGroupPartitionTables.entrySet()) {
+ databasePartitionTables.entrySet()) {
ReadWriteIOUtils.write(storageGroupPartitionTableEntry.getKey(), fileOutputStream);
storageGroupPartitionTableEntry.getValue().serialize(fileOutputStream, protocol);
}
@@ -803,7 +772,7 @@ public class PartitionInfo implements SnapshotProcessor {
}
DatabasePartitionTable databasePartitionTable = new DatabasePartitionTable(storageGroup);
databasePartitionTable.deserialize(fileInputStream, protocol);
- storageGroupPartitionTables.put(storageGroup, databasePartitionTable);
+ databasePartitionTables.put(storageGroup, databasePartitionTable);
}
// restore deletedRegionSet
@@ -820,8 +789,7 @@ public class PartitionInfo implements SnapshotProcessor {
return new GetRegionIdResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), new ArrayList<>());
}
- DatabasePartitionTable sgPartitionTable =
- storageGroupPartitionTables.get(plan.getStorageGroup());
+ DatabasePartitionTable sgPartitionTable = databasePartitionTables.get(plan.getStorageGroup());
return new GetRegionIdResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()),
sgPartitionTable.getRegionId(
@@ -833,8 +801,7 @@ public class PartitionInfo implements SnapshotProcessor {
return new GetTimeSlotListResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), new ArrayList<>());
}
- DatabasePartitionTable sgPartitionTable =
- storageGroupPartitionTables.get(plan.getStorageGroup());
+ DatabasePartitionTable sgPartitionTable = databasePartitionTables.get(plan.getStorageGroup());
return new GetTimeSlotListResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()),
sgPartitionTable.getTimeSlotList(
@@ -846,35 +813,34 @@ public class PartitionInfo implements SnapshotProcessor {
return new GetSeriesSlotListResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()), new ArrayList<>());
}
- DatabasePartitionTable sgPartitionTable =
- storageGroupPartitionTables.get(plan.getStorageGroup());
+ DatabasePartitionTable sgPartitionTable = databasePartitionTables.get(plan.getStorageGroup());
return new GetSeriesSlotListResp(
new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()),
sgPartitionTable.getSeriesSlotList(plan.getPartitionType()));
}
- public int getStorageGroupPartitionTableSize() {
- return storageGroupPartitionTables.size();
- }
-
public void clear() {
nextRegionGroupId.set(-1);
- storageGroupPartitionTables.clear();
+ databasePartitionTables.clear();
regionMaintainTaskList.clear();
}
@Override
public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
PartitionInfo that = (PartitionInfo) o;
return nextRegionGroupId.get() == that.nextRegionGroupId.get()
- && storageGroupPartitionTables.equals(that.storageGroupPartitionTables)
+ && databasePartitionTables.equals(that.databasePartitionTables)
&& regionMaintainTaskList.equals(that.regionMaintainTaskList);
}
@Override
public int hashCode() {
- return Objects.hash(nextRegionGroupId, storageGroupPartitionTables, regionMaintainTaskList);
+ return Objects.hash(nextRegionGroupId, databasePartitionTables, regionMaintainTaskList);
}
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
index ccd4b4f9f4..20c5b25f87 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java
@@ -92,7 +92,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
private static final Logger LOGGER = LoggerFactory.getLogger(ClusterSchemaInfo.class);
// StorageGroup read write lock
- private final ReentrantReadWriteLock storageGroupReadWriteLock;
+ private final ReentrantReadWriteLock databaseReadWriteLock;
private final ConfigMTree mTree;
private static final String SNAPSHOT_FILENAME = "cluster_schema.bin";
@@ -102,7 +102,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
private final TemplateTable templateTable;
public ClusterSchemaInfo() throws IOException {
- storageGroupReadWriteLock = new ReentrantReadWriteLock();
+ databaseReadWriteLock = new ReentrantReadWriteLock();
try {
mTree = new ConfigMTree();
@@ -125,7 +125,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public TSStatus createDatabase(DatabaseSchemaPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
// Set StorageGroup
TDatabaseSchema storageGroupSchema = plan.getSchema();
@@ -142,7 +142,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(e.getErrorCode()).setMessage(e.getMessage());
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
@@ -155,7 +155,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public TSStatus alterDatabase(DatabaseSchemaPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
TDatabaseSchema alterSchema = plan.getSchema();
PartialPath partialPathName = new PartialPath(alterSchema.getName());
@@ -202,7 +202,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(e.getErrorCode()).setMessage(e.getMessage());
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
@@ -215,7 +215,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public TSStatus deleteDatabase(DeleteDatabasePlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
// Delete StorageGroup
String storageGroup = plan.getName();
@@ -229,7 +229,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode())
.setMessage("Database not exist: " + e.getMessage());
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
@@ -237,7 +237,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
/** @return The number of matched StorageGroups by the specific StorageGroup pattern */
public CountDatabaseResp countMatchedDatabases(CountDatabasePlan plan) {
CountDatabaseResp result = new CountDatabaseResp();
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
PartialPath patternPath = new PartialPath(plan.getStorageGroupPattern());
result.setCount(mTree.getStorageGroupNum(patternPath, false));
@@ -248,7 +248,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
new TSStatus(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode())
.setMessage(ERROR_NAME + ": " + e.getMessage()));
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return result;
}
@@ -256,7 +256,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
/** @return All StorageGroupSchemas that matches to the specific StorageGroup pattern */
public DatabaseSchemaResp getMatchedDatabaseSchemas(GetDatabasePlan plan) {
DatabaseSchemaResp result = new DatabaseSchemaResp();
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
Map<String, TDatabaseSchema> schemaMap = new HashMap<>();
PartialPath patternPath = new PartialPath(plan.getStorageGroupPattern());
@@ -274,14 +274,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
new TSStatus(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode())
.setMessage(ERROR_NAME + ": " + e.getMessage()));
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return result;
}
public TSStatus setTTL(SetTTLPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
PartialPath patternPath = new PartialPath(plan.getStorageGroupPathPattern());
List<PartialPath> matchedPaths = mTree.getBelongedStorageGroups(patternPath);
@@ -301,14 +301,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()).setMessage(ERROR_NAME);
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
public TSStatus setSchemaReplicationFactor(SetSchemaReplicationFactorPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
PartialPath path = new PartialPath(plan.getStorageGroup());
if (mTree.isStorageGroupAlreadySet(path)) {
@@ -324,14 +324,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()).setMessage(ERROR_NAME);
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
public TSStatus setDataReplicationFactor(SetDataReplicationFactorPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
PartialPath path = new PartialPath(plan.getStorageGroup());
if (mTree.isStorageGroupAlreadySet(path)) {
@@ -347,14 +347,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()).setMessage(ERROR_NAME);
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
public TSStatus setTimePartitionInterval(SetTimePartitionIntervalPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
PartialPath path = new PartialPath(plan.getStorageGroup());
if (mTree.isStorageGroupAlreadySet(path)) {
@@ -370,7 +370,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()).setMessage(ERROR_NAME);
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
@@ -383,7 +383,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public TSStatus adjustMaxRegionGroupCount(AdjustMaxRegionGroupNumPlan plan) {
TSStatus result = new TSStatus();
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try {
for (Map.Entry<String, Pair<Integer, Integer>> entry :
plan.getMaxRegionGroupNumMap().entrySet()) {
@@ -398,7 +398,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.error(ERROR_NAME, e);
result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode());
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
return result;
}
@@ -410,20 +410,20 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
/**
* Only leader use this interface.
*
- * @return List<StorageGroupName>, all storageGroups' name
+ * @return List<StorageGroupName>, all Databases' name
*/
public List<String> getDatabaseNames() {
- List<String> storageGroups = new ArrayList<>();
- storageGroupReadWriteLock.readLock().lock();
+ List<String> databases = new ArrayList<>();
+ databaseReadWriteLock.readLock().lock();
try {
List<PartialPath> namePaths = mTree.getAllStorageGroupPaths();
for (PartialPath path : namePaths) {
- storageGroups.add(path.getFullPath());
+ databases.add(path.getFullPath());
}
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
- return storageGroups;
+ return databases;
}
/**
@@ -433,11 +433,11 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
* @throws IllegalPathException If the specified Database's name is illegal
*/
public boolean isDatabaseExisted(String databaseName) throws IllegalPathException {
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
return mTree.isStorageGroupAlreadySet(new PartialPath(databaseName));
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -448,11 +448,11 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
* @throws MetadataException If the specific StorageGroup already exists
*/
public void checkContainsStorageGroup(String storageName) throws MetadataException {
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
mTree.checkStorageGroupAlreadySet(new PartialPath(storageName));
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -465,7 +465,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public TDatabaseSchema getMatchedDatabaseSchemaByName(String storageGroup)
throws DatabaseNotExistsException {
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
return mTree
.getStorageGroupNodeByStorageGroupPath(new PartialPath(storageGroup))
@@ -473,7 +473,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
} catch (MetadataException e) {
throw new DatabaseNotExistsException(storageGroup);
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -485,7 +485,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
*/
public Map<String, TDatabaseSchema> getMatchedDatabaseSchemasByName(List<String> rawPathList) {
Map<String, TDatabaseSchema> schemaMap = new HashMap<>();
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
for (String rawPath : rawPathList) {
PartialPath patternPath = new PartialPath(rawPath);
@@ -498,7 +498,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
} catch (MetadataException e) {
LOGGER.warn(ERROR_NAME, e);
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return schemaMap;
}
@@ -511,7 +511,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
* @return maxSchemaRegionGroupNum or maxDataRegionGroupNum
*/
public int getMinRegionGroupNum(String database, TConsensusGroupType consensusGroupType) {
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
PartialPath path = new PartialPath(database);
TDatabaseSchema storageGroupSchema =
@@ -527,7 +527,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.warn(ERROR_NAME, e);
return -1;
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -539,7 +539,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
* @return maxSchemaRegionGroupNum or maxDataRegionGroupNum
*/
public int getMaxRegionGroupNum(String database, TConsensusGroupType consensusGroupType) {
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
PartialPath path = new PartialPath(database);
TDatabaseSchema storageGroupSchema =
@@ -555,7 +555,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
LOGGER.warn(ERROR_NAME, e);
return -1;
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -576,7 +576,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
File tmpFile = new File(snapshotFile.getAbsolutePath() + "-" + UUID.randomUUID());
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
try (FileOutputStream fileOutputStream = new FileOutputStream(tmpFile);
BufferedOutputStream outputStream = new BufferedOutputStream(fileOutputStream)) {
@@ -595,7 +595,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
"Can't delete temporary snapshot file: {}, retrying...", tmpFile.getAbsolutePath());
}
}
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
}
@@ -613,14 +613,14 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
snapshotFile.getAbsolutePath());
return;
}
- storageGroupReadWriteLock.writeLock().lock();
+ databaseReadWriteLock.writeLock().lock();
try (FileInputStream fileInputStream = new FileInputStream(snapshotFile);
BufferedInputStream bufferedInputStream = new BufferedInputStream(fileInputStream)) {
// Load snapshot of MTree
mTree.clear();
mTree.deserialize(bufferedInputStream);
} finally {
- storageGroupReadWriteLock.writeLock().unlock();
+ databaseReadWriteLock.writeLock().unlock();
}
}
@@ -628,13 +628,13 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
PartialPath partialPath, int level) {
Pair<List<PartialPath>, Set<PartialPath>> matchedPathsInNextLevel =
new Pair(new HashSet<>(), new HashSet<>());
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
matchedPathsInNextLevel = mTree.getNodesListInGivenLevel(partialPath, level, true);
} catch (MetadataException e) {
LOGGER.error("Error get matched paths in given level.", e);
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return matchedPathsInNextLevel;
}
@@ -643,13 +643,13 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
PartialPath partialPath) {
Pair<Set<TSchemaNode>, Set<PartialPath>> matchedPathsInNextLevel =
new Pair<>(new HashSet<>(), new HashSet<>());
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
matchedPathsInNextLevel = mTree.getChildNodePathInNextLevel(partialPath);
} catch (MetadataException e) {
LOGGER.error("Error get matched paths in next level.", e);
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return matchedPathsInNextLevel;
}
@@ -872,7 +872,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
public Map<String, TDatabaseSchema> getMatchedStorageGroupSchemasByOneName(
String[] storageGroupPathPattern) {
Map<String, TDatabaseSchema> schemaMap = new HashMap<>();
- storageGroupReadWriteLock.readLock().lock();
+ databaseReadWriteLock.readLock().lock();
try {
PartialPath patternPath = new PartialPath(storageGroupPathPattern);
List<PartialPath> matchedPaths = mTree.getBelongedStorageGroups(patternPath);
@@ -883,7 +883,7 @@ public class ClusterSchemaInfo implements SnapshotProcessor {
} catch (MetadataException e) {
LOGGER.warn(ERROR_NAME, e);
} finally {
- storageGroupReadWriteLock.readLock().unlock();
+ databaseReadWriteLock.readLock().unlock();
}
return schemaMap;
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
index ad8f9bd09e..1771d16841 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java
@@ -45,7 +45,7 @@ import org.apache.iotdb.confignode.exception.AddPeerException;
import org.apache.iotdb.confignode.exception.DatabaseNotExistsException;
import org.apache.iotdb.confignode.manager.ClusterSchemaManager;
import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.manager.load.LoadManager;
import org.apache.iotdb.confignode.manager.node.NodeManager;
import org.apache.iotdb.confignode.manager.node.heartbeat.NodeHeartbeatSample;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
index d424a1eff9..33fec29316 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/DataNodeRemoveHandler.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation;
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
import org.apache.iotdb.commons.cluster.NodeStatus;
+import org.apache.iotdb.commons.utils.NodeUrlUtils;
import org.apache.iotdb.confignode.client.DataNodeRequestType;
import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool;
import org.apache.iotdb.confignode.conf.ConfigNodeConfig;
@@ -34,6 +35,7 @@ import org.apache.iotdb.confignode.consensus.request.write.partition.UpdateRegio
import org.apache.iotdb.confignode.consensus.response.datanode.DataNodeToStatusResp;
import org.apache.iotdb.confignode.manager.ConfigManager;
import org.apache.iotdb.confignode.manager.node.heartbeat.BaseNodeCache;
+import org.apache.iotdb.confignode.manager.partition.PartitionMetrics;
import org.apache.iotdb.confignode.persistence.node.NodeInfo;
import org.apache.iotdb.confignode.procedure.scheduler.LockQueue;
import org.apache.iotdb.mpp.rpc.thrift.TCreatePeerReq;
@@ -551,11 +553,16 @@ public class DataNodeRemoveHandler {
/**
* Remove data node in node info
*
- * @param tDataNodeLocation data node location
+ * @param dataNodeLocation data node location
*/
- public void removeDataNodePersistence(TDataNodeLocation tDataNodeLocation) {
- List<TDataNodeLocation> removeDataNodes = Collections.singletonList(tDataNodeLocation);
+ public void removeDataNodePersistence(TDataNodeLocation dataNodeLocation) {
+ // Remove consensus record
+ List<TDataNodeLocation> removeDataNodes = Collections.singletonList(dataNodeLocation);
configManager.getConsensusManager().write(new RemoveDataNodePlan(removeDataNodes));
+
+ // Remove metrics
+ PartitionMetrics.unbindDataNodePartitionMetrics(
+ NodeUrlUtils.convertTEndPointUrl(dataNodeLocation.getClientRpcEndPoint()));
}
/**
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
similarity index 94%
rename from confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java
rename to confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
index c00c90d816..a9b094243e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteStorageGroupProcedure.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java
@@ -31,6 +31,7 @@ import org.apache.iotdb.confignode.client.async.AsyncDataNodeClientPool;
import org.apache.iotdb.confignode.client.async.handlers.AsyncClientHandler;
import org.apache.iotdb.confignode.consensus.request.write.region.OfferRegionMaintainTasksPlan;
import org.apache.iotdb.confignode.consensus.request.write.storagegroup.PreDeleteDatabasePlan;
+import org.apache.iotdb.confignode.manager.partition.PartitionMetrics;
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionDeleteTask;
import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv;
import org.apache.iotdb.confignode.procedure.exception.ProcedureException;
@@ -53,19 +54,20 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
-public class DeleteStorageGroupProcedure
+public class DeleteDatabaseProcedure
extends StateMachineProcedure<ConfigNodeProcedureEnv, DeleteStorageGroupState> {
- private static final Logger LOG = LoggerFactory.getLogger(DeleteStorageGroupProcedure.class);
+ private static final Logger LOG = LoggerFactory.getLogger(DeleteDatabaseProcedure.class);
private static final int RETRY_THRESHOLD = 5;
private TDatabaseSchema deleteSgSchema;
- public DeleteStorageGroupProcedure() {
+ public DeleteDatabaseProcedure() {
super();
}
- public DeleteStorageGroupProcedure(TDatabaseSchema deleteSgSchema) {
+ public DeleteDatabaseProcedure(TDatabaseSchema deleteSgSchema) {
super();
this.deleteSgSchema = deleteSgSchema;
}
@@ -145,9 +147,12 @@ public class DeleteStorageGroupProcedure
env.getConfigManager().getConsensusManager().write(dataRegionDeleteTaskOfferPlan);
}
- // Delete StorageGroupPartitionTable
+ // Delete DatabasePartitionTable
TSStatus deleteConfigResult = env.deleteConfig(deleteSgSchema.getName());
+ // Delete Database metrics
+ PartitionMetrics.unbindDatabasePartitionMetrics(deleteSgSchema.getName());
+
// try sync delete schema region
AsyncClientHandler<TConsensusGroupId, TSStatus> asyncClientHandler =
new AsyncClientHandler<>(DataNodeRequestType.DELETE_REGION);
@@ -267,12 +272,17 @@ public class DeleteStorageGroupProcedure
@Override
public boolean equals(Object that) {
- if (that instanceof DeleteStorageGroupProcedure) {
- DeleteStorageGroupProcedure thatProc = (DeleteStorageGroupProcedure) that;
+ if (that instanceof DeleteDatabaseProcedure) {
+ DeleteDatabaseProcedure thatProc = (DeleteDatabaseProcedure) that;
return thatProc.getProcId() == this.getProcId()
&& thatProc.getState() == this.getState()
&& thatProc.deleteSgSchema.equals(this.getDeleteSgSchema());
}
return false;
}
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(deleteSgSchema);
+ }
}
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java
index ef27ed492f..0d209a5f9e 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ConfigProcedureStore.java
@@ -23,7 +23,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.confignode.consensus.request.write.procedure.DeleteProcedurePlan;
import org.apache.iotdb.confignode.consensus.request.write.procedure.UpdateProcedurePlan;
import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.persistence.ProcedureInfo;
import org.apache.iotdb.confignode.procedure.Procedure;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
index 268d1965c9..ab4e9f8af9 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/procedure/store/ProcedureFactory.java
@@ -25,7 +25,7 @@ import org.apache.iotdb.confignode.procedure.impl.node.AddConfigNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.node.RemoveConfigNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.node.RemoveDataNodeProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.DeactivateTemplateProcedure;
-import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.DeleteTimeSeriesProcedure;
import org.apache.iotdb.confignode.procedure.impl.schema.UnsetTemplateProcedure;
import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
@@ -60,7 +60,7 @@ public class ProcedureFactory implements IProcedureFactory {
Procedure procedure;
switch (procedureType) {
case DELETE_STORAGE_GROUP_PROCEDURE:
- procedure = new DeleteStorageGroupProcedure();
+ procedure = new DeleteDatabaseProcedure();
break;
case ADD_CONFIG_NODE_PROCEDURE:
procedure = new AddConfigNodeProcedure();
@@ -118,7 +118,7 @@ public class ProcedureFactory implements IProcedureFactory {
}
public static ProcedureType getProcedureType(Procedure procedure) {
- if (procedure instanceof DeleteStorageGroupProcedure) {
+ if (procedure instanceof DeleteDatabaseProcedure) {
return ProcedureType.DELETE_STORAGE_GROUP_PROCEDURE;
} else if (procedure instanceof AddConfigNodeProcedure) {
return ProcedureType.ADD_CONFIG_NODE_PROCEDURE;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
index 616743b9fe..198d272a4c 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/ConfigNode.java
@@ -205,18 +205,6 @@ public class ConfigNode implements ConfigNodeMBean {
}
}
- private void initConfigManager() {
- try {
- configManager = new ConfigManager();
- } catch (IOException e) {
- LOGGER.error("Can't start ConfigNode consensus group!", e);
- stop();
- }
- // Add some Metrics for configManager
- configManager.addMetrics();
- LOGGER.info("Successfully initialize ConfigManager.");
- }
-
private void setUpInternalServices() throws StartupException, IOException {
// Setup JMXService
registerManager.register(new JMXService());
@@ -233,6 +221,18 @@ public class ConfigNode implements ConfigNodeMBean {
LOGGER.info("Successfully setup internal services.");
}
+ private void initConfigManager() {
+ try {
+ configManager = new ConfigManager();
+ } catch (IOException e) {
+ LOGGER.error("Can't start ConfigNode consensus group!", e);
+ stop();
+ }
+ // Add some Metrics for configManager
+ configManager.addMetrics();
+ LOGGER.info("Successfully initialize ConfigManager.");
+ }
+
/** Register Non-seed ConfigNode when first startup */
private void sendRegisterConfigNodeRequest() throws StartupException, IOException {
TConfigNodeRegisterReq req =
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceHandlerMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceHandlerMetrics.java
index 86f48bd6ab..7c3483d72a 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceHandlerMetrics.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceHandlerMetrics.java
@@ -28,7 +28,7 @@ import java.util.Objects;
import java.util.concurrent.atomic.AtomicLong;
public class ConfigNodeRPCServiceHandlerMetrics implements IMetricSet {
- private AtomicLong thriftConnectionNumber;
+ private final AtomicLong thriftConnectionNumber;
public ConfigNodeRPCServiceHandlerMetrics(AtomicLong thriftConnectionNumber) {
this.thriftConnectionNumber = thriftConnectionNumber;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceMetrics.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceMetrics.java
index 4b06390cee..e237dd75b0 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceMetrics.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceMetrics.java
@@ -29,7 +29,8 @@ import org.apache.iotdb.metrics.utils.MetricType;
import java.util.Objects;
public class ConfigNodeRPCServiceMetrics implements IMetricSet {
- private AbstractThriftServiceThread thriftServiceThread;
+
+ private final AbstractThriftServiceThread thriftServiceThread;
public ConfigNodeRPCServiceMetrics(AbstractThriftServiceThread thriftServiceThread) {
this.thriftServiceThread = thriftServiceThread;
diff --git a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 0b75e1767a..2a3adb9841 100644
--- a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++ b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -65,7 +65,7 @@ import org.apache.iotdb.confignode.consensus.response.datanode.DataNodeRegisterR
import org.apache.iotdb.confignode.consensus.response.datanode.DataNodeToStatusResp;
import org.apache.iotdb.confignode.consensus.response.partition.RegionInfoListResp;
import org.apache.iotdb.confignode.manager.ConfigManager;
-import org.apache.iotdb.confignode.manager.ConsensusManager;
+import org.apache.iotdb.confignode.manager.consensus.ConsensusManager;
import org.apache.iotdb.confignode.rpc.thrift.IConfigNodeRPCService;
import org.apache.iotdb.confignode.rpc.thrift.TAddConsensusGroupReq;
import org.apache.iotdb.confignode.rpc.thrift.TAuthorizerReq;
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
index 2d6fd19749..2355c53cd6 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java
@@ -107,7 +107,7 @@ import org.apache.iotdb.confignode.consensus.request.write.trigger.UpdateTrigger
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionCreateTask;
import org.apache.iotdb.confignode.persistence.partition.maintainer.RegionDeleteTask;
import org.apache.iotdb.confignode.procedure.Procedure;
-import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure;
import org.apache.iotdb.confignode.procedure.impl.statemachine.CreateRegionGroupsProcedure;
import org.apache.iotdb.confignode.rpc.thrift.TCreateCQReq;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
@@ -752,15 +752,15 @@ public class ConfigPhysicalPlanSerDeTest {
@Test
public void updateProcedureTest() throws IOException {
// test procedure equals DeleteStorageGroupProcedure
- DeleteStorageGroupProcedure deleteStorageGroupProcedure = new DeleteStorageGroupProcedure();
- deleteStorageGroupProcedure.setDeleteSgSchema(new TDatabaseSchema("root.sg"));
+ DeleteDatabaseProcedure deleteDatabaseProcedure = new DeleteDatabaseProcedure();
+ deleteDatabaseProcedure.setDeleteSgSchema(new TDatabaseSchema("root.sg"));
UpdateProcedurePlan updateProcedurePlan0 = new UpdateProcedurePlan();
- updateProcedurePlan0.setProcedure(deleteStorageGroupProcedure);
+ updateProcedurePlan0.setProcedure(deleteDatabaseProcedure);
UpdateProcedurePlan updateProcedurePlan1 =
(UpdateProcedurePlan)
ConfigPhysicalPlan.Factory.create(updateProcedurePlan0.serializeToByteBuffer());
Procedure proc = updateProcedurePlan1.getProcedure();
- Assert.assertEquals(proc, deleteStorageGroupProcedure);
+ Assert.assertEquals(proc, deleteDatabaseProcedure);
// test procedure equals CreateRegionGroupsProcedure
TDataNodeLocation dataNodeLocation0 = new TDataNodeLocation();
@@ -797,11 +797,11 @@ public class ConfigPhysicalPlanSerDeTest {
@Test
public void UpdateProcedurePlanTest() throws IOException {
UpdateProcedurePlan req0 = new UpdateProcedurePlan();
- DeleteStorageGroupProcedure deleteStorageGroupProcedure = new DeleteStorageGroupProcedure();
+ DeleteDatabaseProcedure deleteDatabaseProcedure = new DeleteDatabaseProcedure();
TDatabaseSchema tDatabaseSchema = new TDatabaseSchema();
tDatabaseSchema.setName("root.sg");
- deleteStorageGroupProcedure.setDeleteSgSchema(tDatabaseSchema);
- req0.setProcedure(deleteStorageGroupProcedure);
+ deleteDatabaseProcedure.setDeleteSgSchema(tDatabaseSchema);
+ req0.setProcedure(deleteDatabaseProcedure);
UpdateProcedurePlan req1 =
(UpdateProcedurePlan) ConfigPhysicalPlan.Factory.create(req0.serializeToByteBuffer());
Assert.assertEquals(req0, req1);
diff --git a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteDatabaseProcedureTest.java
similarity index 81%
rename from confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java
rename to confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteDatabaseProcedureTest.java
index 612ac01c60..1d4be74c5f 100644
--- a/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteStorageGroupProcedureTest.java
+++ b/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/DeleteDatabaseProcedureTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.confignode.procedure.impl;
-import org.apache.iotdb.confignode.procedure.impl.schema.DeleteStorageGroupProcedure;
+import org.apache.iotdb.confignode.procedure.impl.schema.DeleteDatabaseProcedure;
import org.apache.iotdb.confignode.procedure.store.ProcedureFactory;
import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema;
import org.apache.iotdb.tsfile.utils.PublicBAOS;
@@ -32,23 +32,22 @@ import java.nio.ByteBuffer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
-public class DeleteStorageGroupProcedureTest {
+public class DeleteDatabaseProcedureTest {
@Test
public void serializeDeserializeTest() {
PublicBAOS byteArrayOutputStream = new PublicBAOS();
DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream);
- DeleteStorageGroupProcedure p1 =
- new DeleteStorageGroupProcedure(new TDatabaseSchema("root.sg"));
+ DeleteDatabaseProcedure p1 = new DeleteDatabaseProcedure(new TDatabaseSchema("root.sg"));
try {
p1.serialize(outputStream);
ByteBuffer buffer =
ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size());
- DeleteStorageGroupProcedure p2 =
- (DeleteStorageGroupProcedure) ProcedureFactory.getInstance().create(buffer);
+ DeleteDatabaseProcedure p2 =
+ (DeleteDatabaseProcedure) ProcedureFactory.getInstance().create(buffer);
assertEquals(p1, p2);
} catch (Exception e) {
diff --git a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
index 6a3921f2fc..89dfb35d65 100644
--- a/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
+++ b/docs/UserGuide/Monitor-Alert/Apache-IoTDB-ConfigNode-Dashboard.json
@@ -11,6 +11,12 @@
],
"__elements": {},
"__requires": [
+ {
+ "type": "panel",
+ "id": "bargauge",
+ "name": "Bar gauge",
+ "version": ""
+ },
{
"type": "grafana",
"id": "grafana",
@@ -73,448 +79,717 @@
"liveNow": false,
"panels": [
{
- "collapsed": true,
- "datasource": {
- "type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
+ "collapsed": false,
"gridPos": {
"h": 1,
"w": 24,
"x": 0,
"y": 0
},
- "id": 1,
- "panels": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "id": 57,
+ "panels": [],
+ "title": "Overview",
+ "type": "row"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The current status of cluster ConfigNodes",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
},
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "thresholds"
- },
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- },
- {
- "color": "red",
- "value": 80
- }
- ]
+ "custom": {
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ }
+ },
+ "mappings": []
+ },
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"ReadOnly\", type=\"ConfigNode\"}"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
}
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Removing\", type=\"ConfigNode\"}"
},
- "overrides": []
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
},
- "gridPos": {
- "h": 7,
- "w": 9,
- "x": 0,
- "y": 1
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Unknown\", type=\"ConfigNode\"}"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
},
- "id": 2,
- "options": {
- "colorMode": "value",
- "graphMode": "area",
- "justifyMode": "auto",
- "orientation": "auto",
- "reduceOptions": {
- "calcs": [
- "lastNotNull"
- ],
- "fields": "",
- "values": false
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "{__name__=\"node_num\", instance=\"localhost:9091\", job=\"confignode\", status=\"Running\", type=\"ConfigNode\"}"
},
- "textMode": "auto"
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
},
- "pluginVersion": "9.3.6",
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "expr": "config_node{instance=~\"$instance\", status=\"Registered\"}",
- "hide": false,
- "legendFormat": "ConfigNode",
- "range": true,
- "refId": "A"
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
},
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "expr": "data_node{instance=~\"$instance\", status=\"Registered\"}",
- "hide": false,
- "legendFormat": "DataNode",
- "range": true,
- "refId": "B"
- }
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 0,
+ "y": 1
+ },
+ "id": 59,
+ "options": {
+ "legend": {
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "values": [
+ "value"
+ ]
+ },
+ "pieType": "pie",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
],
- "title": "Registered Node",
- "type": "stat"
+ "fields": "",
+ "values": false
},
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
+ },
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- }
- },
- "mappings": []
- },
- "overrides": []
- },
- "gridPos": {
- "h": 7,
- "w": 3,
- "x": 9,
- "y": 1
+ "editorMode": "builder",
+ "expr": "node_num{type=\"ConfigNode\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "ConfigNode Current Status",
+ "type": "piechart"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The CPU Core of current ConfigNode",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
},
- "id": 3,
- "options": {
- "legend": {
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
- },
- "pieType": "pie",
- "reduceOptions": {
- "calcs": [
- "lastNotNull"
- ],
- "fields": "",
- "values": false
- },
- "tooltip": {
- "mode": "single",
- "sort": "none"
- }
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
},
- "pluginVersion": "9.2.0",
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "data_node{instance=~\"$instance\", status!=\"Registered\"}",
- "format": "time_series",
- "instant": false,
- "interval": "",
- "legendFormat": "{{status}}",
- "refId": "A"
- }
+ "unit": "Core(s)"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 5,
+ "y": 1
+ },
+ "id": 69,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "none",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
],
- "title": "DataNode",
- "type": "piechart"
+ "fields": "",
+ "values": false
},
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- }
- },
- "mappings": []
- },
- "overrides": []
- },
- "gridPos": {
- "h": 7,
- "w": 3,
- "x": 12,
- "y": 1
- },
- "id": 4,
- "options": {
- "legend": {
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
- },
- "pieType": "pie",
- "reduceOptions": {
- "calcs": [
- "lastNotNull"
- ],
- "fields": "",
- "values": false
- },
- "tooltip": {
- "mode": "single",
- "sort": "none"
- }
- },
- "pluginVersion": "9.2.0",
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "config_node{instance=~\"$instance\", status!=\"Registered\"}",
- "format": "time_series",
- "instant": false,
- "interval": "",
- "legendFormat": "{{status}}",
- "refId": "A"
- }
- ],
- "title": "ConfigNode",
- "type": "piechart"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "axisSoftMax": 2,
- "axisSoftMin": 0,
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
- },
- "showPoints": "never",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
- },
- "thresholdsStyle": {
- "mode": "off"
- }
- },
- "mappings": [
- {
- "options": {
- "0": {
- "index": 1,
- "text": "Unknown"
- },
- "1": {
- "index": 0,
- "text": "Online"
- }
- },
- "type": "value"
- }
- ],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "bool_on_off"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 7,
- "w": 9,
- "x": 15,
- "y": 1
+ "editorMode": "code",
+ "exemplar": false,
+ "expr": "sys_cpu_cores{instance=\"$instance\", name=\"system\"}",
+ "instant": false,
+ "interval": "",
+ "legendFormat": "{{name}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "CPU Core",
+ "type": "stat"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The total disk space of current ConfigNode",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
},
- "id": 5,
- "options": {
- "legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
- },
- "tooltip": {
- "mode": "multi",
- "sort": "desc"
- }
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
},
- "pluginVersion": "8.4.2",
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "cluster_node_status{instance=~\"${instance}\"}",
- "legendFormat": "{{type}} {{name}}",
- "range": true,
- "refId": "A"
- }
+ "unit": "bytes"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 10,
+ "y": 1
+ },
+ "id": 77,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "none",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
],
- "title": "The Status Of Node",
- "type": "timeseries"
- }
- ],
+ "fields": "",
+ "values": false
+ },
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
"targets": [
{
"datasource": {
"type": "prometheus",
- "uid": "lfuOmw1Vk"
+ "uid": "${DS_PROMETHEUS}"
},
+ "editorMode": "builder",
+ "expr": "sys_disk_total_space",
+ "legendFormat": "{{name}}",
+ "range": true,
"refId": "A"
}
],
- "title": "Overview",
- "type": "row"
+ "title": "Total Disk Space",
+ "type": "stat"
},
{
- "collapsed": true,
"datasource": {
"type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "gridPos": {
- "h": 1,
- "w": 24,
- "x": 0,
- "y": 1
+ "uid": "${DS_PROMETHEUS}"
},
- "id": 6,
- "panels": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "description": "Bar of current ConfigNode",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
},
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "thresholds"
+ "mappings": [],
+ "max": 1,
+ "min": 0,
+ "thresholds": {
+ "mode": "percentage",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
+ {
+ "color": "#EAB839",
+ "value": 80
+ },
+ {
+ "color": "red",
+ "value": 90
}
- },
- "overrides": []
- },
- "gridPos": {
- "h": 7,
- "w": 12,
- "x": 0,
- "y": 2
- },
- "id": 7,
- "options": {
- "colorMode": "value",
- "graphMode": "area",
- "justifyMode": "auto",
- "orientation": "auto",
- "reduceOptions": {
- "calcs": [
- "lastNotNull"
- ],
- "fields": "",
- "values": false
- },
- "textMode": "auto"
+ ]
},
- "pluginVersion": "9.3.6",
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "sum(region{instance=~\"$instance\", name=\"total\"})",
- "hide": false,
- "interval": "",
- "legendFormat": "Total Region",
- "range": true,
- "refId": "C"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "region{instance=~\"$instance\", name=\"total\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "{{type}}",
- "range": true,
- "refId": "D"
- }
+ "unit": "percentunit"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 12,
+ "w": 9,
+ "x": 15,
+ "y": 1
+ },
+ "id": 73,
+ "options": {
+ "displayMode": "lcd",
+ "minVizHeight": 10,
+ "minVizWidth": 0,
+ "orientation": "horizontal",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
],
- "title": "Region Number",
- "type": "stat"
+ "fields": "",
+ "values": false
},
+ "showUnfilled": true
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "editorMode": "code",
+ "expr": "process_cpu_load{instance=~\"$instance\"} / 100",
+ "hide": false,
+ "legendFormat": "cpu load",
+ "range": true,
+ "refId": "C"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "1- sys_disk_free_space / sys_disk_total_space",
+ "hide": false,
+ "legendFormat": "disk",
+ "range": true,
+ "refId": "D"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "1 - process_free_mem{instance=\"$instance\"} / process_total_mem{instance=\"$instance\"}",
+ "hide": false,
+ "legendFormat": "process memory",
+ "range": true,
+ "refId": "B"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": false,
+ "expr": "1 - sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"} / sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+ "instant": false,
+ "interval": "",
+ "legendFormat": "system memory",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "type": "bargauge"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The current status of cluster DataNodes",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ }
+ },
+ "mappings": []
+ },
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 0,
+ "y": 7
+ },
+ "id": 65,
+ "options": {
+ "legend": {
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "values": [
+ "value"
+ ]
+ },
+ "pieType": "pie",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "builder",
+ "expr": "node_num{type=\"DataNode\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "DataNode Current Status",
+ "type": "piechart"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The system memory of current ConfigNode",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "bytes"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 5,
+ "y": 7
+ },
+ "id": 71,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "none",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+ "legendFormat": "{{name}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "System Memory",
+ "type": "stat"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The swap memory of current ConfigNode",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "bytes"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 5,
+ "x": 10,
+ "y": 7
+ },
+ "id": 75,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "none",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "text": {},
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "builder",
+ "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
+ "legendFormat": "{{name}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "Swap Memory",
+ "type": "stat"
+ },
+ {
+ "collapsed": true,
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 13
+ },
+ "id": 49,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The number of cluster ConfigNodes",
"fieldConfig": {
"defaults": {
"color": {
@@ -534,12 +809,12 @@
"overrides": []
},
"gridPos": {
- "h": 7,
- "w": 12,
- "x": 12,
- "y": 2
+ "h": 6,
+ "w": 6,
+ "x": 0,
+ "y": 14
},
- "id": 8,
+ "id": 55,
"options": {
"colorMode": "value",
"graphMode": "area",
@@ -561,16 +836,14 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "max(cluster_node_leader_count{}) by (name) ",
- "interval": "",
- "legendFormat": "Total leaders on {{name}} ",
+ "editorMode": "builder",
+ "expr": "sum by(type) (node_num{type=\"ConfigNode\"})",
+ "legendFormat": "__auto",
"range": true,
"refId": "A"
}
],
- "title": "Leadership Distribution",
+ "title": "ConfigNode Count",
"type": "stat"
},
{
@@ -578,6 +851,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The status history of cluster ConfigNodes",
"fieldConfig": {
"defaults": {
"color": {
@@ -590,7 +864,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -624,48 +898,188 @@
]
}
},
- "overrides": []
- },
- "gridPos": {
- "h": 7,
- "w": 12,
- "x": 0,
- "y": 9
- },
- "id": 9,
- "options": {
- "legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
- },
- "tooltip": {
- "mode": "multi",
- "sort": "desc"
- }
- },
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
},
- "exemplar": true,
- "expr": "sum(region{instance=~\"${instance}\", name=~\"EndPoint.*\"}) by (name)",
- "interval": "",
- "legendFormat": "{{name}}",
- "refId": "A"
- }
- ],
- "title": "Total Region in Node",
- "type": "timeseries"
- },
- {
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 18,
+ "x": 6,
+ "y": 14
+ },
+ "id": 61,
+ "options": {
+ "legend": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "builder",
+ "expr": "node_num{type=\"ConfigNode\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "ConfigNode Status History",
+ "type": "timeseries"
+ }
+ ],
+ "title": "ConfigNode",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 14
+ },
+ "id": 51,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The number of cluster DataNodes",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 6,
+ "x": 0,
+ "y": 15
+ },
+ "id": 63,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "area",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "builder",
+ "expr": "sum by(type) (node_num{type=\"DataNode\"})",
+ "legendFormat": "__auto",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "DataNode Count",
+ "type": "stat"
+ },
+ {
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The status history of cluster DataNodes",
"fieldConfig": {
"defaults": {
"color": {
@@ -678,7 +1092,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -712,25 +1126,88 @@
]
}
},
- "overrides": []
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
},
"gridPos": {
- "h": 7,
- "w": 12,
- "x": 12,
- "y": 9
+ "h": 6,
+ "w": 18,
+ "x": 6,
+ "y": 15
},
- "id": 10,
+ "id": 67,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -739,14 +1216,14 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "region{instance=~\"${instance}\", name=~\"EndPoint.*\"}",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
+ "editorMode": "builder",
+ "expr": "node_num{type=\"DataNode\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
"refId": "A"
}
],
- "title": "Region in Node",
+ "title": "DataNode Status History",
"type": "timeseries"
},
{
@@ -754,6 +1231,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of SchemaRegions in each DataNode",
"fieldConfig": {
"defaults": {
"color": {
@@ -766,7 +1244,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -803,17 +1281,19 @@
"overrides": []
},
"gridPos": {
- "h": 7,
+ "h": 6,
"w": 12,
"x": 0,
- "y": 16
+ "y": 21
},
- "id": 11,
+ "id": 97,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
@@ -827,14 +1307,14 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "region{instance=~\"${instance}\", name!=\"total\", name!~\"EndPoint.*\"}",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
+ "editorMode": "builder",
+ "expr": "region_num_in_data_node{type=\"SchemaRegion\"}",
+ "legendFormat": "{{name}}",
+ "range": true,
"refId": "A"
}
],
- "title": "Region in Database",
+ "title": "SchemaRegion Distribution",
"type": "timeseries"
},
{
@@ -842,6 +1322,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of SchemaRegionGroup leadere in each DataNodes",
"fieldConfig": {
"defaults": {
"color": {
@@ -854,7 +1335,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -891,22 +1372,24 @@
"overrides": []
},
"gridPos": {
- "h": 7,
+ "h": 6,
"w": 12,
"x": 12,
- "y": 16
+ "y": 21
},
- "id": 12,
+ "id": 95,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -915,52 +1398,56 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "slot{instance=~\"$instance\"}",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
+ "editorMode": "builder",
+ "expr": "region_group_leader_num_in_data_node{type=\"SchemaRegion\"}",
+ "legendFormat": "{{name}}",
+ "range": true,
"refId": "A"
}
],
- "title": "Slot In Database",
+ "title": "SchemaRegionGroup Leader Distribution",
"type": "timeseries"
- }
- ],
- "targets": [
+ },
{
"datasource": {
"type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "refId": "A"
- }
- ],
- "title": "Region",
- "type": "row"
- },
- {
- "collapsed": true,
- "datasource": {
- "type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "gridPos": {
- "h": 1,
- "w": 24,
- "x": 0,
- "y": 2
- },
- "id": 13,
- "panels": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of DataRegions in each DataNode",
"fieldConfig": {
"defaults": {
"color": {
- "mode": "thresholds"
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
},
"mappings": [],
"thresholds": {
@@ -969,61 +1456,55 @@
{
"color": "green",
"value": null
- },
- {
- "color": "red",
- "value": 80
}
]
- },
- "unit": "Core"
+ }
},
"overrides": []
},
"gridPos": {
- "h": 8,
- "w": 8,
+ "h": 6,
+ "w": 12,
"x": 0,
- "y": 3
+ "y": 27
},
- "id": 14,
+ "id": 99,
"options": {
- "colorMode": "value",
- "graphMode": "area",
- "justifyMode": "auto",
- "orientation": "auto",
- "reduceOptions": {
+ "legend": {
"calcs": [
"lastNotNull"
],
- "fields": "",
- "values": false
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
},
- "textMode": "auto"
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
},
- "pluginVersion": "9.3.6",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "expr": "sys_cpu_cores{instance=\"$instance\",name=\"system\"}",
- "hide": false,
- "legendFormat": "Process CPU Time",
+ "editorMode": "builder",
+ "expr": "region_num_in_data_node{type=\"DataRegion\"}",
+ "legendFormat": "{{name}}",
"range": true,
- "refId": "B"
+ "refId": "A"
}
],
- "title": "CPU Core",
- "type": "stat"
+ "title": "DataRegion Distribution",
+ "type": "timeseries"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of DataRegionGroup leaders in each DataNode",
"fieldConfig": {
"defaults": {
"color": {
@@ -1034,8 +1515,6 @@
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
- "axisSoftMax": 0,
- "axisSoftMin": 0,
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 10,
@@ -1063,35 +1542,36 @@
},
"mappings": [],
"thresholds": {
- "mode": "percentage",
+ "mode": "absolute",
"steps": [
{
"color": "green",
"value": null
}
]
- },
- "unit": "percent"
+ }
},
"overrides": []
},
"gridPos": {
- "h": 8,
- "w": 8,
- "x": 8,
- "y": 3
+ "h": 6,
+ "w": 12,
+ "x": 12,
+ "y": 27
},
- "id": 15,
+ "id": 101,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -1100,35 +1580,98 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
- "interval": "",
- "legendFormat": "System CPU Load",
+ "editorMode": "builder",
+ "expr": "region_num_in_data_node{type=\"DataRegion\"}",
+ "legendFormat": "{{name}}",
"range": true,
"refId": "A"
+ }
+ ],
+ "title": "DataRegionGroup Leader Distribution",
+ "type": "timeseries"
+ }
+ ],
+ "title": "DataNode",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 15
+ },
+ "id": 53,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The number of cluster Databases",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 6,
+ "x": 0,
+ "y": 16
+ },
+ "id": 79,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "area",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
},
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
- "hide": false,
- "legendFormat": "Process CPU Load",
+ "editorMode": "builder",
+ "expr": "database_num",
+ "legendFormat": "__auto",
"range": true,
- "refId": "B"
+ "refId": "A"
}
],
- "title": "CPU Load",
- "type": "timeseries"
+ "title": "Database Count",
+ "type": "stat"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of SeriesSlots in each Database",
"fieldConfig": {
"defaults": {
"color": {
@@ -1139,8 +1682,6 @@
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
- "axisSoftMax": 0,
- "axisSoftMin": 0,
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 10,
@@ -1168,35 +1709,36 @@
},
"mappings": [],
"thresholds": {
- "mode": "percentage",
+ "mode": "absolute",
"steps": [
{
"color": "green",
"value": null
}
]
- },
- "unit": "ns"
+ }
},
"overrides": []
},
"gridPos": {
- "h": 8,
- "w": 8,
- "x": 16,
- "y": 3
+ "h": 6,
+ "w": 18,
+ "x": 6,
+ "y": 16
},
- "id": 16,
+ "id": 103,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -1205,15 +1747,14 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
- "hide": false,
- "legendFormat": "Process CPU Time",
+ "editorMode": "builder",
+ "expr": "series_slot_num_in_database",
+ "legendFormat": "{{name}}",
"range": true,
- "refId": "B"
+ "refId": "A"
}
],
- "title": "CPU Time Per Minute",
+ "title": "SeriesSlot Count",
"type": "timeseries"
},
{
@@ -1221,6 +1762,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of SchemaRegionGroups in each Database",
"fieldConfig": {
"defaults": {
"color": {
@@ -1233,7 +1775,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -1265,28 +1807,29 @@
"value": null
}
]
- },
- "unit": "bytes"
+ }
},
"overrides": []
},
"gridPos": {
- "h": 8,
- "w": 8,
+ "h": 6,
+ "w": 12,
"x": 0,
- "y": 11
+ "y": 22
},
- "id": 17,
+ "id": 105,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -1295,42 +1838,14 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}-sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"}",
- "interval": "",
- "legendFormat": "Used physical memory",
+ "editorMode": "builder",
+ "expr": "region_group_num_in_database{type=\"SchemaRegion\"}",
+ "legendFormat": "{{name}}",
"range": true,
"refId": "A"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "exemplar": true,
- "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "Total physical memory",
- "range": true,
- "refId": "B"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "expr": "sys_committed_vm_size{instance=\"$instance\"}",
- "hide": false,
- "legendFormat": "Committed vm size",
- "range": true,
- "refId": "C"
}
],
- "title": "System Memory",
+ "title": "SchemaRegionGroup Count",
"type": "timeseries"
},
{
@@ -1338,6 +1853,7 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of DataRegionGroups in each DataNode",
"fieldConfig": {
"defaults": {
"color": {
@@ -1350,7 +1866,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -1382,14 +1898,148 @@
"value": null
}
]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 12,
+ "x": 12,
+ "y": 22
+ },
+ "id": 107,
+ "options": {
+ "legend": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
},
- "unit": "bytes"
+ "editorMode": "builder",
+ "expr": "region_group_num_in_database{type=\"DataRegion\"}",
+ "legendFormat": "{{name}}",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "DataRegionGroup Count",
+ "type": "timeseries"
+ }
+ ],
+ "title": "Database",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 16
+ },
+ "id": 81,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The number of cluster SchemaRegions",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 6,
+ "x": 0,
+ "y": 17
+ },
+ "id": 83,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "area",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
+ },
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "builder",
+ "expr": "sum by(type) (region_num{type=\"SchemaRegion\"})",
+ "legendFormat": "__auto",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "SchemaRegion Count",
+ "type": "stat"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The current status of cluster SchemaRegions",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ }
+ },
+ "mappings": []
},
"overrides": [
{
"matcher": {
"id": "byName",
- "options": "Total Swap Size"
+ "options": "Running"
},
"properties": [
{
@@ -1404,7 +2054,37 @@
{
"matcher": {
"id": "byName",
- "options": "Used Swap Size"
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
},
"properties": [
{
@@ -1419,22 +2099,32 @@
]
},
"gridPos": {
- "h": 8,
- "w": 8,
- "x": 8,
- "y": 11
+ "h": 6,
+ "w": 6,
+ "x": 6,
+ "y": 17
},
- "id": 18,
+ "id": 87,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "values": [
+ "value"
+ ]
+ },
+ "pieType": "pie",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -1443,36 +2133,22 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "Used Swap Size",
- "range": true,
- "refId": "B"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
- "hide": false,
- "legendFormat": "Total Swap Size",
+ "editorMode": "builder",
+ "expr": "region_num{type=\"SchemaRegion\"}",
+ "legendFormat": "{{status}}",
"range": true,
- "refId": "C"
+ "refId": "A"
}
],
- "title": "System Swap Size",
- "type": "timeseries"
+ "title": "SchemaRegion Current Status",
+ "type": "piechart"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The status history of cluster SchemaRegions",
"fieldConfig": {
"defaults": {
"color": {
@@ -1485,7 +2161,7 @@
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -1521,68 +2197,106 @@
"value": 80
}
]
- },
- "unit": "bytes"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 8,
- "x": 16,
- "y": 11
- },
- "id": 19,
- "options": {
- "legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
- },
- "tooltip": {
- "mode": "multi",
- "sort": "desc"
- }
- },
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "editorMode": "code",
- "expr": "process_max_mem{instance=~\"${instance}\"}",
- "legendFormat": "Max Memory",
- "range": true,
- "refId": "A"
+ }
},
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
},
- "editorMode": "code",
- "expr": "process_total_mem{instance=~\"${instance}\"}",
- "hide": false,
- "legendFormat": "Total Memory",
- "range": true,
- "refId": "B"
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 12,
+ "x": 12,
+ "y": 17
+ },
+ "id": 89,
+ "options": {
+ "legend": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
},
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
+ },
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "expr": "process_total_mem{instance=~\"${instance}\"} - process_free_mem{instance=~\"${instance}\"}",
- "hide": false,
- "legendFormat": "Used Memory",
+ "editorMode": "builder",
+ "expr": "region_num{type=\"SchemaRegion\"}",
+ "legendFormat": "{{status}}",
"range": true,
- "refId": "C"
+ "refId": "A"
}
],
- "title": "Process Memory",
+ "title": "SchemaRegion Status History",
"type": "timeseries"
},
{
@@ -1590,40 +2304,11 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The number of cluster DataRegions",
"fieldConfig": {
"defaults": {
"color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
- },
- "showPoints": "never",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
- },
- "thresholdsStyle": {
- "mode": "off"
- }
+ "mode": "thresholds"
},
"mappings": [],
"thresholds": {
@@ -1639,59 +2324,178 @@
"overrides": []
},
"gridPos": {
- "h": 8,
- "w": 8,
+ "h": 6,
+ "w": 6,
"x": 0,
- "y": 19
+ "y": 23
},
- "id": 20,
+ "id": 85,
"options": {
- "legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
- "showLegend": true
+ "colorMode": "value",
+ "graphMode": "area",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
},
- "tooltip": {
- "mode": "multi",
- "sort": "desc"
- }
+ "textMode": "auto"
},
+ "pluginVersion": "9.3.6",
"targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
- "interval": "",
- "legendFormat": "Young GC number",
+ "editorMode": "builder",
+ "expr": "sum by (type) (region_num{type=\"DataRegion\"})",
+ "legendFormat": "__auto",
"range": true,
"refId": "A"
+ }
+ ],
+ "title": "DataRegion Count",
+ "type": "stat"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "description": "The current status of cluster DataRegions",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ }
+ },
+ "mappings": []
+ },
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 6,
+ "w": 6,
+ "x": 6,
+ "y": 23
+ },
+ "id": 93,
+ "options": {
+ "legend": {
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "values": [
+ "value"
+ ]
+ },
+ "pieType": "pie",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
},
+ "tooltip": {
+ "mode": "single",
+ "sort": "none"
+ }
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
- "hide": false,
- "interval": "",
- "legendFormat": "Full GC number",
- "refId": "B"
+ "editorMode": "builder",
+ "expr": "region_num{type=\"DataRegion\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
+ "refId": "A"
}
],
- "title": "The Number Of GC Per Minute",
- "type": "timeseries"
+ "title": "DataRegion Current Status",
+ "type": "piechart"
},
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
+ "description": "The status history of cluster DataRegions",
"fieldConfig": {
"defaults": {
"color": {
@@ -1736,28 +2540,90 @@
"value": null
}
]
- },
- "unit": "s"
+ }
},
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 8,
- "x": 8,
- "y": 19
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "ReadOnly"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Removing"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Running"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "green",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "Unknown"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "yellow",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
},
- "id": 21,
+ "gridPos": {
+ "h": 6,
+ "w": 12,
+ "x": 12,
+ "y": 23
+ },
+ "id": 91,
"options": {
"legend": {
- "calcs": [],
- "displayMode": "list",
- "placement": "bottom",
+ "calcs": [
+ "lastNotNull"
+ ],
+ "displayMode": "table",
+ "placement": "right",
"showLegend": true
},
"tooltip": {
- "mode": "multi",
- "sort": "desc"
+ "mode": "single",
+ "sort": "none"
}
},
"targets": [
@@ -1766,27 +2632,100 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
- "interval": "",
- "legendFormat": "The time of young GC",
+ "editorMode": "builder",
+ "expr": "region_num{type=\"DataRegion\"}",
+ "legendFormat": "{{status}}",
+ "range": true,
"refId": "A"
+ }
+ ],
+ "title": "DataRegion Status History",
+ "type": "timeseries"
+ }
+ ],
+ "title": "Region",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": {
+ "type": "prometheus",
+ "uid": "lfuOmw1Vk"
+ },
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 17
+ },
+ "id": 13,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "thresholds"
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
+ }
+ ]
+ },
+ "unit": "Core"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 0,
+ "y": 18
+ },
+ "id": 14,
+ "options": {
+ "colorMode": "value",
+ "graphMode": "area",
+ "justifyMode": "auto",
+ "orientation": "auto",
+ "reduceOptions": {
+ "calcs": [
+ "lastNotNull"
+ ],
+ "fields": "",
+ "values": false
},
+ "textMode": "auto"
+ },
+ "pluginVersion": "9.3.6",
+ "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+ "editorMode": "code",
+ "expr": "sys_cpu_cores{instance=\"$instance\",name=\"system\"}",
"hide": false,
- "interval": "",
- "legendFormat": "The time of full GC",
+ "legendFormat": "Process CPU Time",
+ "range": true,
"refId": "B"
}
],
- "title": "The Time Consumed Of GC Per Minute",
- "type": "timeseries"
+ "title": "CPU Core",
+ "type": "stat"
},
{
"datasource": {
@@ -1803,9 +2742,11 @@
"axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
+ "axisSoftMax": 0,
+ "axisSoftMin": 0,
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 5,
+ "fillOpacity": 10,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -1830,24 +2771,25 @@
},
"mappings": [],
"thresholds": {
- "mode": "absolute",
+ "mode": "percentage",
"steps": [
{
"color": "green",
"value": null
}
]
- }
+ },
+ "unit": "percent"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 8,
- "x": 16,
- "y": 19
+ "x": 8,
+ "y": 18
},
- "id": 22,
+ "id": 15,
"options": {
"legend": {
"calcs": [],
@@ -1868,9 +2810,9 @@
},
"editorMode": "code",
"exemplar": true,
- "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
+ "expr": "sys_cpu_load{instance=\"$instance\",name=\"system\"}",
"interval": "",
- "legendFormat": "Total Number",
+ "legendFormat": "System CPU Load",
"range": true,
"refId": "A"
},
@@ -1879,15 +2821,15 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "jvm_threads_states_threads{instance=~\"$instance\"}",
+ "editorMode": "code",
+ "expr": "process_cpu_load{instance=\"$instance\",name=\"process\"}",
"hide": false,
- "interval": "",
- "legendFormat": "{{state}}",
+ "legendFormat": "Process CPU Load",
+ "range": true,
"refId": "B"
}
],
- "title": "The Number Of Java Thread",
+ "title": "CPU Load",
"type": "timeseries"
},
{
@@ -1901,8 +2843,12 @@
"mode": "palette-classic"
},
"custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
+ "axisSoftMax": 0,
+ "axisSoftMin": 0,
"barAlignment": 0,
"drawStyle": "line",
"fillOpacity": 10,
@@ -1930,24 +2876,25 @@
},
"mappings": [],
"thresholds": {
- "mode": "absolute",
+ "mode": "percentage",
"steps": [
{
- "color": "green"
+ "color": "green",
+ "value": null
}
]
},
- "unit": "bytes"
+ "unit": "ns"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 8,
- "x": 0,
- "y": 27
+ "x": 16,
+ "y": 18
},
- "id": 23,
+ "id": 16,
"options": {
"legend": {
"calcs": [],
@@ -1961,87 +2908,40 @@
}
},
"targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "exemplar": true,
- "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
- "interval": "",
- "legendFormat": "Maximum heap memory",
- "refId": "A"
- },
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
- "exemplar": true,
- "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+ "expr": "sum(rate(process_cpu_time{instance=\"$instance\",name=\"process\"}[1m]))*60",
"hide": false,
- "interval": "",
- "legendFormat": "Used heap memory",
+ "legendFormat": "Process CPU Time",
"range": true,
"refId": "B"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "exemplar": true,
- "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Old Gen\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "Old area",
- "refId": "C"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "exemplar": true,
- "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Eden Space\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "Eden area",
- "refId": "D"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "exemplar": true,
- "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Survivor Space\"}",
- "hide": false,
- "interval": "",
- "legendFormat": "Survivor area",
- "refId": "E"
- }
- ],
- "title": "Heap Memory",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
+ }
+ ],
+ "title": "CPU Time Per Minute",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
},
"custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 10,
+ "fillOpacity": 5,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -2069,7 +2969,8 @@
"mode": "absolute",
"steps": [
{
- "color": "green"
+ "color": "green",
+ "value": null
}
]
},
@@ -2080,10 +2981,10 @@
"gridPos": {
"h": 8,
"w": 8,
- "x": 8,
- "y": 27
+ "x": 0,
+ "y": 26
},
- "id": 24,
+ "id": 17,
"options": {
"legend": {
"calcs": [],
@@ -2104,14 +3005,40 @@
},
"editorMode": "code",
"exemplar": true,
- "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
+ "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}-sys_free_physical_memory_size{instance=\"$instance\", name=\"system\"}",
"interval": "",
- "legendFormat": "off-heap memory",
+ "legendFormat": "Used physical memory",
"range": true,
"refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "sys_total_physical_memory_size{instance=\"$instance\", name=\"system\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Total physical memory",
+ "range": true,
+ "refId": "B"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "sys_committed_vm_size{instance=\"$instance\"}",
+ "hide": false,
+ "legendFormat": "Committed vm size",
+ "range": true,
+ "refId": "C"
}
],
- "title": "Off Heap Memory",
+ "title": "System Memory",
"type": "timeseries"
},
{
@@ -2125,11 +3052,13 @@
"mode": "palette-classic"
},
"custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 10,
+ "fillOpacity": 5,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -2157,38 +3086,24 @@
"mode": "absolute",
"steps": [
{
- "color": "green"
+ "color": "green",
+ "value": null
}
]
},
- "unit": "none"
+ "unit": "bytes"
},
"overrides": [
{
"matcher": {
"id": "byName",
- "options": "warn"
- },
- "properties": [
- {
- "id": "color",
- "value": {
- "fixedColor": "orange",
- "mode": "fixed"
- }
- }
- ]
- },
- {
- "matcher": {
- "id": "byName",
- "options": "trace"
+ "options": "Total Swap Size"
},
"properties": [
{
"id": "color",
"value": {
- "fixedColor": "purple",
+ "fixedColor": "green",
"mode": "fixed"
}
}
@@ -2197,13 +3112,13 @@
{
"matcher": {
"id": "byName",
- "options": "error"
+ "options": "Used Swap Size"
},
"properties": [
{
"id": "color",
"value": {
- "fixedColor": "red",
+ "fixedColor": "yellow",
"mode": "fixed"
}
}
@@ -2214,10 +3129,10 @@
"gridPos": {
"h": 8,
"w": 8,
- "x": 16,
- "y": 27
+ "x": 8,
+ "y": 26
},
- "id": 25,
+ "id": 18,
"options": {
"legend": {
"calcs": [],
@@ -2238,15 +3153,27 @@
},
"editorMode": "code",
"exemplar": true,
- "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
+ "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"} - sys_free_swap_space_size{instance=\"$instance\", name=\"system\"}",
"hide": false,
"interval": "",
- "legendFormat": "{{level}}",
+ "legendFormat": "Used Swap Size",
+ "range": true,
+ "refId": "B"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "sys_total_swap_space_size{instance=\"$instance\", name=\"system\"}",
+ "hide": false,
+ "legendFormat": "Total Swap Size",
"range": true,
"refId": "C"
}
],
- "title": "Log Number Per Minute",
+ "title": "System Swap Size",
"type": "timeseries"
},
{
@@ -2254,18 +3181,19 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "description": "",
"fieldConfig": {
"defaults": {
"color": {
"mode": "palette-classic"
},
"custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 10,
+ "fillOpacity": 5,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -2293,21 +3221,26 @@
"mode": "absolute",
"steps": [
{
- "color": "green"
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "red",
+ "value": 80
}
]
},
- "unit": "ms"
+ "unit": "bytes"
},
"overrides": []
},
"gridPos": {
"h": 8,
"w": 8,
- "x": 0,
- "y": 35
+ "x": 16,
+ "y": 26
},
- "id": 26,
+ "id": 19,
"options": {
"legend": {
"calcs": [],
@@ -2327,15 +3260,37 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
- "exemplar": true,
- "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
- "interval": "",
- "legendFormat": "compiler",
+ "expr": "process_max_mem{instance=~\"${instance}\"}",
+ "legendFormat": "Max Memory",
"range": true,
"refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "process_total_mem{instance=~\"${instance}\"}",
+ "hide": false,
+ "legendFormat": "Total Memory",
+ "range": true,
+ "refId": "B"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "expr": "process_total_mem{instance=~\"${instance}\"} - process_free_mem{instance=~\"${instance}\"}",
+ "hide": false,
+ "legendFormat": "Used Memory",
+ "range": true,
+ "refId": "C"
}
],
- "title": "The Time Consumed of Compilation Per Minute",
+ "title": "Process Memory",
"type": "timeseries"
},
{
@@ -2349,11 +3304,13 @@
"mode": "palette-classic"
},
"custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
"axisLabel": "",
"axisPlacement": "auto",
"barAlignment": 0,
"drawStyle": "line",
- "fillOpacity": 10,
+ "fillOpacity": 5,
"gradientMode": "none",
"hideFrom": {
"legend": false,
@@ -2361,9 +3318,6 @@
"viz": false
},
"lineInterpolation": "linear",
- "lineStyle": {
- "fill": "solid"
- },
"lineWidth": 1,
"pointSize": 5,
"scaleDistribution": {
@@ -2384,7 +3338,8 @@
"mode": "absolute",
"steps": [
{
- "color": "green"
+ "color": "green",
+ "value": null
}
]
}
@@ -2394,10 +3349,10 @@
"gridPos": {
"h": 8,
"w": 8,
- "x": 8,
- "y": 35
+ "x": 0,
+ "y": 34
},
- "id": 27,
+ "id": 20,
"options": {
"legend": {
"calcs": [],
@@ -2418,9 +3373,9 @@
},
"editorMode": "code",
"exemplar": true,
- "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
+ "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
"interval": "",
- "legendFormat": "Unloaded Number",
+ "legendFormat": "Young GC number",
"range": true,
"refId": "A"
},
@@ -2429,908 +3384,1677 @@
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
"exemplar": true,
- "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+ "expr": "sum(rate(jvm_gc_pause_seconds_count{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
"hide": false,
"interval": "",
- "legendFormat": "Loaded Number",
- "range": true,
+ "legendFormat": "Full GC number",
"refId": "B"
}
],
- "title": "The Number Of Class",
+ "title": "The Number Of GC Per Minute",
"type": "timeseries"
- }
- ],
- "targets": [
- {
- "datasource": {
- "type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "refId": "A"
- }
- ],
- "title": "System",
- "type": "row"
- },
- {
- "collapsed": false,
- "datasource": {
- "type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "gridPos": {
- "h": 1,
- "w": 24,
- "x": 0,
- "y": 3
- },
- "id": 29,
- "panels": [],
- "targets": [
+ },
{
"datasource": {
"type": "prometheus",
- "uid": "lfuOmw1Vk"
- },
- "refId": "A"
- }
- ],
- "title": "Disk I/O",
- "type": "row"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
+ "uid": "${DS_PROMETHEUS}"
},
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "axisSoftMax": 1,
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
- },
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
- },
- "thresholdsStyle": {
- "mode": "line+area"
- }
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "s"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 8,
+ "y": 34
+ },
+ "id": 21,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of minor GC\"}[1m]))*60",
+ "interval": "",
+ "legendFormat": "The time of young GC",
+ "refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "sum(rate(jvm_gc_pause_seconds_sum{instance=~\"$instance\", action=\"end of major GC\"}[1m]))*60",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "The time of full GC",
+ "refId": "B"
+ }
+ ],
+ "title": "The Time Consumed Of GC Per Minute",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 16,
+ "y": 34
+ },
+ "id": 22,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "jvm_threads_live_threads{instance=~\"$instance\"}",
+ "interval": "",
+ "legendFormat": "Total Number",
+ "range": true,
+ "refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "jvm_threads_states_threads{instance=~\"$instance\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "{{state}}",
+ "refId": "B"
+ }
+ ],
+ "title": "The Number Of Java Thread",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "bytes"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 0,
+ "y": 42
+ },
+ "id": 23,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "sum(jvm_memory_max_bytes{instance=~\"$instance\",area=\"heap\"})",
+ "interval": "",
+ "legendFormat": "Maximum heap memory",
+ "refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "sum(jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\"})",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Used heap memory",
+ "range": true,
+ "refId": "B"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Old Gen\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Old area",
+ "refId": "C"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Eden Space\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Eden area",
+ "refId": "D"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "jvm_memory_used_bytes{instance=~\"$instance\",area=\"heap\",id=\"PS Survivor Space\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Survivor area",
+ "refId": "E"
+ }
+ ],
+ "title": "Heap Memory",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "bytes"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 8,
+ "y": 42
+ },
+ "id": 24,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "sum(jvm_buffer_memory_used_bytes{instance=~\"$instance\"})",
+ "interval": "",
+ "legendFormat": "off-heap memory",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "Off Heap Memory",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "none"
+ },
+ "overrides": [
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "warn"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "orange",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "trace"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "purple",
+ "mode": "fixed"
+ }
+ }
+ ]
+ },
+ {
+ "matcher": {
+ "id": "byName",
+ "options": "error"
+ },
+ "properties": [
+ {
+ "id": "color",
+ "value": {
+ "fixedColor": "red",
+ "mode": "fixed"
+ }
+ }
+ ]
+ }
+ ]
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 16,
+ "y": 42
+ },
+ "id": 25,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "sum(rate(logback_events_total{instance=~\"${instance}\"}[1m])) by (level) * 60",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "{{level}}",
+ "range": true,
+ "refId": "C"
+ }
+ ],
+ "title": "Log Number Per Minute",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
},
- "mappings": [],
- "max": 1,
- "thresholds": {
- "mode": "percentage",
- "steps": [
- {
- "color": "green",
- "value": null
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "ms"
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 0,
+ "y": 50
+ },
+ "id": 26,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "sum(rate(jvm_compilation_time_ms{instance=~\"${instance}\"}[1m]))*60",
+ "interval": "",
+ "legendFormat": "compiler",
+ "range": true,
+ "refId": "A"
+ }
+ ],
+ "title": "The Time Consumed of Compilation Per Minute",
+ "type": "timeseries"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 10,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineStyle": {
+ "fill": "solid"
+ },
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "never",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
+ },
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 8,
+ "y": 50
+ },
+ "id": 27,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "list",
+ "placement": "bottom",
+ "showLegend": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "desc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "jvm_classes_unloaded_classes{instance=~\"$instance\"}",
+ "interval": "",
+ "legendFormat": "Unloaded Number",
+ "range": true,
+ "refId": "A"
+ },
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "jvm_classes_loaded_classes{instance=~\"$instance\"}",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "Loaded Number",
+ "range": true,
+ "refId": "B"
+ }
+ ],
+ "title": "The Number Of Class",
+ "type": "timeseries"
+ }
+ ],
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "lfuOmw1Vk"
+ },
+ "refId": "A"
+ }
+ ],
+ "title": "System",
+ "type": "row"
+ },
+ {
+ "collapsed": true,
+ "datasource": {
+ "type": "prometheus",
+ "uid": "lfuOmw1Vk"
+ },
+ "gridPos": {
+ "h": 1,
+ "w": 24,
+ "x": 0,
+ "y": 18
+ },
+ "id": 29,
+ "panels": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "axisSoftMax": 1,
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "line+area"
+ }
},
- {
- "color": "#EAB839",
- "value": 80
+ "mappings": [],
+ "max": 1,
+ "thresholds": {
+ "mode": "percentage",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ },
+ {
+ "color": "#EAB839",
+ "value": 80
+ },
+ {
+ "color": "dark-red",
+ "value": 90
+ }
+ ]
},
- {
- "color": "dark-red",
- "value": 90
- }
- ]
+ "unit": "percentunit"
+ },
+ "overrides": []
},
- "unit": "percentunit"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 0,
- "y": 4
- },
- "id": 31,
- "options": {
- "legend": {
- "calcs": [
- "mean"
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 59
+ },
+ "id": 31,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Mean",
+ "sortDesc": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "disk_io_busy_percentage{instance=~\"$instance\"}",
+ "instant": false,
+ "interval": "",
+ "legendFormat": "{{name}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Mean",
- "sortDesc": true
+ "title": "Disk I/O Busy Rate",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "disk_io_busy_percentage{instance=~\"$instance\"}",
- "instant": false,
- "interval": "",
- "legendFormat": "{{name}}",
- "refId": "A"
- }
- ],
- "title": "Disk I/O Busy Rate",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "KBs"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 59
+ },
+ "id": 33,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean",
+ "last"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Last",
+ "sortDesc": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "KBs"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 12,
- "y": 4
- },
- "id": 33,
- "options": {
- "legend": {
- "calcs": [
- "mean",
- "last"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "rate(disk_io_size{instance=~\"$instance\"}[1m])",
+ "hide": false,
+ "interval": "",
+ "legendFormat": "{{name}}-{{type}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Last",
- "sortDesc": true
+ "title": "Disk I/O Throughput",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "rate(disk_io_size{instance=~\"$instance\"}[1m])",
- "hide": false,
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
- "refId": "A"
- }
- ],
- "title": "Disk I/O Throughput",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "iops"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 67
+ },
+ "id": 35,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean",
+ "max"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Max",
+ "sortDesc": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "iops"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 0,
- "y": 12
- },
- "id": 35,
- "options": {
- "legend": {
- "calcs": [
- "mean",
- "max"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "rate(disk_io_ops{instance=~\"$instance\"}[1m])",
+ "interval": "",
+ "legendFormat": "{{name}}-{{type}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Max",
- "sortDesc": true
+ "title": "Disk I/O Ops",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "rate(disk_io_ops{instance=~\"$instance\"}[1m])",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
- "refId": "A"
- }
- ],
- "title": "Disk I/O Ops",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "ms"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 67
+ },
+ "id": 37,
+ "options": {
+ "legend": {
+ "calcs": [],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "ms"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 12,
- "y": 12
- },
- "id": 37,
- "options": {
- "legend": {
- "calcs": [],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "rate(disk_io_time{instance=~\"$instance\"}[1m])",
+ "interval": "",
+ "legendFormat": "{{name}}-{{type}}",
+ "refId": "A"
+ }
+ ],
+ "title": "Disk I/O Time",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "rate(disk_io_time{instance=~\"$instance\"}[1m])",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
- "refId": "A"
- }
- ],
- "title": "Disk I/O Time",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "description": "",
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
- },
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "description": "",
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "ms"
},
- "thresholdsStyle": {
- "mode": "off"
- }
+ "overrides": []
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 0,
+ "y": 75
},
- "unit": "ms"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 8,
- "x": 0,
- "y": 20
- },
- "id": 39,
- "options": {
- "legend": {
- "calcs": [
- "mean",
- "max"
+ "id": 39,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean",
+ "max"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Max",
+ "sortDesc": true
+ },
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
+ }
+ },
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "disk_io_avg_time{instance=~\"$instance\"}",
+ "interval": "",
+ "legendFormat": "{{name}}-{{type}}",
+ "range": true,
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Max",
- "sortDesc": true
+ "title": "Disk Per I/O Avg Time",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "disk_io_avg_time{instance=~\"$instance\"}",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
- "range": true,
- "refId": "A"
- }
- ],
- "title": "Disk Per I/O Avg Time",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "bytes"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 8,
+ "y": 75
+ },
+ "id": 41,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Mean",
+ "sortDesc": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "bytes"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 8,
- "x": 8,
- "y": 20
- },
- "id": 41,
- "options": {
- "legend": {
- "calcs": [
- "mean"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "disk_io_avg_size{instance=~\"$instance\"}",
+ "interval": "",
+ "legendFormat": "{{name}}-{{type}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Mean",
- "sortDesc": true
+ "title": "Disk I/O Avg Size",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "disk_io_avg_size{instance=~\"$instance\"}",
- "interval": "",
- "legendFormat": "{{name}}-{{type}}",
- "refId": "A"
- }
- ],
- "title": "Disk I/O Avg Size",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ }
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 8,
+ "x": 16,
+ "y": 75
+ },
+ "id": 43,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Mean",
+ "sortDesc": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- }
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 8,
- "x": 16,
- "y": 20
- },
- "id": 43,
- "options": {
- "legend": {
- "calcs": [
- "mean"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "editorMode": "code",
+ "exemplar": true,
+ "expr": "disk_io_avg_queue_size{instance=~\"$instance\"}",
+ "interval": "",
+ "legendFormat": "{{name}}",
+ "range": true,
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Mean",
- "sortDesc": true
+ "title": "Disk I/O Avg Queue Size",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "editorMode": "code",
- "exemplar": true,
- "expr": "disk_io_avg_queue_size{instance=~\"$instance\"}",
- "interval": "",
- "legendFormat": "{{name}}",
- "range": true,
- "refId": "A"
- }
- ],
- "title": "Disk I/O Avg Queue Size",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "KBs"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 0,
+ "y": 83
+ },
+ "id": 47,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "KBs"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 0,
- "y": 28
- },
- "id": 47,
- "options": {
- "legend": {
- "calcs": [
- "mean"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "rate(process_io_size{instance=~\"$instance\"}[1m])",
+ "interval": "",
+ "legendFormat": "{{from}}-{{name}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true
+ "title": "Process I/O Throughput",
+ "type": "timeseries"
},
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
- }
- },
- "targets": [
{
"datasource": {
"type": "prometheus",
"uid": "${DS_PROMETHEUS}"
},
- "exemplar": true,
- "expr": "rate(process_io_size{instance=~\"$instance\"}[1m])",
- "interval": "",
- "legendFormat": "{{from}}-{{name}}",
- "refId": "A"
- }
- ],
- "title": "Process I/O Throughput",
- "type": "timeseries"
- },
- {
- "datasource": {
- "type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
- },
- "fieldConfig": {
- "defaults": {
- "color": {
- "mode": "palette-classic"
- },
- "custom": {
- "axisCenteredZero": false,
- "axisColorMode": "text",
- "axisLabel": "",
- "axisPlacement": "auto",
- "barAlignment": 0,
- "drawStyle": "line",
- "fillOpacity": 5,
- "gradientMode": "none",
- "hideFrom": {
- "legend": false,
- "tooltip": false,
- "viz": false
- },
- "lineInterpolation": "linear",
- "lineWidth": 1,
- "pointSize": 5,
- "scaleDistribution": {
- "type": "linear"
+ "fieldConfig": {
+ "defaults": {
+ "color": {
+ "mode": "palette-classic"
+ },
+ "custom": {
+ "axisCenteredZero": false,
+ "axisColorMode": "text",
+ "axisLabel": "",
+ "axisPlacement": "auto",
+ "barAlignment": 0,
+ "drawStyle": "line",
+ "fillOpacity": 5,
+ "gradientMode": "none",
+ "hideFrom": {
+ "legend": false,
+ "tooltip": false,
+ "viz": false
+ },
+ "lineInterpolation": "linear",
+ "lineWidth": 1,
+ "pointSize": 5,
+ "scaleDistribution": {
+ "type": "linear"
+ },
+ "showPoints": "auto",
+ "spanNulls": false,
+ "stacking": {
+ "group": "A",
+ "mode": "none"
+ },
+ "thresholdsStyle": {
+ "mode": "off"
+ }
+ },
+ "mappings": [],
+ "thresholds": {
+ "mode": "absolute",
+ "steps": [
+ {
+ "color": "green",
+ "value": null
+ }
+ ]
+ },
+ "unit": "ops"
},
- "showPoints": "auto",
- "spanNulls": false,
- "stacking": {
- "group": "A",
- "mode": "none"
+ "overrides": []
+ },
+ "gridPos": {
+ "h": 8,
+ "w": 12,
+ "x": 12,
+ "y": 83
+ },
+ "id": 45,
+ "options": {
+ "legend": {
+ "calcs": [
+ "mean"
+ ],
+ "displayMode": "table",
+ "placement": "right",
+ "showLegend": true,
+ "sortBy": "Mean",
+ "sortDesc": true
},
- "thresholdsStyle": {
- "mode": "off"
+ "tooltip": {
+ "mode": "multi",
+ "sort": "asc"
}
},
- "mappings": [],
- "thresholds": {
- "mode": "absolute",
- "steps": [
- {
- "color": "green",
- "value": null
- }
- ]
- },
- "unit": "ops"
- },
- "overrides": []
- },
- "gridPos": {
- "h": 8,
- "w": 12,
- "x": 12,
- "y": 28
- },
- "id": 45,
- "options": {
- "legend": {
- "calcs": [
- "mean"
+ "targets": [
+ {
+ "datasource": {
+ "type": "prometheus",
+ "uid": "${DS_PROMETHEUS}"
+ },
+ "exemplar": true,
+ "expr": "rate(process_io_ops{instance=~\"$instance\"}[1m])",
+ "interval": "",
+ "legendFormat": "{{from}}-{{name}}",
+ "refId": "A"
+ }
],
- "displayMode": "table",
- "placement": "right",
- "showLegend": true,
- "sortBy": "Mean",
- "sortDesc": true
- },
- "tooltip": {
- "mode": "multi",
- "sort": "asc"
+ "title": "I/O System Call Rate",
+ "type": "timeseries"
}
- },
+ ],
"targets": [
{
"datasource": {
"type": "prometheus",
- "uid": "${DS_PROMETHEUS}"
+ "uid": "lfuOmw1Vk"
},
- "exemplar": true,
- "expr": "rate(process_io_ops{instance=~\"$instance\"}[1m])",
- "interval": "",
- "legendFormat": "{{from}}-{{name}}",
"refId": "A"
}
],
- "title": "I/O System Call Rate",
- "type": "timeseries"
+ "title": "Disk I/O",
+ "type": "row"
}
],
"refresh": "15s",
@@ -3404,8 +5128,8 @@
]
},
"timezone": "browser",
- "title": "Apache IoTDB ConfigNode Dashboard",
- "uid": "TbEVYRw7T",
- "version": 3,
+ "title": "yesterday dashboard",
+ "uid": "TbEVYRw7T666",
+ "version": 1,
"weekStart": ""
}
\ No newline at end of file
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeStatus.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeStatus.java
index d19eadfbdd..32f083dd04 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeStatus.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeStatus.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.commons.cluster;
/** Node status for showing cluster */
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeType.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeType.java
index 38f521f837..ff3fbfdada 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeType.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/NodeType.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.commons.cluster;
public enum NodeType {
@@ -34,7 +35,7 @@ public enum NodeType {
public static NodeType parse(String type) {
for (NodeType nodeType : NodeType.values()) {
- if (nodeType.nodeType.equals(type)) {
+ if (type.equals(nodeType.getNodeType())) {
return nodeType;
}
}
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/RegionRoleType.java b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/RegionRoleType.java
index 6e65824f7d..95ee99f23b 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/cluster/RegionRoleType.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/cluster/RegionRoleType.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.commons.cluster;
/** Region Role for showing regions */
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java
index bb399ac979..9644224f04 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java
@@ -41,8 +41,6 @@ public enum Metric {
DATA_WRITTEN,
DATA_READ,
COMPACTION_TASK_COUNT,
- CLUSTER_NODE_STATUS,
- CLUSTER_NODE_LEADER_COUNT,
PROCESS_CPU_LOAD,
PROCESS_CPU_TIME,
PROCESS_MAX_MEM,
@@ -61,11 +59,15 @@ public enum Metric {
SYS_COMMITTED_VM_SIZE,
SYS_DISK_TOTAL_SPACE,
SYS_DISK_FREE_SPACE,
- CONFIG_NODE,
- DATA_NODE,
- STORAGE_GROUP,
- REGION,
- SLOT,
+
+ NODE_NUM,
+ DATABASE_NUM,
+ REGION_NUM,
+ REGION_NUM_IN_DATA_NODE,
+ REGION_GROUP_LEADER_NUM_IN_DATA_NODE,
+ SERIES_SLOT_NUM_IN_DATABASE,
+ REGION_GROUP_NUM_IN_DATABASE,
+
THRIFT_CONNECTIONS,
THRIFT_ACTIVE_THREADS,
IOT_CONSENSUS,