未验证 提交 b4cb7e0b 编写于 作者: wu-sheng's avatar wu-sheng 提交者: GitHub

Envoy metric support and a bug fix (#2321)

* Extend envoy metric service, scope and OAL lexer/parser.

* Finish metric extension and oal.

* Make test env works.

* Finish test of envoy connection and memory metric.

* Change document.

* Finish docker compose test.
上级 082a20eb
......@@ -86,6 +86,8 @@ service-mesh:
bufferFileCleanWhenRestart: ${SW_SERVICE_MESH_BUFFER_FILE_CLEAN_WHEN_RESTART:false}
istio-telemetry:
default:
envoy-metric:
default:
# receiver_zipkin:
# default:
# host: ${SW_RECEIVER_ZIPKIN_HOST:0.0.0.0}
......
......@@ -29,6 +29,7 @@
<logger name="org.elasticsearch.common.network.IfConfig" level="INFO"/>
<logger name="io.grpc.netty" level="INFO"/>
<logger name="org.apache.skywalking.oap.server.receiver.istio.telemetry" level="DEBUG"/>
<logger name="org.apache.skywalking.oap.server.receiver.envoy" level="DEBUG"/>
<Root level="INFO">
<AppenderRef ref="Console"/>
</Root>
......
......@@ -18,9 +18,11 @@ You could go to their project repositories to find out the releases and how to u
- [SkyAPM Node.js agent](https://github.com/SkyAPM/SkyAPM-nodejs). See Node.js server side agent project document for more details.
- [SkyAPM PHP SDK](https://github.com/SkyAPM/SkyAPM-php-sdk). See PHP agent project document for more details.
## On Service Mesh
## Service Mesh
- Istio
- [SkyWalking on Istio](istio/README.md). Introduce how to use Istio Mixer bypass Adapter to work with SkyWalking.
## Proxy
- [Envoy Proxy](https://www.envoyproxy.io/)
- [Sending metrics to Skywalking from Envoy](envoy/README.md). How to send metrics from Envoy to SkyWalking using [Metrics service](https://www.envoyproxy.io/docs/envoy/latest/api-v2/config/metrics/v2/metrics_service.proto.html).
......
......@@ -8,8 +8,9 @@ We have following receivers, and `default` implementors are provided in our Apac
1. **receiver-trace**. gRPC and HTTPRestful services to accept SkyWalking format traces.
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. **istio-telemetry**. Istio telemetry is from Istio official bypass adaptor, this receiver match its gRPC services.
1. **receiver-jvm**. gRPC services accept JVM metric data.
1. **istio-telemetry**. Istio telemetry is from Istio official bypass adaptor, this receiver match its gRPC services.
1. **envoy-metric**. Envoy `metrics_service` supported by this receiver. OAL script support all GAUGE type metrics.
1. **receiver_zipkin**. HTTP service accepts Span in Zipkin v1 and v2 formats. Notice, this receiver only
works as expected in backend single node mode. Cluster mode is not supported. Welcome anyone to improve this.
......@@ -34,6 +35,8 @@ service-mesh:
bufferFileCleanWhenRestart: false
istio-telemetry:
default:
envoy-metric:
default:
receiver_zipkin:
default:
host: 0.0.0.0
......
......@@ -41,6 +41,7 @@ SRC_DATABASE_ACCESS: 'DatabaseAccess';
SRC_SERVICE_INSTANCE_CLR_CPU: 'ServiceInstanceCLRCPU';
SRC_SERVICE_INSTANCE_CLR_GC: 'ServiceInstanceCLRGC';
SRC_SERVICE_INSTANCE_CLR_THREAD: 'ServiceInstanceCLRThread';
SRC_ENVOY_INSTANCE_METRIC: 'EnvoyInstanceMetric';
// Literals
......
......@@ -49,7 +49,8 @@ source
: SRC_ALL | SRC_SERVICE | SRC_DATABASE_ACCESS | SRC_SERVICE_INSTANCE | SRC_ENDPOINT |
SRC_SERVICE_RELATION | SRC_SERVICE_INSTANCE_RELATION | SRC_ENDPOINT_RELATION |
SRC_SERVICE_INSTANCE_JVM_CPU | SRC_SERVICE_INSTANCE_JVM_MEMORY | SRC_SERVICE_INSTANCE_JVM_MEMORY_POOL | SRC_SERVICE_INSTANCE_JVM_GC |// JVM source of service instance
SRC_SERVICE_INSTANCE_CLR_CPU | SRC_SERVICE_INSTANCE_CLR_GC | SRC_SERVICE_INSTANCE_CLR_THREAD
SRC_SERVICE_INSTANCE_CLR_CPU | SRC_SERVICE_INSTANCE_CLR_GC | SRC_SERVICE_INSTANCE_CLR_THREAD |
SRC_ENVOY_INSTANCE_METRIC
;
sourceAttribute
......
......@@ -163,4 +163,14 @@ scopes:
- fieldName: serviceInstanceId
columnName: service_instance_id
typeName: int
ID: false
\ No newline at end of file
ID: false
- name: EnvoyInstanceMetric
columns:
- fieldName: entityId
columnName: entity_id
typeName: java.lang.String
ID: true
- fieldName: serviceInstanceId
columnName: service_instance_id
typeName: int
ID: false
\ No newline at end of file
......@@ -91,3 +91,8 @@ instance_clr_available_completion_port_threads = from(ServiceInstanceCLRThread.a
instance_clr_available_worker_threads = from(ServiceInstanceCLRThread.availableWorkerThreads).max();
instance_clr_max_completion_port_threads = from(ServiceInstanceCLRThread.maxCompletionPortThreads).max();
instance_clr_max_worker_threads = from(ServiceInstanceCLRThread.maxWorkerThreads).max();
// Envoy instance metric
envoy_heap_memory_max_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.memory_heap_size").maxDouble();
envoy_total_connections_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.total_connections").maxDouble();
envoy_parent_connections_used = from(EnvoyInstanceMetric.value).filter(metricName == "server.parent_connections").maxDouble();
/*
* 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.core.analysis.indicator;
import lombok.*;
import org.apache.skywalking.oap.server.core.analysis.indicator.annotation.*;
import org.apache.skywalking.oap.server.core.storage.annotation.Column;
/**
* @author wusheng
*/
@IndicatorFunction(functionName = "maxDouble")
public abstract class MaxDoubleIndicator extends Indicator implements DoubleValueHolder {
protected static final String VALUE = "value";
@Getter @Setter @Column(columnName = VALUE, isValue = true) private double value;
@Entrance
public final void combine(@SourceFrom double count) {
if (count > this.value) {
this.value = count;
}
}
@Override public final void combine(Indicator indicator) {
MaxDoubleIndicator countIndicator = (MaxDoubleIndicator)indicator;
combine(countIndicator.value);
}
@Override public void calculate() {
}
@Override public double getValue() {
return value;
}
}
......@@ -29,7 +29,7 @@ import org.apache.skywalking.oap.server.core.storage.annotation.Column;
* @author liuhaoyang
**/
@IndicatorFunction(functionName = "max")
public abstract class MaxIndicator extends Indicator implements LongValueHolder {
public abstract class MaxLongIndicator extends Indicator implements LongValueHolder {
protected static final String VALUE = "value";
......@@ -43,7 +43,7 @@ public abstract class MaxIndicator extends Indicator implements LongValueHolder
}
@Override public final void combine(Indicator indicator) {
MaxIndicator countIndicator = (MaxIndicator)indicator;
MaxLongIndicator countIndicator = (MaxLongIndicator)indicator;
combine(countIndicator.value);
}
......
......@@ -19,11 +19,12 @@
package org.apache.skywalking.oap.server.core.analysis.indicator.expression;
/**
*
*
* @author wusheng
*/
public class EqualMatch extends BinaryMatchExpression {
@Override public boolean match() {
return left == right;
return left.equals(right);
}
}
......@@ -31,6 +31,11 @@ public class DefaultScopeDefine {
private static final Map<String, Integer> NAME_2_ID = new HashMap<>();
private static final Map<Integer, String> ID_2_NAME = new HashMap<>();
/**
* All metric IDs in [0, 10,000) are reserved in Apache SkyWalking.
*
* If you want to extend the scope, recommend to start with 10,000.
*/
public static final int ALL = 0;
public static final int SERVICE = 1;
public static final int SERVICE_INSTANCE = 2;
......@@ -53,6 +58,7 @@ public class DefaultScopeDefine {
public static final int SERVICE_INSTANCE_CLR_CPU = 19;
public static final int SERVICE_INSTANCE_CLR_GC = 20;
public static final int SERVICE_INSTANCE_CLR_THREAD = 21;
public static final int ENVOY_INSTANCE_METRIC = 22;
public static class Listener implements AnnotationListener {
@Override public Class<? extends Annotation> annotation() {
......
/*
* 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.core.source;
import lombok.*;
import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENVOY_INSTANCE_METRIC;
/**
* The envoy metrics. This group of metrics are in Prometheus metric format family.
*
* This metric source supports Counter and Gauge types.
*
* @author wusheng
*/
@ScopeDeclaration(id = ENVOY_INSTANCE_METRIC, name = "EnvoyInstanceMetric")
public class EnvoyInstanceMetric extends Source {
@Override public int scope() {
return ENVOY_INSTANCE_METRIC;
}
@Override public String getEntityId() {
return String.valueOf(id);
}
/**
* Instance id
*/
@Getter @Setter private int id;
@Getter @Setter private int serviceId;
@Getter @Setter private int serviceInstanceId;
@Getter @Setter private String name;
@Getter @Setter private String serviceName;
@Getter @Setter private String metricName;
@Getter @Setter private double value;
}
......@@ -25,11 +25,11 @@ import org.junit.Test;
/**
* @author liuhaoyang
**/
public class MaxIndicatorTest {
public class MaxLongIndicatorTest {
@Test
public void testEntranceCombine() {
MaxIndicatorImpl impl = new MaxIndicatorImpl();
MaxLongIndicatorImpl impl = new MaxLongIndicatorImpl();
impl.combine(10);
impl.combine(5);
impl.combine(20);
......@@ -39,11 +39,11 @@ public class MaxIndicatorTest {
@Test
public void testSelfCombine() {
MaxIndicatorImpl impl = new MaxIndicatorImpl();
MaxLongIndicatorImpl impl = new MaxLongIndicatorImpl();
impl.combine(10);
impl.combine(5);
MaxIndicatorImpl impl2 = new MaxIndicatorImpl();
MaxLongIndicatorImpl impl2 = new MaxLongIndicatorImpl();
impl.combine(2);
impl.combine(6);
......@@ -51,7 +51,7 @@ public class MaxIndicatorTest {
Assert.assertEquals(10, impl.getValue());
}
public class MaxIndicatorImpl extends MaxIndicator {
public class MaxLongIndicatorImpl extends MaxLongIndicator {
@Override public String id() {
return null;
......
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.apache.skywalking.oap.server.receiver.envoy;
import org.apache.skywalking.oap.server.library.module.ModuleDefine;
/**
* Envoy metric receiver module
*
* @author wusheng
*/
public class EnvoyMetricReceiverModule extends ModuleDefine {
public static final String NAME = "envoy-metric";
public EnvoyMetricReceiverModule() {
super(NAME);
}
@Override public Class[] services() {
return new Class[0];
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.apache.skywalking.oap.server.receiver.envoy;
import org.apache.skywalking.oap.server.core.CoreModule;
import org.apache.skywalking.oap.server.core.server.GRPCHandlerRegister;
import org.apache.skywalking.oap.server.library.module.*;
import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
/**
* @author wusheng
*/
public class EnvoyMetricReceiverProvider extends ModuleProvider {
@Override public String name() {
return "default";
}
@Override public Class<? extends ModuleDefine> module() {
return EnvoyMetricReceiverModule.class;
}
@Override public ModuleConfig createConfigBeanIfAbsent() {
return null;
}
@Override public void prepare() throws ServiceNotProvidedException, ModuleStartException {
}
@Override public void start() throws ServiceNotProvidedException, ModuleStartException {
GRPCHandlerRegister service = getManager().find(CoreModule.NAME).provider().getService(GRPCHandlerRegister.class);
service.addHandler(new MetricServiceGRPCHandler(getManager()));
}
@Override public void notifyAfterCompleted() throws ServiceNotProvidedException, ModuleStartException {
}
@Override public String[] requiredModules() {
return new String[] {TelemetryModule.NAME, CoreModule.NAME};
}
}
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.apache.skywalking.oap.server.receiver.envoy;
import io.envoyproxy.envoy.api.v2.core.Node;
import io.envoyproxy.envoy.service.metrics.v2.*;
import io.grpc.stub.StreamObserver;
import io.prometheus.client.Metrics;
import java.util.List;
import org.apache.skywalking.apm.util.StringUtil;
import org.apache.skywalking.oap.server.core.*;
import org.apache.skywalking.oap.server.core.register.service.*;
import org.apache.skywalking.oap.server.core.source.*;
import org.apache.skywalking.oap.server.library.module.ModuleManager;
import org.apache.skywalking.oap.server.library.util.TimeBucketUtils;
import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
import org.apache.skywalking.oap.server.telemetry.api.*;
import org.slf4j.*;
/**
* @author wusheng
*/
public class MetricServiceGRPCHandler extends MetricsServiceGrpc.MetricsServiceImplBase {
private static final Logger logger = LoggerFactory.getLogger(MetricServiceGRPCHandler.class);
private final IServiceInventoryRegister serviceInventoryRegister;
private final IServiceInstanceInventoryRegister serviceInstanceInventoryRegister;
private final SourceReceiver sourceReceiver;
private CounterMetric counter;
private HistogramMetric histogram;
public MetricServiceGRPCHandler(ModuleManager moduleManager) {
serviceInventoryRegister = moduleManager.find(CoreModule.NAME).provider().getService(IServiceInventoryRegister.class);
serviceInstanceInventoryRegister = moduleManager.find(CoreModule.NAME).provider().getService(IServiceInstanceInventoryRegister.class);
sourceReceiver = moduleManager.find(CoreModule.NAME).provider().getService(SourceReceiver.class);
MetricCreator metricCreator = moduleManager.find(TelemetryModule.NAME).provider().getService(MetricCreator.class);
counter = metricCreator.createCounter("envoy_metric_in_count", "The count of envoy service metric received",
MetricTag.EMPTY_KEY, MetricTag.EMPTY_VALUE);
histogram = metricCreator.createHistogramMetric("envoy_metric_in_latency", "The process latency of service metric receiver",
MetricTag.EMPTY_KEY, MetricTag.EMPTY_VALUE);
}
@Override
public StreamObserver<StreamMetricsMessage> streamMetrics(StreamObserver<StreamMetricsResponse> responseObserver) {
return new StreamObserver<StreamMetricsMessage>() {
private volatile boolean isFirst = true;
private String serviceName = null;
private int serviceId = Const.NONE;
private String serviceInstanceName = null;
private int serviceInstanceId = Const.NONE;
@Override public void onNext(StreamMetricsMessage message) {
if (isFirst) {
isFirst = false;
StreamMetricsMessage.Identifier identifier = message.getIdentifier();
logger.debug("Received identifier msg {}", identifier);
Node node = identifier.getNode();
if (node != null) {
String nodeId = node.getId();
if (!StringUtil.isEmpty(nodeId)) {
serviceInstanceName = nodeId;
}
String cluster = node.getCluster();
if (!StringUtil.isEmpty(cluster)) {
serviceName = cluster;
if (serviceInstanceName == null) {
serviceInstanceName = serviceName;
}
}
}
if (serviceName == null) {
serviceName = serviceInstanceName;
}
}
if (logger.isDebugEnabled()) {
logger.debug("Envoy metric reported from service[{}], service instance[{}]", serviceName, serviceInstanceName);
}
if (serviceInstanceId != Const.NONE) {
List<Metrics.MetricFamily> list = message.getEnvoyMetricsList();
for (int i = 0; i < list.size(); i++) {
counter.inc();
HistogramMetric.Timer timer = histogram.createTimer();
try {
Metrics.MetricFamily metricFamily = list.get(i);
double value = 0;
long timestamp = 0;
switch (metricFamily.getType()) {
case GAUGE:
for (Metrics.Metric metric : metricFamily.getMetricList()) {
timestamp = metric.getTimestampMs();
value = metric.getGauge().getValue();
EnvoyInstanceMetric metricSource = new EnvoyInstanceMetric();
metricSource.setServiceId(serviceId);
metricSource.setServiceName(serviceName);
metricSource.setId(serviceInstanceId);
metricSource.setServiceInstanceId(serviceInstanceId);
metricSource.setName(serviceInstanceName);
metricSource.setMetricName(metricFamily.getName());
metricSource.setValue(value);
metricSource.setTimeBucket(TimeBucketUtils.INSTANCE.getMinuteTimeBucket(timestamp));
sourceReceiver.receive(metricSource);
}
break;
default:
continue;
}
if (i == 0) {
// Send heartbeat
serviceInventoryRegister.heartbeat(serviceId, timestamp);
serviceInstanceInventoryRegister.heartbeat(serviceInstanceId, timestamp);
}
} finally {
timer.finish();
}
}
} else if (serviceName != null && serviceInstanceName != null) {
if (serviceId == Const.NONE) {
logger.debug("Register envoy service [{}].", serviceName);
serviceId = serviceInventoryRegister.getOrCreate(serviceName, null);
}
if (serviceId != Const.NONE) {
logger.debug("Register envoy service instance [{}].", serviceInstanceName);
serviceInstanceId = serviceInstanceInventoryRegister.getOrCreate(serviceId, serviceInstanceName, serviceInstanceName, System.currentTimeMillis(), null);
}
}
}
@Override public void onError(Throwable throwable) {
logger.error("Error in receiving metric from envoy", throwable);
responseObserver.onCompleted();
}
@Override public void onCompleted() {
responseObserver.onNext(StreamMetricsResponse.newBuilder().build());
responseObserver.onCompleted();
}
};
}
}
#
# 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.envoy.EnvoyMetricReceiverModule
\ 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.envoy.EnvoyMetricReceiverProvider
\ 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.envoy;
import com.google.protobuf.TextFormat;
import io.envoyproxy.envoy.service.metrics.v2.*;
import io.grpc.*;
import io.grpc.stub.StreamObserver;
import io.prometheus.client.Metrics;
import java.io.*;
import java.util.concurrent.*;
/**
* @author wusheng
*/
public class MetricServiceGRPCHandlerTestMain {
public static void main(String[] args) throws InterruptedException {
ManagedChannel channel = ManagedChannelBuilder.forAddress("localhost", 11800).usePlaintext(true).build();
MetricsServiceGrpc.MetricsServiceStub stub = MetricsServiceGrpc.newStub(channel);
ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor();
executor.schedule(() -> {
try {
send(stub);
} catch (IOException e) {
e.printStackTrace();
} catch (InterruptedException e) {
e.printStackTrace();
}
}, 1, TimeUnit.SECONDS);
Thread.sleep(5000L);
executor.shutdown();
}
private static void send(
final MetricsServiceGrpc.MetricsServiceStub stub) throws IOException, InterruptedException {
StreamObserver<StreamMetricsMessage> messageStreamObserver = stub.streamMetrics(new StreamObserver<StreamMetricsResponse>() {
@Override public void onNext(StreamMetricsResponse response) {
}
@Override public void onError(Throwable throwable) {
}
@Override public void onCompleted() {
}
});
int countdown = 20;
while (countdown-- > 0) {
try (InputStreamReader isr = new InputStreamReader(getResourceAsStream("envoy-metric.msg"))) {
StreamMetricsMessage.Builder requestBuilder = StreamMetricsMessage.newBuilder();
TextFormat.getParser().merge(isr, requestBuilder);
for (Metrics.MetricFamily.Builder builder : requestBuilder.getEnvoyMetricsBuilderList()) {
for (Metrics.Metric.Builder metricBuilder : builder.getMetricBuilderList()) {
metricBuilder.setTimestampMs(System.currentTimeMillis());
}
}
messageStreamObserver.onNext(requestBuilder.build());
Thread.sleep(200L);
}
}
}
private static InputStream getResourceAsStream(final String resource) {
final InputStream in = getContextClassLoader().getResourceAsStream(resource);
return in == null ? MetricServiceGRPCHandlerTestMain.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.
identifier {
node {
id: "ingress",
cluster: "envoy-proxy",
metadata {
fields {
key: "envoy"
value {
string_value: "isawesome"
}
key: "skywalking"
value {
string_value: "iscool"
}
}
},
locality {
region: "ap-southeast-1"
zone: "zone1"
sub_zone: "subzone1"
},
build_version: "caf7ab123964cedd172a2d4cb29b2f2e05ca9156/1.10.0-dev/Clean/RELEASE/BoringSSL"
}
}
envoy_metrics [
{
name: "cluster.service_stats.update_attempt",
type: COUNTER
metric [
{
counter {
value: 1
},
timestamp_ms: 1551781658343
}
]
},
{
name: "cluster.service_stats.membership_change",
type: COUNTER
metric [
{
counter {
value: 1
},
timestamp_ms: 1551781658343
}
]
},
{
name: "server.parent_connections",
type: GAUGE
metric [
{
gauge {
value: 50
},
timestamp_ms: 1551781658343
}
]
},
{
name: "server.total_connections",
type: GAUGE
metric [
{
gauge {
value: 14
},
timestamp_ms: 1551781658343
}
]
},
{
name: "server.memory_heap_size",
type: GAUGE
metric [
{
gauge {
value: 3145728
},
timestamp_ms: 1551781658343
}
]
}
]
......@@ -98,6 +98,11 @@
<artifactId>skywalking-trace-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>envoy-metrics-receiver-plugin</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.skywalking</groupId>
<artifactId>zipkin-receiver-plugin</artifactId>
......
......@@ -78,14 +78,16 @@ receiver-trace:
slowDBAccessThreshold: ${SW_SLOW_DB_THRESHOLD:default:200,mongodb:100} # The slow database access thresholds. Unit ms.
receiver-jvm:
default:
#service-mesh:
# default:
# bufferPath: ${SW_SERVICE_MESH_BUFFER_PATH:../mesh-buffer/} # Path to trace buffer files, suggest to use absolute path
# bufferOffsetMaxFileSize: ${SW_SERVICE_MESH_OFFSET_MAX_FILE_SIZE:100} # Unit is MB
# bufferDataMaxFileSize: ${SW_SERVICE_MESH_BUFFER_DATA_MAX_FILE_SIZE:500} # Unit is MB
# bufferFileCleanWhenRestart: ${SW_SERVICE_MESH_BUFFER_FILE_CLEAN_WHEN_RESTART:false}
#istio-telemetry:
# default:
service-mesh:
default:
bufferPath: ${SW_SERVICE_MESH_BUFFER_PATH:../mesh-buffer/} # Path to trace buffer files, suggest to use absolute path
bufferOffsetMaxFileSize: ${SW_SERVICE_MESH_OFFSET_MAX_FILE_SIZE:100} # Unit is MB
bufferDataMaxFileSize: ${SW_SERVICE_MESH_BUFFER_DATA_MAX_FILE_SIZE:500} # Unit is MB
bufferFileCleanWhenRestart: ${SW_SERVICE_MESH_BUFFER_FILE_CLEAN_WHEN_RESTART:false}
istio-telemetry:
default:
envoy-metric:
default:
#receiver_zipkin:
# default:
# host: ${SW_RECEIVER_ZIPKIN_HOST:0.0.0.0}
......
......@@ -86,6 +86,8 @@ service-mesh:
bufferFileCleanWhenRestart: ${SW_SERVICE_MESH_BUFFER_FILE_CLEAN_WHEN_RESTART:false}
istio-telemetry:
default:
envoy-metric:
default:
#receiver_zipkin:
# default:
# host: ${SW_RECEIVER_ZIPKIN_HOST:0.0.0.0}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册