Extract interfaces for Index and IndexCollection

Add another type argument for the key type, e.g. Change.Id. Keep
subinterfaces ChangeIndex and ChangeIndexCollection to avoid having to
pass around lots of type arguments; Schema<ChangeData> was bad enough,
adding one more would be tortuous.

Change-Id: Id54b985692ca41ffcaabe9793d8b9035cb1d4e82
This commit is contained in:
Dave Borowitz
2016-03-15 14:18:51 +01:00
parent e023347830
commit da0459d20c
30 changed files with 235 additions and 183 deletions

View File

@@ -14,7 +14,6 @@
package com.google.gerrit.server.index;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.DummyChangeIndex;
import com.google.inject.AbstractModule;
@@ -24,6 +23,6 @@ public class DummyIndexModule extends AbstractModule {
protected void configure() {
install(new IndexModule(1));
bind(IndexConfig.class).toInstance(IndexConfig.createDefault());
bind(ChangeIndex.class).toInstance(new DummyChangeIndex());
bind(Index.class).toInstance(new DummyChangeIndex());
}
}

View File

@@ -0,0 +1,100 @@
// 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.server.index;
import com.google.gerrit.server.query.DataSource;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
import java.io.IOException;
/**
* Secondary index implementation for arbitrary documents.
* <p>
* Documents are inserted into the index and are queried by converting special
* {@link com.google.gerrit.server.query.Predicate} instances into index-aware
* predicates that use the index search results as a source.
* <p>
* Implementations must be thread-safe and should batch inserts/updates where
* appropriate.
*/
public interface Index<K, V> {
/** @return the schema version used by this index. */
Schema<V> getSchema();
/** Close this index. */
void close();
/**
* Update a document in the index.
* <p>
* Semantically equivalent to deleting the document and reinserting it with
* new field values. A document that does not already exist is created. Results
* may not be immediately visible to searchers, but should be visible within a
* reasonable amount of time.
*
* @param obj document object
*
* @throws IOException
*/
void replace(V obj) throws IOException;
/**
* Delete a document from the index by key.
*
* @param key document key
*
* @throws IOException
*/
void delete(K key) throws IOException;
/**
* Delete all documents from the index.
*
* @throws IOException
*/
void deleteAll() throws IOException;
/**
* Convert the given operator predicate into a source searching the index and
* returning only the documents matching that predicate.
* <p>
* This method may be called multiple times for variations on the same
* predicate or multiple predicate subtrees in the course of processing a
* single query, so it should not have any side effects (e.g. starting a
* search in the background).
*
* @param p the predicate to match. Must be a tree containing only AND, OR,
* or NOT predicates as internal nodes, and {@link IndexPredicate}s as
* leaves.
* @param opts query options not implied by the predicate, such as start and
* limit.
* @return a source of documents matching the predicate, returned in a
* defined order depending on the type of documents.
*
* @throws QueryParseException if the predicate could not be converted to an
* indexed data source.
*/
DataSource<V> getSource(Predicate<V> p, QueryOptions opts)
throws QueryParseException;
/**
* Mark whether this index is up-to-date and ready to serve reads.
*
* @param ready whether the index is ready
* @throws IOException
*/
void markReady(boolean ready) throws IOException;
}

View File

@@ -17,9 +17,7 @@ package com.google.gerrit.server.index;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
import com.google.gerrit.extensions.events.LifecycleListener;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import java.util.Collection;
import java.util.Collections;
@@ -27,10 +25,10 @@ import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicReference;
/** Dynamic pointers to the index versions used for searching and writing. */
@Singleton
public class IndexCollection implements LifecycleListener {
private final CopyOnWriteArrayList<ChangeIndex> writeIndexes;
private final AtomicReference<ChangeIndex> searchIndex;
public abstract class IndexCollection<K, V, I extends Index<K, V>>
implements LifecycleListener {
private final CopyOnWriteArrayList<I> writeIndexes;
private final AtomicReference<I> searchIndex;
@Inject
@VisibleForTesting
@@ -40,22 +38,22 @@ public class IndexCollection implements LifecycleListener {
}
/** @return the current search index version. */
public ChangeIndex getSearchIndex() {
public I getSearchIndex() {
return searchIndex.get();
}
public void setSearchIndex(ChangeIndex index) {
ChangeIndex old = searchIndex.getAndSet(index);
public void setSearchIndex(I index) {
I old = searchIndex.getAndSet(index);
if (old != null && old != index && !writeIndexes.contains(old)) {
old.close();
}
}
public Collection<ChangeIndex> getWriteIndexes() {
public Collection<I> getWriteIndexes() {
return Collections.unmodifiableCollection(writeIndexes);
}
public synchronized ChangeIndex addWriteIndex(ChangeIndex index) {
public synchronized I addWriteIndex(I index) {
int version = index.getSchema().getVersion();
for (int i = 0; i < writeIndexes.size(); i++) {
if (writeIndexes.get(i).getSchema().getVersion() == version) {
@@ -83,8 +81,8 @@ public class IndexCollection implements LifecycleListener {
}
}
public ChangeIndex getWriteIndex(int version) {
for (ChangeIndex i : writeIndexes) {
public I getWriteIndex(int version) {
for (I i : writeIndexes) {
if (i.getSchema().getVersion() == version) {
return i;
}
@@ -98,11 +96,11 @@ public class IndexCollection implements LifecycleListener {
@Override
public void stop() {
ChangeIndex read = searchIndex.get();
I read = searchIndex.get();
if (read != null) {
read.close();
}
for (ChangeIndex write : writeIndexes) {
for (I write : writeIndexes) {
if (write != read) {
write.close();
}

View File

@@ -17,7 +17,6 @@ package com.google.gerrit.server.index;
import static com.google.common.base.Preconditions.checkArgument;
import com.google.auto.value.AutoValue;
import com.google.gerrit.server.index.change.ChangeIndex;
import org.eclipse.jgit.lib.Config;
@@ -25,7 +24,7 @@ import org.eclipse.jgit.lib.Config;
* Implementation-specific configuration for secondary indexes.
* <p>
* Contains configuration that is tied to a specific index implementation but is
* otherwise global, i.e. not tied to a specific {@link ChangeIndex} and schema
* otherwise global, i.e. not tied to a specific {@link Index} and schema
* version.
*/
@AutoValue

View File

@@ -22,6 +22,7 @@ import com.google.common.util.concurrent.MoreExecutors;
import com.google.gerrit.lifecycle.LifecycleModule;
import com.google.gerrit.server.config.GerritServerConfig;
import com.google.gerrit.server.git.WorkQueue;
import com.google.gerrit.server.index.change.ChangeIndexCollection;
import com.google.gerrit.server.index.change.ChangeIndexer;
import com.google.gerrit.server.index.change.IndexRewriter;
import com.google.inject.Injector;
@@ -69,8 +70,7 @@ public class IndexModule extends LifecycleModule {
@Override
protected void configure() {
bind(IndexRewriter.class);
bind(IndexCollection.class);
listener().to(IndexCollection.class);
listener().to(ChangeIndexCollection.class);
factory(ChangeIndexer.Factory.class);
}
@@ -79,7 +79,7 @@ public class IndexModule extends LifecycleModule {
ChangeIndexer getChangeIndexer(
@IndexExecutor(INTERACTIVE) ListeningExecutorService executor,
ChangeIndexer.Factory factory,
IndexCollection indexes) {
ChangeIndexCollection indexes) {
// Bind default indexer to interactive executor; callers who need a
// different executor can use the factory directly.
return factory.create(executor, indexes);

View File

@@ -12,17 +12,13 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package com.google.gerrit.server.query.change;
package com.google.gerrit.server.index;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.gerrit.server.index.change.ChangeField.CHANGE;
import static com.google.gerrit.server.index.change.ChangeField.PROJECT;
import com.google.auto.value.AutoValue;
import com.google.common.collect.ImmutableSet;
import com.google.gerrit.server.index.IndexConfig;
import java.util.HashSet;
import java.util.Set;
@AutoValue
@@ -31,23 +27,10 @@ public abstract class QueryOptions {
Set<String> fields) {
checkArgument(start >= 0, "start must be nonnegative: %s", start);
checkArgument(limit > 0, "limit must be positive: %s", limit);
// Always include project since it is needed to load the change from notedb.
if (!fields.contains(CHANGE.getName())
&& !fields.contains(PROJECT.getName())) {
fields = new HashSet<>(fields);
fields.add(PROJECT.getName());
}
return new AutoValue_QueryOptions(config, start, limit,
ImmutableSet.copyOf(fields));
}
public static QueryOptions oneResult() {
return create(IndexConfig.createDefault(), 0, 1,
ImmutableSet.<String> of());
}
public abstract IndexConfig config();
public abstract int start();
public abstract int limit();

View File

@@ -1,4 +1,4 @@
// Copyright (C) 2013 The Android Open Source Project
// 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.
@@ -15,91 +15,8 @@
package com.google.gerrit.server.index.change;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.Schema;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
import com.google.gerrit.server.index.Index;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.query.change.ChangeDataSource;
import com.google.gerrit.server.query.change.QueryOptions;
import java.io.IOException;
/**
* Secondary index implementation for change documents.
* <p>
* {@link ChangeData} objects are inserted into the index and are queried by
* converting special {@link com.google.gerrit.server.query.Predicate} instances
* into index-aware predicates that use the index search results as a source.
* <p>
* Implementations must be thread-safe and should batch inserts/updates where
* appropriate.
*/
public interface ChangeIndex {
/** @return the schema version used by this index. */
Schema<ChangeData> getSchema();
/** Close this index. */
void close();
/**
* Update a change document in the index.
* <p>
* Semantically equivalent to deleting the document and reinserting it with
* new field values. A document that does not already exist is created. Results
* may not be immediately visible to searchers, but should be visible within a
* reasonable amount of time.
*
* @param cd change document
*
* @throws IOException
*/
void replace(ChangeData cd) throws IOException;
/**
* Delete a change document from the index by id.
*
* @param id change id
*
* @throws IOException
*/
void delete(Change.Id id) throws IOException;
/**
* Delete all change documents from the index.
*
* @throws IOException
*/
void deleteAll() throws IOException;
/**
* Convert the given operator predicate into a source searching the index and
* returning only the documents matching that predicate.
* <p>
* This method may be called multiple times for variations on the same
* predicate or multiple predicate subtrees in the course of processing a
* single query, so it should not have any side effects (e.g. starting a
* search in the background).
*
* @param p the predicate to match. Must be a tree containing only AND, OR,
* or NOT predicates as internal nodes, and {@link IndexPredicate}s as
* leaves.
* @param opts query options not implied by the predicate, such as start and
* limit.
* @return a source of documents matching the predicate. Documents must be
* returned in descending updated timestamp order.
*
* @throws QueryParseException if the predicate could not be converted to an
* indexed data source.
*/
ChangeDataSource getSource(Predicate<ChangeData> p, QueryOptions opts)
throws QueryParseException;
/**
* Mark whether this index is up-to-date and ready to serve reads.
*
* @param ready whether the index is ready
* @throws IOException
*/
void markReady(boolean ready) throws IOException;
public interface ChangeIndex extends Index<Change.Id, ChangeData> {
}

View File

@@ -0,0 +1,25 @@
// 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.server.index.change;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.inject.Singleton;
@Singleton
public class ChangeIndexCollection extends
IndexCollection<Change.Id, ChangeData, ChangeIndex> {
}

View File

@@ -26,7 +26,7 @@ import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.CurrentUser;
import com.google.gerrit.server.extensions.events.ChangeIndexedListener;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.Index;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.util.RequestContext;
import com.google.gerrit.server.util.ThreadLocalRequestContext;
@@ -64,7 +64,7 @@ public class ChangeIndexer {
public interface Factory {
ChangeIndexer create(ListeningExecutorService executor, ChangeIndex index);
ChangeIndexer create(ListeningExecutorService executor,
IndexCollection indexes);
ChangeIndexCollection indexes);
}
public static CheckedFuture<?, IOException> allAsList(
@@ -91,7 +91,7 @@ public class ChangeIndexer {
}
};
private final IndexCollection indexes;
private final ChangeIndexCollection indexes;
private final ChangeIndex index;
private final SchemaFactory<ReviewDb> schemaFactory;
private final ChangeData.Factory changeDataFactory;
@@ -121,7 +121,7 @@ public class ChangeIndexer {
ThreadLocalRequestContext context,
DynamicSet<ChangeIndexedListener> indexedListener,
@Assisted ListeningExecutorService executor,
@Assisted IndexCollection indexes) {
@Assisted ChangeIndexCollection indexes) {
this.executor = executor;
this.schemaFactory = schemaFactory;
this.changeDataFactory = changeDataFactory;
@@ -165,7 +165,7 @@ public class ChangeIndexer {
* @param cd change to index.
*/
public void index(ChangeData cd) throws IOException {
for (ChangeIndex i : getWriteIndexes()) {
for (Index<?, ChangeData> i : getWriteIndexes()) {
i.replace(cd);
}
fireChangeIndexedEvent(cd);

View File

@@ -15,11 +15,11 @@
package com.google.gerrit.server.index.change;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.server.index.QueryOptions;
import com.google.gerrit.server.index.Schema;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.query.change.ChangeDataSource;
import com.google.gerrit.server.query.change.QueryOptions;
import java.io.IOException;

View File

@@ -18,9 +18,9 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.Change.Status;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.IndexConfig;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.QueryOptions;
import com.google.gerrit.server.query.AndPredicate;
import com.google.gerrit.server.query.NotPredicate;
import com.google.gerrit.server.query.OrPredicate;
@@ -31,7 +31,6 @@ import com.google.gerrit.server.query.change.ChangeData;
import com.google.gerrit.server.query.change.ChangeStatusPredicate;
import com.google.gerrit.server.query.change.LimitPredicate;
import com.google.gerrit.server.query.change.OrSource;
import com.google.gerrit.server.query.change.QueryOptions;
import com.google.inject.Inject;
import com.google.inject.Singleton;
@@ -121,11 +120,11 @@ public class IndexRewriter {
return null;
}
private final IndexCollection indexes;
private final ChangeIndexCollection indexes;
private final IndexConfig config;
@Inject
IndexRewriter(IndexCollection indexes,
IndexRewriter(ChangeIndexCollection indexes,
IndexConfig config) {
this.indexes = indexes;
this.config = config;
@@ -208,7 +207,8 @@ public class IndexRewriter {
return partitionChildren(in, newChildren, isIndexed, index, opts);
}
private boolean isIndexPredicate(Predicate<ChangeData> in, ChangeIndex index) {
private boolean isIndexPredicate(Predicate<ChangeData> in,
ChangeIndex index) {
if (!(in instanceof IndexPredicate)) {
return false;
}

View File

@@ -14,25 +14,33 @@
package com.google.gerrit.server.index.change;
import static com.google.gerrit.server.index.change.ChangeField.CHANGE;
import static com.google.gerrit.server.index.change.ChangeField.PROJECT;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.MoreObjects;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.primitives.Ints;
import com.google.gerrit.server.index.IndexConfig;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.QueryOptions;
import com.google.gerrit.server.query.DataSource;
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.gerrit.server.query.change.Paginated;
import com.google.gerrit.server.query.change.QueryOptions;
import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.ResultSet;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
/**
* Wrapper combining an {@link IndexPredicate} together with a
@@ -44,6 +52,22 @@ import java.util.List;
*/
public class IndexedChangeQuery extends Predicate<ChangeData>
implements ChangeDataSource, Paginated {
public static QueryOptions oneResult() {
return createOptions(IndexConfig.createDefault(), 0, 1,
ImmutableSet.<String> of());
}
public static QueryOptions createOptions(IndexConfig config, int start,
int limit, Set<String> fields) {
// Always include project since it is needed to load the change from notedb.
if (!fields.contains(CHANGE.getName())
&& !fields.contains(PROJECT.getName())) {
fields = new HashSet<>(fields);
fields.add(PROJECT.getName());
}
return QueryOptions.create(config, start, limit, fields);
}
@VisibleForTesting
static QueryOptions convertOptions(QueryOptions opts) {
// Increase the limit rather than skipping, since we don't know how many
@@ -51,14 +75,14 @@ public class IndexedChangeQuery extends Predicate<ChangeData>
int backendLimit = opts.config().maxLimit();
int limit = Ints.saturatedCast((long) opts.limit() + opts.start());
limit = Math.min(limit, backendLimit);
return QueryOptions.create(opts.config(), 0, limit, opts.fields());
return IndexedChangeQuery.createOptions(opts.config(), 0, limit, opts.fields());
}
private final ChangeIndex index;
private QueryOptions opts;
private Predicate<ChangeData> pred;
private ChangeDataSource source;
private DataSource<ChangeData> source;
public IndexedChangeQuery(ChangeIndex index, Predicate<ChangeData> pred,
QueryOptions opts) throws QueryParseException {
@@ -103,7 +127,7 @@ public class IndexedChangeQuery extends Predicate<ChangeData>
@Override
public ResultSet<ChangeData> read() throws OrmException {
final ChangeDataSource currSource = source;
final DataSource<ChangeData> currSource = source;
final ResultSet<ChangeData> rs = currSource.read();
return new ResultSet<ChangeData>() {

View File

@@ -26,7 +26,6 @@ import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.reviewdb.client.RefNames;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.git.QueueProvider.QueueType;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.IndexExecutor;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.project.NoSuchChangeException;
@@ -52,7 +51,7 @@ public class ReindexAfterUpdate implements GitReferenceUpdatedListener {
private final OneOffRequestContext requestContext;
private final Provider<InternalChangeQuery> queryProvider;
private final ChangeIndexer.Factory indexerFactory;
private final IndexCollection indexes;
private final ChangeIndexCollection indexes;
private final ChangeNotes.Factory notesFactory;
private final ListeningExecutorService executor;
@@ -61,7 +60,7 @@ public class ReindexAfterUpdate implements GitReferenceUpdatedListener {
OneOffRequestContext requestContext,
Provider<InternalChangeQuery> queryProvider,
ChangeIndexer.Factory indexerFactory,
IndexCollection indexes,
ChangeIndexCollection indexes,
ChangeNotes.Factory notesFactory,
@IndexExecutor(QueueType.BATCH) ListeningExecutorService executor) {
this.requestContext = requestContext;

View File

@@ -58,6 +58,7 @@ import com.google.gerrit.server.project.ChangeControl;
import com.google.gerrit.server.project.NoSuchChangeException;
import com.google.gerrit.server.project.ProjectCache;
import com.google.gerrit.server.project.SubmitRuleEvaluator;
import com.google.gerrit.server.query.DataSource;
import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.ResultSet;
import com.google.inject.assistedinject.Assisted;
@@ -315,7 +316,7 @@ public class ChangeData {
private final NotesMigration notesMigration;
private final MergeabilityCache mergeabilityCache;
private final Change.Id legacyId;
private ChangeDataSource returnedBySource;
private DataSource<ChangeData> returnedBySource;
private Project.NameKey project;
private Change change;
private ChangeNotes notes;
@@ -528,11 +529,11 @@ public class ChangeData {
return db;
}
public boolean isFromSource(ChangeDataSource s) {
public boolean isFromSource(DataSource<ChangeData> s) {
return s == returnedBySource;
}
public void cacheFromSource(ChangeDataSource s) {
public void cacheFromSource(DataSource<ChangeData> s) {
returnedBySource = s;
}

View File

@@ -50,11 +50,11 @@ import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.git.strategy.SubmitDryRun;
import com.google.gerrit.server.group.ListMembers;
import com.google.gerrit.server.index.FieldDef;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.IndexConfig;
import com.google.gerrit.server.index.Schema;
import com.google.gerrit.server.index.change.ChangeField;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.ChangeIndexCollection;
import com.google.gerrit.server.index.change.IndexRewriter;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.patch.PatchListCache;
@@ -205,7 +205,7 @@ public class ChangeQueryBuilder extends QueryBuilder<ChangeData> {
GitRepositoryManager repoManager,
ProjectCache projectCache,
Provider<ListChildProjects> listChildProjects,
IndexCollection indexes,
ChangeIndexCollection indexes,
SubmitDryRun submitDryRun,
ConflictsCache conflictsCache,
TrackingFooters trackingFooters,

View File

@@ -17,6 +17,7 @@ package com.google.gerrit.server.query.change;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.change.ChangeField;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.IndexedChangeQuery;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
import com.google.gwtorm.server.OrmException;
@@ -35,7 +36,7 @@ class CommentPredicate extends IndexPredicate<ChangeData> {
Predicate<ChangeData> p = Predicate.and(
new LegacyChangeIdPredicate(object.getId()), this);
for (ChangeData cData
: index.getSource(p, QueryOptions.oneResult()).read()) {
: index.getSource(p, IndexedChangeQuery.oneResult()).read()) {
if (cData.getId().equals(object.getId())) {
return true;
}

View File

@@ -20,6 +20,7 @@ import com.google.common.collect.Iterables;
import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.IndexedChangeQuery;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
import com.google.gwtorm.server.OrmException;
@@ -45,7 +46,7 @@ class FuzzyTopicPredicate extends IndexPredicate<ChangeData> {
try {
Predicate<ChangeData> thisId = new LegacyChangeIdPredicate(cd.getId());
Iterable<ChangeData> results =
index.getSource(and(thisId, this), QueryOptions.oneResult()).read();
index.getSource(and(thisId, this), IndexedChangeQuery.oneResult()).read();
return !Iterables.isEmpty(results);
} catch (QueryParseException e) {
throw new OrmException(e);

View File

@@ -33,10 +33,10 @@ import com.google.gerrit.reviewdb.client.Change;
import com.google.gerrit.reviewdb.client.Project;
import com.google.gerrit.reviewdb.client.RefNames;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.IndexConfig;
import com.google.gerrit.server.index.Schema;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.ChangeIndexCollection;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
@@ -85,14 +85,14 @@ public class InternalChangeQuery {
private final IndexConfig indexConfig;
private final QueryProcessor qp;
private final IndexCollection indexes;
private final ChangeIndexCollection indexes;
private final ChangeData.Factory changeDataFactory;
private final ChangeNotes.Factory notesFactory;
@Inject
InternalChangeQuery(IndexConfig indexConfig,
QueryProcessor queryProcessor,
IndexCollection indexes,
ChangeIndexCollection indexes,
ChangeData.Factory changeDataFactory,
ChangeNotes.Factory notesFactory) {
this.indexConfig = indexConfig;
@@ -305,7 +305,8 @@ public class InternalChangeQuery {
}
}
private static Schema<ChangeData> schema(@Nullable IndexCollection indexes) {
private static Schema<ChangeData> schema(
@Nullable ChangeIndexCollection indexes) {
ChangeIndex index = indexes != null ? indexes.getSearchIndex() : null;
return index != null ? index.getSchema() : null;
}

View File

@@ -17,6 +17,7 @@ package com.google.gerrit.server.query.change;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.change.ChangeField;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.IndexedChangeQuery;
import com.google.gerrit.server.query.Predicate;
import com.google.gerrit.server.query.QueryParseException;
import com.google.gwtorm.server.OrmException;
@@ -39,7 +40,7 @@ class MessagePredicate extends IndexPredicate<ChangeData> {
Predicate<ChangeData> p = Predicate.and(
new LegacyChangeIdPredicate(object.getId()), this);
for (ChangeData cData
: index.getSource(p, QueryOptions.oneResult()).read()) {
: index.getSource(p, IndexedChangeQuery.oneResult()).read()) {
if (cData.getId().equals(object.getId())) {
return true;
}

View File

@@ -14,6 +14,7 @@
package com.google.gerrit.server.query.change;
import com.google.gerrit.server.index.QueryOptions;
import com.google.gwtorm.server.OrmException;
import com.google.gwtorm.server.ResultSet;

View File

@@ -26,11 +26,13 @@ import com.google.gerrit.metrics.MetricMaker;
import com.google.gerrit.metrics.Timer0;
import com.google.gerrit.reviewdb.server.ReviewDb;
import com.google.gerrit.server.CurrentUser;
import com.google.gerrit.server.index.IndexCollection;
import com.google.gerrit.server.index.IndexConfig;
import com.google.gerrit.server.index.IndexPredicate;
import com.google.gerrit.server.index.QueryOptions;
import com.google.gerrit.server.index.change.ChangeIndex;
import com.google.gerrit.server.index.change.ChangeIndexCollection;
import com.google.gerrit.server.index.change.IndexRewriter;
import com.google.gerrit.server.index.change.IndexedChangeQuery;
import com.google.gerrit.server.notedb.ChangeNotes;
import com.google.gerrit.server.project.ChangeControl;
import com.google.gerrit.server.query.Predicate;
@@ -50,7 +52,7 @@ public class QueryProcessor {
private final Provider<CurrentUser> userProvider;
private final ChangeControl.GenericFactory changeControlFactory;
private final ChangeNotes.Factory notesFactory;
private final IndexCollection indexes;
private final ChangeIndexCollection indexes;
private final IndexRewriter rewriter;
private final IndexConfig indexConfig;
private final Metrics metrics;
@@ -65,7 +67,7 @@ public class QueryProcessor {
Provider<CurrentUser> userProvider,
ChangeControl.GenericFactory changeControlFactory,
ChangeNotes.Factory notesFactory,
IndexCollection indexes,
ChangeIndexCollection indexes,
IndexRewriter rewriter,
IndexConfig indexConfig,
Metrics metrics) {
@@ -168,7 +170,7 @@ public class QueryProcessor {
"Cannot go beyond page " + indexConfig.maxPages() + "of results");
}
QueryOptions opts = QueryOptions.create(
QueryOptions opts = IndexedChangeQuery.createOptions(
indexConfig, start, limit + 1, getRequestedFields());
Predicate<ChangeData> s = rewriter.rewrite(q, opts);
if (!(s instanceof ChangeDataSource)) {