]> source.dussan.org Git - sonarqube.git/commitdiff
Refactor the way to declare ES indices.
authorSimon Brandhof <simon.brandhof@sonarsource.com>
Sun, 16 Nov 2014 21:45:40 +0000 (22:45 +0100)
committerSimon Brandhof <simon.brandhof@sonarsource.com>
Wed, 19 Nov 2014 07:27:08 +0000 (08:27 +0100)
- BaseIndex has too many responsibilities
- SearchClient must not override TransportClient, else it can't be easily used in tests

46 files changed:
server/sonar-search/src/main/java/org/sonar/search/SearchServer.java
server/sonar-search/src/main/java/org/sonar/search/SearchSettings.java
server/sonar-server/pom.xml
server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexRequestIterator.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexer.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/DefaultMappingSettings.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/EsClient.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/IndexCreator.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/IndexDefinition.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/IndexHash.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/IssueIndexDefinition.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/NewIndex.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/es/package-info.java [new file with mode: 0644]
server/sonar-server/src/main/java/org/sonar/server/issue/index/IssueAuthorizationIndex.java
server/sonar-server/src/main/java/org/sonar/server/issue/index/IssueIndex.java
server/sonar-server/src/main/java/org/sonar/server/platform/ServerComponents.java
server/sonar-server/src/main/java/org/sonar/server/search/ClusterHealth.java
server/sonar-server/src/main/java/org/sonar/server/search/SearchClient.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyBulkRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyClusterHealthRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyClusterStateRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyClusterStatsRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyCountRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyCreateIndexRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyDeleteByQueryRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyFlushRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyGetRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyIndicesExistsRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyIndicesStatsRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyMultiGetRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyNodesStatsRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyPutMappingRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxyRefreshRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxySearchRequestBuilder.java
server/sonar-server/src/main/java/org/sonar/server/search/request/ProxySearchScrollRequestBuilder.java
server/sonar-server/src/test/java/org/sonar/server/es/BulkIndexRequestIteratorTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/DefaultMappingSettingsTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/EsClientTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/EsTester.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/IndexCreatorTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/IndexHashTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/IssueIndexDefinitionTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/es/NewIndexTest.java [new file with mode: 0644]
server/sonar-server/src/test/java/org/sonar/server/issue/index/IssueIndexMediumTest.java
sonar-core/src/test/java/org/sonar/core/persistence/DatabaseCommands.java
sonar-core/src/test/java/org/sonar/core/persistence/TestDatabase.java

index 735634a9e1c8868adb6c159d23effda5095ce0f2..c1f8a1e329f1f84c8798088e89814220b05826e5 100644 (file)
@@ -20,6 +20,7 @@
 package org.sonar.search;
 
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.hppc.cursors.ObjectCursor;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
@@ -55,7 +56,7 @@ public class SearchServer implements Monitored {
       for (ObjectCursor<Settings> settingCursor : node.client().admin().indices()
         .prepareGetSettings().get().getIndexToSettings().values()) {
         Settings settings = settingCursor.value;
-        String clusterReplicationFactor = settings.get("index.number_of_replicas", "-1");
+        String clusterReplicationFactor = settings.get(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "-1");
         if (Integer.parseInt(clusterReplicationFactor) <= 0) {
           node.stop();
           throw new MessageException("Invalid number of Elasticsearch replicas: " + clusterReplicationFactor);
index 415c48ee793a08a585be5b3e470d9a736b84a05d..2e03649ea92d09d5a56ca65c4abdf3bc980caa58 100644 (file)
@@ -20,6 +20,7 @@
 package org.sonar.search;
 
 import org.apache.commons.lang.StringUtils;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.slf4j.Logger;
@@ -147,10 +148,7 @@ class SearchSettings {
     builder
       .put("index.number_of_shards", "1")
       .put("index.refresh_interval", "30s")
-      .put("index.store.type", "mmapfs")
-      .put("indices.store.throttle.type", "none")
-      .put("index.merge.scheduler.max_thread_count",
-        Math.max(1, Math.min(3, Runtime.getRuntime().availableProcessors() / 2)));
+      .put("indices.store.throttle.type", "none");
   }
 
   private void configureCluster(ImmutableSettings.Builder builder) {
@@ -161,18 +159,16 @@ class SearchSettings {
         LOGGER.info("Elasticsearch cluster enabled. Master node.");
         builder.put("node.master", true);
       } else if (!masterHosts.isEmpty()) {
-        LOGGER.info("Elasticsearch cluster enabled. Slave node connecting to master: {}", masterHosts);
+        LOGGER.info("Elasticsearch cluster enabled. Node connecting to master: {}", masterHosts);
         builder.put("discovery.zen.ping.unicast.hosts", StringUtils.join(masterHosts, ","));
         builder.put("node.master", false);
-
-        // Enforce a N/2+1 number of masters in cluster
         builder.put("discovery.zen.minimum_master_nodes", 1);
       } else {
         throw new MessageException(String.format("Not an Elasticsearch master nor slave. Please check properties %s and %s",
           ProcessConstants.CLUSTER_MASTER, ProcessConstants.CLUSTER_MASTER_HOST));
       }
     }
-    builder.put("index.number_of_replicas", replicationFactor);
+    builder.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicationFactor);
     builder.put("cluster.name", clusterName);
     builder.put("cluster.routing.allocation.awareness.attributes", "rack_id");
     builder.put("node.rack_id", props.value(ProcessConstants.CLUSTER_NODE_NAME, "unknown"));
index e55f9054241ee5ffcdccec06779097fb219b97da..4c826b8542f5a383c8a29b6960a3e3289de009ff 100644 (file)
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.easytesting</groupId>
-      <artifactId>fest-assert</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.hamcrest</groupId>
-      <artifactId>hamcrest-all</artifactId>
+      <groupId>org.codehaus.sonar</groupId>
+      <artifactId>sonar-testing-harness</artifactId>
       <scope>test</scope>
     </dependency>
     <dependency>
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexRequestIterator.java b/server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexRequestIterator.java
new file mode 100644 (file)
index 0000000..966f868
--- /dev/null
@@ -0,0 +1,75 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.action.ActionRequest;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+public class BulkIndexRequestIterator<INPUT> implements Iterator<ActionRequest> {
+
+  public static interface InputConverter<INPUT> {
+    List<ActionRequest> convert(INPUT input);
+  }
+
+  private final Iterator<INPUT> input;
+  private final InputConverter<INPUT> converter;
+  private Iterator<ActionRequest> currents = null;
+
+  public BulkIndexRequestIterator(Iterable<INPUT> input, InputConverter<INPUT> converter) {
+    this.input = input.iterator();
+    this.converter = converter;
+    if (this.input.hasNext()) {
+      this.currents = converter.convert(this.input.next()).iterator();
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    return currents != null && currents.hasNext();
+  }
+
+  @Override
+  public ActionRequest next() {
+    if (currents == null) {
+      throw new NoSuchElementException();
+    }
+    ActionRequest request = currents.next();
+    peekNext();
+    return request;
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  private void peekNext() {
+    if (!currents.hasNext()) {
+      if (input.hasNext()) {
+        currents = converter.convert(input.next()).iterator();
+      } else {
+        currents = null;
+      }
+    }
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexer.java b/server/sonar-server/src/main/java/org/sonar/server/es/BulkIndexer.java
new file mode 100644 (file)
index 0000000..895e7d7
--- /dev/null
@@ -0,0 +1,121 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang.StringUtils;
+import org.elasticsearch.action.ActionRequest;
+import org.elasticsearch.action.admin.indices.settings.get.GetSettingsRequestBuilder;
+import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequestBuilder;
+import org.elasticsearch.action.bulk.BulkRequest;
+import org.elasticsearch.action.bulk.BulkResponse;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.unit.ByteSizeValue;
+import org.sonar.server.search.SearchClient;
+
+import java.util.Iterator;
+
+/**
+ *
+ */
+public class BulkIndexer {
+
+  private static final long FLUSH_BYTE_SIZE = new ByteSizeValue(5, ByteSizeUnit.MB).bytes();
+  
+  private final SearchClient client;
+
+  public BulkIndexer(SearchClient client) {
+    this.client = client;
+  }
+
+  /**
+   * Heavy operation that populates an index from scratch. Replicas are disabled during
+   * the bulk indexation and lucene segments are optimized at the end. No need
+   * to call {@link #refresh(String)} after this method.
+   * 
+   * @see BulkIndexRequestIterator
+   */
+  public void fullIndex(String index, Iterator<ActionRequest> requests) {
+    // deactivate replicas
+    GetSettingsRequestBuilder replicaRequest = client.admin().indices().prepareGetSettings(index);
+    String initialRequestSetting = replicaRequest.get().getSetting(index, IndexMetaData.SETTING_NUMBER_OF_REPLICAS);
+    int initialReplicas = Integer.parseInt(StringUtils.defaultIfEmpty(initialRequestSetting, "0"));
+    if (initialReplicas > 0) {
+      setNumberOfReplicas(index, 0);
+    }
+
+    index(requests);
+    refresh(index);
+    optimize(index);
+
+    if (initialReplicas > 0) {
+      // re-enable replicas
+      setNumberOfReplicas(index, initialReplicas);
+    }
+  }
+
+  private void setNumberOfReplicas(String index, int replicas) {
+    UpdateSettingsRequestBuilder req = client.admin().indices().prepareUpdateSettings(index);
+    req.setSettings(ImmutableMap.<String, Object>of(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, String.valueOf(replicas)));
+    req.get();
+  }
+
+  /**
+   * @see BulkIndexRequestIterator
+   */
+  public void index(Iterator<ActionRequest> requests) {
+    BulkRequest bulkRequest = client.prepareBulk().request();
+    while (requests.hasNext()) {
+      ActionRequest request = requests.next();
+      bulkRequest.add(request);
+      if (bulkRequest.estimatedSizeInBytes() >= FLUSH_BYTE_SIZE) {
+        executeBulk(bulkRequest);
+        bulkRequest = client.prepareBulk().request();
+      }
+    }
+    if (bulkRequest.numberOfActions() > 0) {
+      executeBulk(bulkRequest);
+    }
+  }
+
+  private void executeBulk(BulkRequest bulkRequest) {
+    try {
+      BulkResponse response = client.bulk(bulkRequest).get();
+
+      // TODO check failures
+      // WARNING - complexity of response#hasFailures() and #buildFailureMessages() is O(n)
+    } catch (Exception e) {
+      throw new IllegalStateException("TODO", e);
+    }
+  }
+
+  public void refresh(String index) {
+    client.prepareRefresh(index).get();
+  }
+
+  private void optimize(String index) {
+    client.admin().indices().prepareOptimize(index)
+      .setMaxNumSegments(1)
+      .setWaitForMerge(true)
+      .get();
+  }
+
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/DefaultMappingSettings.java b/server/sonar-server/src/main/java/org/sonar/server/es/DefaultMappingSettings.java
new file mode 100644 (file)
index 0000000..be5c670
--- /dev/null
@@ -0,0 +1,94 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.common.settings.ImmutableSettings;
+
+class DefaultMappingSettings {
+
+  private DefaultMappingSettings() {
+    // only static stuff
+  }
+
+  static ImmutableSettings.Builder defaults() {
+    return ImmutableSettings.builder()
+      .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
+      .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
+      .put("index.refresh_interval", "30s")
+      .put("index.mapper.dynamic", false)
+
+      // Sortable text analyzer
+      .put("index.analysis.analyzer.sortable.type", "custom")
+      .put("index.analysis.analyzer.sortable.tokenizer", "keyword")
+      .putArray("index.analysis.analyzer.sortable.filter", "trim", "lowercase")
+
+      // Edge NGram index-analyzer
+      .put("index.analysis.analyzer.index_grams.type", "custom")
+      .put("index.analysis.analyzer.index_grams.tokenizer", "whitespace")
+      .putArray("index.analysis.analyzer.index_grams.filter", "trim", "lowercase", "gram_filter")
+
+      // Edge NGram search-analyzer
+      .put("index.analysis.analyzer.search_grams.type", "custom")
+      .put("index.analysis.analyzer.search_grams.tokenizer", "whitespace")
+      .putArray("index.analysis.analyzer.search_grams.filter", "trim", "lowercase")
+
+      // Word index-analyzer
+      .put("index.analysis.analyzer.index_words.type", "custom")
+      .put("index.analysis.analyzer.index_words.tokenizer", "standard")
+      .putArray("index.analysis.analyzer.index_words.filter",
+        "standard", "word_filter", "lowercase", "stop", "asciifolding", "porter_stem")
+
+      // Word search-analyzer
+      .put("index.analysis.analyzer.search_words.type", "custom")
+      .put("index.analysis.analyzer.search_words.tokenizer", "standard")
+      .putArray("index.analysis.analyzer.search_words.filter",
+        "standard", "lowercase", "stop", "asciifolding", "porter_stem")
+
+      // Edge NGram filter
+      .put("index.analysis.filter.gram_filter.type", "edgeNGram")
+      .put("index.analysis.filter.gram_filter.min_gram", 2)
+      .put("index.analysis.filter.gram_filter.max_gram", 15)
+      .putArray("index.analysis.filter.gram_filter.token_chars", "letter", "digit", "punctuation", "symbol")
+
+      // Word filter
+      .put("index.analysis.filter.word_filter.type", "word_delimiter")
+      .put("index.analysis.filter.word_filter.generate_word_parts", true)
+      .put("index.analysis.filter.word_filter.catenate_words", true)
+      .put("index.analysis.filter.word_filter.catenate_numbers", true)
+      .put("index.analysis.filter.word_filter.catenate_all", true)
+      .put("index.analysis.filter.word_filter.split_on_case_change", true)
+      .put("index.analysis.filter.word_filter.preserve_original", true)
+      .put("index.analysis.filter.word_filter.split_on_numerics", true)
+      .put("index.analysis.filter.word_filter.stem_english_possessive", true)
+
+      // Path Analyzer
+      .put("index.analysis.analyzer.path_analyzer.type", "custom")
+      .put("index.analysis.analyzer.path_analyzer.tokenizer", "path_hierarchy")
+
+      // UUID Module analyzer
+      .put("index.analysis.tokenizer.dot_tokenizer.type", "pattern")
+      .put("index.analysis.tokenizer.dot_tokenizer.pattern", "\\.")
+      .put("index.analysis.analyzer.uuid_analyzer.type", "custom")
+      .putArray("index.analysis.analyzer.uuid_analyzer.filter", "trim", "lowercase")
+      .put("index.analysis.analyzer.uuid_analyzer.tokenizer", "dot_tokenizer");
+
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/EsClient.java b/server/sonar-server/src/main/java/org/sonar/server/es/EsClient.java
new file mode 100644 (file)
index 0000000..3227328
--- /dev/null
@@ -0,0 +1,182 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder;
+import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
+import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequestBuilder;
+import org.elasticsearch.action.admin.cluster.state.ClusterStateRequestBuilder;
+import org.elasticsearch.action.admin.cluster.stats.ClusterStatsRequestBuilder;
+import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
+import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
+import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequestBuilder;
+import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder;
+import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequestBuilder;
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder;
+import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder;
+import org.elasticsearch.action.bulk.BulkRequestBuilder;
+import org.elasticsearch.action.count.CountRequestBuilder;
+import org.elasticsearch.action.deletebyquery.DeleteByQueryRequestBuilder;
+import org.elasticsearch.action.get.GetRequestBuilder;
+import org.elasticsearch.action.get.MultiGetRequestBuilder;
+import org.elasticsearch.action.search.SearchRequestBuilder;
+import org.elasticsearch.action.search.SearchScrollRequestBuilder;
+import org.elasticsearch.client.Client;
+import org.picocontainer.Startable;
+import org.sonar.core.profiling.Profiling;
+import org.sonar.server.search.ClusterHealth;
+import org.sonar.server.search.SearchClient;
+import org.sonar.server.search.request.ProxyBulkRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterHealthRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterStateRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyCountRequestBuilder;
+import org.sonar.server.search.request.ProxyCreateIndexRequestBuilder;
+import org.sonar.server.search.request.ProxyDeleteByQueryRequestBuilder;
+import org.sonar.server.search.request.ProxyFlushRequestBuilder;
+import org.sonar.server.search.request.ProxyGetRequestBuilder;
+import org.sonar.server.search.request.ProxyIndicesExistsRequestBuilder;
+import org.sonar.server.search.request.ProxyIndicesStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyMultiGetRequestBuilder;
+import org.sonar.server.search.request.ProxyNodesStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyPutMappingRequestBuilder;
+import org.sonar.server.search.request.ProxyRefreshRequestBuilder;
+import org.sonar.server.search.request.ProxySearchRequestBuilder;
+import org.sonar.server.search.request.ProxySearchScrollRequestBuilder;
+
+/**
+ * Facade to connect to Elasticsearch node. Handles correctly errors (logging + exceptions
+ * with context) and profiling of requests.
+ */
+public class EsClient implements Startable {
+
+  private final Profiling profiling;
+  private final Client client;
+
+  public EsClient(SearchClient deprecatedClient) {
+    this.profiling = deprecatedClient.getProfiling();
+    this.client = deprecatedClient;
+  }
+
+  public EsClient(Profiling profiling, Client client) {
+    this.profiling = profiling;
+    this.client = client;
+  }
+
+  public ClusterHealth getClusterHealth() {
+    ClusterHealth health = new ClusterHealth();
+    ClusterStatsResponse clusterStatsResponse = this.prepareClusterStats().get();
+
+    // Cluster health
+    health.setClusterAvailable(clusterStatsResponse.getStatus() != ClusterHealthStatus.RED);
+
+    // Number of nodes
+    health.setNumberOfNodes(clusterStatsResponse.getNodesStats().getCounts().getTotal());
+
+    return health;
+  }
+
+
+  public RefreshRequestBuilder prepareRefresh(String... indices) {
+    return new ProxyRefreshRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public FlushRequestBuilder prepareFlush(String... indices) {
+    return new ProxyFlushRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public IndicesStatsRequestBuilder prepareStats(String... indices) {
+    return new ProxyIndicesStatsRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public NodesStatsRequestBuilder prepareNodesStats(String... nodesIds) {
+    return new ProxyNodesStatsRequestBuilder(client, profiling).setNodesIds(nodesIds);
+  }
+
+  public ClusterStatsRequestBuilder prepareClusterStats() {
+    return new ProxyClusterStatsRequestBuilder(client, profiling);
+  }
+
+  public ClusterStateRequestBuilder prepareState() {
+    return new ProxyClusterStateRequestBuilder(client, profiling);
+  }
+
+  public ClusterHealthRequestBuilder prepareHealth(String... indices) {
+    return new ProxyClusterHealthRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public IndicesExistsRequestBuilder prepareExists(String... indices) {
+    return new ProxyIndicesExistsRequestBuilder(client, profiling, indices);
+  }
+
+  public CreateIndexRequestBuilder prepareCreate(String index) {
+    return new ProxyCreateIndexRequestBuilder(client, profiling, index);
+  }
+
+  public PutMappingRequestBuilder preparePutMapping(String... indices) {
+    return new ProxyPutMappingRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public SearchRequestBuilder prepareSearch(String... indices) {
+    return new ProxySearchRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public SearchScrollRequestBuilder prepareSearchScroll(String scrollId) {
+    return new ProxySearchScrollRequestBuilder(scrollId, client, profiling);
+  }
+
+  public GetRequestBuilder prepareGet() {
+    return new ProxyGetRequestBuilder(client, profiling);
+  }
+
+  public GetRequestBuilder prepareGet(String index, String type, String id) {
+    return new ProxyGetRequestBuilder(client, profiling).setIndex(index).setType(type).setId(id);
+  }
+
+  public MultiGetRequestBuilder prepareMultiGet() {
+    return new ProxyMultiGetRequestBuilder(client, profiling);
+  }
+
+  public CountRequestBuilder prepareCount(String... indices) {
+    return new ProxyCountRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  public BulkRequestBuilder prepareBulk() {
+    return new ProxyBulkRequestBuilder(client, profiling);
+  }
+
+  public DeleteByQueryRequestBuilder prepareDeleteByQuery(String... indices) {
+    return new ProxyDeleteByQueryRequestBuilder(client, profiling).setIndices(indices);
+  }
+
+  @Override
+  public void start() {
+    // nothing to do
+  }
+
+  @Override
+  public void stop() {
+    client.close();
+  }
+
+  protected Client nativeClient() {
+    return client;
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/IndexCreator.java b/server/sonar-server/src/main/java/org/sonar/server/es/IndexCreator.java
new file mode 100644 (file)
index 0000000..01008a4
--- /dev/null
@@ -0,0 +1,117 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.apache.commons.lang.StringUtils;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.picocontainer.Startable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+/**
+ * Create registered indices in Elasticsearch.
+ */
+public class IndexCreator implements Startable {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(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 EsClient client;
+  private final IndexDefinition[] definitions;
+
+  public IndexCreator(EsClient client, IndexDefinition[] definitions) {
+    this.client = client;
+    this.definitions = definitions;
+  }
+
+  @Override
+  public void start() {
+    create();
+  }
+
+  @Override
+  public void stop() {
+    // nothing to do
+  }
+
+  public void create() {
+    // collect definitions
+    IndexDefinition.IndexDefinitionContext context = new IndexDefinition.IndexDefinitionContext();
+    for (IndexDefinition definition : definitions) {
+      definition.define(context);
+    }
+
+    // create indices that do not exist or that have a new definition (different mapping, cluster enabled, ...)
+    for (NewIndex newIndex : context.getIndices().values()) {
+      boolean exists = client.prepareExists(newIndex.getName()).get().isExists();
+      if (exists) {
+        if (needsToDeleteIndex(newIndex)) {
+          LOGGER.info(String.format("Delete index %s (settings changed)", newIndex.getName()));
+          deleteIndex(newIndex.getName());
+          exists = false;
+        }
+      }
+      if (!exists) {
+        createIndex(newIndex);
+      }
+    }
+  }
+
+  private void createIndex(NewIndex newIndex) {
+    LOGGER.info(String.format("Create index %s", newIndex.getName()));
+    ImmutableSettings.Builder settings = newIndex.getSettings();
+    settings.put(SETTING_HASH, new IndexHash().of(newIndex));
+    client
+      .prepareCreate(newIndex.getName())
+      .setSettings(settings)
+      .get();
+
+    // create types
+    for (Map.Entry<String, NewIndex.NewMapping> entry : newIndex.getMappings().entrySet()) {
+      LOGGER.info(String.format("Create type %s/%s", newIndex.getName(), entry.getKey()));
+      client.preparePutMapping(newIndex.getName())
+        .setType(entry.getKey())
+        .setIgnoreConflicts(false)
+        .setSource(entry.getValue().getAttributes())
+        .get();
+    }
+  }
+
+  private void deleteIndex(String indexName) {
+    client.nativeClient().admin().indices().prepareDelete(indexName).get();
+  }
+
+  private boolean needsToDeleteIndex(NewIndex 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 IndexHash().of(index);
+      toBeDeleted = !StringUtils.equals(hash, defHash);
+    }
+    return toBeDeleted;
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/IndexDefinition.java b/server/sonar-server/src/main/java/org/sonar/server/es/IndexDefinition.java
new file mode 100644 (file)
index 0000000..93b0358
--- /dev/null
@@ -0,0 +1,45 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public interface IndexDefinition {
+
+  public static class IndexDefinitionContext {
+    private final Map<String, NewIndex> byKey = new HashMap<String, NewIndex>();
+
+    public NewIndex create(String key) {
+      NewIndex index = byKey.get(key);
+      if (index == null) {
+        index = new NewIndex(key);
+        byKey.put(key, index);
+      }
+      return index;
+    }
+
+    public Map<String, NewIndex> getIndices() {
+      return byKey;
+    }
+  }
+
+  void define(IndexDefinitionContext context);
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/IndexHash.java b/server/sonar-server/src/main/java/org/sonar/server/es/IndexHash.java
new file mode 100644 (file)
index 0000000..42be03b
--- /dev/null
@@ -0,0 +1,86 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.Lists;
+import org.apache.commons.codec.digest.DigestUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+
+class IndexHash {
+
+  private static final char DELIMITER = ',';
+
+  String of(NewIndex index) {
+    return of(index.getSettings().internalMap(), index.getMappings());
+  }
+
+  String of(Map... maps) {
+    StringBuilder sb = new StringBuilder();
+    for (Map map : maps) {
+      appendMap(sb, map);
+    }
+    return DigestUtils.sha1Hex(sb.toString());
+  }
+
+  private void appendObject(StringBuilder sb, Object value) {
+    if (value instanceof NewIndex.NewMapping) {
+      appendMapping(sb, (NewIndex.NewMapping) value);
+    } else if (value instanceof Map) {
+      appendMap(sb, (Map) value);
+    } else if (value instanceof Iterable) {
+      appendIterable(sb, (Iterable) value);
+    } else {
+      sb.append(String.valueOf(value));
+    }
+  }
+
+  private void appendMapping(StringBuilder sb, NewIndex.NewMapping mapping) {
+    appendMap(sb, mapping.getAttributes());
+  }
+
+  private void appendMap(StringBuilder sb, Map attributes) {
+    for (Object entry : sort(attributes).entrySet()) {
+      sb.append(((Map.Entry) entry).getKey());
+      sb.append(DELIMITER);
+      appendObject(sb, ((Map.Entry) entry).getValue());
+      sb.append(DELIMITER);
+    }
+  }
+
+  private void appendIterable(StringBuilder sb, Iterable value) {
+    List sorted = Lists.newArrayList(value);
+    Collections.sort(sorted);
+    for (Object o : sorted) {
+      appendObject(sb, o);
+      sb.append(DELIMITER);
+    }
+  }
+
+  private SortedMap sort(Map map) {
+    return ImmutableSortedMap.copyOf(map);
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/IssueIndexDefinition.java b/server/sonar-server/src/main/java/org/sonar/server/es/IssueIndexDefinition.java
new file mode 100644 (file)
index 0000000..63f1608
--- /dev/null
@@ -0,0 +1,101 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import com.google.common.collect.ImmutableMap;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.sonar.api.config.Settings;
+import org.sonar.process.ProcessConstants;
+import org.sonar.server.issue.index.IssueAuthorizationNormalizer;
+import org.sonar.server.issue.index.IssueNormalizer;
+import org.sonar.server.search.BaseNormalizer;
+
+/**
+ * Definition of ES index "issues", including settings and fields.
+ */
+public class IssueIndexDefinition implements IndexDefinition {
+
+  public static final String INDEX_ISSUES = "issues";
+  public static final String TYPE_ISSUE_AUTHORIZATION = "issueAuthorization";
+  public static final String TYPE_ISSUE = "issue";
+
+  private final Settings settings;
+
+  public IssueIndexDefinition(Settings settings) {
+    this.settings = settings;
+  }
+
+  @Override
+  public void define(IndexDefinitionContext context) {
+    NewIndex index = context.create(INDEX_ISSUES);
+
+    // shards
+    boolean clusterMode = settings.getBoolean(ProcessConstants.CLUSTER_ACTIVATE);
+    if (clusterMode) {
+      index.getSettings().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 4);
+      index.getSettings().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1);
+      // else keep defaults (one shard)
+    }
+
+    // type "issueAuthorization"
+    NewIndex.NewMapping authorizationMapping = index.createMapping(TYPE_ISSUE_AUTHORIZATION);
+    authorizationMapping.setAttribute("_id", ImmutableMap.of("path", IssueAuthorizationNormalizer.IssueAuthorizationField.PROJECT.field()));
+    authorizationMapping.createDateTimeField(BaseNormalizer.UPDATED_AT_FIELD);
+    authorizationMapping.stringFieldBuilder("project").build();
+    authorizationMapping.stringFieldBuilder("groups").build();
+    authorizationMapping.stringFieldBuilder("users").build();
+
+    // type "issue"
+    NewIndex.NewMapping issueMapping = index.createMapping(TYPE_ISSUE);
+    issueMapping.setAttribute("_id", ImmutableMap.of("path", IssueNormalizer.IssueField.KEY.field()));
+    issueMapping.setAttribute("_parent", ImmutableMap.of("type", TYPE_ISSUE_AUTHORIZATION));
+    issueMapping.setAttribute("_routing", ImmutableMap.of("required", true, "path", IssueNormalizer.IssueField.PROJECT.field()));
+    issueMapping.stringFieldBuilder("component").build();
+    issueMapping.stringFieldBuilder("actionPlan").build();
+    // TODO do we really sort by assignee ?
+    issueMapping.stringFieldBuilder("assignee").enableSorting().build();
+    issueMapping.stringFieldBuilder("attributes").build();
+    issueMapping.stringFieldBuilder("authorLogin").build();
+    issueMapping.createDateTimeField("createdAt");
+    issueMapping.createDoubleField("debt");
+    issueMapping.createDoubleField("effort");
+    issueMapping.stringFieldBuilder("filePath").enableSorting().build();
+    issueMapping.createDateTimeField("issueCreatedAt");
+    issueMapping.createDateTimeField("issueUpdatedAt");
+    issueMapping.createDateTimeField("issueClosedAt");
+    issueMapping.stringFieldBuilder("key").enableSorting().build();
+    issueMapping.stringFieldBuilder("language").build();
+    issueMapping.createIntegerField("line");
+    issueMapping.stringFieldBuilder("message").build();
+    issueMapping.stringFieldBuilder("module").build();
+    issueMapping.createUuidPathField("modulePath");
+    // TODO do we need to sort by project ?
+    issueMapping.stringFieldBuilder("project").enableSorting().build();
+    issueMapping.stringFieldBuilder("reporter").build();
+    issueMapping.stringFieldBuilder("resolution").build();
+    issueMapping.stringFieldBuilder("ruleKey").build();
+    issueMapping.stringFieldBuilder("severity").build();
+    // TODO do we need to sort by severity ?
+    issueMapping.createByteField("severityValue");
+    // TODO do we really sort by status ? If yes, then we should sort by "int value", but not by string key
+    issueMapping.stringFieldBuilder("status").enableSorting().build();
+    issueMapping.createDateTimeField("updatedAt");
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/NewIndex.java b/server/sonar-server/src/main/java/org/sonar/server/es/NewIndex.java
new file mode 100644 (file)
index 0000000..c236153
--- /dev/null
@@ -0,0 +1,211 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSortedMap;
+import org.apache.commons.lang.StringUtils;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.sonar.server.search.IndexField;
+
+import javax.annotation.CheckForNull;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+public class NewIndex {
+
+  public static class NewMapping {
+    private final Map<String, Object> attributes = new TreeMap<String, Object>();
+    private final Map<String, Object> properties = new TreeMap<String, Object>();
+
+    private NewMapping() {
+      // defaults
+      attributes.put("dynamic", false);
+      attributes.put("_all", ImmutableSortedMap.of("enabled", false));
+
+      attributes.put("properties", properties);
+    }
+
+    /**
+     * Complete the root json hash of mapping type, for example to set the attribute "_id"
+     */
+    public NewMapping setAttribute(String key, Object value) {
+      attributes.put(key, value);
+      return this;
+    }
+
+    /**
+     * Complete the json hash named "properties" in mapping type, usually to declare fields
+     */
+    public NewMapping setProperty(String key, Object value) {
+      properties.put(key, value);
+      return this;
+    }
+
+    public StringFieldBuilder stringFieldBuilder(String fieldName) {
+      return new StringFieldBuilder(this, fieldName);
+    }
+
+    public NewMapping createBooleanField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "boolean"));
+    }
+
+    public NewMapping createByteField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "byte"));
+    }
+
+    public NewMapping createDateTimeField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "date", "format", "date_time"));
+    }
+
+    public NewMapping createDoubleField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "double"));
+    }
+
+    public NewMapping createIntegerField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "integer"));
+    }
+
+    public NewMapping createLongField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "long"));
+    }
+
+    public NewMapping createShortField(String fieldName) {
+      return setProperty(fieldName, ImmutableMap.of("type", "short"));
+    }
+
+    public NewMapping createUuidPathField(String fieldName) {
+      return setProperty(fieldName, ImmutableSortedMap.of(
+        "type", "string",
+        "index", "analyzed",
+        "analyzer", "uuid_analyzer"));
+    }
+
+    public Map<String, Object> getAttributes() {
+      return attributes;
+    }
+
+    @CheckForNull
+    public Object getProperty(String key) {
+      return properties.get(key);
+    }
+  }
+
+  /**
+   * Helper to define a string field in mapping of index type
+   */
+  public static class StringFieldBuilder {
+    private static final ImmutableMap<String, String> NOT_ANALYZED = ImmutableSortedMap.of(
+      "type", "string",
+      "index", "not_analyzed");
+
+    private final NewMapping newMapping;
+    private final String fieldName;
+    private boolean sortable = false, wordSearch = false, gramSearch = false;
+
+    private StringFieldBuilder(NewMapping newMapping, String fieldName) {
+      this.newMapping = newMapping;
+      this.fieldName = fieldName;
+    }
+
+    /**
+     * Create a inner-field named "sort" with analyzer "sortable"
+     */
+    public StringFieldBuilder enableSorting() {
+      this.sortable = true;
+      return this;
+    }
+
+    public StringFieldBuilder enableWordSearch() {
+      this.wordSearch = true;
+      return this;
+    }
+
+    public StringFieldBuilder enableGramSearch() {
+      this.gramSearch = true;
+      return this;
+    }
+
+    public void build() {
+      Map<String, Object> hash = new TreeMap<String, Object>();
+      if (wordSearch || sortable || gramSearch) {
+        hash.put("type", "multi_field");
+        Map<String, Object> multiFields = new TreeMap<String, Object>();
+
+        if (sortable) {
+          multiFields.put(IndexField.SORT_SUFFIX, ImmutableSortedMap.of(
+            "type", "string",
+            "index", "analyzed",
+            "analyzer", "sortable"));
+        }
+        if (wordSearch) {
+          multiFields.put(IndexField.SEARCH_WORDS_SUFFIX, ImmutableSortedMap.of(
+            "type", "string",
+            "index", "analyzed",
+            "index_analyzer", "index_words",
+            "search_analyzer", "search_words"));
+        }
+        if (gramSearch) {
+          multiFields.put(IndexField.SEARCH_PARTIAL_SUFFIX, ImmutableSortedMap.of(
+            "type", "string",
+            "index", "analyzed",
+            "index_analyzer", "index_grams",
+            "search_analyzer", "search_grams"));
+        }
+        multiFields.put(fieldName, NOT_ANALYZED);
+        hash.put("fields", multiFields);
+      } else {
+        hash.putAll(NOT_ANALYZED);
+      }
+
+      newMapping.setProperty(fieldName, hash);
+    }
+  }
+
+  private final String indexName;
+  private final ImmutableSettings.Builder settings = DefaultMappingSettings.defaults();
+  private final SortedMap<String, NewMapping> mappings = new TreeMap<String, NewMapping>();
+
+  NewIndex(String indexName) {
+    Preconditions.checkArgument(StringUtils.isAllLowerCase(indexName), "Index name must be lower-case: " + indexName);
+    this.indexName = indexName;
+  }
+
+  public String getName() {
+    return indexName;
+  }
+
+  public ImmutableSettings.Builder getSettings() {
+    return settings;
+  }
+
+  public NewMapping createMapping(String typeName) {
+    NewMapping type = new NewMapping();
+    mappings.put(typeName, type);
+    return type;
+  }
+
+  public SortedMap<String, NewMapping> getMappings() {
+    return mappings;
+  }
+}
diff --git a/server/sonar-server/src/main/java/org/sonar/server/es/package-info.java b/server/sonar-server/src/main/java/org/sonar/server/es/package-info.java
new file mode 100644 (file)
index 0000000..3145230
--- /dev/null
@@ -0,0 +1,25 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import javax.annotation.ParametersAreNonnullByDefault;
+
index 8590ae013c5f3303ad4653ea149d02793b819e4a..29df06ef4688aa23c713f9ac4276027942b0fa08 100644 (file)
@@ -26,10 +26,8 @@ import org.elasticsearch.index.query.FilterBuilders;
 import org.sonar.core.issue.db.IssueAuthorizationDto;
 import org.sonar.server.search.BaseIndex;
 import org.sonar.server.search.IndexDefinition;
-import org.sonar.server.search.IndexField;
 import org.sonar.server.search.SearchClient;
 
-import java.util.HashMap;
 import java.util.Map;
 
 public class IssueAuthorizationIndex extends BaseIndex<IssueAuthorizationDoc, IssueAuthorizationDto, String> {
@@ -43,20 +41,19 @@ public class IssueAuthorizationIndex extends BaseIndex<IssueAuthorizationDoc, Is
     return s;
   }
 
+  @Override
+  protected void initializeIndex() {
+    // being refactored
+  }
+
   @Override
   protected Map mapProperties() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    for (IndexField field : IssueAuthorizationNormalizer.IssueAuthorizationField.ALL_FIELDS) {
-      mapping.put(field.field(), mapField(field));
-    }
-    return mapping;
+    throw new UnsupportedOperationException("being refactored");
   }
 
   @Override
   protected Map mapKey() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    mapping.put("path", IssueAuthorizationNormalizer.IssueAuthorizationField.PROJECT.field());
-    return mapping;
+    throw new UnsupportedOperationException("being refactored");
   }
 
   @Override
index 29d5900500b96b10894388fb738b32c9f54bb316..2c99f8dd2e398a5d2ceeffbba42125a65ef93930 100644 (file)
 package org.sonar.server.issue.index;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.*;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.commons.lang.BooleanUtils;
 import org.elasticsearch.action.search.SearchRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchType;
-import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.index.query.*;
+import org.elasticsearch.index.query.BoolFilterBuilder;
+import org.elasticsearch.index.query.FilterBuilder;
+import org.elasticsearch.index.query.FilterBuilders;
+import org.elasticsearch.index.query.OrFilterBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.search.aggregations.AggregationBuilder;
 import org.elasticsearch.search.aggregations.AggregationBuilders;
 import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
@@ -41,12 +48,22 @@ import org.sonar.api.rule.Severity;
 import org.sonar.core.issue.db.IssueDto;
 import org.sonar.server.issue.IssueQuery;
 import org.sonar.server.issue.filter.IssueFilterParameters;
-import org.sonar.server.search.*;
+import org.sonar.server.search.BaseIndex;
+import org.sonar.server.search.FacetValue;
+import org.sonar.server.search.IndexDefinition;
+import org.sonar.server.search.IndexField;
+import org.sonar.server.search.QueryContext;
+import org.sonar.server.search.Result;
+import org.sonar.server.search.SearchClient;
+import org.sonar.server.search.StickyFacetBuilder;
 import org.sonar.server.user.UserSession;
 
 import javax.annotation.Nullable;
-
-import java.util.*;
+import java.util.Collection;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import static com.google.common.collect.Lists.newArrayList;
 
@@ -74,8 +91,8 @@ public class IssueIndex extends BaseIndex<Issue, IssueDto, String> {
   }
 
   @Override
-  protected ImmutableSettings.Builder addCustomIndexSettings(ImmutableSettings.Builder baseIndexSettings) {
-    return baseIndexSettings.put("index.number_of_shards", 4);
+  protected void initializeIndex() {
+    // replaced by IssueIndexDefinition
   }
 
   @Override
@@ -85,47 +102,12 @@ public class IssueIndex extends BaseIndex<Issue, IssueDto, String> {
 
   @Override
   protected Map mapProperties() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    for (IndexField field : IssueNormalizer.IssueField.ALL_FIELDS) {
-      mapping.put(field.field(), mapField(field));
-    }
-    return mapping;
-  }
-
-  @Override
-  protected Map mapDomain() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    mapping.put("dynamic", false);
-    mapping.put("_all", ImmutableMap.of("enabled", false));
-    mapping.put("_id", mapKey());
-    mapping.put("_parent", mapParent());
-    mapping.put("_routing", mapRouting());
-    mapping.put("properties", mapProperties());
-    return mapping;
-  }
-
-  private Object mapParent() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    mapping.put("type", getParentType());
-    return mapping;
-  }
-
-  private String getParentType() {
-    return IndexDefinition.ISSUES_AUTHORIZATION.getIndexType();
-  }
-
-  private Map mapRouting() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    mapping.put("required", true);
-    mapping.put("path", IssueNormalizer.IssueField.PROJECT.field());
-    return mapping;
+    throw new UnsupportedOperationException("Being refactored");
   }
 
   @Override
   protected Map mapKey() {
-    Map<String, Object> mapping = new HashMap<String, Object>();
-    mapping.put("path", IssueNormalizer.IssueField.KEY.field());
-    return mapping;
+    throw new UnsupportedOperationException("Being refactored");
   }
 
   @Override
index 555875e1c84c364c799d7beff70f6c3f1e8e096e..2e01a36ca47b5c64ed84d03a891c7650ceee7344 100644 (file)
@@ -53,7 +53,13 @@ import org.sonar.core.measure.db.MeasureFilterDao;
 import org.sonar.core.metric.DefaultMetricFinder;
 import org.sonar.core.notification.DefaultNotificationManager;
 import org.sonar.core.permission.PermissionFacade;
-import org.sonar.core.persistence.*;
+import org.sonar.core.persistence.DaoUtils;
+import org.sonar.core.persistence.DatabaseVersion;
+import org.sonar.core.persistence.DefaultDatabase;
+import org.sonar.core.persistence.MyBatis;
+import org.sonar.core.persistence.PreviewDatabaseFactory;
+import org.sonar.core.persistence.SemaphoreUpdater;
+import org.sonar.core.persistence.SemaphoresImpl;
 import org.sonar.core.preview.PreviewCache;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.purge.PurgeProfiler;
@@ -79,7 +85,11 @@ import org.sonar.server.activity.index.ActivityNormalizer;
 import org.sonar.server.activity.ws.ActivitiesWebService;
 import org.sonar.server.activity.ws.ActivityMapping;
 import org.sonar.server.authentication.ws.AuthenticationWs;
-import org.sonar.server.batch.*;
+import org.sonar.server.batch.BatchIndex;
+import org.sonar.server.batch.BatchWs;
+import org.sonar.server.batch.GlobalReferentialsAction;
+import org.sonar.server.batch.ProjectReferentialsAction;
+import org.sonar.server.batch.UploadReportAction;
 import org.sonar.server.charts.ChartFactory;
 import org.sonar.server.component.ComponentCleanerService;
 import org.sonar.server.component.ComponentService;
@@ -87,8 +97,22 @@ import org.sonar.server.component.DefaultComponentFinder;
 import org.sonar.server.component.DefaultRubyComponentService;
 import org.sonar.server.component.db.ComponentDao;
 import org.sonar.server.component.db.SnapshotDao;
-import org.sonar.server.component.ws.*;
-import org.sonar.server.computation.*;
+import org.sonar.server.component.ws.ComponentAppAction;
+import org.sonar.server.component.ws.ComponentsWs;
+import org.sonar.server.component.ws.EventsWs;
+import org.sonar.server.component.ws.ProjectsWs;
+import org.sonar.server.component.ws.ResourcesWs;
+import org.sonar.server.computation.AnalysisReportQueue;
+import org.sonar.server.computation.AnalysisReportTaskCleaner;
+import org.sonar.server.computation.AnalysisReportTaskLauncher;
+import org.sonar.server.computation.ComponentIndexationInDatabaseStep;
+import org.sonar.server.computation.ComputationService;
+import org.sonar.server.computation.ComputationStepRegistry;
+import org.sonar.server.computation.DataCleanerStep;
+import org.sonar.server.computation.IndexProjectIssuesStep;
+import org.sonar.server.computation.InvalidatePreviewCacheStep;
+import org.sonar.server.computation.SwitchSnapshotStep;
+import org.sonar.server.computation.SynchronizeProjectPermissionsStep;
 import org.sonar.server.computation.db.AnalysisReportDao;
 import org.sonar.server.computation.ws.ActiveAnalysisReportsAction;
 import org.sonar.server.computation.ws.AnalysisReportHistorySearchAction;
@@ -105,12 +129,35 @@ import org.sonar.server.db.DbClient;
 import org.sonar.server.db.EmbeddedDatabaseFactory;
 import org.sonar.server.db.migrations.DatabaseMigrations;
 import org.sonar.server.db.migrations.DatabaseMigrator;
-import org.sonar.server.debt.*;
+import org.sonar.server.debt.DebtCharacteristicsXMLImporter;
+import org.sonar.server.debt.DebtModelBackup;
+import org.sonar.server.debt.DebtModelLookup;
+import org.sonar.server.debt.DebtModelOperations;
+import org.sonar.server.debt.DebtModelPluginRepository;
+import org.sonar.server.debt.DebtModelService;
+import org.sonar.server.debt.DebtModelXMLExporter;
+import org.sonar.server.debt.DebtRulesXMLImporter;
 import org.sonar.server.design.FileDesignWidget;
 import org.sonar.server.duplication.ws.DuplicationsJsonWriter;
 import org.sonar.server.duplication.ws.DuplicationsParser;
 import org.sonar.server.duplication.ws.DuplicationsWs;
-import org.sonar.server.issue.*;
+import org.sonar.server.es.EsClient;
+import org.sonar.server.es.IndexCreator;
+import org.sonar.server.es.IssueIndexDefinition;
+import org.sonar.server.issue.ActionService;
+import org.sonar.server.issue.AssignAction;
+import org.sonar.server.issue.CommentAction;
+import org.sonar.server.issue.InternalRubyIssueService;
+import org.sonar.server.issue.IssueBulkChangeService;
+import org.sonar.server.issue.IssueChangelogFormatter;
+import org.sonar.server.issue.IssueChangelogService;
+import org.sonar.server.issue.IssueCommentService;
+import org.sonar.server.issue.IssueQueryService;
+import org.sonar.server.issue.IssueService;
+import org.sonar.server.issue.PlanAction;
+import org.sonar.server.issue.ServerIssueStorage;
+import org.sonar.server.issue.SetSeverityAction;
+import org.sonar.server.issue.TransitionAction;
 import org.sonar.server.issue.actionplan.ActionPlanService;
 import org.sonar.server.issue.actionplan.ActionPlanWs;
 import org.sonar.server.issue.db.IssueAuthorizationDao;
@@ -143,32 +190,113 @@ import org.sonar.server.platform.ws.L10nWs;
 import org.sonar.server.platform.ws.RestartHandler;
 import org.sonar.server.platform.ws.ServerWs;
 import org.sonar.server.platform.ws.SystemWs;
-import org.sonar.server.plugins.*;
+import org.sonar.server.plugins.InstalledPluginReferentialFactory;
+import org.sonar.server.plugins.PluginDownloader;
+import org.sonar.server.plugins.ServerExtensionInstaller;
+import org.sonar.server.plugins.ServerPluginJarInstaller;
+import org.sonar.server.plugins.ServerPluginJarsInstaller;
+import org.sonar.server.plugins.ServerPluginRepository;
+import org.sonar.server.plugins.UpdateCenterClient;
+import org.sonar.server.plugins.UpdateCenterMatrixFactory;
 import org.sonar.server.properties.ProjectSettingsFactory;
 import org.sonar.server.qualitygate.QgateProjectFinder;
 import org.sonar.server.qualitygate.QualityGates;
 import org.sonar.server.qualitygate.RegisterQualityGates;
-import org.sonar.server.qualitygate.ws.*;
-import org.sonar.server.qualityprofile.*;
+import org.sonar.server.qualitygate.ws.QGatesAppAction;
+import org.sonar.server.qualitygate.ws.QGatesCopyAction;
+import org.sonar.server.qualitygate.ws.QGatesCreateAction;
+import org.sonar.server.qualitygate.ws.QGatesCreateConditionAction;
+import org.sonar.server.qualitygate.ws.QGatesDeleteConditionAction;
+import org.sonar.server.qualitygate.ws.QGatesDeselectAction;
+import org.sonar.server.qualitygate.ws.QGatesDestroyAction;
+import org.sonar.server.qualitygate.ws.QGatesListAction;
+import org.sonar.server.qualitygate.ws.QGatesRenameAction;
+import org.sonar.server.qualitygate.ws.QGatesSearchAction;
+import org.sonar.server.qualitygate.ws.QGatesSelectAction;
+import org.sonar.server.qualitygate.ws.QGatesSetAsDefaultAction;
+import org.sonar.server.qualitygate.ws.QGatesShowAction;
+import org.sonar.server.qualitygate.ws.QGatesUnsetDefaultAction;
+import org.sonar.server.qualitygate.ws.QGatesUpdateConditionAction;
+import org.sonar.server.qualitygate.ws.QGatesWs;
+import org.sonar.server.qualityprofile.BuiltInProfiles;
+import org.sonar.server.qualityprofile.QProfileBackuper;
+import org.sonar.server.qualityprofile.QProfileCopier;
+import org.sonar.server.qualityprofile.QProfileExporters;
+import org.sonar.server.qualityprofile.QProfileFactory;
+import org.sonar.server.qualityprofile.QProfileLoader;
+import org.sonar.server.qualityprofile.QProfileLookup;
+import org.sonar.server.qualityprofile.QProfileProjectLookup;
+import org.sonar.server.qualityprofile.QProfileProjectOperations;
+import org.sonar.server.qualityprofile.QProfileReset;
+import org.sonar.server.qualityprofile.QProfileService;
+import org.sonar.server.qualityprofile.QProfiles;
+import org.sonar.server.qualityprofile.RegisterQualityProfiles;
+import org.sonar.server.qualityprofile.RuleActivator;
+import org.sonar.server.qualityprofile.RuleActivatorContextFactory;
 import org.sonar.server.qualityprofile.db.ActiveRuleDao;
 import org.sonar.server.qualityprofile.index.ActiveRuleIndex;
 import org.sonar.server.qualityprofile.index.ActiveRuleNormalizer;
-import org.sonar.server.qualityprofile.ws.*;
-import org.sonar.server.rule.*;
+import org.sonar.server.qualityprofile.ws.BulkRuleActivationActions;
+import org.sonar.server.qualityprofile.ws.ProfilesWs;
+import org.sonar.server.qualityprofile.ws.QProfileRestoreBuiltInAction;
+import org.sonar.server.qualityprofile.ws.QProfilesWs;
+import org.sonar.server.qualityprofile.ws.RuleActivationActions;
+import org.sonar.server.rule.DefaultRuleFinder;
+import org.sonar.server.rule.DeprecatedRulesDefinition;
+import org.sonar.server.rule.RegisterRules;
+import org.sonar.server.rule.RubyRuleService;
+import org.sonar.server.rule.RuleCreator;
+import org.sonar.server.rule.RuleDefinitionsLoader;
+import org.sonar.server.rule.RuleDeleter;
+import org.sonar.server.rule.RuleOperations;
+import org.sonar.server.rule.RuleRepositories;
+import org.sonar.server.rule.RuleService;
+import org.sonar.server.rule.RuleUpdater;
 import org.sonar.server.rule.db.RuleDao;
 import org.sonar.server.rule.index.RuleIndex;
 import org.sonar.server.rule.index.RuleNormalizer;
-import org.sonar.server.rule.ws.*;
-import org.sonar.server.search.*;
+import org.sonar.server.rule.ws.ActiveRuleCompleter;
+import org.sonar.server.rule.ws.AppAction;
+import org.sonar.server.rule.ws.DeleteAction;
+import org.sonar.server.rule.ws.RuleMapping;
+import org.sonar.server.rule.ws.RulesWebService;
+import org.sonar.server.rule.ws.SearchAction;
+import org.sonar.server.rule.ws.TagsAction;
+import org.sonar.server.rule.ws.UpdateAction;
+import org.sonar.server.search.IndexClient;
+import org.sonar.server.search.IndexQueue;
+import org.sonar.server.search.IndexSynchronizer;
+import org.sonar.server.search.SearchClient;
+import org.sonar.server.search.SearchHealth;
 import org.sonar.server.source.CodeColorizers;
 import org.sonar.server.source.DeprecatedSourceDecorator;
 import org.sonar.server.source.HtmlSourceDecorator;
 import org.sonar.server.source.SourceService;
-import org.sonar.server.source.ws.*;
+import org.sonar.server.source.ws.RawAction;
+import org.sonar.server.source.ws.ScmAction;
+import org.sonar.server.source.ws.ScmWriter;
 import org.sonar.server.source.ws.ShowAction;
-import org.sonar.server.startup.*;
+import org.sonar.server.source.ws.SourcesWs;
+import org.sonar.server.startup.CleanPreviewAnalysisCache;
+import org.sonar.server.startup.CopyRequirementsFromCharacteristicsToRules;
+import org.sonar.server.startup.GeneratePluginIndex;
+import org.sonar.server.startup.JdbcDriverDeployer;
+import org.sonar.server.startup.LogServerId;
+import org.sonar.server.startup.RegisterDashboards;
+import org.sonar.server.startup.RegisterDebtModel;
+import org.sonar.server.startup.RegisterMetrics;
+import org.sonar.server.startup.RegisterNewMeasureFilters;
+import org.sonar.server.startup.RegisterPermissionTemplates;
+import org.sonar.server.startup.RegisterServletFilters;
+import org.sonar.server.startup.RenameDeprecatedPropertyKeys;
+import org.sonar.server.startup.ServerMetadataPersister;
 import org.sonar.server.test.CoverageService;
-import org.sonar.server.test.ws.*;
+import org.sonar.server.test.ws.CoverageShowAction;
+import org.sonar.server.test.ws.CoverageWs;
+import org.sonar.server.test.ws.TestsCoveredFilesAction;
+import org.sonar.server.test.ws.TestsShowAction;
+import org.sonar.server.test.ws.TestsTestCasesAction;
+import org.sonar.server.test.ws.TestsWs;
 import org.sonar.server.text.MacroInterpreter;
 import org.sonar.server.text.RubyTextService;
 import org.sonar.server.ui.JRubyI18n;
@@ -176,12 +304,23 @@ import org.sonar.server.ui.JRubyProfiling;
 import org.sonar.server.ui.PageDecorations;
 import org.sonar.server.ui.Views;
 import org.sonar.server.updatecenter.ws.UpdateCenterWs;
-import org.sonar.server.user.*;
+import org.sonar.server.user.DefaultUserService;
+import org.sonar.server.user.DoPrivileged;
+import org.sonar.server.user.GroupMembershipFinder;
+import org.sonar.server.user.GroupMembershipService;
+import org.sonar.server.user.NewUserNotifier;
+import org.sonar.server.user.SecurityRealmFactory;
 import org.sonar.server.user.db.GroupDao;
 import org.sonar.server.user.ws.FavoritesWs;
 import org.sonar.server.user.ws.UserPropertiesWs;
 import org.sonar.server.user.ws.UsersWs;
-import org.sonar.server.util.*;
+import org.sonar.server.util.BooleanTypeValidation;
+import org.sonar.server.util.FloatTypeValidation;
+import org.sonar.server.util.IntegerTypeValidation;
+import org.sonar.server.util.StringListTypeValidation;
+import org.sonar.server.util.StringTypeValidation;
+import org.sonar.server.util.TextTypeValidation;
+import org.sonar.server.util.TypeValidations;
 import org.sonar.server.ws.ListingWs;
 import org.sonar.server.ws.WebServiceEngine;
 
@@ -240,6 +379,7 @@ class ServerComponents {
       SearchClient.class,
       IndexClient.class,
       SearchHealth.class,
+      EsClient.class,
 
       // users
       GroupDao.class,
@@ -339,6 +479,7 @@ class ServerComponents {
     pico.addSingleton(Periods.class);
     pico.addSingleton(ServerWs.class);
     pico.addSingleton(BackendCleanup.class);
+    pico.addSingleton(IndexCreator.class);
 
     // batch
     pico.addSingleton(BatchIndex.class);
@@ -486,6 +627,7 @@ class ServerComponents {
     pico.addSingleton(ComponentCleanerService.class);
 
     // issues
+    pico.addSingleton(IssueIndexDefinition.class);
     pico.addSingleton(ServerIssueStorage.class);
     pico.addSingleton(IssueUpdater.class);
     pico.addSingleton(FunctionExecutor.class);
index 45b260ad1924e5cfe897cbbbeca6cbf6fdfa98c0..ab9e61a37c1ed401478dd8c51941e1ed7dfdaf7c 100644 (file)
@@ -24,7 +24,7 @@ public class ClusterHealth {
   private boolean clusterAvailable;
   private int numberOfNodes;
 
-  void setClusterAvailable(boolean clusterAvailable) {
+  public void setClusterAvailable(boolean clusterAvailable) {
     this.clusterAvailable = clusterAvailable;
   }
 
@@ -32,7 +32,7 @@ public class ClusterHealth {
     return clusterAvailable;
   }
 
-  void setNumberOfNodes(int total) {
+  public void setNumberOfNodes(int total) {
     this.numberOfNodes = total;
   }
 
index 6ad4812d4eb658b68efb847890c68e48ad338d8c..a359669d8d79ee483be8ca083f39fc66fb8da8c4 100644 (file)
@@ -107,6 +107,10 @@ public class SearchClient extends TransportClient implements Startable {
     this.profiling = profiling;
   }
 
+  public Profiling getProfiling() {
+    return profiling;
+  }
+
   public ClusterHealth getClusterHealth() {
     ClusterHealth health = new ClusterHealth();
     ClusterStatsResponse clusterStatsResponse = this.prepareClusterStats().get();
index 7a9966e0c8e7495b99523e1d56a4f53eb4f5e139..bb75fdba32fd6d23f9dc3814487bb5ab7a5ad264 100644 (file)
@@ -27,16 +27,16 @@ import org.elasticsearch.action.bulk.BulkResponse;
 import org.elasticsearch.action.delete.DeleteRequest;
 import org.elasticsearch.action.index.IndexRequest;
 import org.elasticsearch.action.update.UpdateRequest;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyBulkRequestBuilder extends BulkRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyBulkRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyBulkRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index 68699d0de416d50bf2dce75059c11f3ced11517f..8cc46b001a7450503ff6bc0363505e308b268cba 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder;
 import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyClusterHealthRequestBuilder extends ClusterHealthRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyClusterHealthRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyClusterHealthRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().cluster());
     this.profiling = profiling;
   }
index 1c3610e1d2a714d63d08e141e7e7c45b23557bef..178eec942f003ec9f92be374e902cf3ca8ebe354 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.cluster.state.ClusterStateRequestBuilder;
 import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyClusterStateRequestBuilder extends ClusterStateRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyClusterStateRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyClusterStateRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().cluster());
     this.profiling = profiling;
   }
index 6a76aac7ce40ec00a6342ec304c57ab3bc392899..cba4108d9f0e604050f1b058dd5dc4a2a69ca6d5 100644 (file)
@@ -24,6 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.cluster.stats.ClusterStatsRequestBuilder;
 import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
@@ -33,7 +34,7 @@ public class ProxyClusterStatsRequestBuilder extends ClusterStatsRequestBuilder
 
   private final Profiling profiling;
 
-  public ProxyClusterStatsRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyClusterStatsRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().cluster());
     this.profiling = profiling;
   }
index 7af342d9dd75d2f11542334508e087b9c7228029..1d997f018d06077e3a20e0ffc21aa8a70f2b660f 100644 (file)
@@ -24,6 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.count.CountRequestBuilder;
 import org.elasticsearch.action.count.CountResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
@@ -33,7 +34,7 @@ public class ProxyCountRequestBuilder extends CountRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyCountRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyCountRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index fdc47425319c3709ad5e89c9bd6e402b5f373b53..e8679305295931a328da7a6ab916313723bdf9e9 100644 (file)
@@ -23,17 +23,17 @@ package org.sonar.server.search.request;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
 import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyCreateIndexRequestBuilder extends CreateIndexRequestBuilder {
 
   private final Profiling profiling;
   private final String index;
 
-  public ProxyCreateIndexRequestBuilder(SearchClient client, Profiling profiling, String index) {
+  public ProxyCreateIndexRequestBuilder(Client client, Profiling profiling, String index) {
     super(client.admin().indices(), index);
     this.profiling = profiling;
     this.index = index;
index 5201b6a5bda4a4aac47c35f9ab1a5245b7e502de..a86b6f75b9f8a9275be53685d6bb4e4c58dea5f1 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.deletebyquery.DeleteByQueryRequestBuilder;
 import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyDeleteByQueryRequestBuilder extends DeleteByQueryRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyDeleteByQueryRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyDeleteByQueryRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index 803843235051b740065b6e8f496ebf472df8cf72..c758784470ed7ae40d098845978f16d152ae5a1f 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.flush.FlushRequestBuilder;
 import org.elasticsearch.action.admin.indices.flush.FlushResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyFlushRequestBuilder extends FlushRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyFlushRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyFlushRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().indices());
     this.profiling = profiling;
   }
index ea764d3649096825b312d707512129b6507470c6..4823e92a2962b99c77ead7c6992ccc332b897ef0 100644 (file)
@@ -23,6 +23,7 @@ package org.sonar.server.search.request;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.get.GetRequestBuilder;
 import org.elasticsearch.action.get.GetResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
@@ -32,7 +33,7 @@ public class ProxyGetRequestBuilder extends GetRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyGetRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyGetRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index a4bc27c0cf45db7252f9ad3349035c0e69b07d61..67364280234dae7a280da8dd3335e3d39ff0d25a 100644 (file)
@@ -24,6 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequestBuilder;
 import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
@@ -33,7 +34,7 @@ public class ProxyIndicesExistsRequestBuilder extends IndicesExistsRequestBuilde
 
   private final Profiling profiling;
 
-  public ProxyIndicesExistsRequestBuilder(SearchClient client, Profiling profiling, String... indices) {
+  public ProxyIndicesExistsRequestBuilder(Client client, Profiling profiling, String... indices) {
     super(client.admin().indices(), indices);
     this.profiling = profiling;
   }
index 8b02e3f25e05e42f050e43f0d4167e87ab710b34..689387cb31da6c1e7aefe5b9f442b30829ff5cc6 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.stats.IndicesStatsRequestBuilder;
 import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyIndicesStatsRequestBuilder extends IndicesStatsRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyIndicesStatsRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyIndicesStatsRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().indices());
     this.profiling = profiling;
   }
index 453557ea790ce650c1384a31f01013c771a7091b..6567240531d3108393989fb295af09fef727eb56 100644 (file)
@@ -24,16 +24,16 @@ import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.get.MultiGetRequest;
 import org.elasticsearch.action.get.MultiGetRequestBuilder;
 import org.elasticsearch.action.get.MultiGetResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyMultiGetRequestBuilder extends MultiGetRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyMultiGetRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyMultiGetRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index 6f921ce7960964f2b8aea0b682c6b3f72fecb0b2..ff43b6cf32d8e41edca24dd9025c2a08a2e536c0 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequestBuilder;
 import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyNodesStatsRequestBuilder extends NodesStatsRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyNodesStatsRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyNodesStatsRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().cluster());
     this.profiling = profiling;
   }
index 69459171ff7aad7a91dad3ca95c9041267203369..ca3cc2cb4a940793dd55392ba71c4fda85c1eb92 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequestBuilder;
 import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyPutMappingRequestBuilder extends PutMappingRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyPutMappingRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyPutMappingRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().indices());
     this.profiling = profiling;
   }
index cc76ea6fb66162dff40d91e07f8674f3398ee9c1..0cb33ea52248856671c97853266b2d6a68cf79cd 100644 (file)
@@ -24,16 +24,16 @@ import org.apache.commons.lang.StringUtils;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.admin.indices.refresh.RefreshRequestBuilder;
 import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxyRefreshRequestBuilder extends RefreshRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxyRefreshRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxyRefreshRequestBuilder(Client client, Profiling profiling) {
     super(client.admin().indices());
     this.profiling = profiling;
   }
index 3b48a189088e4219c0840cef77b32a1a9fb7c5c0..e85924661fce23ecc53ac814dfd1b0aa3103fedd 100644 (file)
@@ -23,13 +23,13 @@ package org.sonar.server.search.request;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.search.SearchRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.common.xcontent.XContentFactory;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -38,7 +38,7 @@ public class ProxySearchRequestBuilder extends SearchRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxySearchRequestBuilder(SearchClient client, Profiling profiling) {
+  public ProxySearchRequestBuilder(Client client, Profiling profiling) {
     super(client);
     this.profiling = profiling;
   }
index aa3c7df81390de2a2ea432a0f1265cd97343d380..f8d996a9444f906b2556aff42a45a8aba22f2a89 100644 (file)
@@ -23,16 +23,16 @@ package org.sonar.server.search.request;
 import org.elasticsearch.action.ListenableActionFuture;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchScrollRequestBuilder;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.sonar.core.profiling.Profiling;
 import org.sonar.core.profiling.StopWatch;
-import org.sonar.server.search.SearchClient;
 
 public class ProxySearchScrollRequestBuilder extends SearchScrollRequestBuilder {
 
   private final Profiling profiling;
 
-  public ProxySearchScrollRequestBuilder(String scrollId, SearchClient client, Profiling profiling) {
+  public ProxySearchScrollRequestBuilder(String scrollId, Client client, Profiling profiling) {
     super(client, scrollId);
     this.profiling = profiling;
   }
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/BulkIndexRequestIteratorTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/BulkIndexRequestIteratorTest.java
new file mode 100644 (file)
index 0000000..73c6de6
--- /dev/null
@@ -0,0 +1,109 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.action.ActionRequest;
+import org.elasticsearch.action.index.IndexRequest;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import static org.fest.assertions.Assertions.assertThat;
+import static org.fest.assertions.Fail.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+public class BulkIndexRequestIteratorTest {
+
+  @Test
+  public void iterate_over_requests() throws Exception {
+    List<String> input = Arrays.asList("foo", "bar", "3requests", "baz");
+    BulkIndexRequestIterator.InputConverter converter = new BulkIndexRequestIterator.InputConverter<String>() {
+      @Override
+      public List<ActionRequest> convert(String input) {
+        if ("3requests".equals(input)) {
+          return Collections.nCopies(3, (ActionRequest) new IndexRequest(input));
+        }
+        return Arrays.asList((ActionRequest) new IndexRequest(input));
+      }
+    };
+
+    BulkIndexRequestIterator<String> it = new BulkIndexRequestIterator<String>(input, converter);
+
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "foo");
+
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "bar");
+
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "3requests");
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "3requests");
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "3requests");
+
+    assertThat(it.hasNext()).isTrue();
+    assertIndex(it.next(), "baz");
+
+    assertThat(it.hasNext()).isFalse();
+    try {
+      it.next();
+      fail();
+    } catch (NoSuchElementException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void empty() throws Exception {
+    List<String> input = Collections.emptyList();
+    BulkIndexRequestIterator.InputConverter converter = mock(BulkIndexRequestIterator.InputConverter.class);
+
+    BulkIndexRequestIterator<String> it = new BulkIndexRequestIterator<String>(input, converter);
+
+    assertThat(it.hasNext()).isFalse();
+    verifyZeroInteractions(converter);
+  }
+
+  @Test
+  public void removal_is_not_supported() throws Exception {
+    List<String> input = Arrays.asList("foo");
+    BulkIndexRequestIterator.InputConverter converter = mock(BulkIndexRequestIterator.InputConverter.class);
+
+    BulkIndexRequestIterator<String> it = new BulkIndexRequestIterator<String>(input, converter);
+
+    try {
+      it.remove();
+      fail();
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+  }
+
+  private void assertIndex(ActionRequest req, String indexName) {
+    assertThat(req).isNotNull();
+    assertThat(req).isInstanceOf(IndexRequest.class);
+    assertThat(((IndexRequest) req).index()).isEqualTo(indexName);
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/DefaultMappingSettingsTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/DefaultMappingSettingsTest.java
new file mode 100644 (file)
index 0000000..3e2a130
--- /dev/null
@@ -0,0 +1,46 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.common.collect.ImmutableMap;
+import org.junit.Test;
+import org.sonar.test.TestUtils;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class DefaultMappingSettingsTest {
+
+  @Test
+  public void defaults() throws Exception {
+    ImmutableMap<String, String> map = DefaultMappingSettings.defaults().build().getAsMap();
+    assertThat(map).isNotEmpty();
+
+    // test some values
+    assertThat(map.get("index.number_of_shards")).isEqualTo("1");
+    assertThat(map.get("index.number_of_replicas")).isEqualTo("0");
+    assertThat(map.get("index.analysis.analyzer.sortable.type")).isEqualTo("custom");
+  }
+
+  @Test
+  public void only_statics() throws Exception {
+    TestUtils.hasOnlyPrivateConstructors(DefaultMappingSettings.class);
+
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/EsClientTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/EsClientTest.java
new file mode 100644 (file)
index 0000000..90c9c08
--- /dev/null
@@ -0,0 +1,76 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.sonar.server.search.request.ProxyBulkRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterHealthRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterStateRequestBuilder;
+import org.sonar.server.search.request.ProxyClusterStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyCountRequestBuilder;
+import org.sonar.server.search.request.ProxyCreateIndexRequestBuilder;
+import org.sonar.server.search.request.ProxyDeleteByQueryRequestBuilder;
+import org.sonar.server.search.request.ProxyFlushRequestBuilder;
+import org.sonar.server.search.request.ProxyGetRequestBuilder;
+import org.sonar.server.search.request.ProxyIndicesExistsRequestBuilder;
+import org.sonar.server.search.request.ProxyIndicesStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyMultiGetRequestBuilder;
+import org.sonar.server.search.request.ProxyNodesStatsRequestBuilder;
+import org.sonar.server.search.request.ProxyPutMappingRequestBuilder;
+import org.sonar.server.search.request.ProxyRefreshRequestBuilder;
+import org.sonar.server.search.request.ProxySearchRequestBuilder;
+import org.sonar.server.search.request.ProxySearchScrollRequestBuilder;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class EsClientTest {
+
+  @Rule
+  public EsTester es = new EsTester();
+
+  @Test
+  public void proxify_requests() throws Exception {
+    EsClient client = es.client();
+    client.start();
+    assertThat(client.nativeClient()).isNotNull();
+    assertThat(client.getClusterHealth().isClusterAvailable()).isTrue();
+    assertThat(client.prepareBulk()).isInstanceOf(ProxyBulkRequestBuilder.class);
+    assertThat(client.prepareClusterStats()).isInstanceOf(ProxyClusterStatsRequestBuilder.class);
+    assertThat(client.prepareCount()).isInstanceOf(ProxyCountRequestBuilder.class);
+    assertThat(client.prepareCreate("fakes")).isInstanceOf(ProxyCreateIndexRequestBuilder.class);
+    assertThat(client.prepareDeleteByQuery()).isInstanceOf(ProxyDeleteByQueryRequestBuilder.class);
+    assertThat(client.prepareExists()).isInstanceOf(ProxyIndicesExistsRequestBuilder.class);
+    assertThat(client.prepareFlush()).isInstanceOf(ProxyFlushRequestBuilder.class);
+    assertThat(client.prepareGet()).isInstanceOf(ProxyGetRequestBuilder.class);
+    assertThat(client.prepareGet("fakes", "fake", "1")).isInstanceOf(ProxyGetRequestBuilder.class);
+    assertThat(client.prepareHealth()).isInstanceOf(ProxyClusterHealthRequestBuilder.class);
+    assertThat(client.prepareMultiGet()).isInstanceOf(ProxyMultiGetRequestBuilder.class);
+    assertThat(client.prepareNodesStats()).isInstanceOf(ProxyNodesStatsRequestBuilder.class);
+    assertThat(client.preparePutMapping()).isInstanceOf(ProxyPutMappingRequestBuilder.class);
+    assertThat(client.prepareRefresh()).isInstanceOf(ProxyRefreshRequestBuilder.class);
+    assertThat(client.prepareSearch()).isInstanceOf(ProxySearchRequestBuilder.class);
+    assertThat(client.prepareSearchScroll("1234")).isInstanceOf(ProxySearchScrollRequestBuilder.class);
+    assertThat(client.prepareState()).isInstanceOf(ProxyClusterStateRequestBuilder.class);
+    assertThat(client.prepareStats()).isInstanceOf(ProxyIndicesStatsRequestBuilder.class);
+
+    client.stop();
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/EsTester.java b/server/sonar-server/src/test/java/org/sonar/server/es/EsTester.java
new file mode 100644 (file)
index 0000000..8283766
--- /dev/null
@@ -0,0 +1,96 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexResponse;
+import org.elasticsearch.cluster.ClusterName;
+import org.elasticsearch.cluster.metadata.IndexMetaData;
+import org.elasticsearch.cluster.node.DiscoveryNode;
+import org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeBuilder;
+import org.junit.rules.ExternalResource;
+import org.sonar.api.config.Settings;
+import org.sonar.core.profiling.Profiling;
+
+import java.util.Date;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class EsTester extends ExternalResource {
+
+  /**
+   * This instance is shared for performance reasons. Never stopped.
+   */
+  private static Node sharedNode;
+  private static EsClient client;
+
+  @Override
+  protected void before() throws Throwable {
+    if (sharedNode == null) {
+      String nodeName = EsTester.class.getName();
+      sharedNode = NodeBuilder.nodeBuilder().local(true).data(true).settings(ImmutableSettings.builder()
+        .put(ClusterName.SETTING, nodeName)
+        .put("node.name", nodeName)
+        // the two following properties are probably not used because they are
+        // declared on indices too
+        .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
+        .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
+        // limit the number of threads created (see org.elasticsearch.common.util.concurrent.EsExecutors)
+        .put("processors", 1)
+        .put("http.enabled", false)
+        .put("index.store.type", "ram")
+        .put("config.ignore_system_properties", true)
+        .put("gateway.type", "none"))
+        .build();
+      sharedNode.start();
+      assertThat(DiscoveryNode.localNode(sharedNode.settings())).isTrue();
+      client = new EsClient(new Profiling(new Settings()), sharedNode.client());
+
+    } else {
+      // delete the indices created by previous tests
+      DeleteIndexResponse response = sharedNode.client().admin().indices().prepareDelete("_all").get();
+      assertThat(response.isAcknowledged()).isTrue();
+    }
+  }
+
+  public void truncateIndices() {
+    client.prepareDeleteByQuery(client.prepareState().get()
+      .getState().getMetaData().concreteAllIndices())
+      .setQuery(QueryBuilders.matchAllQuery())
+      .get();
+    client.prepareRefresh(client.prepareState().get()
+      .getState().getMetaData().concreteAllIndices())
+      .setForce(true)
+      .get();
+    client.prepareFlush(client.prepareState().get()
+      .getState().getMetaData().concreteAllIndices())
+      .get();
+  }
+
+  public Node node() {
+    return sharedNode;
+  }
+
+  public EsClient client() {
+    return client;
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/IndexCreatorTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/IndexCreatorTest.java
new file mode 100644 (file)
index 0000000..2591702
--- /dev/null
@@ -0,0 +1,124 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+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 javax.annotation.CheckForNull;
+import java.io.IOException;
+import java.util.Map;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class IndexCreatorTest {
+
+  @Rule
+  public EsTester es = new EsTester();
+
+  @Test
+  public void create_index() throws Exception {
+    assertThat(mappings()).isEmpty();
+
+    IndexCreator creator = new IndexCreator(es.client(), new IndexDefinition[]{new FakeIndexDefinition()});
+    creator.start();
+
+    // check that index is created with related mapping
+    ImmutableOpenMap<String, ImmutableOpenMap<String, MappingMetaData>> mappings = mappings();
+    MappingMetaData mapping = mappings.get("fakes").get("fake");
+    assertThat(mapping.type()).isEqualTo("fake");
+    assertThat(mapping.getSourceAsMap()).isNotEmpty();
+    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();
+  }
+
+  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 recreate_index_on_definition_changes() throws Exception {
+    assertThat(mappings()).isEmpty();
+
+    // v1
+    IndexCreator creator = new IndexCreator(es.client(), new IndexDefinition[]{new FakeIndexDefinition()});
+    creator.start();
+    creator.stop();
+    String hashV1 = setting("fakes", "index.sonar_hash");
+    assertThat(hashV1).isNotEmpty();
+
+    // v2
+    creator = new IndexCreator(es.client(), new IndexDefinition[]{new FakeIndexDefinitionV2()});
+    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();
+  }
+
+  private ImmutableOpenMap<String, ImmutableOpenMap<String, MappingMetaData>> mappings() {
+    return es.client().nativeClient().admin().indices().prepareGetMappings().get().mappings();
+  }
+
+  @CheckForNull
+  private Map<String, Object> field(MappingMetaData mapping, String field) throws IOException {
+    Map<String, Object> props = (Map) mapping.getSourceAsMap().get("properties");
+    return (Map<String, Object>) props.get(field);
+  }
+
+  private int countMappingFields(MappingMetaData mapping) throws IOException {
+    return ((Map) mapping.getSourceAsMap().get("properties")).size();
+  }
+
+  public static class FakeIndexDefinition implements IndexDefinition {
+    @Override
+    public void define(IndexDefinitionContext context) {
+      NewIndex index = context.create("fakes");
+      NewIndex.NewMapping mapping = index.createMapping("fake");
+      mapping.stringFieldBuilder("key").build();
+      mapping.createDateTimeField("updatedAt");
+    }
+  }
+
+  public static class FakeIndexDefinitionV2 implements IndexDefinition {
+    @Override
+    public void define(IndexDefinitionContext context) {
+      NewIndex index = context.create("fakes");
+      NewIndex.NewMapping mapping = index.createMapping("fake");
+      mapping.stringFieldBuilder("key").build();
+      mapping.createDateTimeField("updatedAt");
+      mapping.createIntegerField("newField");
+    }
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/IndexHashTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/IndexHashTest.java
new file mode 100644 (file)
index 0000000..8f53727
--- /dev/null
@@ -0,0 +1,53 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class IndexHashTest {
+
+  @Test
+  public void of() throws Exception {
+    NewIndex indexV1 = createIndex();
+    String hashV1 = new IndexHash().of(indexV1);
+    assertThat(hashV1).isNotEmpty();
+    // always the same
+    assertThat(hashV1).isEqualTo(new IndexHash().of(indexV1));
+
+    NewIndex indexV2 = createIndex();
+    indexV2.getMappings().get("fake").createIntegerField("max");
+    String hashV2 = new IndexHash().of(indexV2);
+    assertThat(hashV2).isNotEmpty().isNotEqualTo(hashV1);
+  }
+
+  private NewIndex createIndex() {
+    NewIndex index = new NewIndex("fakes");
+    NewIndex.NewMapping mapping = index.createMapping("fake");
+    mapping.setAttribute("list_attr", Arrays.asList("foo", "bar"));
+    mapping.stringFieldBuilder("key").build();
+    mapping.createDateTimeField("updatedAt");
+    return index;
+  }
+
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/IssueIndexDefinitionTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/IssueIndexDefinitionTest.java
new file mode 100644 (file)
index 0000000..1a5d442
--- /dev/null
@@ -0,0 +1,58 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import org.junit.Test;
+import org.sonar.api.config.Settings;
+import org.sonar.process.ProcessConstants;
+
+import static org.fest.assertions.Assertions.assertThat;
+
+public class IssueIndexDefinitionTest {
+
+  IndexDefinition.IndexDefinitionContext context = new IndexDefinition.IndexDefinitionContext();
+
+  @Test
+  public void define() throws Exception {
+    IssueIndexDefinition def = new IssueIndexDefinition(new Settings());
+    def.define(context);
+
+    assertThat(context.getIndices()).hasSize(1);
+    NewIndex issuesIndex = context.getIndices().get("issues");
+    assertThat(issuesIndex).isNotNull();
+    assertThat(issuesIndex.getMappings().keySet()).containsOnly("issue", "issueAuthorization");
+
+    // no cluster by default
+    assertThat(issuesIndex.getSettings().get("index.number_of_shards")).isEqualTo("1");
+    assertThat(issuesIndex.getSettings().get("index.number_of_replicas")).isEqualTo("0");
+  }
+
+  @Test
+  public void enable_cluster() throws Exception {
+    Settings settings = new Settings();
+    settings.setProperty(ProcessConstants.CLUSTER_ACTIVATE, true);
+    IssueIndexDefinition def = new IssueIndexDefinition(settings);
+    def.define(context);
+
+    NewIndex issuesIndex = context.getIndices().get("issues");
+    assertThat(issuesIndex.getSettings().get("index.number_of_shards")).isEqualTo("4");
+    assertThat(issuesIndex.getSettings().get("index.number_of_replicas")).isEqualTo("1");
+  }
+}
diff --git a/server/sonar-server/src/test/java/org/sonar/server/es/NewIndexTest.java b/server/sonar-server/src/test/java/org/sonar/server/es/NewIndexTest.java
new file mode 100644 (file)
index 0000000..80df925
--- /dev/null
@@ -0,0 +1,105 @@
+/*
+ * SonarQube, open source software quality management tool.
+ * Copyright (C) 2008-2014 SonarSource
+ * mailto:contact AT sonarsource DOT com
+ *
+ * SonarQube 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.
+ *
+ * SonarQube 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;
+
+import com.google.common.collect.ImmutableMap;
+import org.elasticsearch.common.settings.Settings;
+import org.fest.assertions.MapAssert;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.fest.assertions.Assertions.assertThat;
+import static org.fest.assertions.Fail.fail;
+
+public class NewIndexTest {
+
+  @Test
+  public void most_basic_index() throws Exception {
+    NewIndex index = new NewIndex("issues");
+    assertThat(index.getName()).isEqualTo("issues");
+    assertThat(index.getMappings()).isEmpty();
+    Settings settings = index.getSettings().build();
+    // test some basic settings
+    assertThat(settings.get("index.number_of_shards")).isNotEmpty();
+  }
+
+  @Test
+  public void index_name_is_lower_case() throws Exception {
+    try {
+      new NewIndex("Issues");
+      fail();
+    } catch (IllegalArgumentException e) {
+      assertThat(e).hasMessage("Index name must be lower-case: Issues");
+    }
+  }
+
+  @Test
+  public void define_fields() throws Exception {
+    NewIndex index = new NewIndex("issues");
+    NewIndex.NewMapping mapping = index.createMapping("issue");
+    mapping.setAttribute("dynamic", "true");
+    mapping.setProperty("foo_field", ImmutableMap.of("type", "string"));
+    mapping.createBooleanField("boolean_field");
+    mapping.createByteField("byte_field");
+    mapping.createDateTimeField("dt_field");
+    mapping.createDoubleField("double_field");
+    mapping.createIntegerField("int_field");
+    mapping.createLongField("long_field");
+    mapping.createShortField("short_field");
+    mapping.createUuidPathField("uuid_path_field");
+
+    mapping = index.getMappings().get("issue");
+    assertThat(mapping.getAttributes().get("dynamic")).isEqualTo("true");
+    assertThat(mapping).isNotNull();
+    assertThat(mapping.getProperty("foo_field")).isInstanceOf(Map.class);
+    assertThat((Map)mapping.getProperty("foo_field")).includes(MapAssert.entry("type", "string"));
+    assertThat((Map)mapping.getProperty("byte_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("double_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("dt_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("int_field")).includes(MapAssert.entry("type", "integer"));
+    assertThat((Map)mapping.getProperty("long_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("short_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("uuid_path_field")).isNotEmpty();
+    assertThat((Map)mapping.getProperty("unknown")).isNull();
+  }
+
+  @Test
+  public void define_string_field() throws Exception {
+    NewIndex index = new NewIndex("issues");
+    NewIndex.NewMapping mapping = index.createMapping("issue");
+    mapping.stringFieldBuilder("basic_field").build();
+    mapping.stringFieldBuilder("all_capabilities_field")
+      .enableGramSearch()
+      .enableWordSearch()
+      .enableSorting()
+      .build();
+
+    Map<String, Object> props = (Map) mapping.getProperty("basic_field");
+    assertThat(props.get("type")).isEqualTo("string");
+    assertThat(props.get("index")).isEqualTo("not_analyzed");
+    assertThat(props.get("fields")).isNull();
+
+    props = (Map) mapping.getProperty("all_capabilities_field");
+    assertThat(props.get("type")).isEqualTo("multi_field");
+    // no need to test values, it's not the scope of this test
+    assertThat((Map)props.get("fields")).isNotEmpty();
+  }
+}
index 2d3d884711008f82603454d6f168bc15a177e0ca..334785d408393da81aedc5d52c842250e5768ff8 100644 (file)
@@ -20,7 +20,6 @@
 package org.sonar.server.issue.index;
 
 import com.google.common.collect.ImmutableMap;
-import org.elasticsearch.common.settings.Settings;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -50,7 +49,10 @@ import org.sonar.server.issue.db.IssueDao;
 import org.sonar.server.platform.BackendCleanup;
 import org.sonar.server.rule.RuleTesting;
 import org.sonar.server.rule.db.RuleDao;
-import org.sonar.server.search.*;
+import org.sonar.server.search.FacetValue;
+import org.sonar.server.search.IndexDefinition;
+import org.sonar.server.search.QueryContext;
+import org.sonar.server.search.Result;
 import org.sonar.server.tester.ServerTester;
 import org.sonar.server.user.MockUserSession;
 
@@ -836,19 +838,6 @@ public class IssueIndexMediumTest {
     assertThat(results.get(2).getValue()).isEqualTo(1);
   }
 
-  @Test
-  public void index_has_4_shards() {
-
-    // 0 Assert configuration is correct
-    String shardSettingKey = "index.number_of_shards";
-    Settings settings = index.getIndexSettings();
-    assertThat(settings.get(shardSettingKey)).isEqualTo("4");
-
-    // 1 Assert index has 4 shards
-    assertThat(tester.get(SearchClient.class).admin().indices().prepareGetSettings(IndexDefinition.ISSUES.getIndexName())
-      .get().getSetting(IndexDefinition.ISSUES.getIndexName(), shardSettingKey)).isEqualTo("4");
-  }
-
   @Test
   public void delete_issues_from_one_project() {
     // ARRANGE
index 75b9d9bd64aa919f1dee1285301c2ff99e101a98..03b0237a59d9bb53865814b5be146dd090b79058 100644 (file)
@@ -21,6 +21,7 @@ package org.sonar.core.persistence;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import org.apache.commons.dbutils.DbUtils;
 import org.apache.commons.lang.StringUtils;
 import org.dbunit.dataset.datatype.DefaultDataTypeFactory;
 import org.dbunit.dataset.datatype.IDataTypeFactory;
@@ -30,9 +31,14 @@ import org.dbunit.ext.mssql.MsSqlDataTypeFactory;
 import org.dbunit.ext.mysql.MySqlDataTypeFactory;
 import org.dbunit.ext.oracle.Oracle10DataTypeFactory;
 import org.dbunit.ext.postgresql.PostgresqlDataTypeFactory;
-import org.sonar.core.persistence.dialect.*;
+import org.sonar.core.persistence.dialect.Dialect;
+import org.sonar.core.persistence.dialect.MsSql;
+import org.sonar.core.persistence.dialect.MySql;
+import org.sonar.core.persistence.dialect.Oracle;
+import org.sonar.core.persistence.dialect.PostgreSql;
 
 import javax.sql.DataSource;
+
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -116,21 +122,23 @@ public abstract class DatabaseCommands {
 
   public void truncateDatabase(DataSource dataSource) throws SQLException {
     Connection connection = dataSource.getConnection();
-    connection.setAutoCommit(false);
-
-    Statement statement = connection.createStatement();
-    for (String table : DatabaseVersion.TABLES) {
-      try {
-        statement.executeUpdate("TRUNCATE TABLE " + table);
-        connection.commit();
-      } catch (Exception e) {
-        // ignore
-        connection.rollback();
+    Statement statement = null;
+    try {
+      connection.setAutoCommit(false);
+      statement = connection.createStatement();
+      for (String table : DatabaseVersion.TABLES) {
+        try {
+          statement.executeUpdate("TRUNCATE TABLE " + table);
+          connection.commit();
+        } catch (Exception e) {
+          // ignore
+          connection.rollback();
+        }
       }
+    } finally {
+      DbUtils.closeQuietly(connection);
+      DbUtils.closeQuietly(statement);
     }
-
-    statement.close();
-    connection.close();
   }
 
   public void resetPrimaryKeys(DataSource dataSource) throws SQLException {
index 56eadb4aace07f2fcfb7fd092cf88a6c7b679b94..3f3bd71c3e8a8afc48e23ea7e01b322706e60efc 100644 (file)
@@ -127,7 +127,15 @@ public class TestDatabase extends ExternalResource {
     myBatis = new MyBatis(db, new Logback(), queue);
     myBatis.start();
 
-    commands.truncateDatabase(db.getDataSource());
+    truncateTables();
+  }
+
+  public void truncateTables() {
+    try {
+      commands.truncateDatabase(db.getDataSource());
+    } catch (SQLException e) {
+      throw new IllegalStateException("Fail to truncate db tables", e);
+    }
   }
 
   @Override