未验证 提交 9f3ff517 编写于 作者: W wankai123 提交者: GitHub

Support envoy cluster manager metrics (#7294)

上级 8df362b9
......@@ -240,6 +240,7 @@ jobs:
cd skywalking-kubernetes
git reset --hard $SW_KUBERNETES_COMMIT_SHA
cd chart
mkdir -p skywalking/files/conf.d/oap/ && cp ../../test/e2e/e2e-test/src/test/resources/metadata-service-mapping.yaml skywalking/files/conf.d/oap/metadata-service-mapping.yaml
helm dep up skywalking
helm -n istio-system install skywalking skywalking \
--set fullnameOverride=skywalking \
......
......@@ -81,6 +81,7 @@ Release Notes.
* Performance: compile LAL DSL statically and run with type checked.
* Add pagination to event query protocol.
* Performance: optimize Envoy error logs persistence performance.
* Support envoy `cluster manager` metrics.
* Performance: remove the synchronous persistence mechanism from batch ElasticSearch DAO. Because the current enhanced
persistent session mechanism, don't require the data queryable immediately after the insert and update anymore.
* Performance: share `flushInterval` setting for both metrics and record data, due
......
......@@ -85,7 +85,7 @@ static_resources:
- endpoint:
address:
socket_address:
address: host.docker.internal
address: skywalking
port_value: 11800
- name: service_google
......
......@@ -73,7 +73,8 @@ Under this circumstance, emitting the metrics to SyWalking is as simple as addin
```shell
istioctl install -y \
--set profile=demo `# replace the profile as per your need` \
--set meshConfig.defaultConfig.envoyMetricsService.address=<skywalking.address.port.11800> # replace <skywalking.address.port.11800> with your actual SkyWalking OAP address
--set meshConfig.defaultConfig.envoyMetricsService.address=<skywalking.address.port.11800> \ # replace <skywalking.address.port.11800> with your actual SkyWalking OAP address
--set 'meshConfig.defaultConfig.proxyStatsMatcher.inclusionRegexps[0]=.*'
```
If you already have Istio installed, you can use the following command to apply the config without re-installing Istio:
......@@ -81,7 +82,8 @@ If you already have Istio installed, you can use the following command to apply
```shell
istioctl manifest install -y \
--set profile=demo `# replace the profile as per your need` \
--set meshConfig.defaultConfig.envoyMetricsService.address=<skywalking.address.port.11800> # replace <skywalking.address.port.11800> with your actual SkyWalking OAP address
--set meshConfig.defaultConfig.envoyMetricsService.address=<skywalking.address.port.11800> \ # replace <skywalking.address.port.11800> with your actual SkyWalking OAP address
--set 'meshConfig.defaultConfig.proxyStatsMatcher.inclusionRegexps[0]=.*'
```
# Metrics data
......
......@@ -29,31 +29,49 @@
# "-P-6H+3M" -- parses as "+6 hours and -3 minutes"
# </pre>
expSuffix: tag({tags -> tags.cluster = 'istio-dp::' + tags.cluster}).instance(['cluster'], ['instance'])
expSuffix: tag({tags -> tags.app = 'istio-dp::' + tags.app}).instance(['app'], ['instance'])
metricPrefix: envoy
metricsRules:
- name: heap_memory_used
exp: server_memory_heap_size
- name: heap_memory_max_used
exp: server_memory_heap_size.max(['cluster', 'instance'])
exp: server_memory_heap_size.max(['app', 'instance'])
- name: memory_allocated
exp: server_memory_allocated
- name: memory_allocated_max
exp: server_memory_allocated.max(['cluster', 'instance'])
exp: server_memory_allocated.max(['app', 'instance'])
- name: memory_physical_size
exp: server_memory_physical_size
- name: memory_physical_size_max
exp: server_memory_physical_size.max(['cluster', 'instance'])
exp: server_memory_physical_size.max(['app', 'instance'])
- name: total_connections_used
exp: server_total_connections.max(['cluster', 'instance'])
exp: server_total_connections.max(['app', 'instance'])
- name: parent_connections_used
exp: server_parent_connections.max(['cluster', 'instance'])
exp: server_parent_connections.max(['app', 'instance'])
- name: worker_threads
exp: server_concurrency
- name: worker_threads_max
exp: server_concurrency.max(['cluster', 'instance'])
exp: server_concurrency.max(['app', 'instance'])
- name: bug_failures
exp: server_envoy_bug_failures
# envoy_cluster_metrics
- name: cluster_membership_healthy
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+membership_healthy').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').tagNotMatch('cluster_name' , '.+kube-system').sum(['app', 'instance' , 'cluster_name'])
- name: cluster_upstream_cx_active
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_cx_active').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name'])
- name: cluster_upstream_cx_increase
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_cx_total').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name']).increase('PT1M')
- name: cluster_upstream_rq_active
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_rq_active').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name'])
- name: cluster_upstream_rq_increase
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_rq_total').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name']).increase('PT1M')
- name: cluster_upstream_rq_pending_active
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_rq_pending_active').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name'])
- name: cluster_lb_healthy_panic_increase
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+lb_healthy_panic').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name'])
- name: cluster_upstream_cx_none_healthy_increase
exp: envoy_cluster_metrics.tagMatch('metrics_name' , '.+upstream_cx_none_healthy').tagMatch('metrics_name' , 'cluster.outbound.+|cluster.inbound.+').sum(['app', 'instance' , 'cluster_name']).increase('PT1M')
......@@ -13,5 +13,5 @@
# See the License for the specific language governing permissions and
# limitations under the License.
serviceName: ${LABELS."service.istio.io/canonical-name",LABELS."app.kubernetes.io/name",LABELS.app}
serviceName: ${LABELS."service.istio.io/canonical-revision"}.${LABELS."service.istio.io/canonical-name",LABELS."app.kubernetes.io/name",LABELS.app}.${NAMESPACE}
serviceInstanceName: ${NAME}
......@@ -72,6 +72,98 @@ templates:
"metricName": "envoy_bug_failures",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Membership Healthy",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_membership_healthy",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream CX Active",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_cx_active",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream CX Increase",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_cx_increase",
"queryMetricType": "readLabeledMetricsValues",
"unit": "pm",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream RQ Active",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_rq_active",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream RQ Increase",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_rq_increase",
"queryMetricType": "readLabeledMetricsValues",
"unit": "pm",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream RQ Pending Active",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_rq_pending_active",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "LB Healthy Panic Increase",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_lb_healthy_panic_increase",
"queryMetricType": "readLabeledMetricsValues",
"unit": "pm",
"chartType": "ChartLine"
},
{
"width": "3",
"title": "Upstream CX None Healthy Increase",
"height": 350,
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "envoy_cluster_upstream_cx_none_healthy_increase",
"queryMetricType": "readLabeledMetricsValues",
"unit": "pm",
"chartType": "ChartLine"
}
]
}
......
......@@ -28,6 +28,7 @@ import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rule;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rules;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
import org.apache.skywalking.oap.server.library.module.ModuleStartException;
import org.apache.skywalking.oap.server.receiver.envoy.metrics.adapters.ClusterManagerMetricsAdapter;
public class EnvoyMetricReceiverConfig extends ModuleConfig {
@Getter
......@@ -38,6 +39,8 @@ public class EnvoyMetricReceiverConfig extends ModuleConfig {
private String k8sServiceNameRule;
private final ServiceMetaInfoFactory serviceMetaInfoFactory = new ServiceMetaInfoFactoryImpl();
@Getter
private final ClusterManagerMetricsAdapter clusterManagerMetricsAdapter = new ClusterManagerMetricsAdapter(this);
public List<String> getAlsHTTPAnalysis() {
if (Strings.isNullOrEmpty(alsHTTPAnalysis)) {
......
......@@ -23,9 +23,11 @@ import io.envoyproxy.envoy.service.metrics.v3.StreamMetricsMessage;
import io.envoyproxy.envoy.service.metrics.v3.StreamMetricsResponse;
import io.grpc.stub.StreamObserver;
import io.prometheus.client.Metrics;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.apm.util.StringUtil;
......@@ -51,7 +53,8 @@ public class MetricServiceGRPCHandler extends MetricsServiceGrpc.MetricsServiceI
private final EnvoyMetricReceiverConfig config;
public MetricServiceGRPCHandler(final ModuleManager moduleManager, final EnvoyMetricReceiverConfig config) throws ModuleStartException {
public MetricServiceGRPCHandler(final ModuleManager moduleManager,
final EnvoyMetricReceiverConfig config) throws ModuleStartException {
this.config = config;
MetricsCreator metricsCreator = moduleManager.find(TelemetryModule.NAME)
......@@ -89,28 +92,38 @@ public class MetricServiceGRPCHandler extends MetricsServiceGrpc.MetricsServiceI
if (isFirst) {
isFirst = false;
service = config.serviceMetaInfoFactory().fromStruct(message.getIdentifier().getNode().getMetadata());
service = config.serviceMetaInfoFactory()
.fromStruct(message.getIdentifier().getNode().getMetadata());
}
if (log.isDebugEnabled()) {
log.debug("Envoy metrics reported from service[{}]", service);
}
if (service != null && StringUtil.isNotEmpty(service.getServiceName()) && StringUtil.isNotEmpty(service.getServiceInstanceName())) {
List<Metrics.MetricFamily> metricFamilies = new ArrayList<>();
if (service != null && StringUtil.isNotEmpty(service.getServiceName()) && StringUtil.isNotEmpty(
service.getServiceInstanceName())) {
List<Metrics.MetricFamily> list = message.getEnvoyMetricsList();
Map<String, List<Metric>> groupingMetrics = new HashMap<>();
for (final Metrics.MetricFamily metricFamily : list) {
counter.inc();
try (final HistogramMetrics.Timer ignored = histogram.createTimer()) {
final ProtoMetricFamily2MetricsAdapter adapter = new ProtoMetricFamily2MetricsAdapter(metricFamily);
final Stream<Metric> metrics = adapter.adapt().peek(it -> {
it.getLabels().putIfAbsent("cluster", service.getServiceName());
final ProtoMetricFamily2MetricsAdapter adapter = new ProtoMetricFamily2MetricsAdapter(
metricFamily, config.getClusterManagerMetricsAdapter());
adapter.adapt().forEach(it -> {
it.getLabels().putIfAbsent("app", service.getServiceName());
it.getLabels().putIfAbsent("instance", service.getServiceInstanceName());
List<Metric> metricList = groupingMetrics.computeIfAbsent(
it.getName(),
name -> new ArrayList<>()
);
metricList.add(it);
});
converters.forEach(converter -> converter.toMeter(metrics));
}
}
groupingMetrics.forEach(
(name, metrics) ->
converters.forEach(converter -> converter.toMeter(metrics.stream())));
}
}
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.skywalking.oap.server.receiver.envoy.metrics.adapters;
import com.google.protobuf.Struct;
import com.google.protobuf.Value;
import io.prometheus.client.Metrics;
import java.util.Map;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.skywalking.apm.util.StringUtil;
import org.apache.skywalking.oap.server.receiver.envoy.EnvoyMetricReceiverConfig;
import org.apache.skywalking.oap.server.receiver.envoy.als.ServiceMetaInfo;
@Slf4j
@RequiredArgsConstructor
public class ClusterManagerMetricsAdapter {
private static final String DEFAULT_VALUE = "-";
private final EnvoyMetricReceiverConfig config;
public String adaptMetricsName(final Metrics.MetricFamily metricFamily) {
return "envoy_cluster_metrics";
}
public Map<String, String> adaptLabels(final Metrics.MetricFamily metricFamily, final Map<String, String> labels) {
String metricsName = metricFamily.getName();
labels.putIfAbsent("metrics_name", metricsName);
String clusterName = null;
try {
clusterName = buildUpstreamServiceMetaInfo(metricFamily).getServiceName();
} catch (Exception e) {
log.error("Failed to build upstream serviceMetaInfo from metrics name. ", e);
}
if (StringUtil.isNotEmpty(clusterName)) {
labels.putIfAbsent("cluster_name", clusterName);
}
return labels;
}
protected ServiceMetaInfo buildUpstreamServiceMetaInfo(final Metrics.MetricFamily metricFamily) throws Exception {
String metricsName = metricFamily.getName();
String serviceName = DEFAULT_VALUE;
String ns = DEFAULT_VALUE;
String version = DEFAULT_VALUE;
String[] splitArrGeneral = StringUtils.split(metricsName, ".");
if (metricsName.startsWith("cluster.outbound")) {
String[] splitArrBound = metricsName.split("\\|");
if (splitArrBound.length > 3) {
String[] splitArrClusterName = StringUtils.split(splitArrBound[3], ".");
version = splitArrBound[2];
if (splitArrClusterName.length > 1) {
if (StringUtil.isBlank(version)) {
version = "*";
}
serviceName = splitArrClusterName[0];
ns = splitArrClusterName[1];
}
}
} else if (metricsName.startsWith("cluster.inbound")) {
String[] splitArrBound = metricsName.split("\\|");
if (splitArrBound.length > 1) {
String[] splitArrClusterName = StringUtils.split(splitArrBound[0], ".");
if (splitArrClusterName.length > 1) {
serviceName = splitArrClusterName[1] + ":" + splitArrBound[1];
}
}
} else if (splitArrGeneral.length == 3) {
serviceName = splitArrGeneral[1];
}
Value nsValue = Value.newBuilder().setStringValue(ns).build();
Value nameValue = Value.newBuilder().setStringValue(serviceName).build();
Value versionValue = Value.newBuilder().setStringValue(version).build();
Struct labelStruct = Struct.newBuilder()
.putFields("service.istio.io/canonical-name", nameValue)
.putFields("app.kubernetes.io/name", nameValue)
.putFields("app", nameValue)
.putFields("service.istio.io/canonical-revision", versionValue)
.putFields("version", versionValue).build();
Value label = Value.newBuilder().setStructValue(labelStruct).build();
Struct struct = Struct.newBuilder().putFields("NAMESPACE", nsValue).putFields("LABELS", label).build();
return config.serviceMetaInfoFactory().fromStruct(struct);
}
}
......@@ -18,14 +18,13 @@
package org.apache.skywalking.oap.server.receiver.envoy.metrics.adapters;
import com.google.common.base.Splitter;
import io.prometheus.client.Metrics;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import lombok.RequiredArgsConstructor;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Counter;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Gauge;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Histogram;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Metric;
......@@ -35,9 +34,19 @@ import static java.util.stream.Collectors.toMap;
@RequiredArgsConstructor
public class ProtoMetricFamily2MetricsAdapter {
protected final Metrics.MetricFamily metricFamily;
private final ClusterManagerMetricsAdapter clusterManagerMetricsAdapter;
public Stream<Metric> adapt() {
switch (metricFamily.getType()) {
case COUNTER:
return metricFamily.getMetricList()
.stream()
.map(it -> Counter.builder()
.name(adaptMetricsName(it))
.value(it.getCounter().getValue())
.timestamp(adaptTimestamp(it))
.labels(adaptLabels(it))
.build());
case GAUGE:
return metricFamily.getMetricList()
.stream()
......@@ -63,15 +72,11 @@ public class ProtoMetricFamily2MetricsAdapter {
}
}
@SuppressWarnings("unused")
public String adaptMetricsName(final Metrics.Metric metric) {
if (metricFamily.getName().startsWith("cluster.inbound|")) {
return Splitter.on(".")
.splitToList(metricFamily.getName())
.stream()
.filter(it -> !it.startsWith("inbound|"))
.collect(Collectors.joining("_"));
if (metricFamily.getName().startsWith("cluster.")) {
return clusterManagerMetricsAdapter.adaptMetricsName(metricFamily);
}
return metricFamily.getName();
}
......@@ -80,9 +85,14 @@ public class ProtoMetricFamily2MetricsAdapter {
}
public Map<String, String> adaptLabels(final Metrics.Metric metric) {
return metric.getLabelList()
.stream()
.collect(toMap(Metrics.LabelPair::getName, Metrics.LabelPair::getValue));
Map<String, String> labels = metric.getLabelList()
.stream()
.collect(toMap(Metrics.LabelPair::getName, Metrics.LabelPair::getValue));
if (metricFamily.getName().startsWith("cluster.")) {
return clusterManagerMetricsAdapter.adaptLabels(metricFamily, labels);
}
return labels;
}
public long adaptTimestamp(final Metrics.Metric metric) {
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.skywalking.oap.server.receiver.envoy;
import io.prometheus.client.Metrics;
import java.util.HashMap;
import lombok.SneakyThrows;
import org.apache.skywalking.oap.server.receiver.envoy.als.mx.FieldsHelper;
import org.apache.skywalking.oap.server.receiver.envoy.metrics.adapters.ClusterManagerMetricsAdapter;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.powermock.reflect.Whitebox;
import static org.hamcrest.CoreMatchers.equalTo;
public class ClusterManagerMetricsAdapterTest {
private ClusterManagerMetricsAdapter clusterManagerMetricsAdapter;
private Metrics.MetricFamily generalName = Metrics.MetricFamily.newBuilder().setName("cluster.sds-grpc.upstream_cx_total").build();
private Metrics.MetricFamily cbNameOutboundFQDN = Metrics.MetricFamily.newBuilder().setName("cluster.outbound|9080||reviews.default.svc.cluster.local.circuit_breakers.default.cx_pool_open").build();
private Metrics.MetricFamily cbNameOutboundFQDNSubset = Metrics.MetricFamily.newBuilder().setName("cluster.outbound|9080|v1|reviews.default.svc.cluster.local.circuit_breakers.default.cx_pool_open").build();
private Metrics.MetricFamily cbNameInboundFQDN = Metrics.MetricFamily.newBuilder().setName("cluster.inbound|9080||.upstream_cx_total").build();
@SneakyThrows
@Before
public void setUp() {
Whitebox.setInternalState(FieldsHelper.SINGLETON, "initialized", false);
EnvoyMetricReceiverConfig config = new EnvoyMetricReceiverConfig();
clusterManagerMetricsAdapter = new ClusterManagerMetricsAdapter(config);
FieldsHelper.SINGLETON.init("metadata-service-mapping.yaml", config.serviceMetaInfoFactory().clazz());
}
@Test
public void testAdaptMetricsName() {
Assert.assertThat(clusterManagerMetricsAdapter.adaptMetricsName(generalName), equalTo("envoy_cluster_metrics"));
}
@Test
public void testAdaptLabels() {
Assert.assertThat(
clusterManagerMetricsAdapter.adaptLabels(generalName, new HashMap<>()).toString(),
equalTo("{cluster_name=-.sds-grpc.-, metrics_name=" + generalName.getName() + "}")
);
Assert.assertThat(
clusterManagerMetricsAdapter.adaptLabels(cbNameOutboundFQDN, new HashMap<>()).toString(),
equalTo("{cluster_name=*.reviews.default, metrics_name=" + cbNameOutboundFQDN.getName() + "}")
);
Assert.assertThat(
clusterManagerMetricsAdapter.adaptLabels(cbNameOutboundFQDNSubset, new HashMap<>()).toString(),
equalTo("{cluster_name=v1.reviews.default, metrics_name=" + cbNameOutboundFQDNSubset.getName() + "}")
);
Assert.assertThat(
clusterManagerMetricsAdapter.adaptLabels(cbNameInboundFQDN, new HashMap<>()).toString(),
equalTo("{cluster_name=-.inbound:9080.-, metrics_name=" + cbNameInboundFQDN.getName() + "}")
);
}
}
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
serviceName: ${LABELS."service.istio.io/canonical-revision"}.${LABELS."service.istio.io/canonical-name",LABELS."app.kubernetes.io/name",LABELS.app}.${NAMESPACE}
serviceInstanceName: ${NAME}
......@@ -15,14 +15,14 @@
services:
- key: not null
label: istio-dp::ratings
label: istio-dp::e2e::ratings
- key: not null
label: istio-dp::reviews
label: istio-dp::e2e::reviews
- key: not null
label: istio-dp::productpage
label: istio-dp::e2e::productpage
- key: not null
label: istio-dp::details
label: istio-dp::e2e::details
- key: not null
label: istio-dp::istio-ingressgateway
label: istio-dp::e2e::istio-ingressgateway
- key: not null
label: istio-dp::istio-egressgateway
label: istio-dp::e2e::istio-egressgateway
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册