Providing metrics information to Prometheus

Former-commit-id: fcca6c8438877872d6c6e65bde522039f1a8668d
This commit is contained in:
zhangwei 2024-03-21 09:11:12 +08:00
parent 918d95ad72
commit 021092b6b4
13 changed files with 322 additions and 278 deletions

View File

@ -24,9 +24,9 @@ type (
}
CenterIndex {
name string `json:"name"`
cpu float32 `json:"cpu"`
memory float32 `json:"memory"`
storage float32 `json:"storage"`
cpu string `json:"cpu"`
memory string `json:"memory"`
storage string `json:"storage"`
centerType string `json:"centerType"`
}
)
@ -35,9 +35,16 @@ type (
clusterLoadRecords []ClusterLoadRecord `json:"clusterLoadRecords"`
}
ClusterLoadRecord {
AdapterId int64 `json:"adapterId"`
ClusterName string `json:"clusterName"`
CpuAvail float64 `json:"cpuAvail"`
CpuTotal float64 `json:"cpuTotal"`
CpuUsage float64 `json:"cpuUsage"`
MemoryAvail float64 `json:"memoryAvail"`
MemoryUsage float64 `json:"memoryUsage"`
MemoryTotal float64 `json:"memoryTotal"`
DiskAvail float64 `json:"diskAvail"`
DiskTotal float64 `json:"diskTotal"`
DiskUsage float64 `json:"diskUsage"`
}
)
@ -79,7 +86,6 @@ type commitTaskReq {
MatchLabels map[string]string `json:"matchLabels,optional"`
YamlList []string `json:"yamlList"`
ClusterName string `json:"clusterName"`
}
type (

View File

@ -49,6 +49,7 @@ type Config struct {
Password string
}
SnowflakeConf SnowflakeConf
PromUrl string
}
// SnowflakeConf 雪花算法机器id配置

View File

@ -33,7 +33,7 @@ func SyncParticipantRpc(svc *svc.ServiceContext) {
if err != nil {
return
}
svc.PromClient[participant.Id] = promClient
svc.MonitorClient[participant.Id] = promClient
}
}
}

View File

@ -134,16 +134,6 @@ func RegisterHandlers(server *rest.Server, serverCtx *svc.ServiceContext) {
Path: "/core/metrics",
Handler: core.MetricsHandler(serverCtx),
},
{
Method: http.MethodGet,
Path: "/core/pullTaskInfo",
Handler: core.PullTaskInfoHandler(serverCtx),
},
{
Method: http.MethodGet,
Path: "/core/pushTaskInfo",
Handler: core.PushTaskInfoHandler(serverCtx),
},
},
rest.WithPrefix("/pcm/v1"),
)

View File

@ -27,14 +27,14 @@ func NewControllerMetricsLogic(ctx context.Context, svcCtx *svc.ServiceContext)
func (l *ControllerMetricsLogic) ControllerMetrics(req *types.ControllerMetricsReq) (resp *types.ControllerMetricsResp, err error) {
resp = &types.ControllerMetricsResp{}
if _, ok := l.svcCtx.PromClient[req.ParticipantId]; ok {
if _, ok := l.svcCtx.MonitorClient[req.ParticipantId]; ok {
if len(req.Pod) != 0 {
resp.Data = l.svcCtx.PromClient[req.ParticipantId].GetNamedMetricsByTime(req.Metrics, req.Start, req.End, 60*time.Minute, tracker.PodOption{
resp.Data = l.svcCtx.MonitorClient[req.ParticipantId].GetNamedMetricsByTime(req.Metrics, req.Start, req.End, 60*time.Minute, tracker.PodOption{
PodName: req.Pod,
})
} else {
resp.Data = l.svcCtx.PromClient[req.ParticipantId].GetNamedMetricsByTime(req.Metrics, req.Start, req.End, 60*time.Minute, tracker.ControllerOption{
resp.Data = l.svcCtx.MonitorClient[req.ParticipantId].GetNamedMetricsByTime(req.Metrics, req.Start, req.End, 60*time.Minute, tracker.ControllerOption{
WorkloadName: req.WorkloadName,
})
}

View File

@ -2,6 +2,8 @@ package core
import (
"context"
"github.com/prometheus/common/model"
"gitlink.org.cn/JointCloud/pcm-coordinator/pkg/tracker"
"gitlink.org.cn/JointCloud/pcm-coordinator/api/internal/svc"
"gitlink.org.cn/JointCloud/pcm-coordinator/api/internal/types"
@ -24,31 +26,41 @@ func NewCenterResourcesLogic(ctx context.Context, svcCtx *svc.ServiceContext) *C
}
func (l *CenterResourcesLogic) CenterResources() (resp *types.CenterResourcesResp, err error) {
// todo: add your logic here and delete this line
centerIndex1 := types.CenterIndex{
Name: "阿里云",
Cpu: float32(12.33),
Memory: float32(64.55),
Storage: float32(33.90),
CenterType: "cloud",
}
centerIndex2 := types.CenterIndex{
Name: "A超算中心",
Cpu: float32(42.36),
Memory: float32(66.55),
Storage: float32(23.231),
CenterType: "hpc",
}
centerIndex3 := types.CenterIndex{
Name: "智算中心",
Cpu: float32(78.33),
Memory: float32(36.55),
Storage: float32(88.93),
CenterType: "ai",
}
resp = &types.CenterResourcesResp{}
resp.CentersIndex = append(resp.CentersIndex, centerIndex1)
resp.CentersIndex = append(resp.CentersIndex, centerIndex2)
resp.CentersIndex = append(resp.CentersIndex, centerIndex3)
rawData, err := l.svcCtx.PromClient.GetRawData("resource_top3", tracker.ClusterOption{})
if err != nil {
return nil, err
}
var centersIndex []*types.CenterIndex
data := rawData.(model.Vector)
for _, d := range data {
for _, v := range d.Metric {
centersIndex = append(centersIndex, &types.CenterIndex{Name: string(v)})
}
}
for _, centerIndex := range centersIndex {
// Query the types of resource centers
//l.svcCtx.DbEngin.Raw().Scan(&centerIndex.CenterType)
cpuRawData, err := l.svcCtx.PromClient.GetRawData("cluster_cpu_usage", tracker.ClusterOption{ClusterName: centerIndex.Name})
cpuData := cpuRawData.(model.Vector)
if err != nil {
return nil, err
}
centerIndex.Cpu = cpuData[0].Value.String()
memoryRawData, err := l.svcCtx.PromClient.GetRawData("cluster_memory_usage", tracker.ClusterOption{ClusterName: centerIndex.Name})
if err != nil {
return nil, err
}
memoryData := memoryRawData.(model.Vector)
centerIndex.Memory = memoryData[0].Value.String()
diskRawData, err := l.svcCtx.PromClient.GetRawData("cluster_disk_usage", tracker.ClusterOption{ClusterName: centerIndex.Name})
if err != nil {
return nil, err
}
diskData := diskRawData.(model.Vector)
centerIndex.Storage = diskData[0].Value.String()
resp.CentersIndex = append(resp.CentersIndex, *centerIndex)
}
return resp, nil
}

View File

@ -5,6 +5,7 @@ import (
"gitlink.org.cn/JointCloud/pcm-coordinator/api/internal/svc"
"gitlink.org.cn/JointCloud/pcm-coordinator/api/internal/types"
"gitlink.org.cn/JointCloud/pcm-coordinator/pkg/tracker"
"strconv"
"github.com/zeromicro/go-zero/core/logx"
)
@ -26,9 +27,17 @@ func NewSyncClusterLoadLogic(ctx context.Context, svcCtx *svc.ServiceContext) *S
func (l *SyncClusterLoadLogic) SyncClusterLoad(req *types.SyncClusterLoadReq) error {
if len(req.ClusterLoadRecords) != 0 {
for _, record := range req.ClusterLoadRecords {
tracker.ClusterCpuGauge.WithLabelValues(record.ClusterName).Set(record.CpuUsage)
tracker.ClusterMemoryGauge.WithLabelValues(record.ClusterName).Set(record.MemoryUsage)
tracker.ClusterDiskGauge.WithLabelValues(record.ClusterName).Set(record.DiskUsage)
tracker.ClusterCpuUsageGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.CpuUsage)
tracker.ClusterCpuAvailGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.CpuAvail)
tracker.ClusterCpuTotalGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.CpuTotal)
tracker.ClusterMemoryUsageGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.MemoryUsage)
tracker.ClusterMemoryAvailGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.MemoryAvail)
tracker.ClusterMemoryTotalGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.MemoryTotal)
tracker.ClusterDiskUsageGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.DiskUsage)
tracker.ClusterDiskAvailGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.DiskAvail)
tracker.ClusterDiskTotalGauge.WithLabelValues(record.ClusterName, strconv.FormatInt(record.AdapterId, 10)).Set(record.DiskTotal)
}
}
return nil

View File

@ -59,8 +59,9 @@ type ServiceContext struct {
Downloader *s3manager.Downloader
Uploader *s3manager.Uploader
K8sRpc kubernetesclient.Kubernetes
PromClient map[int64]tracker.Prometheus
MonitorClient map[int64]tracker.Prometheus
ParticipantRpc participantservice.ParticipantService
PromClient tracker.Prometheus
}
func NewServiceContext(c config.Config) *ServiceContext {
@ -72,8 +73,14 @@ func NewServiceContext(c config.Config) *ServiceContext {
DisableSSL: aws.Bool(false), //是否禁用https,这里表示不禁用,即使用HTTPS
S3ForcePathStyle: aws.Bool(true), //使用路径样式而非虚拟主机样式,区别请参考:https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html
})
promClient, err := tracker.NewPrometheus(c.PromUrl)
if err != nil {
logx.Errorf("InitPrometheus err: %v", err)
panic("InitSnowflake err")
}
//添加snowflake支持
err := utils.InitSnowflake(c.SnowflakeConf.MachineId)
err = utils.InitSnowflake(c.SnowflakeConf.MachineId)
if err != nil {
logx.Errorf("InitSnowflake err: %v", err)
panic("InitSnowflake err")
@ -122,10 +129,11 @@ func NewServiceContext(c config.Config) *ServiceContext {
OctopusRpc: octopusclient.NewOctopus(zrpc.MustNewClient(c.OctopusRpcConf)),
OpenstackRpc: openstackclient.NewOpenstack(zrpc.MustNewClient(c.OpenstackRpcConf)),
K8sRpc: kubernetesclient.NewKubernetes(zrpc.MustNewClient(c.K8sNativeConf)),
PromClient: make(map[int64]tracker.Prometheus),
MonitorClient: make(map[int64]tracker.Prometheus),
ParticipantRpc: participantservice.NewParticipantService(zrpc.MustNewClient(c.PcmCoreRpcConf)),
DockerClient: dockerClient,
Downloader: downloader,
Uploader: uploader,
PromClient: promClient,
}
}

View File

@ -18,9 +18,9 @@ type CenterResourcesResp struct {
type CenterIndex struct {
Name string `json:"name"`
Cpu float32 `json:"cpu"`
Memory float32 `json:"memory"`
Storage float32 `json:"storage"`
Cpu string `json:"cpu"`
Memory string `json:"memory"`
Storage string `json:"storage"`
CenterType string `json:"centerType"`
}
@ -29,9 +29,16 @@ type SyncClusterLoadReq struct {
}
type ClusterLoadRecord struct {
AdapterId int64 `json:"adapterId"`
ClusterName string `json:"clusterName"`
CpuAvail float64 `json:"cpuAvail"`
CpuTotal float64 `json:"cpuTotal"`
CpuUsage float64 `json:"cpuUsage"`
MemoryAvail float64 `json:"memoryAvail"`
MemoryUsage float64 `json:"memoryUsage"`
MemoryTotal float64 `json:"memoryTotal"`
DiskAvail float64 `json:"diskAvail"`
DiskTotal float64 `json:"diskTotal"`
DiskUsage float64 `json:"diskUsage"`
}
@ -93,7 +100,7 @@ type CommitVmTaskReq struct {
NsID string `json:"nsID"`
Replicas int64 `json:"replicas,optional"`
MatchLabels map[string]string `json:"matchLabels,optional"`
Server ServerCommit `json:"server,optional"`
Server []ServerCommit `json:"server,optional"`
Platform string `json:"platform,optional"`
}

View File

@ -14,7 +14,10 @@
package tracker
import "time"
import (
"github.com/prometheus/common/model"
"time"
)
type Interface interface {
//GetMetric(expr string, time time.Time) Metric
@ -27,4 +30,5 @@ type Interface interface {
//// meter
//GetNamedMeters(meters []string, time time.Time, opts []QueryOption) []Metric
//GetNamedMetersOverTime(metrics []string, start, end time.Time, step time.Duration, opts []QueryOption) []Metric
GetRawData(expr string, o QueryOption) (model.Value, error)
}

View File

@ -27,35 +27,13 @@ const (
var promQLTemplates = map[string]string{
//namespace
"cluster_cpu_usage": "sum by (cluster_name)(cluster_cpu_usage{$1})",
"cluster_memory_usage": "sum by (cluster_name)(cluster_memory_usage{$1})",
"cluster_disk_usage": "sum by (cluster_name)(cluster_disk_usage{$1})",
"resource_top3": "topk(3,sum by (cluster_name)(cluster_cpu_usage +cluster_memory_usage +cluster_disk_usage)/3)",
"namespace_cpu_usage": `round(namespace:container_cpu_usage_seconds_total:sum_rate{namespace!="", $1}, 0.001)`,
"namespace_memory_usage": `namespace:container_memory_usage_bytes:sum{namespace!="", $1}`,
"namespace_memory_usage_wo_cache": `namespace:container_memory_usage_bytes_wo_cache:sum{namespace!="", $1}`,
"namespace_net_bytes_transmitted": `sum by (namespace) (irate(container_network_transmit_bytes_total{namespace!="", pod!="", interface!~"^(cali.+|tunl.+|dummy.+|kube.+|flannel.+|cni.+|docker.+|veth.+|lo.*)", job="kubelet"}[5m]) * on (namespace) group_left(workspace) kube_namespace_labels{$1}) or on(namespace) max by(namespace) (kube_namespace_labels{$1} * 0)`,
"namespace_net_bytes_received": `sum by (namespace) (irate(container_network_receive_bytes_total{namespace!="", pod!="", interface!~"^(cali.+|tunl.+|dummy.+|kube.+|flannel.+|cni.+|docker.+|veth.+|lo.*)", job="kubelet"}[5m]) * on (namespace) group_left(workspace) kube_namespace_labels{$1}) or on(namespace) max by(namespace) (kube_namespace_labels{$1} * 0)`,
"namespace_pod_count": `sum by (namespace) (kube_pod_status_phase{phase!~"Failed|Succeeded", namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1}) or on(namespace) max by(namespace) (kube_namespace_labels{$1} * 0)`,
"namespace_pod_running_count": `sum by (namespace) (kube_pod_status_phase{phase="Running", namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1}) or on(namespace) max by(namespace) (kube_namespace_labels{$1} * 0)`,
"namespace_pod_succeeded_count": `sum by (namespace) (kube_pod_status_phase{phase="Succeeded", namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1}) or on(namespace) max by(namespace) (kube_namespace_labels{$1} * 0)`,
"namespace_pod_abnormal_count": `namespace:pod_abnormal:count{namespace!="", $1}`,
"namespace_pod_abnormal_ratio": `namespace:pod_abnormal:ratio{namespace!="", $1}`,
"namespace_memory_limit_hard": `min by (namespace) (kube_resourcequota{resourcequota!="quota", type="hard", namespace!="", resource="limits.memory"} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_cpu_limit_hard": `min by (namespace) (kube_resourcequota{resourcequota!="quota", type="hard", namespace!="", resource="limits.cpu"} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_pod_count_hard": `min by (namespace) (kube_resourcequota{resourcequota!="quota", type="hard", namespace!="", resource="count/pods"} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_cronjob_count": `sum by (namespace) (kube_cronjob_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_pvc_count": `sum by (namespace) (kube_persistentvolumeclaim_info{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_daemonset_count": `sum by (namespace) (kube_daemonset_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_deployment_count": `sum by (namespace) (kube_deployment_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_endpoint_count": `sum by (namespace) (kube_endpoint_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_hpa_count": `sum by (namespace) (kube_horizontalpodautoscaler_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_job_count": `sum by (namespace) (kube_job_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_statefulset_count": `sum by (namespace) (kube_statefulset_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_replicaset_count": `count by (namespace) (kube_replicaset_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_service_count": `sum by (namespace) (kube_service_info{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_secret_count": `sum by (namespace) (kube_secret_info{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_configmap_count": `sum by (namespace) (kube_configmap_info{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_ingresses_extensions_count": `sum by (namespace) (kube_ingress_labels{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"namespace_s2ibuilder_count": `sum by (namespace) (s2i_s2ibuilder_created{namespace!=""} * on (namespace) group_left(workspace) kube_namespace_labels{$1})`,
"controller_cpu_usage_rate": `sum( node_namespace_pod_container:container_cpu_usage_seconds_total:sum_irate{}* on(namespace,pod) group_left(workload) namespace_workload_pod:kube_pod_owner:relabel{$1}) by (workload)/sum( kube_pod_container_resource_limits{job="kube-state-metrics", resource="cpu"}* on(namespace,pod) group_left(workload) namespace_workload_pod:kube_pod_owner:relabel{ }) by (workload)`,
"controller_memory_usage_rate": `sum( container_memory_working_set_bytes{job="kubelet", metrics_path="/metrics/cadvisor", container!="", image!=""} * on(namespace,pod) group_left(workload) namespace_workload_pod:kube_pod_owner:relabel{$1}) by (workload)/sum( kube_pod_container_resource_limits{job="kube-state-metrics", resource="memory"}* on(namespace,pod) group_left(workload) namespace_workload_pod:kube_pod_owner:relabel{ }) by (workload)`,
// pod
@ -81,7 +59,7 @@ func makeExpr(metric string, opts QueryOptions) string {
tmpl := promQLTemplates[metric]
switch opts.Level {
case LevelCluster:
return tmpl
return makeClusterMetricExpr(tmpl, opts)
case LevelNode:
return makeNodeMetricExpr(tmpl, opts)
case LevelWorkspace:
@ -105,6 +83,14 @@ func makeExpr(metric string, opts QueryOptions) string {
}
}
func makeClusterMetricExpr(tmpl string, o QueryOptions) string {
var clusterSelector string
if o.ClusterName != "" {
clusterSelector = fmt.Sprintf(`cluster_name="%s"`, o.ClusterName)
}
return strings.Replace(tmpl, "$1", clusterSelector, -1)
}
func makeNodeMetricExpr(tmpl string, o QueryOptions) string {
var nodeSelector string
if o.NodeName != "" {
@ -177,19 +163,12 @@ func makePVCMetricExpr(tmpl string, o QueryOptions) string {
// GET /namespaces/{namespace}/persistentvolumeclaims/{persistentvolumeclaim} or
// GET /namespaces/{namespace}/persistentvolumeclaims
if o.Namespace != "" {
if o.PersistentVolumeClaimName != "" {
pvcSelector = fmt.Sprintf(`namespace="%s", persistentvolumeclaim="%s"`, o.Namespace, o.PersistentVolumeClaimName)
} else {
pvcSelector = fmt.Sprintf(`namespace="%s", persistentvolumeclaim=~"%s"`, o.Namespace, o.ResourceFilter)
}
return strings.Replace(tmpl, "$1", pvcSelector, -1)
}
// For monitoring persistentvolumeclaims of the specific storageclass
// GET /storageclasses/{storageclass}/persistentvolumeclaims
if o.StorageClassName != "" {
pvcSelector = fmt.Sprintf(`storageclass="%s", persistentvolumeclaim=~"%s"`, o.StorageClassName, o.ResourceFilter)
}
return strings.Replace(tmpl, "$1", pvcSelector, -1)
}

View File

@ -68,6 +68,7 @@ type QueryOptions struct {
NamespacedResourcesFilter string
QueryType string
ResourceFilter string
ClusterName string
NodeName string
WorkspaceName string
Namespace string
@ -77,10 +78,6 @@ type QueryOptions struct {
PodName string
PodsName string
ContainerName string
StorageClassName string
PersistentVolumeClaimName string
PVCFilter string
ApplicationName string
ServiceName string
Ingress string
Job string
@ -92,10 +89,13 @@ func NewQueryOptions() *QueryOptions {
return &QueryOptions{}
}
type ClusterOption struct{}
type ClusterOption struct {
ClusterName string
}
func (_ ClusterOption) Apply(o *QueryOptions) {
func (c ClusterOption) Apply(o *QueryOptions) {
o.Level = LevelCluster
o.ClusterName = c.ClusterName
}
type NodeOption struct {
@ -110,8 +110,6 @@ func (no NodeOption) Apply(o *QueryOptions) {
o.Level = LevelNode
o.ResourceFilter = no.ResourceFilter
o.NodeName = no.NodeName
o.PVCFilter = no.PVCFilter
o.StorageClassName = no.StorageClassName
o.QueryType = no.QueryType
}
@ -126,8 +124,6 @@ func (wo WorkspaceOption) Apply(o *QueryOptions) {
o.Level = LevelWorkspace
o.ResourceFilter = wo.ResourceFilter
o.WorkspaceName = wo.WorkspaceName
o.PVCFilter = wo.PVCFilter
o.StorageClassName = wo.StorageClassName
}
type NamespaceOption struct {
@ -143,8 +139,6 @@ func (no NamespaceOption) Apply(o *QueryOptions) {
o.ResourceFilter = no.ResourceFilter
o.WorkspaceName = no.WorkspaceName
o.Namespace = no.NamespaceName
o.PVCFilter = no.PVCFilter
o.StorageClassName = no.StorageClassName
}
type ApplicationsOption struct {
@ -183,8 +177,6 @@ type ApplicationOption struct {
func (ao ApplicationOption) Apply(o *QueryOptions) {
o.Level = LevelApplication
o.Namespace = ao.NamespaceName
o.ApplicationName = ao.Application
o.StorageClassName = ao.StorageClassName
app_components := strings.Join(ao.ApplicationComponents[:], "|")
@ -303,11 +295,6 @@ func (po PVCOption) Apply(o *QueryOptions) {
o.Level = LevelPVC
o.ResourceFilter = po.ResourceFilter
o.Namespace = po.NamespaceName
o.StorageClassName = po.StorageClassName
o.PersistentVolumeClaimName = po.PersistentVolumeClaimName
// for meter
o.PVCFilter = po.PersistentVolumeClaimName
}
type IngressOption struct {

View File

@ -27,22 +27,53 @@ import (
)
var (
ClusterCpuGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
ClusterCpuUsageGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_cpu_usage",
Help: "Cluster CPU Utilization Rate.",
}, []string{"cluster_name"})
ClusterMemoryGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
}, []string{"cluster_name", "adapter_id"})
ClusterCpuAvailGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_cpu_avail",
Help: "Cluster CPU Available.",
}, []string{"cluster_name", "adapter_id"})
ClusterCpuTotalGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_cpu_total",
Help: "Cluster CPU Total.",
}, []string{"cluster_name", "adapter_id"})
ClusterMemoryUsageGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_memory_usage",
Help: "Cluster Memory Utilization Rate.",
}, []string{"cluster_name"})
ClusterDiskGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
}, []string{"cluster_name", "adapter_id"})
ClusterMemoryAvailGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_memory_avail",
Help: "Cluster Memory Available.",
}, []string{"cluster_name", "adapter_id"})
ClusterMemoryTotalGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_memory_total",
Help: "Cluster Memory Total.",
}, []string{"cluster_name", "adapter_id"})
ClusterDiskUsageGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_disk_usage",
Help: "Cluster Disk Utilization Rate.",
}, []string{"cluster_name"})
}, []string{"cluster_name", "adapter_id"})
ClusterDiskAvailGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_disk_avail",
Help: "Cluster Disk Available.",
}, []string{"cluster_name", "adapter_id"})
ClusterDiskTotalGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{
Name: "cluster_disk_total",
Help: "Cluster Disk Total.",
}, []string{"cluster_name", "adapter_id"})
metrics = []prometheus.Collector{
ClusterCpuGauge,
ClusterMemoryGauge,
ClusterDiskGauge,
ClusterCpuUsageGauge,
ClusterCpuAvailGauge,
ClusterCpuTotalGauge,
ClusterMemoryUsageGauge,
ClusterMemoryAvailGauge,
ClusterMemoryTotalGauge,
ClusterDiskUsageGauge,
ClusterDiskAvailGauge,
ClusterDiskTotalGauge,
}
)
@ -225,3 +256,13 @@ func genMetricFilter(o QueryOption) func(metric model.Metric) bool {
return true
}
}
func (p Prometheus) GetRawData(expr string, o QueryOption) (model.Value, error) {
opts := NewQueryOptions()
o.Apply(opts)
value, _, err := p.client.Query(context.Background(), makeExpr(expr, *opts), time.Now())
if err != nil {
return nil, err
}
return value, nil
}