Merge changes from topic 'lucene-stability'
* changes: Write each Lucene index using a dedicated background thread Do not allow index futures to be cancelled Run all Lucene queries on background threads Optimize loading change from Lucene index
This commit is contained in:
commit
1b5c1b1bbe
@ -19,7 +19,11 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
|
|||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.util.concurrent.AbstractFuture;
|
import com.google.common.util.concurrent.AbstractFuture;
|
||||||
|
import com.google.common.util.concurrent.AsyncFunction;
|
||||||
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import com.google.gerrit.server.config.SitePaths;
|
import com.google.gerrit.server.config.SitePaths;
|
||||||
import com.google.gerrit.server.index.FieldDef;
|
import com.google.gerrit.server.index.FieldDef;
|
||||||
@ -54,8 +58,10 @@ import org.slf4j.LoggerFactory;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.sql.Timestamp;
|
import java.sql.Timestamp;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
@ -84,6 +90,7 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
private final SitePaths sitePaths;
|
private final SitePaths sitePaths;
|
||||||
private final Directory dir;
|
private final Directory dir;
|
||||||
private final String name;
|
private final String name;
|
||||||
|
private final ListeningExecutorService writerThread;
|
||||||
private final TrackingIndexWriter writer;
|
private final TrackingIndexWriter writer;
|
||||||
private final ReferenceManager<IndexSearcher> searcherManager;
|
private final ReferenceManager<IndexSearcher> searcherManager;
|
||||||
private final ControlledRealTimeReopenThread<IndexSearcher> reopenThread;
|
private final ControlledRealTimeReopenThread<IndexSearcher> reopenThread;
|
||||||
@ -117,7 +124,7 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
delegateWriter = autoCommitWriter;
|
delegateWriter = autoCommitWriter;
|
||||||
|
|
||||||
autoCommitExecutor = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder()
|
autoCommitExecutor = new ScheduledThreadPoolExecutor(1, new ThreadFactoryBuilder()
|
||||||
.setNameFormat("Commit-%d " + index)
|
.setNameFormat(index + " Commit-%d")
|
||||||
.setDaemon(true)
|
.setDaemon(true)
|
||||||
.build());
|
.build());
|
||||||
autoCommitExecutor.scheduleAtFixedRate(new Runnable() {
|
autoCommitExecutor.scheduleAtFixedRate(new Runnable() {
|
||||||
@ -148,11 +155,18 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
|
|
||||||
notDoneNrtFutures = Sets.newConcurrentHashSet();
|
notDoneNrtFutures = Sets.newConcurrentHashSet();
|
||||||
|
|
||||||
|
writerThread = MoreExecutors.listeningDecorator(
|
||||||
|
Executors.newFixedThreadPool(1,
|
||||||
|
new ThreadFactoryBuilder()
|
||||||
|
.setNameFormat(index + " Write-%d")
|
||||||
|
.setDaemon(true)
|
||||||
|
.build()));
|
||||||
|
|
||||||
reopenThread = new ControlledRealTimeReopenThread<>(
|
reopenThread = new ControlledRealTimeReopenThread<>(
|
||||||
writer, searcherManager,
|
writer, searcherManager,
|
||||||
0.500 /* maximum stale age (seconds) */,
|
0.500 /* maximum stale age (seconds) */,
|
||||||
0.010 /* minimum stale age (seconds) */);
|
0.010 /* minimum stale age (seconds) */);
|
||||||
reopenThread.setName("NRT " + name);
|
reopenThread.setName(index + " NRT");
|
||||||
reopenThread.setPriority(Math.min(
|
reopenThread.setPriority(Math.min(
|
||||||
Thread.currentThread().getPriority() + 2,
|
Thread.currentThread().getPriority() + 2,
|
||||||
Thread.MAX_PRIORITY));
|
Thread.MAX_PRIORITY));
|
||||||
@ -193,6 +207,15 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
autoCommitExecutor.shutdown();
|
autoCommitExecutor.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
writerThread.shutdown();
|
||||||
|
try {
|
||||||
|
if (!writerThread.awaitTermination(5, TimeUnit.SECONDS)) {
|
||||||
|
log.warn("shutting down " + name + " index with pending Lucene writes");
|
||||||
|
}
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
log.warn("interrupted waiting for pending Lucene writes of " + name +
|
||||||
|
" index", e);
|
||||||
|
}
|
||||||
reopenThread.close();
|
reopenThread.close();
|
||||||
|
|
||||||
// Closing the reopen thread sets its generation to Long.MAX_VALUE, but we
|
// Closing the reopen thread sets its generation to Long.MAX_VALUE, but we
|
||||||
@ -222,16 +245,45 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ListenableFuture<?> insert(Document doc) throws IOException {
|
ListenableFuture<?> insert(final Document doc) {
|
||||||
return new NrtFuture(writer.addDocument(doc));
|
return submit(new Callable<Long>() {
|
||||||
|
@Override
|
||||||
|
public Long call() throws IOException, InterruptedException {
|
||||||
|
return writer.addDocument(doc);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
ListenableFuture<?> replace(Term term, Document doc) throws IOException {
|
ListenableFuture<?> replace(final Term term, final Document doc) {
|
||||||
return new NrtFuture(writer.updateDocument(term, doc));
|
return submit(new Callable<Long>() {
|
||||||
|
@Override
|
||||||
|
public Long call() throws IOException, InterruptedException {
|
||||||
|
return writer.updateDocument(term, doc);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
ListenableFuture<?> delete(Term term) throws IOException {
|
ListenableFuture<?> delete(final Term term) {
|
||||||
return new NrtFuture(writer.deleteDocuments(term));
|
return submit(new Callable<Long>() {
|
||||||
|
@Override
|
||||||
|
public Long call() throws IOException, InterruptedException {
|
||||||
|
return writer.deleteDocuments(term);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private ListenableFuture<?> submit(Callable<Long> task) {
|
||||||
|
ListenableFuture<Long> future =
|
||||||
|
Futures.nonCancellationPropagating(writerThread.submit(task));
|
||||||
|
return Futures.transformAsync(future, new AsyncFunction<Long, Void>() {
|
||||||
|
@Override
|
||||||
|
public ListenableFuture<Void> apply(Long gen) throws InterruptedException {
|
||||||
|
// Tell the reopen thread a future is waiting on this
|
||||||
|
// generation so it uses the min stale time when refreshing.
|
||||||
|
reopenThread.waitForGeneration(gen, 0);
|
||||||
|
return new NrtFuture(gen);
|
||||||
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -305,9 +357,6 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
|
|
||||||
NrtFuture(long gen) {
|
NrtFuture(long gen) {
|
||||||
this.gen = gen;
|
this.gen = gen;
|
||||||
// Tell the reopen thread we are waiting on this generation so it uses the
|
|
||||||
// min stale time when refreshing.
|
|
||||||
isGenAvailableNowForCurrentSearcher();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -323,12 +372,10 @@ public abstract class AbstractLuceneIndex<K, V> implements Index<K, V> {
|
|||||||
public Void get(long timeout, TimeUnit unit) throws InterruptedException,
|
public Void get(long timeout, TimeUnit unit) throws InterruptedException,
|
||||||
TimeoutException, ExecutionException {
|
TimeoutException, ExecutionException {
|
||||||
if (!isDone()) {
|
if (!isDone()) {
|
||||||
if (reopenThread.waitForGeneration(gen,
|
if (!reopenThread.waitForGeneration(gen, (int) unit.toMillis(timeout))) {
|
||||||
(int) MILLISECONDS.convert(timeout, unit))) {
|
|
||||||
set(null);
|
|
||||||
} else {
|
|
||||||
throw new TimeoutException();
|
throw new TimeoutException();
|
||||||
}
|
}
|
||||||
|
set(null);
|
||||||
}
|
}
|
||||||
return super.get(timeout, unit);
|
return super.get(timeout, unit);
|
||||||
}
|
}
|
||||||
|
@ -25,10 +25,11 @@ import static com.google.gerrit.server.index.change.ChangeIndexRewriter.CLOSED_S
|
|||||||
import static com.google.gerrit.server.index.change.ChangeIndexRewriter.OPEN_STATUSES;
|
import static com.google.gerrit.server.index.change.ChangeIndexRewriter.OPEN_STATUSES;
|
||||||
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ArrayListMultimap;
|
import com.google.common.collect.ArrayListMultimap;
|
||||||
import com.google.common.collect.FluentIterable;
|
import com.google.common.collect.FluentIterable;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Multimap;
|
import com.google.common.collect.Multimap;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
@ -58,6 +59,7 @@ import com.google.gerrit.server.query.change.ChangeData;
|
|||||||
import com.google.gerrit.server.query.change.ChangeDataSource;
|
import com.google.gerrit.server.query.change.ChangeDataSource;
|
||||||
import com.google.gwtorm.protobuf.ProtobufCodec;
|
import com.google.gwtorm.protobuf.ProtobufCodec;
|
||||||
import com.google.gwtorm.server.OrmException;
|
import com.google.gwtorm.server.OrmException;
|
||||||
|
import com.google.gwtorm.server.OrmRuntimeException;
|
||||||
import com.google.gwtorm.server.ResultSet;
|
import com.google.gwtorm.server.ResultSet;
|
||||||
import com.google.inject.Provider;
|
import com.google.inject.Provider;
|
||||||
import com.google.inject.assistedinject.Assisted;
|
import com.google.inject.assistedinject.Assisted;
|
||||||
@ -85,11 +87,14 @@ import org.slf4j.LoggerFactory;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -240,7 +245,7 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
public ChangeDataSource getSource(Predicate<ChangeData> p, QueryOptions opts)
|
public ChangeDataSource getSource(Predicate<ChangeData> p, QueryOptions opts)
|
||||||
throws QueryParseException {
|
throws QueryParseException {
|
||||||
Set<Change.Status> statuses = ChangeIndexRewriter.getPossibleStatus(p);
|
Set<Change.Status> statuses = ChangeIndexRewriter.getPossibleStatus(p);
|
||||||
List<ChangeSubIndex> indexes = Lists.newArrayListWithCapacity(2);
|
List<ChangeSubIndex> indexes = new ArrayList<>(2);
|
||||||
if (!Sets.intersection(statuses, OPEN_STATUSES).isEmpty()) {
|
if (!Sets.intersection(statuses, OPEN_STATUSES).isEmpty()) {
|
||||||
indexes.add(openIndex);
|
indexes.add(openIndex);
|
||||||
}
|
}
|
||||||
@ -298,6 +303,22 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ResultSet<ChangeData> read() throws OrmException {
|
public ResultSet<ChangeData> read() throws OrmException {
|
||||||
|
if (Thread.interrupted()) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw new OrmException("interrupted");
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<String> fields = fields(opts);
|
||||||
|
return new ChangeDataResults(
|
||||||
|
executor.submit(new Callable<List<Document>>() {
|
||||||
|
@Override
|
||||||
|
public List<Document> call() throws IOException {
|
||||||
|
return doRead(fields);
|
||||||
|
}
|
||||||
|
}), fields);
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<Document> doRead(Set<String> fields) throws IOException {
|
||||||
IndexSearcher[] searchers = new IndexSearcher[indexes.size()];
|
IndexSearcher[] searchers = new IndexSearcher[indexes.size()];
|
||||||
try {
|
try {
|
||||||
int realLimit = opts.start() + opts.limit();
|
int realLimit = opts.start() + opts.limit();
|
||||||
@ -308,35 +329,12 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
TopDocs docs = TopDocs.merge(sort, realLimit, hits);
|
TopDocs docs = TopDocs.merge(sort, realLimit, hits);
|
||||||
|
|
||||||
List<ChangeData> result =
|
List<Document> result = new ArrayList<>(docs.scoreDocs.length);
|
||||||
Lists.newArrayListWithCapacity(docs.scoreDocs.length);
|
|
||||||
Set<String> fields = fields(opts);
|
|
||||||
String idFieldName = LEGACY_ID.getName();
|
|
||||||
for (int i = opts.start(); i < docs.scoreDocs.length; i++) {
|
for (int i = opts.start(); i < docs.scoreDocs.length; i++) {
|
||||||
ScoreDoc sd = docs.scoreDocs[i];
|
ScoreDoc sd = docs.scoreDocs[i];
|
||||||
Document doc = searchers[sd.shardIndex].doc(sd.doc, fields);
|
result.add(searchers[sd.shardIndex].doc(sd.doc, fields));
|
||||||
result.add(toChangeData(doc, fields, idFieldName));
|
|
||||||
}
|
}
|
||||||
|
return result;
|
||||||
final List<ChangeData> r = Collections.unmodifiableList(result);
|
|
||||||
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);
|
|
||||||
} finally {
|
} finally {
|
||||||
for (int i = 0; i < indexes.size(); i++) {
|
for (int i = 0; i < indexes.size(); i++) {
|
||||||
if (searchers[i] != null) {
|
if (searchers[i] != null) {
|
||||||
@ -351,6 +349,45 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class ChangeDataResults implements ResultSet<ChangeData> {
|
||||||
|
private final Future<List<Document>> future;
|
||||||
|
private final Set<String> fields;
|
||||||
|
|
||||||
|
ChangeDataResults(Future<List<Document>> future, Set<String> fields) {
|
||||||
|
this.future = future;
|
||||||
|
this.fields = fields;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<ChangeData> iterator() {
|
||||||
|
return toList().iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<ChangeData> toList() {
|
||||||
|
try {
|
||||||
|
List<Document> docs = future.get();
|
||||||
|
List<ChangeData> result = new ArrayList<>(docs.size());
|
||||||
|
String idFieldName = LEGACY_ID.getName();
|
||||||
|
for (Document doc : docs) {
|
||||||
|
result.add(toChangeData(fields(doc, fields), fields, idFieldName));
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
close();
|
||||||
|
throw new OrmRuntimeException(e);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
Throwables.propagateIfPossible(e.getCause());
|
||||||
|
throw new OrmRuntimeException(e.getCause());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
future.cancel(false /* do not interrupt Lucene */);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Set<String> fields(QueryOptions opts) {
|
private Set<String> fields(QueryOptions opts) {
|
||||||
// Ensure we request enough fields to construct a ChangeData.
|
// Ensure we request enough fields to construct a ChangeData.
|
||||||
Set<String> fs = opts.fields();
|
Set<String> fs = opts.fields();
|
||||||
@ -376,19 +413,33 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
ImmutableSet.of(LEGACY_ID.getName(), PROJECT.getName()));
|
ImmutableSet.of(LEGACY_ID.getName(), PROJECT.getName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
private ChangeData toChangeData(Document doc, Set<String> fields,
|
private static Multimap<String, IndexableField> fields(Document doc,
|
||||||
String idFieldName) {
|
Set<String> fields) {
|
||||||
|
Multimap<String, IndexableField> stored =
|
||||||
|
ArrayListMultimap.create(fields.size(), 4);
|
||||||
|
for (IndexableField f : doc) {
|
||||||
|
String name = f.name();
|
||||||
|
if (fields.contains(name)) {
|
||||||
|
stored.put(name, f);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return stored;
|
||||||
|
}
|
||||||
|
|
||||||
|
private ChangeData toChangeData(Multimap<String, IndexableField> doc,
|
||||||
|
Set<String> fields, String idFieldName) {
|
||||||
ChangeData cd;
|
ChangeData cd;
|
||||||
// Either change or the ID field was guaranteed to be included in the call
|
// Either change or the ID field was guaranteed to be included in the call
|
||||||
// to fields() above.
|
// to fields() above.
|
||||||
BytesRef cb = doc.getBinaryValue(CHANGE_FIELD);
|
IndexableField cb = Iterables.getFirst(doc.get(CHANGE_FIELD), null);
|
||||||
if (cb != null) {
|
if (cb != null) {
|
||||||
|
BytesRef proto = cb.binaryValue();
|
||||||
cd = changeDataFactory.create(db.get(),
|
cd = changeDataFactory.create(db.get(),
|
||||||
ChangeProtoField.CODEC.decode(cb.bytes, cb.offset, cb.length));
|
ChangeProtoField.CODEC.decode(proto.bytes, proto.offset, proto.length));
|
||||||
} else {
|
} else {
|
||||||
Change.Id id =
|
IndexableField f = Iterables.getFirst(doc.get(idFieldName), null);
|
||||||
new Change.Id(doc.getField(idFieldName).numericValue().intValue());
|
Change.Id id = new Change.Id(f.numericValue().intValue());
|
||||||
IndexableField project = doc.getField(PROJECT.getName());
|
IndexableField project = Iterables.getFirst(doc.get(PROJECT.getName()), null);
|
||||||
if (project == null) {
|
if (project == null) {
|
||||||
// Old schema without project field: we can safely assume NoteDb is
|
// Old schema without project field: we can safely assume NoteDb is
|
||||||
// disabled.
|
// disabled.
|
||||||
@ -429,7 +480,7 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
return cd;
|
return cd;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodePatchSets(Document doc, ChangeData cd) {
|
private void decodePatchSets(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
List<PatchSet> patchSets =
|
List<PatchSet> patchSets =
|
||||||
decodeProtos(doc, PATCH_SET_FIELD, PatchSetProtoField.CODEC);
|
decodeProtos(doc, PATCH_SET_FIELD, PatchSetProtoField.CODEC);
|
||||||
if (!patchSets.isEmpty()) {
|
if (!patchSets.isEmpty()) {
|
||||||
@ -439,14 +490,14 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeApprovals(Document doc, ChangeData cd) {
|
private void decodeApprovals(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
cd.setCurrentApprovals(
|
cd.setCurrentApprovals(
|
||||||
decodeProtos(doc, APPROVAL_FIELD, PatchSetApprovalProtoField.CODEC));
|
decodeProtos(doc, APPROVAL_FIELD, PatchSetApprovalProtoField.CODEC));
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeChangedLines(Document doc, ChangeData cd) {
|
private void decodeChangedLines(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
IndexableField added = doc.getField(ADDED_FIELD);
|
IndexableField added = Iterables.getFirst(doc.get(ADDED_FIELD), null);
|
||||||
IndexableField deleted = doc.getField(DELETED_FIELD);
|
IndexableField deleted = Iterables.getFirst(doc.get(DELETED_FIELD), null);
|
||||||
if (added != null && deleted != null) {
|
if (added != null && deleted != null) {
|
||||||
cd.setChangedLines(
|
cd.setChangedLines(
|
||||||
added.numericValue().intValue(),
|
added.numericValue().intValue(),
|
||||||
@ -460,23 +511,26 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeMergeable(Document doc, ChangeData cd) {
|
private void decodeMergeable(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
String mergeable = doc.get(MERGEABLE_FIELD);
|
IndexableField f = Iterables.getFirst(doc.get(MERGEABLE_FIELD), null);
|
||||||
if ("1".equals(mergeable)) {
|
if (f != null) {
|
||||||
cd.setMergeable(true);
|
String mergeable = f.stringValue();
|
||||||
} else if ("0".equals(mergeable)) {
|
if ("1".equals(mergeable)) {
|
||||||
cd.setMergeable(false);
|
cd.setMergeable(true);
|
||||||
|
} else if ("0".equals(mergeable)) {
|
||||||
|
cd.setMergeable(false);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeReviewedBy(Document doc, ChangeData cd) {
|
private void decodeReviewedBy(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
IndexableField[] reviewedBy = doc.getFields(REVIEWEDBY_FIELD);
|
Collection<IndexableField> reviewedBy = doc.get(REVIEWEDBY_FIELD);
|
||||||
if (reviewedBy.length > 0) {
|
if (reviewedBy.size() > 0) {
|
||||||
Set<Account.Id> accounts =
|
Set<Account.Id> accounts =
|
||||||
Sets.newHashSetWithExpectedSize(reviewedBy.length);
|
Sets.newHashSetWithExpectedSize(reviewedBy.size());
|
||||||
for (IndexableField r : reviewedBy) {
|
for (IndexableField r : reviewedBy) {
|
||||||
int id = r.numericValue().intValue();
|
int id = r.numericValue().intValue();
|
||||||
if (reviewedBy.length == 1 && id == ChangeField.NOT_REVIEWED) {
|
if (reviewedBy.size() == 1 && id == ChangeField.NOT_REVIEWED) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
accounts.add(new Account.Id(id));
|
accounts.add(new Account.Id(id));
|
||||||
@ -485,9 +539,9 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeHashtags(Document doc, ChangeData cd) {
|
private void decodeHashtags(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
IndexableField[] hashtag = doc.getFields(HASHTAG_FIELD);
|
Collection<IndexableField> hashtag = doc.get(HASHTAG_FIELD);
|
||||||
Set<String> hashtags = Sets.newHashSetWithExpectedSize(hashtag.length);
|
Set<String> hashtags = Sets.newHashSetWithExpectedSize(hashtag.size());
|
||||||
for (IndexableField r : hashtag) {
|
for (IndexableField r : hashtag) {
|
||||||
hashtags.add(r.binaryValue().utf8ToString());
|
hashtags.add(r.binaryValue().utf8ToString());
|
||||||
}
|
}
|
||||||
@ -495,18 +549,18 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
private void decodeStarredBy(Document doc, ChangeData cd) {
|
private void decodeStarredBy(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
IndexableField[] starredBy = doc.getFields(STARREDBY_FIELD);
|
Collection<IndexableField> starredBy = doc.get(STARREDBY_FIELD);
|
||||||
Set<Account.Id> accounts =
|
Set<Account.Id> accounts =
|
||||||
Sets.newHashSetWithExpectedSize(starredBy.length);
|
Sets.newHashSetWithExpectedSize(starredBy.size());
|
||||||
for (IndexableField r : starredBy) {
|
for (IndexableField r : starredBy) {
|
||||||
accounts.add(new Account.Id(r.numericValue().intValue()));
|
accounts.add(new Account.Id(r.numericValue().intValue()));
|
||||||
}
|
}
|
||||||
cd.setStarredBy(accounts);
|
cd.setStarredBy(accounts);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeStar(Document doc, ChangeData cd) {
|
private void decodeStar(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
IndexableField[] star = doc.getFields(STAR_FIELD);
|
Collection<IndexableField> star = doc.get(STAR_FIELD);
|
||||||
Multimap<Account.Id, String> stars = ArrayListMultimap.create();
|
Multimap<Account.Id, String> stars = ArrayListMultimap.create();
|
||||||
for (IndexableField r : star) {
|
for (IndexableField r : star) {
|
||||||
StarredChangesUtil.StarField starField =
|
StarredChangesUtil.StarField starField =
|
||||||
@ -518,10 +572,10 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
cd.setStars(stars);
|
cd.setStars(stars);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void decodeReviewers(Document doc, ChangeData cd) {
|
private void decodeReviewers(Multimap<String, IndexableField> doc, ChangeData cd) {
|
||||||
cd.setReviewers(
|
cd.setReviewers(
|
||||||
ChangeField.parseReviewerFieldValues(
|
ChangeField.parseReviewerFieldValues(
|
||||||
FluentIterable.of(doc.getFields(REVIEWER_FIELD))
|
FluentIterable.from(doc.get(REVIEWER_FIELD))
|
||||||
.transform(
|
.transform(
|
||||||
new Function<IndexableField, String>() {
|
new Function<IndexableField, String>() {
|
||||||
@Override
|
@Override
|
||||||
@ -531,14 +585,16 @@ public class LuceneChangeIndex implements ChangeIndex {
|
|||||||
})));
|
})));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static <T> List<T> decodeProtos(Document doc, String fieldName,
|
private static <T> List<T> decodeProtos(Multimap<String, IndexableField> doc,
|
||||||
ProtobufCodec<T> codec) {
|
String fieldName, ProtobufCodec<T> codec) {
|
||||||
BytesRef[] bytesRefs = doc.getBinaryValues(fieldName);
|
Collection<IndexableField> fields = doc.get(fieldName);
|
||||||
if (bytesRefs.length == 0) {
|
if (fields.isEmpty()) {
|
||||||
return Collections.emptyList();
|
return Collections.emptyList();
|
||||||
}
|
}
|
||||||
List<T> result = new ArrayList<>(bytesRefs.length);
|
|
||||||
for (BytesRef r : bytesRefs) {
|
List<T> result = new ArrayList<>(fields.size());
|
||||||
|
for (IndexableField f : fields) {
|
||||||
|
BytesRef r = f.binaryValue();
|
||||||
result.add(codec.decode(r.bytes, r.offset, r.length));
|
result.add(codec.decode(r.bytes, r.offset, r.length));
|
||||||
}
|
}
|
||||||
return result;
|
return result;
|
||||||
|
@ -246,7 +246,8 @@ public class ChangeIndexer {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private CheckedFuture<?, IOException> submit(Callable<?> task) {
|
private CheckedFuture<?, IOException> submit(Callable<?> task) {
|
||||||
return Futures.makeChecked(executor.submit(task), MAPPER);
|
return Futures.makeChecked(
|
||||||
|
Futures.nonCancellationPropagating(executor.submit(task)), MAPPER);
|
||||||
}
|
}
|
||||||
|
|
||||||
private class IndexTask implements Callable<Void> {
|
private class IndexTask implements Callable<Void> {
|
||||||
|
@ -31,6 +31,7 @@ import com.google.gerrit.server.index.IndexRewriter;
|
|||||||
import com.google.gerrit.server.index.QueryOptions;
|
import com.google.gerrit.server.index.QueryOptions;
|
||||||
import com.google.gerrit.server.index.SchemaDefinitions;
|
import com.google.gerrit.server.index.SchemaDefinitions;
|
||||||
import com.google.gwtorm.server.OrmException;
|
import com.google.gwtorm.server.OrmException;
|
||||||
|
import com.google.gwtorm.server.OrmRuntimeException;
|
||||||
import com.google.gwtorm.server.ResultSet;
|
import com.google.gwtorm.server.ResultSet;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.google.inject.Provider;
|
import com.google.inject.Provider;
|
||||||
@ -136,6 +137,8 @@ public abstract class QueryProcessor<T> {
|
|||||||
throws OrmException, QueryParseException {
|
throws OrmException, QueryParseException {
|
||||||
try {
|
try {
|
||||||
return query(null, queries);
|
return query(null, queries);
|
||||||
|
} catch (OrmRuntimeException e) {
|
||||||
|
throw new OrmException(e.getMessage(), e);
|
||||||
} catch (OrmException e) {
|
} catch (OrmException e) {
|
||||||
Throwables.propagateIfInstanceOf(e.getCause(), QueryParseException.class);
|
Throwables.propagateIfInstanceOf(e.getCause(), QueryParseException.class);
|
||||||
throw e;
|
throw e;
|
||||||
|
Loading…
Reference in New Issue
Block a user