You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2021/03/23 09:04:03 UTC

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #6602: Enhance the export service.

kezhenxu94 commented on a change in pull request #6602:
URL: https://github.com/apache/skywalking/pull/6602#discussion_r599326924



##########
File path: docs/en/setup/backend/metrics-exporter.md
##########
@@ -61,8 +73,8 @@ exporter:
 
 ## For target exporter service 
 ### subscription implementation
-Return the expected metrics name list, all the names must match the OAL script definition. Return empty list, if you want
-to export all metrics.
+Return the expected metrics name list with event type(increment or total), all the names must match the OAL script definition. 

Review comment:
       "all the names must match the OAL script definition."
   
   Need to mention MAL metrics definitions here as well?

##########
File path: oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporter.java
##########
@@ -67,27 +67,46 @@ public GRPCExporter(GRPCExporterSetting setting) {
         blockingStub = MetricExportServiceGrpc.newBlockingStub(channel);
         exportBuffer = new DataCarrier<ExportData>(setting.getBufferChannelNum(), setting.getBufferChannelSize());
         exportBuffer.consume(this, 1, 200);
-        subscriptionSet = new HashSet<>();
+        subscriptionList = new ArrayList<>();
     }
 
     @Override
     public void export(ExportEvent event) {
-        if (ExportEvent.EventType.TOTAL == event.getType()) {
-            Metrics metrics = event.getMetrics();
-            if (metrics instanceof WithMetadata) {
-                MetricsMetaInfo meta = ((WithMetadata) metrics).getMeta();
-                if (subscriptionSet.size() == 0 || subscriptionSet.contains(meta.getMetricsName())) {
-                    exportBuffer.produce(new ExportData(meta, metrics));
-                }
+        Metrics metrics = event.getMetrics();
+        if (metrics instanceof WithMetadata) {
+            MetricsMetaInfo meta = ((WithMetadata) metrics).getMeta();
+            if (subscriptionList.size() == 0 && ExportEvent.EventType.INCREMENT.equals(event.getType())) {
+                exportBuffer.produce(new ExportData(meta, metrics, event.getType()));
+            } else {
+                subscriptionList.forEach(subscriptionMetric -> {
+                    if (subscriptionMetric.getMetricName().equals(meta.getMetricsName()) &&
+                        eventTypeMatch(event.getType(), subscriptionMetric.getEventType())) {
+                        exportBuffer.produce(new ExportData(meta, metrics, event.getType()));
+                    }
+                });
             }
+
+            fetchSubscriptionList();
         }
     }
 
-    public void initSubscriptionList() {
-        SubscriptionsResp subscription = blockingStub.withDeadlineAfter(10, TimeUnit.SECONDS)
-                                                     .subscription(SubscriptionReq.newBuilder().build());
-        subscription.getMetricNamesList().forEach(subscriptionSet::add);
-        LOGGER.debug("Get exporter subscription list, {}", subscriptionSet);
+    /**
+     * Read the subscription list.
+     */
+    public void fetchSubscriptionList() {
+        final long currentTimeMillis = System.currentTimeMillis();
+        if (currentTimeMillis - lastFetchTimestamp > 30_000) {
+            try {
+                lastFetchTimestamp = currentTimeMillis;
+                SubscriptionsResp subscription = blockingStub.withDeadlineAfter(10, TimeUnit.SECONDS)
+                                                             .subscription(SubscriptionReq.newBuilder().build());
+                subscriptionList.clear();
+                subscriptionList.addAll(subscription.getMetricsList());

Review comment:
       These 2 operations are not atomic and `export` is executed in another thread, causing `subscriptionList.size() == 0` (line 78) incorrectly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org