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/07 15:49:43 UTC

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

muse-dev[bot] commented on a change in pull request #6499:
URL: https://github.com/apache/skywalking/pull/6499#discussion_r589049140



##########
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,68 +42,165 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The mappings of the template .
+     */
+    private final Map<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 HashMap<>();
     }
 
     @Override
     protected boolean isExists(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
         try {
-            if (model.isTimeSeries()) {
-                return esClient.isExistsTemplate(model.getName()) && esClient.isExistsIndex(
-                    TimeSeriesUtils.latestWriteIndexName(model));
-            } else {
-                return esClient.isExistsIndex(model.getName());
+            if (!model.isTimeSeries()) {
+                return esClient.isExistsIndex(tableName);
             }
+            boolean exist = esClient.isExistsTemplate(tableName)
+                && esClient.isExistsIndex(TimeSeriesUtils.latestWriteIndexName(model));
+            if (RunningMode.isInitMode() && IndexController.INSTANCE.isAggregationMode(model)) {
+                appendTemplateMapping(tableName, (Map<String, Object>) esClient.getTemplate(tableName).get("mappings"));
+                exist = exist && isTemplateMappingCompatible(tableName, createMapping(model));
+            }
+            return exist;
         } catch (IOException e) {
             throw new StorageException(e.getMessage());
         }
     }
 
     @Override
     protected void createTable(Model model) throws StorageException {
+        if (model.isTimeSeries()) {
+            createTimeSeriesTable(model);
+        } else {
+            createNormalTable(model);
+        }
+    }
+
+    private void createNormalTable(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
+        try {
+            if (!esClient.isExistsIndex(tableName)) {
+                boolean isAcknowledged = esClient.createIndex(tableName);
+                log.info("create {} index finished, isAcknowledged: {}", tableName, isAcknowledged);
+                if (!isAcknowledged) {
+                    throw new StorageException("create " + tableName + " time series index failure, ");
+                }
+            }
+        } catch (IOException e) {
+            throw new StorageException("cannot create the normal index", e);
+        }
+    }
 
+    private void createTimeSeriesTable(Model model) throws StorageException {
+        ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
         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 indexName = TimeSeriesUtils.latestWriteIndexName(model);
         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 (!isAcknowledged) {
-                        throw new StorageException("create " + model.getName() + " index template failure, ");
-                    }
-                }
-                indexName = TimeSeriesUtils.latestWriteIndexName(model);
+            boolean updateTemplate = !esClient.isExistsTemplate(tableName);
+            if (!updateTemplate) {
+                appendTemplateMapping(tableName, (Map<String, Object>) esClient.getTemplate(tableName)

Review comment:
       *NULL_DEREFERENCE:*  object returned by `esClient.getTemplate(tableName).get("mappings")` could be null and is dereferenced by call to `appendTemplateMapping(...)` at line 119.
   (at-me [in a reply](https://docs.muse.dev/docs/talk-to-muse/) with `help` or `ignore`)

##########
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,68 +42,165 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The mappings of the template .
+     */
+    private final Map<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 HashMap<>();
     }
 
     @Override
     protected boolean isExists(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
         try {
-            if (model.isTimeSeries()) {
-                return esClient.isExistsTemplate(model.getName()) && esClient.isExistsIndex(
-                    TimeSeriesUtils.latestWriteIndexName(model));
-            } else {
-                return esClient.isExistsIndex(model.getName());
+            if (!model.isTimeSeries()) {
+                return esClient.isExistsIndex(tableName);
             }
+            boolean exist = esClient.isExistsTemplate(tableName)
+                && esClient.isExistsIndex(TimeSeriesUtils.latestWriteIndexName(model));
+            if (RunningMode.isInitMode() && IndexController.INSTANCE.isAggregationMode(model)) {
+                appendTemplateMapping(tableName, (Map<String, Object>) esClient.getTemplate(tableName).get("mappings"));
+                exist = exist && isTemplateMappingCompatible(tableName, createMapping(model));
+            }
+            return exist;
         } catch (IOException e) {
             throw new StorageException(e.getMessage());
         }
     }
 
     @Override
     protected void createTable(Model model) throws StorageException {
+        if (model.isTimeSeries()) {
+            createTimeSeriesTable(model);
+        } else {
+            createNormalTable(model);
+        }
+    }
+
+    private void createNormalTable(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
+        try {
+            if (!esClient.isExistsIndex(tableName)) {
+                boolean isAcknowledged = esClient.createIndex(tableName);
+                log.info("create {} index finished, isAcknowledged: {}", tableName, isAcknowledged);
+                if (!isAcknowledged) {
+                    throw new StorageException("create " + tableName + " time series index failure, ");
+                }
+            }
+        } catch (IOException e) {
+            throw new StorageException("cannot create the normal index", e);
+        }
+    }
 
+    private void createTimeSeriesTable(Model model) throws StorageException {
+        ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
         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 indexName = TimeSeriesUtils.latestWriteIndexName(model);
         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 (!isAcknowledged) {
-                        throw new StorageException("create " + model.getName() + " index template failure, ");
-                    }
-                }
-                indexName = TimeSeriesUtils.latestWriteIndexName(model);
+            boolean updateTemplate = !esClient.isExistsTemplate(tableName);
+            if (!updateTemplate) {
+                appendTemplateMapping(tableName, (Map<String, Object>) esClient.getTemplate(tableName)
+                                                                               .get("mappings"));
+                updateTemplate = !isTemplateMappingCompatible(tableName, mapping);
             }
-            if (!esClient.isExistsIndex(indexName)) {
-                boolean isAcknowledged = esClient.createIndex(indexName);
-                log.info("create {} index finished, isAcknowledged: {}", indexName, isAcknowledged);
+            if (updateTemplate) {
+                Map<String, Object> templateMapping = appendTemplateMapping(tableName, mapping);
+                boolean isAcknowledged = esClient.createOrUpdateTemplate(tableName, settings, templateMapping);
+                log.info("create {} index template finished, isAcknowledged: {}", tableName, isAcknowledged);
                 if (!isAcknowledged) {
-                    throw new StorageException("create " + indexName + " time series index failure, ");
+                    throw new IOException("create " + tableName + " index template failure, ");
                 }
-            }
 
+                if (esClient.isExistsIndex(indexName)) {
+                    Map<String, Object> historyMapping = (Map<String, Object>) esClient.getIndex(indexName)
+                                                                                       .get("mappings");
+                    Map<String, Object> appendMapping = extractAppendMapping(templateMapping, historyMapping);
+                    if (appendMapping.size() > 0) {
+                        esClient.updateIndexMapping(indexName, appendMapping);
+                    }
+                } else {
+                    isAcknowledged = esClient.createIndex(indexName);
+                    log.info("create {} index finished, isAcknowledged: {}", indexName, isAcknowledged);
+                    if (!isAcknowledged) {
+                        throw new StorageException("create " + indexName + " time series index failure, ");
+                    }
+                }
+            }
         } catch (IOException e) {
-            throw new StorageException(e.getMessage());
+            throw new StorageException("cannot create " + tableName + " index template", e);
+        }
+    }
+
+    private Map<String, Object> extractAppendMapping(final Map<String, Object> latestMapping,
+                                                     final Map<String, Object> historyMapping) {
+        Map<String, Object> checkingFields = getColumnProperties(latestMapping);
+        Map<String, Object> existFields = getColumnProperties(historyMapping);
+        Map<String, Object> newFields = checkingFields.entrySet()
+                                                      .stream()
+                                                      .filter(item -> !existFields.containsKey(item.getKey()))
+                                                      .collect(Collectors.toMap(
+                                                          Map.Entry::getKey, Map.Entry::getValue));
+        Map<String, Object> mapping = createEmptyMapping();
+        Map<String, Object> properties = getColumnProperties(mapping);
+        newFields.forEach(properties::put);
+        return mapping;
+    }
+
+    /**
+     * Append the mapping to the tables with the same table name key.
+     */
+    private Map<String, Object> appendTemplateMapping(String tableName, Map<String, Object> mapping) {
+        if (Objects.isNull(mapping) || mapping.size() == 0) {
+            return Optional.ofNullable(tables.get(tableName)).orElse(new HashMap<>());
+        }
+        if (!tables.containsKey(tableName)) {
+            tables.put(tableName, mapping);
+            return mapping;
+        }
+        Map<String, Object> existMapping = tables.get(tableName);
+        Map<String, Object> appendMapping = extractAppendMapping(mapping, existMapping);
+        Map<String, Object> newColumns = getColumnProperties(appendMapping);
+        Map<String, Object> existFields = getColumnProperties(existMapping);
+        newColumns.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");

Review comment:
       *NULL_DEREFERENCE:*  object returned by `mapping.get("type")` could be null and is dereferenced at line 189.
   (at-me [in a reply](https://docs.muse.dev/docs/talk-to-muse/) with `help` or `ignore`)

##########
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,68 +42,165 @@
 @Slf4j
 public class StorageEsInstaller extends ModelInstaller {
     private final Gson gson = new Gson();
-
     private final StorageModuleElasticsearchConfig config;
     protected final ColumnTypeEsMapping columnTypeEsMapping;
 
+    /**
+     * The mappings of the template .
+     */
+    private final Map<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 HashMap<>();
     }
 
     @Override
     protected boolean isExists(Model model) throws StorageException {
         ElasticSearchClient esClient = (ElasticSearchClient) client;
+        String tableName = IndexController.INSTANCE.getTableName(model);
+        IndexController.LogicIndicesRegister.registerRelation(model.getName(), tableName);
         try {
-            if (model.isTimeSeries()) {
-                return esClient.isExistsTemplate(model.getName()) && esClient.isExistsIndex(
-                    TimeSeriesUtils.latestWriteIndexName(model));
-            } else {
-                return esClient.isExistsIndex(model.getName());
+            if (!model.isTimeSeries()) {
+                return esClient.isExistsIndex(tableName);
             }
+            boolean exist = esClient.isExistsTemplate(tableName)
+                && esClient.isExistsIndex(TimeSeriesUtils.latestWriteIndexName(model));
+            if (RunningMode.isInitMode() && IndexController.INSTANCE.isAggregationMode(model)) {
+                appendTemplateMapping(tableName, (Map<String, Object>) esClient.getTemplate(tableName).get("mappings"));

Review comment:
       *NULL_DEREFERENCE:*  object returned by `esClient.getTemplate(tableName).get("mappings")` could be null and is dereferenced by call to `appendTemplateMapping(...)` at line 74.
   (at-me [in a reply](https://docs.muse.dev/docs/talk-to-muse/) with `help` or `ignore`)




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