]> source.dussan.org Git - sonarqube.git/commitdiff
SONAR-8798 store index definition hashes in separate es index
authorDaniel Schwarz <daniel.schwarz@sonarsource.com>
Mon, 26 Jun 2017 15:19:38 +0000 (17:19 +0200)
committerDaniel Schwarz <bartfastiel@users.noreply.github.com>
Wed, 9 Aug 2017 13:09:54 +0000 (15:09 +0200)
13 files changed:
server/sonar-server/src/main/java/org/sonar/server/es/IndexCreator.java
server/sonar-server/src/main/java/org/sonar/server/es/IndexDefinitionHash.java
server/sonar-server/src/main/java/org/sonar/server/es/IndexerStartupTask.java
server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndex.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndexDefinition.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/metadata/package-info.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/platform/platformlevel/PlatformLevel4.java
server/sonar-server/src/test/java/org/sonar/server/es/EsTester.java
server/sonar-server/src/test/java/org/sonar/server/es/IndexCreatorTest.java
server/sonar-server/src/test/java/org/sonar/server/es/IndexDefinitionHashTest.java
server/sonar-server/src/test/java/org/sonar/server/es/IndexerStartupTaskTest.java
server/sonar-server/src/test/java/org/sonar/server/es/metadata/MetadataIndexTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/platform/monitoring/EsMonitorTest.java

index 96fd1ec256716d015885a19ba767dabc99623b91..a2dcfb4895b335c93cac5ff69aec1d9f067f3a43 100644 (file)
@@ -29,6 +29,9 @@ import org.picocontainer.Startable;
 import org.sonar.api.server.ServerSide;
 import org.sonar.api.utils.log.Logger;
 import org.sonar.api.utils.log.Loggers;
+import org.sonar.server.es.IndexDefinitions.Index;
+import org.sonar.server.es.metadata.MetadataIndex;
+import org.sonar.server.es.metadata.MetadataIndexDefinition;
 
 /**
  * Creates/deletes all indices in Elasticsearch during server startup.
@@ -38,32 +41,39 @@ public class IndexCreator implements Startable {
 
   private static final Logger LOGGER = Loggers.get(IndexCreator.class);
 
-  /**
-   * Internal setting stored on index to know its version. It's used to re-create index
-   * when something changed between versions.
-   */
-  private static final String SETTING_HASH = "sonar_hash";
-
+  private final MetadataIndexDefinition metadataIndexDefinition;
+  private final MetadataIndex metadataIndex;
   private final EsClient client;
   private final IndexDefinitions definitions;
 
-  public IndexCreator(EsClient client, IndexDefinitions definitions) {
+  public IndexCreator(EsClient client, IndexDefinitions definitions, MetadataIndexDefinition metadataIndexDefinition, MetadataIndex metadataIndex) {
     this.client = client;
     this.definitions = definitions;
+    this.metadataIndexDefinition = metadataIndexDefinition;
+    this.metadataIndex = metadataIndex;
   }
 
   @Override
   public void start() {
+
+    // create the "metadata" index first
+    if (!client.prepareIndicesExist(MetadataIndexDefinition.INDEX_TYPE_METADATA.getIndex()).get().isExists()) {
+      IndexDefinition.IndexDefinitionContext context = new IndexDefinition.IndexDefinitionContext();
+      metadataIndexDefinition.define(context);
+      NewIndex index = context.getIndices().values().iterator().next();
+      createIndex(new Index(index), false);
+    }
+
     // create indices that do not exist or that have a new definition (different mapping, cluster enabled, ...)
-    for (IndexDefinitions.Index index : definitions.getIndices().values()) {
+    for (Index index : definitions.getIndices().values()) {
       boolean exists = client.prepareIndicesExist(index.getName()).get().isExists();
-      if (exists && needsToDeleteIndex(index)) {
+      if (exists && !index.getName().equals(MetadataIndexDefinition.INDEX_TYPE_METADATA.getIndex()) && needsToDeleteIndex(index)) {
         LOGGER.info(String.format("Delete index %s (settings changed)", index.getName()));
         deleteIndex(index.getName());
         exists = false;
       }
       if (!exists) {
-        createIndex(index);
+        createIndex(index, true);
       }
     }
   }
@@ -73,11 +83,16 @@ public class IndexCreator implements Startable {
     // nothing to do
   }
 
-  private void createIndex(IndexDefinitions.Index index) {
+  private void createIndex(Index index, boolean useMetadata) {
     LOGGER.info(String.format("Create index %s", index.getName()));
     Settings.Builder settings = Settings.builder();
     settings.put(index.getSettings());
-    settings.put(SETTING_HASH, new IndexDefinitionHash().of(index));
+    if (useMetadata) {
+      metadataIndex.setHash(index.getName(), IndexDefinitionHash.of(index));
+      for (IndexDefinitions.IndexType type : index.getTypes().values()) {
+        metadataIndex.setInitialized(new IndexType(index.getName(), type.getName()), false);
+      }
+    }
     CreateIndexResponse indexResponse = client
       .prepareCreate(index.getName())
       .setSettings(settings)
@@ -105,13 +120,11 @@ public class IndexCreator implements Startable {
     client.nativeClient().admin().indices().prepareDelete(indexName).get();
   }
 
-  private boolean needsToDeleteIndex(IndexDefinitions.Index index) {
-    boolean toBeDeleted = false;
-    String hash = client.nativeClient().admin().indices().prepareGetSettings(index.getName()).get().getSetting(index.getName(), "index." + SETTING_HASH);
-    if (hash != null) {
-      String defHash = new IndexDefinitionHash().of(index);
-      toBeDeleted = !StringUtils.equals(hash, defHash);
-    }
-    return toBeDeleted;
+  private boolean needsToDeleteIndex(Index index) {
+    return metadataIndex.getHash(index.getName())
+      .map(hash -> {
+        String defHash = IndexDefinitionHash.of(index);
+        return !StringUtils.equals(hash, defHash);
+      }).orElse(true);
   }
 }
index c550498fd09d357c22e3be77c76915b29edd5f16..e50546ceeec24c3bedddbe9f697c6d30dcefdf78 100644 (file)
@@ -37,11 +37,14 @@ class IndexDefinitionHash {
 
   private static final char DELIMITER = ',';
 
-  String of(IndexDefinitions.Index index) {
+  private IndexDefinitionHash() {
+  }
+
+  static String of(IndexDefinitions.Index index) {
     return of(index.getSettings().getAsMap(), index.getTypes());
   }
 
-  String of(Map... maps) {
+  private static String of(Map... maps) {
     StringBuilder sb = new StringBuilder();
     for (Map map : maps) {
       appendMap(sb, map);
@@ -49,7 +52,7 @@ class IndexDefinitionHash {
     return DigestUtils.sha256Hex(sb.toString());
   }
 
-  private void appendObject(StringBuilder sb, Object value) {
+  private static void appendObject(StringBuilder sb, Object value) {
     if (value instanceof IndexDefinitions.IndexType) {
       appendIndexType(sb, (IndexDefinitions.IndexType) value);
     } else if (value instanceof Map) {
@@ -61,11 +64,11 @@ class IndexDefinitionHash {
     }
   }
 
-  private void appendIndexType(StringBuilder sb, IndexDefinitions.IndexType type) {
+  private static void appendIndexType(StringBuilder sb, IndexDefinitions.IndexType type) {
     appendMap(sb, type.getAttributes());
   }
 
-  private void appendMap(StringBuilder sb, Map attributes) {
+  private static void appendMap(StringBuilder sb, Map attributes) {
     for (Object entry : sort(attributes).entrySet()) {
       sb.append(((Map.Entry) entry).getKey());
       sb.append(DELIMITER);
@@ -74,7 +77,7 @@ class IndexDefinitionHash {
     }
   }
 
-  private void appendIterable(StringBuilder sb, Iterable value) {
+  private static void appendIterable(StringBuilder sb, Iterable value) {
     List sorted = Lists.newArrayList(value);
     Collections.sort(sorted);
     for (Object o : sorted) {
@@ -83,7 +86,7 @@ class IndexDefinitionHash {
     }
   }
 
-  private SortedMap sort(Map map) {
+  private static SortedMap sort(Map map) {
     return ImmutableSortedMap.copyOf(map);
   }
 }
index c30695280bcfb061faa3fa7ede27907f71f26558..cabbe77ea37441b3db22153e3efe2641f476125b 100644 (file)
@@ -22,15 +22,13 @@ package org.sonar.server.es;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
-import org.elasticsearch.action.admin.indices.close.CloseIndexAction;
-import org.elasticsearch.action.admin.indices.open.OpenIndexAction;
 import org.elasticsearch.client.Client;
-import org.elasticsearch.common.settings.Settings.Builder;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.api.config.Configuration;
 import org.sonar.api.utils.log.Logger;
 import org.sonar.api.utils.log.Loggers;
 import org.sonar.api.utils.log.Profiler;
+import org.sonar.server.es.metadata.MetadataIndex;
 
 import static java.util.Arrays.stream;
 import static java.util.stream.Collectors.toSet;
@@ -38,15 +36,16 @@ import static java.util.stream.Collectors.toSet;
 public class IndexerStartupTask {
 
   private static final Logger LOG = Loggers.get(IndexerStartupTask.class);
-  private static final String SETTING_PREFIX_INITIAL_INDEXING_FINISHED = "sonarqube_initial_indexing_finished.";
 
   private final EsClient esClient;
   private final Configuration config;
+  private final MetadataIndex metadataIndex;
   private final StartupIndexer[] indexers;
 
-  public IndexerStartupTask(EsClient esClient, Configuration config, StartupIndexer... indexers) {
+  public IndexerStartupTask(EsClient esClient, Configuration config, MetadataIndex metadataIndex, StartupIndexer... indexers) {
     this.esClient = esClient;
     this.config = config;
+    this.metadataIndex = metadataIndex;
     this.indexers = indexers;
   }
 
@@ -62,58 +61,24 @@ public class IndexerStartupTask {
   }
 
   private void indexUninitializedTypes(StartupIndexer indexer) {
-    Set<IndexType> uninizializedTypes = getUninitializedTypes(indexer);
-    if (!uninizializedTypes.isEmpty()) {
+    Set<IndexType> uninitializedTypes = getUninitializedTypes(indexer);
+    if (!uninitializedTypes.isEmpty()) {
       Profiler profiler = Profiler.create(LOG);
-      profiler.startInfo(getLogMessage(uninizializedTypes, "..."));
-      indexer.indexOnStartup(uninizializedTypes);
-      uninizializedTypes.forEach(this::setInitialized);
-      profiler.stopInfo(getLogMessage(uninizializedTypes, "done"));
+      profiler.startInfo(getLogMessage(uninitializedTypes, "..."));
+      indexer.indexOnStartup(uninitializedTypes);
+      uninitializedTypes.forEach(this::setInitialized);
+      profiler.stopInfo(getLogMessage(uninitializedTypes, "done"));
     }
   }
 
   private Set<IndexType> getUninitializedTypes(StartupIndexer indexer) {
-    return indexer.getIndexTypes().stream().filter(this::isUninitialized).collect(toSet());
-  }
-
-  private boolean isUninitialized(IndexType indexType) {
-    return isUninitialized(indexType, esClient);
-  }
-
-  private static boolean isUninitialized(IndexType indexType, EsClient esClient) {
-    String setting = esClient.nativeClient().admin().indices().prepareGetSettings(indexType.getIndex()).get().getSetting(indexType.getIndex(),
-      getInitializedSettingName(indexType));
-    return !"true".equals(setting);
+    return indexer.getIndexTypes().stream().filter(indexType -> !metadataIndex.getInitialized(indexType)).collect(toSet());
   }
 
   private void setInitialized(IndexType indexType) {
     String index = indexType.getIndex();
     waitForIndexGreen(index);
-    closeIndex(index);
-    setIndexSetting(index, getInitializedSettingName(indexType), true);
-    openIndex(index);
-    waitForIndexYellow(index);
-  }
-
-  private void closeIndex(String index) {
-    Client nativeClient = esClient.nativeClient();
-    CloseIndexAction.INSTANCE.newRequestBuilder(nativeClient).setIndices(index).get();
-  }
-
-  private void setIndexSetting(String index, String name, boolean value) {
-    Client nativeClient = esClient.nativeClient();
-    Builder setting = org.elasticsearch.common.settings.Settings.builder().put(name, value);
-    nativeClient.admin().indices().prepareUpdateSettings(index).setSettings(setting).get();
-  }
-
-  private void openIndex(String index) {
-    Client nativeClient = esClient.nativeClient();
-    OpenIndexAction.INSTANCE.newRequestBuilder(nativeClient).setIndices(index).get();
-  }
-
-  private void waitForIndexYellow(String index) {
-    Client nativeClient = esClient.nativeClient();
-    ClusterHealthAction.INSTANCE.newRequestBuilder(nativeClient).setIndices(index).setWaitForYellowStatus().get(TimeValue.timeValueMinutes(10));
+    metadataIndex.setInitialized(indexType, true);
   }
 
   private void waitForIndexGreen(String index) {
@@ -121,10 +86,6 @@ public class IndexerStartupTask {
     ClusterHealthAction.INSTANCE.newRequestBuilder(nativeClient).setIndices(index).setWaitForGreenStatus().get(TimeValue.timeValueMinutes(10));
   }
 
-  private static String getInitializedSettingName(IndexType indexType) {
-    return "index." + SETTING_PREFIX_INITIAL_INDEXING_FINISHED + indexType.getType();
-  }
-
   private String getLogMessage(Set<IndexType> emptyTypes, String suffix) {
     String s = emptyTypes.size() == 1 ? "" : "s";
     String typeList = emptyTypes.stream().map(Object::toString).collect(Collectors.joining(","));
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndex.java b/server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndex.java
new file mode 100644 (file)
index 0000000..c0d8155
--- /dev/null
@@ -0,0 +1,79 @@
+/*
+ * SonarQube
+ * Copyright (C) 2009-2017 SonarSource SA
+ * mailto:info AT sonarsource DOT com
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 3 of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public License
+ * along with this program; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+package org.sonar.server.es.metadata;
+
+import java.util.Optional;
+import org.elasticsearch.action.get.GetRequestBuilder;
+import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.index.get.GetField;
+import org.sonar.server.es.EsClient;
+import org.sonar.server.es.IndexType;
+
+public class MetadataIndex {
+
+  private final EsClient esClient;
+
+  public MetadataIndex(EsClient esClient) {
+    this.esClient = esClient;
+  }
+
+  public Optional<String> getHash(String index) {
+    return getMetadata(hashId(index));
+  }
+
+  public void setHash(String index, String hash) {
+    setMetadata(hash, hashId(index));
+  }
+
+  private static String hashId(String index) {
+    return index + ".indexStructure";
+  }
+
+  public boolean getInitialized(IndexType indexType) {
+    return getMetadata(initializedId(indexType)).map(Boolean::parseBoolean).orElse(false);
+  }
+
+  public void setInitialized(IndexType indexType, boolean initialized) {
+    setMetadata(String.valueOf(initialized), initializedId(indexType));
+  }
+
+  private static String initializedId(IndexType indexType) {
+    return indexType.getIndex() + "." + indexType.getType() + ".initialized";
+  }
+
+  private Optional<String> getMetadata(String id) {
+    GetRequestBuilder request = esClient.prepareGet(MetadataIndexDefinition.INDEX_TYPE_METADATA, id).setFields(MetadataIndexDefinition.FIELD_VALUE);
+    GetResponse response = request.get();
+    if (response.isExists()) {
+      GetField field = response.getField(MetadataIndexDefinition.FIELD_VALUE);
+      String value = String.valueOf(field.getValue());
+      return Optional.of(value);
+    }
+    return Optional.empty();
+  }
+
+  private void setMetadata(String hash, String id) {
+    esClient.prepareIndex(MetadataIndexDefinition.INDEX_TYPE_METADATA)
+      .setId(id)
+      .setSource(MetadataIndexDefinition.FIELD_VALUE, hash)
+      .setRefresh(true)
+      .get();
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndexDefinition.java b/server/sonar-server/src/main/java/org/sonar/server/es/metadata/MetadataIndexDefinition.java
new file mode 100644 (file)
index 0000000..469f16a
--- /dev/null
@@ -0,0 +1,49 @@
+/*
+ * SonarQube
+ * Copyright (C) 2009-2017 SonarSource SA
+ * mailto:info AT sonarsource DOT com
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 3 of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public License
+ * along with this program; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+package org.sonar.server.es.metadata;
+
+import org.sonar.api.config.Configuration;
+import org.sonar.server.es.IndexDefinition.IndexDefinitionContext;
+import org.sonar.server.es.IndexType;
+import org.sonar.server.es.NewIndex;
+
+public class MetadataIndexDefinition {
+
+  public static final IndexType INDEX_TYPE_METADATA = new IndexType("metadatas", "metadata");
+  public static final String FIELD_VALUE = "value";
+
+  private static final int DEFAULT_NUMBER_OF_SHARDS = 1;
+
+  private final Configuration configuration;
+
+  public MetadataIndexDefinition(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  public void define(IndexDefinitionContext context) {
+    NewIndex index = context.create(INDEX_TYPE_METADATA.getIndex());
+    index.refreshHandledByIndexer();
+    index.configureShards(configuration, DEFAULT_NUMBER_OF_SHARDS);
+
+    NewIndex.NewIndexType mapping = index.createType(INDEX_TYPE_METADATA.getType());
+
+    mapping.stringFieldBuilder(FIELD_VALUE).disableSearch().build();
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/metadata/package-info.java b/server/sonar-server/src/main/java/org/sonar/server/es/metadata/package-info.java
new file mode 100644 (file)
index 0000000..83e47cf
--- /dev/null
@@ -0,0 +1,23 @@
+/*
+ * SonarQube
+ * Copyright (C) 2009-2017 SonarSource SA
+ * mailto:info AT sonarsource DOT com
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 3 of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public License
+ * along with this program; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+@ParametersAreNonnullByDefault
+package org.sonar.server.es.metadata;
+
+import javax.annotation.ParametersAreNonnullByDefault;
index 5a8c8ca0a44d3a407ac12e0fbc8926429b208834..2dd4f1f7378c9d3314c0ae90093beeb6ace9ab2a 100644 (file)
@@ -55,6 +55,8 @@ import org.sonar.server.es.IndexCreator;
 import org.sonar.server.es.IndexDefinitions;
 import org.sonar.server.es.ProjectIndexersImpl;
 import org.sonar.server.es.RecoveryIndexer;
+import org.sonar.server.es.metadata.MetadataIndex;
+import org.sonar.server.es.metadata.MetadataIndexDefinition;
 import org.sonar.server.event.NewAlerts;
 import org.sonar.server.favorite.FavoriteModule;
 import org.sonar.server.issue.AddTagsAction;
@@ -230,7 +232,10 @@ public class PlatformLevel4 extends PlatformLevel {
 
   @Override
   protected void configureLevel() {
-    addIfStartupLeader(IndexCreator.class);
+    addIfStartupLeader(
+      IndexCreator.class,
+      MetadataIndexDefinition.class,
+      MetadataIndex.class);
 
     add(
       PluginDownloader.class,
index 077d997c2e86375d34d1968ba934ed80d232a84f..9be16c06f721870ad2c19f1de03610c6bf903cad 100644 (file)
@@ -52,6 +52,8 @@ import org.junit.rules.ExternalResource;
 import org.sonar.api.config.internal.MapSettings;
 import org.sonar.core.config.ConfigurationProvider;
 import org.sonar.core.platform.ComponentContainer;
+import org.sonar.server.es.metadata.MetadataIndex;
+import org.sonar.server.es.metadata.MetadataIndexDefinition;
 
 import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.collect.Lists.newArrayList;
@@ -79,6 +81,8 @@ public class EsTester extends ExternalResource {
       container.addSingleton(client);
       container.addSingleton(IndexDefinitions.class);
       container.addSingleton(IndexCreator.class);
+      container.addSingleton(MetadataIndex.class);
+      container.addSingleton(MetadataIndexDefinition.class);
       container.startComponents();
     }
   }
index cb4c73c9c5a7a91c7cc4321d830f1bafe52f8da0..6cd13fcef3fde56977f85f877cf05e971aaf2058 100644 (file)
@@ -22,19 +22,27 @@ package org.sonar.server.es;
 import java.io.IOException;
 import java.util.Map;
 import javax.annotation.CheckForNull;
-import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
 import org.elasticsearch.cluster.metadata.MappingMetaData;
 import org.elasticsearch.common.collect.ImmutableOpenMap;
 import org.junit.Rule;
 import org.junit.Test;
 import org.sonar.api.config.internal.MapSettings;
+import org.sonar.server.es.metadata.MetadataIndex;
+import org.sonar.server.es.metadata.MetadataIndexDefinition;
 
 import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
 
 public class IndexCreatorTest {
 
   @Rule
   public EsTester es = new EsTester();
+  private MetadataIndexDefinition metadataIndexDefinition = new MetadataIndexDefinition(new MapSettings().asConfig());
+  private MetadataIndex metadataIndex = new MetadataIndex(es.client());
 
   @Test
   public void create_index() throws Exception {
@@ -42,7 +50,7 @@ public class IndexCreatorTest {
 
     IndexDefinitions registry = new IndexDefinitions(new IndexDefinition[] {new FakeIndexDefinition()}, new MapSettings().asConfig());
     registry.start();
-    IndexCreator creator = new IndexCreator(es.client(), registry);
+    IndexCreator creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndex);
     creator.start();
 
     // check that index is created with related mapping
@@ -53,13 +61,29 @@ public class IndexCreatorTest {
     assertThat(countMappingFields(mapping)).isEqualTo(2);
     assertThat(field(mapping, "updatedAt").get("type")).isEqualTo("date");
 
-    assertThat(setting("fakes", "index.sonar_hash")).isNotEmpty();
-
     // of course do not delete indices on stop
     creator.stop();
     assertThat(mappings()).isNotEmpty();
   }
 
+  @Test
+  public void mark_all_non_existing_index_types_as_uninitialized() throws Exception {
+    MetadataIndex metadataIndexMock = mock(MetadataIndex.class);
+    IndexDefinitions registry = new IndexDefinitions(new IndexDefinition[] {context -> {
+      NewIndex i = context.create("i");
+      i.createType("t1");
+      i.createType("t2");
+    }}, new MapSettings().asConfig());
+    registry.start();
+    IndexCreator creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndexMock);
+    creator.start();
+
+    verify(metadataIndexMock).setHash(eq("i"), anyString());
+    verify(metadataIndexMock).setInitialized(eq(new IndexType("i", "t1")), eq(false));
+    verify(metadataIndexMock).setInitialized(eq(new IndexType("i", "t2")), eq(false));
+    verifyNoMoreInteractions(metadataIndexMock);
+  }
+
   @Test
   public void recreate_index_on_definition_changes() throws Exception {
     assertThat(mappings()).isEmpty();
@@ -67,30 +91,54 @@ public class IndexCreatorTest {
     // v1
     IndexDefinitions registry = new IndexDefinitions(new IndexDefinition[] {new FakeIndexDefinition()}, new MapSettings().asConfig());
     registry.start();
-    IndexCreator creator = new IndexCreator(es.client(), registry);
+    IndexCreator creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndex);
     creator.start();
     creator.stop();
-    String hashV1 = setting("fakes", "index.sonar_hash");
-    assertThat(hashV1).isNotEmpty();
+
+    IndexType fakeIndexType = new IndexType("fakes", "fake");
+    String id = "1";
+    es.client().prepareIndex(fakeIndexType).setId(id).setSource(new FakeDoc().getFields()).setRefresh(true).get();
+    assertThat(es.client().prepareGet(fakeIndexType, id).get().isExists()).isTrue();
 
     // v2
     registry = new IndexDefinitions(new IndexDefinition[] {new FakeIndexDefinitionV2()}, new MapSettings().asConfig());
     registry.start();
-    creator = new IndexCreator(es.client(), registry);
+    creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndex);
     creator.start();
     ImmutableOpenMap<String, ImmutableOpenMap<String, MappingMetaData>> mappings = mappings();
     MappingMetaData mapping = mappings.get("fakes").get("fake");
     assertThat(countMappingFields(mapping)).isEqualTo(3);
     assertThat(field(mapping, "updatedAt").get("type")).isEqualTo("date");
     assertThat(field(mapping, "newField").get("type")).isEqualTo("integer");
-    String hashV2 = setting("fakes", "index.sonar_hash");
-    assertThat(hashV2).isNotEqualTo(hashV1);
     creator.stop();
+
+    assertThat(es.client().prepareGet(fakeIndexType, id).get().isExists()).isFalse();
   }
 
-  private String setting(String indexName, String settingKey) {
-    GetSettingsResponse indexSettings = es.client().nativeClient().admin().indices().prepareGetSettings(indexName).get();
-    return indexSettings.getSetting(indexName, settingKey);
+  @Test
+  public void do_not_recreate_index_on_unchanged_definition() throws Exception {
+    assertThat(mappings()).isEmpty();
+
+    // v1
+    IndexDefinitions registry = new IndexDefinitions(new IndexDefinition[] {new FakeIndexDefinition()}, new MapSettings().asConfig());
+    registry.start();
+    IndexCreator creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndex);
+    creator.start();
+    creator.stop();
+
+    IndexType fakeIndexType = new IndexType("fakes", "fake");
+    String id = "1";
+    es.client().prepareIndex(fakeIndexType).setId(id).setSource(new FakeDoc().getFields()).setRefresh(true).get();
+    assertThat(es.client().prepareGet(fakeIndexType, id).get().isExists()).isTrue();
+
+    // v1
+    registry = new IndexDefinitions(new IndexDefinition[] {new FakeIndexDefinition()}, new MapSettings().asConfig());
+    registry.start();
+    creator = new IndexCreator(es.client(), registry, metadataIndexDefinition, metadataIndex);
+    creator.start();
+    creator.stop();
+
+    assertThat(es.client().prepareGet(fakeIndexType, id).get().isExists()).isTrue();
   }
 
   private ImmutableOpenMap<String, ImmutableOpenMap<String, MappingMetaData>> mappings() {
index 946543195b99518b1e6e9bd5c2cd027143459ff1..06d8bb1fdb12534bf7f15f924ce42b29d40dcf67 100644 (file)
@@ -30,14 +30,14 @@ public class IndexDefinitionHashTest {
   @Test
   public void of() {
     IndexDefinitions.Index indexV1 = new IndexDefinitions.Index(createIndex());
-    String hashV1 = new IndexDefinitionHash().of(indexV1);
+    String hashV1 = IndexDefinitionHash.of(indexV1);
     assertThat(hashV1).isNotEmpty();
     // always the same
-    assertThat(hashV1).isEqualTo(new IndexDefinitionHash().of(indexV1));
+    assertThat(hashV1).isEqualTo(IndexDefinitionHash.of(indexV1));
 
     NewIndex newIndexV2 = createIndex();
     newIndexV2.getTypes().get("fake").createIntegerField("max");
-    String hashV2 = new IndexDefinitionHash().of(new IndexDefinitions.Index(newIndexV2));
+    String hashV2 = IndexDefinitionHash.of(new IndexDefinitions.Index(newIndexV2));
     assertThat(hashV2).isNotEmpty().isNotEqualTo(hashV1);
   }
 
index ddce70f478ff7b3b3d0a0621143dd06b2cd04af6..7f536617d19541ca1e885ffc3099e4527caa9eda 100644 (file)
 package org.sonar.server.es;
 
 import com.google.common.collect.ImmutableSet;
+import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.sonar.api.config.internal.MapSettings;
-import org.sonar.api.utils.System2;
-import org.sonar.db.DbTester;
+import org.sonar.server.es.metadata.MetadataIndex;
 
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
@@ -35,58 +36,56 @@ import static org.sonar.server.es.FakeIndexDefinition.INDEX_TYPE_FAKE;
 
 public class IndexerStartupTaskTest {
 
-  private System2 system2 = System2.INSTANCE;
-  private MapSettings settings = new MapSettings();
-
-  @Rule
-  public DbTester db = DbTester.create(system2);
-
   @Rule
   public EsTester es = new EsTester(new FakeIndexDefinition());
 
-  @Test
-  public void only_index_once() throws Exception {
-    insertDocumentIntoIndex();
+  private final MapSettings settings = new MapSettings();
+  private final MetadataIndex metadataIndex = mock(MetadataIndex.class);
+  private final StartupIndexer indexer = mock(StartupIndexer.class);
+  private final IndexerStartupTask underTest = new IndexerStartupTask(es.client(), settings.asConfig(), metadataIndex, indexer);
 
-    StartupIndexer indexer1 = createIndexer();
-    emulateStartup(indexer1);
+  @Before
+  public void setUp() throws Exception {
+    doReturn(ImmutableSet.of(INDEX_TYPE_FAKE)).when(indexer).getIndexTypes();
+  }
 
-    // do index on first run
-    verify(indexer1).getIndexTypes();
-    verify(indexer1).indexOnStartup(Mockito.eq(ImmutableSet.of(INDEX_TYPE_FAKE)));
+  @Test
+  public void index_if_not_initialized() throws Exception {
+    doReturn(false).when(metadataIndex).getInitialized(INDEX_TYPE_FAKE);
 
-    StartupIndexer indexer2 = createIndexer();
-    emulateStartup(indexer2);
+    underTest.execute();
 
-    // do not index on second run
-    verify(indexer2).getIndexTypes();
-    verifyNoMoreInteractions(indexer2);
+    verify(indexer).getIndexTypes();
+    verify(indexer).indexOnStartup(Mockito.eq(ImmutableSet.of(INDEX_TYPE_FAKE)));
   }
 
   @Test
-  public void do_not_index_if_indexes_are_disabled() throws Exception {
-    settings.setProperty("sonar.internal.es.disableIndexes", "true");
+  public void set_initialized_after_indexation() throws Exception {
+    doReturn(false).when(metadataIndex).getInitialized(INDEX_TYPE_FAKE);
 
-    insertDocumentIntoIndex();
+    underTest.execute();
 
-    StartupIndexer indexer = createIndexer();
-    emulateStartup(indexer);
+    verify(metadataIndex).setInitialized(eq(INDEX_TYPE_FAKE), eq(true));
+  }
 
-    // do not index
+  @Test
+  public void do_not_index_if_already_initialized() throws Exception {
+    doReturn(true).when(metadataIndex).getInitialized(INDEX_TYPE_FAKE);
+
+    underTest.execute();
+
+    verify(indexer).getIndexTypes();
     verifyNoMoreInteractions(indexer);
   }
 
-  private void insertDocumentIntoIndex() {
+  @Test
+  public void do_not_index_if_indexes_are_disabled() throws Exception {
+    settings.setProperty("sonar.internal.es.disableIndexes", "true");
     es.putDocuments(INDEX_TYPE_FAKE, new FakeDoc());
-  }
 
-  private StartupIndexer createIndexer() {
-    StartupIndexer indexer = mock(StartupIndexer.class);
-    doReturn(ImmutableSet.of(INDEX_TYPE_FAKE)).when(indexer).getIndexTypes();
-    return indexer;
-  }
+    underTest.execute();
 
-  private void emulateStartup(StartupIndexer indexer) {
-    new IndexerStartupTask(es.client(), settings.asConfig(), indexer).execute();
+    // do not index
+    verifyNoMoreInteractions(indexer);
   }
 }
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/metadata/MetadataIndexTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/metadata/MetadataIndexTest.java
new file mode 100644 (file)
index 0000000..13cb0dc
--- /dev/null
@@ -0,0 +1,64 @@
+/*
+ * SonarQube
+ * Copyright (C) 2009-2017 SonarSource SA
+ * mailto:info AT sonarsource DOT com
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 3 of the License, or (at your option) any later version.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public License
+ * along with this program; if not, write to the Free Software Foundation,
+ * Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301, USA.
+ */
+package org.sonar.server.es.metadata;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.sonar.server.es.EsTester;
+import org.sonar.server.es.FakeIndexDefinition;
+import org.sonar.server.es.IndexType;
+
+import static org.apache.commons.lang.RandomStringUtils.randomAlphanumeric;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class MetadataIndexTest {
+
+  @Rule
+  public EsTester es = new EsTester(new FakeIndexDefinition());
+  private final MetadataIndex underTest = new MetadataIndex(es.client());
+  private final String index = randomAlphanumeric(20);
+  private final String type = randomAlphanumeric(20);
+
+
+  @Test
+  public void type_should_be_not_initialized_by_default() throws Exception {
+    IndexType indexType = new IndexType("examples", "example");
+    assertThat(underTest.getInitialized(indexType)).isFalse();
+  }
+
+  @Test
+  public void type_should_be_initialized_after_explicitly_set_to_initialized() throws Exception {
+    IndexType indexType = new IndexType("examples", "example");
+    underTest.setInitialized(indexType, true);
+    assertThat(underTest.getInitialized(indexType)).isTrue();
+  }
+
+  @Test
+  public void hash_should_be_empty_by_default() throws Exception {
+    assertThat(underTest.getHash(index)).isEmpty();
+  }
+
+  @Test
+  public void hash_should_be_able_to_be_automatically_set() throws Exception {
+    String hash = randomAlphanumeric(20);
+    underTest.setHash(index, hash);
+    assertThat(underTest.getHash(index)).hasValue(hash);
+  }
+}
\ No newline at end of file
index 0230612bdefcc979e52286678e1762e06d737fea..345d23793ccc15e3d09c950fb5e7e2c175ba4008 100644 (file)
@@ -71,8 +71,7 @@ public class EsMonitorTest {
     Map indicesAttributes = (Map) attributes.get("Indices");
 
     // one index "issues"
-    assertThat(indicesAttributes).hasSize(1);
-    Map indexAttributes = (Map) indicesAttributes.values().iterator().next();
+    Map indexAttributes = (Map) indicesAttributes.get(IssueIndexDefinition.INDEX_TYPE_ISSUE.getIndex());
     assertThat(indexAttributes.get("Docs")).isEqualTo(0L);
     assertThat((int) indexAttributes.get("Shards")).isGreaterThan(0);
     assertThat(indexAttributes.get("Store Size")).isNotNull();