未验证 提交 b39c4306 编写于 作者: G Gao Hongtao 提交者: GitHub

Observing Istio control plane (#5835)

* Add Istio control plane otel MAL expressions and UI template
* Remove OpenCensus receiver, add OpenTelemetry receiver
* Transfer oc receiver to ot receiver
* Create indices on boot
* Document istio metrics and otel receiver
* Update CHANGES.md
* Fix potential NPE
* Add group to meter and cleanup istio mixer
* Update e2e case due to the change of meter metric name
* Update prometheus fetcher config, fix sql builder issue of h2
* Disable istio and so11y metrics by default.
* Fix micrometer testcase
上级 ec24c039
......@@ -52,6 +52,8 @@ Release Notes.
* Fix the incorrect dynamic configuration key bug of `endpoint-name-grouping`.
* Remove unused min date timebucket in jdbc deletehistory logical
* Fix "transaction too large error" when use TiDB as storage.
* Add otel rules to ui template to observe Istio control plane.
* Remove istio mixer
#### UI
* Fix incorrect label in radial chart in topology.
......@@ -61,6 +63,8 @@ Release Notes.
* Add VNode FAQ doc.
* Add logic endpoint section in the agent setup doc.
* Adjust configuration names and system environment names of the sharing server module
* Tweak Istio metrics collection doc.
* Add otel receiver.
All issues and pull requests are [here](https://github.com/apache/skywalking/milestone/62?closed=1)
......
......@@ -37,7 +37,7 @@ including
1. Java, .NET Core, NodeJS, PHP, and Python auto-instrument agents.
1. Go agent.
1. LUA agent especially for Nginx, OpenResty.
1. Service Mesh Observability. Support Mixer telemetry. Recommend to use Envoy Access Log Service (ALS) for better performance, first introduced at [KubeCon 2019](https://www.youtube.com/watch?v=tERm39ju9ew).
1. Service Mesh Observability. Support Istio telemetry metrics. Recommend to use Envoy Access Log Service (ALS) for better performance, first introduced at [KubeCon 2019](https://www.youtube.com/watch?v=tERm39ju9ew).
1. Metrics system, including Prometheus, OpenTelemetry, Spring Sleuth(Micrometer).
1. Browser application performance, including metrics and error logs.
1. Zipkin v1/v2 and Jaeger gRPC format with limited topology and metrics analysis.(Experimental).
......
......@@ -54,7 +54,7 @@
<include>oal/*.oal</include>
<include>fetcher-prom-rules/*.yaml</include>
<include>meter-analyzer-config/*.yaml</include>
<include>oc-rules/*</include>
<include>otel-oc-rules/*</include>
<include>ui-initialized-templates/*</include>
</includes>
<outputDirectory>/config</outputDirectory>
......
......@@ -54,7 +54,7 @@
<include>oal/*.oal</include>
<include>fetcher-prom-rules/*.yaml</include>
<include>meter-analyzer-config/*.yaml</include>
<include>oc-rules/*</include>
<include>otel-oc-rules/*</include>
<include>ui-initialized-templates/*</include>
</includes>
<outputDirectory>/config</outputDirectory>
......
......@@ -14,9 +14,8 @@ which are wide used. Here is Istio Architecture:
![Istio Architecture](https://istio.io/latest/docs/ops/deployment/architecture/arch.svg)
Service Mesh probe can choose to collect data from **Control Panel** or **Data Panel**. In Istio,
it means collecting telemetry data from Mixer(Control Panel) or Envoy sidecar(Data Panel). Underlying
they are same data, the probe collects two telemetry entities from client side and server side per request.
Service Mesh probe can choose to collect data from **Data Panel**. In Istio, it means collecting telemetry data from
Envoy sidecar(Data Panel). The probe collects two telemetry entities from client side and server side per request.
## How does Service Mesh make backend work?
From the probe, you can see there must have no trace related in this kind of probe, so why SkyWalking
......
......@@ -40,7 +40,7 @@ Note, make sure the [`receiver-browser`](backend/backend-receivers.md) has been
## Service Mesh
- Istio
- [SkyWalking on Istio](istio/README.md). Introduces how to use Istio Mixer bypass adapter to work with SkyWalking.
- [SkyWalking on Istio](istio/README.md). Introduces how to analyze Istio metrics.
- Envoy
- Use [ALS (access log service)](https://www.envoyproxy.io/docs/envoy/latest/api-v2/service/accesslog/v2/als.proto) to observe service mesh, without Mixer. Follow [document](envoy/als_setting.md) for guides.
......
......@@ -46,6 +46,8 @@ staticConfig:
# Labels assigned to all metrics fetched from the targets.
labels:
[ <labelname>: <labelvalue> ... ]
# insert group_name into metric name, for example, meter_<group_name>_<raw_metric_name>
group: <group_name>
# default metric level function appends to all expression in this file.
defaultMetricLevel: <exp>
# Metrics rule allow you to recompute queries.
......
......@@ -9,12 +9,11 @@ We have following receivers, and `default` implementors are provided in our Apac
1. **receiver-register**. gRPC and HTTPRestful services to provide service, service instance and endpoint register.
1. **service-mesh**. gRPC services accept data from inbound mesh probes.
1. **receiver-jvm**. gRPC services accept JVM metrics data.
1. **istio-telemetry**. Istio telemetry is from Istio official bypass adaptor, this receiver match its gRPC services.
1. **envoy-metric**. Envoy `metrics_service` and `ALS(access log service)` supported by this receiver. OAL script support all GAUGE type metrics.
1. **receiver-profile**. gRPC services accept profile task status and snapshot reporter.
1. **receiver_zipkin**. See [details](#zipkin-receiver).
1. **receiver_jaeger**. See [details](#jaeger-receiver).
1. **receiver-oc**. See [details](#opencensus-receiver).
1. **receiver-otel**. See [details](#opentelemetry-receiver).
1. **receiver-meter**. See [details](backend-meter.md).
1. **receiver-browser**. gRPC services to accept browser performance data and error log.
......@@ -36,10 +35,6 @@ service-mesh:
selector: ${SW_SERVICE_MESH:default}
default:
istio-telemetry:
selector: ${SW_ISTIO_TELEMETRY:default}
default:
envoy-metric:
selector: ${SW_ENVOY_METRIC:default}
default:
......@@ -157,21 +152,25 @@ receiver_jaeger:
NOTICE, Jaeger receiver is only provided in `apache-skywalking-apm-x.y.z.tar.gz` tar.
## Opencensus receiver
## OpenTelemetry receiver
Opencensus receiver supports to ingest agent metrics by meter-system. OAP can load the configuration at bootstrap.
If the new configuration is not well-formed, OAP fails to start up. The files are located at `$CLASSPATH/oc-rules`.
OpenTelemetry receiver supports to ingest agent metrics by meter-system. OAP can load the configuration at bootstrap.
If the new configuration is not well-formed, OAP fails to start up. The files are located at `$CLASSPATH/otel-<handler>-rules`.
Eg, the `oc` handler loads fules from `$CLASSPATH/otel-oc-rules`,
The file is written in YAML format, defined by the scheme described in [prometheus-fetcher](./backend-fetcher.md).
Notice, `receiver-oc` only support `metricsRules` node of scheme due to the push mode it opts to.
Supported handlers:
* `oc`: [OpenCensus](https://github.com/open-telemetry/opentelemetry-collector/blob/master/exporter/opencensusexporter/README.md) gRPC service handler.
To active the `default` implementation:
The rule file should be in YAML format, defined by the scheme described in [prometheus-fetcher](./backend-fetcher.md).
Notice, `receiver-otel` only support `group`, `defaultMetricLevel` and `metricsRules` nodes of scheme due to the push mode it opts to.
To active the `oc` handler and `istio` relevant rules:
```yaml
receiver-oc:
selector: ${SW_OC_RECEIVER:-}
receiver-otel:
selector: ${SW_OTEL_RECEIVER:default}
default:
gRPCHost: ${SW_OC_RECEIVER_GRPC_HOST:0.0.0.0}
gRPCPort: ${SW_OC_RECEIVER_GRPC_PORT:55678}
enabledHandlers: ${SW_OTEL_RECEIVER_ENABLED_HANDLERS:"oc"}
enabledOcRules: ${SW_OTEL_RECEIVER_ENABLED_OC_RULES:"istio-controlplane"}
```
## Meter receiver
......
......@@ -171,18 +171,13 @@ core|default|role|Option values, `Mixed/Receiver/Aggregator`. **Receiver** mode
| receiver-clr| default| Read [receiver doc](backend-receivers.md) for more details | - | - |
| receiver-profile| default| Read [receiver doc](backend-receivers.md) for more details | - | - |
| service-mesh| default| Read [receiver doc](backend-receivers.md) for more details | - | - |
| istio-telemetry| default| Read [receiver doc](backend-receivers.md) for more details | - | - |
| envoy-metric| default| Read [receiver doc](backend-receivers.md) for more details | - | - |
| - | - | acceptMetricsService | Open Envoy Metrics Service analysis | SW_ENVOY_METRIC_SERVICE | true|
| - | - | alsHTTPAnalysis | Open Envoy Access Log Service analysis. Value = `k8s-mesh` means open the analysis | SW_ENVOY_METRIC_ALS_HTTP_ANALYSIS | - |
| - | - | k8sServiceNameRule | `k8sServiceNameRule` allows you to customize the service name in ALS via Kubernetes metadata, the available variables are `pod`, `service`, e.g., you can use `${service.metadata.name}-${pod.metadata.labels.version}` to append the version number to the service name. Be careful, when using environment variables to pass this configuration, use single quotes(`''`) to avoid it being evaluated by the shell. | - |
| receiver-oc | default | Read [receiver doc](backend-receivers.md) for more details | - | - |
| - | - | gRPCHost|Binding IP of gRPC service. Services include gRPC data report and internal communication among OAP nodes| SW_OC_RECEIVER_GRPC_HOST | - |
| - | - | gRPCPort| Binding port of gRPC service | SW_OC_RECEIVER_GRPC_PORT | - |
| - | - | gRPCThreadPoolSize|Pool size of gRPC server| - | CPU core * 4|
| - | - | gRPCThreadPoolQueueSize| The queue size of gRPC server| - | 10000|
| - | - | maxConcurrentCallsPerConnection | The maximum number of concurrent calls permitted for each incoming connection. Defaults to no limit. | - | - |
| - | - | maxMessageSize | Sets the maximum message size allowed to be received on the server. Empty means 4 MiB | - | 4M(based on Netty) |
| receiver-otel | default | Read [receiver doc](backend-receivers.md) for more details | - | - |
| - | - | enabledHandlers|Enabled handlers for otel| SW_OTEL_RECEIVER_ENABLED_HANDLERS | - |
| - | - | enabledOcRules|Enabled metric rules for OC handler | SW_OTEL_RECEIVER_ENABLED_OC_RULES | - |
| receiver_zipkin |default| Read [receiver doc](backend-receivers.md) | - | - |
| - | - | restHost| Binding IP of restful service. |SW_RECEIVER_ZIPKIN_HOST|0.0.0.0|
| - | - | restPort | Binding port of restful service | SW_RECEIVER_ZIPKIN_PORT|9411|
......
# Work with Istio
Instructions for transport Istio's metrics to SkyWalking OAP server.
Instructions for transport Istio's metrics to the SkyWalking OAP server.
## Prerequisites
Istio should be installed in kubernetes cluster. Follow [Istio getting start](https://istio.io/docs/setup/getting-started/)
Istio should be installed in the kubernetes cluster. Follow [Istio getting start](https://istio.io/docs/setup/getting-started/)
to finish it.
## Deploy Skywalking backend
Follow the [deploying backend in kubernetes](../backend/backend-k8s.md) to install oap server in kubernetes cluster.
Follow the [deploying backend in kubernetes](../backend/backend-k8s.md) to install the OAP server in the kubernetes cluster.
Referring to [OpenTelemetry receiver](../backend/backend-receivers.md#OpenTelemetry-receiver) to ingest metrics.
`otel-receiver` defaults to be inactive. Set env var `SW_OTEL_RECEIVER` to `default` to enable it.
## Setup Istio to send metrics to oap
Our scripts are wrote based on Istio 1.3.3.
## Deploy OpenTelemetry collector
OpenTelemetry collector is the location Istio telemetry sends metrics, then processing and sending them to SkyWalking
backend.
1. Install Istio metric template
Following the [Getting Started](https://opentelemetry.io/docs/collector/getting-started/) to deploy this collector. There
are several components available in the collector, and they could be combined for different scenarios.
For the sake of brevity, we use the Prometheus receiver to retrieve metrics from Istio control and data plane,
then send them to SkyWalking by OpenCensus exporter.
`kubectl apply -f https://raw.githubusercontent.com/istio/istio/1.3.3/mixer/template/metric/template.yaml`
#### Prometheus receiver
Refer to [Prometheus Receiver](https://github.com/open-telemetry/opentelemetry-collector/blob/master/receiver/prometheusreceiver/README.md)
to set up this receiver. you could find more configuration details in [Prometheus Integration of Istio](https://istio.io/latest/docs/ops/integrations/prometheus/#configuration)
to figure out how to direct Prometheus receiver to query Istio metrics.
SkyWalking supports receiving multi-cluster metrics in a single OAP cluster. A `cluster` label should be appended to every metric
fetched by this receiver even there's only a single cluster needed to be collected.
You could leverage `relabel` to add it like below:
2. Install SkyWalking adapter
```
relabel_configs:
- source_labels: []
target_label: cluster
replacement: <cluster name>
```
`kubectl apply -f skywalkingadapter.yml`
or opt to [Resource Processor](https://github.com/open-telemetry/opentelemetry-collector/blob/master/processor/resourceprocessor/README.md):
Find the `skywalkingadapter.yml` at [here](yaml/skywalkingadapter.yml).
```
processors:
resource:
attributes:
- key: cluster
value: "<cluster name>"
action: upsert
```
NOTICE, due to Istio Mixer is default OFF, we recommend you to consider our [ALS solution](../envoy/als_setting.md)
\ No newline at end of file
Notice, if you try the sample of istio Prometheus Kubernetes configuration,
the issues described [here](https://github.com/open-telemetry/opentelemetry-collector/issues/2163) might block you.
Try to use the solution indicated in this issue if it's not fixed.
#### OpenCensus exporter
Follow [OpenCensus exporter configuration](https://github.com/open-telemetry/opentelemetry-collector/blob/master/exporter/opencensusexporter/README.md)
to set up a connection between OpenTelemetry collector and OAP cluster. `endpoint` is the address of OAP gRPC service.
## Observe Istio
Open Istio Dashboard in SkyWaling UI by clicking `Dashboard` -> `Istio`, then you're able to view charts and diagrams
generated by Istio metrics. You also could view them by `swctl` and set up alarm rules based on them.
NOTICE, if you want metrics of Istio managed services, including topology among them, we recommend you to consider our [ALS solution](../envoy/als_setting.md)
# 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.
apiVersion: "config.istio.io/v1alpha2"
kind: adapter
metadata:
name: swadapter
namespace: istio-system
spec:
description:
session_based: false
templates:
- metric
---
apiVersion: "config.istio.io/v1alpha2"
kind: handler
metadata:
name: sw
namespace: istio-system
spec:
adapter: swadapter
connection:
address: "oap.skywalking.svc.cluster.local:11800"
---
# instance for template metric
apiVersion: "config.istio.io/v1alpha2"
kind: instance
metadata:
name: swmetric
namespace: istio-system
spec:
template: metric
params:
value: request.size | 0
dimensions:
sourceService: source.workload.name | ""
sourceNamespace: source.workload.namespace | ""
sourceUID: source.uid | ""
destinationService: destination.workload.name | ""
destinationNamespace: destination.workload.namespace | ""
destinationUID: destination.uid | ""
requestMethod: request.method | ""
requestPath: request.path | ""
requestScheme: request.scheme | ""
requestTime: request.time
responseTime: response.time
responseCode: response.code | 200
reporter: conditional((context.reporter.kind | "inbound") == "outbound", "source", "destination")
apiProtocol: api.protocol | ""
---
# rule to dispatch to handler sw
apiVersion: "config.istio.io/v1alpha2"
kind: rule
metadata:
name: swmetric-rule
namespace: istio-system
spec:
actions:
- handler: sw.istio-system
instances:
- swmetric
......@@ -31,7 +31,9 @@ import lombok.RequiredArgsConstructor;
import lombok.ToString;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.oap.meter.analyzer.dsl.DSL;
import org.apache.skywalking.oap.meter.analyzer.dsl.DownsamplingType;
import org.apache.skywalking.oap.meter.analyzer.dsl.Expression;
import org.apache.skywalking.oap.meter.analyzer.dsl.ExpressionParsingContext;
import org.apache.skywalking.oap.meter.analyzer.dsl.Result;
import org.apache.skywalking.oap.meter.analyzer.dsl.Sample;
import org.apache.skywalking.oap.meter.analyzer.dsl.SampleFamily;
......@@ -68,7 +70,10 @@ public class Analyzer {
public static Analyzer build(final String metricName, final String expression,
final MeterSystem meterSystem) {
Expression e = DSL.parse(expression);
return new Analyzer(metricName, e, meterSystem);
ExpressionParsingContext ctx = e.parse();
Analyzer analyzer = new Analyzer(metricName, e, meterSystem);
analyzer.init(ctx);
return analyzer;
}
private static final String FUNCTION_NAME_TEMP = "%s%s";
......@@ -79,7 +84,9 @@ public class Analyzer {
private final MeterSystem meterSystem;
private boolean createdMetric;
private MetricType metricType;
private int[] percentiles;
/**
* analyse intends to parse expression with input samples to meter-system metrics.
......@@ -91,54 +98,64 @@ public class Analyzer {
if (!r.isSuccess()) {
return;
}
SampleFamily.Context ctx = r.getData().context;
SampleFamily.RunningContext ctx = r.getData().context;
Sample[] ss = r.getData().samples;
generateTraffic(ctx.getMeterEntity());
if (ctx.isHistogram()) {
Stream.of(ss).map(s -> Tuple.of(composeGroup(s.getLabels(), k -> !Objects.equals("le", k)), s))
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())))
.forEach((group, subSs) -> {
if (subSs.size() < 1) {
return;
}
long[] bb = new long[subSs.size()];
long[] vv = new long[bb.length];
for (int i = 0; i < subSs.size(); i++) {
Sample s = subSs.get(i);
bb[i] = Long.parseLong(s.getLabels().get("le"));
vv[i] = (long) s.getValue();
}
BucketedValues bv = new BucketedValues(bb, vv);
long time = subSs.get(0).getTimestamp();
if (ctx.getPercentiles() == null || ctx.getPercentiles().length < 1) {
Preconditions.checkState(createMetric(ctx.getMeterEntity().getScopeType(), "histogram", ctx));
AcceptableValue<BucketedValues> v = meterSystem.buildMetrics(metricName, BucketedValues.class);
v.accept(ctx.getMeterEntity(), bv);
send(v, time);
return;
}
Preconditions.checkState(createMetric(ctx.getMeterEntity().getScopeType(), "histogramPercentile", ctx));
AcceptableValue<PercentileArgument> v = meterSystem.buildMetrics(metricName, PercentileArgument.class);
v.accept(ctx.getMeterEntity(), new PercentileArgument(bv, ctx.getPercentiles()));
send(v, time);
});
return;
switch (metricType) {
case single:
AcceptableValue<Long> sv = meterSystem.buildMetrics(metricName, Long.class);
sv.accept(ctx.getMeterEntity(), getValue(ss[0]));
send(sv, ss[0].getTimestamp());
break;
case labeled:
AcceptableValue<DataTable> lv = meterSystem.buildMetrics(metricName, DataTable.class);
DataTable dt = new DataTable();
for (Sample each : ss) {
dt.put(composeGroup(each.getLabels()), getValue(each));
}
lv.accept(ctx.getMeterEntity(), dt);
send(lv, ss[0].getTimestamp());
break;
case histogram:
case histogramPercentile:
Stream.of(ss).map(s -> Tuple.of(composeGroup(s.getLabels(), k -> !Objects.equals("le", k)), s))
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())))
.forEach((group, subSs) -> {
if (subSs.size() < 1) {
return;
}
long[] bb = new long[subSs.size()];
long[] vv = new long[bb.length];
for (int i = 0; i < subSs.size(); i++) {
Sample s = subSs.get(i);
bb[i] = Long.parseLong(s.getLabels().get("le"));
vv[i] = getValue(s);
}
BucketedValues bv = new BucketedValues(bb, vv);
long time = subSs.get(0).getTimestamp();
if (metricType == MetricType.histogram) {
AcceptableValue<BucketedValues> v = meterSystem.buildMetrics(metricName, BucketedValues.class);
v.accept(ctx.getMeterEntity(), bv);
send(v, time);
return;
}
AcceptableValue<PercentileArgument> v = meterSystem.buildMetrics(metricName, PercentileArgument.class);
v.accept(ctx.getMeterEntity(), new PercentileArgument(bv, percentiles));
send(v, time);
});
break;
}
if (ss.length == 1) {
Preconditions.checkState(createMetric(ctx.getMeterEntity().getScopeType(), "", ctx));
AcceptableValue<Long> v = meterSystem.buildMetrics(metricName, Long.class);
v.accept(ctx.getMeterEntity(), (long) ss[0].getValue());
send(v, ss[0].getTimestamp());
return;
}
private long getValue(Sample sample) {
if (sample.getValue() <= 0.0) {
return 0L;
}
Preconditions.checkState(createMetric(ctx.getMeterEntity().getScopeType(), "labeled", ctx));
AcceptableValue<DataTable> v = meterSystem.buildMetrics(metricName, DataTable.class);
DataTable dt = new DataTable();
for (Sample each : ss) {
dt.put(composeGroup(each.getLabels()), (long) each.getValue());
if (sample.getValue() < 1.0) {
return 1L;
}
v.accept(ctx.getMeterEntity(), dt);
send(v, ss[0].getTimestamp());
return Math.round(sample.getValue());
}
private String composeGroup(ImmutableMap<String, String> labels) {
......@@ -150,12 +167,41 @@ public class Analyzer {
.collect(Collectors.joining("-"));
}
private boolean createMetric(final ScopeType scopeType, final String dataType, final SampleFamily.Context ctx) {
if (createdMetric) {
return true;
@RequiredArgsConstructor
private enum MetricType {
// metrics is aggregated by histogram function.
histogram("histogram"),
// metrics is aggregated by histogram based percentile function.
histogramPercentile("histogramPercentile"),
// metrics is aggregated by labeled function.
labeled("labeled"),
// metrics is aggregated by single value function.
single("");
private final String literal;
}
private void init(final ExpressionParsingContext ctx) {
if (ctx.isHistogram()) {
if (ctx.getPercentiles() != null && ctx.getPercentiles().length > 0) {
metricType = MetricType.histogramPercentile;
this.percentiles = ctx.getPercentiles();
} else {
metricType = MetricType.histogram;
}
} else {
if (ctx.getLabels().isEmpty()) {
metricType = MetricType.single;
} else {
metricType = MetricType.labeled;
}
}
String functionName = String.format(FUNCTION_NAME_TEMP, ctx.getDownsampling().toString().toLowerCase(), Strings.capitalize(dataType));
return meterSystem.create(metricName, functionName, scopeType) && (createdMetric = true);
Preconditions.checkState(createMetric(ctx.getScopeType(), metricType.literal, ctx.getDownsampling()));
}
private boolean createMetric(final ScopeType scopeType, final String dataType, final DownsamplingType downsamplingType) {
String functionName = String.format(FUNCTION_NAME_TEMP, downsamplingType.toString().toLowerCase(), Strings.capitalize(dataType));
return meterSystem.create(metricName, functionName, scopeType);
}
private void send(final AcceptableValue<?> v, final long time) {
......
......@@ -26,7 +26,7 @@ import java.util.StringJoiner;
import java.util.stream.Stream;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.oap.meter.analyzer.dsl.SampleFamily;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.MetricsRule;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rule;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterSystem;
import org.elasticsearch.common.Strings;
......@@ -47,9 +47,10 @@ public class MetricConvert {
private final List<Analyzer> analyzers;
public MetricConvert(List<MetricsRule> rules, String defaultMetricLevel, MeterSystem service) {
this.analyzers = rules.stream().map(r -> Analyzer.build(formatMetricName(r.getName()),
Strings.isEmpty(defaultMetricLevel) ? r.getExp() : String.format("(%s).%s", r.getExp(), defaultMetricLevel), service))
public MetricConvert(Rule rule, MeterSystem service) {
Preconditions.checkState(!Strings.isNullOrEmpty(rule.getGroup()));
this.analyzers = rule.getMetricsRules().stream().map(r -> Analyzer.build(formatMetricName(rule, r.getName()),
Strings.isEmpty(rule.getDefaultMetricLevel()) ? r.getExp() : String.format("(%s).%s", r.getExp(), rule.getDefaultMetricLevel()), service))
.collect(toList());
}
......@@ -72,9 +73,9 @@ public class MetricConvert {
}
}
private String formatMetricName(String meterRuleName) {
private String formatMetricName(Rule rule, String meterRuleName) {
StringJoiner metricName = new StringJoiner("_");
metricName.add("meter").add(meterRuleName);
metricName.add("meter").add(rule.getGroup()).add(meterRuleName);
return metricName.toString();
}
}
......@@ -39,6 +39,6 @@ public final class DSL {
cc.setScriptBaseClass(DelegatingScript.class.getName());
GroovyShell sh = new GroovyShell(new Binding(), cc);
DelegatingScript script = (DelegatingScript) sh.parse(expression);
return new Expression(script);
return new Expression(expression, script);
}
}
......@@ -23,18 +23,34 @@ import groovy.lang.ExpandoMetaClass;
import groovy.lang.GroovyObjectSupport;
import groovy.util.DelegatingScript;
import java.time.Instant;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
/**
* Expression is a reusable monadic container type which represents a DSL expression.
*/
@Slf4j
@RequiredArgsConstructor
public class Expression {
private final String literal;
private final DelegatingScript expression;
Expression(DelegatingScript expression) {
this.expression = expression;
/**
* Parse the expression statically.
*
* @return Parsed context of the expression.
*/
public ExpressionParsingContext parse() {
try (ExpressionParsingContext ctx = ExpressionParsingContext.create()) {
Result r = run(ImmutableMap.of());
if (!r.isSuccess() && r.isThrowable()) {
throw new ExpressionParsingException("failed to parse expression: " + literal + ", error:" + r.getError());
}
ctx.validate(literal);
return ctx;
}
}
/**
......@@ -57,18 +73,12 @@ public class Expression {
}
public SampleFamily avg(SampleFamily sf) {
if (sf == SampleFamily.EMPTY) {
return SampleFamily.EMPTY;
}
sf.context.downsampling = DownsamplingType.AVG;
ExpressionParsingContext.get().ifPresent(ctx -> ctx.downsampling = DownsamplingType.AVG);
return sf;
}
public SampleFamily latest(SampleFamily sf) {
if (sf == SampleFamily.EMPTY) {
return SampleFamily.EMPTY;
}
sf.context.downsampling = DownsamplingType.LATEST;
ExpressionParsingContext.get().ifPresent(ctx -> ctx.downsampling = DownsamplingType.LATEST);
return sf;
}
......
/*
* 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.meter.analyzer.dsl;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import lombok.Builder;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.ToString;
import org.apache.skywalking.oap.server.core.analysis.meter.ScopeType;
/**
* ExpressionParsingContext contains states in parsing phase of an expression.
*/
@Getter
@ToString
@EqualsAndHashCode
@Builder
public class ExpressionParsingContext implements Closeable {
static ExpressionParsingContext create() {
if (CACHE.get() == null) {
CACHE.set(ExpressionParsingContext.builder()
.downsampling(DownsamplingType.AVG)
.scopeLabels(Lists.newArrayList())
.aggregationLabels(Lists.newArrayList()).build());
}
return CACHE.get();
}
static Optional<ExpressionParsingContext> get() {
return Optional.ofNullable(CACHE.get());
}
private final static ThreadLocal<ExpressionParsingContext> CACHE = new ThreadLocal<>();
boolean isHistogram;
int[] percentiles;
List<String> aggregationLabels;
List<String> scopeLabels;
DownsamplingType downsampling;
ScopeType scopeType;
/**
* Get labels no scope related.
*
* @return labels
*/
public List<String> getLabels() {
List<String> result = new ArrayList<>(aggregationLabels);
result.removeAll(scopeLabels);
return result;
}
/**
* Validate context after parsing
* @param exp expression literal
*/
public void validate(String exp) {
Preconditions.checkNotNull(scopeType, exp + ": one of service(), instance() or endpoint() should be invoke");
}
@Override
public void close() {
CACHE.remove();
}
}
......@@ -16,20 +16,13 @@
*
*/
package org.apache.skywalking.oap.server.receiver.opencensus;
package org.apache.skywalking.oap.meter.analyzer.dsl;
import lombok.Getter;
import lombok.Setter;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
@Getter
@Setter
public class OCMetricReceiverConfig extends ModuleConfig {
private String gRPCHost = "0.0.0.0";
private int gRPCPort = -1;
private int maxConcurrentCallsPerConnection;
private int maxMessageSize;
private int gRPCThreadPoolSize;
private int gRPCThreadPoolQueueSize;
private String rulePath = "oc-rules";
/**
* ExpressionParsingException is throw in expression parsing phase.
*/
public class ExpressionParsingException extends RuntimeException {
public ExpressionParsingException(final String message) {
super(message);
}
}
......@@ -43,7 +43,7 @@ public class Result {
*/
public static Result fail(final Throwable throwable) {
log.info("Expression fails: {}", throwable.getMessage());
return new Result(false, SampleFamily.EMPTY);
return new Result(false, true, throwable.getMessage(), SampleFamily.EMPTY);
}
/**
......@@ -54,7 +54,7 @@ public class Result {
*/
public static Result fail(String message) {
log.info("Expression fails: {}", message);
return new Result(false, SampleFamily.EMPTY);
return new Result(false, false, message, SampleFamily.EMPTY);
}
/**
......@@ -64,7 +64,7 @@ public class Result {
*/
public static Result fail() {
log.info("Expression fails");
return new Result(false, SampleFamily.EMPTY);
return new Result(false, false, null, SampleFamily.EMPTY);
}
/**
......@@ -77,10 +77,14 @@ public class Result {
if (log.isDebugEnabled()) {
log.debug("Result is successful, sample family is {}", sf);
}
return new Result(true, sf);
return new Result(true, false, null, sf);
}
private final boolean success;
private final boolean isThrowable;
private final String error;
private final SampleFamily data;
}
......@@ -18,6 +18,7 @@
package org.apache.skywalking.oap.meter.analyzer.dsl;
import com.google.common.base.CharMatcher;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
......@@ -47,6 +48,7 @@ import lombok.RequiredArgsConstructor;
import lombok.Setter;
import lombok.ToString;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterEntity;
import org.apache.skywalking.oap.server.core.analysis.meter.ScopeType;
import static java.util.stream.Collectors.groupingBy;
import static java.util.stream.Collectors.mapping;
......@@ -59,40 +61,21 @@ import static java.util.stream.Collectors.toList;
@EqualsAndHashCode
@ToString
public class SampleFamily {
public static final SampleFamily EMPTY = new SampleFamily(new Sample[0], Context.EMPTY);
public static final SampleFamily EMPTY = new SampleFamily(new Sample[0], RunningContext.EMPTY);
public static SampleFamily build(Sample... samples) {
return build(null, samples);
}
static SampleFamily build(Context ctx, Sample... samples) {
static SampleFamily build(RunningContext ctx, Sample... samples) {
Preconditions.checkNotNull(samples);
Preconditions.checkArgument(samples.length > 0);
return new SampleFamily(samples, Optional.ofNullable(ctx).orElseGet(Context::instance));
}
static SampleFamily buildHistogram(Sample... samples) {
return buildHistogram(null, samples);
}
static SampleFamily buildHistogram(Context ctx, Sample... samples) {
Preconditions.checkNotNull(samples);
Preconditions.checkArgument(samples.length > 0);
ctx = Optional.ofNullable(ctx).orElseGet(Context::instance);
ctx.isHistogram = true;
return new SampleFamily(samples, ctx);
}
static SampleFamily buildHistogramPercentile(SampleFamily histogram, int[] percentiles) {
Preconditions.checkNotNull(histogram);
Preconditions.checkArgument(histogram.context.isHistogram);
histogram.context.percentiles = percentiles;
return new SampleFamily(histogram.samples, histogram.context);
return new SampleFamily(samples, Optional.ofNullable(ctx).orElseGet(RunningContext::instance));
}
public final Sample[] samples;
public final Context context;
public final RunningContext context;
/**
* Following operations are used in DSL
......@@ -181,6 +164,7 @@ public class SampleFamily {
/* Aggregation operators */
public SampleFamily sum(List<String> by) {
ExpressionParsingContext.get().ifPresent(ctx -> ctx.aggregationLabels.addAll(by));
if (this == EMPTY) {
return EMPTY;
}
......@@ -260,12 +244,13 @@ public class SampleFamily {
public SampleFamily histogram(String le, TimeUnit unit) {
long scale = unit.toMillis(1);
Preconditions.checkArgument(scale > 0);
ExpressionParsingContext.get().ifPresent(ctx -> ctx.isHistogram = true);
if (this == EMPTY) {
return EMPTY;
}
AtomicDouble pre = new AtomicDouble();
AtomicReference<String> preLe = new AtomicReference<>("0");
return SampleFamily.buildHistogram(this.context, Stream.concat(
return SampleFamily.build(this.context, Stream.concat(
Arrays.stream(samples).filter(s -> !s.labels.containsKey(le)),
Arrays.stream(samples)
.filter(s -> s.labels.containsKey(le))
......@@ -283,14 +268,20 @@ public class SampleFamily {
public SampleFamily histogram_percentile(List<Integer> percentiles) {
Preconditions.checkArgument(percentiles.size() > 0);
if (this == EMPTY) {
return EMPTY;
}
return SampleFamily.buildHistogramPercentile(this, percentiles.stream().mapToInt(i -> i).toArray());
int[] p = percentiles.stream().mapToInt(i -> i).toArray();
ExpressionParsingContext.get().ifPresent(ctx -> {
Preconditions.checkState(ctx.isHistogram, "histogram() should be invoked before invoking histogram_percentile()");
ctx.percentiles = p;
});
return this;
}
public SampleFamily service(List<String> labelKeys) {
Preconditions.checkArgument(labelKeys.size() > 0);
ExpressionParsingContext.get().ifPresent(ctx -> {
ctx.scopeType = ScopeType.SERVICE;
ctx.scopeLabels.addAll(labelKeys);
});
if (this == EMPTY) {
return EMPTY;
}
......@@ -301,6 +292,12 @@ public class SampleFamily {
public SampleFamily instance(List<String> serviceKeys, List<String> instanceKeys) {
Preconditions.checkArgument(serviceKeys.size() > 0);
Preconditions.checkArgument(instanceKeys.size() > 0);
ExpressionParsingContext.get().ifPresent(ctx -> ctx.scopeType = ScopeType.SERVICE_INSTANCE);
ExpressionParsingContext.get().ifPresent(ctx -> {
ctx.scopeType = ScopeType.SERVICE_INSTANCE;
ctx.scopeLabels.addAll(serviceKeys);
ctx.scopeLabels.addAll(instanceKeys);
});
if (this == EMPTY) {
return EMPTY;
}
......@@ -311,6 +308,12 @@ public class SampleFamily {
public SampleFamily endpoint(List<String> serviceKeys, List<String> endpointKeys) {
Preconditions.checkArgument(serviceKeys.size() > 0);
Preconditions.checkArgument(endpointKeys.size() > 0);
ExpressionParsingContext.get().ifPresent(ctx -> ctx.scopeType = ScopeType.ENDPOINT);
ExpressionParsingContext.get().ifPresent(ctx -> {
ctx.scopeType = ScopeType.ENDPOINT;
ctx.scopeLabels.addAll(serviceKeys);
ctx.scopeLabels.addAll(endpointKeys);
});
if (this == EMPTY) {
return EMPTY;
}
......@@ -319,7 +322,8 @@ public class SampleFamily {
}
private String dim(List<String> labelKeys) {
return labelKeys.stream().map(k -> samples[0].labels.getOrDefault(k, "")).collect(Collectors.joining("."));
String name = labelKeys.stream().map(k -> samples[0].labels.getOrDefault(k, "")).collect(Collectors.joining("."));
return CharMatcher.is('.').trimFrom(name);
}
private SampleFamily left(List<String> labelKeys) {
......@@ -349,6 +353,9 @@ public class SampleFamily {
}
SampleFamily newValue(Function<Double, Double> transform) {
if (this == EMPTY) {
return EMPTY;
}
Sample[] ss = new Sample[samples.length];
for (int i = 0; i < ss.length; i++) {
ss[i] = samples[i].newValue(transform);
......@@ -392,20 +399,14 @@ public class SampleFamily {
@Getter
@Setter
@Builder
public static class Context {
public static class RunningContext {
static Context EMPTY = Context.builder().build();
static RunningContext EMPTY = RunningContext.builder().build();
static Context instance() {
return Context.builder().downsampling(DownsamplingType.AVG).build();
static RunningContext instance() {
return RunningContext.builder().build();
}
boolean isHistogram;
int[] percentiles;
DownsamplingType downsampling;
MeterEntity meterEntity;
}
}
......@@ -22,14 +22,13 @@ import com.google.common.collect.ImmutableMap;
import io.vavr.Tuple;
import io.vavr.Tuple2;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.stream.Stream;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.oap.meter.analyzer.MetricConvert;
import org.apache.skywalking.oap.meter.analyzer.dsl.Sample;
import org.apache.skywalking.oap.meter.analyzer.dsl.SampleFamily;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.MetricsRule;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rule;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterSystem;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Counter;
import org.apache.skywalking.oap.server.library.util.prometheus.metrics.Gauge;
......@@ -53,8 +52,8 @@ public class PrometheusMetricConverter {
private final MetricConvert convert;
public PrometheusMetricConverter(List<MetricsRule> rules, String defaultMetricLevel, MeterSystem service) {
this.convert = new MetricConvert(rules, defaultMetricLevel, service);
public PrometheusMetricConverter(Rule rule, MeterSystem service) {
this.convert = new MetricConvert(rule, service);
}
/**
......@@ -67,7 +66,7 @@ public class PrometheusMetricConverter {
convert.toMeter(data);
}
private ImmutableMap<String, SampleFamily> convertPromMetricToSampleFamily(Stream<Metric> metricStream) {
public ImmutableMap<String, SampleFamily> convertPromMetricToSampleFamily(Stream<Metric> metricStream) {
return metricStream
.peek(metric -> log.debug("Prom metric to be convert to SampleFamily: {}", metric))
.flatMap(this::convertMetric)
......
......@@ -33,6 +33,7 @@ public class Rule {
private String fetcherTimeout;
private String metricsPath;
private StaticConfig staticConfig;
private String group;
private String defaultMetricLevel;
private List<MetricsRule> metricsRules;
}
......@@ -24,6 +24,7 @@ import java.io.FileReader;
import java.io.IOException;
import java.io.Reader;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
......@@ -40,6 +41,10 @@ public class Rules {
private static final Logger LOG = LoggerFactory.getLogger(Rule.class);
public static List<Rule> loadRules(final String path) throws ModuleStartException {
return loadRules(path, Collections.emptyList());
}
public static List<Rule> loadRules(final String path, List<String> enabledRules) throws ModuleStartException {
File[] rules;
try {
rules = ResourceUtils.getPathFiles(path);
......@@ -50,8 +55,14 @@ public class Rules {
.filter(File::isFile)
.map(f -> {
try (Reader r = new FileReader(f)) {
String fileName = f.getName();
int dotIndex = fileName.lastIndexOf('.');
fileName = (dotIndex == -1) ? fileName : fileName.substring(0, dotIndex);
if (!enabledRules.contains(fileName)) {
return null;
}
Rule rule = new Yaml().loadAs(r, Rule.class);
rule.setName(f.getName().replace(".", "_"));
rule.setName(fileName);
return rule;
} catch (IOException e) {
LOG.debug("Reading file {} failed", f, e);
......
......@@ -130,7 +130,7 @@ public class ArithmeticTest {
of("http_success_request", SampleFamily.EMPTY,
"http_error_request", SampleFamily.EMPTY),
"http_success_request + http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -186,7 +186,7 @@ public class ArithmeticTest {
of("http_success_request", SampleFamily.EMPTY,
"http_error_request", SampleFamily.EMPTY),
"http_success_request - http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -242,7 +242,7 @@ public class ArithmeticTest {
of("http_success_request", SampleFamily.EMPTY,
"http_error_request", SampleFamily.EMPTY),
"http_success_request * http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -253,7 +253,7 @@ public class ArithmeticTest {
Sample.builder().labels(of("idc", "t2")).value(3).build()
)),
"http_success_request * http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -264,7 +264,7 @@ public class ArithmeticTest {
Sample.builder().labels(of("idc", "t2")).value(3).build()
)),
"http_error_request * http_success_request ",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -292,7 +292,7 @@ public class ArithmeticTest {
of("http_success_request", SampleFamily.EMPTY,
"http_error_request", SampleFamily.EMPTY),
"http_success_request / http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -303,7 +303,7 @@ public class ArithmeticTest {
Sample.builder().labels(of("idc", "t2")).value(3).build()
)),
"http_success_request / http_error_request",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......
......@@ -57,7 +57,7 @@ public class BasicTest {
"default",
of("instance_cpu_percentage", SampleFamily.EMPTY),
"instance_cpu_percentage",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......@@ -67,22 +67,6 @@ public class BasicTest {
Result.success(SampleFamily.build(Sample.builder().value(1600592418480.0).build())),
false,
},
{
"downsampling-avg",
of("instance_cpu_percentage", SampleFamily.build(Sample.builder().value(1600592418480.0).build())),
"avg instance_cpu_percentage",
Result.success(SampleFamily.build(SampleFamily.Context.builder().downsampling(DownsamplingType.AVG).build()
, Sample.builder().value(1600592418480.0).build())),
false,
},
{
"downsampling-latest",
of("instance_cpu_percentage", SampleFamily.build(Sample.builder().value(1600592418480.0).build())),
"latest instance_cpu_percentage",
Result.success(SampleFamily.build(SampleFamily.Context.builder().downsampling(DownsamplingType.LATEST).build()
, Sample.builder().value(1600592418480.0).build())),
false,
}
});
}
......
/*
* 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.meter.analyzer.dsl;
import com.google.common.collect.Lists;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.oap.server.core.analysis.meter.ScopeType;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.fail;
@Slf4j
@RunWith(Parameterized.class)
public class ExpressionParsingTest {
@Parameterized.Parameter
public String name;
@Parameterized.Parameter(1)
public String expression;
@Parameterized.Parameter(2)
public ExpressionParsingContext want;
@Parameterized.Parameter(3)
public boolean isThrow;
@Parameterized.Parameters(name = "{index}: {0}")
public static Collection<Object[]> data() {
return Arrays.asList(new Object[][] {
{
"mini",
"foo.instance(['service'], ['host'])",
ExpressionParsingContext.builder()
.downsampling(DownsamplingType.AVG)
.scopeLabels(Arrays.asList("service", "host"))
.scopeType(ScopeType.SERVICE_INSTANCE)
.aggregationLabels(Lists.newArrayList()).build(),
false,
},
{
"all",
"latest (foo - 1).tagEqual('bar', '1').sum(['tt']).irate().histogram().histogram_percentile([50,99]).service(['rr'])",
ExpressionParsingContext.builder()
.scopeType(ScopeType.SERVICE)
.scopeLabels(Collections.singletonList("rr"))
.aggregationLabels(Collections.singletonList("tt"))
.downsampling(DownsamplingType.LATEST)
.isHistogram(true)
.percentiles(new int[]{50, 99}).build(),
false,
},
});
}
@Test
public void test() {
Expression e = DSL.parse(expression);
ExpressionParsingContext r = null;
try {
r = e.parse();
} catch (Throwable t) {
if (isThrow) {
return;
}
log.error("Test failed", t);
fail("Should not throw anything");
}
if (isThrow) {
fail("Should throw something");
}
assertThat(r, is(want));
}
}
\ No newline at end of file
......@@ -90,7 +90,7 @@ public class FunctionTest {
Sample.builder().labels(of("le", String.valueOf(Integer.MAX_VALUE))).value(410).build())
),
"instance_cpu_percentage.histogram()",
Result.success(SampleFamily.buildHistogram(
Result.success(SampleFamily.build(
Sample.builder().labels(of("le", "0")).value(100).build(),
Sample.builder().labels(of("le", "25")).value(22).build(),
Sample.builder().labels(of("le", "750")).value(178).build(),
......@@ -107,12 +107,12 @@ public class FunctionTest {
Sample.builder().labels(of("le", String.valueOf(Integer.MAX_VALUE))).value(410).build())
),
"instance_cpu_percentage.histogram().histogram_percentile([75,99])",
Result.success(SampleFamily.buildHistogramPercentile(SampleFamily.buildHistogram(
Result.success(SampleFamily.build(
Sample.builder().labels(of("le", "0")).value(100).build(),
Sample.builder().labels(of("le", "25")).value(22).build(),
Sample.builder().labels(of("le", "750")).value(178).build(),
Sample.builder().labels(of("le", "1250")).value(110).build())
, new int[]{75, 99})),
),
false,
},
});
......
......@@ -57,7 +57,7 @@ public class TagFilterTest {
"default",
of("instance_cpu_percentage", SampleFamily.EMPTY),
"instance_cpu_percentage",
Result.fail(),
Result.fail("Parsed result is an EMPTY sample family"),
false,
},
{
......
......@@ -81,11 +81,6 @@
<artifactId>skywalking-mesh-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>skywalking-istio-telemetry-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>skywalking-management-receiver-plugin</artifactId>
......@@ -118,7 +113,7 @@
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>opencensus-receiver-plugin</artifactId>
<artifactId>otel-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
......@@ -265,7 +260,7 @@
<exclude>oal/</exclude>
<exclude>fetcher-prom-rules/</exclude>
<exclude>meter-analyzer-config/</exclude>
<exclude>oc-rules/</exclude>
<exclude>otel-oc-rules/</exclude>
<exclude>ui-initialized-templates/</exclude>
</excludes>
</configuration>
......
......@@ -254,10 +254,6 @@ service-mesh:
selector: ${SW_SERVICE_MESH:default}
default:
istio-telemetry:
selector: ${SW_ISTIO_TELEMETRY:default}
default:
envoy-metric:
selector: ${SW_ENVOY_METRIC:default}
default:
......@@ -270,9 +266,9 @@ envoy-metric:
k8sServiceNameRule: ${K8S_SERVICE_NAME_RULE:"${service.metadata.name}"}
prometheus-fetcher:
selector: ${SW_PROMETHEUS_FETCHER:default}
selector: ${SW_PROMETHEUS_FETCHER:-}
default:
active: ${SW_PROMETHEUS_FETCHER_ACTIVE:false}
enabledRules: ${SW_PROMETHEUS_FETCHER_ENABLED_RULES:"self"}
kafka-fetcher:
selector: ${SW_KAFKA_FETCHER:-}
......@@ -290,11 +286,11 @@ receiver-meter:
selector: ${SW_RECEIVER_METER:default}
default:
receiver-oc:
selector: ${SW_OC_RECEIVER:-}
receiver-otel:
selector: ${SW_OTEL_RECEIVER:-}
default:
gRPCHost: ${SW_OC_RECEIVER_GRPC_HOST:0.0.0.0}
gRPCPort: ${SW_OC_RECEIVER_GRPC_PORT:55678}
enabledHandlers: ${SW_OTEL_RECEIVER_ENABLED_HANDLERS:"oc"}
enabledOcRules: ${SW_OTEL_RECEIVER_ENABLED_OC_RULES:"istio-controlplane"}
receiver_zipkin:
selector: ${SW_RECEIVER_ZIPKIN:-}
......
......@@ -39,42 +39,43 @@ staticConfig:
labels:
service: oap-server
defaultMetricLevel: instance(['service'], ['instance'])
group: oap
metricsRules:
- name: instance_cpu_percentage
exp: (process_cpu_seconds_total * 100).rate('PT1M')
exp: (process_cpu_seconds_total * 100).sum(['service', 'instance']).rate('PT1M')
- name: instance_jvm_memory_bytes_used
exp: jvm_memory_bytes_used.sum(['service', 'instance'])
- name: instance_jvm_young_gc_count
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').increase('PT1M')
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').sum(['service', 'instance']).increase('PT1M')
- name: instance_jvm_young_gc_time
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation') * 1000
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').sum(['service', 'instance']) * 1000
- name: instance_jvm_old_gc_count
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').increase('PT1M')
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').sum(['service', 'instance']).increase('PT1M')
- name: instance_jvm_old_gc_time
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation') * 1000
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').sum(['service', 'instance']) * 1000
- name: instance_trace_count
exp: trace_in_latency_count.increase('PT1M')
exp: trace_in_latency_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_trace_latency_percentile
exp: trace_in_latency.increase('PT1M').histogram().histogram_percentile([50,70,90,99])
exp: trace_in_latency.sum(['le', 'service', 'instance']).increase('PT1M').histogram().histogram_percentile([50,70,90,99])
- name: instance_trace_analysis_error_count
exp: trace_analysis_error_count.increase('PT1M')
exp: trace_analysis_error_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_mesh_count
exp: mesh_analysis_latency_count.increase('PT1M')
exp: mesh_analysis_latency_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_mesh_latency_percentile
exp: mesh_analysis_latency.increase('PT1M').histogram().histogram_percentile([50,70,90,99])
exp: mesh_analysis_latency.sum(['le', 'service', 'instance']).increase('PT1M').histogram().histogram_percentile([50,70,90,99])
- name: instance_mesh_analysis_error_count
exp: mesh_analysis_error_count.increase('PT1M')
exp: mesh_analysis_error_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_metrics_first_aggregation
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '1').sum(['service', 'instance']).increase('PT1M')
- name: instance_metrics_second_aggregation
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '2').sum(['service', 'instance']).increase('PT1M')
- name: instance_persistence_execute_percentile
exp: persistence_timer_bulk_execute_latency.increase('PT5M').histogram().histogram_percentile([50,70,90,99])
exp: persistence_timer_bulk_execute_latency.sum(['le', 'service', 'instance']).increase('PT5M').histogram().histogram_percentile([50,70,90,99])
- name: instance_persistence_prepare_percentile
exp: persistence_timer_bulk_prepare_latency.increase('PT5M').histogram().histogram_percentile([50,70,90,99])
exp: persistence_timer_bulk_prepare_latency.sum(['le', 'service', 'instance']).increase('PT5M').histogram().histogram_percentile([50,70,90,99])
- name: instance_persistence_error_count
exp: persistence_timer_bulk_error_count.increase('PT1M')
exp: persistence_timer_bulk_error_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_persistence_execute_count
exp: persistence_timer_bulk_execute_latency_count.increase('PT1M')
exp: persistence_timer_bulk_execute_latency_count.sum(['service', 'instance']).increase('PT1M')
- name: instance_persistence_prepare_count
exp: persistence_timer_bulk_prepare_latency_count.increase('PT1M')
exp: persistence_timer_bulk_prepare_latency_count.sum(['service', 'instance']).increase('PT1M')
# 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.
# This will parse a textual representation of a duration. The formats
# accepted are based on the ISO-8601 duration format {@code PnDTnHnMn.nS}
# with days considered to be exactly 24 hours.
# <p>
# Examples:
# <pre>
# "PT20.345S" -- parses as "20.345 seconds"
# "PT15M" -- parses as "15 minutes" (where a minute is 60 seconds)
# "PT10H" -- parses as "10 hours" (where an hour is 3600 seconds)
# "P2D" -- parses as "2 days" (where a day is 24 hours or 86400 seconds)
# "P2DT3H4M" -- parses as "2 days, 3 hours and 4 minutes"
# "P-6H3M" -- parses as "-6 hours and +3 minutes"
# "-P6H3M" -- parses as "-6 hours and -3 minutes"
# "-P-6H+3M" -- parses as "+6 hours and -3 minutes"
# </pre>
group: istio
defaultMetricLevel: service(['cluster', 'app'])
metricsRules:
## Resource usage
# Pilot Versions
- name: pilot_version
exp: istio_build.tagEqual('component', 'pilot').sum(['cluster', 'app', 'tag'])
# Memory
- name: virtual_memory
exp: process_virtual_memory_bytes.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: resident_memory
exp: process_resident_memory_bytes.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: go_alloc
exp: go_memstats_alloc_bytes.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: go_heap_inuse
exp: go_memstats_heap_inuse_bytes.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: go_stack_inuse
exp: go_memstats_stack_inuse_bytes.tagEqual('app', 'istiod').sum(['cluster', 'app'])
# CPU
- name: cpu
exp: (process_cpu_seconds_total * 100).tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
# Goroutines
- name: go_goroutines
exp: go_goroutines.tagEqual('app', 'istiod').sum(['cluster', 'app'])
## Pilot push info
# Pilot pushes
- name: pilot_xds_pushes
exp: pilot_xds_pushes.tagMatch('type', 'lds|cds|rds|eds').sum(['cluster', 'app', 'type']).irate()
# Pilot Errors
- name: pilot_xds_cds_reject
exp: pilot_xds_cds_reject.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_xds_eds_reject
exp: pilot_xds_eds_reject.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_xds_rds_reject
exp: pilot_xds_rds_reject.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_xds_lds_reject
exp: pilot_xds_lds_reject.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_xds_write_timeout
exp: pilot_xds_write_timeout.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
- name: pilot_total_xds_internal_errors
exp: pilot_total_xds_internal_errors.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
- name: pilot_total_xds_rejects
exp: pilot_total_xds_rejects.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
- name: pilot_xds_push_context_errors
exp: pilot_xds_push_context_errors.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
- name: pilot_xds_pushes_error
exp: pilot_xds_pushes.tagEqual('app', 'istiod').tagNotMatch('type', 'lds|cds|rds|eds').sum(['cluster', 'app', 'type']).rate('PT1M')
- name: pilot_xds_push_errors
exp: pilot_xds_push_errors.tagEqual('app', 'istiod').sum(['cluster', 'app', 'type']).rate('PT1M')
- name: pilot_xds_push_timeout
exp: pilot_xds_push_timeout.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
- name: pilot_xds_push_timeout_failures
exp: pilot_xds_push_timeout_failures.tagEqual('app', 'istiod').sum(['cluster', 'app']).rate('PT1M')
# Proxy Push Time
- name: pilot_proxy_push_percentile
exp: pilot_proxy_convergence_time.sum(['cluster', 'app', 'le']).rate('PT1M').histogram().histogram_percentile([50,90,99])
# Conflicts
- name: pilot_conflict_il
exp: pilot_conflict_inbound_listener.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_conflict_ol_http_tcp
exp: pilot_conflict_outbound_listener_http_over_current_tcp.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_conflict_ol_tcp_tcp
exp: pilot_conflict_outbound_listener_tcp_over_current_tcp.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_conflict_ol_tcp_http
exp: pilot_conflict_outbound_listener_tcp_over_current_http.tagEqual('app', 'istiod').sum(['cluster', 'app'])
# ADS Monitoring
- name: pilot_virt_services
exp: pilot_virt_services.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_services
exp: pilot_services.tagEqual('app', 'istiod').sum(['cluster', 'app'])
- name: pilot_xds
exp: pilot_xds.tagEqual('app', 'istiod').sum(['cluster', 'app'])
## Webhooks
# Configuration Validation
- name: galley_validation_passed
exp: galley_validation_passed.sum(['cluster', 'app']).rate('PT1M')
- name: galley_validation_failed
exp: galley_validation_failed.sum(['cluster', 'app']).rate('PT1M')
# Sidecar Injection
- name: sidecar_injection_success_total
exp: sidecar_injection_success_total.sum(['cluster', 'app']).rate('PT1M')
- name: sidecar_injection_failure_total
exp: sidecar_injection_failure_total.sum(['cluster', 'app']).rate('PT1M')
......@@ -28,42 +28,45 @@
# "-P6H3M" -- parses as "-6 hours and -3 minutes"
# "-P-6H+3M" -- parses as "+6 hours and -3 minutes"
# </pre>
group: oap
# Self-monitoring defaults to prometheus fetcher.
defaultMetricLevel: instance(['service'], ['host_name'])
metricsRules:
- name: instance_cpu_percentage
exp: (process_cpu_seconds_total * 100).rate('PT1M').instance(['service'], ['host_name'])
exp: (process_cpu_seconds_total * 100).sum(['service', 'host_name']).rate('PT1M')
- name: instance_jvm_memory_bytes_used
exp: jvm_memory_bytes_used.sum(['service', 'instance']).instance(['service'], ['host_name'])
exp: jvm_memory_bytes_used.sum(['service', 'host_name'])
- name: instance_jvm_young_gc_count
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').increase('PT1M').instance(['service'], ['host_name'])
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').sum(['service', 'host_name']).increase('PT1M')
- name: instance_jvm_young_gc_time
exp: (jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation') * 1000).instance(['service'], ['host_name'])
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS Scavenge|Copy|ParNew|G1 Young Generation').sum(['service', 'host_name']) * 1000
- name: instance_jvm_old_gc_count
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').increase('PT1M').instance(['service'], ['host_name'])
exp: jvm_gc_collection_seconds_count.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').sum(['service', 'host_name']).increase('PT1M')
- name: instance_jvm_old_gc_time
exp: (jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation') * 1000).instance(['service'], ['host_name'])
exp: jvm_gc_collection_seconds_sum.tagMatch('gc', 'PS MarkSweep|MarkSweepCompact|ConcurrentMarkSweep|G1 Old Generation').sum(['service', 'host_name']) * 1000
- name: instance_trace_count
exp: trace_in_latency_count.increase('PT1M').instance(['service'], ['host_name'])
exp: trace_in_latency_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_trace_latency_percentile
exp: trace_in_latency.increase('PT1M').histogram().histogram_percentile([50,70,90,99]).instance(['service'], ['host_name'])
exp: trace_in_latency.sum(['le', 'service', 'host_name']).increase('PT1M').histogram().histogram_percentile([50,70,90,99])
- name: instance_trace_analysis_error_count
exp: trace_analysis_error_count.increase('PT1M').instance(['service'], ['host_name'])
exp: trace_analysis_error_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_mesh_count
exp: mesh_analysis_latency_count.increase('PT1M').instance(['service'], ['host_name'])
exp: mesh_analysis_latency_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_mesh_latency_percentile
exp: mesh_analysis_latency.increase('PT1M').histogram().histogram_percentile([50,70,90,99]).instance(['service'], ['host_name'])
exp: mesh_analysis_latency.sum(['le', 'service', 'host_name']).increase('PT1M').histogram().histogram_percentile([50,70,90,99])
- name: instance_mesh_analysis_error_count
exp: mesh_analysis_error_count.increase('PT1M').instance(['service'], ['host_name'])
exp: mesh_analysis_error_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_metrics_first_aggregation
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '1').sum(['service', 'instance']).increase('PT1M').instance(['service'], ['host_name'])
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '1').sum(['service', 'host_name']).increase('PT1M')
- name: instance_metrics_second_aggregation
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '2').sum(['service', 'instance']).increase('PT1M').instance(['service'], ['host_name'])
exp: metrics_aggregation.tagEqual('dimensionality', 'minute', 'level', '2').sum(['service', 'host_name']).increase('PT1M')
- name: instance_persistence_execute_percentile
exp: persistence_timer_bulk_execute_latency.increase('PT5M').histogram().histogram_percentile([50,70,90,99]).instance(['service'], ['host_name'])
exp: persistence_timer_bulk_execute_latency.sum(['le', 'service', 'host_name']).increase('PT5M').histogram().histogram_percentile([50,70,90,99])
- name: instance_persistence_prepare_percentile
exp: persistence_timer_bulk_prepare_latency.increase('PT5M').histogram().histogram_percentile([50,70,90,99]).instance(['service'], ['host_name'])
exp: persistence_timer_bulk_prepare_latency.sum(['le', 'service', 'host_name']).increase('PT5M').histogram().histogram_percentile([50,70,90,99])
- name: instance_persistence_error_count
exp: persistence_timer_bulk_error_count.increase('PT1M').instance(['service'], ['host_name'])
exp: persistence_timer_bulk_error_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_persistence_execute_count
exp: persistence_timer_bulk_execute_latency_count.increase('PT1M').instance(['service'], ['host_name'])
exp: persistence_timer_bulk_execute_latency_count.sum(['service', 'host_name']).increase('PT1M')
- name: instance_persistence_prepare_count
exp: persistence_timer_bulk_prepare_latency_count.increase('PT1M').instance(['service'], ['host_name'])
exp: persistence_timer_bulk_prepare_latency_count.sum(['service', 'host_name']).increase('PT1M')
# 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.
# UI templates initialized file includes the default template when the SkyWalking OAP starts up at the first time.
#
# Also, SkyWalking would detect the existing templates in the database, once they are missing, all templates in this file
# could be added automatically.
templates:
- name: "Istio"
# The type includes DASHBOARD, TOPOLOGY_INSTANCE, TOPOLOGY_ENDPOINT.
# DASHBOARD type templates could have multiple definitions, by using different names.
# TOPOLOGY_INSTANCE, TOPOLOGY_ENDPOINT type templates should be defined once, as they are used in the topology page only.
type: "DASHBOARD"
# Configuration could be defined through UI, and use `export` to format in the standard JSON.
configuration: |-
[
{
"name": "Istio",
"type": "service",
"children": [
{
"name": "Control Plane",
"children": [
{
"width": 3,
"title": "Istio Versions",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_pilot_version",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea"
},
{
"width": 3,
"title": "Memory",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_virtual_memory,meter_resident_memory,meter_go_alloc,meter_go_heap_inuse,meter_go_stack_inuse",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea",
"unit": "MB",
"aggregation": "/",
"aggregationNum": "1048576"
},
{
"width": 3,
"title": "CPU",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_cpu",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea",
"unit": "%"
},
{
"width": 3,
"title": "Goroutines",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_go_goroutines",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine"
},
{
"width": "4",
"title": "Pilot pushes",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "meter_istio_pilot_xds_pushes",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartBar"
},
{
"width": "4",
"title": "Pilot Errors",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "UNKNOWN",
"metricName": "meter_istio_pilot_xds_cds_reject,meter_pilot_xds_eds_reject,meter_pilot_xds_rds_reject,meter_pilot_xds_lds_reject,meter_pilot_xds_write_timeout,meter_pilot_total_xds_internal_errors,meter_pilot_total_xds_rejects,meter_pilot_xds_push_context_errors,meter_pilot_xds_pushes_error,meter_pilot_xds_push_errors,meter_pilot_xds_push_timeout,meter_pilot_xds_push_timeout_failures",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea"
},
{
"width": "4",
"title": "Proxy Push Time",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "meter_istio_pilot_proxy_push_percentile",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartArea",
"metricLabels": "50,90,99",
"labelsIndex": "50,90,99",
"unit": "%"
},
{
"width": "4",
"title": "Conflicts",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_pilot_conflict_il,meter_pilot_conflict_ol_http_tcp,meter_pilot_conflict_ol_tcp_tcp,meter_pilot_conflict_ol_tcp_http",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar"
},
{
"width": "4",
"title": "ADS Monitoring",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_pilot_virt_services,meter_pilot_services,meter_pilot_xds",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea"
},
{
"width": "6",
"title": "Configuration Validation",
"height": 350,
"entityType": "Service",
"independentSelector": true,
"metricType": "REGULAR_VALUE",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine",
"metricName": "meter_istio_galley_validation_passed,meter_galley_validation_failed"
},
{
"width": "6",
"title": "Sidecar Injection",
"height": 350,
"entityType": "Service",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_istio_sidecar_injection_success_total",
"queryMetricType": "readMetricsValues",
"chartType": "ChartArea"
}
]
}
]
}
]
# Activated as the DASHBOARD type, makes this templates added into the UI page automatically.
# False means providing a basic template, user needs to add it manually.
activated: true
# True means wouldn't show up on the dashboard. Only keeps the definition in the storage.
disabled: false
\ No newline at end of file
......@@ -37,7 +37,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_cpu_percentage",
"metricName": "meter_oap_instance_cpu_percentage",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine",
"unit": "%"
......@@ -49,7 +49,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_jvm_memory_bytes_used",
"metricName": "meter_oap_instance_jvm_memory_bytes_used",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine",
"unit": "MB",
......@@ -63,7 +63,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_jvm_young_gc_count,meter_instance_jvm_old_gc_count",
"metricName": "meter_oap_instance_jvm_young_gc_count,meter_instance_jvm_old_gc_count",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar",
"unit": "Per Minute"
......@@ -75,7 +75,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_jvm_young_gc_time,meter_instance_jvm_old_gc_time",
"metricName": "meter_oap_instance_jvm_young_gc_time,meter_instance_jvm_old_gc_time",
"queryMetricType": "readMetricsValues",
"chartType": "ChartLine",
"unit": "Millisecond"
......@@ -87,7 +87,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_trace_count,meter_instance_trace_analysis_error_count",
"metricName": "meter_oap_instance_trace_count,meter_instance_trace_analysis_error_count",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar",
"unit": "Per Minute"
......@@ -99,7 +99,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "meter_instance_trace_latency_percentile",
"metricName": "meter_oap_instance_trace_latency_percentile",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine",
"labelsIndex": "50,70,90,99",
......@@ -113,7 +113,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_mesh_count,meter_instance_mesh_analysis_error_count",
"metricName": "meter_oap_instance_mesh_count,meter_instance_mesh_analysis_error_count",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar",
"unit": "Per Minute"
......@@ -125,7 +125,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "meter_instance_mesh_latency_percentile",
"metricName": "meter_oap_instance_mesh_latency_percentile",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine",
"metricLabels": "50,70,90,99",
......@@ -139,7 +139,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_metrics_first_aggregation,meter_instance_metrics_second_aggregation",
"metricName": "meter_oap_instance_metrics_first_aggregation,meter_instance_metrics_second_aggregation",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar",
"unit": "Per Minute"
......@@ -151,7 +151,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "REGULAR_VALUE",
"metricName": "meter_instance_persistence_prepare_count,meter_instance_persistence_execute_count,meter_instance_persistence_error_count",
"metricName": "meter_oap_instance_persistence_prepare_count,meter_instance_persistence_execute_count,meter_instance_persistence_error_count",
"queryMetricType": "readMetricsValues",
"chartType": "ChartBar",
"unit": "Per 5 Minutes"
......@@ -166,7 +166,7 @@ templates:
"unit": "Millisecond",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine",
"metricName": "meter_instance_persistence_prepare_percentile",
"metricName": "meter_oap_instance_persistence_prepare_percentile",
"metricLabels": "50,70,90,99",
"labelsIndex": "50,70,90,99"
},
......@@ -177,7 +177,7 @@ templates:
"entityType": "ServiceInstance",
"independentSelector": false,
"metricType": "LABELED_VALUE",
"metricName": "meter_instance_persistence_execute_percentile",
"metricName": "meter_oap_instance_persistence_execute_percentile",
"queryMetricType": "readLabeledMetricsValues",
"chartType": "ChartLine",
"metricLabels": "50,70,90,99",
......
......@@ -18,13 +18,25 @@
package org.apache.skywalking.oap.server.fetcher.prometheus.provider;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import lombok.Getter;
import org.apache.skywalking.apm.util.StringUtil;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
@Getter
public class PrometheusFetcherConfig extends ModuleConfig {
private boolean active;
private String enabledRules;
private final String rulePath = "fetcher-prom-rules";
List<String> getEnabledRules() {
return Arrays.stream(Optional.ofNullable(enabledRules).orElse("").toString()
.split(","))
.map(String::trim)
.filter(StringUtil::isNotEmpty)
.collect(Collectors.toList());
}
}
......@@ -22,6 +22,7 @@ import com.google.common.collect.Maps;
import io.vavr.CheckedFunction1;
import java.io.ByteArrayInputStream;
import java.io.InputStream;
import java.net.URI;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
......@@ -82,10 +83,7 @@ public class PrometheusFetcherProvider extends ModuleProvider {
@Override
public void prepare() throws ServiceNotProvidedException, ModuleStartException {
if (!config.isActive()) {
return;
}
rules = Rules.loadRules(config.getRulePath());
rules = Rules.loadRules(config.getRulePath(), config.getEnabledRules());
ses = Executors.newScheduledThreadPool(rules.size(), Executors.defaultThreadFactory());
}
......@@ -95,14 +93,14 @@ public class PrometheusFetcherProvider extends ModuleProvider {
@Override
public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
if (!config.isActive()) {
if (rules.isEmpty()) {
return;
}
final MeterSystem service = getManager().find(CoreModule.NAME).provider().getService(MeterSystem.class);
rules.forEach(r -> {
ses.scheduleAtFixedRate(new Runnable() {
private final PrometheusMetricConverter converter = new PrometheusMetricConverter(r.getMetricsRules(), r.getDefaultMetricLevel(), service);
private final PrometheusMetricConverter converter = new PrometheusMetricConverter(r, service);
@Override public void run() {
if (Objects.isNull(r.getStaticConfig())) {
......@@ -112,7 +110,9 @@ public class PrometheusFetcherProvider extends ModuleProvider {
long now = System.currentTimeMillis();
converter.toMeter(sc.getTargets().stream()
.map(CheckedFunction1.liftTry(target -> {
String content = HttpClient.builder().url(target.getUrl()).caFilePath(target.getSslCaFilePath()).build().request();
URI url = new URI(target.getUrl());
URI targetURL = url.resolve(r.getMetricsPath());
String content = HttpClient.builder().url(targetURL.toString()).caFilePath(target.getSslCaFilePath()).build().request();
List<Metric> result = new ArrayList<>();
try (InputStream targetStream = new ByteArrayInputStream(content.getBytes(Charsets.UTF_8))) {
Parser p = Parsers.text(targetStream);
......
......@@ -18,6 +18,7 @@
package org.apache.skywalking.oap.server.fetcher.prometheus.provider.rule;
import java.util.Collections;
import java.util.List;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rule;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rules;
......@@ -31,7 +32,7 @@ public class RulesTest {
@Test
public void testFetcherPrometheusRulesLoader() throws ModuleStartException {
List<Rule> rr = Rules.loadRules("fetcher-prom-rules");
List<Rule> rr = Rules.loadRules("fetcher-prom-rules", Collections.singletonList("localhost"));
assertThat(rr.size(), is(1));
}
......
......@@ -78,9 +78,6 @@ receiver-profile:
service-mesh:
default:
istio-telemetry:
default:
query:
graphql:
path: ${QUERY_GRAPHQL_PATH:/graphql}
......
/*
* 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.opencensus;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
import org.apache.skywalking.oap.meter.analyzer.prometheus.PrometheusMetricConverter;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.MetricsRule;
import org.apache.skywalking.oap.meter.analyzer.prometheus.rule.Rules;
import org.apache.skywalking.oap.server.core.CoreModule;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterSystem;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
import org.apache.skywalking.oap.server.library.module.ModuleDefine;
import org.apache.skywalking.oap.server.library.module.ModuleProvider;
import org.apache.skywalking.oap.server.library.module.ModuleStartException;
import org.apache.skywalking.oap.server.library.module.ServiceNotProvidedException;
import org.apache.skywalking.oap.server.library.server.ServerException;
import org.apache.skywalking.oap.server.library.server.grpc.GRPCServer;
import org.apache.skywalking.oap.server.receiver.sharing.server.SharingServerModule;
public class OCMetricReceiverProvider extends ModuleProvider {
public static final String NAME = "default";
private OCMetricReceiverConfig config;
private GRPCServer grpcServer = null;
private List<MetricsRule> rules;
@Override
public String name() {
return NAME;
}
@Override
public Class<? extends ModuleDefine> module() {
return OCMetricReceiverModule.class;
}
@Override
public ModuleConfig createConfigBeanIfAbsent() {
config = new OCMetricReceiverConfig();
return config;
}
@Override
public void prepare() throws ServiceNotProvidedException, ModuleStartException {
if (config.getGRPCPort() <= 0) {
return;
}
rules = Rules.loadRules(config.getRulePath()).stream()
.flatMap(rule -> rule.getMetricsRules().stream())
.collect(Collectors.toList());
grpcServer = new GRPCServer(config.getGRPCHost(), config.getGRPCPort());
if (config.getMaxMessageSize() > 0) {
grpcServer.setMaxMessageSize(config.getMaxMessageSize());
}
if (config.getMaxConcurrentCallsPerConnection() > 0) {
grpcServer.setMaxConcurrentCallsPerConnection(config.getMaxConcurrentCallsPerConnection());
}
if (config.getGRPCThreadPoolQueueSize() > 0) {
grpcServer.setThreadPoolQueueSize(config.getGRPCThreadPoolQueueSize());
}
if (config.getGRPCThreadPoolSize() > 0) {
grpcServer.setThreadPoolSize(config.getGRPCThreadPoolSize());
}
grpcServer.initialize();
}
@Override
public void start() throws ServiceNotProvidedException, ModuleStartException {
if (Objects.nonNull(grpcServer)) {
final MeterSystem service = getManager().find(CoreModule.NAME).provider().getService(MeterSystem.class);
grpcServer.addHandler(new OCMetricHandler(new PrometheusMetricConverter(rules, null, service)));
}
}
@Override
public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
try {
if (Objects.nonNull(grpcServer)) {
grpcServer.start();
}
} catch (ServerException e) {
throw new ModuleStartException(e.getMessage(), e);
}
}
@Override
public String[] requiredModules() {
return new String[] {SharingServerModule.NAME};
}
}
......@@ -25,7 +25,7 @@
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>opencensus-receiver-plugin</artifactId>
<artifactId>otel-receiver-plugin</artifactId>
<packaging>jar</packaging>
<dependencies>
......
/*
* 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.otel;
import com.google.common.collect.ImmutableSet;
import com.google.common.reflect.ClassPath;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterSystem;
import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
public interface Handler {
static List<Handler> all() throws HandlerInitializationException {
ClassPath classpath;
try {
classpath = ClassPath.from(Handler.class.getClassLoader());
} catch (IOException e) {
throw new HandlerInitializationException("failed to load handler classes", e);
}
ImmutableSet<ClassPath.ClassInfo> classes = classpath.getTopLevelClassesRecursive(Handler.class.getPackage().getName());
List<Handler> result = new ArrayList<>();
for (ClassPath.ClassInfo each : classes) {
Class<?> c = each.load();
if (Arrays.stream(c.getInterfaces()).anyMatch(interfaceClass -> interfaceClass.isAssignableFrom(Handler.class))) {
try {
result.add((Handler) c.newInstance());
} catch (InstantiationException | IllegalAccessException e) {
throw new HandlerInitializationException("failed to get instances of handler classed", e);
}
}
}
return result;
}
String type();
void active(List<String> enabledRules, MeterSystem service,
GRPCHandlerRegister grpcHandlerRegister);
}
......@@ -16,19 +16,16 @@
*
*/
package org.apache.skywalking.oap.server.receiver.istio.telemetry.module;
package org.apache.skywalking.oap.server.receiver.otel;
import org.apache.skywalking.oap.server.library.module.ModuleDefine;
import org.apache.skywalking.oap.server.library.module.ModuleStartException;
public class IstioTelemetryReceiverModule extends ModuleDefine {
public static final String NAME = "istio-telemetry";
public IstioTelemetryReceiverModule() {
super(NAME);
public class HandlerInitializationException extends ModuleStartException {
public HandlerInitializationException(String message) {
super(message);
}
@Override
public Class[] services() {
return new Class[0];
public HandlerInitializationException(String message, Throwable cause) {
super(message, cause);
}
}
/*
* 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.otel;
import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.skywalking.apm.util.StringUtil;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
@Slf4j
public class OtelMetricReceiverConfig extends ModuleConfig {
private String enabledHandlers;
private String enabledOcRules;
public List<String> getEnabledHandlers() {
return split(enabledHandlers);
}
List<String> getEnabledRulesFrom(String handler) {
Field f;
try {
f = this.getClass().getDeclaredField(String.format("enabled%sRules", StringUtils.capitalize(handler)));
} catch (NoSuchFieldException e) {
if (log.isDebugEnabled()) {
log.debug("failed to get disabled rule field of {}", handler, e);
}
return Collections.emptyList();
}
f.setAccessible(true);
try {
return split(f.get(this));
} catch (IllegalAccessException e) {
log.warn("failed to access disabled rule list of {}", handler, e);
return Collections.emptyList();
}
}
private List<String> split(Object str) {
return Arrays.stream(Optional.ofNullable(str).orElse("").toString()
.split(","))
.map(String::trim)
.filter(StringUtil::isNotEmpty)
.collect(Collectors.toList());
}
}
......@@ -16,14 +16,14 @@
*
*/
package org.apache.skywalking.oap.server.receiver.opencensus;
package org.apache.skywalking.oap.server.receiver.otel;
import org.apache.skywalking.oap.server.library.module.ModuleDefine;
public class OCMetricReceiverModule extends ModuleDefine {
public static final String NAME = "receiver-oc";
public class OtelMetricReceiverModule extends ModuleDefine {
public static final String NAME = "receiver-otel";
public OCMetricReceiverModule() {
public OtelMetricReceiverModule() {
super(NAME);
}
......
......@@ -16,34 +16,36 @@
*
*/
package org.apache.skywalking.oap.server.receiver.istio.telemetry.provider;
package org.apache.skywalking.oap.server.receiver.otel;
import org.apache.skywalking.aop.server.receiver.mesh.MeshReceiverModule;
import org.apache.skywalking.oap.server.core.CoreModule;
import org.apache.skywalking.oap.server.core.analysis.meter.MeterSystem;
import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
import org.apache.skywalking.oap.server.library.module.ModuleConfig;
import org.apache.skywalking.oap.server.library.module.ModuleDefine;
import org.apache.skywalking.oap.server.library.module.ModuleProvider;
import org.apache.skywalking.oap.server.library.module.ModuleStartException;
import org.apache.skywalking.oap.server.library.module.ServiceNotProvidedException;
import org.apache.skywalking.oap.server.receiver.istio.telemetry.module.IstioTelemetryReceiverModule;
import org.apache.skywalking.oap.server.receiver.sharing.server.SharingServerModule;
import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
public class IstioTelemetryReceiverProvider extends ModuleProvider {
public class OtelMetricReceiverProvider extends ModuleProvider {
public static final String NAME = "default";
private OtelMetricReceiverConfig config;
@Override
public String name() {
return "default";
return NAME;
}
@Override
public Class<? extends ModuleDefine> module() {
return IstioTelemetryReceiverModule.class;
return OtelMetricReceiverModule.class;
}
@Override
public ModuleConfig createConfigBeanIfAbsent() {
return null;
config = new OtelMetricReceiverConfig();
return config;
}
@Override
......@@ -52,24 +54,24 @@ public class IstioTelemetryReceiverProvider extends ModuleProvider {
@Override
public void start() throws ServiceNotProvidedException, ModuleStartException {
GRPCHandlerRegister service = getManager().find(SharingServerModule.NAME)
.provider()
.getService(GRPCHandlerRegister.class);
service.addHandler(new IstioTelemetryGRPCHandler(getManager()));
if (config.getEnabledHandlers().isEmpty()) {
return;
}
GRPCHandlerRegister grpcHandlerRegister = getManager().find(SharingServerModule.NAME)
.provider()
.getService(GRPCHandlerRegister.class);
final MeterSystem service = getManager().find(CoreModule.NAME).provider().getService(MeterSystem.class);
Handler.all().stream()
.filter(h -> config.getEnabledHandlers().contains(h.type()))
.forEach(h -> h.active(config.getEnabledRulesFrom(h.type()), service, grpcHandlerRegister));
}
@Override
public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
}
@Override
public String[] requiredModules() {
return new String[] {
TelemetryModule.NAME,
CoreModule.NAME,
MeshReceiverModule.NAME,
SharingServerModule.NAME
};
return new String[] {SharingServerModule.NAME};
}
}
\ No newline at end of file
}
......@@ -16,7 +16,7 @@
*
*/
package org.apache.skywalking.oap.server.receiver.opencensus;
package org.apache.skywalking.oap.server.receiver.otel.oc;
import com.google.protobuf.Timestamp;
import io.grpc.stub.StreamObserver;
......@@ -33,26 +33,32 @@ import java.time.Instant;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.skywalking.oap.meter.analyzer.MetricConvert;
import org.apache.skywalking.oap.meter.analyzer.prometheus.PrometheusMetricConverter;
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.core.analysis.meter.MeterSystem;
import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
import org.apache.skywalking.oap.server.library.module.ModuleStartException;
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.Summary;
import org.apache.skywalking.oap.server.receiver.otel.Handler;
import static java.util.stream.Collectors.toList;
@RequiredArgsConstructor
@Slf4j
public class OCMetricHandler extends MetricsServiceGrpc.MetricsServiceImplBase {
public class OCMetricHandler extends MetricsServiceGrpc.MetricsServiceImplBase implements Handler {
private final PrometheusMetricConverter prometheusMetric;
private List<PrometheusMetricConverter> metrics;
@Override public StreamObserver<ExportMetricsServiceRequest> export(
StreamObserver<ExportMetricsServiceResponse> responseObserver) {
return new StreamObserver<ExportMetricsServiceRequest>() {
@Override public void onNext(ExportMetricsServiceRequest request) {
prometheusMetric.toMeter(request.getMetricsList().stream()
metrics.forEach(m -> m.toMeter(request.getMetricsList().stream()
.flatMap(metric -> metric.getTimeseriesList().stream().map(timeSeries ->
Tuple.of(metric.getMetricDescriptor(),
buildLabels(metric.getMetricDescriptor().getLabelKeysList(), timeSeries.getLabelValuesList()),
......@@ -80,7 +86,7 @@ public class OCMetricHandler extends MetricsServiceGrpc.MetricsServiceImplBase {
throw new UnsupportedOperationException("Unsupported OC type:" + t._1.getType());
}
}))
.flatMap(tryIt -> MetricConvert.log(tryIt, "Convert OC metric to prometheus metric")));
.flatMap(tryIt -> MetricConvert.log(tryIt, "Convert OC metric to prometheus metric"))));
}
@Override public void onError(Throwable throwable) {
......@@ -122,4 +128,25 @@ public class OCMetricHandler extends MetricsServiceGrpc.MetricsServiceImplBase {
Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos()).toEpochMilli();
}
@Override public String type() {
return "oc";
}
@Override public void active(List<String> enabledRules,
MeterSystem service, GRPCHandlerRegister grpcHandlerRegister) {
List<Rule> rules;
try {
rules = Rules.loadRules("otel-oc-rules", enabledRules);
} catch (ModuleStartException e) {
log.warn("failed to load otel-oc-rules");
return;
}
if (rules.isEmpty()) {
return;
}
this.metrics = rules.stream().map(r ->
new PrometheusMetricConverter(r, service))
.collect(toList());
grpcHandlerRegister.addHandler(this);
}
}
......@@ -17,4 +17,4 @@
#
org.apache.skywalking.oap.server.receiver.opencensus.OCMetricReceiverModule
\ No newline at end of file
org.apache.skywalking.oap.server.receiver.otel.OtelMetricReceiverModule
\ No newline at end of file
......@@ -16,4 +16,4 @@
#
#
org.apache.skywalking.oap.server.receiver.opencensus.OCMetricReceiverProvider
\ No newline at end of file
org.apache.skywalking.oap.server.receiver.otel.OtelMetricReceiverProvider
\ No newline at end of file
......@@ -31,7 +31,6 @@
<module>skywalking-trace-receiver-plugin</module>
<module>zipkin-receiver-plugin</module>
<module>skywalking-mesh-receiver-plugin</module>
<module>skywalking-istio-telemetry-receiver-plugin</module>
<module>skywalking-management-receiver-plugin</module>
<module>skywalking-jvm-receiver-plugin</module>
<module>envoy-metrics-receiver-plugin</module>
......@@ -40,7 +39,7 @@
<module>jaeger-receiver-plugin</module>
<module>receiver-proto</module>
<module>skywalking-profile-receiver-plugin</module>
<module>opencensus-receiver-plugin</module>
<module>otel-receiver-plugin</module>
<module>skywalking-meter-receiver-plugin</module>
<module>skywalking-browser-receiver-plugin</module>
</modules>
......
// Copyright 2017 Istio Authors
//
// Licensed 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.
// THIS FILE IS AUTOMATICALLY GENERATED.
syntax = "proto3";
// The `metric` template is designed to let you describe runtime metric to dispatch to
// monitoring backends.
//
// Example config:
//
// ```yaml
// apiVersion: "config.istio.io/v1alpha2"
// kind: metric
// metadata:
// name: requestsize
// namespace: istio-system
// spec:
// value: request.size | 0
// dimensions:
// source_service: source.service | "unknown"
// source_version: source.labels["version"] | "unknown"
// destination_service: destination.service | "unknown"
// destination_version: destination.labels["version"] | "unknown"
// response_code: response.code | 200
// monitored_resource_type: '"UNSPECIFIED"'
// ```
//
// The `metric` template represents a single piece of data to report.
//
// When writing the configuration, the value for the fields associated with this template can either be a
// literal or an [expression](https://istio.io/docs/reference//config/policy-and-telemetry/expression-language/). Please note that if the datatype of a field is not istio.policy.v1beta1.Value,
// then the expression's [inferred type](https://istio.io/docs/reference//config/policy-and-telemetry/expression-language/#type-checking) must match the datatype of the field.
package metric;
import "gogoproto/gogo.proto";
import "mixer/adapter/model/v1beta1/extensions.proto";
import "google/protobuf/any.proto";
import "mixer/adapter/model/v1beta1/report.proto";
import "policy/v1beta1/value_type.proto";
import "policy/v1beta1/type.proto";
option java_package = "io.istio";
option java_outer_classname = "IstioMetricProto";
option (istio.mixer.adapter.model.v1beta1.template_variety) = TEMPLATE_VARIETY_REPORT;
option (istio.mixer.adapter.model.v1beta1.template_name) = "metric";
option (gogoproto.goproto_getters_all) = false;
option (gogoproto.equal_all) = false;
option (gogoproto.gostring_all) = false;
// HandleMetricService is implemented by backends that wants to handle request-time 'metric' instances.
service HandleMetricService {
// HandleMetric is called by Mixer at request-time to deliver 'metric' instances to the backend.
rpc HandleMetric(HandleMetricRequest) returns (istio.mixer.adapter.model.v1beta1.ReportResult);
}
// Request message for HandleMetric method.
message HandleMetricRequest {
// 'metric' instances.
repeated InstanceMsg instances = 1;
// Adapter specific handler configuration.
//
// Note: Backends can also implement [InfrastructureBackend][https://istio.io/docs/reference/config/mixer/istio.mixer.adapter.model.v1beta1.html#InfrastructureBackend]
// service and therefore opt to receive handler configuration during session creation through [InfrastructureBackend.CreateSession][TODO: Link to this fragment]
// call. In that case, adapter_config will have type_url as 'google.protobuf.Any.type_url' and would contain string
// value of session_id (returned from InfrastructureBackend.CreateSession).
google.protobuf.Any adapter_config = 2;
// Id to dedupe identical requests from Mixer.
string dedup_id = 3;
}
// Contains instance payload for 'metric' template. This is passed to infrastructure backends during request-time
// through HandleMetricService.HandleMetric.
message InstanceMsg {
// Name of the instance as specified in configuration.
string name = 72295727;
// The value being reported.
istio.policy.v1beta1.Value value = 1;
// The unique identity of the particular metric to report.
map<string, istio.policy.v1beta1.Value> dimensions = 2;
// Optional. An expression to compute the type of the monitored resource this metric is being reported on.
// If the metric backend supports monitored resources, these fields are used to populate that resource. Otherwise
// these fields will be ignored by the adapter.
string monitored_resource_type = 3;
// Optional. A set of expressions that will form the dimensions of the monitored resource this metric is being reported on.
// If the metric backend supports monitored resources, these fields are used to populate that resource. Otherwise
// these fields will be ignored by the adapter.
map<string, istio.policy.v1beta1.Value> monitored_resource_dimensions = 4;
}
// Contains inferred type information about specific instance of 'metric' template. This is passed to
// infrastructure backends during configuration-time through [InfrastructureBackend.CreateSession][TODO: Link to this fragment].
message Type {
// The value being reported.
istio.policy.v1beta1.ValueType value = 1;
// The unique identity of the particular metric to report.
map<string, istio.policy.v1beta1.ValueType> dimensions = 2;
// Optional. A set of expressions that will form the dimensions of the monitored resource this metric is being reported on.
// If the metric backend supports monitored resources, these fields are used to populate that resource. Otherwise
// these fields will be ignored by the adapter.
map<string, istio.policy.v1beta1.ValueType> monitored_resource_dimensions = 4;
}
// Represents instance configuration schema for 'metric' template.
message InstanceParam {
// The value being reported.
string value = 1;
// The unique identity of the particular metric to report.
map<string, string> dimensions = 2;
// Optional. An expression to compute the type of the monitored resource this metric is being reported on.
// If the metric backend supports monitored resources, these fields are used to populate that resource. Otherwise
// these fields will be ignored by the adapter.
string monitored_resource_type = 3;
// Optional. A set of expressions that will form the dimensions of the monitored resource this metric is being reported on.
// If the metric backend supports monitored resources, these fields are used to populate that resource. Otherwise
// these fields will be ignored by the adapter.
map<string, string> monitored_resource_dimensions = 4;
}
// Copyright 2018 Istio Authors
//
// Licensed 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.
syntax = "proto3";
package istio.mixer.adapter.model.v1beta1;
option go_package="istio.io/api/mixer/adapter/model/v1beta1";
option java_package = "io.istio.api.mixer.adapter.model.v1beta1";
option java_outer_classname = "ExtensionsProto";
import "google/protobuf/descriptor.proto";
// The available varieties of templates, controlling the semantics of what an adapter does with each instance.
enum TemplateVariety {
// Makes the template applicable for Mixer's check calls. Instances of such template are created during
// report calls in Mixer and passed to the handlers based on the rule configurations.
TEMPLATE_VARIETY_CHECK = 0;
// Makes the template applicable for Mixer's report calls. Instances of such template are created during
// check calls in Mixer and passed to the handlers based on the rule configurations.
TEMPLATE_VARIETY_REPORT = 1;
// Makes the template applicable for Mixer's quota calls. Instances of such template are created during
// quota check calls in Mixer and passed to the handlers based on the rule configurations.
TEMPLATE_VARIETY_QUOTA = 2;
// Makes the template applicable for Mixer's attribute generation phase. Instances of such template are created during
// pre-processing attribute generation phase and passed to the handlers based on the rule configurations.
TEMPLATE_VARIETY_ATTRIBUTE_GENERATOR = 3;
}
// File level options for the template.
extend google.protobuf.FileOptions {
// Required: option for the TemplateVariety.
TemplateVariety template_variety = 72295727;
// Optional: option for the template name.
// If not specified, the last segment of the template proto's package name is used to
// derive the template name.
string template_name = 72295888;
}
// Copyright 2018 Istio Authors
//
// Licensed 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.
syntax = "proto3";
package istio.mixer.adapter.model.v1beta1;
option go_package="istio.io/api/mixer/adapter/model/v1beta1";
option java_package = "io.istio.api.mixer.adapter.model.v1beta1";
option java_outer_classname = "ReportProto";
import "gogoproto/gogo.proto";
option (gogoproto.goproto_getters_all) = false;
option (gogoproto.equal_all) = false;
option (gogoproto.gostring_all) = false;
// Expresses the result of a report call.
message ReportResult {}
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
~
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>server-receiver-plugin</artifactId>
<groupId>org.apache.skywalking</groupId>
<version>8.3.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>skywalking-istio-telemetry-receiver-plugin</artifactId>
<packaging>jar</packaging>
<dependencies>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>skywalking-mesh-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>receiver-proto</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-resources-plugin</artifactId>
<version>2.4.3</version>
<configuration>
<encoding>${project.build.sourceEncoding}</encoding>
</configuration>
</plugin>
<plugin>
<groupId>org.xolstice.maven.plugins</groupId>
<artifactId>protobuf-maven-plugin</artifactId>
<version>${protobuf-maven-plugin.version}</version>
<configuration>
<!--
The version of protoc must match protobuf-java. If you don't depend on
protobuf-java directly, you will be transitively depending on the
protobuf-java version that grpc depends on.
-->
<protocArtifact>
com.google.protobuf:protoc:${com.google.protobuf.protoc.version}:exe:${os.detected.classifier}
</protocArtifact>
<pluginId>grpc-java</pluginId>
<pluginArtifact>
io.grpc:protoc-gen-grpc-java:${protoc-gen-grpc-java.plugin.version}:exe:${os.detected.classifier}
</pluginArtifact>
</configuration>
<executions>
<execution>
<goals>
<goal>compile</goal>
<goal>compile-custom</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>
\ No newline at end of file
/*
* 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.istio.telemetry.provider;
import com.google.common.base.Joiner;
import com.google.protobuf.Timestamp;
import io.grpc.stub.StreamObserver;
import io.istio.HandleMetricServiceGrpc;
import io.istio.IstioMetricProto;
import io.istio.api.mixer.adapter.model.v1beta1.ReportProto;
import io.istio.api.policy.v1beta1.TypeProto;
import java.time.Duration;
import java.time.Instant;
import java.util.Map;
import org.apache.skywalking.aop.server.receiver.mesh.TelemetryDataDispatcher;
import org.apache.skywalking.apm.network.common.v3.DetectPoint;
import org.apache.skywalking.apm.network.servicemesh.v3.Protocol;
import org.apache.skywalking.apm.network.servicemesh.v3.ServiceMeshMetric;
import org.apache.skywalking.oap.server.library.module.ModuleManager;
import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
import org.apache.skywalking.oap.server.telemetry.api.CounterMetrics;
import org.apache.skywalking.oap.server.telemetry.api.HistogramMetrics;
import org.apache.skywalking.oap.server.telemetry.api.MetricsCreator;
import org.apache.skywalking.oap.server.telemetry.api.MetricsTag;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Handle istio telemetry data.
*/
public class IstioTelemetryGRPCHandler extends HandleMetricServiceGrpc.HandleMetricServiceImplBase {
private static final Logger LOGGER = LoggerFactory.getLogger(IstioTelemetryGRPCHandler.class);
private static final Joiner JOINER = Joiner.on(".");
private CounterMetrics counter;
private HistogramMetrics histogram;
public IstioTelemetryGRPCHandler(ModuleManager moduleManager) {
MetricsCreator metricsCreator = moduleManager.find(TelemetryModule.NAME)
.provider()
.getService(MetricsCreator.class);
counter = metricsCreator.createCounter("istio_mesh_grpc_in_count", "The count of istio service mesh telemetry", MetricsTag.EMPTY_KEY, MetricsTag.EMPTY_VALUE);
histogram = metricsCreator.createHistogramMetric("istio_mesh_grpc_in_latency", "The process latency of istio service mesh telemetry", MetricsTag.EMPTY_KEY, MetricsTag.EMPTY_VALUE);
}
@Override
public void handleMetric(IstioMetricProto.HandleMetricRequest request,
StreamObserver<ReportProto.ReportResult> responseObserver) {
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Received msg {}", request);
}
for (IstioMetricProto.InstanceMsg i : request.getInstancesList()) {
counter.inc();
HistogramMetrics.Timer timer = histogram.createTimer();
try {
String requestMethod = string(i, "requestMethod");
String requestPath = string(i, "requestPath");
String requestScheme = string(i, "requestScheme");
long responseCode = int64(i, "responseCode");
String reporter = string(i, "reporter");
String protocol = string(i, "apiProtocol");
String endpoint;
boolean status = true;
Protocol netProtocol;
if (protocol.equals("http") || protocol.equals("https") || requestScheme.equals("http") || requestScheme
.equals("https")) {
endpoint = requestScheme + "/" + requestMethod + "/" + requestPath;
status = responseCode >= 200 && responseCode < 400;
netProtocol = Protocol.HTTP;
} else {
//grpc
endpoint = protocol + "/" + requestPath;
netProtocol = Protocol.gRPC;
}
Instant requestTime = time(i, "requestTime");
Instant responseTime = time(i, "responseTime");
int latency = Math.toIntExact(Duration.between(requestTime, responseTime).toMillis());
DetectPoint detectPoint;
if (reporter.equals("source")) {
detectPoint = DetectPoint.client;
} else {
detectPoint = DetectPoint.server;
}
String sourceServiceName;
if (has(i, "sourceNamespace")) {
sourceServiceName = JOINER.join(string(i, "sourceService"), string(i, "sourceNamespace"));
} else {
sourceServiceName = string(i, "sourceService");
}
String destServiceName;
if (has(i, "destinationNamespace")) {
destServiceName = JOINER.join(string(i, "destinationService"), string(i, "destinationNamespace"));
} else {
destServiceName = string(i, "destinationService");
}
ServiceMeshMetric.Builder metrics = ServiceMeshMetric.newBuilder()
.setStartTime(requestTime.toEpochMilli())
.setEndTime(responseTime.toEpochMilli())
.setSourceServiceName(sourceServiceName)
.setSourceServiceInstance(string(i, "sourceUID"))
.setDestServiceName(destServiceName)
.setDestServiceInstance(string(i, "destinationUID"))
.setEndpoint(endpoint)
.setLatency(latency)
.setResponseCode(Math.toIntExact(responseCode))
.setStatus(status)
.setProtocol(netProtocol)
.setDetectPoint(detectPoint);
LOGGER.debug("Transformed metrics {}", metrics);
TelemetryDataDispatcher.process(metrics);
} finally {
timer.finish();
}
}
responseObserver.onNext(ReportProto.ReportResult.newBuilder().build());
responseObserver.onCompleted();
}
private String string(final IstioMetricProto.InstanceMsg instanceMsg, final String key) {
Map<String, TypeProto.Value> map = instanceMsg.getDimensionsMap();
assertDimension(map, key);
return map.get(key).getStringValue();
}
private long int64(final IstioMetricProto.InstanceMsg instanceMsg, final String key) {
Map<String, TypeProto.Value> map = instanceMsg.getDimensionsMap();
assertDimension(map, key);
return map.get(key).getInt64Value();
}
private Instant time(final IstioMetricProto.InstanceMsg instanceMsg, final String key) {
Map<String, TypeProto.Value> map = instanceMsg.getDimensionsMap();
assertDimension(map, key);
Timestamp timestamp = map.get(key).getTimestampValue().getValue();
return Instant.ofEpochSecond(timestamp.getSeconds(), timestamp.getNanos());
}
private void assertDimension(final Map<String, TypeProto.Value> map, final String key) {
if (!map.containsKey(key)) {
throw new IllegalArgumentException(String.format("Lack dimension %s", key));
}
}
private boolean has(final IstioMetricProto.InstanceMsg instanceMsg, final String key) {
Map<String, TypeProto.Value> map = instanceMsg.getDimensionsMap();
return map.containsKey(key);
}
}
#
# 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.
#
#
org.apache.skywalking.oap.server.receiver.istio.telemetry.module.IstioTelemetryReceiverModule
\ No newline at end of file
#
# 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.
#
#
org.apache.skywalking.oap.server.receiver.istio.telemetry.provider.IstioTelemetryReceiverProvider
\ No newline at end of file
/*
* 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.istio.telemetry.handler;
import com.google.protobuf.TextFormat;
import io.grpc.ManagedChannel;
import io.grpc.ManagedChannelBuilder;
import io.istio.HandleMetricServiceGrpc;
import io.istio.IstioMetricProto;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.LinkedList;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
public class IstioTelemetryHandlerMainTest {
public static void main(String[] args) throws InterruptedException {
ManagedChannel channel = ManagedChannelBuilder.forAddress("localhost", 11800).usePlaintext().build();
HandleMetricServiceGrpc.HandleMetricServiceBlockingStub stub = HandleMetricServiceGrpc.newBlockingStub(channel);
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
executor.schedule(() -> {
try {
send(stub);
} catch (IOException e) {
e.printStackTrace();
}
}, 1, TimeUnit.SECONDS);
Thread.sleep(5000L);
executor.shutdown();
}
private static void send(final HandleMetricServiceGrpc.HandleMetricServiceBlockingStub stub) throws IOException {
for (String s : readData()) {
IstioMetricProto.HandleMetricRequest.Builder requestBuilder = IstioMetricProto.HandleMetricRequest.newBuilder();
try (InputStreamReader isr = new InputStreamReader(getResourceAsStream(String.format("fixture/%s", s)))) {
TextFormat.getParser().merge(isr, requestBuilder);
}
stub.handleMetric(requestBuilder.build());
}
}
private static Iterable<String> readData() throws IOException {
Iterable<String> result = new LinkedList<>();
try (InputStream in = getResourceAsStream("fixture"); BufferedReader br = new BufferedReader(new InputStreamReader(in))) {
String resource;
while ((resource = br.readLine()) != null) {
((LinkedList<String>) result).add(resource);
}
}
return result;
}
private static InputStream getResourceAsStream(final String resource) {
final InputStream in = getContextClassLoader().getResourceAsStream(resource);
return in == null ? IstioTelemetryHandlerMainTest.class.getResourceAsStream(resource) : in;
}
private static ClassLoader getContextClassLoader() {
return Thread.currentThread().getContextClassLoader();
}
}
# 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.
instances {
value {
int64_value: 948
}
dimensions {
key: "sourceService"
value {
string_value: "istio-ingressgateway"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-ingressgateway-6f58fdc8d7-m29dq.istio-system"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 883239533
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 882149494
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-8p5sr.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
instances {
value {
int64_value: 688
}
dimensions {
key: "sourceService"
value {
string_value: "reviews-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://reviews-v1-59cbdd7959-g69ll.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 905792473
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 904701150
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-8p5sr.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296801"
# 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.
instances {
value {
int64_value: 940
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 887512227
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 886307886
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-dt82j.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
instances {
value {
int64_value: 689
}
dimensions {
key: "sourceService"
value {
string_value: "details-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://details-v1-7bcdcc4fd6-qb42z.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 895223970
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 894249545
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-dt82j.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296802"
# 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.
instances {
value {
int64_value: 940
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 887512227
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 886307886
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-dt82j.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
instances {
value {
int64_value: 689
}
dimensions {
key: "sourceService"
value {
string_value: "details-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://details-v1-7bcdcc4fd6-qb42z.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Check"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 895223970
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 894249545
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-dt82j.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296802"
# 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.
instances {
value {
int64_value: 0
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/details/0"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "GET"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 899188567
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 892490629
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "details-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://details-v1-7bcdcc4fd6-qb42z.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "source"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296803"
# 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.
instances {
value {
int64_value: 0
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 911565364
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/reviews/0"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://reviews-v1-59cbdd7959-g69ll.default"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 904205514
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "reviews-v1"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "GET"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306074"
# 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.
instances {
value {
int64_value: 0
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 912063918
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/reviews/0"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://reviews-v1-59cbdd7959-g69ll.default"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 903831844
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "reviews-v1"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "GET"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "source"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306075"
# 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.
instances {
value {
int64_value: 0
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 915154754
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/productpage"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "istio-ingressgateway"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 884254913
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "GET"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-ingressgateway-6f58fdc8d7-m29dq.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306076"
# 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.
instances {
value {
int64_value: 0
}
dimensions {
key: "sourceService"
value {
string_value: "istio-ingressgateway"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-ingressgateway-6f58fdc8d7-m29dq.istio-system"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/productpage"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "GET"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 915765768
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415799
nanos: 881503925
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "source"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296804"
# 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.
instances {
value {
int64_value: 1346
}
dimensions {
key: "sourceService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-8p5sr.istio-system"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 887532113
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 882648986
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-bs9v4.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296805"
# 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.
instances {
value {
int64_value: 1321
}
dimensions {
key: "sourceService"
value {
string_value: "istio-policy"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-policy-7fbd997765-dt82j.istio-system"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 892667623
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 888995363
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-bs9v4.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296806"
# 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.
instances {
value {
int64_value: 814
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 903133331
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "details-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-vx5dl.istio-system"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 899051593
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://details-v1-7bcdcc4fd6-qb42z.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306077"
# 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.
instances {
value {
int64_value: 858
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 903322857
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 900194688
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-bs9v4.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
instances {
value {
int64_value: 1154
}
dimensions {
key: "sourceService"
value {
string_value: "istio-ingressgateway"
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://istio-ingressgateway-6f58fdc8d7-m29dq.istio-system"
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 919832087
}
}
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 916615524
}
}
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-bs9v4.istio-system"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9307733128061296807"
# 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.
instances {
value {
int64_value: 904
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 917907619
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-vx5dl.istio-system"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 914305682
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306078"
# 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.
instances {
value {
int64_value: 860
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 917979066
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "reviews-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-vx5dl.istio-system"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 913319273
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://reviews-v1-59cbdd7959-g69ll.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
instances {
value {
int64_value: 1068
}
dimensions {
key: "responseTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 918964543
}
}
}
}
dimensions {
key: "requestPath"
value {
string_value: "/istio.mixer.v1.Mixer/Report"
}
}
dimensions {
key: "responseCode"
value {
int64_value: 200
}
}
dimensions {
key: "sourceService"
value {
string_value: "productpage-v1"
}
}
dimensions {
key: "destinationUID"
value {
string_value: "kubernetes://istio-telemetry-796dbc5d46-vx5dl.istio-system"
}
}
dimensions {
key: "requestTime"
value {
timestamp_value {
value {
seconds: 1537415800
nanos: 916174445
}
}
}
}
dimensions {
key: "destinationService"
value {
string_value: "istio-telemetry"
}
}
dimensions {
key: "requestMethod"
value {
string_value: "POST"
}
}
dimensions {
key: "apiProtocol"
value {
string_value: ""
}
}
dimensions {
key: "sourceUID"
value {
string_value: "kubernetes://productpage-v1-8584c875d8-fq8wh.default"
}
}
dimensions {
key: "reporter"
value {
string_value: "destination"
}
}
dimensions {
key: "requestScheme"
value {
string_value: "http"
}
}
name: "swmetric.instance.istio-system"
}
dedup_id: "9219286855927306079"
......@@ -63,7 +63,7 @@ public class H2MetadataQueryDAO implements IMetadataQueryDAO {
sql.append(ServiceTraffic.NODE_TYPE).append("=?");
condition.add(NodeType.Normal.value());
if (StringUtil.isNotEmpty(group)) {
sql.append(ServiceTraffic.GROUP).append("=?");
sql.append(" and ").append(ServiceTraffic.GROUP).append("=?");
condition.add(group);
}
sql.append(" limit ").append(metadataQueryMaxSize);
......
......@@ -105,12 +105,12 @@ public class MetricsQuery extends AbstractQuery<MetricsQuery> {
SERVICE_INSTANCE_RELATION_SERVER_CPM
};
public static String METER_INSTANCE_CPU_PERCENTAGE = "meter_instance_cpu_percentage";
public static String METER_INSTANCE_JVM_MEMORY_BYTES_USED = "meter_instance_jvm_memory_bytes_used";
public static String METER_INSTANCE_TRACE_COUNT = "meter_instance_trace_count";
public static String METER_INSTANCE_METRICS_FIRST_AGGREGATION = "meter_instance_metrics_first_aggregation";
public static String METER_INSTANCE_PERSISTENCE_PREPARE_COUNT = "meter_instance_persistence_prepare_count";
public static String METER_INSTANCE_PERSISTENCE_EXECUTE_COUNT = "meter_instance_persistence_execute_count";
public static String METER_INSTANCE_CPU_PERCENTAGE = "meter_oap_instance_cpu_percentage";
public static String METER_INSTANCE_JVM_MEMORY_BYTES_USED = "meter_oap_instance_jvm_memory_bytes_used";
public static String METER_INSTANCE_TRACE_COUNT = "meter_oap_instance_trace_count";
public static String METER_INSTANCE_METRICS_FIRST_AGGREGATION = "meter_oap_instance_metrics_first_aggregation";
public static String METER_INSTANCE_PERSISTENCE_PREPARE_COUNT = "meter_oap_instance_persistence_prepare_count";
public static String METER_INSTANCE_PERSISTENCE_EXECUTE_COUNT = "meter_oap_instance_persistence_execute_count";
public static String[] ALL_SO11Y_LINER_METRICS = {
METER_INSTANCE_CPU_PERCENTAGE,
......@@ -121,7 +121,7 @@ public class MetricsQuery extends AbstractQuery<MetricsQuery> {
METER_INSTANCE_PERSISTENCE_EXECUTE_COUNT
};
public static String METER_INSTANCE_PERSISTENCE_EXECUTE_PERCENTILE = "meter_instance_persistence_execute_percentile";
public static String METER_INSTANCE_PERSISTENCE_EXECUTE_PERCENTILE = "meter_oap_instance_persistence_execute_percentile";
public static String[] ALL_SO11Y_LABELED_METRICS = {
METER_INSTANCE_PERSISTENCE_EXECUTE_PERCENTILE
......
......@@ -28,7 +28,7 @@ import org.apache.skywalking.e2e.AbstractQuery;
@EqualsAndHashCode(callSuper = true)
public class ReadMetricsQuery extends AbstractQuery<ReadMetricsQuery> {
public static String METER_INSTANCE_CPU_PERCENTAGE = "meter_instance_cpu_percentage";
public static String METER_INSTANCE_CPU_PERCENTAGE = "meter_oap_instance_cpu_percentage";
public static String[] ALL_SO11Y_LINER_METRICS = {
METER_INSTANCE_CPU_PERCENTAGE
......
......@@ -21,7 +21,7 @@ services:
file: ../base-compose.yml
service: oap
environment:
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
SW_KAFKA_FETCHER: default
SW_KAFKA_FETCHER_SERVERS: broker-a:9092,broker-b:9092
......
......@@ -76,7 +76,7 @@ services:
file: ../base-compose.yml
service: oap
environment:
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
SW_KAFKA_FETCHER: default
SW_KAFKA_FETCHER_SERVERS: broker-a:9092,broker-b:9092
......
......@@ -22,7 +22,7 @@ services:
service: oap
environment:
SW_AUTHENTICATION: test-token
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
ui:
......
......@@ -18,7 +18,7 @@ version: '2.1'
services:
oap:
environment:
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
extends:
file: ../../base-compose.yml
......
......@@ -27,7 +27,7 @@ services:
SW_CORE_GRPC_SSL_KEY_PATH: /skywalking/certs/server-key.pem
SW_CORE_GRPC_SSL_CERT_CHAIN_PATH: /skywalking/certs/server.crt
SW_CORE_GRPC_SSL_TRUSTED_CA_PATH: /skywalking/certs/ca.crt
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
ui:
......
......@@ -37,7 +37,7 @@ services:
service: oap
environment:
SW_STORAGE: elasticsearch
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
depends_on:
es:
......
......@@ -37,7 +37,7 @@ services:
service: oap-es7
environment:
SW_STORAGE: elasticsearch7
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
depends_on:
es:
......
......@@ -37,7 +37,7 @@ services:
service: oap-es7
environment:
SW_STORAGE: elasticsearch7
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
depends_on:
es:
......
......@@ -34,7 +34,7 @@ services:
service: oap
environment:
SW_STORAGE: influxdb
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
depends_on:
influxdb:
......
......@@ -38,7 +38,7 @@ services:
service: oap
environment:
SW_STORAGE: mysql
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
depends_on:
mysql:
......
......@@ -37,7 +37,7 @@ services:
service: oap
environment:
SW_STORAGE: tidb
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
SW_JDBC_URL: "jdbc:mysql://tidb:4000/test"
SW_DATA_SOURCE_PASSWORD: ""
......
......@@ -37,7 +37,7 @@ services:
service: oap
environment:
SW_STORAGE: tidb
SW_PROMETHEUS_FETCHER_ACTIVE: "true"
SW_PROMETHEUS_FETCHER: "default"
SW_TELEMETRY: prometheus
SW_JDBC_URL: "jdbc:mysql://tidb:4000/test"
SW_DATA_SOURCE_PASSWORD: ""
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册