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

[GitHub] [skywalking] wu-sheng commented on a change in pull request #6499: [Incompatible Enhancement]New index policy of ElasticSearch storage option

wu-sheng commented on a change in pull request #6499:
URL: https://github.com/apache/skywalking/pull/6499#discussion_r588855550



##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {

Review comment:
       I think this not a `Partitioner`. I think this is an `PhysicalIndexManager`?

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {
+    /**
+     * The telemetry data, that generated by OAL or MAL, would be partitioned to storage by the functions of the OAL or
+     * MAL. And, the other record data would be insulated storage by themselves definitions.

Review comment:
       All data in SkyWalking is telemetry data. Don't use this words. INSTANCE should not have comments. All comments should on the class level.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {
+    /**
+     * The telemetry data, that generated by OAL or MAL, would be partitioned to storage by the functions of the OAL or
+     * MAL. And, the other record data would be insulated storage by themselves definitions.
+     */
+    INSTANCE;
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    public static final String LOGIC_TABLE_NAME = "logic_table";
+
+    /**
+     * The relations of the logic table and the physical table.
+     */
+    private final Map<String, String> tableRelations = new ConcurrentHashMap<>();
+
+    public String getTableName(Model model) {
+        return StringUtil.isNotBlank(
+            model.getAggregationFunctionName()) ? model.getAggregationFunctionName() : model.getName();
+    }
+
+    /**
+     * Generate the index doc ID. When a model is the aggregation storage mode, the logicTableName is a part of new ID
+     * to avoid conflicts.
+     */
+    public String generateDocId(Model model, String originalID) {
+        if (!isAggregationMode(model)) {
+            return originalID;
+        }
+        return this.generateDocId(model.getName(), originalID);
+    }
+
+    /**
+     * Generate the index doc ID.
+     */
+    public String generateDocId(String logicTableName, String originalID) {
+        return encoder.encodeToString(logicTableName.getBytes(StandardCharsets.UTF_8)) + originalID;
+    }
+
+    /**
+     * Check the mode of the Model definition.
+     */
+    public boolean isAggregationMode(Model model) {
+        return StringUtil.isNotBlank(model.getAggregationFunctionName());
+    }
+
+    /**
+     * Append the value to the {@link #LOGIC_TABLE_NAME} when running in the aggregation mode.
+     */
+    /**
+     * When a model is the aggregation storage mode, a column named {@link #LOGIC_TABLE_NAME} would be append to the
+     * physical index. The value of the column is the original table name in other storages, such as the OAL name.
+     */
+    public Map<String, Object> appendLogicTableColumn(Model model, Map<String, Object> columns) {
+        if (!isAggregationMode(model)) {
+            return columns;
+        }
+        columns.put(LOGIC_TABLE_NAME, this.getLogicTableColumnVal(model.getName()));
+        return columns;
+    }
+
+    /**
+     * BASE64 encode the original logic table name.
+     */
+    public String getLogicTableColumnVal(String logicTableName) {

Review comment:
       Why do we need `metrics_name` to BASE64 encoding? We used to use the name directly as the physical index name, I assume it should be fine to use the literal string, which is better to read.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {
+    /**
+     * The telemetry data, that generated by OAL or MAL, would be partitioned to storage by the functions of the OAL or
+     * MAL. And, the other record data would be insulated storage by themselves definitions.
+     */
+    INSTANCE;
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    public static final String LOGIC_TABLE_NAME = "logic_table";
+
+    /**
+     * The relations of the logic table and the physical table.
+     */
+    private final Map<String, String> tableRelations = new ConcurrentHashMap<>();
+
+    public String getTableName(Model model) {
+        return StringUtil.isNotBlank(
+            model.getAggregationFunctionName()) ? model.getAggregationFunctionName() : model.getName();

Review comment:
       Should use `isAggregationMode(model)`, right?

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -38,27 +41,50 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The relation is between physical table and the specific structure definition.
+     */
+    private final ConcurrentHashMap<String, Map<String, Object>> tables;

Review comment:
       I think should use `StoragePartitioner` to hold, and put a class called `PhysicalIndex` the value of `StoragePartitioner#tableRelations`. 
   According to your code style, you should isolate these two.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -100,6 +139,48 @@ protected void createTable(Model model) throws StorageException {
         } catch (IOException e) {
             throw new StorageException(e.getMessage());
         }
+
+    }
+
+    /**
+     * Append the mapping to the tables with the same table name key.
+     */
+    private Map<String, Object> appendTemplateMapping(String tableName, Map<String, Object> mapping) {
+        if (!tables.containsKey(tableName)) {
+            tables.put(tableName, mapping);
+            return mapping;
+        }
+        Map<String, Object> existMapping = tables.get(tableName);
+        Map<String, Object> existFields = getColumnProperties(existMapping);
+        Map<String, Object> checkingFields = getColumnProperties(mapping);
+        Map<String, Object> newFields = checkingFields.entrySet()
+                                                      .stream()
+                                                      .filter(item -> !existFields.containsKey(item.getKey()))
+                                                      .collect(Collectors.toMap(
+                                                          Map.Entry::getKey, Map.Entry::getValue));
+        newFields.forEach(existFields::put);
+        return existMapping;
+    }
+
+    protected Map<String, Object> getColumnProperties(Map<String, Object> mapping) {
+        if (Objects.isNull(mapping) || mapping.size() == 0) {
+            return new HashMap<>();
+        }
+        return (Map<String, Object>) ((Map<String, Object>) mapping.get(ElasticSearchClient.TYPE)).get("properties");
+    }
+
+    /**
+     * Whether the tables contains the input mapping with the same table name key.
+     */
+    private boolean containsTemplateMapping(String tableName, Map<String, Object> mapping) {

Review comment:
       `containsTemplateMapping`->`isTemplateMappingCompatible`

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -38,27 +41,50 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The relation is between physical table and the specific structure definition.
+     */
+    private final ConcurrentHashMap<String, Map<String, Object>> tables;
+
     public StorageEsInstaller(Client client,
                               ModuleManager moduleManager,
-                              final StorageModuleElasticsearchConfig config) {
+                              StorageModuleElasticsearchConfig config) throws StorageException {
         super(client, moduleManager);
         this.columnTypeEsMapping = new ColumnTypeEsMapping();
         this.config = config;
+        this.tables = new ConcurrentHashMap<>(loadHistoryTables(client));

Review comment:
       You should read template settings when you face the creation event only.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {
+    /**
+     * The telemetry data, that generated by OAL or MAL, would be partitioned to storage by the functions of the OAL or
+     * MAL. And, the other record data would be insulated storage by themselves definitions.
+     */
+    INSTANCE;
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    public static final String LOGIC_TABLE_NAME = "logic_table";
+
+    /**
+     * The relations of the logic table and the physical table.
+     */
+    private final Map<String, String> tableRelations = new ConcurrentHashMap<>();

Review comment:
       `ConcurrentHashMap`? Is there a race condition?

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -38,27 +41,50 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The relation is between physical table and the specific structure definition.
+     */
+    private final ConcurrentHashMap<String, Map<String, Object>> tables;
+
     public StorageEsInstaller(Client client,
                               ModuleManager moduleManager,
-                              final StorageModuleElasticsearchConfig config) {
+                              StorageModuleElasticsearchConfig config) throws StorageException {
         super(client, moduleManager);
         this.columnTypeEsMapping = new ColumnTypeEsMapping();
         this.config = config;
+        this.tables = new ConcurrentHashMap<>(loadHistoryTables(client));

Review comment:
       I don't recommend to do this. Reading all template impact the user side **security policy**.
   And according to existing codes, if this template and index exist, we will treat them as a correct version.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -68,26 +94,39 @@ protected boolean isExists(Model model) throws StorageException {
     @Override
     protected void createTable(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
-
         Map<String, Object> settings = createSetting(model);
         Map<String, Object> mapping = createMapping(model);
-        log.info("index {}'s columnTypeEsMapping builder str: {}", esClient.formatIndexName(model.getName()), mapping
-            .toString());
+        String tableName = StoragePartitioner.INSTANCE.getTableName(model);
+        StoragePartitioner.INSTANCE.registerTableRelation(model.getName(), tableName);
+        log.info("index {}'s columnTypeEsMapping builder str: {}",
+                 esClient.formatIndexName(tableName), mapping.toString()
+        );
 
+        String indexName;
         try {
-            String indexName;
-            if (!model.isTimeSeries()) {
-                indexName = model.getName();
-            } else {
-                if (!esClient.isExistsTemplate(model.getName())) {
-                    boolean isAcknowledged = esClient.createTemplate(model.getName(), settings, mapping);
-                    log.info(
-                        "create {} index template finished, isAcknowledged: {}", model.getName(), isAcknowledged);
+            if (model.isTimeSeries()) {
+                Map<String, Object> template = esClient.getTemplate(tableName);
+                if (!template.isEmpty()) {
+                    appendTemplateMapping(tableName, (Map<String, Object>) template.get("mappings"));
+                }
+                if (!containsTemplateMapping(tableName, mapping)) {
+                    Map<String, Object> templateMapping = appendTemplateMapping(tableName, mapping);
+                    boolean isAcknowledged = esClient.createTemplate(tableName, settings, templateMapping);

Review comment:
       Are you meaning we could recreate a template w/o deleting the old one?

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StorageEsInstaller.java
##########
@@ -68,26 +94,39 @@ protected boolean isExists(Model model) throws StorageException {
     @Override
     protected void createTable(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
-
         Map<String, Object> settings = createSetting(model);
         Map<String, Object> mapping = createMapping(model);
-        log.info("index {}'s columnTypeEsMapping builder str: {}", esClient.formatIndexName(model.getName()), mapping
-            .toString());
+        String tableName = StoragePartitioner.INSTANCE.getTableName(model);
+        StoragePartitioner.INSTANCE.registerTableRelation(model.getName(), tableName);
+        log.info("index {}'s columnTypeEsMapping builder str: {}",
+                 esClient.formatIndexName(tableName), mapping.toString()
+        );
 
+        String indexName;
         try {
-            String indexName;
-            if (!model.isTimeSeries()) {
-                indexName = model.getName();
-            } else {
-                if (!esClient.isExistsTemplate(model.getName())) {
-                    boolean isAcknowledged = esClient.createTemplate(model.getName(), settings, mapping);
-                    log.info(
-                        "create {} index template finished, isAcknowledged: {}", model.getName(), isAcknowledged);
+            if (model.isTimeSeries()) {
+                Map<String, Object> template = esClient.getTemplate(tableName);
+                if (!template.isEmpty()) {
+                    appendTemplateMapping(tableName, (Map<String, Object>) template.get("mappings"));
+                }
+                if (!containsTemplateMapping(tableName, mapping)) {
+                    Map<String, Object> templateMapping = appendTemplateMapping(tableName, mapping);
+                    boolean isAcknowledged = esClient.createTemplate(tableName, settings, templateMapping);

Review comment:
       You need to polish this new creating logic, it is not clear to me.

##########
File path: oap-server/server-storage-plugin/storage-elasticsearch-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/elasticsearch/base/StoragePartitioner.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.elasticsearch.base;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.apm.util.StringUtil;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+@Slf4j
+public enum StoragePartitioner {
+    /**
+     * The telemetry data, that generated by OAL or MAL, would be partitioned to storage by the functions of the OAL or
+     * MAL. And, the other record data would be insulated storage by themselves definitions.
+     */
+    INSTANCE;
+    private final Base64.Encoder encoder = Base64.getEncoder();
+    public static final String LOGIC_TABLE_NAME = "logic_table";
+
+    /**
+     * The relations of the logic table and the physical table.
+     */
+    private final Map<String, String> tableRelations = new ConcurrentHashMap<>();
+
+    public String getTableName(Model model) {
+        return StringUtil.isNotBlank(
+            model.getAggregationFunctionName()) ? model.getAggregationFunctionName() : model.getName();
+    }
+
+    /**
+     * Generate the index doc ID. When a model is the aggregation storage mode, the logicTableName is a part of new ID
+     * to avoid conflicts.
+     */
+    public String generateDocId(Model model, String originalID) {
+        if (!isAggregationMode(model)) {
+            return originalID;
+        }
+        return this.generateDocId(model.getName(), originalID);
+    }
+
+    /**
+     * Generate the index doc ID.
+     */
+    public String generateDocId(String logicTableName, String originalID) {
+        return encoder.encodeToString(logicTableName.getBytes(StandardCharsets.UTF_8)) + originalID;
+    }
+
+    /**
+     * Check the mode of the Model definition.
+     */
+    public boolean isAggregationMode(Model model) {
+        return StringUtil.isNotBlank(model.getAggregationFunctionName());
+    }
+
+    /**
+     * Append the value to the {@link #LOGIC_TABLE_NAME} when running in the aggregation mode.
+     */
+    /**
+     * When a model is the aggregation storage mode, a column named {@link #LOGIC_TABLE_NAME} would be append to the
+     * physical index. The value of the column is the original table name in other storages, such as the OAL name.
+     */
+    public Map<String, Object> appendLogicTableColumn(Model model, Map<String, Object> columns) {
+        if (!isAggregationMode(model)) {
+            return columns;
+        }
+        columns.put(LOGIC_TABLE_NAME, this.getLogicTableColumnVal(model.getName()));
+        return columns;
+    }
+
+    /**
+     * BASE64 encode the original logic table name.
+     */
+    public String getLogicTableColumnVal(String logicTableName) {
+        return encoder.encodeToString(logicTableName.getBytes(StandardCharsets.UTF_8));
+    }
+
+    public String getPhysicialTableName(String logicName) {

Review comment:
       `Physicial` typo.




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