Merge changes from topic 'elasticsearch'
* changes:
Introduce gerrit-index module
Add support for secondary index with Elasticsearch
diff --git a/Documentation/config-gerrit.txt b/Documentation/config-gerrit.txt
index 004d32b..1c7981a 100644
--- a/Documentation/config-gerrit.txt
+++ b/Documentation/config-gerrit.txt
@@ -2461,6 +2461,10 @@
+
A link:http://lucene.apache.org/[Lucene] index is used.
+
++
+* `ELASTICSEARCH`
++
+An link:http://www.elasticsearch.org/[Elasticsearch] index is used.
+
By default, `LUCENE`.
@@ -2585,6 +2589,43 @@
maxBufferedDocs = 500
----
+
+==== Elasticsearch configuration
+
+WARNING: ElasticSearch implementation is incomplete. Right now it is
+still using parts of Lucene index.
+
+Open and closed changes are indexed in a single index, separated
+into types 'open_changes' and 'closed_changes' respectively.
+
+The following settings are only used when the index type is
+`ELASTICSEARCH`.
+
+[[index.protocol]]index.protocol::
++
+Elasticsearch server protocol [http|https].
++
+Defaults to `http`.
+
+[[index.hostname]]index.hostname::
++
+Elasticsearch server hostname.
+
+Defaults to `localhost`.
+
+[[index.port]]index.port::
++
+Elasticsearch server port.
++
+Defauls to `9200`.
+
+[[index.name]]index.name::
++
+This setting can be used to index changes from multiple Gerrit
+instances in a single Elasticsearch cluster.
++
+Defaults to 'gerrit'.
+
[[ldap]]
=== Section ldap
diff --git a/gerrit-elasticsearch/BUCK b/gerrit-elasticsearch/BUCK
new file mode 100644
index 0000000..a2641df
--- /dev/null
+++ b/gerrit-elasticsearch/BUCK
@@ -0,0 +1,51 @@
+java_library(
+ name = 'elasticsearch',
+ srcs = glob(['src/main/java/**/*.java']),
+ deps = [
+ '//gerrit-antlr:query_exception',
+ '//gerrit-extension-api:api',
+ '//gerrit-lucene:lucene', # only for LuceneAccountIndex
+ '//gerrit-reviewdb:client',
+ '//gerrit-reviewdb:server',
+ '//gerrit-server:server',
+ '//gerrit-index:index',
+ '//lib:gson',
+ '//lib:guava',
+ '//lib:gwtorm',
+ '//lib:protobuf',
+ '//lib/commons:codec',
+ '//lib/commons:lang',
+ '//lib/elasticsearch:elasticsearch',
+ '//lib/elasticsearch:jest',
+ '//lib/elasticsearch:jest-common',
+ '//lib/guice:guice',
+ '//lib/guice:guice-assistedinject',
+ '//lib/jgit/org.eclipse.jgit:jgit',
+ '//lib/joda:joda-time',
+ '//lib/log:api',
+ '//lib/lucene:lucene-analyzers-common',
+ '//lib/lucene:lucene-core',
+ ],
+ visibility = ['PUBLIC'],
+)
+
+java_test(
+ name = 'elasticsearch_tests',
+ labels = ['elastic'],
+ srcs = glob(['src/test/java/**/*.java']),
+ deps = [
+ ':elasticsearch',
+ '//gerrit-extension-api:api',
+ '//gerrit-server:server',
+ '//gerrit-server:testutil',
+ '//gerrit-server:query_tests',
+ '//lib:gson',
+ '//lib:guava',
+ '//lib:junit',
+ '//lib:truth',
+ '//lib/elasticsearch:elasticsearch',
+ '//lib/guice:guice',
+ '//lib/jgit/org.eclipse.jgit:jgit',
+ '//lib/jgit/org.eclipse.jgit.junit:junit',
+ ],
+)
diff --git a/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/AbstractElasticIndex.java b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/AbstractElasticIndex.java
new file mode 100644
index 0000000..a46edc7
--- /dev/null
+++ b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/AbstractElasticIndex.java
@@ -0,0 +1,207 @@
+// Copyright (C) 2014 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import static com.google.common.base.MoreObjects.firstNonNull;
+import static com.google.common.base.Preconditions.checkState;
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterables;
+import com.google.gerrit.index.IndexUtils;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.server.index.FieldDef.FillArgs;
+import com.google.gerrit.server.index.Index;
+import com.google.gerrit.server.index.Schema;
+import com.google.gerrit.server.index.Schema.Values;
+import com.google.inject.Inject;
+import com.google.inject.assistedinject.Assisted;
+
+import org.eclipse.jgit.lib.Config;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.concurrent.TimeUnit;
+
+import io.searchbox.client.JestClientFactory;
+import io.searchbox.client.JestResult;
+import io.searchbox.client.config.HttpClientConfig;
+import io.searchbox.client.http.JestHttpClient;
+import io.searchbox.core.Bulk;
+import io.searchbox.core.Delete;
+import io.searchbox.indices.CreateIndex;
+import io.searchbox.indices.DeleteIndex;
+import io.searchbox.indices.IndicesExists;
+
+abstract class AbstractElasticIndex<K, V> implements Index<K, V> {
+ private static final String DEFAULT_INDEX_NAME = "gerrit";
+
+ private final Schema<V> schema;
+ private final FillArgs fillArgs;
+ private final SitePaths sitePaths;
+
+ protected final boolean refresh;
+ protected final String indexName;
+ protected final JestHttpClient client;
+
+
+ @Inject
+ AbstractElasticIndex(@GerritServerConfig Config cfg,
+ FillArgs fillArgs,
+ SitePaths sitePaths,
+ @Assisted Schema<V> schema) {
+ this.fillArgs = fillArgs;
+ this.sitePaths = sitePaths;
+ this.schema = schema;
+ String protocol = getRequiredConfigOption(cfg, "protocol");
+ String hostname = getRequiredConfigOption(cfg, "hostname");
+ String port = getRequiredConfigOption(cfg, "port");
+
+ this.indexName =
+ firstNonNull(cfg.getString("index", null, "name"), DEFAULT_INDEX_NAME);
+
+ // By default Elasticsearch has a 1s delay before changes are available in
+ // the index. Setting refresh(true) on calls to the index makes the index
+ // refresh immediately.
+ //
+ // Discovery should be disabled during test mode to prevent spurious
+ // connection failures caused by the client starting up and being ready
+ // before the test node.
+ //
+ // This setting should only be set to true during testing, and is not
+ // documented.
+ this.refresh = cfg.getBoolean("index", "elasticsearch", "test", false);
+
+ String url = buildUrl(protocol, hostname, port);
+ JestClientFactory factory = new JestClientFactory();
+ factory.setHttpClientConfig(new HttpClientConfig
+ .Builder(url)
+ .multiThreaded(true)
+ .discoveryEnabled(!refresh)
+ .discoveryFrequency(1L, TimeUnit.MINUTES)
+ .build());
+ client = (JestHttpClient) factory.getObject();
+ }
+
+ @Override
+ public Schema<V> getSchema() {
+ return schema;
+ }
+
+ @Override
+ public void close() {
+ client.shutdownClient();
+ }
+
+ @Override
+ public void markReady(boolean ready) throws IOException {
+ IndexUtils.setReady(sitePaths, indexName, schema.getVersion(), ready);
+ }
+
+ @Override
+ public void delete(K c) throws IOException {
+ Bulk bulk = addActions(new Bulk.Builder(), c).refresh(refresh).build();
+ JestResult result = client.execute(bulk);
+ if (!result.isSucceeded()) {
+ throw new IOException(String.format(
+ "Failed to delete change %s in index %s: %s", c, indexName,
+ result.getErrorMessage()));
+ }
+ }
+
+ @Override
+ public void deleteAll() throws IOException {
+ // Delete the index, if it exists.
+ JestResult result = client.execute(
+ new IndicesExists.Builder(indexName).build());
+ if (result.isSucceeded()) {
+ result = client.execute(
+ new DeleteIndex.Builder(indexName).build());
+ if (!result.isSucceeded()) {
+ throw new IOException(String.format(
+ "Failed to delete index %s: %s", indexName,
+ result.getErrorMessage()));
+ }
+ }
+
+ // Recreate the index.
+ result = client.execute(
+ new CreateIndex.Builder(indexName).settings(getMappings()).build());
+ if (!result.isSucceeded()) {
+ String error = String.format("Failed to create index %s: %s",
+ indexName, result.getErrorMessage());
+ throw new IOException(error);
+ }
+ }
+
+ protected abstract Bulk.Builder addActions(Bulk.Builder builder, K c);
+
+ protected abstract String getMappings();
+
+ protected abstract String getId(V v);
+
+ protected Delete delete(String type, K c) {
+ String id = c.toString();
+ return new Delete.Builder(id)
+ .index(indexName)
+ .type(type)
+ .build();
+ }
+
+ protected io.searchbox.core.Index insert(String type, V v) throws IOException {
+ String id = getId(v);
+ String doc = toDoc(v);
+ return new io.searchbox.core.Index.Builder(doc)
+ .index(indexName)
+ .type(type)
+ .id(id)
+ .build();
+ }
+
+ private String toDoc(V v) throws IOException {
+ XContentBuilder builder = jsonBuilder().startObject();
+ for (Values<V> values : schema.buildFields(v, fillArgs)) {
+ String name = values.getField().getName();
+ if (values.getField().isRepeatable()) {
+ builder.array(name, values.getValues());
+ } else {
+ Object element = Iterables.getOnlyElement(values.getValues(), "");
+ if (!(element instanceof String) || !((String) element).isEmpty()) {
+ builder.field(name, element);
+ }
+ }
+ }
+ return builder.endObject().string();
+ }
+
+ private String getRequiredConfigOption(Config cfg, String name) {
+ String option = cfg.getString("index", null, name);
+ checkState(!Strings.isNullOrEmpty(option), "index." + name + " must be supplied");
+ return option;
+ }
+
+ private String buildUrl(String protocol, String hostname, String port) {
+ try {
+ return new URL(protocol, hostname, Integer.parseInt(port), "").toString();
+ } catch (MalformedURLException | NumberFormatException e) {
+ throw new RuntimeException(
+ "Cannot build url to Elasticsearch from values: protocol=" + protocol
+ + " hostname=" + hostname + " port=" + port, e);
+ }
+ }
+}
diff --git a/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticChangeIndex.java b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticChangeIndex.java
new file mode 100644
index 0000000..c55ea1c
--- /dev/null
+++ b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticChangeIndex.java
@@ -0,0 +1,389 @@
+// Copyright (C) 2014 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import static com.google.gerrit.server.index.change.ChangeIndexRewriter.CLOSED_STATUSES;
+import static com.google.gerrit.server.index.change.ChangeIndexRewriter.OPEN_STATUSES;
+import static com.google.gson.FieldNamingPolicy.LOWER_CASE_WITH_UNDERSCORES;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.gerrit.elasticsearch.ElasticMapping.MappingProperties;
+import com.google.gerrit.index.IndexUtils;
+import com.google.gerrit.reviewdb.client.Account;
+import com.google.gerrit.reviewdb.client.Change;
+import com.google.gerrit.reviewdb.client.Change.Id;
+import com.google.gerrit.reviewdb.client.Project;
+import com.google.gerrit.reviewdb.server.ReviewDb;
+import com.google.gerrit.server.ReviewerSet;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.server.index.FieldDef;
+import com.google.gerrit.server.index.FieldDef.FillArgs;
+import com.google.gerrit.server.index.FieldType;
+import com.google.gerrit.server.index.QueryOptions;
+import com.google.gerrit.server.index.Schema;
+import com.google.gerrit.server.index.change.ChangeField;
+import com.google.gerrit.server.index.change.ChangeField.ChangeProtoField;
+import com.google.gerrit.server.index.change.ChangeField.PatchSetApprovalProtoField;
+import com.google.gerrit.server.index.change.ChangeField.PatchSetProtoField;
+import com.google.gerrit.server.index.change.ChangeIndex;
+import com.google.gerrit.server.index.change.ChangeIndexRewriter;
+import com.google.gerrit.server.query.Predicate;
+import com.google.gerrit.server.query.QueryParseException;
+import com.google.gerrit.server.query.change.ChangeData;
+import com.google.gerrit.server.query.change.ChangeDataSource;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gwtorm.protobuf.ProtobufCodec;
+import com.google.gwtorm.server.OrmException;
+import com.google.gwtorm.server.ResultSet;
+import com.google.inject.Provider;
+import com.google.inject.assistedinject.Assisted;
+import com.google.inject.assistedinject.AssistedInject;
+
+import org.apache.commons.codec.binary.Base64;
+import org.eclipse.jgit.lib.Config;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import io.searchbox.client.JestResult;
+import io.searchbox.core.Bulk;
+import io.searchbox.core.Bulk.Builder;
+import io.searchbox.core.Search;
+import io.searchbox.core.search.sort.Sort;
+import io.searchbox.core.search.sort.Sort.Sorting;
+
+/** Secondary index implementation using Elasticsearch. */
+class ElasticChangeIndex extends AbstractElasticIndex<Change.Id, ChangeData>
+ implements ChangeIndex {
+ private static final Logger log =
+ LoggerFactory.getLogger(ElasticChangeIndex.class);
+
+ static class ChangeMapping {
+ MappingProperties openChanges;
+ MappingProperties closedChanges;
+
+ ChangeMapping(Schema<ChangeData> schema) {
+ ElasticMapping.Builder mappingBuilder = new ElasticMapping.Builder();
+ for (FieldDef<?, ?> field : schema.getFields().values()) {
+ String name = field.getName();
+ FieldType<?> fieldType = field.getType();
+ if (fieldType == FieldType.EXACT) {
+ mappingBuilder.addExactField(name);
+ } else if (fieldType == FieldType.TIMESTAMP) {
+ mappingBuilder.addTimestamp(name);
+ } else if (fieldType == FieldType.INTEGER
+ || fieldType == FieldType.INTEGER_RANGE
+ || fieldType == FieldType.LONG) {
+ mappingBuilder.addNumber(name);
+ } else if (fieldType == FieldType.PREFIX
+ || fieldType == FieldType.FULL_TEXT
+ || fieldType == FieldType.STORED_ONLY) {
+ mappingBuilder.addString(name);
+ } else {
+ throw new IllegalArgumentException(
+ "Unsupported filed type " + fieldType.getName());
+ }
+ }
+ MappingProperties mapping = mappingBuilder.build();
+ openChanges = mapping;
+ closedChanges = mapping;
+ }
+ }
+
+ static final String OPEN_CHANGES = "open_changes";
+ static final String CLOSED_CHANGES = "closed_changes";
+
+ private final Gson gson;
+ private final ChangeMapping mapping;
+ private final Provider<ReviewDb> db;
+ private final ElasticQueryBuilder queryBuilder;
+ private final ChangeData.Factory changeDataFactory;
+
+ @AssistedInject
+ ElasticChangeIndex(
+ @GerritServerConfig Config cfg,
+ Provider<ReviewDb> db,
+ ChangeData.Factory changeDataFactory,
+ FillArgs fillArgs,
+ SitePaths sitePaths,
+ @Assisted Schema<ChangeData> schema) {
+ super(cfg, fillArgs, sitePaths, schema);
+ this.db = db;
+ this.changeDataFactory = changeDataFactory;
+ mapping = new ChangeMapping(schema);
+
+ this.queryBuilder = new ElasticQueryBuilder();
+ this.gson = new GsonBuilder()
+ .setFieldNamingPolicy(LOWER_CASE_WITH_UNDERSCORES).create();
+ }
+
+ private static <T> List<T> decodeProtos(JsonObject doc, String fieldName,
+ ProtobufCodec<T> codec) {
+ return FluentIterable.from(doc.getAsJsonArray(fieldName))
+ .transform(i -> codec.decode(Base64.decodeBase64(i.toString())))
+ .toList();
+ }
+
+ @Override
+ public void replace(ChangeData cd) throws IOException {
+ String deleteIndex;
+ String insertIndex;
+
+ try {
+ if (cd.change().getStatus().isOpen()) {
+ insertIndex = OPEN_CHANGES;
+ deleteIndex = CLOSED_CHANGES;
+ } else {
+ insertIndex = CLOSED_CHANGES;
+ deleteIndex = OPEN_CHANGES;
+ }
+ } catch (OrmException e) {
+ throw new IOException(e);
+ }
+
+ Bulk bulk = new Bulk.Builder()
+ .defaultIndex(indexName)
+ .defaultType("changes")
+ .addAction(insert(insertIndex, cd))
+ .addAction(delete(deleteIndex, cd.getId()))
+ .refresh(refresh)
+ .build();
+ JestResult result = client.execute(bulk);
+ if (!result.isSucceeded()) {
+ throw new IOException(String.format(
+ "Failed to replace change %s in index %s: %s", cd.getId(), indexName,
+ result.getErrorMessage()));
+ }
+ }
+
+ @Override
+ public ChangeDataSource getSource(Predicate<ChangeData> p, QueryOptions opts)
+ throws QueryParseException {
+ Set<Change.Status> statuses = ChangeIndexRewriter.getPossibleStatus(p);
+ List<String> indexes = Lists.newArrayListWithCapacity(2);
+ if (!Sets.intersection(statuses, OPEN_STATUSES).isEmpty()) {
+ indexes.add(OPEN_CHANGES);
+ }
+ if (!Sets.intersection(statuses, CLOSED_STATUSES).isEmpty()) {
+ indexes.add(CLOSED_CHANGES);
+ }
+ return new QuerySource(indexes, p, opts);
+ }
+
+ @Override
+ protected Builder addActions(Builder builder, Id c) {
+ return builder
+ .addAction(delete(OPEN_CHANGES, c))
+ .addAction(delete(OPEN_CHANGES, c));
+ }
+
+ @Override
+ protected String getMappings() {
+ return gson.toJson(ImmutableMap.of("mappings", mapping));
+ }
+
+ @Override
+ protected String getId(ChangeData cd) {
+ return cd.getId().toString();
+ }
+
+ private class QuerySource implements ChangeDataSource {
+ private final Search search;
+ private final Set<String> fields;
+
+ public QuerySource(List<String> types, Predicate<ChangeData> p,
+ QueryOptions opts) throws QueryParseException {
+ List<Sort> sorts = ImmutableList.of(
+ new Sort(ChangeField.UPDATED.getName(), Sorting.DESC),
+ new Sort(ChangeField.LEGACY_ID.getName(), Sorting.DESC));
+ for (Sort sort : sorts) {
+ sort.setIgnoreUnmapped();
+ }
+ QueryBuilder qb = queryBuilder.toQueryBuilder(p);
+ fields = IndexUtils.fields(opts);
+ SearchSourceBuilder searchSource = new SearchSourceBuilder()
+ .query(qb)
+ .from(opts.start())
+ .size(opts.limit())
+ .fields(Lists.newArrayList(fields));
+
+ search = new Search.Builder(searchSource.toString())
+ .addType(types)
+ .addSort(sorts)
+ .addIndex(indexName)
+ .build();
+ }
+
+ @Override
+ public int getCardinality() {
+ return 10;
+ }
+
+ @Override
+ public ResultSet<ChangeData> read() throws OrmException {
+ try {
+ List<ChangeData> results = Collections.emptyList();
+ JestResult result = client.execute(search);
+ if (result.isSucceeded()) {
+ JsonObject obj = result.getJsonObject().getAsJsonObject("hits");
+ if (obj.get("hits") != null) {
+ JsonArray json = obj.getAsJsonArray("hits");
+ results = Lists.newArrayListWithCapacity(json.size());
+ for (int i = 0; i < json.size(); i++) {
+ results.add(toChangeData(json.get(i)));
+ }
+ }
+ } else {
+ log.error(result.getErrorMessage());
+ }
+ final List<ChangeData> r = Collections.unmodifiableList(results);
+ return new ResultSet<ChangeData>() {
+ @Override
+ public Iterator<ChangeData> iterator() {
+ return r.iterator();
+ }
+
+ @Override
+ public List<ChangeData> toList() {
+ return r;
+ }
+
+ @Override
+ public void close() {
+ // Do nothing.
+ }
+ };
+ } catch (IOException e) {
+ throw new OrmException(e);
+ }
+ }
+
+ @Override
+ public boolean hasChange() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return search.toString();
+ }
+
+ private ChangeData toChangeData(JsonElement json) {
+ JsonElement sourceElement = json.getAsJsonObject().get("_source");
+ if (sourceElement == null) {
+ sourceElement = json.getAsJsonObject().get("fields");
+ }
+ JsonObject source = sourceElement.getAsJsonObject();
+ JsonElement c = source.get(ChangeField.CHANGE.getName());
+
+ if (c == null) {
+ int id = source.get(ChangeField.LEGACY_ID.getName()).getAsInt();
+ String projectName =
+ source.get(ChangeField.PROJECT.getName()).getAsString();
+ if (projectName == null) {
+ return changeDataFactory.createOnlyWhenNoteDbDisabled(
+ db.get(), new Change.Id(id));
+ }
+ return changeDataFactory.create(
+ db.get(), new Project.NameKey(projectName), new Change.Id(id));
+ }
+
+ ChangeData cd = changeDataFactory.create(db.get(),
+ ChangeProtoField.CODEC.decode(Base64.decodeBase64(c.getAsString())));
+
+ // Patch sets.
+ cd.setPatchSets(decodeProtos(
+ source, ChangeField.PATCH_SET.getName(), PatchSetProtoField.CODEC));
+
+ // Approvals.
+ if (source.get(ChangeField.APPROVAL.getName()) != null) {
+ cd.setCurrentApprovals(decodeProtos(source,
+ ChangeField.APPROVAL.getName(), PatchSetApprovalProtoField.CODEC));
+ } else if (fields.contains(ChangeField.APPROVAL.getName())) {
+ cd.setCurrentApprovals(Collections.emptyList());
+ }
+
+ JsonElement addedElement = source.get(ChangeField.ADDED.getName());
+ JsonElement deletedElement = source.get(ChangeField.DELETED.getName());
+ if (addedElement != null && deletedElement != null) {
+ // Changed lines.
+ int added = addedElement.getAsInt();
+ int deleted = deletedElement.getAsInt();
+ if (added != 0 && deleted != 0) {
+ cd.setChangedLines(added, deleted);
+ }
+ }
+
+ // Mergeable.
+ JsonElement mergeableElement = source.get(ChangeField.MERGEABLE.getName());
+ if (mergeableElement != null) {
+ String mergeable = mergeableElement.getAsString();
+ if ("1".equals(mergeable)) {
+ cd.setMergeable(true);
+ } else if ("0".equals(mergeable)) {
+ cd.setMergeable(false);
+ }
+ }
+
+ // Reviewed-by.
+ if (source.get(ChangeField.REVIEWEDBY.getName()) != null) {
+ JsonArray reviewedBy =
+ source.get(ChangeField.REVIEWEDBY.getName()).getAsJsonArray();
+ if (reviewedBy.size() > 0) {
+ Set<Account.Id> accounts =
+ Sets.newHashSetWithExpectedSize(reviewedBy.size());
+ for (int i = 0; i < reviewedBy.size() ; i++) {
+ int aId = reviewedBy.get(i).getAsInt();
+ if (reviewedBy.size() == 1 && aId == ChangeField.NOT_REVIEWED) {
+ break;
+ }
+ accounts.add(new Account.Id(aId));
+ }
+ cd.setReviewedBy(accounts);
+ }
+ } else if (fields.contains(ChangeField.REVIEWEDBY.getName())) {
+ cd.setReviewedBy(Collections.emptySet());
+ }
+
+ if (source.get(ChangeField.REVIEWER.getName()) != null) {
+ cd.setReviewers(
+ ChangeField.parseReviewerFieldValues(FluentIterable
+ .from(
+ source.get(ChangeField.REVIEWER.getName()).getAsJsonArray())
+ .transform(JsonElement::getAsString)));
+ } else if (fields.contains(ChangeField.REVIEWER.getName())) {
+ cd.setReviewers(ReviewerSet.empty());
+ }
+
+ return cd;
+ }
+ }
+}
diff --git a/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticIndexModule.java b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticIndexModule.java
new file mode 100644
index 0000000..e108dca
--- /dev/null
+++ b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticIndexModule.java
@@ -0,0 +1,73 @@
+// Copyright (C) 2014 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import com.google.gerrit.index.SingleVersionModule;
+import com.google.gerrit.lifecycle.LifecycleModule;
+import com.google.gerrit.lucene.LuceneAccountIndex;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.index.IndexConfig;
+import com.google.gerrit.server.index.IndexModule;
+import com.google.gerrit.server.index.account.AccountIndex;
+import com.google.gerrit.server.index.change.ChangeIndex;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
+import com.google.inject.assistedinject.FactoryModuleBuilder;
+
+import org.eclipse.jgit.lib.Config;
+
+import java.util.Map;
+
+public class ElasticIndexModule extends LifecycleModule {
+ private final int threads;
+ private final Map<String, Integer> singleVersions;
+
+ public static ElasticIndexModule singleVersionWithExplicitVersions(
+ Map<String, Integer> versions, int threads) {
+ return new ElasticIndexModule(versions, threads);
+ }
+
+ public static ElasticIndexModule latestVersionWithOnlineUpgrade() {
+ return new ElasticIndexModule(null, 0);
+ }
+
+ private ElasticIndexModule(Map<String, Integer> singleVersions, int threads) {
+ this.singleVersions = singleVersions;
+ this.threads = threads;
+ }
+
+ @Override
+ protected void configure() {
+ install(
+ new FactoryModuleBuilder()
+ .implement(ChangeIndex.class, ElasticChangeIndex.class)
+ .build(ChangeIndex.Factory.class));
+ install(
+ new FactoryModuleBuilder()
+ // until we implement Elasticsearch index for accounts we need to
+ // use Lucene to make all tests green and Gerrit server to work
+ .implement(AccountIndex.class, LuceneAccountIndex.class)
+ .build(AccountIndex.Factory.class));
+
+ install(new IndexModule(threads));
+ install(new SingleVersionModule(singleVersions));
+ }
+
+ @Provides
+ @Singleton
+ IndexConfig getIndexConfig(@GerritServerConfig Config cfg) {
+ return IndexConfig.fromConfig(cfg);
+ }
+}
diff --git a/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticMapping.java b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticMapping.java
new file mode 100644
index 0000000..e3f7e96
--- /dev/null
+++ b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticMapping.java
@@ -0,0 +1,79 @@
+// Copyright (C) 2016 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+class ElasticMapping {
+ static class Builder {
+ private final ImmutableMap.Builder<String, FieldProperties> fields =
+ new ImmutableMap.Builder<>();
+
+ MappingProperties build() {
+ MappingProperties properties = new MappingProperties();
+ properties.properties = fields.build();
+ return properties;
+ }
+
+ Builder addExactField(String name) {
+ FieldProperties key = new FieldProperties("string");
+ key.index = "not_analyzed";
+ FieldProperties properties = new FieldProperties("string");
+ properties.fields = ImmutableMap.of("key", key);
+ fields.put(name, properties);
+ return this;
+ }
+
+ Builder addTimestamp(String name) {
+ FieldProperties properties = new FieldProperties("date");
+ properties.type = "date";
+ properties.format = "dateOptionalTime";
+ fields.put(name, properties);
+ return this;
+ }
+
+ Builder addNumber(String name) {
+ fields.put(name, new FieldProperties("long"));
+ return this;
+ }
+
+ Builder addString(String name) {
+ fields.put(name, new FieldProperties("string"));
+ return this;
+ }
+
+ Builder add(String name, String type) {
+ fields.put(name, new FieldProperties(type));
+ return this;
+ }
+ }
+
+ static class MappingProperties {
+ Map<String, FieldProperties> properties;
+ }
+
+ static class FieldProperties {
+ String type;
+ String index;
+ String format;
+ Map<String, FieldProperties> fields;
+
+ FieldProperties(String type) {
+ this.type = type;
+ }
+ }
+}
diff --git a/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticQueryBuilder.java b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticQueryBuilder.java
new file mode 100644
index 0000000..51b14a4
--- /dev/null
+++ b/gerrit-elasticsearch/src/main/java/com/google/gerrit/elasticsearch/ElasticQueryBuilder.java
@@ -0,0 +1,181 @@
+// Copyright (C) 2014 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import com.google.gerrit.server.index.FieldDef;
+import com.google.gerrit.server.index.FieldType;
+import com.google.gerrit.server.index.IndexPredicate;
+import com.google.gerrit.server.index.IntegerRangePredicate;
+import com.google.gerrit.server.index.RegexPredicate;
+import com.google.gerrit.server.index.TimestampRangePredicate;
+import com.google.gerrit.server.query.AndPredicate;
+import com.google.gerrit.server.query.NotPredicate;
+import com.google.gerrit.server.query.OrPredicate;
+import com.google.gerrit.server.query.Predicate;
+import com.google.gerrit.server.query.QueryParseException;
+import com.google.gerrit.server.query.change.AfterPredicate;
+
+import org.apache.lucene.search.BooleanQuery;
+import org.elasticsearch.index.query.BoolQueryBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.joda.time.DateTime;
+
+public class ElasticQueryBuilder {
+
+ protected <T> QueryBuilder toQueryBuilder(Predicate<T> p)
+ throws QueryParseException {
+ if (p instanceof AndPredicate) {
+ return and(p);
+ } else if (p instanceof OrPredicate) {
+ return or(p);
+ } else if (p instanceof NotPredicate) {
+ return not(p);
+ } else if (p instanceof IndexPredicate) {
+ return fieldQuery((IndexPredicate<T>) p);
+ } else {
+ throw new QueryParseException("cannot create query for index: " + p);
+ }
+ }
+
+ private <T> BoolQueryBuilder and(Predicate<T> p)
+ throws QueryParseException {
+ try {
+ BoolQueryBuilder b = QueryBuilders.boolQuery();
+ for (Predicate<T> c : p.getChildren()) {
+ b.must(toQueryBuilder(c));
+ }
+ return b;
+ } catch (BooleanQuery.TooManyClauses e) {
+ throw new QueryParseException("cannot create query for index: " + p, e);
+ }
+ }
+
+ private <T> BoolQueryBuilder or(Predicate<T> p)
+ throws QueryParseException {
+ try {
+ BoolQueryBuilder q = QueryBuilders.boolQuery();
+ for (Predicate<T> c : p.getChildren()) {
+ q.should(toQueryBuilder(c));
+ }
+ return q;
+ } catch (BooleanQuery.TooManyClauses e) {
+ throw new QueryParseException("cannot create query for index: " + p, e);
+ }
+ }
+
+ private <T> QueryBuilder not(Predicate<T> p)
+ throws QueryParseException {
+ Predicate<T> n = p.getChild(0);
+ if (n instanceof TimestampRangePredicate) {
+ return notTimestamp((TimestampRangePredicate<T>) n);
+ }
+
+ // Lucene does not support negation, start with all and subtract.
+ BoolQueryBuilder q = QueryBuilders.boolQuery();
+ q.must(QueryBuilders.matchAllQuery());
+ q.mustNot(toQueryBuilder(n));
+ return q;
+ }
+
+ private <T> QueryBuilder fieldQuery(IndexPredicate<T> p)
+ throws QueryParseException {
+ FieldType<?> type = p.getType();
+ FieldDef<?,?> field = p.getField();
+ String name = field.getName();
+ String value = p.getValue();
+
+ if (type == FieldType.INTEGER) {
+ // QueryBuilder encodes integer fields as prefix coded bits,
+ // which elasticsearch's queryString can't handle.
+ // Create integer terms with string representations instead.
+ return QueryBuilders.termQuery(name, value);
+ } else if (type == FieldType.INTEGER_RANGE) {
+ return intRangeQuery(p);
+ } else if (type == FieldType.TIMESTAMP) {
+ return timestampQuery(p);
+ } else if (type == FieldType.EXACT) {
+ return exactQuery(p);
+ } else if (type == FieldType.PREFIX) {
+ return QueryBuilders.matchPhrasePrefixQuery(name, value);
+ } else if (type == FieldType.FULL_TEXT) {
+ return QueryBuilders.matchPhraseQuery(name, value);
+ } else {
+ throw FieldType.badFieldType(p.getType());
+ }
+ }
+
+ private <T> QueryBuilder intRangeQuery(IndexPredicate<T> p)
+ throws QueryParseException {
+ if (p instanceof IntegerRangePredicate) {
+ IntegerRangePredicate<T> r = (IntegerRangePredicate<T>) p;
+ int minimum = r.getMinimumValue();
+ int maximum = r.getMaximumValue();
+ if (minimum == maximum) {
+ // Just fall back to a standard integer query.
+ return QueryBuilders.termQuery(p.getField().getName(), minimum);
+ }
+ return QueryBuilders.rangeQuery(p.getField().getName())
+ .gte(minimum)
+ .lte(maximum);
+ }
+ throw new QueryParseException("not an integer range: " + p);
+ }
+
+ private <T> QueryBuilder notTimestamp(TimestampRangePredicate<T> r)
+ throws QueryParseException {
+ if (r.getMinTimestamp().getTime() == 0) {
+ return QueryBuilders.rangeQuery(r.getField().getName())
+ .gt(new DateTime(r.getMaxTimestamp().getTime()));
+ }
+ throw new QueryParseException("cannot negate: " + r);
+ }
+
+ private <T> QueryBuilder timestampQuery(IndexPredicate<T> p)
+ throws QueryParseException {
+ if (p instanceof TimestampRangePredicate) {
+ TimestampRangePredicate<T> r =
+ (TimestampRangePredicate<T>) p;
+ if (p instanceof AfterPredicate) {
+ return QueryBuilders.rangeQuery(r.getField().getName())
+ .gte(new DateTime(r.getMinTimestamp().getTime()));
+ }
+ return QueryBuilders.rangeQuery(r.getField().getName())
+ .gte(new DateTime(r.getMinTimestamp().getTime()))
+ .lte(new DateTime(r.getMaxTimestamp().getTime()));
+ }
+ throw new QueryParseException("not a timestamp: " + p);
+ }
+
+ private <T> QueryBuilder exactQuery(IndexPredicate<T> p){
+ String name = p.getField().getName();
+ String value = p.getValue();
+
+ if (value.isEmpty()) {
+ return new BoolQueryBuilder().mustNot(QueryBuilders.existsQuery(name));
+ } else if (p instanceof RegexPredicate) {
+ if (value.startsWith("^")) {
+ value = value.substring(1);
+ }
+ if (value.endsWith("$") && !value.endsWith("\\$")
+ && !value.endsWith("\\\\$")) {
+ value = value.substring(0, value.length() - 1);
+ }
+ return QueryBuilders.regexpQuery(name + ".key", value);
+ } else {
+ return QueryBuilders.termQuery(name + ".key", value);
+ }
+ }
+}
diff --git a/gerrit-elasticsearch/src/test/java/com/google/gerrit/elasticsearch/ElasticQueryChangesTest.java b/gerrit-elasticsearch/src/test/java/com/google/gerrit/elasticsearch/ElasticQueryChangesTest.java
new file mode 100644
index 0000000..e2e7585
--- /dev/null
+++ b/gerrit-elasticsearch/src/test/java/com/google/gerrit/elasticsearch/ElasticQueryChangesTest.java
@@ -0,0 +1,178 @@
+// Copyright (C) 2014 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.elasticsearch;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.gerrit.elasticsearch.ElasticChangeIndex.CLOSED_CHANGES;
+import static com.google.gerrit.elasticsearch.ElasticChangeIndex.OPEN_CHANGES;
+
+import com.google.common.base.Strings;
+import com.google.common.io.Files;
+import com.google.gerrit.elasticsearch.ElasticChangeIndex.ChangeMapping;
+import com.google.gerrit.server.index.IndexModule.IndexType;
+import com.google.gerrit.server.index.change.ChangeSchemaDefinitions;
+import com.google.gerrit.server.query.change.AbstractQueryChangesTest;
+import com.google.gerrit.testutil.InMemoryModule;
+import com.google.gson.FieldNamingPolicy;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+import org.eclipse.jgit.lib.Config;
+import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeBuilder;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+public class ElasticQueryChangesTest extends AbstractQueryChangesTest {
+ private static final Gson gson = new GsonBuilder()
+ .setFieldNamingPolicy(FieldNamingPolicy.LOWER_CASE_WITH_UNDERSCORES)
+ .create();
+ private static Node node;
+ private static String port;
+ private static File elasticDir;
+
+ static class NodeInfo {
+ String httpAddress;
+ }
+
+ static class Info {
+ Map<String, NodeInfo> nodes;
+ }
+
+ @BeforeClass
+ public static void startIndexService()
+ throws InterruptedException, ExecutionException {
+ if (node != null) {
+ // do not start Elasticsearch twice
+ return;
+ }
+ elasticDir = Files.createTempDir();
+ Path elasticDirPath = elasticDir.toPath();
+ Settings settings = Settings.settingsBuilder()
+ .put("cluster.name", "gerrit")
+ .put("node.name", "Gerrit Elasticsearch Test Node")
+ .put("node.local", true)
+ .put("discovery.zen.ping.multicast.enabled", false)
+ .put("index.store.fs.memory.enabled", true)
+ .put("index.gateway.type", "none")
+ .put("index.max_result_window", Integer.MAX_VALUE)
+ .put("gateway.type", "default")
+ .put("http.port", 0)
+ .put("discovery.zen.ping.unicast.hosts", "[\"localhost\"]")
+ .put("path.home", elasticDirPath.toAbsolutePath())
+ .put("path.data", elasticDirPath.resolve("data").toAbsolutePath())
+ .put("path.work", elasticDirPath.resolve("work").toAbsolutePath())
+ .put("path.logs", elasticDirPath.resolve("logs").toAbsolutePath())
+ .build();
+
+ // Start the node
+ node = NodeBuilder.nodeBuilder()
+ .settings(settings)
+ .node();
+
+ // Wait for it to be ready
+ node.client()
+ .admin()
+ .cluster()
+ .prepareHealth()
+ .setWaitForYellowStatus()
+ .execute()
+ .actionGet();
+
+ createIndexes();
+
+ assertThat(node.isClosed()).isFalse();
+ port = getHttpPort();
+ }
+
+ @After
+ public void cleanupIndex() {
+ node.client().admin().indices().prepareDelete("gerrit").execute();
+ createIndexes();
+ }
+
+ @AfterClass
+ public static void stopElasticsearchServer() {
+ if (node != null) {
+ node.close();
+ node = null;
+ }
+ if (elasticDir != null && elasticDir.delete()) {
+ elasticDir = null;
+ }
+ }
+
+ @Override
+ protected Injector createInjector() {
+ Config elasticsearchConfig = new Config(config);
+ InMemoryModule.setDefaults(elasticsearchConfig);
+ elasticsearchConfig.setEnum("index", null, "type", IndexType.ELASTICSEARCH);
+ elasticsearchConfig.setString("index", null, "protocol", "http");
+ elasticsearchConfig.setString("index", null, "hostname", "localhost");
+ elasticsearchConfig.setString("index", null, "port", port);
+ elasticsearchConfig.setString("index", null, "name", "gerrit");
+ elasticsearchConfig.setBoolean("index", "elasticsearch", "test", true);
+ return Guice.createInjector(
+ new InMemoryModule(elasticsearchConfig, notesMigration));
+ }
+
+ private static void createIndexes() {
+ ChangeMapping openChangesMapping =
+ new ChangeMapping(ChangeSchemaDefinitions.INSTANCE.getLatest());
+ ChangeMapping closedChangesMapping =
+ new ChangeMapping(ChangeSchemaDefinitions.INSTANCE.getLatest());
+ openChangesMapping.closedChanges = null;
+ closedChangesMapping.openChanges = null;
+ node.client()
+ .admin()
+ .indices()
+ .prepareCreate("gerrit")
+ .addMapping(OPEN_CHANGES, gson.toJson(openChangesMapping))
+ .addMapping(CLOSED_CHANGES, gson.toJson(closedChangesMapping))
+ .execute()
+ .actionGet();
+ }
+
+ private static String getHttpPort()
+ throws InterruptedException, ExecutionException {
+ String nodes = node.client().admin().cluster()
+ .nodesInfo(new NodesInfoRequest("*")).get().toString();
+ Gson gson = new GsonBuilder()
+ .setFieldNamingPolicy(FieldNamingPolicy.LOWER_CASE_WITH_UNDERSCORES)
+ .create();
+ Info info = gson.fromJson(nodes, Info.class);
+
+ checkState(info.nodes != null && info.nodes.size() == 1);
+ Iterator<NodeInfo> values = info.nodes.values().iterator();
+ String httpAddress = values.next().httpAddress;
+
+ checkState(
+ !Strings.isNullOrEmpty(httpAddress) && httpAddress.indexOf(':') > 0);
+ return httpAddress.substring(httpAddress.indexOf(':') + 1,
+ httpAddress.length());
+ }
+}
diff --git a/gerrit-index/BUCK b/gerrit-index/BUCK
new file mode 100644
index 0000000..ea97f88
--- /dev/null
+++ b/gerrit-index/BUCK
@@ -0,0 +1,13 @@
+java_library(
+ name = 'index',
+ srcs = glob(['src/main/java/**/*.java']),
+ deps = [
+ '//gerrit-extension-api:api',
+ '//gerrit-server:server',
+ '//gerrit-patch-jgit:server',
+ '//lib/guice:guice',
+ '//lib/jgit/org.eclipse.jgit:jgit',
+ '//lib:guava',
+ ],
+ visibility = ['PUBLIC'],
+)
diff --git a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/GerritIndexStatus.java b/gerrit-index/src/main/java/com/google/gerrit/index/GerritIndexStatus.java
similarity index 84%
rename from gerrit-lucene/src/main/java/com/google/gerrit/lucene/GerritIndexStatus.java
rename to gerrit-index/src/main/java/com/google/gerrit/index/GerritIndexStatus.java
index f43e385..cafd30e 100644
--- a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/GerritIndexStatus.java
+++ b/gerrit-index/src/main/java/com/google/gerrit/index/GerritIndexStatus.java
@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
-package com.google.gerrit.lucene;
+package com.google.gerrit.index;
import com.google.common.primitives.Ints;
import com.google.gerrit.server.config.SitePaths;
@@ -24,13 +24,13 @@
import java.io.IOException;
-class GerritIndexStatus {
+public class GerritIndexStatus {
private static final String SECTION = "index";
private static final String KEY_READY = "ready";
private final FileBasedConfig cfg;
- GerritIndexStatus(SitePaths sitePaths)
+ public GerritIndexStatus(SitePaths sitePaths)
throws ConfigInvalidException, IOException {
cfg = new FileBasedConfig(
sitePaths.index_dir.resolve("gerrit_index.config").toFile(),
@@ -39,16 +39,16 @@
convertLegacyConfig();
}
- void setReady(String indexName, int version, boolean ready) {
+ public void setReady(String indexName, int version, boolean ready) {
cfg.setBoolean(SECTION, indexDirName(indexName, version), KEY_READY, ready);
}
- boolean getReady(String indexName, int version) {
+ public boolean getReady(String indexName, int version) {
return cfg.getBoolean(SECTION, indexDirName(indexName, version), KEY_READY,
false);
}
- void save() throws IOException {
+ public void save() throws IOException {
cfg.save();
}
@@ -62,8 +62,8 @@
if (ready != null) {
dirty = false;
cfg.unset(SECTION, subsection, KEY_READY);
- cfg.setString(SECTION,
- indexDirName(ChangeSchemaDefinitions.NAME, v), KEY_READY, ready);
+ cfg.setString(SECTION, indexDirName(ChangeSchemaDefinitions.NAME, v),
+ KEY_READY, ready);
}
}
}
diff --git a/gerrit-index/src/main/java/com/google/gerrit/index/IndexUtils.java b/gerrit-index/src/main/java/com/google/gerrit/index/IndexUtils.java
new file mode 100644
index 0000000..f00f5c2
--- /dev/null
+++ b/gerrit-index/src/main/java/com/google/gerrit/index/IndexUtils.java
@@ -0,0 +1,67 @@
+// Copyright (C) 2013 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.index;
+
+import static com.google.gerrit.server.index.change.ChangeField.CHANGE;
+import static com.google.gerrit.server.index.change.ChangeField.LEGACY_ID;
+import static com.google.gerrit.server.index.change.ChangeField.PROJECT;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.gerrit.server.config.SitePaths;
+import com.google.gerrit.server.index.QueryOptions;
+
+import org.eclipse.jgit.errors.ConfigInvalidException;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+public final class IndexUtils {
+ public static final Map<String, String> CUSTOM_CHAR_MAPPING =
+ ImmutableMap.of("_", " ", ".", " ");
+
+ public static void setReady(SitePaths sitePaths, String name, int version,
+ boolean ready) throws IOException {
+ try {
+ GerritIndexStatus cfg = new GerritIndexStatus(sitePaths);
+ cfg.setReady(name, version, ready);
+ cfg.save();
+ } catch (ConfigInvalidException e) {
+ throw new IOException(e);
+ }
+ }
+
+ public static Set<String> fields(QueryOptions opts) {
+ // Ensure we request enough fields to construct a ChangeData. We need both
+ // change ID and project, which can either come via the Change field or
+ // separate fields.
+ Set<String> fs = opts.fields();
+ if (fs.contains(CHANGE.getName())) {
+ // A Change is always sufficient.
+ return fs;
+ }
+ if (fs.contains(PROJECT.getName()) && fs.contains(LEGACY_ID.getName())) {
+ return fs;
+ }
+ return Sets.union(fs,
+ ImmutableSet.of(LEGACY_ID.getName(), PROJECT.getName()));
+ }
+
+ private IndexUtils() {
+ // hide default constructor
+ }
+}
diff --git a/gerrit-index/src/main/java/com/google/gerrit/index/SingleVersionModule.java b/gerrit-index/src/main/java/com/google/gerrit/index/SingleVersionModule.java
new file mode 100644
index 0000000..55b9b57
--- /dev/null
+++ b/gerrit-index/src/main/java/com/google/gerrit/index/SingleVersionModule.java
@@ -0,0 +1,106 @@
+// Copyright (C) 2013 The Android Open Source Project
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.gerrit.index;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.gerrit.extensions.events.LifecycleListener;
+import com.google.gerrit.lifecycle.LifecycleModule;
+import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.index.Index;
+import com.google.gerrit.server.index.IndexDefinition;
+import com.google.gerrit.server.index.Schema;
+import com.google.inject.Inject;
+import com.google.inject.ProvisionException;
+import com.google.inject.Singleton;
+import com.google.inject.TypeLiteral;
+import com.google.inject.name.Named;
+import com.google.inject.name.Names;
+
+import org.eclipse.jgit.lib.Config;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+@Singleton
+public class SingleVersionModule extends LifecycleModule {
+ static final String SINGLE_VERSIONS = "LuceneIndexModule/SingleVersions";
+
+ private final Map<String, Integer> singleVersions;
+
+ public SingleVersionModule(Map<String, Integer> singleVersions) {
+ this.singleVersions = singleVersions;
+ }
+
+ @Override
+ public void configure() {
+ listener().to(SingleVersionListener.class);
+ bind(new TypeLiteral<Map<String, Integer>>() {})
+ .annotatedWith(Names.named(SINGLE_VERSIONS))
+ .toInstance(singleVersions);
+ }
+
+ @Singleton
+ static class SingleVersionListener implements LifecycleListener {
+ private final Set<String> disabled;
+ private final Collection<IndexDefinition<?, ?, ?>> defs;
+ private final Map<String, Integer> singleVersions;
+
+ @Inject
+ SingleVersionListener(
+ @GerritServerConfig Config cfg,
+ Collection<IndexDefinition<?, ?, ?>> defs,
+ @Named(SINGLE_VERSIONS) Map<String, Integer> singleVersions) {
+ this.defs = defs;
+ this.singleVersions = singleVersions;
+
+ disabled = ImmutableSet.copyOf(
+ cfg.getStringList("index", null, "testDisable"));
+ }
+
+ @Override
+ public void start() {
+ for (IndexDefinition<?, ?, ?> def : defs) {
+ start(def);
+ }
+ }
+
+ private <K, V, I extends Index<K, V>> void start(
+ IndexDefinition<K, V, I> def) {
+ if (disabled.contains(def.getName())) {
+ return;
+ }
+ Schema<V> schema;
+ Integer v = singleVersions.get(def.getName());
+ if (v == null) {
+ schema = def.getLatest();
+ } else {
+ schema = def.getSchemas().get(v);
+ if (schema == null) {
+ throw new ProvisionException(String.format(
+ "Unrecognized %s schema version: %s", def.getName(), v));
+ }
+ }
+ I index = def.getIndexFactory().create(schema);
+ def.getIndexCollection().setSearchIndex(index);
+ def.getIndexCollection().addWriteIndex(index);
+ }
+
+ @Override
+ public void stop() {
+ // Do nothing; indexes are closed by IndexCollection.
+ }
+ }
+}
\ No newline at end of file
diff --git a/gerrit-lucene/BUCK b/gerrit-lucene/BUCK
index 771a021..f4f097c 100644
--- a/gerrit-lucene/BUCK
+++ b/gerrit-lucene/BUCK
@@ -27,6 +27,7 @@
'//gerrit-extension-api:api',
'//gerrit-reviewdb:server',
'//gerrit-server:server',
+ '//gerrit-index:index',
'//lib:guava',
'//lib:gwtorm',
'//lib/guice:guice',
diff --git a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/AbstractLuceneIndex.java b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/AbstractLuceneIndex.java
index eb0dfaa..e869afb 100644
--- a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/AbstractLuceneIndex.java
+++ b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/AbstractLuceneIndex.java
@@ -25,6 +25,7 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.gerrit.index.IndexUtils;
import com.google.gerrit.server.config.SitePaths;
import com.google.gerrit.server.index.FieldDef;
import com.google.gerrit.server.index.FieldDef.FillArgs;
@@ -51,7 +52,6 @@
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
-import org.eclipse.jgit.errors.ConfigInvalidException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -75,17 +75,6 @@
return f.getName() + "_SORT";
}
- public static void setReady(SitePaths sitePaths, String name, int version,
- boolean ready) throws IOException {
- try {
- GerritIndexStatus cfg = new GerritIndexStatus(sitePaths);
- cfg.setReady(name, version, ready);
- cfg.save();
- } catch (ConfigInvalidException e) {
- throw new IOException(e);
- }
- }
-
private final Schema<V> schema;
private final SitePaths sitePaths;
private final Directory dir;
@@ -198,7 +187,7 @@
@Override
public void markReady(boolean ready) throws IOException {
- setReady(sitePaths, name, schema.getVersion(), ready);
+ IndexUtils.setReady(sitePaths, name, schema.getVersion(), ready);
}
@Override
diff --git a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneChangeIndex.java b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneChangeIndex.java
index 2decff5..4775ac4 100644
--- a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneChangeIndex.java
+++ b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneChangeIndex.java
@@ -18,7 +18,6 @@
import static com.google.gerrit.lucene.AbstractLuceneIndex.sortFieldName;
import static com.google.gerrit.lucene.LuceneVersionManager.CHANGES_PREFIX;
import static com.google.gerrit.server.git.QueueProvider.QueueType.INTERACTIVE;
-import static com.google.gerrit.server.index.change.ChangeField.CHANGE;
import static com.google.gerrit.server.index.change.ChangeField.LEGACY_ID;
import static com.google.gerrit.server.index.change.ChangeField.PROJECT;
import static com.google.gerrit.server.index.change.ChangeIndexRewriter.CLOSED_STATUSES;
@@ -27,7 +26,6 @@
import com.google.common.base.Throwables;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
@@ -35,6 +33,7 @@
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.gerrit.index.IndexUtils;
import com.google.gerrit.reviewdb.client.Account;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.PatchSet;
@@ -317,7 +316,7 @@
throw new OrmException("interrupted");
}
- final Set<String> fields = fields(opts);
+ final Set<String> fields = IndexUtils.fields(opts);
return new ChangeDataResults(
executor.submit(new Callable<List<Document>>() {
@Override
@@ -405,22 +404,6 @@
}
}
- private Set<String> fields(QueryOptions opts) {
- // Ensure we request enough fields to construct a ChangeData. We need both
- // change ID and project, which can either come via the Change field or
- // separate fields.
- Set<String> fs = opts.fields();
- if (fs.contains(CHANGE.getName())) {
- // A Change is always sufficient.
- return fs;
- }
- if (fs.contains(PROJECT.getName()) && fs.contains(LEGACY_ID.getName())) {
- return fs;
- }
- return Sets.union(fs,
- ImmutableSet.of(LEGACY_ID.getName(), PROJECT.getName()));
- }
-
private static Multimap<String, IndexableField> fields(Document doc,
Set<String> fields) {
Multimap<String, IndexableField> stored =
diff --git a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneIndexModule.java b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneIndexModule.java
index f5d5146..58890176 100644
--- a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneIndexModule.java
+++ b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneIndexModule.java
@@ -15,37 +15,23 @@
package com.google.gerrit.lucene;
import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
-import com.google.gerrit.extensions.events.LifecycleListener;
+import com.google.gerrit.index.SingleVersionModule;
import com.google.gerrit.lifecycle.LifecycleModule;
import com.google.gerrit.server.config.GerritServerConfig;
-import com.google.gerrit.server.index.Index;
import com.google.gerrit.server.index.IndexConfig;
-import com.google.gerrit.server.index.IndexDefinition;
import com.google.gerrit.server.index.IndexModule;
-import com.google.gerrit.server.index.Schema;
import com.google.gerrit.server.index.account.AccountIndex;
import com.google.gerrit.server.index.change.ChangeIndex;
-import com.google.inject.Inject;
import com.google.inject.Provides;
-import com.google.inject.ProvisionException;
import com.google.inject.Singleton;
-import com.google.inject.TypeLiteral;
import com.google.inject.assistedinject.FactoryModuleBuilder;
-import com.google.inject.name.Named;
-import com.google.inject.name.Names;
import org.apache.lucene.search.BooleanQuery;
import org.eclipse.jgit.lib.Config;
-import java.util.Collection;
import java.util.Map;
-import java.util.Set;
public class LuceneIndexModule extends LifecycleModule {
- private static final String SINGLE_VERSIONS =
- "LuceneIndexModule/SingleVersions";
-
public static LuceneIndexModule singleVersionAllLatest(int threads) {
return new LuceneIndexModule(ImmutableMap.<String, Integer> of(), threads);
}
@@ -86,7 +72,7 @@
if (singleVersions == null) {
install(new MultiVersionModule());
} else {
- install(new SingleVersionModule());
+ install(new SingleVersionModule(singleVersions));
}
}
@@ -104,66 +90,4 @@
listener().to(LuceneVersionManager.class);
}
}
-
- private class SingleVersionModule extends LifecycleModule {
- @Override
- public void configure() {
- listener().to(SingleVersionListener.class);
- bind(new TypeLiteral<Map<String, Integer>>() {})
- .annotatedWith(Names.named(SINGLE_VERSIONS))
- .toInstance(singleVersions);
- }
- }
-
- @Singleton
- static class SingleVersionListener implements LifecycleListener {
- private final Set<String> disabled;
- private final Collection<IndexDefinition<?, ?, ?>> defs;
- private final Map<String, Integer> singleVersions;
-
- @Inject
- SingleVersionListener(
- @GerritServerConfig Config cfg,
- Collection<IndexDefinition<?, ?, ?>> defs,
- @Named(SINGLE_VERSIONS) Map<String, Integer> singleVersions) {
- this.defs = defs;
- this.singleVersions = singleVersions;
-
- disabled = ImmutableSet.copyOf(
- cfg.getStringList("index", null, "testDisable"));
- }
-
- @Override
- public void start() {
- for (IndexDefinition<?, ?, ?> def : defs) {
- start(def);
- }
- }
-
- private <K, V, I extends Index<K, V>> void start(
- IndexDefinition<K, V, I> def) {
- if (disabled.contains(def.getName())) {
- return;
- }
- Schema<V> schema;
- Integer v = singleVersions.get(def.getName());
- if (v == null) {
- schema = def.getLatest();
- } else {
- schema = def.getSchemas().get(v);
- if (schema == null) {
- throw new ProvisionException(String.format(
- "Unrecognized %s schema version: %s", def.getName(), v));
- }
- }
- I index = def.getIndexFactory().create(schema);
- def.getIndexCollection().setSearchIndex(index);
- def.getIndexCollection().addWriteIndex(index);
- }
-
- @Override
- public void stop() {
- // Do nothing; indexes are closed by IndexCollection.
- }
- }
}
diff --git a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneVersionManager.java b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneVersionManager.java
index b46f1f6..2f871fc 100644
--- a/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneVersionManager.java
+++ b/gerrit-lucene/src/main/java/com/google/gerrit/lucene/LuceneVersionManager.java
@@ -20,6 +20,7 @@
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import com.google.gerrit.extensions.events.LifecycleListener;
+import com.google.gerrit.index.GerritIndexStatus;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.config.SitePaths;
import com.google.gerrit.server.index.Index;
diff --git a/gerrit-pgm/BUCK b/gerrit-pgm/BUCK
index 8852133..5f2ef43 100644
--- a/gerrit-pgm/BUCK
+++ b/gerrit-pgm/BUCK
@@ -47,7 +47,7 @@
':init-api',
':util',
'//gerrit-common:annotations',
- '//gerrit-lucene:lucene',
+ '//gerrit-index:index',
'//lib:args4j',
'//lib:derby',
'//lib:gwtjsonrpc',
@@ -66,6 +66,7 @@
REST_UTIL_DEPS = [
'//gerrit-cache-h2:cache-h2',
+ '//gerrit-elasticsearch:elasticsearch',
'//gerrit-util-cli:cli',
'//lib:args4j',
'//lib:gwtorm',
@@ -120,6 +121,7 @@
':init-api',
':util',
'//gerrit-cache-h2:cache-h2',
+ '//gerrit-elasticsearch:elasticsearch',
'//gerrit-gpg:gpg',
'//gerrit-lucene:lucene',
'//gerrit-oauth:oauth',
diff --git a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Daemon.java b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Daemon.java
index eb17530..9d4120c 100644
--- a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Daemon.java
+++ b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Daemon.java
@@ -20,6 +20,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import com.google.gerrit.common.EventBroker;
+import com.google.gerrit.elasticsearch.ElasticIndexModule;
import com.google.gerrit.extensions.client.AuthType;
import com.google.gerrit.gpg.GpgModule;
import com.google.gerrit.httpd.AllRequestFilter;
@@ -408,15 +409,18 @@
return cfgInjector.createChildInjector(modules);
}
- private AbstractModule createIndexModule() {
+ private Module createIndexModule() {
if (slave) {
return new DummyIndexModule();
}
+ if (luceneModule != null) {
+ return luceneModule;
+ }
switch (indexType) {
case LUCENE:
- return luceneModule != null
- ? luceneModule
- : LuceneIndexModule.latestVersionWithOnlineUpgrade();
+ return LuceneIndexModule.latestVersionWithOnlineUpgrade();
+ case ELASTICSEARCH:
+ return ElasticIndexModule.latestVersionWithOnlineUpgrade();
default:
throw new IllegalStateException("unsupported index.type = " + indexType);
}
@@ -426,6 +430,7 @@
indexType = IndexModule.getIndexType(cfgInjector);
switch (indexType) {
case LUCENE:
+ case ELASTICSEARCH:
break;
default:
throw new IllegalStateException("unsupported index.type = " + indexType);
diff --git a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Reindex.java b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Reindex.java
index 501b115..ee0d02f 100644
--- a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Reindex.java
+++ b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/Reindex.java
@@ -20,6 +20,7 @@
import com.google.common.collect.Sets;
import com.google.gerrit.common.Die;
+import com.google.gerrit.elasticsearch.ElasticIndexModule;
import com.google.gerrit.extensions.config.FactoryModule;
import com.google.gerrit.lifecycle.LifecycleManager;
import com.google.gerrit.lucene.LuceneIndexModule;
@@ -161,6 +162,10 @@
indexModule = LuceneIndexModule.singleVersionWithExplicitVersions(
versions, threads);
break;
+ case ELASTICSEARCH:
+ indexModule = ElasticIndexModule
+ .singleVersionWithExplicitVersions(versions, threads);
+ break;
default:
throw new IllegalStateException("unsupported index.type");
}
diff --git a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/init/InitIndex.java b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/init/InitIndex.java
index 185063b..c8d8edb 100644
--- a/gerrit-pgm/src/main/java/com/google/gerrit/pgm/init/InitIndex.java
+++ b/gerrit-pgm/src/main/java/com/google/gerrit/pgm/init/InitIndex.java
@@ -15,7 +15,8 @@
package com.google.gerrit.pgm.init;
import com.google.common.collect.Iterables;
-import com.google.gerrit.lucene.AbstractLuceneIndex;
+import com.google.common.collect.Sets;
+import com.google.gerrit.index.IndexUtils;
import com.google.gerrit.pgm.init.api.ConsoleUI;
import com.google.gerrit.pgm.init.api.InitFlags;
import com.google.gerrit.pgm.init.api.InitStep;
@@ -61,9 +62,17 @@
type = index.select("Type", "type", type);
}
+ if (type == IndexType.ELASTICSEARCH) {
+ index.select("Transport protocol", "protocol", "http",
+ Sets.newHashSet("http", "https"));
+ index.string("Hostname", "hostname", "localhost");
+ index.string("Port", "port", "9200");
+ index.string("Index Name", "name", "gerrit");
+ }
+
if ((site.isNew || isEmptySite()) && type == IndexType.LUCENE) {
for (SchemaDefinitions<?> def : IndexModule.ALL_SCHEMA_DEFS) {
- AbstractLuceneIndex.setReady(
+ IndexUtils.setReady(
site, def.getName(), def.getLatest().getVersion(), true);
}
} else {
diff --git a/gerrit-server/BUCK b/gerrit-server/BUCK
index 66fc545..a50df82 100644
--- a/gerrit-server/BUCK
+++ b/gerrit-server/BUCK
@@ -91,6 +91,7 @@
':server',
'//gerrit-common:server',
'//gerrit-cache-h2:cache-h2',
+ '//gerrit-elasticsearch:elasticsearch',
'//gerrit-extension-api:api',
'//gerrit-gpg:gpg',
'//gerrit-lucene:lucene',
@@ -181,6 +182,7 @@
'//gerrit-server/src/main/prolog:common',
'//lib/antlr:java_runtime',
],
+ visibility = ['PUBLIC'],
)
java_test(
diff --git a/gerrit-server/src/main/java/com/google/gerrit/server/index/IndexModule.java b/gerrit-server/src/main/java/com/google/gerrit/server/index/IndexModule.java
index 6a25862..9e0be86 100644
--- a/gerrit-server/src/main/java/com/google/gerrit/server/index/IndexModule.java
+++ b/gerrit-server/src/main/java/com/google/gerrit/server/index/IndexModule.java
@@ -55,7 +55,7 @@
*/
public class IndexModule extends LifecycleModule {
public enum IndexType {
- LUCENE
+ LUCENE, ELASTICSEARCH
}
public static final ImmutableCollection<SchemaDefinitions<?>> ALL_SCHEMA_DEFS =
diff --git a/gerrit-server/src/test/java/com/google/gerrit/testutil/InMemoryModule.java b/gerrit-server/src/test/java/com/google/gerrit/testutil/InMemoryModule.java
index 84fd9d7..71401d7 100644
--- a/gerrit-server/src/test/java/com/google/gerrit/testutil/InMemoryModule.java
+++ b/gerrit-server/src/test/java/com/google/gerrit/testutil/InMemoryModule.java
@@ -220,6 +220,9 @@
case LUCENE:
install(luceneIndexModule());
break;
+ case ELASTICSEARCH:
+ install(elasticIndexModule());
+ break;
default:
throw new ProvisionException(
"index type unsupported in tests: " + indexType);
@@ -242,14 +245,21 @@
}
private Module luceneIndexModule() {
+ return indexModule("com.google.gerrit.lucene.LuceneIndexModule");
+ }
+
+ private Module elasticIndexModule() {
+ return indexModule("com.google.gerrit.elasticsearch.ElasticIndexModule");
+ }
+
+ private Module indexModule(String moduleClassName) {
try {
Map<String, Integer> singleVersions = new HashMap<>();
int version = cfg.getInt("index", "lucene", "testVersion", -1);
if (version > 0) {
singleVersions.put(ChangeSchemaDefinitions.INSTANCE.getName(), version);
}
- Class<?> clazz =
- Class.forName("com.google.gerrit.lucene.LuceneIndexModule");
+ Class<?> clazz = Class.forName(moduleClassName);
Method m = clazz.getMethod(
"singleVersionWithExplicitVersions", Map.class, int.class);
return (Module) m.invoke(null, singleVersions, 0);
diff --git a/gerrit-war/BUCK b/gerrit-war/BUCK
index 6d74a83..5dd1b04 100644
--- a/gerrit-war/BUCK
+++ b/gerrit-war/BUCK
@@ -5,6 +5,7 @@
srcs = glob(['src/main/java/**/*.java']),
deps = [
'//gerrit-cache-h2:cache-h2',
+ '//gerrit-elasticsearch:elasticsearch',
'//gerrit-extension-api:api',
'//gerrit-gpg:gpg',
'//gerrit-httpd:httpd',
diff --git a/gerrit-war/src/main/java/com/google/gerrit/httpd/WebAppInitializer.java b/gerrit-war/src/main/java/com/google/gerrit/httpd/WebAppInitializer.java
index fc0beae..df0284a 100644
--- a/gerrit-war/src/main/java/com/google/gerrit/httpd/WebAppInitializer.java
+++ b/gerrit-war/src/main/java/com/google/gerrit/httpd/WebAppInitializer.java
@@ -20,6 +20,7 @@
import com.google.common.base.Splitter;
import com.google.gerrit.common.EventBroker;
import com.google.gerrit.extensions.client.AuthType;
+import com.google.gerrit.elasticsearch.ElasticIndexModule;
import com.google.gerrit.gpg.GpgModule;
import com.google.gerrit.httpd.auth.oauth.OAuthModule;
import com.google.gerrit.httpd.auth.openid.OpenIdModule;
@@ -343,6 +344,8 @@
switch (indexType) {
case LUCENE:
return LuceneIndexModule.latestVersionWithOnlineUpgrade();
+ case ELASTICSEARCH:
+ return ElasticIndexModule.latestVersionWithOnlineUpgrade();
default:
throw new IllegalStateException("unsupported index.type = " + indexType);
}
diff --git a/lib/commons/BUCK b/lib/commons/BUCK
index 7c27477..55c07a6 100644
--- a/lib/commons/BUCK
+++ b/lib/commons/BUCK
@@ -47,6 +47,13 @@
)
maven_jar(
+ name = 'lang3',
+ id = 'org.apache.commons:commons-lang3:3.3.2',
+ sha1 = '90a3822c38ec8c996e84c16a3477ef632cbc87a3',
+ license = 'Apache2.0',
+)
+
+maven_jar(
name = 'net',
id = 'commons-net:commons-net:3.5',
sha1 = '342fc284019f590e1308056990fdb24a08f06318',
diff --git a/lib/elasticsearch/BUCK b/lib/elasticsearch/BUCK
new file mode 100644
index 0000000..86594ce
--- /dev/null
+++ b/lib/elasticsearch/BUCK
@@ -0,0 +1,104 @@
+include_defs('//lib/maven.defs')
+
+# Java client library for Elasticsearch.
+maven_jar(
+ name = 'elasticsearch',
+ id = 'org.elasticsearch:elasticsearch:2.4.0',
+ sha1 = 'aeb9704a76fa8654c348f38fcbb993a952a7ab07',
+ attach_source = True,
+ repository = MAVEN_CENTRAL,
+ license = 'Apache2.0',
+ deps = [
+ ':jna',
+ ':hppc',
+ ':jsr166e',
+ ':netty',
+ ':t-digest',
+ ':compress-lzf',
+ '//lib/joda:joda-time',
+ '//lib/lucene:lucene-codecs',
+ '//lib/lucene:lucene-highlighter',
+ '//lib/lucene:lucene-join',
+ '//lib/lucene:lucene-memory',
+ '//lib/lucene:lucene-sandbox',
+ '//lib/lucene:lucene-suggest',
+ '//lib/lucene:lucene-queries',
+ '//lib/lucene:lucene-spatial',
+ '//lib/jackson:jackson-core',
+ '//lib/jackson:jackson-dataformat-cbor',
+ '//lib/jackson:jackson-dataformat-smile',
+ ]
+)
+
+# Java REST client for Elasticsearch.
+VERSION = '0.1.7'
+
+maven_jar(
+ name = 'jest-common',
+ id = 'io.searchbox:jest-common:' + VERSION,
+ sha1 = 'ff6e2694405557a3a02b444cb7f7da28c4d99f07',
+ license = 'Apache2.0',
+)
+
+maven_jar(
+ name = 'jest',
+ id = 'io.searchbox:jest:' + VERSION,
+ sha1 = '686619c7141edb50b562ad2a39d32ea4cf20b567',
+ license = 'Apache2.0',
+ deps = [
+ ':elasticsearch',
+ ':jest-common',
+ '//lib/commons:lang3',
+ '//lib/httpcomponents:httpasyncclient',
+ '//lib/httpcomponents:httpclient',
+ '//lib/httpcomponents:httpcore-nio',
+ '//lib/httpcomponents:httpcore-niossl',
+ ],
+)
+
+maven_jar(
+ name = 'compress-lzf',
+ id = 'com.ning:compress-lzf:1.0.2',
+ sha1 = '62896e6fca184c79cc01a14d143f3ae2b4f4b4ae',
+ license = 'Apache2.0',
+ visibility = ['//lib/elasticsearch:elasticsearch'],
+)
+
+maven_jar(
+ name = 'hppc',
+ id = 'com.carrotsearch:hppc:0.7.1',
+ sha1 = '8b5057f74ea378c0150a1860874a3ebdcb713767',
+ license = 'Apache2.0',
+ visibility = ['//lib/elasticsearch:elasticsearch'],
+)
+
+maven_jar(
+ name = 'jsr166e',
+ id = 'com.twitter:jsr166e:1.1.0',
+ sha1 = '233098147123ee5ddcd39ffc57ff648be4b7e5b2',
+ license = 'Apache2.0',
+ visibility = ['//lib/elasticsearch:elasticsearch'],
+)
+
+maven_jar(
+ name = 'netty',
+ id = 'io.netty:netty:3.10.0.Final',
+ sha1 = 'ad61cd1bba067e6634ddd3e160edf0727391ac30',
+ license = 'Apache2.0',
+ visibility = ['//lib/elasticsearch:elasticsearch'],
+)
+
+maven_jar(
+ name = 't-digest',
+ id = 'com.tdunning:t-digest:3.0',
+ sha1 = '84ccf145ac2215e6bfa63baa3101c0af41017cfc',
+ license = 'Apache2.0',
+ visibility = ['//lib/elasticsearch:elasticsearch'],
+)
+
+maven_jar(
+ name = 'jna',
+ id = 'net.java.dev.jna:jna:4.1.0',
+ sha1 = '1c12d070e602efd8021891cdd7fd18bc129372d4',
+ license = 'Apache2.0',
+)
diff --git a/lib/httpcomponents/BUCK b/lib/httpcomponents/BUCK
index 03669f2..1e56f94 100644
--- a/lib/httpcomponents/BUCK
+++ b/lib/httpcomponents/BUCK
@@ -39,3 +39,25 @@
src_sha1 = '5394d3715181a87009032335a55b0a9789f6e26f',
license = 'Apache2.0',
)
+
+maven_jar(
+ name = 'httpasyncclient',
+ id = 'org.apache.httpcomponents:httpasyncclient:4.1.2',
+ sha1 = '95aa3e6fb520191a0970a73cf09f62948ee614be',
+ license = 'Apache2.0',
+)
+
+maven_jar(
+ name = 'httpcore-nio',
+ id = 'org.apache.httpcomponents:httpcore-nio:' + VERSION,
+ sha1 = 'a8c5e3c3bfea5ce23fb647c335897e415eb442e3',
+ license = 'Apache2.0',
+)
+
+maven_jar(
+ name = 'httpcore-niossl',
+ id = 'org.apache.httpcomponents:httpcore-niossl:4.0-alpha6',
+ sha1 = '9c662e7247ca8ceb1de5de629f685c9ef3e4ab58',
+ license = 'Apache2.0',
+ attach_source = False,
+)
diff --git a/lib/jackson/BUCK b/lib/jackson/BUCK
new file mode 100644
index 0000000..46056b5
--- /dev/null
+++ b/lib/jackson/BUCK
@@ -0,0 +1,26 @@
+include_defs('//lib/maven.defs')
+
+VERSION = '2.6.6'
+
+maven_jar(
+ name = 'jackson-core',
+ id = 'com.fasterxml.jackson.core:jackson-core:' + VERSION,
+ sha1 = '02eb801df67aacaf5b1deb4ac626e1964508e47b',
+ license = 'Apache2.0',
+)
+
+maven_jar(
+ name = 'jackson-dataformat-smile',
+ id = 'com.fasterxml.jackson.dataformat:jackson-dataformat-smile:' + VERSION,
+ sha1 = 'ccbfc948748ed2754a58c1af9e0a02b5cc1aed69',
+ license = 'Apache2.0',
+)
+
+maven_jar(
+ name = 'jackson-dataformat-cbor',
+ id = 'com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:' + VERSION,
+ sha1 = '34c7b7ff495fc6b049612bdc9db0900a68e112f8',
+ license = 'Apache2.0'
+)
+
+
diff --git a/lib/lucene/BUCK b/lib/lucene/BUCK
index dee8ce8..8f2efa2 100644
--- a/lib/lucene/BUCK
+++ b/lib/lucene/BUCK
@@ -14,6 +14,17 @@
)
maven_jar(
+ name = 'lucene-codecs',
+ id = 'org.apache.lucene:lucene-codecs:' + VERSION,
+ sha1 = 'e01fe463d9490bb1b4a6a168e771f7b7255a50b1',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
name = 'lucene-core',
id = 'org.apache.lucene:lucene-core:' + VERSION,
sha1 = 'de5e5c3161ea01e89f2a09a14391f9b7ed66cdbb',
@@ -22,7 +33,7 @@
'META-INF/LICENSE.txt',
'META-INF/NOTICE.txt',
],
- visibility = [],
+ visibility = ['//gerrit-elasticsearch:elasticsearch'],
)
maven_jar(
@@ -51,6 +62,39 @@
)
maven_jar(
+ name = 'lucene-highlighter',
+ id = 'org.apache.lucene:lucene-highlighter:' + VERSION,
+ sha1 = 'd127ac514e9df965ab0b57d92bbe0c68d3d145b8',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
+ name = 'lucene-join',
+ id = 'org.apache.lucene:lucene-join:'+ VERSION,
+ sha1 = 'dac1b322508f3f2696ecc49a97311d34d8382054',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
+ name = 'lucene-memory',
+ id = 'org.apache.lucene:lucene-memory:' + VERSION,
+ sha1 = '7409db9863d8fbc265c27793c6cc7511304182c2',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
name = 'lucene-misc',
id = 'org.apache.lucene:lucene-misc:' + VERSION,
sha1 = '37bbe5a2fb429499dfbe75d750d1778881fff45d',
@@ -63,6 +107,49 @@
)
maven_jar(
+ name = 'lucene-sandbox',
+ id = 'org.apache.lucene:lucene-sandbox:' + VERSION,
+ sha1 = '30a91f120706ba66732d5a974b56c6971b3c8a16',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
+ name = 'lucene-spatial',
+ id = 'org.apache.lucene:lucene-spatial:' + VERSION,
+ sha1 = '8ed7a9a43d78222038573dd1c295a61f3c0bb0db',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+maven_jar(
+ name = 'lucene-suggest',
+ id = 'org.apache.lucene:lucene-suggest:' + VERSION,
+ sha1 = 'e8316b37dddcf2092a54dab2ce6aad0d5ad78585',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
+ name = 'lucene-queries',
+ id = 'org.apache.lucene:lucene-queries:' + VERSION,
+ sha1 = '692f1ad887cf4e006a23f45019e6de30f3312d3f',
+ license = 'Apache2.0',
+ exclude = [
+ 'META-INF/LICENSE.txt',
+ 'META-INF/NOTICE.txt',
+ ],
+)
+
+maven_jar(
name = 'lucene-queryparser',
id = 'org.apache.lucene:lucene-queryparser:' + VERSION,
sha1 = '8ac921563e744463605284c6d9d2d95e1be5b87c',
@@ -73,3 +160,4 @@
'META-INF/NOTICE.txt',
],
)
+