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 2018/04/23 06:25:30 UTC

[GitHub] wu-sheng closed pull request #1107: The implementation of getThermodynamic in overview page.

wu-sheng closed pull request #1107: The implementation of getThermodynamic in overview page.
URL: https://github.com/apache/incubator-skywalking/pull/1107
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/main/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListener.java b/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/main/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListener.java
index e923bc57a..0f5ba2186 100644
--- a/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/main/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListener.java
+++ b/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/main/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListener.java
@@ -26,6 +26,8 @@
 import org.apache.skywalking.apm.collector.analysis.segment.parser.define.listener.LocalSpanListener;
 import org.apache.skywalking.apm.collector.analysis.segment.parser.define.listener.SpanListener;
 import org.apache.skywalking.apm.collector.analysis.segment.parser.define.listener.SpanListenerFactory;
+import org.apache.skywalking.apm.collector.configuration.ConfigurationModule;
+import org.apache.skywalking.apm.collector.configuration.service.IResponseTimeDistributionConfigService;
 import org.apache.skywalking.apm.collector.core.annotations.trace.GraphComputingMetric;
 import org.apache.skywalking.apm.collector.core.graph.Graph;
 import org.apache.skywalking.apm.collector.core.graph.GraphManager;
@@ -47,6 +49,11 @@
     private boolean isError = false;
     private int entrySpanDuration = 0;
     private int firstSpanDuration = 0;
+    private final IResponseTimeDistributionConfigService configService;
+
+    ResponseTimeDistributionSpanListener(ModuleManager moduleManager) {
+        this.configService = moduleManager.find(ConfigurationModule.NAME).getService(IResponseTimeDistributionConfigService.class);
+    }
 
     @Override public boolean containsPoint(Point point) {
         return Point.First.equals(point) || Point.Entry.equals(point) || Point.Exit.equals(point) || Point.Local.equals(point);
@@ -101,8 +108,8 @@ public void parseFirst(SpanDecorator spanDecorator, int applicationId, int insta
     }
 
     int getStep() {
-        int abovePoint = 3000;
-        int interval = 50;
+        int responseTimeMaxStep = configService.getResponseTimeStep() * configService.getResponseTimeMaxStep();
+        int responseTimeStep = configService.getResponseTimeStep();
 
         int duration;
         if (entrySpanDuration == 0) {
@@ -111,12 +118,12 @@ int getStep() {
             duration = entrySpanDuration;
         }
 
-        if (duration > abovePoint) {
-            return abovePoint / interval;
-        } else if (duration <= interval) {
+        if (duration > responseTimeMaxStep) {
+            return responseTimeMaxStep / responseTimeStep;
+        } else if (duration <= responseTimeStep) {
             return 0;
         } else {
-            return (int)Math.ceil((double)duration / (double)interval) - 1;
+            return (int)Math.ceil((double)duration / (double)responseTimeStep) - 1;
         }
     }
 
@@ -124,7 +131,7 @@ int getStep() {
 
         @GraphComputingMetric(name = "/segment/parse/createSpanListeners/responseTimeDistributionSpanListener")
         @Override public SpanListener create(ModuleManager moduleManager) {
-            return new ResponseTimeDistributionSpanListener();
+            return new ResponseTimeDistributionSpanListener(moduleManager);
         }
     }
 }
diff --git a/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/test/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListenerTestCase.java b/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/test/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListenerTestCase.java
index b67a9c34a..339e00042 100644
--- a/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/test/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListenerTestCase.java
+++ b/apm-collector/apm-collector-analysis/analysis-metric/metric-provider/src/test/java/org/apache/skywalking/apm/collector/analysis/metric/provider/worker/global/std/ResponseTimeDistributionSpanListenerTestCase.java
@@ -18,8 +18,15 @@
 
 package org.apache.skywalking.apm.collector.analysis.metric.provider.worker.global.std;
 
+import java.util.LinkedList;
+import org.apache.skywalking.apm.collector.configuration.ConfigurationModule;
+import org.apache.skywalking.apm.collector.configuration.service.IResponseTimeDistributionConfigService;
+import org.apache.skywalking.apm.collector.core.module.Module;
+import org.apache.skywalking.apm.collector.core.module.ModuleManager;
+import org.apache.skywalking.apm.collector.core.module.ModuleProvider;
 import org.junit.Assert;
 import org.junit.Test;
+import org.mockito.Mockito;
 import org.powermock.reflect.Whitebox;
 
 /**
@@ -29,7 +36,22 @@
 
     @Test
     public void testStep() {
-        ResponseTimeDistributionSpanListener listener = new ResponseTimeDistributionSpanListener();
+        ModuleProvider moduleProvider = Mockito.mock(ModuleProvider.class);
+        LinkedList<ModuleProvider> loadedProviders = new LinkedList<>();
+        loadedProviders.add(moduleProvider);
+
+        IResponseTimeDistributionConfigService service = Mockito.mock(IResponseTimeDistributionConfigService.class);
+        Mockito.when(service.getResponseTimeStep()).thenReturn(50);
+        Mockito.when(service.getResponseTimeMaxStep()).thenReturn(40);
+
+        Module module = Mockito.mock(Module.class);
+        Whitebox.setInternalState(module, "loadedProviders", loadedProviders);
+        Mockito.when(module.getService(IResponseTimeDistributionConfigService.class)).thenReturn(service);
+
+        ModuleManager moduleManager = Mockito.mock(ModuleManager.class);
+        Mockito.when(moduleManager.find(ConfigurationModule.NAME)).thenReturn(module);
+
+        ResponseTimeDistributionSpanListener listener = new ResponseTimeDistributionSpanListener(moduleManager);
 
         Whitebox.setInternalState(listener, "entrySpanDuration", 0);
         Whitebox.setInternalState(listener, "firstSpanDuration", 200);
@@ -42,6 +64,6 @@ public void testStep() {
         Assert.assertEquals(1, listener.getStep());
 
         Whitebox.setInternalState(listener, "entrySpanDuration", 3100);
-        Assert.assertEquals(60, listener.getStep());
+        Assert.assertEquals(40, listener.getStep());
     }
 }
diff --git a/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModule.java b/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModule.java
index 18ede2260..e4040526a 100644
--- a/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModule.java
+++ b/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModule.java
@@ -25,6 +25,7 @@
 import org.apache.skywalking.apm.collector.configuration.service.IComponentLibraryCatalogService;
 import org.apache.skywalking.apm.collector.configuration.service.IInstanceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.IInstanceReferenceAlarmRuleConfig;
+import org.apache.skywalking.apm.collector.configuration.service.IResponseTimeDistributionConfigService;
 import org.apache.skywalking.apm.collector.configuration.service.IServiceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.IServiceReferenceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.core.module.Module;
@@ -46,6 +47,6 @@
             IApdexThresholdService.class,
             IServiceAlarmRuleConfig.class, IInstanceAlarmRuleConfig.class, IApplicationAlarmRuleConfig.class,
             IServiceReferenceAlarmRuleConfig.class, IInstanceReferenceAlarmRuleConfig.class, IApplicationReferenceAlarmRuleConfig.class,
-            IComponentLibraryCatalogService.class};
+            IComponentLibraryCatalogService.class, IResponseTimeDistributionConfigService.class};
     }
 }
diff --git a/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/service/IResponseTimeDistributionConfigService.java b/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/service/IResponseTimeDistributionConfigService.java
new file mode 100644
index 000000000..08047fac2
--- /dev/null
+++ b/apm-collector/apm-collector-configuration/collector-configuration-define/src/main/java/org/apache/skywalking/apm/collector/configuration/service/IResponseTimeDistributionConfigService.java
@@ -0,0 +1,31 @@
+/*
+ * 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.apm.collector.configuration.service;
+
+import org.apache.skywalking.apm.collector.core.module.Service;
+
+/**
+ * @author peng-yongsheng
+ */
+public interface IResponseTimeDistributionConfigService extends Service {
+
+    int getResponseTimeStep();
+
+    int getResponseTimeMaxStep();
+}
diff --git a/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleConfig.java b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleConfig.java
index 754a1eed7..7d836a29e 100644
--- a/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleConfig.java
+++ b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleConfig.java
@@ -33,6 +33,8 @@
     private int instanceAverageResponseTimeThreshold;
     private double applicationErrorRateThreshold;
     private int applicationAverageResponseTimeThreshold;
+    private int responseTimeDistributionDuration;
+    private int responseTimeDistributionMaxDurationns;
 
     public String getNamespace() {
         return namespace;
@@ -97,4 +99,20 @@ public int getApplicationAverageResponseTimeThreshold() {
     public void setApplicationAverageResponseTimeThreshold(int applicationAverageResponseTimeThreshold) {
         this.applicationAverageResponseTimeThreshold = applicationAverageResponseTimeThreshold;
     }
+
+    public int getResponseTimeDistributionDuration() {
+        return responseTimeDistributionDuration;
+    }
+
+    public void setResponseTimeDistributionDuration(int responseTimeDistributionDuration) {
+        this.responseTimeDistributionDuration = responseTimeDistributionDuration;
+    }
+
+    public int getResponseTimeDistributionMaxDurationns() {
+        return responseTimeDistributionMaxDurationns;
+    }
+
+    public void setResponseTimeDistributionMaxDurationns(int responseTimeDistributionMaxDurationns) {
+        this.responseTimeDistributionMaxDurationns = responseTimeDistributionMaxDurationns;
+    }
 }
diff --git a/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleProvider.java b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleProvider.java
index 108926b87..9cfba732e 100644
--- a/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleProvider.java
+++ b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/ConfigurationModuleProvider.java
@@ -30,10 +30,12 @@
 import org.apache.skywalking.apm.collector.configuration.service.IComponentLibraryCatalogService;
 import org.apache.skywalking.apm.collector.configuration.service.IInstanceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.IInstanceReferenceAlarmRuleConfig;
+import org.apache.skywalking.apm.collector.configuration.service.IResponseTimeDistributionConfigService;
 import org.apache.skywalking.apm.collector.configuration.service.IServiceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.IServiceReferenceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.InstanceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.InstanceReferenceAlarmRuleConfig;
+import org.apache.skywalking.apm.collector.configuration.service.ResponseTimeDistributionConfigService;
 import org.apache.skywalking.apm.collector.configuration.service.ServiceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.configuration.service.ServiceReferenceAlarmRuleConfig;
 import org.apache.skywalking.apm.collector.core.module.Module;
@@ -69,13 +71,16 @@ public ConfigurationModuleProvider() {
 
     @Override public void prepare() throws ServiceNotProvidedException {
         String namespace = StringUtils.isNotEmpty(config.getNamespace()) ? config.getNamespace() : Const.EMPTY_STRING;
-        Integer applicationApdexThreshold = config.getApplicationApdexThreshold() == 0 ? 2000 : config.getApplicationApdexThreshold();
-        Double serviceErrorRateThreshold = config.getServiceErrorRateThreshold() == 0 ? 10.00 : config.getServiceErrorRateThreshold();
-        Integer serviceAverageResponseTimeThreshold = config.getServiceAverageResponseTimeThreshold() == 0 ? 2000 : config.getServiceAverageResponseTimeThreshold();
-        Double instanceErrorRateThreshold = config.getInstanceErrorRateThreshold() == 0 ? 10.00 : config.getInstanceErrorRateThreshold();
-        Integer instanceAverageResponseTimeThreshold = config.getInstanceAverageResponseTimeThreshold() == 0 ? 2000 : config.getInstanceAverageResponseTimeThreshold();
-        Double applicationErrorRateThreshold = config.getApplicationErrorRateThreshold() == 0 ? 10.00 : config.getApplicationErrorRateThreshold();
-        Integer applicationAverageResponseTimeThreshold = config.getApplicationAverageResponseTimeThreshold() == 0 ? 2000 : config.getApplicationAverageResponseTimeThreshold();
+        int applicationApdexThreshold = config.getApplicationApdexThreshold() == 0 ? 2000 : config.getApplicationApdexThreshold();
+        double serviceErrorRateThreshold = config.getServiceErrorRateThreshold() == 0 ? 10.00 : config.getServiceErrorRateThreshold();
+        int serviceAverageResponseTimeThreshold = config.getServiceAverageResponseTimeThreshold() == 0 ? 2000 : config.getServiceAverageResponseTimeThreshold();
+        double instanceErrorRateThreshold = config.getInstanceErrorRateThreshold() == 0 ? 10.00 : config.getInstanceErrorRateThreshold();
+        int instanceAverageResponseTimeThreshold = config.getInstanceAverageResponseTimeThreshold() == 0 ? 2000 : config.getInstanceAverageResponseTimeThreshold();
+        double applicationErrorRateThreshold = config.getApplicationErrorRateThreshold() == 0 ? 10.00 : config.getApplicationErrorRateThreshold();
+        int applicationAverageResponseTimeThreshold = config.getApplicationAverageResponseTimeThreshold() == 0 ? 2000 : config.getApplicationAverageResponseTimeThreshold();
+
+        int responseTimeDistributionDuration = config.getResponseTimeDistributionDuration() == 0 ? 50 : config.getResponseTimeDistributionDuration();
+        int responseTimeDistributionMaxDurations = config.getResponseTimeDistributionMaxDurationns() == 0 ? 40 : config.getResponseTimeDistributionMaxDurationns();
 
         this.registerServiceImplementation(ICollectorConfig.class, new CollectorConfigService(namespace));
         this.registerServiceImplementation(IComponentLibraryCatalogService.class, new ComponentLibraryCatalogService());
@@ -86,6 +91,7 @@ public ConfigurationModuleProvider() {
         this.registerServiceImplementation(IServiceReferenceAlarmRuleConfig.class, new ServiceReferenceAlarmRuleConfig(serviceErrorRateThreshold, serviceAverageResponseTimeThreshold));
         this.registerServiceImplementation(IInstanceReferenceAlarmRuleConfig.class, new InstanceReferenceAlarmRuleConfig(instanceErrorRateThreshold, instanceAverageResponseTimeThreshold));
         this.registerServiceImplementation(IApplicationReferenceAlarmRuleConfig.class, new ApplicationReferenceAlarmRuleConfig(applicationErrorRateThreshold, applicationAverageResponseTimeThreshold));
+        this.registerServiceImplementation(IResponseTimeDistributionConfigService.class, new ResponseTimeDistributionConfigService(responseTimeDistributionDuration, responseTimeDistributionMaxDurations));
     }
 
     @Override public void start() {
diff --git a/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/service/ResponseTimeDistributionConfigService.java b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/service/ResponseTimeDistributionConfigService.java
new file mode 100644
index 000000000..4201abd14
--- /dev/null
+++ b/apm-collector/apm-collector-configuration/collector-configuration-provider/src/main/java/org/apache/skywalking/apm/collector/configuration/service/ResponseTimeDistributionConfigService.java
@@ -0,0 +1,41 @@
+/*
+ * 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.apm.collector.configuration.service;
+
+/**
+ * @author peng-yongsheng
+ */
+public class ResponseTimeDistributionConfigService implements IResponseTimeDistributionConfigService {
+
+    private final int responseTimeStep;
+    private final int responseTimeMaxStep;
+
+    public ResponseTimeDistributionConfigService(int responseTimeStep, int responseTimeMaxStep) {
+        this.responseTimeStep = responseTimeStep;
+        this.responseTimeMaxStep = responseTimeMaxStep + 1;
+    }
+
+    @Override public int getResponseTimeStep() {
+        return responseTimeStep;
+    }
+
+    @Override public int getResponseTimeMaxStep() {
+        return responseTimeMaxStep;
+    }
+}
diff --git a/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/StorageModule.java b/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/StorageModule.java
index 2197a81ae..e9e3b3889 100644
--- a/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/StorageModule.java
+++ b/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/StorageModule.java
@@ -123,6 +123,7 @@
 import org.apache.skywalking.apm.collector.storage.dao.ui.IInstanceUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IMemoryMetricUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.INetworkAddressUIDAO;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentDurationUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IServiceAlarmUIDAO;
@@ -260,6 +261,7 @@ private void addUiDAO(List<Class> classes) {
         classes.add(IMemoryMetricUIDAO.class);
 
         classes.add(IGlobalTraceUIDAO.class);
+        classes.add(IResponseTimeDistributionUIDAO.class);
         classes.add(IInstanceMetricUIDAO.class);
         classes.add(IApplicationComponentUIDAO.class);
         classes.add(IApplicationMappingUIDAO.class);
diff --git a/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/dao/ui/IResponseTimeDistributionUIDAO.java b/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/dao/ui/IResponseTimeDistributionUIDAO.java
new file mode 100644
index 000000000..a326a6eb2
--- /dev/null
+++ b/apm-collector/apm-collector-storage/collector-storage-define/src/main/java/org/apache/skywalking/apm/collector/storage/dao/ui/IResponseTimeDistributionUIDAO.java
@@ -0,0 +1,106 @@
+/*
+ * 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.apm.collector.storage.dao.ui;
+
+import java.util.List;
+import org.apache.skywalking.apm.collector.storage.base.dao.DAO;
+import org.apache.skywalking.apm.collector.storage.ui.common.Step;
+
+/**
+ * Interface to be implemented for execute database query operation
+ * from {@link org.apache.skywalking.apm.collector.storage.table.global.ResponseTimeDistributionTable#TABLE}.
+ *
+ * @author peng-yongsheng
+ * @see org.apache.skywalking.apm.collector.storage.table.global.ResponseTimeDistributionTable
+ * @see org.apache.skywalking.apm.collector.storage.StorageModule
+ */
+public interface IResponseTimeDistributionUIDAO extends DAO {
+
+    /**
+     * <p>SQL as: select CALLS, ERROR_CALLS, SUCCESS_CALLS from RESPONSE_TIME_DISTRIBUTION
+     * where ID in (${responseTimeSteps}),
+     *
+     * <p>Rule of ID generation is "${responseTimeStep.getDurationPoint}_${responseTimeStep.getStep}",
+     *
+     * <p>Every element in return list must match ResponseTimeStep list, which also means that,
+     * the two list must be in same size, and yAxis match.
+     *
+     * <p>If some element of the return list can't be found, the implementor must set 0 as
+     * default value.
+     *
+     * <p>Use {@link org.apache.skywalking.apm.collector.storage.utils.TimePyramidTableNameBuilder#build(Step, String)}
+     * to generate table name which mixed with step name.
+     *
+     * @param step the step which represent time formats
+     * @param responseTimeSteps the time points in the time span
+     * @see org.apache.skywalking.apm.collector.storage.ui.common.Step
+     */
+    void loadMetrics(Step step, List<ResponseTimeStep> responseTimeSteps);
+
+    class ResponseTimeStep {
+        private final int step;
+        private final long durationPoint;
+        private final int yAxis;
+        private long calls;
+        private long errorCalls;
+        private long successCalls;
+
+        public ResponseTimeStep(long durationPoint, int yAxis, int step) {
+            this.step = step;
+            this.durationPoint = durationPoint;
+            this.yAxis = yAxis;
+        }
+
+        public int getStep() {
+            return step;
+        }
+
+        public long getDurationPoint() {
+            return durationPoint;
+        }
+
+        public int getyAxis() {
+            return yAxis;
+        }
+
+        public long getCalls() {
+            return calls;
+        }
+
+        public void setCalls(long calls) {
+            this.calls = calls;
+        }
+
+        public long getErrorCalls() {
+            return errorCalls;
+        }
+
+        public void setErrorCalls(long errorCalls) {
+            this.errorCalls = errorCalls;
+        }
+
+        public long getSuccessCalls() {
+            return successCalls;
+        }
+
+        public void setSuccessCalls(long successCalls) {
+            this.successCalls = successCalls;
+        }
+    }
+}
diff --git a/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/StorageModuleEsProvider.java b/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/StorageModuleEsProvider.java
index 86cd66f26..1298475a0 100644
--- a/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/StorageModuleEsProvider.java
+++ b/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/StorageModuleEsProvider.java
@@ -137,6 +137,7 @@
 import org.apache.skywalking.apm.collector.storage.dao.ui.IInstanceUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IMemoryMetricUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.INetworkAddressUIDAO;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentDurationUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IServiceAlarmUIDAO;
@@ -246,6 +247,7 @@
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.InstanceMetricEsUIDAO;
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.MemoryMetricEsUIDAO;
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.NetworkAddressEsUIDAO;
+import org.apache.skywalking.apm.collector.storage.es.dao.ui.ResponseTimeDistributionEsUIDAO;
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.SegmentDurationEsUIDAO;
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.SegmentEsUIDAO;
 import org.apache.skywalking.apm.collector.storage.es.dao.ui.ServiceAlarmEsUIDAO;
@@ -450,6 +452,7 @@ private void registerUiDAO() throws ServiceNotProvidedException {
         this.registerServiceImplementation(IServiceAlarmUIDAO.class, new ServiceAlarmEsUIDAO(elasticSearchClient));
 
         this.registerServiceImplementation(IApplicationAlarmListUIDAO.class, new ApplicationAlarmListEsUIDAO(elasticSearchClient));
+        this.registerServiceImplementation(IResponseTimeDistributionUIDAO.class, new ResponseTimeDistributionEsUIDAO(elasticSearchClient));
     }
 
     private void registerAlarmDAO() throws ServiceNotProvidedException {
diff --git a/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/dao/ui/ResponseTimeDistributionEsUIDAO.java b/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/dao/ui/ResponseTimeDistributionEsUIDAO.java
new file mode 100644
index 000000000..37776335a
--- /dev/null
+++ b/apm-collector/apm-collector-storage/collector-storage-es-provider/src/main/java/org/apache/skywalking/apm/collector/storage/es/dao/ui/ResponseTimeDistributionEsUIDAO.java
@@ -0,0 +1,72 @@
+/*
+ * 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.apm.collector.storage.es.dao.ui;
+
+import java.util.List;
+import org.apache.skywalking.apm.collector.client.elasticsearch.ElasticSearchClient;
+import org.apache.skywalking.apm.collector.core.util.Const;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
+import org.apache.skywalking.apm.collector.storage.es.base.dao.EsDAO;
+import org.apache.skywalking.apm.collector.storage.table.global.ResponseTimeDistributionTable;
+import org.apache.skywalking.apm.collector.storage.ui.common.Step;
+import org.apache.skywalking.apm.collector.storage.utils.TimePyramidTableNameBuilder;
+import org.elasticsearch.action.get.MultiGetItemResponse;
+import org.elasticsearch.action.get.MultiGetRequestBuilder;
+import org.elasticsearch.action.get.MultiGetResponse;
+
+/**
+ * @author peng-yongsheng
+ */
+public class ResponseTimeDistributionEsUIDAO extends EsDAO implements IResponseTimeDistributionUIDAO {
+
+    public ResponseTimeDistributionEsUIDAO(ElasticSearchClient client) {
+        super(client);
+    }
+
+    @Override
+    public void loadMetrics(Step step, List<ResponseTimeStep> responseTimeSteps) {
+        String tableName = TimePyramidTableNameBuilder.build(step, ResponseTimeDistributionTable.TABLE);
+
+        MultiGetRequestBuilder prepareMultiGet = getClient().prepareMultiGet(responseTimeSteps, new ElasticSearchClient.MultiGetRowHandler<ResponseTimeStep>() {
+            @Override
+            public void accept(ResponseTimeStep responseTimeStep) {
+                String id = String.valueOf(responseTimeStep.getDurationPoint()) + Const.ID_SPLIT + String.valueOf(responseTimeStep.getStep());
+                this.add(tableName, ResponseTimeDistributionTable.TABLE_TYPE, id);
+            }
+        });
+
+        MultiGetResponse multiGetResponse = prepareMultiGet.get();
+        for (int i = 0; i < multiGetResponse.getResponses().length; i++) {
+            MultiGetItemResponse response = multiGetResponse.getResponses()[i];
+            if (response.getResponse().isExists()) {
+                long calls = ((Number)response.getResponse().getSource().get(ResponseTimeDistributionTable.CALLS.getName())).longValue();
+                long errorCalls = ((Number)response.getResponse().getSource().get(ResponseTimeDistributionTable.ERROR_CALLS.getName())).longValue();
+                long successCalls = ((Number)response.getResponse().getSource().get(ResponseTimeDistributionTable.SUCCESS_CALLS.getName())).longValue();
+
+                responseTimeSteps.get(i).setCalls(calls);
+                responseTimeSteps.get(i).setErrorCalls(errorCalls);
+                responseTimeSteps.get(i).setSuccessCalls(successCalls);
+            } else {
+                responseTimeSteps.get(i).setCalls(0);
+                responseTimeSteps.get(i).setErrorCalls(0);
+                responseTimeSteps.get(i).setSuccessCalls(0);
+            }
+        }
+    }
+}
diff --git a/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/StorageModuleH2Provider.java b/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/StorageModuleH2Provider.java
index d8c134e3f..bb1f6fa3c 100644
--- a/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/StorageModuleH2Provider.java
+++ b/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/StorageModuleH2Provider.java
@@ -20,6 +20,8 @@
 
 import org.apache.skywalking.apm.collector.client.h2.H2Client;
 import org.apache.skywalking.apm.collector.client.h2.H2ClientException;
+import org.apache.skywalking.apm.collector.cluster.ClusterModule;
+import org.apache.skywalking.apm.collector.configuration.ConfigurationModule;
 import org.apache.skywalking.apm.collector.core.module.Module;
 import org.apache.skywalking.apm.collector.core.module.ModuleConfig;
 import org.apache.skywalking.apm.collector.core.module.ModuleProvider;
@@ -129,6 +131,7 @@
 import org.apache.skywalking.apm.collector.storage.dao.ui.IInstanceUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IMemoryMetricUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.INetworkAddressUIDAO;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentDurationUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.ISegmentUIDAO;
 import org.apache.skywalking.apm.collector.storage.dao.ui.IServiceAlarmUIDAO;
@@ -238,6 +241,7 @@
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.InstanceMetricH2UIDAO;
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.MemoryMetricH2UIDAO;
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.NetworkAddressH2UIDAO;
+import org.apache.skywalking.apm.collector.storage.h2.dao.ui.ResponseTimeDistributionH2UIDAO;
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.SegmentDurationH2UIDAO;
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.SegmentH2UIDAO;
 import org.apache.skywalking.apm.collector.storage.h2.dao.ui.ServiceAlarmH2UIDAO;
@@ -299,7 +303,7 @@ public StorageModuleH2Provider() {
     }
 
     @Override public String[] requiredModules() {
-        return new String[] {RemoteModule.NAME};
+        return new String[] {ClusterModule.NAME, ConfigurationModule.NAME, RemoteModule.NAME};
     }
 
     private void registerCacheDAO() throws ServiceNotProvidedException {
@@ -423,6 +427,7 @@ private void registerUiDAO() throws ServiceNotProvidedException {
         this.registerServiceImplementation(IServiceAlarmUIDAO.class, new ServiceAlarmH2UIDAO(h2Client));
 
         this.registerServiceImplementation(IApplicationAlarmListUIDAO.class, new ApplicationAlarmListH2UIDAO(h2Client));
+        this.registerServiceImplementation(IResponseTimeDistributionUIDAO.class, new ResponseTimeDistributionH2UIDAO(h2Client));
     }
 
     private void registerAlarmDAO() throws ServiceNotProvidedException {
diff --git a/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/dao/ui/ResponseTimeDistributionH2UIDAO.java b/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/dao/ui/ResponseTimeDistributionH2UIDAO.java
new file mode 100644
index 000000000..84ad9c653
--- /dev/null
+++ b/apm-collector/apm-collector-storage/collector-storage-h2-provider/src/main/java/org/apache/skywalking/apm/collector/storage/h2/dao/ui/ResponseTimeDistributionH2UIDAO.java
@@ -0,0 +1,39 @@
+/*
+ * 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.apm.collector.storage.h2.dao.ui;
+
+import java.util.List;
+import org.apache.skywalking.apm.collector.client.h2.H2Client;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
+import org.apache.skywalking.apm.collector.storage.h2.base.dao.H2DAO;
+import org.apache.skywalking.apm.collector.storage.ui.common.Step;
+
+/**
+ * @author peng-yongsheng
+ */
+public class ResponseTimeDistributionH2UIDAO extends H2DAO implements IResponseTimeDistributionUIDAO {
+
+    public ResponseTimeDistributionH2UIDAO(H2Client client) {
+        super(client);
+    }
+
+    @Override public void loadMetrics(Step step, List<ResponseTimeStep> responseTimeSteps) {
+
+    }
+}
diff --git a/apm-collector/apm-collector-storage/pom.xml b/apm-collector/apm-collector-storage/pom.xml
index 4ad541281..e5f314ff3 100644
--- a/apm-collector/apm-collector-storage/pom.xml
+++ b/apm-collector/apm-collector-storage/pom.xml
@@ -45,6 +45,11 @@
             <artifactId>client-component</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>collector-configuration-define</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.skywalking</groupId>
             <artifactId>collector-remote-define</artifactId>
diff --git a/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/query/OverViewLayerQuery.java b/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/query/OverViewLayerQuery.java
index 72e09a1c7..0e1bd9d03 100644
--- a/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/query/OverViewLayerQuery.java
+++ b/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/query/OverViewLayerQuery.java
@@ -35,6 +35,7 @@
 import org.apache.skywalking.apm.collector.ui.service.ApplicationService;
 import org.apache.skywalking.apm.collector.ui.service.ClusterTopologyService;
 import org.apache.skywalking.apm.collector.ui.service.NetworkAddressService;
+import org.apache.skywalking.apm.collector.ui.service.ResponseTimeDistributionService;
 import org.apache.skywalking.apm.collector.ui.service.ServiceNameService;
 import org.apache.skywalking.apm.collector.ui.utils.DurationUtils;
 
@@ -51,6 +52,7 @@
     private NetworkAddressService networkAddressService;
     private ServiceNameService serviceNameService;
     private AlarmService alarmService;
+    private ResponseTimeDistributionService timeDistributionService;
 
     public OverViewLayerQuery(ModuleManager moduleManager) {
         this.moduleManager = moduleManager;
@@ -91,6 +93,13 @@ private AlarmService getAlarmService() {
         return alarmService;
     }
 
+    private ResponseTimeDistributionService getTimeDistributionService() {
+        if (isNull(timeDistributionService)) {
+            this.timeDistributionService = new ResponseTimeDistributionService(moduleManager);
+        }
+        return timeDistributionService;
+    }
+
     public Topology getClusterTopology(Duration duration) throws ParseException {
         long startTimeBucket = DurationUtils.INSTANCE.exchangeToTimeBucket(duration.getStart());
         long endTimeBucket = DurationUtils.INSTANCE.exchangeToTimeBucket(duration.getEnd());
@@ -150,6 +159,9 @@ public ConjecturalAppBrief getConjecturalApps(Duration duration) throws ParseExc
     }
 
     public Thermodynamic getThermodynamic(Duration duration, ValueType type) throws ParseException {
-        return new Thermodynamic();
+        long startTimeBucket = DurationUtils.INSTANCE.exchangeToTimeBucket(duration.getStart());
+        long endTimeBucket = DurationUtils.INSTANCE.exchangeToTimeBucket(duration.getEnd());
+
+        return getTimeDistributionService().getThermodynamic(duration.getStep(), startTimeBucket, endTimeBucket, type);
     }
 }
diff --git a/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/service/ResponseTimeDistributionService.java b/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/service/ResponseTimeDistributionService.java
new file mode 100644
index 000000000..86d6c197b
--- /dev/null
+++ b/apm-collector/apm-collector-ui/collector-ui-jetty-provider/src/main/java/org/apache/skywalking/apm/collector/ui/service/ResponseTimeDistributionService.java
@@ -0,0 +1,91 @@
+/*
+ * 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.apm.collector.ui.service;
+
+import java.text.ParseException;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.skywalking.apm.collector.configuration.ConfigurationModule;
+import org.apache.skywalking.apm.collector.configuration.service.IResponseTimeDistributionConfigService;
+import org.apache.skywalking.apm.collector.core.module.ModuleManager;
+import org.apache.skywalking.apm.collector.storage.StorageModule;
+import org.apache.skywalking.apm.collector.storage.dao.ui.IResponseTimeDistributionUIDAO;
+import org.apache.skywalking.apm.collector.storage.ui.common.Step;
+import org.apache.skywalking.apm.collector.storage.ui.common.ValueType;
+import org.apache.skywalking.apm.collector.storage.ui.overview.Thermodynamic;
+import org.apache.skywalking.apm.collector.storage.utils.DurationPoint;
+import org.apache.skywalking.apm.collector.ui.utils.DurationUtils;
+
+/**
+ * @author peng-yongsheng
+ */
+public class ResponseTimeDistributionService {
+
+    private final IResponseTimeDistributionConfigService responseTimeDistributionConfigService;
+    private final IResponseTimeDistributionUIDAO responseTimeDistributionUIDAO;
+
+    public ResponseTimeDistributionService(ModuleManager moduleManager) {
+        this.responseTimeDistributionConfigService = moduleManager.find(ConfigurationModule.NAME).getService(IResponseTimeDistributionConfigService.class);
+        this.responseTimeDistributionUIDAO = moduleManager.find(StorageModule.NAME).getService(IResponseTimeDistributionUIDAO.class);
+    }
+
+    public Thermodynamic getThermodynamic(Step step, long startTimeBucket, long endTimeBucket,
+        ValueType type) throws ParseException {
+        List<DurationPoint> durationPoints = DurationUtils.INSTANCE.getDurationPoints(step, startTimeBucket, endTimeBucket);
+
+        List<IResponseTimeDistributionUIDAO.ResponseTimeStep> responseTimeSteps = new LinkedList<>();
+        for (int i = 0; i < durationPoints.size(); i++) {
+            for (int j = 0; j < responseTimeDistributionConfigService.getResponseTimeMaxStep(); j++) {
+                responseTimeSteps.add(new IResponseTimeDistributionUIDAO.ResponseTimeStep(durationPoints.get(i).getPoint(), i, j));
+            }
+        }
+
+        responseTimeDistributionUIDAO.loadMetrics(step, responseTimeSteps);
+
+        Thermodynamic thermodynamic = new Thermodynamic();
+        thermodynamic.setResponseTimeStep(responseTimeDistributionConfigService.getResponseTimeStep());
+        responseTimeSteps.forEach(responseTimeStep -> {
+            long calls = 0;
+            switch (type) {
+                case ALL:
+                    calls = responseTimeStep.getCalls();
+                    break;
+                case RIGHT:
+                    calls = responseTimeStep.getSuccessCalls();
+                    break;
+                case WRONG:
+                    calls = responseTimeStep.getErrorCalls();
+                    break;
+            }
+
+            List<Long> metric = trans(responseTimeStep.getyAxis(), responseTimeStep.getStep(), calls);
+            thermodynamic.getNodes().add(metric);
+        });
+
+        return thermodynamic;
+    }
+
+    private List<Long> trans(long xAxis, long yAxis, long calls) {
+        List<Long> metric = new LinkedList<>();
+        metric.add(xAxis);
+        metric.add(yAxis);
+        metric.add(calls);
+        return metric;
+    }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


With regards,
Apache Git Services