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 2020/01/15 16:20:37 UTC

[GitHub] [skywalking] dmsolr opened a new pull request #4239: [WIP] Provide influxdb as a new storage plugin

dmsolr opened a new pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239
 
 
   Please answer these questions before submitting pull request
   
   - Why submit this pull request?
   - [ ] Bug fix
   - [X] New feature provided
   - [ ] Improve performance
   ___
   ### New feature or improvement
   - Describe the details and related test reports.
   
   This PR provides a new storage plugin for OAP which uses two-level storage. 
   
   The first level is a relationship database stored something likes `inventory`, which has transactions and updates. Stored in MySQL.(Now, I make H2 as relationship database. Because it is easy for test.)
   
   The second level is a time-series database, stored `metrics`, `logs`, and `traces`. Those data have strong time-lined and no-transaction, no-updated, no-deleted. Stored in InfluxDB.
   
   InfluxDB is a very powerful time-series database. It is good at manipulating time-series data. So, it is suitable for our trace records that have strong time-lined, no-updated, and no-deleted.
   
   Now, it is an experimental feature. We need to continuously improve it with you. And it is a meaningful experience for others NoSQL/TSDB likes Apache Druid, OpenTSDB.
   
   ------------------------------------
   ## TODO list
   - [x] feature
   - [ ] e2e
   - [ ] document 
   - [ ] change MySQL as the level storage

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379220907
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+                                         new OkHttpClient.Builder().readTimeout(3, TimeUnit.MINUTES)
+                                                                   .writeTimeout(3, TimeUnit.MINUTES),
+                                         InfluxDB.ResponseFormat.MSGPACK
+        );
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    private InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Execute a query against InfluxDB and return a set of {@link QueryResult.Result}s. Normally, InfluxDB supports
+     * combining multiple statements into one query, so that we do get multi-results.
+     *
+     * @param query Query
+     * @return a set of {@link QueryResult.Result}s.
+     * @throws IOException if there is an error on the InfluxDB server or communication error.
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement.
+     *
+     * @param query Query
+     * @return a set of {@link QueryResult.Series}s
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        List<QueryResult.Result> results = query(query);
+
+        if (CollectionUtils.isEmpty(results)) {
+            return null;
+        }
+        return results.get(0).getSeries();
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement but return a single {@link QueryResult.Series}.
+     *
+     * @param query Query
+     * @return {@link QueryResult.Series}
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public QueryResult.Series queryForSingleSeries(Query query) throws IOException {
+        List<QueryResult.Series> series = queryForSeries(query);
+        if (CollectionUtils.isEmpty(series)) {
+            return null;
+        }
+        return series.get(0);
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success. Notice, drop series don't support to drop series by range
+     *
+     * @param measurement String
+     * @param timeBucket  long
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
+        QueryResult result = getInflux().query(query);
+
+        if (result.hasError()) {
+            throw new IOException("Statement: " + query.getCommand() + ", ErrorMsg: " + result.getError());
+        }
+    }
+
+    public void deleteByQuery(String measurement, long timestamp) throws IOException {
+        this.query(new Query("delete from " + measurement + " where time < " + timestamp + "ms"));
+    }
+
+    /**
+     * Write a {@link Point} into InfluxDB. Note that, the {@link Point} is written into buffer of InfluxDB Client and
+     * wait for buffer flushing.
+     *
+     * @param point Point
 
 Review comment:
   @dmsolr you should put the **description** after the parameter name and return name, **NOT** the type of the parameter and return object, they're meaningless, and will cheat the JavaDoc compiler, if they are already self-documented, simply remove the JavaDoc.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376276636
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+
+@Slf4j
+public class BatchDAO implements IBatchDAO {
+    private final DataCarrier<PrepareRequest> dataCarrier;
+    private final InfluxClient client;
+
+    public BatchDAO(InfluxClient client) {
+        this.client = client;
+
+        String name = "INFLUX_ASYNC_BATCH_PERSISTENT";
+        BulkConsumePool.Creator creator = new BulkConsumePool.Creator(name, 1, 20L);
+
+        try {
+            ConsumerPoolFactory.INSTANCE.createIfAbsent(name, creator);
+        } catch (Exception e) {
+            throw new UnexpectedException(e.getMessage(), e);
+        }
+
+        this.dataCarrier = new DataCarrier(1, 10000);
+        this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(name), new InfluxBatchConsumer(this));
+    }
+
+    @Override
+    public void asynchronous(InsertRequest insertRequest) {
+        dataCarrier.produce(insertRequest);
 
 Review comment:
   Then you should not require datacarrier? Right?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@fd5f94c`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `8.55%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff            @@
   ##             master   #4239   +/-   ##
   ========================================
     Coverage          ?   26.1%           
   ========================================
     Files             ?    1187           
     Lines             ?   26275           
     Branches          ?    3749           
   ========================================
     Hits              ?    6858           
     Misses            ?   18807           
     Partials          ?     610
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <ø> (ø)` | |
   | [...ng/apm/plugin/grpc/v1/OperationNameFormatUtil.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9PcGVyYXRpb25OYW1lRm9ybWF0VXRpbC5qYXZh) | `0% <ø> (ø)` | |
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <ø> (ø)` | |
   | [...ywalking/apm/plugin/avro/ResponderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1Jlc3BvbmRlckludGVyY2VwdG9yLmphdmE=) | `0% <0%> (ø)` | |
   | [...storage/plugin/influxdb/installer/H2Installer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvSDJJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...plugin/grpc/v1/client/BlockingCallInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQmxvY2tpbmdDYWxsSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/ClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvQ2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [.../skywalking/apm/plugin/avro/SWServerRPCPlugin.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXZyby1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9hdnJvL1NXU2VydmVyUlBDUGx1Z2luLmphdmE=) | `0% <0%> (ø)` | |
   | [...g/apm/plugin/grpc/v1/client/TracingClientCall.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9jbGllbnQvVHJhY2luZ0NsaWVudENhbGwuamF2YQ==) | `0% <0%> (ø)` | |
   | [...1/server/AbstractServerImplBuilderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZ3JwYy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZ3JwYy92MS9zZXJ2ZXIvQWJzdHJhY3RTZXJ2ZXJJbXBsQnVpbGRlckludGVyY2VwdG9yLmphdmE=) | `0% <0%> (ø)` | |
   | ... and [18 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [fd5f94c...63aafad](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376264109
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+
+@Slf4j
+public class BatchDAO implements IBatchDAO {
+    private final DataCarrier<PrepareRequest> dataCarrier;
+    private final InfluxClient client;
+
+    public BatchDAO(InfluxClient client) {
+        this.client = client;
+
+        String name = "INFLUX_ASYNC_BATCH_PERSISTENT";
+        BulkConsumePool.Creator creator = new BulkConsumePool.Creator(name, 1, 20L);
+
+        try {
+            ConsumerPoolFactory.INSTANCE.createIfAbsent(name, creator);
+        } catch (Exception e) {
+            throw new UnexpectedException(e.getMessage(), e);
+        }
+
+        this.dataCarrier = new DataCarrier(1, 10000);
+        this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(name), new InfluxBatchConsumer(this));
+    }
+
+    @Override
+    public void asynchronous(InsertRequest insertRequest) {
+        dataCarrier.produce(insertRequest);
+    }
+
+    @Override
+    public void synchronous(List<PrepareRequest> prepareRequests) {
+        if (CollectionUtils.isEmpty(prepareRequests)) {
+            return;
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("batch sql statements execute, data size: {}", prepareRequests.size());
+        }
+
+        final BatchPoints.Builder builder = BatchPoints.builder();
+        prepareRequests.forEach(e -> {
+            builder.point(((InfluxInsertRequest)e).getPoint());
+        });
+
+        client.write(builder.build());
 
 Review comment:
   The data is non-blocking write. They are queryable after the method executed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380443588
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/installer/MetaTableDefine.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.storage.plugin.influxdb.installer;
+
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.jdbc.TableMetaInfo;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENDPOINT_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.NETWORK_ADDRESS;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INSTANCE_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INVENTORY;
+
+/**
+ * Here defines which table is stored in metadata database(H2/MySQL).
+ */
+public class MetaTableDefine {
+
+    /**
+     * Test a {@link Model} is stored in H2/MySQL or not.
+     *
+     * @param model Model
+     * @return true if the {@link Model} is stored in H2/MySQL
+     */
+    public static boolean contains(Model model) {
+        switch (model.getScopeId()) {
+            case SERVICE_INVENTORY:
+            case SERVICE_INSTANCE_INVENTORY:
+            case NETWORK_ADDRESS:
+            case ENDPOINT_INVENTORY:
+            case PROFILE_TASK:
 
 Review comment:
   Use `model#capableOfTimeSeries` == true check. If YES, then it is a metadata table.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374612571
 
 

 ##########
 File path: oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/model/Model.java
 ##########
 @@ -44,5 +48,11 @@ public Model(String name, List<ModelColumn> columns, boolean capableOfTimeSeries
         this.scopeId = scopeId;
         this.name = ModelName.build(downsampling, name);
         this.record = record;
+        this.storageColumns = Maps.newTreeMap();
+        columns.forEach(column -> { storageColumns.put(column.getColumnName().getStorageName(), column); });
+    }
+
+    public ModelColumn getColumnByStorageCName(String storageCName) {
 
 Review comment:
   why add unused method 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379832246
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/ProfileThreadSnapshotQuery.java
 ##########
 @@ -0,0 +1,165 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.storage.profile.IProfileThreadSnapshotQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class ProfileThreadSnapshotQuery implements IProfileThreadSnapshotQueryDAO {
+    private final InfluxClient client;
+
+    public ProfileThreadSnapshotQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public List<BasicTrace> queryProfiledSegments(String taskId) throws IOException {
+        WhereQueryImpl query = select(ProfileThreadSnapshotRecord.SEGMENT_ID)
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where()
+            .and(eq(ProfileThreadSnapshotRecord.TASK_ID, taskId))
+            .and(eq(ProfileThreadSnapshotRecord.SEQUENCE, 0));
+
+        final LinkedList<String> segments = new LinkedList<>();
+        QueryResult.Series series = client.queryForSingleSeries(query);
+        if (series == null) {
+            return Collections.emptyList();
+        }
+        series.getValues().forEach(values -> {
+            segments.add((String) values.get(1));
+        });
+
+        if (segments.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        query = select()
+            .function("bottom", SegmentRecord.START_TIME, segments.size())
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where()
+            .and(contains(SegmentRecord.SEGMENT_ID, Joiner.on("|").join(segments)));
+
+        ArrayList<BasicTrace> result = Lists.newArrayListWithCapacity(segments.size());
+        client.queryForSingleSeries(query)
+              .getValues()
+              .stream()
+              .sorted((a, b) -> Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue()))
+              .forEach(values -> {
+                  BasicTrace basicTrace = new BasicTrace();
+
+                  basicTrace.setSegmentId((String) values.get(2));
+                  basicTrace.setStart(String.valueOf(values.get(3)));
+                  basicTrace.getEndpointNames().add((String) values.get(4));
+                  basicTrace.setDuration((int) values.get(5));
+                  basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+                  String traceIds = (String) values.get(7);
+                  basicTrace.getTraceIds().add(traceIds);
+
+                  result.add(basicTrace);
+              });
+
+        return result;
+    }
+
+    @Override
+    public int queryMinSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("min", segmentId, start, end);
+    }
+
+    @Override
+    public int queryMaxSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("max", segmentId, start, end);
+    }
+
+    @Override
+    public List<ProfileThreadSnapshotRecord> queryRecords(String segmentId, int minSequence,
+                                                          int maxSequence) throws IOException {
+        WhereQueryImpl query = select(
+            ProfileThreadSnapshotRecord.TASK_ID,
+            ProfileThreadSnapshotRecord.SEGMENT_ID,
+            ProfileThreadSnapshotRecord.DUMP_TIME,
+            ProfileThreadSnapshotRecord.SEQUENCE,
+            ProfileThreadSnapshotRecord.STACK_BINARY
+        )
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where(eq(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId))
+            .and(gte(ProfileThreadSnapshotRecord.SEQUENCE, minSequence))
+            .and(lte(ProfileThreadSnapshotRecord.SEQUENCE, maxSequence));
+
+        ArrayList<ProfileThreadSnapshotRecord> result = new ArrayList<>(maxSequence - minSequence);
+        client.queryForSingleSeries(query).getValues().forEach(values -> {
+            ProfileThreadSnapshotRecord record = new ProfileThreadSnapshotRecord();
+
+            record.setTaskId((String) values.get(1));
+            record.setSegmentId((String) values.get(2));
+            record.setDumpTime(((Number) values.get(3)).longValue());
+            record.setSequence((int) values.get(4));
+            String dataBinaryBase64 = String.valueOf(values.get(5));
+            if (StringUtil.isNotEmpty(dataBinaryBase64)) {
+                record.setStackBinary(Base64.getDecoder().decode(dataBinaryBase64));
 
 Review comment:
   better to use `org.apache.skywalking.apm.agent.core.base64.Base64` for consistence, and the default charset here is different from other places

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4b02679`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   25.84%           
   =========================================
     Files             ?     1189           
     Lines             ?    26218           
     Branches          ?     3728           
   =========================================
     Hits              ?     6775           
     Misses            ?    18840           
     Partials          ?      603
   ```
   
   
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4b02679...9128aec](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379863652
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/MetricsDAO.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.oap.server.storage.plugin.influxdb.base;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.model.ModelColumn;
+import org.apache.skywalking.oap.server.core.storage.type.StorageDataType;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class MetricsDAO implements IMetricsDAO {
+    public static final String TAG_ENTITY_ID = "_entity_id";
+
+    private final StorageBuilder<Metrics> storageBuilder;
+    private final InfluxClient client;
+
+    public MetricsDAO(InfluxClient client, StorageBuilder<Metrics> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public List<Metrics> multiGet(Model model, List<String> ids) throws IOException {
+        WhereQueryImpl<SelectQueryImpl> query = select()
+            .regex("*::field")
 
 Review comment:
   They are different in name only. 
   If you think `raw` is more appropiate, I will fix them.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380077043
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   I think we should do that. Creating an issue to track this?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581094650
 
 
   Please be noticed
   
   > This branch is out-of-date with the base branch
   
   Do you merge the master branch with this branch correctly? Please fix them before we can do the review.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380044341
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/AggregationQuery.java
 ##########
 @@ -0,0 +1,152 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.query.entity.Order;
+import org.apache.skywalking.oap.server.core.query.entity.TopNEntity;
+import org.apache.skywalking.oap.server.core.register.EndpointInventory;
+import org.apache.skywalking.oap.server.core.register.ServiceInstanceInventory;
+import org.apache.skywalking.oap.server.core.storage.model.ModelName;
+import org.apache.skywalking.oap.server.core.storage.query.IAggregationQueryDAO;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.MetricsDAO;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.SelectSubQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class AggregationQuery implements IAggregationQueryDAO {
+    private InfluxClient client;
+
+    public AggregationQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public List<TopNEntity> getServiceTopN(String indName, String valueCName, int topN, Downsampling downsampling,
+                                           long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getAllServiceInstanceTopN(String indName, String valueCName, int topN,
+                                                      Downsampling downsampling,
+                                                      long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getServiceInstanceTopN(int serviceId, String indName, String valueCName, int topN,
+                                                   Downsampling downsampling,
+                                                   long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(
+            downsampling, indName,
+            subQuery(ServiceInstanceInventory.SERVICE_ID, serviceId, indName, valueCName, startTB, endTB), order, topN
+        );
+    }
+
+    @Override
+    public List<TopNEntity> getAllEndpointTopN(String indName, String valueCName, int topN, Downsampling downsampling,
+                                               long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getEndpointTopN(int serviceId, String indName, String valueCName, int topN,
+                                            Downsampling downsampling,
+                                            long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(
+            downsampling, indName,
+            subQuery(EndpointInventory.SERVICE_ID, serviceId, indName, valueCName, startTB, endTB), order, topN
+        );
+    }
+
+    private List<TopNEntity> getTopNEntity(Downsampling downsampling,
+                                           String name,
+                                           SelectSubQueryImpl<SelectQueryImpl> subQuery,
+                                           Order order,
+                                           int topN) throws IOException {
+        String measurement = ModelName.build(downsampling, name);
+        Comparator<TopNEntity> comparator = DESCENDING;
+        String functionName = "top";
+        if (order == Order.ASC) {
+            functionName = "bottom";
+            comparator = ASCENDING;
+        }
+
+        SelectQueryImpl query = select().function(functionName, "mean", topN).as("value")
+                                        .column(MetricsDAO.TAG_ENTITY_ID)
+                                        .from(client.getDatabase(), measurement);
+        query.setSubQuery(subQuery);
+
+        List<QueryResult.Series> series = client.queryForSeries(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), series);
+        }
+        if (series == null || series.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        List<List<Object>> dataset = series.get(0).getValues();
+        List<TopNEntity> entities = Lists.newArrayListWithCapacity(dataset.size());
+        dataset.forEach(values -> {
+            final TopNEntity entity = new TopNEntity();
+            entity.setId((String) values.get(2));
+            entity.setValue(((Double) values.get(1)).longValue());
+            entities.add(entity);
+        });
+        Collections.sort(entities, comparator);
 
 Review comment:
   As you mentioned last time, the result is unordered. Could you add the comments some places to ease the readers?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376711649
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/main/proto/common/CLR.proto
 ##########
 @@ -0,0 +1,46 @@
+/*
+ * 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.
+ *
+ */
+
+syntax = "proto3";
 
 Review comment:
   All these protos should be removed, please follow this, https://github.com/apache/skywalking/pull/4329#issue-372713174

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375117861
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
 
 Review comment:
   Empty comment?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375122463
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success.
+     *
+     * @param measurement
+     * @param timeBucket
+     * @throws IOException
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
 
 Review comment:
   `=`?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-583730593
 
 
   I don't want this big PR blocked too long, it is already a long time, we could create an issue to track your test change proposal.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374654305
 
 

 ##########
 File path: oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/metrics/Metrics.java
 ##########
 @@ -32,8 +35,13 @@
     public static final String TIME_BUCKET = "time_bucket";
     public static final String ENTITY_ID = "entity_id";
 
-    @Getter @Setter @Column(columnName = TIME_BUCKET) private long timeBucket;
-    @Getter @Setter private long survivalTime = 0L;
+    @Getter
+    @Setter
+    @Column(columnName = TIME_BUCKET)
+    private long timeBucket;
+    @Getter
+    @Setter
+    private long survivalTime = 0L;
 
 Review comment:
   Including all following changes of this class.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/918673d7bec7906c4fef7f304cc86eca1534df99?src=pr&el=desc) will **increase** coverage by `0.36%`.
   > The diff coverage is `36.7%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.38%   25.75%   +0.36%     
   ==========================================
     Files        1196     1200       +4     
     Lines       27474    27598     +124     
     Branches     3787     3817      +30     
   ==========================================
   + Hits         6974     7107     +133     
   + Misses      19884    19857      -27     
   - Partials      616      634      +18
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...storage/plugin/influxdb/installer/H2Installer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvSDJJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...er/storage/plugin/influxdb/query/MetricsQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9NZXRyaWNzUXVlcnkuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...storage/plugin/influxdb/base/HistoryDeleteDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL0hpc3RvcnlEZWxldGVEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...rage/plugin/influxdb/installer/MySQLInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvTXlTUUxJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...kywalking/oap/server/core/analysis/TimeBucket.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvVGltZUJ1Y2tldC5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...gin/influxdb/query/ProfileThreadSnapshotQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGhyZWFkU25hcHNob3RRdWVyeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...ver/storage/plugin/influxdb/base/PointBuilder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL1BvaW50QnVpbGRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...lking/apm/agent/core/util/CustomizeExpression.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvdXRpbC9DdXN0b21pemVFeHByZXNzaW9uLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [45 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [918673d...58c5674](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376689951
 
 

 ##########
 File path: docs/en/setup/backend/backend-storage.md
 ##########
 @@ -226,6 +227,27 @@ storage:
 All connection related settings including link url, username and password are in `application.yml`. 
 These settings can refer to the configuration of *MySQL* above.
 
+## InfluxDB
+InfluxDB as storage since SkyWalking 7.0. It depends on `H2/MySQL` storage-plugin to store `metadata` likes `Inventory` and `ProfileTask`. So, when we set `InfluxDB` as storage provider, we need to configure `InfluxDB`'s properties and `H2/MySQL`.
 
 Review comment:
   `likes` -> `like`.
   `we need to configure InfluxDB's properties and H2/MySQL` -> `We need to configure properties of InfluxDB and Metabase.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381174084
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/NoneStreamDAO.java
 ##########
 @@ -0,0 +1,57 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeInteger;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.config.NoneStream;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord;
+import org.apache.skywalking.oap.server.core.storage.INoneStreamDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.Point;
+
+public class NoneStreamDAO implements INoneStreamDAO {
+    public static final String TAG_SERVICE_ID = "_service_id";
+    private static final int PADDING_SIZE = 1_000_000;
+    private static final AtomicRangeInteger SUFFIX = new AtomicRangeInteger(0, PADDING_SIZE);
+
+    private InfluxClient client;
+    private StorageBuilder<NoneStream> storageBuilder;
+
+    public NoneStreamDAO(InfluxClient client, StorageBuilder<NoneStream> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public void insert(final Model model, final NoneStream noneStream) throws IOException {
+        final long timestamp = TimeBucket.getTimestamp(
+            noneStream.getTimeBucket(), model.getDownsampling()) * PADDING_SIZE + SUFFIX.getAndIncrement();
+
+        Point point = new InfluxInsertRequest(model, noneStream, storageBuilder)
+            .time(timestamp, TimeUnit.NANOSECONDS)
 
 Review comment:
   All data in influxdb is nanosecond.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380040883
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/installer/MetaTableDefine.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.storage.plugin.influxdb.installer;
+
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.jdbc.TableMetaInfo;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENDPOINT_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.NETWORK_ADDRESS;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INSTANCE_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INVENTORY;
+
+/**
+ * Here defines which table is stored in metadata database(H2/MySQL).
+ */
+public class MetaTableDefine {
+
+    /**
+     * Test a {@link Model} is stored in H2/MySQL or not.
+     *
+     * @param model Model
+     * @return true if the {@link Model} is stored in H2/MySQL
+     */
+    public static boolean contains(Model model) {
+        switch (model.getScopeId()) {
+            case SERVICE_INVENTORY:
+            case SERVICE_INSTANCE_INVENTORY:
+            case NETWORK_ADDRESS:
+            case ENDPOINT_INVENTORY:
+            case PROFILE_TASK:
 
 Review comment:
   There is another missing here, `PROFILE_TASK_LOG`. Could you check why this is missed but still tests passed? Does InfluxDB have the profile e2e?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/8d36fccc74f4ea46b1e61ef0821da595b289084e?src=pr&el=desc) will **decrease** coverage by `0.1%`.
   > The diff coverage is `63.15%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   25.94%   25.83%   -0.11%     
   ==========================================
     Files        1189     1189              
     Lines       26218    26218              
     Branches     3728     3728              
   ==========================================
   - Hits         6802     6774      -28     
   - Misses      18813    18840      +27     
   - Partials      603      604       +1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...pache/skywalking/apm/agent/core/base64/Base64.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvYmFzZTY0L0Jhc2U2NC5qYXZh) | `100% <100%> (ø)` | :arrow_up: |
   | [...walking/apm/agent/core/context/ContextCarrier.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC9Db250ZXh0Q2Fycmllci5qYXZh) | `70% <57.14%> (ø)` | :arrow_up: |
   | [...walking/apm/agent/core/context/TracingContext.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC9UcmFjaW5nQ29udGV4dC5qYXZh) | `61.43% <61.53%> (ø)` | :arrow_up: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `57.37% <0%> (-19.68%)` | :arrow_down: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `67.85% <0%> (-17.86%)` | :arrow_down: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `28.73% <0%> (-8.05%)` | :arrow_down: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `28.08% <0%> (-3.38%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `80.88% <0%> (-1.48%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [8d36fcc...296bbaf](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378009081
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
+      - name: Single Node Tests(InfluxDB/JDK8)
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-influxdb
       - name: Single Node Tests(MySQL/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-mysql
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-mysql
       - name: Single Node Tests(JDK9)
-        run: export E2E_VERSION=jdk9-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk9-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK11)
-        run: export E2E_VERSION=jdk11-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk11-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK12)
-        run: export E2E_VERSION=jdk12-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk12-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Agent Reboot Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-agent-reboot
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-agent-reboot
 
-  Cluster:
+  Cluster_N_TTL:
 
 Review comment:
   > What do you mean N_TTL?
   
   Possibly `N` == `And`, we don't need to save 2 letters to cause confusion, no?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379833534
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/LogQuery.java
 ##########
 @@ -0,0 +1,154 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.Const;
+import org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord;
+import org.apache.skywalking.oap.server.core.query.entity.ContentType;
+import org.apache.skywalking.oap.server.core.query.entity.Log;
+import org.apache.skywalking.oap.server.core.query.entity.LogState;
+import org.apache.skywalking.oap.server.core.query.entity.Logs;
+import org.apache.skywalking.oap.server.core.query.entity.Pagination;
+import org.apache.skywalking.oap.server.core.storage.query.ILogQueryDAO;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.ConjunctionClause;
+
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.CONTENT;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.CONTENT_TYPE;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.ENDPOINT_ID;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.IS_ERROR;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.SERVICE_ID;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.SERVICE_INSTANCE_ID;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.STATUS_CODE;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.TIMESTAMP;
+import static org.apache.skywalking.oap.server.core.analysis.manual.log.AbstractLogRecord.TRACE_ID;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class LogQuery implements ILogQueryDAO {
+    private final InfluxClient client;
+
+    public LogQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public Logs queryLogs(String metricName, int serviceId, int serviceInstanceId, int endpointId, String traceId,
+                          LogState state, String stateCode, Pagination paging, int from, int limit,
+                          long startTB, long endTB) throws IOException {
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select().regex("*::field")
+            .from(client.getDatabase(), metricName)
+            .where();
+        if (serviceId != Const.NONE) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != Const.NONE) {
+            recallQuery.and(eq(SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != Const.NONE) {
+            recallQuery.and(eq(ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(TRACE_ID, traceId));
+        }
+        switch (state) {
+            case ERROR: {
+                recallQuery.and(eq(IS_ERROR, true));
+                break;
+            }
+            case SUCCESS: {
+                recallQuery.and(eq(IS_ERROR, false));
+                break;
+            }
+        }
+        if (!Strings.isNullOrEmpty(stateCode)) {
+            recallQuery.and(eq(STATUS_CODE, stateCode));
+        }
+        recallQuery.and(gte(AbstractLogRecord.TIME_BUCKET, startTB))
+                   .and(lte(AbstractLogRecord.TIME_BUCKET, endTB));
+
+        if (from > Const.NONE) {
+            limit += from;
 
 Review comment:
   why do you need this? seems the `limit` doesn't include the `offset`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375122894
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/MetricsDAO.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.model.ModelColumn;
+import org.apache.skywalking.oap.server.core.storage.type.StorageDataType;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class MetricsDAO implements IMetricsDAO {
+    private final StorageBuilder<Metrics> storageBuilder;
+    private final InfluxClient client;
+
+    public MetricsDAO(InfluxClient client, StorageBuilder<Metrics> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public List<Metrics> multiGet(Model model, List<String> ids) throws IOException {
+        WhereQueryImpl<SelectQueryImpl> query = select("*::field")
+            .from(client.getDatabase(), model.getName())
+            .where(contains("id", Joiner.on("|").join(ids)));
+        List<QueryResult.Series> series = client.queryForSeries(query);
+        if (series == null || series.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        final List<Metrics> metrics = Lists.newArrayList();
+        List<String> columns = series.get(0).getColumns();
+        Map<String, String> storageAndColumnNames = Maps.newHashMap();
+        for (ModelColumn column : model.getColumns()) {
+            storageAndColumnNames.put(column.getColumnName().getName(), column.getColumnName().getStorageName());
+        }
+
+        series.get(0).getValues().forEach(values -> {
 
 Review comment:
   Another `#get(0)`. What does this mean? 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375128689
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   InfluxDB support user to send a request with multi statements. This is for a single statement in a request(query).
   So, `#get(0)` means get the first result of the query.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374619003
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/pom.xml
 ##########
 @@ -0,0 +1,50 @@
+<?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 specrm ific 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-storage-plugin</artifactId>
+        <groupId>org.apache.skywalking</groupId>
+        <version>7.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>storage-influxdb-plugin</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>storage-elasticsearch-plugin</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>storage-jdbc-hikaricp-plugin</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.influxdb</groupId>
+            <artifactId>influxdb-java</artifactId>
+            <version>2.15</version>
 
 Review comment:
   make it a property

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374611069
 
 

 ##########
 File path: test/e2e/e2e-influxdb/src/docker/rc.d/rc0-prepare.sh
 ##########
 @@ -0,0 +1,21 @@
+#!/usr/bin/env bash
+# Licensed to the SkyAPM 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.
+
+echo "InfluxDB with H2 database is storage provider..."
+
+# Modify application.yml to set MySQL as storage provider.
 
 Review comment:
   ```suggestion
   # Modify application.yml to set InfluxDB as storage provider.
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380049944
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/installer/MetaTableDefine.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.storage.plugin.influxdb.installer;
+
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.jdbc.TableMetaInfo;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENDPOINT_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.NETWORK_ADDRESS;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INSTANCE_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INVENTORY;
+
+/**
+ * Here defines which table is stored in metadata database(H2/MySQL).
+ */
+public class MetaTableDefine {
+
+    /**
+     * Test a {@link Model} is stored in H2/MySQL or not.
+     *
+     * @param model Model
+     * @return true if the {@link Model} is stored in H2/MySQL
+     */
+    public static boolean contains(Model model) {
+        switch (model.getScopeId()) {
+            case SERVICE_INVENTORY:
+            case SERVICE_INSTANCE_INVENTORY:
+            case NETWORK_ADDRESS:
+            case ENDPOINT_INVENTORY:
+            case PROFILE_TASK:
 
 Review comment:
   profile_task_log is not metadata. It is stored on InfluxDB.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-586780704
 
 
   @dmsolr Please recheck. The ElasticSearch test fails agian. I don't find any change related to you. Please take a look, what is the issue.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380045349
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   remove this, and other files with the `@author`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380046157
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   Why isn't this detected by CI process?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-584540142
 
 
   The strict format rule has been merged, please update this PR.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374654620
 
 

 ##########
 File path: oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/model/Model.java
 ##########
 @@ -35,6 +38,7 @@
     private final List<ModelColumn> columns;
     private final int scopeId;
     private final boolean record;
+    private final TreeMap<String, ModelColumn> storageColumns;
 
 Review comment:
   This should not be added.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374708505
 
 

 ##########
 File path: tools/dependencies/known-oap-backend-dependencies-es7.txt
 ##########
 @@ -159,3 +159,8 @@ sundr-core-0.9.2.jar
 swagger-annotations-1.5.12.jar
 t-digest-3.2.jar
 zookeeper-3.4.10.jar
+converter-moshi-2.5.0.jar
+influxdb-java-2.15.jar
+logging-interceptor-3.13.1.jar
+moshi-1.5.0.jar
+msgpack-core-0.8.16.jar
 
 Review comment:
   I am confusing about this. So, I need to take some time to know it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/58c5674a7be0efa5c67e98c97e314f4f35cdf88d?src=pr&el=desc) will **not change** coverage.
   > The diff coverage is `35.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff           @@
   ##           master    #4239   +/-   ##
   =======================================
     Coverage   25.75%   25.75%           
   =======================================
     Files        1200     1200           
     Lines       27598    27598           
     Branches     3817     3817           
   =======================================
     Hits         7107     7107           
     Misses      19857    19857           
     Partials      634      634
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...lking/oap/server/core/register/RegisterSource.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcmVnaXN0ZXIvUmVnaXN0ZXJTb3VyY2UuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...ywalking/oap/server/core/alarm/provider/Rules.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9SdWxlcy5qYXZh) | `100% <ø> (ø)` | :arrow_up: |
   | [...server/core/analysis/worker/PersistenceWorker.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvd29ya2VyL1BlcnNpc3RlbmNlV29ya2VyLmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...ywalking/oap/server/core/analysis/data/Window.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvZGF0YS9XaW5kb3cuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...ing/oap/server/core/register/ServiceInventory.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcmVnaXN0ZXIvU2VydmljZUludmVudG9yeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../core/analysis/worker/MetricsPersistentWorker.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvd29ya2VyL01ldHJpY3NQZXJzaXN0ZW50V29ya2VyLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...rage/plugin/jdbc/h2/dao/H2ProfileTaskQueryDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1qZGJjLWhpa2FyaWNwLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2pkYmMvaDIvZGFvL0gyUHJvZmlsZVRhc2tRdWVyeURBTy5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...vicecomb/v1/TransportClientHandlerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vc2VydmljZWNvbWItcGx1Z2luL3NlcnZpY2Vjb21iLWphdmEtY2hhc3Npcy0xLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vc2VydmljZWNvbWIvdjEvVHJhbnNwb3J0Q2xpZW50SGFuZGxlckludGVyY2VwdG9yLmphdmE=) | `57.14% <0%> (ø)` | :arrow_up: |
   | [...tion/service/ServiceRelationClientSideMetrics.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvbWFudWFsL3JlbGF0aW9uL3NlcnZpY2UvU2VydmljZVJlbGF0aW9uQ2xpZW50U2lkZU1ldHJpY3MuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...ing/oap/server/core/source/DefaultScopeDefine.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvc291cmNlL0RlZmF1bHRTY29wZURlZmluZS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [44 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [58c5674...2896232](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/12414c80e1819ccdd6d774bede6d9d33fe6af457?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `93.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   26.24%   26.27%   +0.02%     
   ==========================================
     Files        1194     1195       +1     
     Lines       26404    26409       +5     
     Branches     3767     3768       +1     
   ==========================================
   + Hits         6931     6939       +8     
   + Misses      18860    18858       -2     
   + Partials      613      612       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../core/profile/analyze/ProfileAnalyzeCollector.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVBbmFseXplQ29sbGVjdG9yLmphdmE=) | `100% <ø> (ø)` | :arrow_up: |
   | [...lking/oap/query/graphql/resolver/ProfileQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcXVlcnktcGx1Z2luL3F1ZXJ5LWdyYXBocWwtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9xdWVyeS9ncmFwaHFsL3Jlc29sdmVyL1Byb2ZpbGVRdWVyeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...p/server/core/profile/analyze/ProfileAnalyzer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVBbmFseXplci5qYXZh) | `77.77% <100%> (ø)` | :arrow_up: |
   | [...oap/server/core/query/entity/ProfileStackTree.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvZW50aXR5L1Byb2ZpbGVTdGFja1RyZWUuamF2YQ==) | `100% <100%> (ø)` | |
   | [.../server/core/profile/analyze/ProfileStackNode.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVTdGFja05vZGUuamF2YQ==) | `90.38% <100%> (+0.28%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `48.27% <0%> (+3.44%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [12414c8...ab75c86](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374654869
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,107 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class  BatchDAO implements IBatchDAO {
 
 Review comment:
   This file format seems not right.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375229729
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   Could we add a `#singleStatementQuery` method for most cases?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379833111
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/MetricsDAO.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.oap.server.storage.plugin.influxdb.base;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.model.ModelColumn;
+import org.apache.skywalking.oap.server.core.storage.type.StorageDataType;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class MetricsDAO implements IMetricsDAO {
+    public static final String TAG_ENTITY_ID = "_entity_id";
+
+    private final StorageBuilder<Metrics> storageBuilder;
+    private final InfluxClient client;
+
+    public MetricsDAO(InfluxClient client, StorageBuilder<Metrics> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public List<Metrics> multiGet(Model model, List<String> ids) throws IOException {
+        WhereQueryImpl<SelectQueryImpl> query = select()
+            .regex("*::field")
 
 Review comment:
   did not find related documentation, but will `.raw("*::field")` be more efficient?
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379832725
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   0. is there any particular reason why you paginate here instead of paginating in InfluxDB server side (using the `.limit`)?
   0. why do you sort them again in a different order (`SEGMENT_ID`?) when you already have `top` by `QueryOrder` in line 74 - 80, hope I did not miss any context

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581094442
 
 
   @dmsolr Besides the license issue, is this PR ready to be reviewed?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581281855
 
 
   Recommend to add e2e for this new storage in this PR. At least it could recheck this new feature is basically ready.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r368263056
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/InfluxStorageDAO.java
 ##########
 @@ -53,6 +53,6 @@ public IRecordDAO newRecordDao(StorageBuilder<Record> storageBuilder) {
 
     @Override
     public INoneStreamDAO newNoneStreamDao(StorageBuilder<NoneStream> storageBuilder) {
-        return new NoneStreamDAO();
+        return new NoneStreamDAO(influxClient, storageBuilder);
 
 Review comment:
   That sounds the relationship database better.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374598967
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/pom.xml
 ##########
 @@ -0,0 +1,50 @@
+<?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 specrm ific 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-storage-plugin</artifactId>
+        <groupId>org.apache.skywalking</groupId>
+        <version>7.0.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>storage-influxdb-plugin</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.skywalking</groupId>
+            <artifactId>storage-elasticsearch-plugin</artifactId>
+            <version>${project.version}</version>
+        </dependency>
 
 Review comment:
   why does influx plugin depend on ElasticSearch plugin

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **increase** coverage by `0.06%`.
   > The diff coverage is `10.28%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.36%   25.43%   +0.06%     
   ==========================================
     Files        1201     1203       +2     
     Lines       27753    27832      +79     
     Branches     3825     3837      +12     
   ==========================================
   + Hits         7040     7079      +39     
   - Misses      20079    20114      +35     
   - Partials      634      639       +5
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...rver/storage/plugin/influxdb/query/TraceQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9UcmFjZVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/AlarmQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9BbGFybVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...torage/plugin/influxdb/query/ProfileTaskQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/installer/MetaTableDefine.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvTWV0YVRhYmxlRGVmaW5lLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [13 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...4114303](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379879410
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   I got it. InfluxDB only ORDER BY time supported at this time. As far as I know, most of TSDB have the same issue.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376689989
 
 

 ##########
 File path: docs/en/setup/backend/backend-storage.md
 ##########
 @@ -226,6 +227,27 @@ storage:
 All connection related settings including link url, username and password are in `application.yml`. 
 These settings can refer to the configuration of *MySQL* above.
 
+## InfluxDB
+InfluxDB as storage since SkyWalking 7.0. It depends on `H2/MySQL` storage-plugin to store `metadata` likes `Inventory` and `ProfileTask`. So, when we set `InfluxDB` as storage provider, we need to configure `InfluxDB`'s properties and `H2/MySQL`.
+
+```yaml
+storage
+  influx:
+    # Metadata storage provider configuration
+    metabaseType: ${SW_STORAGE_METABASE_TYPE:H2}
 
 Review comment:
   Please provide type options.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376746175
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,203 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
 
 Review comment:
   I think this should be `private` to avoid misuse

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380509427
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/ProfileTaskQuery.java
 ##########
 @@ -95,7 +96,7 @@ public ProfileTask getById(final String id) throws IOException {
         WhereQueryImpl query = select("ID", ProfileTaskRecord.SERVICE_ID,
                                       ProfileTaskRecord.ENDPOINT_NAME, ProfileTaskRecord.START_TIME,
                                       ProfileTaskRecord.CREATE_TIME,
-                                      ProfileTaskRecord.DURATION,
+                                      "\"" + ProfileTaskRecord.DURATION + "\"", // scape, the 'duration' is identifier
 
 Review comment:
   Check `ModelInstaller#overrideColumnName`, you could replace the name in the storage implementation. That is a more elegant way.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **increase** coverage by `0.11%`.
   > The diff coverage is `20.37%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.36%   25.48%   +0.11%     
   ==========================================
     Files        1201     1201              
     Lines       27753    27784      +31     
     Branches     3825     3829       +4     
   ==========================================
   + Hits         7040     7080      +40     
   + Misses      20079    20066      -13     
   - Partials      634      638       +4
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...walking/oap/server/core/analysis/Downsampling.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvRG93bnNhbXBsaW5nLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...lasticsearch/StorageModuleElasticsearchConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hDb25maWcuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...p/server/core/profile/analyze/ProfileAnalyzer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVBbmFseXplci5qYXZh) | `65.95% <100%> (+51.37%)` | :arrow_up: |
   | [...age/plugin/elasticsearch/base/TimeSeriesUtils.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9UaW1lU2VyaWVzVXRpbHMuamF2YQ==) | `33.33% <43.75%> (+24.99%)` | :arrow_up: |
   | [...apache/skywalking/oal/rt/parser/MetricsHolder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9vYWwtcnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FsL3J0L3BhcnNlci9NZXRyaWNzSG9sZGVyLmphdmE=) | `87.5% <75%> (+0.83%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...170b15d](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375120216
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxStorageConfig.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.oap.server.storage.plugin.influxdb;
+
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.skywalking.oap.server.library.module.ModuleConfig;
+
+@Setter
+@Getter
+public class InfluxStorageConfig extends ModuleConfig {
 
 Review comment:
   Comments are required.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380443678
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/installer/MetaTableDefine.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.storage.plugin.influxdb.installer;
+
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.jdbc.TableMetaInfo;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENDPOINT_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.NETWORK_ADDRESS;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INSTANCE_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INVENTORY;
+
+/**
+ * Here defines which table is stored in metadata database(H2/MySQL).
+ */
+public class MetaTableDefine {
+
+    /**
+     * Test a {@link Model} is stored in H2/MySQL or not.
+     *
+     * @param model Model
+     * @return true if the {@link Model} is stored in H2/MySQL
+     */
+    public static boolean contains(Model model) {
+        switch (model.getScopeId()) {
+            case SERVICE_INVENTORY:
+            case SERVICE_INSTANCE_INVENTORY:
+            case NETWORK_ADDRESS:
+            case ENDPOINT_INVENTORY:
+            case PROFILE_TASK:
 
 Review comment:
   And `PROFILE_TASK` is not a metadata entity

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **increase** coverage by `0.07%`.
   > The diff coverage is `10.28%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.36%   25.43%   +0.07%     
   ==========================================
     Files        1201     1203       +2     
     Lines       27753    27832      +79     
     Branches     3825     3837      +12     
   ==========================================
   + Hits         7040     7080      +40     
   - Misses      20079    20114      +35     
   - Partials      634      638       +4
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...rver/storage/plugin/influxdb/query/TraceQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9UcmFjZVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/AlarmQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9BbGFybVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...torage/plugin/influxdb/query/ProfileTaskQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/installer/MetaTableDefine.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvTWV0YVRhYmxlRGVmaW5lLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [12 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...4ef910d](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4fda55d`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   25.58%           
   =========================================
     Files             ?     1196           
     Lines             ?    27470           
     Branches          ?     3786           
   =========================================
     Hits              ?     7028           
     Misses            ?    19826           
     Partials          ?      616
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...server/storage/plugin/influxdb/query/LogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Mb2dRdWVyeS5qYXZh) | `0% <ø> (ø)` | |
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <0%> (ø)` | |
   | [...r/storage/plugin/influxdb/query/TopologyQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Ub3BvbG9neVF1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   | [...storage/plugin/influxdb/InfluxStorageProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlUHJvdmlkZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4fda55d...4c06a1d](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@55f57e5`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `48.83%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   26.36%           
   =========================================
     Files             ?     1193           
     Lines             ?    26320           
     Branches          ?     3748           
   =========================================
     Hits              ?     6940           
     Misses            ?    18773           
     Partials          ?      607
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...walking/apm/agent/core/context/trace/NoopSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Ob29wU3Bhbi5qYXZh) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/ExitSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FeGl0U3Bhbi5qYXZh) | `53.57% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/LocalSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Mb2NhbFNwYW4uamF2YQ==) | `0% <ø> (ø)` | |
   | [...storage/plugin/influxdb/installer/H2Installer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvSDJJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...rage/plugin/influxdb/installer/MySQLInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9pbnN0YWxsZXIvTXlTUUxJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...storage/plugin/influxdb/base/HistoryDeleteDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL0hpc3RvcnlEZWxldGVEQU8uamF2YQ==) | `0% <0%> (ø)` | |
   | [...vation/opentracing/span/SpanSetTagInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC1vcGVudHJhY2luZy1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vb3BlbnRyYWNpbmcvc3Bhbi9TcGFuU2V0VGFnSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [.../customize/interceptor/BaseInterceptorMethods.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy9jdXN0b21pemUtZW5oYW5jZS1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9jdXN0b21pemUvaW50ZXJjZXB0b3IvQmFzZUludGVyY2VwdG9yTWV0aG9kcy5qYXZh) | `0% <0%> (ø)` | |
   | [...walking/apm/plugin/elasticsearch/v5/Constants.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZWxhc3RpY3NlYXJjaC01LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZWxhc3RpY3NlYXJjaC92NS9Db25zdGFudHMuamF2YQ==) | `0% <0%> (ø)` | |
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | |
   | ... and [12 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [55f57e5...d1b2860](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/fb1b87538d949f3a2b87e2e14836f931312ee1b7?src=pr&el=desc) will **decrease** coverage by `0.02%`.
   > The diff coverage is `35.13%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   26.27%   26.25%   -0.03%     
   ==========================================
     Files        1196     1194       -2     
     Lines       26451    26399      -52     
     Branches     3767     3766       -1     
   ==========================================
   - Hits         6951     6931      -20     
   + Misses      18886    18855      -31     
   + Partials      614      613       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...storage/plugin/influxdb/InfluxStorageProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlUHJvdmlkZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...g/apache/skywalking/apm/agent/SkyWalkingAgent.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9hZ2VudC9Ta3lXYWxraW5nQWdlbnQuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/trace/ignore/TraceIgnoreExtendService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy90cmFjZS1pZ25vcmUtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vdHJhY2UvaWdub3JlL1RyYWNlSWdub3JlRXh0ZW5kU2VydmljZS5qYXZh) | `92.85% <100%> (+10.5%)` | :arrow_up: |
   | [...gin/trace/ignore/conf/IgnoreConfigInitializer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy90cmFjZS1pZ25vcmUtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vdHJhY2UvaWdub3JlL2NvbmYvSWdub3JlQ29uZmlnSW5pdGlhbGl6ZXIuamF2YQ==) | `38.23% <100%> (-7.92%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `79.41% <100%> (ø)` | :arrow_up: |
   | [...apm/agent/core/plugin/loader/AgentClassLoader.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcGx1Z2luL2xvYWRlci9BZ2VudENsYXNzTG9hZGVyLmphdmE=) | `4.1% <5.55%> (-6.79%)` | :arrow_down: |
   | [.../apm/agent/core/conf/SnifferConfigInitializer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9TbmlmZmVyQ29uZmlnSW5pdGlhbGl6ZXIuamF2YQ==) | `70.37% <85.71%> (-1.23%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [fb1b875...c148bfb](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374619657
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,131 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    public static final String TIME = "time";
+    public static final String TAG_ENTITY_ID = "entity_id";
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + "\r\nSQL Statement: " + query.getCommand(), e);
 
 Review comment:
   use `System.lineSeparator()` instead of hardcoded `\r\n`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375072105
 
 

 ##########
 File path: oap-server/server-bootstrap/src/main/resources/application.yml
 ##########
 @@ -75,29 +75,29 @@ core:
     enableDatabaseSession: ${SW_CORE_ENABLE_DATABASE_SESSION:true}
     topNReportPeriod: ${SW_CORE_TOPN_REPORT_PERIOD:10} # top_n record worker report cycle, unit is minute
 storage:
-#  elasticsearch:
-#    nameSpace: ${SW_NAMESPACE:""}
-#    clusterNodes: ${SW_STORAGE_ES_CLUSTER_NODES:localhost:9200}
-#    protocol: ${SW_STORAGE_ES_HTTP_PROTOCOL:"http"}
-#    #trustStorePath: ${SW_SW_STORAGE_ES_SSL_JKS_PATH:"../es_keystore.jks"}
-#    #trustStorePass: ${SW_SW_STORAGE_ES_SSL_JKS_PASS:""}
-#    user: ${SW_ES_USER:""}
-#    password: ${SW_ES_PASSWORD:""}
-#    indexShardsNumber: ${SW_STORAGE_ES_INDEX_SHARDS_NUMBER:2}
-#    indexReplicasNumber: ${SW_STORAGE_ES_INDEX_REPLICAS_NUMBER:0}
-#    # Those data TTL settings will override the same settings in core module.
-#    recordDataTTL: ${SW_STORAGE_ES_RECORD_DATA_TTL:7} # Unit is day
-#    otherMetricsDataTTL: ${SW_STORAGE_ES_OTHER_METRIC_DATA_TTL:45} # Unit is day
-#    monthMetricsDataTTL: ${SW_STORAGE_ES_MONTH_METRIC_DATA_TTL:18} # Unit is month
-#    # Batch process setting, refer to https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.5/java-docs-bulk-processor.html
-#    bulkActions: ${SW_STORAGE_ES_BULK_ACTIONS:1000} # Execute the bulk every 1000 requests
-#    flushInterval: ${SW_STORAGE_ES_FLUSH_INTERVAL:10} # flush the bulk every 10 seconds whatever the number of requests
-#    concurrentRequests: ${SW_STORAGE_ES_CONCURRENT_REQUESTS:2} # the number of concurrent requests
-#    resultWindowMaxSize: ${SW_STORAGE_ES_QUERY_MAX_WINDOW_SIZE:10000}
-#    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
-#    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
-#    profileTaskQueryMaxSize: ${SW_STORAGE_ES_QUERY_PROFILE_TASK_SIZE:200}
-#    advanced: ${SW_STORAGE_ES_ADVANCED:""}
+  #  elasticsearch:
+  #    nameSpace: ${SW_NAMESPACE:""}
+  #    clusterNodes: ${SW_STORAGE_ES_CLUSTER_NODES:localhost:9200}
+  #    protocol: ${SW_STORAGE_ES_HTTP_PROTOCOL:"http"}
+  #    #trustStorePath: ${SW_SW_STORAGE_ES_SSL_JKS_PATH:"../es_keystore.jks"}
+  #    #trustStorePass: ${SW_SW_STORAGE_ES_SSL_JKS_PASS:""}
+  #    user: ${SW_ES_USER:""}
+  #    password: ${SW_ES_PASSWORD:""}
+  #    indexShardsNumber: ${SW_STORAGE_ES_INDEX_SHARDS_NUMBER:2}
+  #    indexReplicasNumber: ${SW_STORAGE_ES_INDEX_REPLICAS_NUMBER:0}
+  #    # Those data TTL settings will override the same settings in core module.
+  #    recordDataTTL: ${SW_STORAGE_ES_RECORD_DATA_TTL:7} # Unit is day
+  #    otherMetricsDataTTL: ${SW_STORAGE_ES_OTHER_METRIC_DATA_TTL:45} # Unit is day
+  #    monthMetricsDataTTL: ${SW_STORAGE_ES_MONTH_METRIC_DATA_TTL:18} # Unit is month
+  #    # Batch process setting, refer to https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.5/java-docs-bulk-processor.html
+  #    bulkActions: ${SW_STORAGE_ES_BULK_ACTIONS:1000} # Execute the bulk every 1000 requests
+  #    flushInterval: ${SW_STORAGE_ES_FLUSH_INTERVAL:10} # flush the bulk every 10 seconds whatever the number of requests
+  #    concurrentRequests: ${SW_STORAGE_ES_CONCURRENT_REQUESTS:2} # the number of concurrent requests
+  #    resultWindowMaxSize: ${SW_STORAGE_ES_QUERY_MAX_WINDOW_SIZE:10000}
+  #    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
+  #    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
+  #    profileTaskQueryMaxSize: ${SW_STORAGE_ES_QUERY_PROFILE_TASK_SIZE:200}
+  #    advanced: ${SW_STORAGE_ES_ADVANCED:""}
 
 Review comment:
   These formats are still changed.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374642128
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxTTLCalculatorProvider.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import org.apache.skywalking.oap.server.core.DataTTLConfig;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder;
+import org.joda.time.DateTime;
+
+public class InfluxTTLCalculatorProvider {
 
 Review comment:
   There is `GeneralStorageTTL` existing, the influxdb implementation should be as same as that one, since it also uses the core configuration.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4fda55d`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   25.57%           
   =========================================
     Files             ?     1196           
     Lines             ?    27470           
     Branches          ?     3786           
   =========================================
     Hits              ?     7026           
     Misses            ?    19827           
     Partials          ?      617
   ```
   
   
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4fda55d...4c06a1d](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379902233
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -71,15 +73,19 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Cluster Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=elasticsearch
+      - name: Cluster Tests (InfluxDB/ZK/JDK8)
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=influxdb
       - name: Cluster With Gateway Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
 
 Review comment:
   > Why this doesn't have `--storage=elasticsearch`? What is the default? Or what happens if no `--storage=`?
   
   Not every e2e case is aware of the parameter `--storage`, `e2e-cluster-with-gateway` is one of them, it's intruduced recently and only `e2e-cluster` and `e2e-profile` are refactored then, IIRC
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/58c5674a7be0efa5c67e98c97e314f4f35cdf88d?src=pr&el=desc) will **decrease** coverage by `0.33%`.
   > The diff coverage is `28.11%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   25.75%   25.41%   -0.34%     
   ==========================================
     Files        1200     1201       +1     
     Lines       27598    27701     +103     
     Branches     3817     3820       +3     
   ==========================================
   - Hits         7107     7040      -67     
   - Misses      19857    20027     +170     
     Partials      634      634
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...lking/oap/server/core/register/RegisterSource.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcmVnaXN0ZXIvUmVnaXN0ZXJTb3VyY2UuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...king/oap/server/library/util/GRPCStreamStatus.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItbGlicmFyeS9saWJyYXJ5LXV0aWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9saWJyYXJ5L3V0aWwvR1JQQ1N0cmVhbVN0YXR1cy5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...ywalking/oap/server/core/alarm/provider/Rules.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9SdWxlcy5qYXZh) | `100% <ø> (ø)` | :arrow_up: |
   | [...server/core/analysis/worker/PersistenceWorker.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvd29ya2VyL1BlcnNpc3RlbmNlV29ya2VyLmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...ywalking/oap/server/core/analysis/data/Window.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvZGF0YS9XaW5kb3cuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...ing/oap/server/core/register/ServiceInventory.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcmVnaXN0ZXIvU2VydmljZUludmVudG9yeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...intrelation/EndpointRelationServerSideMetrics.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvbWFudWFsL2VuZHBvaW50cmVsYXRpb24vRW5kcG9pbnRSZWxhdGlvblNlcnZlclNpZGVNZXRyaWNzLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...tion/service/ServiceRelationClientSideMetrics.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvbWFudWFsL3JlbGF0aW9uL3NlcnZpY2UvU2VydmljZVJlbGF0aW9uQ2xpZW50U2lkZU1ldHJpY3MuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...servicecomb/TransportClientHandlerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vc2VydmljZWNvbWItcGx1Z2luL3NlcnZpY2Vjb21iLWphdmEtY2hhc3Npcy0wLngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vc2VydmljZWNvbWIvVHJhbnNwb3J0Q2xpZW50SGFuZGxlckludGVyY2VwdG9yLmphdmE=) | `57.14% <0%> (ø)` | :arrow_up: |
   | ... and [108 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [58c5674...e6105b7](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380048045
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   > Why isn't this detected by CI process?
   
   CI belongs to the root module and its submodules, e2e and plugin tests are neither of them, maybe set up the same Checkstyle plugin in e2e and plugin tests?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-574940697
 
 
   Also, as we should market this feature, before 7.0.0 release, @dmsolr you should do some performance tests comparing this with ES storage in the same case. I am assuming this one has advantages :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374638027
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,131 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 Review comment:
   Agree. `@slf4j` good enough for me.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374641089
 
 

 ##########
 File path: tools/dependencies/known-oap-backend-dependencies-es7.txt
 ##########
 @@ -159,3 +159,8 @@ sundr-core-0.9.2.jar
 swagger-annotations-1.5.12.jar
 t-digest-3.2.jar
 zookeeper-3.4.10.jar
+converter-moshi-2.5.0.jar
+influxdb-java-2.15.jar
+logging-interceptor-3.13.1.jar
+moshi-1.5.0.jar
+msgpack-core-0.8.16.jar
 
 Review comment:
   All these dependencies licenses should be checked and LICENSE file should be updated(NOTICE should be update if the license is Apache 2.0 and there is a NOTICE existing).

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379832246
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/ProfileThreadSnapshotQuery.java
 ##########
 @@ -0,0 +1,165 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.storage.profile.IProfileThreadSnapshotQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class ProfileThreadSnapshotQuery implements IProfileThreadSnapshotQueryDAO {
+    private final InfluxClient client;
+
+    public ProfileThreadSnapshotQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public List<BasicTrace> queryProfiledSegments(String taskId) throws IOException {
+        WhereQueryImpl query = select(ProfileThreadSnapshotRecord.SEGMENT_ID)
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where()
+            .and(eq(ProfileThreadSnapshotRecord.TASK_ID, taskId))
+            .and(eq(ProfileThreadSnapshotRecord.SEQUENCE, 0));
+
+        final LinkedList<String> segments = new LinkedList<>();
+        QueryResult.Series series = client.queryForSingleSeries(query);
+        if (series == null) {
+            return Collections.emptyList();
+        }
+        series.getValues().forEach(values -> {
+            segments.add((String) values.get(1));
+        });
+
+        if (segments.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        query = select()
+            .function("bottom", SegmentRecord.START_TIME, segments.size())
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where()
+            .and(contains(SegmentRecord.SEGMENT_ID, Joiner.on("|").join(segments)));
+
+        ArrayList<BasicTrace> result = Lists.newArrayListWithCapacity(segments.size());
+        client.queryForSingleSeries(query)
+              .getValues()
+              .stream()
+              .sorted((a, b) -> Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue()))
+              .forEach(values -> {
+                  BasicTrace basicTrace = new BasicTrace();
+
+                  basicTrace.setSegmentId((String) values.get(2));
+                  basicTrace.setStart(String.valueOf(values.get(3)));
+                  basicTrace.getEndpointNames().add((String) values.get(4));
+                  basicTrace.setDuration((int) values.get(5));
+                  basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+                  String traceIds = (String) values.get(7);
+                  basicTrace.getTraceIds().add(traceIds);
+
+                  result.add(basicTrace);
+              });
+
+        return result;
+    }
+
+    @Override
+    public int queryMinSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("min", segmentId, start, end);
+    }
+
+    @Override
+    public int queryMaxSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("max", segmentId, start, end);
+    }
+
+    @Override
+    public List<ProfileThreadSnapshotRecord> queryRecords(String segmentId, int minSequence,
+                                                          int maxSequence) throws IOException {
+        WhereQueryImpl query = select(
+            ProfileThreadSnapshotRecord.TASK_ID,
+            ProfileThreadSnapshotRecord.SEGMENT_ID,
+            ProfileThreadSnapshotRecord.DUMP_TIME,
+            ProfileThreadSnapshotRecord.SEQUENCE,
+            ProfileThreadSnapshotRecord.STACK_BINARY
+        )
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where(eq(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId))
+            .and(gte(ProfileThreadSnapshotRecord.SEQUENCE, minSequence))
+            .and(lte(ProfileThreadSnapshotRecord.SEQUENCE, maxSequence));
+
+        ArrayList<ProfileThreadSnapshotRecord> result = new ArrayList<>(maxSequence - minSequence);
+        client.queryForSingleSeries(query).getValues().forEach(values -> {
+            ProfileThreadSnapshotRecord record = new ProfileThreadSnapshotRecord();
+
+            record.setTaskId((String) values.get(1));
+            record.setSegmentId((String) values.get(2));
+            record.setDumpTime(((Number) values.get(3)).longValue());
+            record.setSequence((int) values.get(4));
+            String dataBinaryBase64 = String.valueOf(values.get(5));
+            if (StringUtil.isNotEmpty(dataBinaryBase64)) {
+                record.setStackBinary(Base64.getDecoder().decode(dataBinaryBase64));
 
 Review comment:
   better to use `org.apache.skywalking.apm.agent.core.base64.Base64` for consistence, and the default charset are different from other places

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380045349
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   remove this, and those in other files

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379828752
 
 

 ##########
 File path: oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/TimeBucket.java
 ##########
 @@ -32,10 +32,119 @@ public static long getRecordTimeBucket(long time) {
         return getTimeBucket(time, Downsampling.Second);
     }
 
+    /**
+     * Record time bucket format in Minute Unit.
+     *
+     * @param time Timestamp
+     * @return time in minute format.
+     */
     public static long getMinuteTimeBucket(long time) {
         return getTimeBucket(time, Downsampling.Minute);
     }
 
+    /**
+     * Convert TimeBucket to Timestamp in millisecond.
+     *
+     * @param timeBucket long
+     * @return timestamp in millisecond unit
+     */
+    public static long getTimestamp(long timeBucket) {
+        if (isSecondBucket(timeBucket)) {
+            return getTimestamp(timeBucket, Downsampling.Second);
+        } else if (isMinuteBucket(timeBucket)) {
+            return getTimestamp(timeBucket, Downsampling.Minute);
+        } else if (isHourBucket(timeBucket)) {
+            return getTimestamp(timeBucket, Downsampling.Hour);
+        } else if (isDayBucket(timeBucket)) {
+            return getTimestamp(timeBucket, Downsampling.Day);
+        } else if (isMonthBucket(timeBucket)) {
+            return getTimestamp(timeBucket, Downsampling.Month);
+        } else {
+            throw new UnexpectedException("Unknown downsampling value.");
+        }
+    }
+
+    /**
+     * The format of timeBucket in minute Unit is "yyyyMMddHHmmss", so which means the TimeBucket mush between
 
 Review comment:
   `mush` -> `must be`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381078944
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,191 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag of time_bucket.
+     */
+    public static final String TAG_TIME_BUCKET = "_time_bucket";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+                                         new OkHttpClient.Builder().readTimeout(3, TimeUnit.MINUTES)
+                                                                   .writeTimeout(3, TimeUnit.MINUTES),
+                                         InfluxDB.ResponseFormat.MSGPACK
+        );
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    private InfluxDB getInflux() {
+        return influx;
 
 Review comment:
   recommend add a null value to judge here,subsequence queries are based on the available influx

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r373958815
 
 

 ##########
 File path: oap-server/server-bootstrap/src/main/resources/application.yml
 ##########
 @@ -146,6 +146,11 @@ storage:
 #    password: ${SW_STORAGE_PASSWORD:}
 #    database: ${SW_STORAGE_DATABASE:skywalking}
 #    metadataQueryMaxSize: ${SW_STORAGE_H2_QUERY_MAX_SIZE:5000}
+#    recordDataTTL: ${SW_STORAGE_RECORD_DATA_TTL:7} # keeps 7 days
+#    minuteMetricsDataTTL: ${SW_STORAGE_MINUTE_METRICS_DATA_TTL:64800} # Unit is minute, keeps 45 days
+#    hourMetricsDataTTL: ${SW_STORAGE_HOUR_METRICS_DATA_TTL:1080} # Unit is hour, keeps 45 days
+#    dayMetricsDataTTL: ${SW_STORAGE_DAY_METRICS_DATA_TTL:45} # Unit is day, keeps 45 days
+#    monthMetricsDataTTL: ${SW_STORAGE_MONTH_METRICS_DATA_TTL:18} # Unit is month, keeps 18 months
 
 Review comment:
   If these are using the as same unit as the core module, you shouldn't add them. You could use the core's.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **increase** coverage by `0.06%`.
   > The diff coverage is `9.9%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.36%   25.43%   +0.06%     
   ==========================================
     Files        1201     1203       +2     
     Lines       27753    27835      +82     
     Branches     3825     3837      +12     
   ==========================================
   + Hits         7040     7080      +40     
   - Misses      20079    20117      +38     
   - Partials      634      638       +4
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...rver/storage/plugin/influxdb/query/TraceQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9UcmFjZVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/AlarmQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9BbGFybVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...torage/plugin/influxdb/query/ProfileTaskQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   | [...lasticsearch/StorageModuleElasticsearchConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hDb25maWcuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [13 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...a2a97f9](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-583774959
 
 
   > All `proto` files under `test/e2e` should be removed, instead, add the protocol repository as submodule, just like what `e2e-ttl-es` does, otherwise, they may be outdated
   > 
   > And I hope you could refactor the `e2e-ttl-es` to something like `e2e-ttl` that can be reused for multiple storage types, like `e2e-profile`, we can use the same codes to test multiple storage types
   
   I am planning to do it, but it is a low priority in my schedule.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380052010
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   We don't check `./test` directory.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381080169
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,191 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag of time_bucket.
+     */
+    public static final String TAG_TIME_BUCKET = "_time_bucket";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+                                         new OkHttpClient.Builder().readTimeout(3, TimeUnit.MINUTES)
+                                                                   .writeTimeout(3, TimeUnit.MINUTES),
+                                         InfluxDB.ResponseFormat.MSGPACK
+        );
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    private InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Execute a query against InfluxDB and return a set of {@link QueryResult.Result}s. Normally, InfluxDB supports
+     * combining multiple statements into one query, so that we do get multi-results.
+     *
+     * @throws IOException if there is an error on the InfluxDB server or communication error.
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement.
+     *
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        List<QueryResult.Result> results = query(query);
+
+        if (CollectionUtils.isEmpty(results)) {
+            return null;
+        }
+        return results.get(0).getSeries();
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement but return a single {@link QueryResult.Series}.
+     *
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public QueryResult.Series queryForSingleSeries(Query query) throws IOException {
+        List<QueryResult.Series> series = queryForSeries(query);
+        if (CollectionUtils.isEmpty(series)) {
+            return null;
+        }
+        return series.get(0);
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success. Notice, drop series don't support to drop series by range
+     *
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
+        QueryResult result = getInflux().query(query);
+
+        if (result.hasError()) {
+            throw new IOException("Statement: " + query.getCommand() + ", ErrorMsg: " + result.getError());
+        }
+    }
+
+    public void deleteByQuery(String measurement, long timestamp) throws IOException {
 
 Review comment:
   change method name `deleteBefore` or else better? 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374623410
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/MetricsDAO.java
 ##########
 @@ -0,0 +1,95 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.model.ModelColumn;
+import org.apache.skywalking.oap.server.core.storage.type.StorageDataType;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class MetricsDAO implements IMetricsDAO {
+    private final StorageBuilder<Metrics> storageBuilder;
+    private final InfluxClient client;
+
+    public MetricsDAO(InfluxClient client, StorageBuilder<Metrics> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public List<Metrics> multiGet(Model model, List<String> ids) throws IOException {
+        WhereQueryImpl query = select("*::field")
 
 Review comment:
   Raw use of parameterized class `WhereQueryImpl`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379832871
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
+        return traceBrief;
+    }
+
+    @Override
+    public List<SegmentRecord> queryByTraceId(String traceId) throws IOException {
+        WhereQueryImpl query = select().column(SegmentRecord.SEGMENT_ID)
 
 Review comment:
   All the `WhereQueryImpl` may be replaced by  `org.influxdb.dto.Query`?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581870855
 
 
   The corresponding `yml` in `docker-entrypoint.sh` should be also updated

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379828221
 
 

 ##########
 File path: docker/oap-es7/docker-entrypoint.sh
 ##########
 @@ -149,6 +149,37 @@ storage:
 EOT
 }
 
+generateStorageInfluxDB() {
+    cat <<EOT >> ${var_application_file}
+storage:
+  influx:
+    # Metadata storage provider configuration
+    metabaseType: ${SW_STORAGE_METABASE_TYPE:H2} # There are 2 options as Metabase provider, H2 or MySQL.
+    h2Props:
+      dataSourceClassName: ${SW_STORAGE_METABASE_DRIVER:org.h2.jdbcx.JdbcDataSource}
+      dataSource.url: ${SW_STORAGE_METABASE_URL:jdbc:h2:mem:skywalking-oap-db}
+      dataSource.user: ${SW_STORAGE_METABASE_USER:sa}
+      dataSource.password: ${SW_STORAGE_METABASE_PASSWORD:}
+    mysqlProps:
+      jdbcUrl: ${SW_STORAGE_METABASE_URL:"jdbc:mysql://localhost:3306/swtest"}
+      dataSource.user: ${SW_STORAGE_METABASE_USER:root}
+      dataSource.password: ${SW_STORAGE_METABASE_PASSWORD:root@1234}
+      dataSource.cachePrepStmts: ${SW_STORAGE_METABASE_CACHE_PREP_STMTS:true}
+      dataSource.prepStmtCacheSize: ${SW_STORAGE_METABASE_PREP_STMT_CACHE_SQL_SIZE:250}
+      dataSource.prepStmtCacheSqlLimit: ${SW_STORAGE_METABASE_PREP_STMT_CACHE_SQL_LIMIT:2048}
+      dataSource.useServerPrepStmts: ${SW_STORAGE_METABASE_USE_SERVER_PREP_STMTS:true}
+    metadataQueryMaxSize: ${SW_STORAGE_METABASE_QUERY_MAX_SIZE:5000}
+    # InfluxDB configuration
+    url: ${SW_STORAGE_INFLUXDB_URL:http://localhost:8086}
+    user: ${SW_STORAGE_INFLUXDB_USER:root}
+    password: ${SW_STORAGE_INFLUXDB_PASSWORD:}
+    database: ${SW_STORAGE_INFLUXDB_DATABASE:skywalking}
+    actions: ${SW_STORAGE_INFLUXDB_ACTIONS:1000} # the number of actions to collect
+    duration: ${SW_STORAGE_INFLUXDB_DURATION:1000} # the time to wait at most (milliseconds)
+    fetchTaskLogMaxSize: ${SW_STORAGE_INFLUXDB_FETCH_TASK_LOG_MAX_SIZE:5000} # the max number of fetch task log in a request
 
 Review comment:
   Should escape the `$` character

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581261228
 
 
   > I will adjust e2e-MySQL compatible
   
   What is the meaning of this?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374601890
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/AggregationQuery.java
 ##########
 @@ -0,0 +1,138 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.query.entity.Order;
+import org.apache.skywalking.oap.server.core.query.entity.TopNEntity;
+import org.apache.skywalking.oap.server.core.register.EndpointInventory;
+import org.apache.skywalking.oap.server.core.register.ServiceInstanceInventory;
+import org.apache.skywalking.oap.server.core.storage.model.ModelName;
+import org.apache.skywalking.oap.server.core.storage.query.IAggregationQueryDAO;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.SelectSubQueryImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.*;
+
+public class AggregationQuery implements IAggregationQueryDAO {
+    private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private final InfluxClient client;
+
+    public AggregationQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public List<TopNEntity> getServiceTopN(String indName, String valueCName, int topN, Downsampling downsampling,
+                                           long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getAllServiceInstanceTopN(String indName, String valueCName, int topN, Downsampling downsampling,
+                                                      long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getServiceInstanceTopN(int serviceId, String indName, String valueCName, int topN, Downsampling downsampling,
+                                                   long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(ServiceInstanceInventory.SERVICE_ID, serviceId, indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getAllEndpointTopN(String indName, String valueCName, int topN, Downsampling downsampling,
+                                               long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    @Override
+    public List<TopNEntity> getEndpointTopN(int serviceId, String indName, String valueCName, int topN, Downsampling downsampling,
+                                            long startTB, long endTB, Order order) throws IOException {
+        return getTopNEntity(downsampling, indName, subQuery(EndpointInventory.SERVICE_ID, serviceId, indName, valueCName, startTB, endTB), order, topN);
+    }
+
+    private final List<TopNEntity> getTopNEntity(Downsampling downsampling, String name, SelectSubQueryImpl<SelectQueryImpl> subQuery, Order order, int topN) throws IOException {
 
 Review comment:
   As private methods cannot be meaningfully overridden, declaring them final is redundant.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379867129
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   2. And there may be performance issue when paging deeply

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379832725
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   0. is there any particular reason why you paginate here instead of paginating in InfluxDB server side (using the `.limit`)?
   0. why do you sort them again in a different order (`SEGMENT_ID`?) when you already have `top` by `QueryOrder` in line 74 - 80
   
   hope I did not miss any context

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378850400
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/PointBuilder.java
 ##########
 @@ -0,0 +1,138 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeInteger;
+import org.apache.skywalking.oap.server.core.alarm.AlarmRecord;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.analysis.record.Record;
+import org.apache.skywalking.oap.server.core.analysis.topn.TopN;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskLogRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+import org.apache.skywalking.oap.server.core.storage.StorageData;
+import org.apache.skywalking.oap.server.core.storage.model.ColumnName;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.model.ModelColumn;
+import org.apache.skywalking.oap.server.core.storage.type.StorageDataType;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.Point;
+
+import static org.apache.skywalking.oap.server.core.analysis.TimeBucket.getTimestamp;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ALARM;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.DATABASE_SLOW_STATEMENT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.HTTP_ACCESS_LOG;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_LOG;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SEGMENT;
+
+/**
+ * A helper help to build a InfluxDB Point from StorageData.
+ */
+public class PointBuilder {
 
 Review comment:
   Could you explain this class a little more? I am a little confused. All existing entities have the shared `prepareBatchInsert` and `prepareBatchUpdate` logic without explicit entity name required. Using the literal string name here is highly unstable. These names could be changed in any PR, and new entity could be added.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374708701
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxTTLCalculatorProvider.java
 ##########
 @@ -0,0 +1,102 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import org.apache.skywalking.oap.server.core.DataTTLConfig;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder;
+import org.joda.time.DateTime;
+
+public class InfluxTTLCalculatorProvider {
 
 Review comment:
   got it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381125075
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/HistoryDeleteDAO.java
 ##########
 @@ -0,0 +1,71 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.io.IOException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.CoreModule;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.config.ConfigService;
+import org.apache.skywalking.oap.server.core.storage.IHistoryDeleteDAO;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.ttl.StorageTTL;
+import org.apache.skywalking.oap.server.core.storage.ttl.TTLCalculator;
+import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.joda.time.DateTime;
+
+@Slf4j
+public class HistoryDeleteDAO implements IHistoryDeleteDAO {
+    private final ModuleDefineHolder moduleDefineHolder;
+    private final InfluxClient client;
+    private final StorageTTL storageTTL;
+
+    public HistoryDeleteDAO(ModuleDefineHolder moduleDefineHolder, InfluxClient client, StorageTTL storageTTL) {
+        this.moduleDefineHolder = moduleDefineHolder;
+        this.storageTTL = storageTTL;
+        this.client = client;
+    }
+
+    @Override
+    public void deleteHistory(Model model, String timeBucketColumnName) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("TTL execution log, model: {}", model.getName());
+        }
+        try {
+            ConfigService configService = moduleDefineHolder.find(CoreModule.NAME)
+                                                            .provider()
+                                                            .getService(ConfigService.class);
+
+            TTLCalculator ttlCalculator;
+            if (model.isRecord()) {
+                ttlCalculator = storageTTL.recordCalculator();
+            } else {
+                ttlCalculator = storageTTL.metricsCalculator(model.getDownsampling());
+            }
+
+            client.deleteByQuery(
+                model.getName(),
+                TimeBucket.getTimestamp(ttlCalculator.timeBefore(DateTime.now(), configService.getDataTTLConfig()) + 1)
 
 Review comment:
   The condition, we delete the history data, is `less than and equal`.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581257953
 
 
   > Is this PR ready to review?
   
   Yes.
   Besides, I will adjust e2e-MySQL compatible with other storage-plugin(InfluxDB) in a new PR.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378846947
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,210 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+                                         new OkHttpClient.Builder().readTimeout(3, TimeUnit.MINUTES)
+                                                                   .writeTimeout(3, TimeUnit.MINUTES),
+                                         InfluxDB.ResponseFormat.MSGPACK
+        );
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    private InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Execute a query against InfluxDB and return a set of {@link QueryResult.Result}s. Normally, InfluxDB supports
+     * combining multiple statements into one query, so that we do get multi-results.
+     *
+     * @param query Query
+     * @return a set of {@link QueryResult.Result}s.
+     * @throws IOException if there is an error on the InfluxDB server or communication error.
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement.
+     *
+     * @param query Query
+     * @return a set of {@link QueryResult.Series}s
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        List<QueryResult.Result> results = query(query);
+
+        if (CollectionUtils.isEmpty(results)) {
+            return null;
+        }
+        return results.get(0).getSeries();
+    }
+
+    /**
+     * Execute a query against InfluxDB with a single statement but return a single {@link QueryResult.Series}.
+     *
+     * @param query Query
+     * @return {@link QueryResult.Series}
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public QueryResult.Series queryForSingleSeries(Query query) throws IOException {
+        List<QueryResult.Series> series = queryForSeries(query);
+        if (CollectionUtils.isEmpty(series)) {
+            return null;
+        }
+        return series.get(0);
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success. Notice, drop series don't support to drop series by range
+     *
+     * @param measurement String
+     * @param timeBucket  long
+     * @throws IOException if there is an error on the InfluxDB server or communication error
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
+        QueryResult result = getInflux().query(query);
+
+        if (result.hasError()) {
+            throw new IOException("Statement: " + query.getCommand() + ", ErrorMsg: " + result.getError());
+        }
+    }
+
+    public void deleteByQuery(String measurement, long timestamp) throws IOException {
+        this.query(new Query("delete from " + measurement + " where time < " + timestamp + "ms"));
+    }
+
+    /**
+     * Write a {@link Point} into InfluxDB. Note that, the {@link Point} is written into buffer of InfluxDB Client and
+     * wait for buffer flushing.
+     *
+     * @param point Point
 
 Review comment:
   From here, all comments about @param and @return are either empty and meaningless. Could you fix this?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/c148bfb6bd5ed585e39a1e1bdc2dfe061fd511f8?src=pr&el=desc) will **decrease** coverage by `<.01%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   26.25%   26.24%   -0.01%     
   ==========================================
     Files        1194     1194              
     Lines       26399    26404       +5     
     Branches     3766     3767       +1     
   ==========================================
     Hits         6931     6931              
   - Misses      18855    18860       +5     
     Partials      613      613
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../server/storage/plugin/influxdb/base/BatchDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL0JhdGNoREFPLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...gin/influxdb/query/ProfileThreadSnapshotQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGhyZWFkU25hcHNob3RRdWVyeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/AlarmQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9BbGFybVF1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...server/storage/plugin/influxdb/query/LogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Mb2dRdWVyeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...erver/storage/plugin/influxdb/base/MetricsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL01ldHJpY3NEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/TraceQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9UcmFjZVF1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...r/storage/plugin/influxdb/query/TopologyQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Ub3BvbG9neVF1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...ver/storage/plugin/influxdb/base/PointBuilder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL1BvaW50QnVpbGRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [c148bfb...12414c8](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4b02679`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   25.86%           
   =========================================
     Files             ?     1189           
     Lines             ?    26216           
     Branches          ?     3729           
   =========================================
     Hits              ?     6780           
     Misses            ?    18833           
     Partials          ?      603
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ver/storage/plugin/influxdb/base/PointBuilder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL1BvaW50QnVpbGRlci5qYXZh) | `0% <0%> (ø)` | |
   | [...er/storage/plugin/influxdb/query/MetricsQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9NZXRyaWNzUXVlcnkuamF2YQ==) | `0% <0%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4b02679...9128aec](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378014946
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
+      - name: Single Node Tests(InfluxDB/JDK8)
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-influxdb
       - name: Single Node Tests(MySQL/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-mysql
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-mysql
       - name: Single Node Tests(JDK9)
-        run: export E2E_VERSION=jdk9-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk9-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK11)
-        run: export E2E_VERSION=jdk11-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk11-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK12)
-        run: export E2E_VERSION=jdk12-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk12-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Agent Reboot Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-agent-reboot
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-agent-reboot
 
-  Cluster:
+  Cluster_N_TTL:
 
 Review comment:
   Yes. I will revert it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng merged pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng merged pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239
 
 
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-583730492
 
 
   @kezhenxu94 I don't want the e2e use submodule actually. Setting static proto files is good for compatible check and easy to check. I think we talked about this before, right?
   
   For test reactor, it is not a block. It is test optimization.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581206795
 
 
   Is this PR ready to review?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375128898
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success.
+     *
+     * @param measurement
+     * @param timeBucket
+     * @throws IOException
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
+        QueryResult result = getInflux().query(query);
+
+        if (result.hasError()) {
+            throw new IOException("Statement: " + query.getCommand() + ", ErrorMsg: " + result.getError());
+        }
+    }
+
+    /**
+     * Data management, to delete data by a statement. If an exception isn't thrown, it means execution success.
+     *
+     * @param statement
+     * @throws IOException
+     */
+    public void queryForDelete(String statement) throws IOException {
 
 Review comment:
   Yes, I will remove

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374597179
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,131 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
 Review comment:
   I don't really recommend to use this kind of codes to get the current class just for logging, `java.lang.invoke.MethodHandles#lookup` creates new object for every call, which I don't think is necessary, and if you are trying to avoid "copy-and-paste" mistakes, why not just use the `@Slf4j` annotation of Lombok, WDYT @wu-sheng 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **increase** coverage by `0.07%`.
   > The diff coverage is `10.09%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.36%   25.43%   +0.07%     
   ==========================================
     Files        1201     1203       +2     
     Lines       27753    27833      +80     
     Branches     3825     3837      +12     
   ==========================================
   + Hits         7040     7080      +40     
   - Misses      20079    20115      +36     
   - Partials      634      638       +4
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...rver/storage/plugin/influxdb/query/TraceQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9UcmFjZVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [...rver/storage/plugin/influxdb/query/AlarmQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9BbGFybVF1ZXJ5LmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...torage/plugin/influxdb/query/ProfileTaskQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | |
   | [...lasticsearch/StorageModuleElasticsearchConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hDb25maWcuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | ... and [13 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...3bef73a](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374611296
 
 

 ##########
 File path: test/e2e/e2e-influxdb/src/main/java/org/apache/skywalking/e2e/sample/client/SampleClientApplication.java
 ##########
 @@ -0,0 +1,34 @@
+/*
+ * 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.e2e.sample.client;
+
+import org.springframework.boot.SpringApplication;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.data.jpa.repository.config.EnableJpaRepositories;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   remove the `@author`s please

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-584171718
 
 
   #4335 is going to be merged today. There would be the last two new storage methods introduced.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-588101582
 
 
   We are waiting for nearly 24 hours, we will merge soon. Anyone has suggestions feels free to submit a separated PR

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r368274574
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/InfluxStorageDAO.java
 ##########
 @@ -53,6 +53,6 @@ public IRecordDAO newRecordDao(StorageBuilder<Record> storageBuilder) {
 
     @Override
     public INoneStreamDAO newNoneStreamDao(StorageBuilder<NoneStream> storageBuilder) {
-        return new NoneStreamDAO();
+        return new NoneStreamDAO(influxClient, storageBuilder);
 
 Review comment:
   There would be much data, just easy to query should be enough. Anyway, based on your decision. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379828284
 
 

 ##########
 File path: docker/oap/docker-entrypoint.sh
 ##########
 @@ -150,6 +150,37 @@ storage:
 EOT
 }
 
+generateStorageInfluxDB() {
+    cat <<EOT >> ${var_application_file}
+storage:
+  influx:
+    # Metadata storage provider configuration
+    metabaseType: ${SW_STORAGE_METABASE_TYPE:H2} # There are 2 options as Metabase provider, H2 or MySQL.
+    h2Props:
+      dataSourceClassName: ${SW_STORAGE_METABASE_DRIVER:org.h2.jdbcx.JdbcDataSource}
+      dataSource.url: ${SW_STORAGE_METABASE_URL:jdbc:h2:mem:skywalking-oap-db}
+      dataSource.user: ${SW_STORAGE_METABASE_USER:sa}
+      dataSource.password: ${SW_STORAGE_METABASE_PASSWORD:}
+    mysqlProps:
+      jdbcUrl: ${SW_STORAGE_METABASE_URL:"jdbc:mysql://localhost:3306/swtest"}
+      dataSource.user: ${SW_STORAGE_METABASE_USER:root}
+      dataSource.password: ${SW_STORAGE_METABASE_PASSWORD:root@1234}
+      dataSource.cachePrepStmts: ${SW_STORAGE_METABASE_CACHE_PREP_STMTS:true}
+      dataSource.prepStmtCacheSize: ${SW_STORAGE_METABASE_PREP_STMT_CACHE_SQL_SIZE:250}
+      dataSource.prepStmtCacheSqlLimit: ${SW_STORAGE_METABASE_PREP_STMT_CACHE_SQL_LIMIT:2048}
+      dataSource.useServerPrepStmts: ${SW_STORAGE_METABASE_USE_SERVER_PREP_STMTS:true}
+    metadataQueryMaxSize: ${SW_STORAGE_METABASE_QUERY_MAX_SIZE:5000}
+    # InfluxDB configuration
+    url: ${SW_STORAGE_INFLUXDB_URL:http://localhost:8086}
+    user: ${SW_STORAGE_INFLUXDB_USER:root}
+    password: ${SW_STORAGE_INFLUXDB_PASSWORD:}
+    database: ${SW_STORAGE_INFLUXDB_DATABASE:skywalking}
+    actions: ${SW_STORAGE_INFLUXDB_ACTIONS:1000} # the number of actions to collect
+    duration: ${SW_STORAGE_INFLUXDB_DURATION:1000} # the time to wait at most (milliseconds)
+    fetchTaskLogMaxSize: ${SW_STORAGE_INFLUXDB_FETCH_TASK_LOG_MAX_SIZE:5000} # the max number of fetch task log in a request
 
 Review comment:
   Should escape the `$` character
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] JaredTan95 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
JaredTan95 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-574931015
 
 
   Powerful storage plugin~

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380054244
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/installer/MetaTableDefine.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.storage.plugin.influxdb.installer;
+
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.jdbc.TableMetaInfo;
+
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.ENDPOINT_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.NETWORK_ADDRESS;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.PROFILE_TASK_SEGMENT_SNAPSHOT;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INSTANCE_INVENTORY;
+import static org.apache.skywalking.oap.server.core.source.DefaultScopeDefine.SERVICE_INVENTORY;
+
+/**
+ * Here defines which table is stored in metadata database(H2/MySQL).
+ */
+public class MetaTableDefine {
+
+    /**
+     * Test a {@link Model} is stored in H2/MySQL or not.
+     *
+     * @param model Model
+     * @return true if the {@link Model} is stored in H2/MySQL
+     */
+    public static boolean contains(Model model) {
+        switch (model.getScopeId()) {
+            case SERVICE_INVENTORY:
+            case SERVICE_INSTANCE_INVENTORY:
+            case NETWORK_ADDRESS:
+            case ENDPOINT_INVENTORY:
+            case PROFILE_TASK:
 
 Review comment:
   What is the principle? Could you add some comments and guidelines about this? 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/c148bfb6bd5ed585e39a1e1bdc2dfe061fd511f8?src=pr&el=desc) will **decrease** coverage by `0.13%`.
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   26.25%   26.11%   -0.14%     
   ==========================================
     Files        1194     1194              
     Lines       26399    26403       +4     
     Branches     3766     3766              
   ==========================================
   - Hits         6931     6896      -35     
   - Misses      18855    18895      +40     
   + Partials      613      612       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../server/storage/plugin/influxdb/base/BatchDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL0JhdGNoREFPLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...age/plugin/influxdb/query/ProfileTaskLogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Qcm9maWxlVGFza0xvZ1F1ZXJ5LmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...apm/agent/core/remote/GRPCStreamServiceStatus.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENTdHJlYW1TZXJ2aWNlU3RhdHVzLmphdmE=) | `29.16% <0%> (-25.01%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `63.93% <0%> (-16.4%)` | :arrow_down: |
   | [.../core/remote/ServiceAndEndpointRegisterClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1NlcnZpY2VBbmRFbmRwb2ludFJlZ2lzdGVyQ2xpZW50LmphdmE=) | `35.95% <0%> (-13.49%)` | :arrow_down: |
   | [.../skywalking/apm/agent/core/remote/GRPCChannel.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENDaGFubmVsLmphdmE=) | `81.25% <0%> (-6.25%)` | :arrow_down: |
   | [.../agent/core/context/trace/AbstractTracingSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9BYnN0cmFjdFRyYWNpbmdTcGFuLmphdmE=) | `61.06% <0%> (-3.54%)` | :arrow_down: |
   | [...king/apm/agent/core/remote/GRPCChannelManager.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENDaGFubmVsTWFuYWdlci5qYXZh) | `69.23% <0%> (-1.29%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [c148bfb...4cd472e](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@55f57e5`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `61.76%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff            @@
   ##             master   #4239   +/-   ##
   ========================================
     Coverage          ?   26.4%           
   ========================================
     Files             ?    1193           
     Lines             ?   26320           
     Branches          ?    3748           
   ========================================
     Hits              ?    6949           
     Misses            ?   18766           
     Partials          ?     605
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...alking/apm/agent/core/context/trace/LocalSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Mb2NhbFNwYW4uamF2YQ==) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/NoopSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Ob29wU3Bhbi5qYXZh) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/ExitSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FeGl0U3Bhbi5qYXZh) | `53.57% <ø> (ø)` | |
   | [.../customize/interceptor/BaseInterceptorMethods.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy9jdXN0b21pemUtZW5oYW5jZS1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL3BsdWdpbi9jdXN0b21pemUvaW50ZXJjZXB0b3IvQmFzZUludGVyY2VwdG9yTWV0aG9kcy5qYXZh) | `0% <0%> (ø)` | |
   | [...walking/apm/plugin/elasticsearch/v5/Constants.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZWxhc3RpY3NlYXJjaC01LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vZWxhc3RpY3NlYXJjaC92NS9Db25zdGFudHMuamF2YQ==) | `0% <0%> (ø)` | |
   | [...vation/opentracing/span/SpanSetTagInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC1vcGVudHJhY2luZy1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vb3BlbnRyYWNpbmcvc3Bhbi9TcGFuU2V0VGFnSW50ZXJjZXB0b3IuamF2YQ==) | `0% <0%> (ø)` | |
   | [...y/socketio/NettySocketIOConnectionInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vbmV0dHktc29ja2V0aW8tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vbmV0dHkvc29ja2V0aW8vTmV0dHlTb2NrZXRJT0Nvbm5lY3Rpb25JbnRlcmNlcHRvci5qYXZh) | `76.92% <100%> (ø)` | |
   | [...ng/span/ConstructorWithSpanBuilderInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC1vcGVudHJhY2luZy1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vb3BlbnRyYWNpbmcvc3Bhbi9Db25zdHJ1Y3RvcldpdGhTcGFuQnVpbGRlckludGVyY2VwdG9yLmphdmE=) | `86.66% <100%> (ø)` | |
   | [...lking/apm/plugin/gson/GsonFromJsonInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy9nc29uLTIuOC54LXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9hcG0vcGx1Z2luL2dzb24vR3NvbkZyb21Kc29uSW50ZXJjZXB0b3IuamF2YQ==) | `77.77% <100%> (ø)` | |
   | [...lking/apm/plugin/esjob/JobExecutorInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vZWxhc3RpYy1qb2ItMi54LXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9hcG0vcGx1Z2luL2Vzam9iL0pvYkV4ZWN1dG9ySW50ZXJjZXB0b3IuamF2YQ==) | `100% <100%> (ø)` | |
   | ... and [6 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [55f57e5...748dce8](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375121308
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+
+@Slf4j
+public class BatchDAO implements IBatchDAO {
+    private final DataCarrier<PrepareRequest> dataCarrier;
+    private final InfluxClient client;
+
+    public BatchDAO(InfluxClient client) {
+        this.client = client;
+
+        String name = "INFLUX_ASYNC_BATCH_PERSISTENT";
+        BulkConsumePool.Creator creator = new BulkConsumePool.Creator(name, 1, 20L);
+
+        try {
+            ConsumerPoolFactory.INSTANCE.createIfAbsent(name, creator);
+        } catch (Exception e) {
+            throw new UnexpectedException(e.getMessage(), e);
+        }
+
+        this.dataCarrier = new DataCarrier(1, 10000);
+        this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(name), new InfluxBatchConsumer(this));
+    }
+
+    @Override
+    public void asynchronous(InsertRequest insertRequest) {
+        dataCarrier.produce(insertRequest);
+    }
+
+    @Override
+    public void synchronous(List<PrepareRequest> prepareRequests) {
+        if (CollectionUtils.isEmpty(prepareRequests)) {
+            return;
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("batch sql statements execute, data size: {}", prepareRequests.size());
+        }
+
+        final BatchPoints.Builder builder = BatchPoints.builder();
+        prepareRequests.forEach(e -> {
+            builder.point(((InfluxInsertRequest)e).getPoint());
+        });
+
+        client.write(builder.build());
 
 Review comment:
   Is this a blocking write? Meaning, the data is queryable when this method finished.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/918673d7bec7906c4fef7f304cc86eca1534df99?src=pr&el=desc) will **increase** coverage by `0.27%`.
   > The diff coverage is `52.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   25.38%   25.65%   +0.27%     
   ==========================================
     Files        1196     1197       +1     
     Lines       27474    27518      +44     
     Branches     3787     3802      +15     
   ==========================================
   + Hits         6974     7061      +87     
   + Misses      19884    19833      -51     
   - Partials      616      624       +8
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...lking/apm/agent/core/util/CustomizeExpression.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvdXRpbC9DdXN0b21pemVFeHByZXNzaW9uLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...vation/trace/TraceAnnotationMethodInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC10cmFjZS1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vdHJhY2UvVHJhY2VBbm5vdGF0aW9uTWV0aG9kSW50ZXJjZXB0b3IuamF2YQ==) | `62.5% <61.11%> (+12.5%)` | :arrow_up: |
   | [...tivation/trace/TagAnnotationMethodInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC10cmFjZS1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vdHJhY2UvVGFnQW5ub3RhdGlvbk1ldGhvZEludGVyY2VwdG9yLmphdmE=) | `68.96% <81.25%> (+68.96%)` | :arrow_up: |
   | [...kywalking/apm/toolkit/activation/util/TagUtil.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXRvb2xraXQtYWN0aXZhdGlvbi9hcG0tdG9vbGtpdC10cmFjZS1hY3RpdmF0aW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS90b29sa2l0L2FjdGl2YXRpb24vdXRpbC9UYWdVdGlsLmphdmE=) | `83.33% <83.33%> (ø)` | |
   | [...king/apm/agent/core/remote/GRPCChannelManager.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENDaGFubmVsTWFuYWdlci5qYXZh) | `70.51% <0%> (+1.28%)` | :arrow_up: |
   | [.../agent/core/context/trace/AbstractTracingSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9BYnN0cmFjdFRyYWNpbmdTcGFuLmphdmE=) | `64.28% <0%> (+3.57%)` | :arrow_up: |
   | [.../skywalking/apm/agent/core/remote/GRPCChannel.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENDaGFubmVsLmphdmE=) | `87.5% <0%> (+6.25%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `45.16% <0%> (+10.75%)` | :arrow_up: |
   | ... and [4 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [918673d...767c85a](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376259790
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   Sorry, I make a mistake. This method works for `single statement`. `InfluxClient#query()` is for `multi-statements`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r367179454
 
 

 ##########
 File path: dist-material/application.yml
 ##########
 @@ -120,11 +120,11 @@ storage:
 #    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
 #    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
 #    profileTaskQueryMaxSize: ${SW_STORAGE_ES_QUERY_PROFILE_TASK_SIZE:200}
-  h2:
-    driver: ${SW_STORAGE_H2_DRIVER:org.h2.jdbcx.JdbcDataSource}
-    url: ${SW_STORAGE_H2_URL:jdbc:h2:mem:skywalking-oap-db}
-    user: ${SW_STORAGE_H2_USER:sa}
-    metadataQueryMaxSize: ${SW_STORAGE_H2_QUERY_MAX_SIZE:5000}
+#  h2:
+#    driver: ${SW_STORAGE_H2_DRIVER:org.h2.jdbcx.JdbcDataSource}
+#    url: ${SW_STORAGE_H2_URL:jdbc:h2:mem:skywalking-oap-db}
+#    user: ${SW_STORAGE_H2_USER:sa}
+#    metadataQueryMaxSize: ${SW_STORAGE_H2_QUERY_MAX_SIZE:5000}
 
 Review comment:
   This should not be changed in PR.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374653664
 
 

 ##########
 File path: oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/metrics/Metrics.java
 ##########
 @@ -32,8 +35,13 @@
     public static final String TIME_BUCKET = "time_bucket";
     public static final String ENTITY_ID = "entity_id";
 
-    @Getter @Setter @Column(columnName = TIME_BUCKET) private long timeBucket;
-    @Getter @Setter private long survivalTime = 0L;
+    @Getter
+    @Setter
+    @Column(columnName = TIME_BUCKET)
+    private long timeBucket;
+    @Getter
+    @Setter
+    private long survivalTime = 0L;
 
 Review comment:
   This class should not change and not relate to this PR, please revert.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4b02679`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff            @@
   ##             master   #4239   +/-   ##
   ========================================
     Coverage          ?   26.1%           
   ========================================
     Files             ?    1189           
     Lines             ?   26208           
     Branches          ?    3729           
   ========================================
     Hits              ?    6842           
     Misses            ?   18767           
     Partials          ?     599
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <ø> (ø)` | |
   | [...er/storage/plugin/influxdb/query/MetricsQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9NZXRyaWNzUXVlcnkuamF2YQ==) | `0% <0%> (ø)` | |
   | [...erver/storage/plugin/influxdb/base/MetricsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL01ldHJpY3NEQU8uamF2YQ==) | `0% <0%> (ø)` | |
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ver/storage/plugin/influxdb/base/PointBuilder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL1BvaW50QnVpbGRlci5qYXZh) | `0% <0%> (ø)` | |
   | [...storage/plugin/influxdb/InfluxStorageProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlUHJvdmlkZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...server/storage/plugin/influxdb/query/LogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Mb2dRdWVyeS5qYXZh) | `0% <0%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4b02679...9fdc886](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-583796531
 
 
   Profile e2e is failing, please recheck.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-582234659
 
 
   > The corresponding `yml` in `docker-entrypoint.sh` should be also updated
   
   Sorry, I don't understand. We need to update which file? @kezhenxu94 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379867032
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   1. Resolved
   2. The question is more about why you sort them in a different order without taking care of the given `orderBy`

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581039390
 
 
   > How can I confirm that the submodule is the newest? If it is not the newest, what should I do?
   
   Check the commit IDs of submodules across checking with the other repos' commit ID.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/a4e13a6f18019cb17f0092a696dcfc3631a15565?src=pr&el=desc) will **increase** coverage by `0.1%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##           master    #4239     +/-   ##
   =========================================
   + Coverage   25.35%   25.45%   +0.1%     
   =========================================
     Files        1201     1201             
     Lines       27785    27785             
     Branches     3829     3829             
   =========================================
   + Hits         7046     7074     +28     
   + Misses      20102    20076     -26     
   + Partials      637      635      -2
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../agent/core/context/trace/AbstractTracingSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9BYnN0cmFjdFRyYWNpbmdTcGFuLmphdmE=) | `61.6% <0%> (+0.89%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `27.95% <0%> (+1.07%)` | :arrow_up: |
   | [...king/apm/agent/core/remote/GRPCChannelManager.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENDaGFubmVsTWFuYWdlci5qYXZh) | `69.23% <0%> (+1.28%)` | :arrow_up: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `79.45% <0%> (+1.36%)` | :arrow_up: |
   | [...alking/apm/agent/core/remote/AgentIDDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0FnZW50SUREZWNvcmF0b3IuamF2YQ==) | `85.18% <0%> (+18.51%)` | :arrow_up: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `80% <0%> (+20%)` | :arrow_up: |
   | [...apm/agent/core/remote/GRPCStreamServiceStatus.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL0dSUENTdHJlYW1TZXJ2aWNlU3RhdHVzLmphdmE=) | `82.35% <0%> (+41.17%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [a4e13a6...bc127d7](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375131166
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   Do we use multiple statements? I think `#get(0)` should be included `#query`(return series). If we have multiple stat case, we should have a different API.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581879651
 
 
   Thanks for your review and reminder, @wu-sheng and @kezhenxu94 
   I will fix them later.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381083645
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/HistoryDeleteDAO.java
 ##########
 @@ -0,0 +1,71 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.io.IOException;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.CoreModule;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.config.ConfigService;
+import org.apache.skywalking.oap.server.core.storage.IHistoryDeleteDAO;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.core.storage.ttl.StorageTTL;
+import org.apache.skywalking.oap.server.core.storage.ttl.TTLCalculator;
+import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.joda.time.DateTime;
+
+@Slf4j
+public class HistoryDeleteDAO implements IHistoryDeleteDAO {
+    private final ModuleDefineHolder moduleDefineHolder;
+    private final InfluxClient client;
+    private final StorageTTL storageTTL;
+
+    public HistoryDeleteDAO(ModuleDefineHolder moduleDefineHolder, InfluxClient client, StorageTTL storageTTL) {
+        this.moduleDefineHolder = moduleDefineHolder;
+        this.storageTTL = storageTTL;
+        this.client = client;
+    }
+
+    @Override
+    public void deleteHistory(Model model, String timeBucketColumnName) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("TTL execution log, model: {}", model.getName());
+        }
+        try {
+            ConfigService configService = moduleDefineHolder.find(CoreModule.NAME)
+                                                            .provider()
+                                                            .getService(ConfigService.class);
+
+            TTLCalculator ttlCalculator;
+            if (model.isRecord()) {
+                ttlCalculator = storageTTL.recordCalculator();
+            } else {
+                ttlCalculator = storageTTL.metricsCalculator(model.getDownsampling());
+            }
+
+            client.deleteByQuery(
+                model.getName(),
+                TimeBucket.getTimestamp(ttlCalculator.timeBefore(DateTime.now(), configService.getDataTTLConfig()) + 1)
 
 Review comment:
   +1 can be deleted? delete is <

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
aderm commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381148243
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/NoneStreamDAO.java
 ##########
 @@ -0,0 +1,57 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeInteger;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.config.NoneStream;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord;
+import org.apache.skywalking.oap.server.core.storage.INoneStreamDAO;
+import org.apache.skywalking.oap.server.core.storage.StorageBuilder;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.Point;
+
+public class NoneStreamDAO implements INoneStreamDAO {
+    public static final String TAG_SERVICE_ID = "_service_id";
+    private static final int PADDING_SIZE = 1_000_000;
+    private static final AtomicRangeInteger SUFFIX = new AtomicRangeInteger(0, PADDING_SIZE);
+
+    private InfluxClient client;
+    private StorageBuilder<NoneStream> storageBuilder;
+
+    public NoneStreamDAO(InfluxClient client, StorageBuilder<NoneStream> storageBuilder) {
+        this.client = client;
+        this.storageBuilder = storageBuilder;
+    }
+
+    @Override
+    public void insert(final Model model, final NoneStream noneStream) throws IOException {
+        final long timestamp = TimeBucket.getTimestamp(
+            noneStream.getTimeBucket(), model.getDownsampling()) * PADDING_SIZE + SUFFIX.getAndIncrement();
+
+        Point point = new InfluxInsertRequest(model, noneStream, storageBuilder)
+            .time(timestamp, TimeUnit.NANOSECONDS)
 
 Review comment:
   why this have Nanosecond? or millisecond?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r381126590
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,191 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ * InfluxDB connection maintainer, provides base data write/query API.
+ */
+@Slf4j
+public class InfluxClient implements Client {
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag of time_bucket.
+     */
+    public static final String TAG_TIME_BUCKET = "_time_bucket";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+                                         new OkHttpClient.Builder().readTimeout(3, TimeUnit.MINUTES)
+                                                                   .writeTimeout(3, TimeUnit.MINUTES),
+                                         InfluxDB.ResponseFormat.MSGPACK
+        );
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.enableBatch(config.getActions(), config.getDuration(), TimeUnit.MILLISECONDS);
+        influx.setDatabase(database);
+    }
+
+    /**
+     * To get a connection of InfluxDB.
+     *
+     * @return InfluxDB's connection
+     */
+    private InfluxDB getInflux() {
+        return influx;
 
 Review comment:
   I don't think so. OAP crashes and exits, when Influx initializes failed. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379902233
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -71,15 +73,19 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Cluster Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=elasticsearch
+      - name: Cluster Tests (InfluxDB/ZK/JDK8)
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=influxdb
       - name: Cluster With Gateway Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
 
 Review comment:
   > Why this doesn't have `--storage=elasticsearch`? What is the default? Or what happens if no `--storage=`?
   
   Not every e2e case is aware of the parameter `--storage`, `e2e-cluster-with-gateway` is one of them, the parameter is introduced recently and only `e2e-cluster` and `e2e-profile` are refactored then, IIRC
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378014958
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
 
 Review comment:
   Got it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] JaredTan95 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
JaredTan95 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575453526
 
 
   > The first level is a relationship database stored something likes inventory, which has transactions and updates. Stored in MySQL.(Now, I make H2 as relationship database. Because it is easy for test.)
   
   BTW, also supported `TiDB`. :)

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375190983
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   Yes.
   In `ITraceQueryDAO#queryBasicTraces(...)`, we have 2 statements in a request. The one gets the number of traces, and another recalls all trace data.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@0133890`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `15.6%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@           Coverage Diff            @@
   ##             master   #4239   +/-   ##
   ========================================
     Coverage          ?   26.1%           
   ========================================
     Files             ?    1178           
     Lines             ?   26101           
     Branches          ?    3724           
   ========================================
     Hits              ?    6814           
     Misses            ?   18682           
     Partials          ?     605
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ing/oap/server/core/source/DefaultScopeDefine.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvc291cmNlL0RlZmF1bHRTY29wZURlZmluZS5qYXZh) | `0% <ø> (ø)` | |
   | [...erver/receiver/jvm/provider/JVMModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctanZtLXJlY2VpdmVyLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3JlY2VpdmVyL2p2bS9wcm92aWRlci9KVk1Nb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/EntrySpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FbnRyeVNwYW4uamF2YQ==) | `54.83% <ø> (ø)` | |
   | [...lking/oap/server/core/query/TraceQueryService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvVHJhY2VRdWVyeVNlcnZpY2UuamF2YQ==) | `0% <ø> (ø)` | |
   | [...walking/apm/agent/core/context/trace/ExitSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9FeGl0U3Bhbi5qYXZh) | `53.57% <ø> (ø)` | |
   | [.../provider/parser/SegmentParserListenerManager.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctdHJhY2UtcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvdHJhY2UvcHJvdmlkZXIvcGFyc2VyL1NlZ21lbnRQYXJzZXJMaXN0ZW5lck1hbmFnZXIuamF2YQ==) | `0% <ø> (ø)` | |
   | [...alking/apm/agent/core/context/trace/LocalSpan.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29udGV4dC90cmFjZS9Mb2NhbFNwYW4uamF2YQ==) | `0% <ø> (ø)` | |
   | [...king/oap/server/library/module/ModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItbGlicmFyeS9saWJyYXJ5LW1vZHVsZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2xpYnJhcnkvbW9kdWxlL01vZHVsZVByb3ZpZGVyLmphdmE=) | `50% <ø> (ø)` | |
   | [...iver/register/provider/RegisterModuleProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctcmVnaXN0ZXItcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvcmVnaXN0ZXIvcHJvdmlkZXIvUmVnaXN0ZXJNb2R1bGVQcm92aWRlci5qYXZh) | `0% <ø> (ø)` | |
   | [.../apm/agent/core/profile/TracingThreadSnapshot.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9UcmFjaW5nVGhyZWFkU25hcHNob3QuamF2YQ==) | `0% <0%> (ø)` | |
   | ... and [33 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [0133890...a879e22](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r380045349
 
 

 ##########
 File path: test/e2e/e2e-ttl/e2e-ttl-influxdb/src/test/java/org/apache/skywalking/e2e/StorageTTLITCase.java
 ##########
 @@ -0,0 +1,285 @@
+/*
+ * 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.e2e;
+
+import io.grpc.ManagedChannel;
+import io.grpc.ManagedChannelBuilder;
+import io.grpc.internal.DnsNameResolverProvider;
+import io.grpc.netty.NettyChannelBuilder;
+import io.grpc.stub.StreamObserver;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.network.common.DetectPoint;
+import org.apache.skywalking.apm.network.servicemesh.MeshProbeDownstream;
+import org.apache.skywalking.apm.network.servicemesh.Protocol;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetric;
+import org.apache.skywalking.apm.network.servicemesh.ServiceMeshMetricServiceGrpc;
+import org.apache.skywalking.e2e.metrics.AllOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.AtLeastOneOfMetricsMatcher;
+import org.apache.skywalking.e2e.metrics.Metrics;
+import org.apache.skywalking.e2e.metrics.MetricsQuery;
+import org.apache.skywalking.e2e.metrics.MetricsValueMatcher;
+import org.apache.skywalking.e2e.service.Service;
+import org.apache.skywalking.e2e.service.ServicesQuery;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.apache.skywalking.e2e.metrics.MetricsQuery.SERVICE_RESP_TIME;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * @author kezhenxu94
+ */
 
 Review comment:
   remove this

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379900290
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -71,15 +73,19 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Cluster Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=elasticsearch
+      - name: Cluster Tests (InfluxDB/ZK/JDK8)
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=influxdb
       - name: Cluster With Gateway Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
 
 Review comment:
   Why this doesn't have `--storage=elasticsearch`? What is the default? Or what happens if no `--storage=`?
   
   I expect a fail when missing this parameter.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375122025
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+
+@Slf4j
+public class BatchDAO implements IBatchDAO {
+    private final DataCarrier<PrepareRequest> dataCarrier;
+    private final InfluxClient client;
+
+    public BatchDAO(InfluxClient client) {
+        this.client = client;
+
+        String name = "INFLUX_ASYNC_BATCH_PERSISTENT";
+        BulkConsumePool.Creator creator = new BulkConsumePool.Creator(name, 1, 20L);
+
+        try {
+            ConsumerPoolFactory.INSTANCE.createIfAbsent(name, creator);
+        } catch (Exception e) {
+            throw new UnexpectedException(e.getMessage(), e);
+        }
+
+        this.dataCarrier = new DataCarrier(1, 10000);
+        this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(name), new InfluxBatchConsumer(this));
+    }
+
+    @Override
+    public void asynchronous(InsertRequest insertRequest) {
+        dataCarrier.produce(insertRequest);
 
 Review comment:
   Doesn't InfluxDB have an async write mode? I remember time serious DB is good at writing. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r367179675
 
 

 ##########
 File path: dist-material/application.yml
 ##########
 @@ -135,6 +135,16 @@ storage:
 #      dataSource.prepStmtCacheSqlLimit: ${SW_DATA_SOURCE_PREP_STMT_CACHE_SQL_LIMIT:2048}
 #      dataSource.useServerPrepStmts: ${SW_DATA_SOURCE_USE_SERVER_PREP_STMTS:true}
 #    metadataQueryMaxSize: ${SW_STORAGE_MYSQL_QUERY_MAX_SIZE:5000}
+  influx:
+    myDriver: ${SW_STORAGE_MY_DRIVER:org.h2.jdbcx.JdbcDataSource}
 
 Review comment:
   You should support to set metadata DB type. Otherwise, you have to link to single one DB implementation.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374739055
 
 

 ##########
 File path: tools/dependencies/known-oap-backend-dependencies-es7.txt
 ##########
 @@ -159,3 +159,8 @@ sundr-core-0.9.2.jar
 swagger-annotations-1.5.12.jar
 t-digest-3.2.jar
 zookeeper-3.4.10.jar
+converter-moshi-2.5.0.jar
+influxdb-java-2.15.jar
+logging-interceptor-3.13.1.jar
+moshi-1.5.0.jar
+msgpack-core-0.8.16.jar
 
 Review comment:
   Confused what parts? All new dependency should be added into the LICENSE, like you do for `influxdb-java`. You need to find the licenses for all other jars.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-582253156
 
 
   `docker-entrypoint.sh` files

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581039128
 
 
   I don't mean github action.  I am trying on my local branch which I have implemented.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375118648
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
 
 Review comment:
   What does this `#get(0)` represent?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581038623
 
 
   > How can I confirm that the submodule is the newest? If it is not the newest, what should I do?
   > I found my e2e always failed in my devel branch.
   > 
   > @wu-sheng , @kezhenxu94
   
   I don't think it's a matter of the sub-modules, the `interface IMetricsQueryDAO` is recently modified and you did not modify your `MetricsQuery` corespondingly

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
kezhenxu94 commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379867511
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
+        return traceBrief;
+    }
+
+    @Override
+    public List<SegmentRecord> queryByTraceId(String traceId) throws IOException {
+        WhereQueryImpl query = select().column(SegmentRecord.SEGMENT_ID)
 
 Review comment:
   > Could you explain more? `WhereQueryImpl` is harmless and easy to append clauses.
   
   - Program to `interface` not `Impl`
   - `WhereQueryImpl` is generic-typed, but you simply ignored them, causing many warnings
   
   Anyway, just ignore if you don’t care the warnings

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581094420
 
 
   Dependency changed, you need to update the LICENSE/NOTICE and dependency check files.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r375119872
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/InfluxClient.java
 ##########
 @@ -0,0 +1,194 @@
+/*
+ * 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.storage.plugin.influxdb;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import okhttp3.OkHttpClient;
+import org.apache.skywalking.oap.server.core.analysis.Downsampling;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.library.client.Client;
+import org.influxdb.InfluxDB;
+import org.influxdb.InfluxDBFactory;
+import org.influxdb.dto.BatchPoints;
+import org.influxdb.dto.Point;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.time.TimeInterval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.ti;
+
+/**
+ *
+ */
+public class InfluxClient implements Client {
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+    private InfluxStorageConfig config;
+    private InfluxDB influx;
+
+    /**
+     * A constant, the name of time field in Time-series database.
+     */
+    public static final String TIME = "time";
+    /**
+     * A constant, the name of tag.
+     */
+    public static final String TAG_ENTITY_ID = "entity_id";
+
+    private final String database;
+
+    public InfluxClient(InfluxStorageConfig config) {
+        this.config = config;
+        this.database = config.getDatabase();
+    }
+
+    public final String getDatabase() {
+        return database;
+    }
+
+    @Override
+    public void connect() {
+        influx = InfluxDBFactory.connect(config.getUrl(), config.getUser(), config.getPassword(),
+            new OkHttpClient.Builder(), InfluxDB.ResponseFormat.MSGPACK);
+        influx.query(new Query("CREATE DATABASE " + database));
+
+        influx.setDatabase(database);
+        influx.enableBatch();
+    }
+
+    /**
+     * To get a connection of InfluxDB
+     *
+     * @return InfluxDB's connection
+     */
+    public InfluxDB getInflux() {
+        return influx;
+    }
+
+    /**
+     * Request with a {@link Query} to InfluxDB and return a set of {@link QueryResult.Result}s.
+     *
+     * @param query
+     * @return a set of Result.
+     * @throws IOException
+     */
+    public List<QueryResult.Result> query(Query query) throws IOException {
+        if (logger.isDebugEnabled()) {
+            logger.debug("SQL Statement: {}", query.getCommand());
+        }
+
+        try {
+            QueryResult result = getInflux().query(query);
+            if (result.hasError()) {
+                throw new IOException(result.getError());
+            }
+            return result.getResults();
+        } catch (Exception e) {
+            throw new IOException(e.getMessage() + System.lineSeparator() + "SQL Statement: " + query.getCommand(), e);
+        }
+    }
+
+    /**
+     * Request with one statement to InfluxDB and return a set of {@link QueryResult.Series}s.
+     *
+     * @param query
+     * @return a set of Series
+     * @throws IOException
+     */
+    public List<QueryResult.Series> queryForSeries(Query query) throws IOException {
+        return query(query).get(0).getSeries();
+    }
+
+    /**
+     * Data management, to drop a time-series by measurement and time-series name specified. If an exception isn't
+     * thrown, it means execution success.
+     *
+     * @param measurement
+     * @param timeBucket
+     * @throws IOException
+     */
+    public void dropSeries(String measurement, long timeBucket) throws IOException {
+        Query query = new Query("DROP SERIES FROM " + measurement + " WHERE time_bucket='" + timeBucket + "'");
+        QueryResult result = getInflux().query(query);
+
+        if (result.hasError()) {
+            throw new IOException("Statement: " + query.getCommand() + ", ErrorMsg: " + result.getError());
+        }
+    }
+
+    /**
+     * Data management, to delete data by a statement. If an exception isn't thrown, it means execution success.
+     *
+     * @param statement
+     * @throws IOException
+     */
+    public void queryForDelete(String statement) throws IOException {
 
 Review comment:
   Don't see anyone uses this. Please confirm. And `queryForDelete` is not a good idea. For time-series data, it is better to delete by time bucket only.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-583730593
 
 
   I don't want this big PR blocked too long, it is last a long time, we could create an issue to track your test change proposal.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r373895500
 
 

 ##########
 File path: dist-material/release-docs/NOTICE
 ##########
 @@ -905,4 +905,31 @@ be obtained at:
   * HOMEPAGE:
     * https://github.com/catapult-project/catapult
 
+------
+
+===========================================================================
 
 Review comment:
   As MIT license you don't need to update NOTICE. NOTICE is for Apache 2.0 only. Please remove this part.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378012577
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
 
 Review comment:
   Upgrade to 1.5 please, based on, https://github.com/apache/skywalking/issues/4343#issuecomment-584990302

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379863606
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
+        return traceBrief;
+    }
+
+    @Override
+    public List<SegmentRecord> queryByTraceId(String traceId) throws IOException {
+        WhereQueryImpl query = select().column(SegmentRecord.SEGMENT_ID)
 
 Review comment:
   Could you explain more? `WhereQueryImpl` is harmless and easy to append clauses.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r368257153
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/InfluxStorageDAO.java
 ##########
 @@ -53,6 +53,6 @@ public IRecordDAO newRecordDao(StorageBuilder<Record> storageBuilder) {
 
     @Override
     public INoneStreamDAO newNoneStreamDao(StorageBuilder<NoneStream> storageBuilder) {
-        return new NoneStreamDAO();
+        return new NoneStreamDAO(influxClient, storageBuilder);
 
 Review comment:
   `INoneStreamDAO` is used for web interactive job, such as @mrproliu 's profiling task. Basically, this should have little data. Is the influxDB suitable for this?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379879952
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/ProfileThreadSnapshotQuery.java
 ##########
 @@ -0,0 +1,165 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.storage.profile.IProfileThreadSnapshotQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.WhereQueryImpl;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.contains;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+public class ProfileThreadSnapshotQuery implements IProfileThreadSnapshotQueryDAO {
+    private final InfluxClient client;
+
+    public ProfileThreadSnapshotQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public List<BasicTrace> queryProfiledSegments(String taskId) throws IOException {
+        WhereQueryImpl query = select(ProfileThreadSnapshotRecord.SEGMENT_ID)
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where()
+            .and(eq(ProfileThreadSnapshotRecord.TASK_ID, taskId))
+            .and(eq(ProfileThreadSnapshotRecord.SEQUENCE, 0));
+
+        final LinkedList<String> segments = new LinkedList<>();
+        QueryResult.Series series = client.queryForSingleSeries(query);
+        if (series == null) {
+            return Collections.emptyList();
+        }
+        series.getValues().forEach(values -> {
+            segments.add((String) values.get(1));
+        });
+
+        if (segments.isEmpty()) {
+            return Collections.emptyList();
+        }
+
+        query = select()
+            .function("bottom", SegmentRecord.START_TIME, segments.size())
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where()
+            .and(contains(SegmentRecord.SEGMENT_ID, Joiner.on("|").join(segments)));
+
+        ArrayList<BasicTrace> result = Lists.newArrayListWithCapacity(segments.size());
+        client.queryForSingleSeries(query)
+              .getValues()
+              .stream()
+              .sorted((a, b) -> Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue()))
+              .forEach(values -> {
+                  BasicTrace basicTrace = new BasicTrace();
+
+                  basicTrace.setSegmentId((String) values.get(2));
+                  basicTrace.setStart(String.valueOf(values.get(3)));
+                  basicTrace.getEndpointNames().add((String) values.get(4));
+                  basicTrace.setDuration((int) values.get(5));
+                  basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+                  String traceIds = (String) values.get(7);
+                  basicTrace.getTraceIds().add(traceIds);
+
+                  result.add(basicTrace);
+              });
+
+        return result;
+    }
+
+    @Override
+    public int queryMinSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("min", segmentId, start, end);
+    }
+
+    @Override
+    public int queryMaxSequence(String segmentId, long start, long end) throws IOException {
+        return querySequenceWithAgg("max", segmentId, start, end);
+    }
+
+    @Override
+    public List<ProfileThreadSnapshotRecord> queryRecords(String segmentId, int minSequence,
+                                                          int maxSequence) throws IOException {
+        WhereQueryImpl query = select(
+            ProfileThreadSnapshotRecord.TASK_ID,
+            ProfileThreadSnapshotRecord.SEGMENT_ID,
+            ProfileThreadSnapshotRecord.DUMP_TIME,
+            ProfileThreadSnapshotRecord.SEQUENCE,
+            ProfileThreadSnapshotRecord.STACK_BINARY
+        )
+            .from(client.getDatabase(), ProfileThreadSnapshotRecord.INDEX_NAME)
+            .where(eq(ProfileThreadSnapshotRecord.SEGMENT_ID, segmentId))
+            .and(gte(ProfileThreadSnapshotRecord.SEQUENCE, minSequence))
+            .and(lte(ProfileThreadSnapshotRecord.SEQUENCE, maxSequence));
+
+        ArrayList<ProfileThreadSnapshotRecord> result = new ArrayList<>(maxSequence - minSequence);
+        client.queryForSingleSeries(query).getValues().forEach(values -> {
+            ProfileThreadSnapshotRecord record = new ProfileThreadSnapshotRecord();
+
+            record.setTaskId((String) values.get(1));
+            record.setSegmentId((String) values.get(2));
+            record.setDumpTime(((Number) values.get(3)).longValue());
+            record.setSequence((int) values.get(4));
+            String dataBinaryBase64 = String.valueOf(values.get(5));
+            if (StringUtil.isNotEmpty(dataBinaryBase64)) {
+                record.setStackBinary(Base64.getDecoder().decode(dataBinaryBase64));
 
 Review comment:
   This class is included in skywaking-agent-core module. We don't import it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] JaredTan95 commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
JaredTan95 commented on a change in pull request #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r367760135
 
 

 ##########
 File path: oap-server/server-bootstrap/src/main/resources/application.yml
 ##########
 @@ -98,28 +98,37 @@ storage:
 #    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
 #    profileTaskQueryMaxSize: ${SW_STORAGE_ES_QUERY_PROFILE_TASK_SIZE:200}
 #    advanced: ${SW_STORAGE_ES_ADVANCED:""}
-  elasticsearch7:
-    nameSpace: ${SW_NAMESPACE:""}
-    clusterNodes: ${SW_STORAGE_ES_CLUSTER_NODES:localhost:9200}
-    protocol: ${SW_STORAGE_ES_HTTP_PROTOCOL:"http"}
-    #trustStorePath: ${SW_SW_STORAGE_ES_SSL_JKS_PATH:"../es_keystore.jks"}
-    #trustStorePass: ${SW_SW_STORAGE_ES_SSL_JKS_PASS:""}
-    user: ${SW_ES_USER:""}
-    password: ${SW_ES_PASSWORD:""}
-    indexShardsNumber: ${SW_STORAGE_ES_INDEX_SHARDS_NUMBER:2}
-    indexReplicasNumber: ${SW_STORAGE_ES_INDEX_REPLICAS_NUMBER:0}
-    # Those data TTL settings will override the same settings in core module.
-    recordDataTTL: ${SW_STORAGE_ES_RECORD_DATA_TTL:7} # Unit is day
-    otherMetricsDataTTL: ${SW_STORAGE_ES_OTHER_METRIC_DATA_TTL:45} # Unit is day
-    monthMetricsDataTTL: ${SW_STORAGE_ES_MONTH_METRIC_DATA_TTL:18} # Unit is month
-    # Batch process setting, refer to https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.5/java-docs-bulk-processor.html
-    bulkActions: ${SW_STORAGE_ES_BULK_ACTIONS:1000} # Execute the bulk every 1000 requests
-    flushInterval: ${SW_STORAGE_ES_FLUSH_INTERVAL:10} # flush the bulk every 10 seconds whatever the number of requests
-    concurrentRequests: ${SW_STORAGE_ES_CONCURRENT_REQUESTS:2} # the number of concurrent requests
-    resultWindowMaxSize: ${SW_STORAGE_ES_QUERY_MAX_WINDOW_SIZE:10000}
-    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
-    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
-    advanced: ${SW_STORAGE_ES_ADVANCED:""}
+  influx:
+    myUrl: ${SW_STORAGE_MY_URL:org.h2.jdbcx.JdbcDataSource}
 
 Review comment:
   ```
       myUrl: ${SW_STORAGE_MY_URL:org.h2.jdbcx.JdbcDataSource}
       myDriver: ${SW_STORAGE_MY_DRIVER:jdbc:h2:mem:skywalking-oap-db}
   ```
   ---> 
   
   ```
       myUrl: ${SW_STORAGE_MY_URL:jdbc:h2:mem:skywalking-oap-db}
       myDriver: ${SW_STORAGE_MY_DRIVER:org.h2.jdbcx.JdbcDataSource}
   ```

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/12414c80e1819ccdd6d774bede6d9d33fe6af457?src=pr&el=desc) will **increase** coverage by `0.02%`.
   > The diff coverage is `93.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   + Coverage   26.24%   26.27%   +0.02%     
   ==========================================
     Files        1194     1195       +1     
     Lines       26404    26409       +5     
     Branches     3767     3768       +1     
   ==========================================
   + Hits         6931     6939       +8     
   + Misses      18860    18858       -2     
   + Partials      613      612       -1
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [.../core/profile/analyze/ProfileAnalyzeCollector.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVBbmFseXplQ29sbGVjdG9yLmphdmE=) | `100% <ø> (ø)` | :arrow_up: |
   | [...lking/oap/query/graphql/resolver/ProfileQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcXVlcnktcGx1Z2luL3F1ZXJ5LWdyYXBocWwtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9xdWVyeS9ncmFwaHFsL3Jlc29sdmVyL1Byb2ZpbGVRdWVyeS5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...p/server/core/profile/analyze/ProfileAnalyzer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVBbmFseXplci5qYXZh) | `77.77% <100%> (ø)` | :arrow_up: |
   | [...oap/server/core/query/entity/ProfileStackTree.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcXVlcnkvZW50aXR5L1Byb2ZpbGVTdGFja1RyZWUuamF2YQ==) | `100% <100%> (ø)` | |
   | [.../server/core/profile/analyze/ProfileStackNode.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvcHJvZmlsZS9hbmFseXplL1Byb2ZpbGVTdGFja05vZGUuamF2YQ==) | `90.38% <100%> (+0.28%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `48.27% <0%> (+3.44%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [12414c8...ab75c86](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379903387
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -71,15 +73,19 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Cluster Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=elasticsearch
+      - name: Cluster Tests (InfluxDB/ZK/JDK8)
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=influxdb
       - name: Cluster With Gateway Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
 
 Review comment:
   I am OK with keeping the current status.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379863712
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/query/TraceQuery.java
 ##########
 @@ -0,0 +1,217 @@
+/*
+ * 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.storage.plugin.influxdb.query;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+import org.apache.skywalking.oap.server.core.query.entity.BasicTrace;
+import org.apache.skywalking.oap.server.core.query.entity.QueryOrder;
+import org.apache.skywalking.oap.server.core.query.entity.Span;
+import org.apache.skywalking.oap.server.core.query.entity.TraceBrief;
+import org.apache.skywalking.oap.server.core.query.entity.TraceState;
+import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
+import org.apache.skywalking.oap.server.library.util.BooleanUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.base.RecordDAO;
+import org.elasticsearch.common.Strings;
+import org.influxdb.dto.Query;
+import org.influxdb.dto.QueryResult;
+import org.influxdb.querybuilder.SelectQueryImpl;
+import org.influxdb.querybuilder.WhereQueryImpl;
+import org.influxdb.querybuilder.clauses.Clause;
+
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.eq;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.gte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.lte;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.regex;
+import static org.influxdb.querybuilder.BuiltQuery.QueryBuilder.select;
+
+@Slf4j
+public class TraceQuery implements ITraceQueryDAO {
+    private final InfluxClient client;
+
+    public TraceQuery(InfluxClient client) {
+        this.client = client;
+    }
+
+    @Override
+    public TraceBrief queryBasicTraces(long startSecondTB,
+                                       long endSecondTB,
+                                       long minDuration,
+                                       long maxDuration,
+                                       String endpointName,
+                                       int serviceId,
+                                       int serviceInstanceId,
+                                       int endpointId,
+                                       String traceId,
+                                       int limit,
+                                       int from,
+                                       TraceState traceState,
+                                       QueryOrder queryOrder)
+        throws IOException {
+
+        String orderBy = SegmentRecord.START_TIME;
+        if (queryOrder == QueryOrder.BY_DURATION) {
+            orderBy = SegmentRecord.LATENCY;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> recallQuery = select()
+            .function("top", orderBy, limit + from)
+            .column(SegmentRecord.SEGMENT_ID)
+            .column(SegmentRecord.START_TIME)
+            .column(SegmentRecord.ENDPOINT_NAME)
+            .column(SegmentRecord.LATENCY)
+            .column(SegmentRecord.IS_ERROR)
+            .column(SegmentRecord.TRACE_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            recallQuery.and(gte(SegmentRecord.TIME_BUCKET, startSecondTB))
+                       .and(lte(SegmentRecord.TIME_BUCKET, endSecondTB));
+        }
+        if (minDuration != 0) {
+            recallQuery.and(gte(SegmentRecord.LATENCY, minDuration));
+        }
+        if (maxDuration != 0) {
+            recallQuery.and(lte(SegmentRecord.LATENCY, maxDuration));
+        }
+        if (!Strings.isNullOrEmpty(endpointName)) {
+            recallQuery.and(regex(SegmentRecord.ENDPOINT_NAME, "/" + endpointName.replaceAll("/", "\\\\/") + "/"));
+        }
+        if (serviceId != 0) {
+            recallQuery.and(eq(RecordDAO.TAG_SERVICE_ID, String.valueOf(serviceId)));
+        }
+        if (serviceInstanceId != 0) {
+            recallQuery.and(eq(SegmentRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
+        }
+        if (endpointId != 0) {
+            recallQuery.and(eq(SegmentRecord.ENDPOINT_ID, endpointId));
+        }
+        if (!Strings.isNullOrEmpty(traceId)) {
+            recallQuery.and(eq(SegmentRecord.TRACE_ID, traceId));
+        }
+        switch (traceState) {
+            case ERROR:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.TRUE));
+                break;
+            case SUCCESS:
+                recallQuery.and(eq(SegmentRecord.IS_ERROR, BooleanUtils.FALSE));
+                break;
+        }
+
+        WhereQueryImpl<SelectQueryImpl> countQuery = select()
+            .count(SegmentRecord.ENDPOINT_ID)
+            .from(client.getDatabase(), SegmentRecord.INDEX_NAME)
+            .where();
+        for (Clause clause : recallQuery.getClauses()) {
+            countQuery.where(clause);
+        }
+        Query query = new Query(countQuery.getCommand() + recallQuery.getCommand());
+
+        List<QueryResult.Result> results = client.query(query);
+        if (log.isDebugEnabled()) {
+            log.debug("SQL: {} result set: {}", query.getCommand(), results);
+        }
+        if (results.size() != 2) {
+            throw new IOException("Expecting to get 2 Results, but it is " + results.size());
+        }
+        List<QueryResult.Series> counter = results.get(0).getSeries();
+        List<QueryResult.Series> result = results.get(1).getSeries();
+        if (result == null || result.isEmpty()) {
+            return new TraceBrief();
+        }
+
+        TraceBrief traceBrief = new TraceBrief();
+        traceBrief.setTotal(((Number) counter.get(0).getValues().get(0).get(1)).intValue());
+
+        result.get(0).getValues().stream().sorted((a, b) -> {
+            return Long.compare(((Number) b.get(1)).longValue(), ((Number) a.get(1)).longValue());
+        }).skip(from).forEach(values -> {
+            BasicTrace basicTrace = new BasicTrace();
+
+            basicTrace.setSegmentId((String) values.get(2));
+            basicTrace.setStart(String.valueOf((long) values.get(3)));
+            basicTrace.getEndpointNames().add((String) values.get(4));
+            basicTrace.setDuration((int) values.get(5));
+            basicTrace.setError(BooleanUtils.valueToBoolean((int) values.get(6)));
+            basicTrace.getTraceIds().add((String) values.get(7));
+
+            traceBrief.getTraces().add(basicTrace);
+        });
 
 Review comment:
   It took me a lot of time to get it clearly. It is a bad story.
   
   1. top/bottom cannot work with limit.
   2. I think top/bottom collects data by PriorityQueue. And it does not re-sort them in the result set. So we get the dataset unorder. Anyway, we always get the un-ordered result set in reality.
   
   I mean, the function `top` works after `limit`, if limit enabled.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-581037225
 
 
   How can I confirm that the submodule is the newest? If it is not the newest, what should I do?
   I found my e2e always failed in my devel branch.
   
   @wu-sheng , @kezhenxu94 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > :exclamation: No coverage uploaded for pull request base (`master@4b02679`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference#section-missing-base-commit).
   > The diff coverage is `0%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #4239   +/-   ##
   =========================================
     Coverage          ?   26.11%           
   =========================================
     Files             ?     1189           
     Lines             ?    26208           
     Branches          ?     3729           
   =========================================
     Hits              ?     6844           
     Misses            ?    18765           
     Partials          ?      599
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...r/storage/plugin/influxdb/InfluxStorageConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlQ29uZmlnLmphdmE=) | `0% <ø> (ø)` | |
   | [...er/storage/plugin/influxdb/query/MetricsQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9NZXRyaWNzUXVlcnkuamF2YQ==) | `0% <0%> (ø)` | |
   | [...erver/storage/plugin/influxdb/base/MetricsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL01ldHJpY3NEQU8uamF2YQ==) | `0% <0%> (ø)` | |
   | [...p/server/storage/plugin/influxdb/InfluxClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhDbGllbnQuamF2YQ==) | `0% <0%> (ø)` | |
   | [...ver/storage/plugin/influxdb/base/PointBuilder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9iYXNlL1BvaW50QnVpbGRlci5qYXZh) | `0% <0%> (ø)` | |
   | [...storage/plugin/influxdb/InfluxStorageProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9JbmZsdXhTdG9yYWdlUHJvdmlkZXIuamF2YQ==) | `0% <0%> (ø)` | |
   | [...server/storage/plugin/influxdb/query/LogQuery.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1pbmZsdXhkYi1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9pbmZsdXhkYi9xdWVyeS9Mb2dRdWVyeS5qYXZh) | `0% <0%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [4b02679...f6e2834](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378009922
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
+      - name: Single Node Tests(InfluxDB/JDK8)
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-influxdb
       - name: Single Node Tests(MySQL/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-mysql
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-mysql
       - name: Single Node Tests(JDK9)
-        run: export E2E_VERSION=jdk9-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk9-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK11)
-        run: export E2E_VERSION=jdk11-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk11-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK12)
-        run: export E2E_VERSION=jdk12-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk12-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Agent Reboot Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-agent-reboot
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-agent-reboot
 
-  Cluster:
+  Cluster_N_TTL:
 
 Review comment:
   Three things
   1. N represent AND is not a common case. `+` and `&` are better
   1. This name should not be changed randomly, `Cluster` is a required test, without this check(named as `cluster`), we can't merge PR unless we request the INFRA change the settings.
   1. If you want the test group names more clear, moving the TTL out of the cluster test group makes sense to me. But notice, it will take more time as it requires to compile the project again.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r374659028
 
 

 ##########
 File path: oap-server/server-bootstrap/src/main/resources/application.yml
 ##########
 @@ -98,33 +98,33 @@ storage:
 #    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
 #    profileTaskQueryMaxSize: ${SW_STORAGE_ES_QUERY_PROFILE_TASK_SIZE:200}
 #    advanced: ${SW_STORAGE_ES_ADVANCED:""}
-  elasticsearch7:
-    nameSpace: ${SW_NAMESPACE:""}
-    clusterNodes: ${SW_STORAGE_ES_CLUSTER_NODES:localhost:9200}
-    protocol: ${SW_STORAGE_ES_HTTP_PROTOCOL:"http"}
-    #trustStorePath: ${SW_SW_STORAGE_ES_SSL_JKS_PATH:"../es_keystore.jks"}
-    #trustStorePass: ${SW_SW_STORAGE_ES_SSL_JKS_PASS:""}
-    user: ${SW_ES_USER:""}
-    password: ${SW_ES_PASSWORD:""}
-    indexShardsNumber: ${SW_STORAGE_ES_INDEX_SHARDS_NUMBER:2}
-    indexReplicasNumber: ${SW_STORAGE_ES_INDEX_REPLICAS_NUMBER:0}
-    # Those data TTL settings will override the same settings in core module.
-    recordDataTTL: ${SW_STORAGE_ES_RECORD_DATA_TTL:7} # Unit is day
-    otherMetricsDataTTL: ${SW_STORAGE_ES_OTHER_METRIC_DATA_TTL:45} # Unit is day
-    monthMetricsDataTTL: ${SW_STORAGE_ES_MONTH_METRIC_DATA_TTL:18} # Unit is month
-    # Batch process setting, refer to https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.5/java-docs-bulk-processor.html
-    bulkActions: ${SW_STORAGE_ES_BULK_ACTIONS:1000} # Execute the bulk every 1000 requests
-    flushInterval: ${SW_STORAGE_ES_FLUSH_INTERVAL:10} # flush the bulk every 10 seconds whatever the number of requests
-    concurrentRequests: ${SW_STORAGE_ES_CONCURRENT_REQUESTS:2} # the number of concurrent requests
-    resultWindowMaxSize: ${SW_STORAGE_ES_QUERY_MAX_WINDOW_SIZE:10000}
-    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
-    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
-    advanced: ${SW_STORAGE_ES_ADVANCED:""}
-#  h2:
-#    driver: ${SW_STORAGE_H2_DRIVER:org.h2.jdbcx.JdbcDataSource}
-#    url: ${SW_STORAGE_H2_URL:jdbc:h2:mem:skywalking-oap-db}
-#    user: ${SW_STORAGE_H2_USER:sa}
-#    metadataQueryMaxSize: ${SW_STORAGE_H2_QUERY_MAX_SIZE:5000}
+#  elasticsearch7:
+#    nameSpace: ${SW_NAMESPACE:""}
+#    clusterNodes: ${SW_STORAGE_ES_CLUSTER_NODES:localhost:9200}
+#    protocol: ${SW_STORAGE_ES_HTTP_PROTOCOL:"http"}
+#    #trustStorePath: ${SW_SW_STORAGE_ES_SSL_JKS_PATH:"../es_keystore.jks"}
+#    #trustStorePass: ${SW_SW_STORAGE_ES_SSL_JKS_PASS:""}
+#    user: ${SW_ES_USER:""}
+#    password: ${SW_ES_PASSWORD:""}
+#    indexShardsNumber: ${SW_STORAGE_ES_INDEX_SHARDS_NUMBER:2}
+#    indexReplicasNumber: ${SW_STORAGE_ES_INDEX_REPLICAS_NUMBER:0}
+#    # Those data TTL settings will override the same settings in core module.
+#    recordDataTTL: ${SW_STORAGE_ES_RECORD_DATA_TTL:7} # Unit is day
+#    otherMetricsDataTTL: ${SW_STORAGE_ES_OTHER_METRIC_DATA_TTL:45} # Unit is day
+#    monthMetricsDataTTL: ${SW_STORAGE_ES_MONTH_METRIC_DATA_TTL:18} # Unit is month
+#    # Batch process setting, refer to https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.5/java-docs-bulk-processor.html
+#    bulkActions: ${SW_STORAGE_ES_BULK_ACTIONS:1000} # Execute the bulk every 1000 requests
+#    flushInterval: ${SW_STORAGE_ES_FLUSH_INTERVAL:10} # flush the bulk every 10 seconds whatever the number of requests
+#    concurrentRequests: ${SW_STORAGE_ES_CONCURRENT_REQUESTS:2} # the number of concurrent requests
+#    resultWindowMaxSize: ${SW_STORAGE_ES_QUERY_MAX_WINDOW_SIZE:10000}
+#    metadataQueryMaxSize: ${SW_STORAGE_ES_QUERY_MAX_SIZE:5000}
+#    segmentQueryMaxSize: ${SW_STORAGE_ES_QUERY_SEGMENT_SIZE:200}
+#    advanced: ${SW_STORAGE_ES_ADVANCED:""}
+  h2:
+    driver: ${SW_STORAGE_H2_DRIVER:org.h2.jdbcx.JdbcDataSource}
+    url: ${SW_STORAGE_H2_URL:jdbc:h2:mem:skywalking-oap-db}
+    user: ${SW_STORAGE_H2_USER:sa}
+    metadataQueryMaxSize: ${SW_STORAGE_H2_QUERY_MAX_SIZE:5000}
 
 Review comment:
   This shouldn't be changed. If start in IDE, we keep in ES

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r376263409
 
 

 ##########
 File path: oap-server/server-storage-plugin/storage-influxdb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/influxdb/base/BatchDAO.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.storage.plugin.influxdb.base;
+
+import java.util.List;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.commons.datacarrier.DataCarrier;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.BulkConsumePool;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.ConsumerPoolFactory;
+import org.apache.skywalking.apm.commons.datacarrier.consumer.IConsumer;
+import org.apache.skywalking.oap.server.core.UnexpectedException;
+import org.apache.skywalking.oap.server.core.storage.IBatchDAO;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.PrepareRequest;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
+import org.apache.skywalking.oap.server.storage.plugin.influxdb.InfluxClient;
+import org.influxdb.dto.BatchPoints;
+
+@Slf4j
+public class BatchDAO implements IBatchDAO {
+    private final DataCarrier<PrepareRequest> dataCarrier;
+    private final InfluxClient client;
+
+    public BatchDAO(InfluxClient client) {
+        this.client = client;
+
+        String name = "INFLUX_ASYNC_BATCH_PERSISTENT";
+        BulkConsumePool.Creator creator = new BulkConsumePool.Creator(name, 1, 20L);
+
+        try {
+            ConsumerPoolFactory.INSTANCE.createIfAbsent(name, creator);
+        } catch (Exception e) {
+            throw new UnexpectedException(e.getMessage(), e);
+        }
+
+        this.dataCarrier = new DataCarrier(1, 10000);
+        this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(name), new InfluxBatchConsumer(this));
+    }
+
+    @Override
+    public void asynchronous(InsertRequest insertRequest) {
+        dataCarrier.produce(insertRequest);
 
 Review comment:
   Yes, it does. I will fix.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-587352263
 
 
   e2e fail because the profile task DAO is still accessing the H2/MySQL. It seems you miss a change. Please fix it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r378004330
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -39,19 +39,21 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Single Node Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-single-service
+      - name: Single Node Tests(InfluxDB/JDK8)
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-influxdb
       - name: Single Node Tests(MySQL/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-mysql
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-mysql
       - name: Single Node Tests(JDK9)
-        run: export E2E_VERSION=jdk9-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk9-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK11)
-        run: export E2E_VERSION=jdk11-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk11-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Single Node Tests(JDK12)
-        run: export E2E_VERSION=jdk12-1.3 && bash -x test/e2e/run.sh e2e-single-service
+        run: export E2E_VERSION=jdk12-1.4 && bash -x test/e2e/run.sh e2e-single-service
       - name: Agent Reboot Tests(JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-agent-reboot
+        run: export E2E_VERSION=jdk8-1.4 && bash -x test/e2e/run.sh e2e-agent-reboot
 
-  Cluster:
+  Cluster_N_TTL:
 
 Review comment:
   What do you mean N_TTL?

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-586716319
 
 
   The ElasticSearch fails twice, I am rerunning it.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-586781392
 
 
   > @dmsolr Please recheck. The ElasticSearch test fails agian. I don't find any change related to you. Please take a look, what is the issue.
   
   I had checked it yesterday. The error message is "No space leave on device". And this exception threw on other PR. 

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/13698b55ed1e501475fa4069d0fcadd133b581c5?src=pr&el=desc) will **decrease** coverage by `<.01%`.
   > The diff coverage is `18.86%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   25.36%   25.35%   -0.01%     
   ==========================================
     Files        1201     1201              
     Lines       27753    27785      +32     
     Branches     3825     3829       +4     
   ==========================================
   + Hits         7040     7046       +6     
   - Misses      20079    20102      +23     
   - Partials      634      637       +3
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...sticsearch/StorageModuleElasticsearchProvider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/HistoryDeleteEsDAO.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9IaXN0b3J5RGVsZXRlRXNEQU8uamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/elasticsearch/base/StorageEsInstaller.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9TdG9yYWdlRXNJbnN0YWxsZXIuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../server/core/alarm/provider/grpc/GRPCCallback.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItYWxhcm0tcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvY29yZS9hbGFybS9wcm92aWRlci9ncnBjL0dSUENDYWxsYmFjay5qYXZh) | `59.42% <0%> (-1.45%)` | :arrow_down: |
   | [...walking/oap/server/core/analysis/Downsampling.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvRG93bnNhbXBsaW5nLmphdmE=) | `0% <0%> (ø)` | :arrow_up: |
   | [...lasticsearch/StorageModuleElasticsearchConfig.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvU3RvcmFnZU1vZHVsZUVsYXN0aWNzZWFyY2hDb25maWcuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [...icsearch7/StorageModuleElasticsearch7Provider.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoNy1wbHVnaW4vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9zdG9yYWdlL3BsdWdpbi9lbGFzdGljc2VhcmNoNy9TdG9yYWdlTW9kdWxlRWxhc3RpY3NlYXJjaDdQcm92aWRlci5qYXZh) | `0% <0%> (ø)` | :arrow_up: |
   | [...age/plugin/elasticsearch/base/TimeSeriesUtils.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItc3RvcmFnZS1wbHVnaW4vc3RvcmFnZS1lbGFzdGljc2VhcmNoLXBsdWdpbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL3N0b3JhZ2UvcGx1Z2luL2VsYXN0aWNzZWFyY2gvYmFzZS9UaW1lU2VyaWVzVXRpbHMuamF2YQ==) | `33.33% <43.75%> (+24.99%)` | :arrow_up: |
   | [...apache/skywalking/oal/rt/parser/MetricsHolder.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9vYWwtcnQvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FsL3J0L3BhcnNlci9NZXRyaWNzSG9sZGVyLmphdmE=) | `87.5% <75%> (+0.83%)` | :arrow_up: |
   | [.../agent/core/profile/ProfileTaskChannelService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcHJvZmlsZS9Qcm9maWxlVGFza0NoYW5uZWxTZXJ2aWNlLmphdmE=) | `26.88% <0%> (-1.08%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [13698b5...a4e13a6](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on issue #4239: [WIP] Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-574739482
 
 
   I don't deal with TTL yet. That can be implemented by RetentationPolicy, a InfluxDB built-in feature.

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/fb1b87538d949f3a2b87e2e14836f931312ee1b7?src=pr&el=desc) will **decrease** coverage by `0.01%`.
   > The diff coverage is `39.39%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   26.27%   26.26%   -0.02%     
   ==========================================
     Files        1196     1194       -2     
     Lines       26451    26399      -52     
     Branches     3767     3766       -1     
   ==========================================
   - Hits         6951     6933      -18     
   + Misses      18886    18854      -32     
   + Partials      614      612       -2
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...g/apache/skywalking/apm/agent/SkyWalkingAgent.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9hZ2VudC9Ta3lXYWxraW5nQWdlbnQuamF2YQ==) | `0% <0%> (ø)` | :arrow_up: |
   | [.../plugin/trace/ignore/TraceIgnoreExtendService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy90cmFjZS1pZ25vcmUtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vdHJhY2UvaWdub3JlL1RyYWNlSWdub3JlRXh0ZW5kU2VydmljZS5qYXZh) | `92.85% <100%> (+10.5%)` | :arrow_up: |
   | [...gin/trace/ignore/conf/IgnoreConfigInitializer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvb3B0aW9uYWwtcGx1Z2lucy90cmFjZS1pZ25vcmUtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vdHJhY2UvaWdub3JlL2NvbmYvSWdub3JlQ29uZmlnSW5pdGlhbGl6ZXIuamF2YQ==) | `38.23% <100%> (-7.92%)` | :arrow_down: |
   | [...m/agent/core/remote/TraceSegmentServiceClient.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcmVtb3RlL1RyYWNlU2VnbWVudFNlcnZpY2VDbGllbnQuamF2YQ==) | `79.41% <100%> (ø)` | :arrow_up: |
   | [...apm/agent/core/plugin/loader/AgentClassLoader.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvcGx1Z2luL2xvYWRlci9BZ2VudENsYXNzTG9hZGVyLmphdmE=) | `4.1% <5.55%> (-6.79%)` | :arrow_down: |
   | [.../apm/agent/core/conf/SnifferConfigInitializer.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvY29uZi9TbmlmZmVyQ29uZmlnSW5pdGlhbGl6ZXIuamF2YQ==) | `70.37% <85.71%> (-1.23%)` | :arrow_down: |
   | [...ache/skywalking/apm/agent/core/jvm/JVMService.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLWFnZW50LWNvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvYXBtL2FnZW50L2NvcmUvanZtL0pWTVNlcnZpY2UuamF2YQ==) | `83.6% <0%> (+3.27%)` | :arrow_up: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [fb1b875...47a6028](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#issuecomment-575965825
 
 
   # [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=h1) Report
   > Merging [#4239](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=desc) into [master](https://codecov.io/gh/apache/skywalking/commit/e6105b7f01c414b97618106f21e52b760b0e828d?src=pr&el=desc) will **decrease** coverage by `0.04%`.
   > The diff coverage is `1.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/skywalking/pull/4239/graphs/tree.svg?width=650&token=qrILxY5yA8&height=150&src=pr)](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #4239      +/-   ##
   ==========================================
   - Coverage   25.41%   25.36%   -0.05%     
   ==========================================
     Files        1201     1201              
     Lines       27701    27753      +52     
     Branches     3820     3825       +5     
   ==========================================
     Hits         7040     7040              
   - Misses      20027    20079      +52     
     Partials      634      634
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...pm/plugin/armeria/Armeria085ServerInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXJtZXJpYS0wLjg1LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vYXJtZXJpYS9Bcm1lcmlhMDg1U2VydmVySW50ZXJjZXB0b3IuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...r/trace/provider/parser/listener/SpanListener.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctdHJhY2UtcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvdHJhY2UvcHJvdmlkZXIvcGFyc2VyL2xpc3RlbmVyL1NwYW5MaXN0ZW5lci5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...ywalking/oap/server/library/buffer/DataStream.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItbGlicmFyeS9saWJyYXJ5LWJ1ZmZlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2xpYnJhcnkvYnVmZmVyL0RhdGFTdHJlYW0uamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...king/oap/server/library/util/GRPCStreamStatus.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItbGlicmFyeS9saWJyYXJ5LXV0aWwvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NreXdhbGtpbmcvb2FwL3NlcnZlci9saWJyYXJ5L3V0aWwvR1JQQ1N0cmVhbVN0YXR1cy5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [...trace/provider/parser/decorator/SpanDecorator.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctdHJhY2UtcmVjZWl2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvdHJhY2UvcHJvdmlkZXIvcGFyc2VyL2RlY29yYXRvci9TcGFuRGVjb3JhdG9yLmphdmE=) | `0% <ø> (ø)` | :arrow_up: |
   | [.../apache/skywalking/oap/server/core/CoreModule.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvQ29yZU1vZHVsZS5qYXZh) | `3.92% <ø> (ø)` | :arrow_up: |
   | [...pm/plugin/armeria/Armeria086ClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXJtZXJpYS0wLjg1LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vYXJtZXJpYS9Bcm1lcmlhMDg2Q2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...g/apm/plugin/armeria/ArmeriaClientInterceptor.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-YXBtLXNuaWZmZXIvYXBtLXNkay1wbHVnaW4vYXJtZXJpYS0wLjg1LngtcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL2FwbS9wbHVnaW4vYXJtZXJpYS9Bcm1lcmlhQ2xpZW50SW50ZXJjZXB0b3IuamF2YQ==) | `0% <ø> (ø)` | :arrow_up: |
   | [...r/receiver/sharing/server/SharingServerModule.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItcmVjZWl2ZXItcGx1Z2luL3NreXdhbGtpbmctc2hhcmluZy1zZXJ2ZXItcGx1Z2luL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9za3l3YWxraW5nL29hcC9zZXJ2ZXIvcmVjZWl2ZXIvc2hhcmluZy9zZXJ2ZXIvU2hhcmluZ1NlcnZlck1vZHVsZS5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | [.../oap/server/core/analysis/worker/ExportWorker.java](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree#diff-b2FwLXNlcnZlci9zZXJ2ZXItY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvc2t5d2Fsa2luZy9vYXAvc2VydmVyL2NvcmUvYW5hbHlzaXMvd29ya2VyL0V4cG9ydFdvcmtlci5qYXZh) | `0% <ø> (ø)` | :arrow_up: |
   | ... and [83 more](https://codecov.io/gh/apache/skywalking/pull/4239/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=footer). Last update [e6105b7...13698b5](https://codecov.io/gh/apache/skywalking/pull/4239?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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


With regards,
Apache Git Services

[GitHub] [skywalking] dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin

Posted by GitBox <gi...@apache.org>.
dmsolr commented on a change in pull request #4239: Provide influxdb as a new storage plugin
URL: https://github.com/apache/skywalking/pull/4239#discussion_r379902871
 
 

 ##########
 File path: .github/workflows/e2e.yaml
 ##########
 @@ -71,15 +73,19 @@ jobs:
           ./mvnw --batch-mode -Dcheckstyle.skip -Drat.skip -T2 -Dmaven.compile.fork -Dmaven.compiler.maxmem=3072 -DskipTests clean install
           ./mvnw --batch-mode -f test/e2e/pom.xml -pl e2e-base clean install
       - name: Cluster Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=elasticsearch
+      - name: Cluster Tests (InfluxDB/ZK/JDK8)
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster/e2e-cluster-test-runner --storage=influxdb
       - name: Cluster With Gateway Tests (ES6/ZK/JDK8)
-        run: export E2E_VERSION=jdk8-1.3 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
+        run: export E2E_VERSION=jdk8-1.5 && bash -x test/e2e/run.sh e2e-cluster-with-gateway/e2e-cluster-with-gateway-test-runner
 
 Review comment:
   Not add e2e-cluster-with-gateway test yet. Do we need it?
   
   By the way, you are right, @kezhenxu94 .
   (I am trying to add more testcases into e2e-ttl-influxdb, so I have not refactored it)

----------------------------------------------------------------
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


With regards,
Apache Git Services