Optionally allow to reuse existing documents during reindexing
Change I1fd78958f6fa0848f81630abab0428520f412baa introduced a feature
to skip indexing of changes which were already present in the index.
This prevented unnecessary work, if Gerrit had to be shutdown while
performing an online reindex. However, this change also broke offline
reindexing, since this forcefully deleted all documents in the index
before reindexing, which wasn't transparent to the indexers.
This change is a reimplementation of the feature allowing to reuse
existing up-to-date documents in the index. The
`index.reuseExistingDocuments` option has to be set in the
`gerrit.config` to enable this feature. This works for offline and
online reindexing.
Existing but stale documents will be reindexed regardless.
The issue with offline reindexing is fixed, by not emptying the index
before indexing, if `index.reuseExistingDocuments` has been set.
Release-Notes: Optionally allow to reuse existing documents during reindexing
Change-Id: I3c5e7c1565c46738c4f5d37303f6ede296f604eb
diff --git a/Documentation/config-gerrit.txt b/Documentation/config-gerrit.txt
index 3ddc3ee..d9d4643 100644
--- a/Documentation/config-gerrit.txt
+++ b/Documentation/config-gerrit.txt
@@ -3488,6 +3488,21 @@
Excluded projects can later be reindexed by for example using the
link:cmd-index-changes-in-project.html[index changes in project command].
+[[index.reuseExistingDocuments]]index.reuseExistingDocuments::
++
+Whether to reuse index documents that already exist during reindexing.
++
+Currently, only supported by the changes index.
++
+This feature is useful, if the Gerrit server has to be restarted
+during an ongoing index online upgrade, since this would cause
+a complete reindexing otherwise that might take an extensive time.
++
+Each existing document in the index will be checked for staleness
+and reindexed if found to be stale.
++
+Defaults to false.
+
[[index.paginationType]]index.paginationType::
+
The pagination type to use when index queries are repeated to
diff --git a/java/com/google/gerrit/acceptance/ChangeIndexedCounter.java b/java/com/google/gerrit/acceptance/ChangeIndexedCounter.java
index 5991646..98287c8 100644
--- a/java/com/google/gerrit/acceptance/ChangeIndexedCounter.java
+++ b/java/com/google/gerrit/acceptance/ChangeIndexedCounter.java
@@ -41,6 +41,10 @@
return countsByChange.get(info._number);
}
+ public long getTotalCount() {
+ return countsByChange.asMap().values().stream().reduce(0L, Long::sum);
+ }
+
public void assertReindexOf(ChangeInfo info) {
assertReindexOf(info, 1);
}
diff --git a/java/com/google/gerrit/index/testing/FakeIndexVersionManager.java b/java/com/google/gerrit/index/testing/FakeIndexVersionManager.java
index 5044e38..40d51fd 100644
--- a/java/com/google/gerrit/index/testing/FakeIndexVersionManager.java
+++ b/java/com/google/gerrit/index/testing/FakeIndexVersionManager.java
@@ -40,7 +40,12 @@
SitePaths sitePaths,
PluginSetContext<OnlineUpgradeListener> listeners,
Collection<IndexDefinition<?, ?, ?>> defs) {
- super(sitePaths, listeners, defs, VersionManager.getOnlineUpgrade(cfg));
+ super(
+ sitePaths,
+ listeners,
+ defs,
+ VersionManager.getOnlineUpgrade(cfg),
+ cfg.getBoolean("index", "reuseExistingDocuments", false));
}
@Override
diff --git a/java/com/google/gerrit/lucene/LuceneVersionManager.java b/java/com/google/gerrit/lucene/LuceneVersionManager.java
index f3ba73d..265d3e0 100644
--- a/java/com/google/gerrit/lucene/LuceneVersionManager.java
+++ b/java/com/google/gerrit/lucene/LuceneVersionManager.java
@@ -49,7 +49,12 @@
SitePaths sitePaths,
PluginSetContext<OnlineUpgradeListener> listeners,
Collection<IndexDefinition<?, ?, ?>> defs) {
- super(sitePaths, listeners, defs, VersionManager.getOnlineUpgrade(cfg));
+ super(
+ sitePaths,
+ listeners,
+ defs,
+ VersionManager.getOnlineUpgrade(cfg),
+ cfg.getBoolean("index", "reuseExistingDocuments", false));
}
@Override
diff --git a/java/com/google/gerrit/pgm/Reindex.java b/java/com/google/gerrit/pgm/Reindex.java
index 2ed2b76..f4814ba 100644
--- a/java/com/google/gerrit/pgm/Reindex.java
+++ b/java/com/google/gerrit/pgm/Reindex.java
@@ -258,7 +258,9 @@
requireNonNull(
index, () -> String.format("no active search index configured for %s", def.getName()));
index.markReady(false);
- index.deleteAll();
+ if (!globalConfig.getBoolean("index", null, "reuseExistingDocuments", false)) {
+ index.deleteAll();
+ }
SiteIndexer<K, V, I> siteIndexer = def.getSiteIndexer();
siteIndexer.setProgressOut(System.err);
diff --git a/java/com/google/gerrit/server/index/IndexModule.java b/java/com/google/gerrit/server/index/IndexModule.java
index 96870ea..a8095f2 100644
--- a/java/com/google/gerrit/server/index/IndexModule.java
+++ b/java/com/google/gerrit/server/index/IndexModule.java
@@ -25,6 +25,7 @@
import com.google.common.util.concurrent.MoreExecutors;
import com.google.gerrit.extensions.events.LifecycleListener;
import com.google.gerrit.extensions.registration.DynamicSet;
+import com.google.gerrit.index.IndexConfig;
import com.google.gerrit.index.IndexDefinition;
import com.google.gerrit.index.IndexType;
import com.google.gerrit.index.SchemaDefinitions;
@@ -34,6 +35,7 @@
import com.google.gerrit.index.project.ProjectSchemaDefinitions;
import com.google.gerrit.lifecycle.LifecycleModule;
import com.google.gerrit.server.config.GerritServerConfig;
+import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.git.MultiProgressMonitor;
import com.google.gerrit.server.git.WorkQueue;
import com.google.gerrit.server.index.account.AccountIndexCollection;
@@ -47,6 +49,7 @@
import com.google.gerrit.server.index.change.ChangeIndexRewriter;
import com.google.gerrit.server.index.change.ChangeIndexer;
import com.google.gerrit.server.index.change.ChangeSchemaDefinitions;
+import com.google.gerrit.server.index.change.StalenessChecker;
import com.google.gerrit.server.index.group.GroupIndexCollection;
import com.google.gerrit.server.index.group.GroupIndexDefinition;
import com.google.gerrit.server.index.group.GroupIndexRewriter;
@@ -129,6 +132,7 @@
bind(ChangeIndexCollection.class);
listener().to(ChangeIndexCollection.class);
factory(ChangeIndexer.Factory.class);
+ factory(StalenessChecker.Factory.class);
bind(GroupIndexRewriter.class);
// GroupIndexCollection is already bound very high up in SchemaModule.
@@ -255,6 +259,13 @@
return MoreExecutors.listeningDecorator(workQueue.createQueue(threads, "Index-Batch", true));
}
+ @Provides
+ @Singleton
+ StalenessChecker getChangeStalenessChecker(
+ ChangeIndexCollection indexes, GitRepositoryManager repoManager, IndexConfig indexConfig) {
+ return new StalenessChecker(indexes, repoManager, indexConfig);
+ }
+
@Singleton
private static class ShutdownIndexExecutors implements LifecycleListener {
private final ListeningExecutorService interactiveExecutor;
diff --git a/java/com/google/gerrit/server/index/OnlineReindexer.java b/java/com/google/gerrit/server/index/OnlineReindexer.java
index eef394d..98abf46 100644
--- a/java/com/google/gerrit/server/index/OnlineReindexer.java
+++ b/java/com/google/gerrit/server/index/OnlineReindexer.java
@@ -42,18 +42,21 @@
private final PluginSetContext<OnlineUpgradeListener> listeners;
private I index;
private final AtomicBoolean running = new AtomicBoolean();
+ private final boolean reuseExistingDocuments;
public OnlineReindexer(
IndexDefinition<K, V, I> def,
int oldVersion,
int newVersion,
- PluginSetContext<OnlineUpgradeListener> listeners) {
+ PluginSetContext<OnlineUpgradeListener> listeners,
+ boolean reuseExistingDocuments) {
this.name = def.getName();
this.indexes = def.getIndexCollection();
this.batchIndexer = def.getSiteIndexer();
this.oldVersion = oldVersion;
this.newVersion = newVersion;
this.listeners = listeners;
+ this.reuseExistingDocuments = reuseExistingDocuments;
}
/** Starts the background process. */
@@ -106,7 +109,7 @@
"Starting online reindex of %s from schema version %s to %s",
name, version(indexes.getSearchIndex()), version(index));
- if (oldVersion != newVersion) {
+ if (!reuseExistingDocuments && oldVersion != newVersion) {
index.deleteAll();
}
SiteIndexer.Result result = batchIndexer.indexAll(index);
diff --git a/java/com/google/gerrit/server/index/VersionManager.java b/java/com/google/gerrit/server/index/VersionManager.java
index 39930a6..2c38caf 100644
--- a/java/com/google/gerrit/server/index/VersionManager.java
+++ b/java/com/google/gerrit/server/index/VersionManager.java
@@ -59,6 +59,7 @@
}
protected final boolean onlineUpgrade;
+ protected final boolean reuseExistingDocuments;
protected final String runReindexMsg;
protected final SitePaths sitePaths;
@@ -72,7 +73,8 @@
SitePaths sitePaths,
PluginSetContext<OnlineUpgradeListener> listeners,
Collection<IndexDefinition<?, ?, ?>> defs,
- boolean onlineUpgrade) {
+ boolean onlineUpgrade,
+ boolean reuseExistingDocuments) {
this.sitePaths = sitePaths;
this.listeners = listeners;
this.defs = Maps.newHashMapWithExpectedSize(defs.size());
@@ -82,6 +84,7 @@
this.reindexers = Maps.newHashMapWithExpectedSize(defs.size());
this.onlineUpgrade = onlineUpgrade;
+ this.reuseExistingDocuments = reuseExistingDocuments;
this.runReindexMsg =
"No index versions for index '%s' ready; run java -jar "
+ sitePaths.gerrit_war.toAbsolutePath()
@@ -190,7 +193,7 @@
if (!reindexers.containsKey(def.getName())) {
int latest = write.get(0).version;
OnlineReindexer<K, V, I> reindexer =
- new OnlineReindexer<>(def, search.version, latest, listeners);
+ new OnlineReindexer<>(def, search.version, latest, listeners, reuseExistingDocuments);
reindexers.put(def.getName(), reindexer);
}
}
diff --git a/java/com/google/gerrit/server/index/change/AllChangesIndexer.java b/java/com/google/gerrit/server/index/change/AllChangesIndexer.java
index 3935108..5f35861 100644
--- a/java/com/google/gerrit/server/index/change/AllChangesIndexer.java
+++ b/java/com/google/gerrit/server/index/change/AllChangesIndexer.java
@@ -52,6 +52,7 @@
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.eclipse.jgit.lib.Config;
import org.eclipse.jgit.lib.ObjectId;
@@ -84,9 +85,11 @@
private final GitRepositoryManager repoManager;
private final ListeningExecutorService executor;
private final ChangeIndexer.Factory indexerFactory;
+ private final StalenessChecker.Factory stalenessCheckerFactory;
private final ChangeNotes.Factory notesFactory;
private final ProjectCache projectCache;
private final Set<Project.NameKey> projectsToSkip;
+ private final boolean reuseExistingDocuments;
@Inject
AllChangesIndexer(
@@ -95,6 +98,7 @@
GitRepositoryManager repoManager,
@IndexExecutor(BATCH) ListeningExecutorService executor,
ChangeIndexer.Factory indexerFactory,
+ StalenessChecker.Factory stalenessCheckerFactory,
ChangeNotes.Factory notesFactory,
ProjectCache projectCache,
@GerritServerConfig Config config) {
@@ -103,6 +107,7 @@
this.repoManager = repoManager;
this.executor = executor;
this.indexerFactory = indexerFactory;
+ this.stalenessCheckerFactory = stalenessCheckerFactory;
this.notesFactory = notesFactory;
this.projectCache = projectCache;
this.projectsToSkip =
@@ -110,6 +115,7 @@
.stream()
.map(p -> Project.NameKey.parse(p))
.collect(Collectors.toSet());
+ this.reuseExistingDocuments = config.getBoolean("index", null, "reuseExistingDocuments", false);
}
@AutoValue
@@ -218,20 +224,27 @@
}
private class ProjectSliceIndexer implements Callable<Void> {
- private final ChangeIndexer indexer;
private final ProjectSlice projectSlice;
private final ProgressMonitor done;
private final ProgressMonitor failed;
+ private final Consumer<ChangeData> indexAction;
private ProjectSliceIndexer(
ChangeIndexer indexer,
ProjectSlice projectSlice,
ProgressMonitor done,
ProgressMonitor failed) {
- this.indexer = indexer;
this.projectSlice = projectSlice;
this.done = done;
this.failed = failed;
+ if (reuseExistingDocuments) {
+ indexAction =
+ cd -> {
+ var unused = indexer.reindexIfStale(cd);
+ };
+ } else {
+ indexAction = cd -> indexer.index(cd);
+ }
}
@Override
@@ -271,7 +284,7 @@
return;
}
try {
- indexer.index(changeDataFactory.create(r.notes()));
+ indexAction.accept(changeDataFactory.create(r.notes()));
done.update(1);
verboseWriter.format(
"Reindexed change %d (project: %s)\n", r.id().get(), r.notes().getProjectName().get());
@@ -385,13 +398,15 @@
for (int slice = 0; slice < slices; slice++) {
ProjectSlice projectSlice = ProjectSlice.create(name, slice, slices, metaIdByChange);
+ ChangeIndexer indexer;
+ if (reuseExistingDocuments) {
+ indexer =
+ indexerFactory.create(executor, index, stalenessCheckerFactory.create(index));
+ } else {
+ indexer = indexerFactory.create(executor, index);
+ }
ListenableFuture<?> future =
- executor.submit(
- reindexProjectSlice(
- indexerFactory.create(executor, index),
- projectSlice,
- doneTask,
- failedTask));
+ executor.submit(reindexProjectSlice(indexer, projectSlice, doneTask, failedTask));
String description = "project " + name + " (" + slice + "/" + slices + ")";
addErrorListener(future, description, projTask, ok);
sliceIndexerFutures.add(future);
diff --git a/java/com/google/gerrit/server/index/change/ChangeIndexer.java b/java/com/google/gerrit/server/index/change/ChangeIndexer.java
index 562f9c4..4ec390d 100644
--- a/java/com/google/gerrit/server/index/change/ChangeIndexer.java
+++ b/java/com/google/gerrit/server/index/change/ChangeIndexer.java
@@ -70,6 +70,9 @@
public interface Factory {
ChangeIndexer create(ListeningExecutorService executor, ChangeIndex index);
+ ChangeIndexer create(
+ ListeningExecutorService executor, ChangeIndex index, StalenessChecker stalenessChecker);
+
ChangeIndexer create(ListeningExecutorService executor, ChangeIndexCollection indexes);
}
@@ -121,6 +124,31 @@
ChangeNotes.Factory notesFactory,
ThreadLocalRequestContext context,
PluginSetContext<ChangeIndexedListener> indexedListeners,
+ @IndexExecutor(BATCH) ListeningExecutorService batchExecutor,
+ IsFirstInsertForEntry isFirstInsertForEntry,
+ @Assisted ListeningExecutorService executor,
+ @Assisted ChangeIndex index,
+ @Assisted StalenessChecker stalenessChecker) {
+ this.executor = executor;
+ this.changeDataFactory = changeDataFactory;
+ this.notesFactory = notesFactory;
+ this.context = context;
+ this.indexedListeners = indexedListeners;
+ this.batchExecutor = batchExecutor;
+ this.autoReindexIfStale = autoReindexIfStale(cfg);
+ this.isFirstInsertForEntry = isFirstInsertForEntry;
+ this.index = index;
+ this.indexes = null;
+ this.stalenessChecker = stalenessChecker;
+ }
+
+ @AssistedInject
+ ChangeIndexer(
+ @GerritServerConfig Config cfg,
+ ChangeData.Factory changeDataFactory,
+ ChangeNotes.Factory notesFactory,
+ ThreadLocalRequestContext context,
+ PluginSetContext<ChangeIndexedListener> indexedListeners,
StalenessChecker stalenessChecker,
@IndexExecutor(BATCH) ListeningExecutorService batchExecutor,
@Assisted ListeningExecutorService executor,
@@ -350,7 +378,7 @@
* @param id ID of the change to index.
* @return future for reindexing the change; returns true if the change was stale.
*/
- public ListenableFuture<Boolean> reindexIfStale(Project.NameKey project, Change.Id id) {
+ public ListenableFuture<Boolean> asyncReindexIfStale(Project.NameKey project, Change.Id id) {
ReindexIfStaleTask task = new ReindexIfStaleTask(project, id);
if (queuedReindexIfStaleTasks.add(task)) {
return submit(task, batchExecutor);
@@ -358,6 +386,41 @@
return Futures.immediateFuture(false);
}
+ /**
+ * Synchronously check if a change is stale, and reindex if it is.
+ *
+ * @param cd the change data to be checked for staleness.
+ * @return true if the change was stale, false if it was up-to-date
+ */
+ public boolean reindexIfStale(ChangeData cd) {
+ return reindexIfStale(cd.project(), cd.getId());
+ }
+
+ /**
+ * Synchronously check if a change is stale, and reindex if it is.
+ *
+ * @param project the project to which the change belongs.
+ * @param id ID of the change to index.
+ * @return true if the change was stale, false if it was up-to-date
+ */
+ public boolean reindexIfStale(Project.NameKey project, Change.Id id) {
+ try {
+ StalenessCheckResult stalenessCheckResult = stalenessChecker.check(id);
+ if (stalenessCheckResult.isStale()) {
+ logger.atInfo().log("Reindexing stale document %s", stalenessCheckResult);
+ indexImpl(changeDataFactory.create(project, id));
+ return true;
+ }
+ } catch (Exception e) {
+ if (!isCausedByRepositoryNotFoundException(e)) {
+ throw e;
+ }
+ logger.atFine().log(
+ "Change %s belongs to deleted project %s, aborting reindexing the change.", id, project);
+ }
+ return false;
+ }
+
private void autoReindexIfStale(ChangeData cd) {
autoReindexIfStale(cd.project(), cd.getId());
}
@@ -366,7 +429,7 @@
if (autoReindexIfStale) {
// Don't retry indefinitely; if this fails the change will be stale.
@SuppressWarnings("unused")
- Future<?> possiblyIgnoredError = reindexIfStale(project, id);
+ Future<?> possiblyIgnoredError = asyncReindexIfStale(project, id);
}
}
@@ -546,22 +609,7 @@
@Override
public Boolean callImpl() throws Exception {
remove();
- try {
- StalenessCheckResult stalenessCheckResult = stalenessChecker.check(id);
- if (stalenessCheckResult.isStale()) {
- logger.atInfo().log("Reindexing stale document %s", stalenessCheckResult);
- indexImpl(changeDataFactory.create(project, id));
- return true;
- }
- } catch (Exception e) {
- if (!isCausedByRepositoryNotFoundException(e)) {
- throw e;
- }
- logger.atFine().log(
- "Change %s belongs to deleted project %s, aborting reindexing the change.",
- id.get(), project.get());
- }
- return false;
+ return reindexIfStale(project, id);
}
@Override
diff --git a/java/com/google/gerrit/server/index/change/IndexedChangeQuery.java b/java/com/google/gerrit/server/index/change/IndexedChangeQuery.java
index 00642a9..f7ff13c 100644
--- a/java/com/google/gerrit/server/index/change/IndexedChangeQuery.java
+++ b/java/com/google/gerrit/server/index/change/IndexedChangeQuery.java
@@ -21,6 +21,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.gerrit.entities.Change;
import com.google.gerrit.index.IndexConfig;
@@ -51,6 +52,10 @@
*/
public class IndexedChangeQuery extends IndexedQuery<Change.Id, ChangeData>
implements ChangeDataSource, Matchable<ChangeData> {
+ public static QueryOptions oneResult() {
+ IndexConfig config = IndexConfig.createDefault();
+ return createOptions(config, 0, 1, config.pageSizeMultiplier(), 1, ImmutableSet.of());
+ }
public static QueryOptions createOptions(
IndexConfig config, int start, int limit, Set<String> fields) {
diff --git a/java/com/google/gerrit/server/index/change/StalenessChecker.java b/java/com/google/gerrit/server/index/change/StalenessChecker.java
index eb4af01..83f6189 100644
--- a/java/com/google/gerrit/server/index/change/StalenessChecker.java
+++ b/java/com/google/gerrit/server/index/change/StalenessChecker.java
@@ -27,6 +27,7 @@
import com.google.common.collect.SetMultimap;
import com.google.common.collect.Sets;
import com.google.common.flogger.FluentLogger;
+import com.google.gerrit.common.Nullable;
import com.google.gerrit.common.UsedAt;
import com.google.gerrit.entities.Change;
import com.google.gerrit.entities.Project;
@@ -36,8 +37,8 @@
import com.google.gerrit.server.git.GitRepositoryManager;
import com.google.gerrit.server.index.StalenessCheckResult;
import com.google.gerrit.server.query.change.ChangeData;
-import com.google.inject.Inject;
-import com.google.inject.Singleton;
+import com.google.inject.assistedinject.Assisted;
+import com.google.inject.assistedinject.AssistedInject;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
@@ -50,34 +51,58 @@
* Checker that compares values stored in the change index to metadata in NoteDb to detect index
* documents that should have been updated (= stale).
*/
-@Singleton
public class StalenessChecker {
private static final FluentLogger logger = FluentLogger.forEnclosingClass();
+ public interface Factory {
+ StalenessChecker create(ChangeIndex index);
+ }
+
public static final ImmutableSet<String> FIELDS =
ImmutableSet.of(
ChangeField.CHANGE_SPEC.getName(),
ChangeField.REF_STATE_SPEC.getName(),
ChangeField.REF_STATE_PATTERN_SPEC.getName());
- private final ChangeIndexCollection indexes;
+ @Nullable private final ChangeIndexCollection indexes;
+ @Nullable private final ChangeIndex index;
private final GitRepositoryManager repoManager;
private final IndexConfig indexConfig;
- @Inject
- StalenessChecker(
+ public StalenessChecker(
ChangeIndexCollection indexes, GitRepositoryManager repoManager, IndexConfig indexConfig) {
this.indexes = indexes;
+ this.index = null;
this.repoManager = repoManager;
this.indexConfig = indexConfig;
}
+ @AssistedInject
+ StalenessChecker(
+ GitRepositoryManager repoManager, IndexConfig indexConfig, @Assisted ChangeIndex index) {
+ this.indexes = null;
+ this.repoManager = repoManager;
+ this.indexConfig = indexConfig;
+ this.index = index;
+ }
+
/**
* Returns a {@link StalenessCheckResult} with structured information about staleness of the
* provided {@link com.google.gerrit.entities.Change.Id}.
*/
public StalenessCheckResult check(Change.Id id) {
- ChangeIndex i = indexes.getSearchIndex();
+ if (index != null) {
+ return check(id, index);
+ }
+ return check(id, indexes.getSearchIndex());
+ }
+
+ /**
+ * Returns a {@link StalenessCheckResult} with structured information about staleness of the
+ * provided {@link com.google.gerrit.entities.Change.Id} in the provided {@link
+ * com.google.gerrit.server.index.change.ChangeIndex}.
+ */
+ private StalenessCheckResult check(Change.Id id, ChangeIndex i) {
if (i == null) {
return StalenessCheckResult
.notStale(); // No index; caller couldn't do anything if it is stale.
diff --git a/javatests/com/google/gerrit/acceptance/pgm/AbstractReindexTests.java b/javatests/com/google/gerrit/acceptance/pgm/AbstractReindexTests.java
index c5059c3..66c4078 100644
--- a/javatests/com/google/gerrit/acceptance/pgm/AbstractReindexTests.java
+++ b/javatests/com/google/gerrit/acceptance/pgm/AbstractReindexTests.java
@@ -30,6 +30,7 @@
import com.google.gerrit.entities.Change;
import com.google.gerrit.entities.Project;
import com.google.gerrit.extensions.api.GerritApi;
+import com.google.gerrit.extensions.api.changes.ReviewInput;
import com.google.gerrit.extensions.common.ChangeInput;
import com.google.gerrit.index.IndexDefinition;
import com.google.gerrit.index.Schema;
@@ -67,9 +68,45 @@
Files.createDirectory(sitePaths.index_dir);
assertServerStartupFails();
- runGerrit("reindex", "-d", sitePaths.site_path.toString(), "--show-stack-trace");
+ runGerrit("reindex", "-d", sitePaths.site_path.toString(), "--show-stack-trace", "--verbose");
+ assertReady(ChangeSchemaDefinitions.INSTANCE.getLatest().getVersion());
+ assertIndexQueries();
+ }
+
+ @Test
+ public void reindexWithSkipExistingDocumentsEnabled() throws Exception {
+ updateConfig(config -> config.setBoolean("index", null, "reuseExistingDocuments", true));
+ setUpChange();
+
+ MoreFiles.deleteRecursively(sitePaths.index_dir, RecursiveDeleteOption.ALLOW_INSECURE);
+ Files.createDirectory(sitePaths.index_dir);
+ assertServerStartupFails();
+
+ runGerrit("reindex", "-d", sitePaths.site_path.toString(), "--show-stack-trace", "--verbose");
assertReady(ChangeSchemaDefinitions.INSTANCE.getLatest().getVersion());
+ runGerrit("reindex", "-d", sitePaths.site_path.toString(), "--show-stack-trace", "--verbose");
+ assertIndexQueries();
+
+ Files.copy(sitePaths.index_dir, sitePaths.resolve("index-backup"));
+ try (ServerContext ctx = startServer()) {
+ GerritApi gApi = ctx.getInjector().getInstance(GerritApi.class);
+ gApi.changes().id(changeId).revision(1).review(ReviewInput.approve());
+ // Query change index
+ assertThat(gApi.changes().query("label:Code-Review+2").get().stream().map(c -> c.changeId))
+ .containsExactly(changeId);
+ }
+ MoreFiles.deleteRecursively(sitePaths.index_dir, RecursiveDeleteOption.ALLOW_INSECURE);
+ Files.copy(sitePaths.resolve("index-backup"), sitePaths.index_dir);
+ runGerrit("reindex", "-d", sitePaths.site_path.toString(), "--show-stack-trace", "--verbose");
+ try (ServerContext ctx = startServer()) {
+ GerritApi gApi = ctx.getInjector().getInstance(GerritApi.class);
+ assertThat(gApi.changes().query("label:Code-Review+2").get().stream().map(c -> c.changeId))
+ .containsExactly(changeId);
+ }
+ }
+
+ private void assertIndexQueries() throws Exception {
try (ServerContext ctx = startServer()) {
GerritApi gApi = ctx.getInjector().getInstance(GerritApi.class);
// Query change index
diff --git a/javatests/com/google/gerrit/acceptance/server/index/change/BUILD b/javatests/com/google/gerrit/acceptance/server/index/change/BUILD
new file mode 100644
index 0000000..3be5249
--- /dev/null
+++ b/javatests/com/google/gerrit/acceptance/server/index/change/BUILD
@@ -0,0 +1,7 @@
+load("//javatests/com/google/gerrit/acceptance:tests.bzl", "acceptance_tests")
+
+acceptance_tests(
+ srcs = glob(["*IT.java"]),
+ group = "server_index",
+ labels = ["server"],
+)
diff --git a/javatests/com/google/gerrit/acceptance/server/index/change/LuceneChangeIndexerIT.java b/javatests/com/google/gerrit/acceptance/server/index/change/LuceneChangeIndexerIT.java
new file mode 100644
index 0000000..b8af367
--- /dev/null
+++ b/javatests/com/google/gerrit/acceptance/server/index/change/LuceneChangeIndexerIT.java
@@ -0,0 +1,137 @@
+// Copyright (C) 2024 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.acceptance.server.index.change;
+
+import static com.google.common.truth.Truth.assertThat;
+
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.ListMultimap;
+import com.google.gerrit.acceptance.AbstractDaemonTest;
+import com.google.gerrit.acceptance.ChangeIndexedCounter;
+import com.google.gerrit.acceptance.ExtensionRegistry;
+import com.google.gerrit.acceptance.ExtensionRegistry.Registration;
+import com.google.gerrit.acceptance.PushOneCommit;
+import com.google.gerrit.acceptance.config.GerritConfig;
+import com.google.gerrit.entities.Project.NameKey;
+import com.google.gerrit.index.IndexDefinition;
+import com.google.gerrit.index.RefState;
+import com.google.gerrit.index.SiteIndexer.Result;
+import com.google.gerrit.server.index.change.AllChangesIndexer;
+import com.google.gerrit.server.index.change.ChangeIndex;
+import com.google.gerrit.server.query.change.ChangeData;
+import com.google.gerrit.testing.ConfigSuite;
+import com.google.inject.Inject;
+import java.util.Collection;
+import java.util.Set;
+import org.eclipse.jgit.lib.Config;
+import org.eclipse.jgit.lib.ObjectId;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LuceneChangeIndexerIT extends AbstractDaemonTest {
+ @ConfigSuite.Default
+ public static Config defaultConfig() {
+ Config cfg = new Config();
+ cfg.setBoolean("index", null, "autoReindexIfStale", false);
+ cfg.setString("index", null, "type", "lucene");
+ return cfg;
+ }
+
+ @Inject private ExtensionRegistry extensionRegistry;
+
+ @Inject private Collection<IndexDefinition<?, ?, ?>> indexDefs;
+ private AllChangesIndexer allChangesIndexer;
+ private ChangeIndex index;
+
+ @Before
+ public void setup() {
+ IndexDefinition<?, ?, ?> changeIndex =
+ indexDefs.stream().filter(i -> i.getName().equals("changes")).findFirst().get();
+ allChangesIndexer = (AllChangesIndexer) changeIndex.getSiteIndexer();
+ index = (ChangeIndex) changeIndex.getIndexCollection().getWriteIndexes().iterator().next();
+ }
+
+ @Test
+ @GerritConfig(name = "index.reuseExistingDocuments", value = "false")
+ public void testReindexWithoutReuse() throws Exception {
+ ChangeIndexedCounter changeIndexedCounter = new ChangeIndexedCounter();
+ try (Registration registration =
+ extensionRegistry.newRegistration().add(changeIndexedCounter)) {
+ createChange();
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(1);
+ changeIndexedCounter.clear();
+ reindexChanges();
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(1);
+
+ createIndexWithMissingChangeAndReindex(changeIndexedCounter);
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(2);
+
+ createIndexWithStaleChangeAndReindex(changeIndexedCounter);
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(3);
+ }
+ }
+
+ @Test
+ @GerritConfig(name = "index.reuseExistingDocuments", value = "true")
+ public void testReindexWithReuse() throws Exception {
+ ChangeIndexedCounter changeIndexedCounter = new ChangeIndexedCounter();
+ try (Registration registration =
+ extensionRegistry.newRegistration().add(changeIndexedCounter)) {
+ createChange();
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(1);
+ changeIndexedCounter.clear();
+ reindexChanges();
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(0);
+
+ createIndexWithMissingChangeAndReindex(changeIndexedCounter);
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(1);
+
+ createIndexWithStaleChangeAndReindex(changeIndexedCounter);
+ assertThat(changeIndexedCounter.getTotalCount()).isEqualTo(1);
+ }
+ }
+
+ private void createIndexWithMissingChangeAndReindex(ChangeIndexedCounter changeIndexedCounter)
+ throws Exception {
+ PushOneCommit.Result res = createChange();
+ index.delete(res.getChange().getId());
+ changeIndexedCounter.clear();
+ reindexChanges();
+ }
+
+ private void createIndexWithStaleChangeAndReindex(ChangeIndexedCounter changeIndexedCounter)
+ throws Exception {
+ PushOneCommit.Result res = createChange();
+ ChangeData wrongChangeData = res.getChange();
+ ListMultimap<NameKey, RefState> refStates =
+ LinkedListMultimap.create(wrongChangeData.getRefStates());
+ refStates.replaceValues(
+ project,
+ Set.of(
+ RefState.create(
+ "refs/changes/abcd",
+ ObjectId.fromString("deadbeefdeadbeefdeadbeefdeadbeefdeadbeef"))));
+ wrongChangeData.setRefStates(ImmutableSetMultimap.copyOf(refStates));
+ index.replace(wrongChangeData);
+ changeIndexedCounter.clear();
+ reindexChanges();
+ }
+
+ private void reindexChanges() throws Exception {
+ Result res = allChangesIndexer.indexAll(index);
+ assertThat(res.success()).isTrue();
+ }
+}
diff --git a/javatests/com/google/gerrit/server/query/change/AbstractQueryChangesTest.java b/javatests/com/google/gerrit/server/query/change/AbstractQueryChangesTest.java
index 6258b18..7a7cff5 100644
--- a/javatests/com/google/gerrit/server/query/change/AbstractQueryChangesTest.java
+++ b/javatests/com/google/gerrit/server/query/change/AbstractQueryChangesTest.java
@@ -3576,7 +3576,7 @@
getChangeApi(change).addReviewer(anotherUser.toString());
assertQuery("reviewer:self", change);
- assertThat(indexer.reindexIfStale(project, change.getId()).get()).isFalse();
+ assertThat(indexer.reindexIfStale(project, change.getId())).isFalse();
// Remove reviewer behind index's back.
ChangeUpdate update = newUpdate(change);
@@ -3585,7 +3585,7 @@
// Index is stale.
assertQuery("reviewer:self", change);
- assertThat(indexer.reindexIfStale(project, change.getId()).get()).isTrue();
+ assertThat(indexer.reindexIfStale(project, change.getId())).isTrue();
assertQuery("reviewer:self");
// Index is not stale when a draft comment exists
@@ -3594,7 +3594,7 @@
in.message = "nit: trailing whitespace";
in.path = Patch.COMMIT_MSG;
getChangeApi(change).current().createDraft(in);
- assertThat(indexer.reindexIfStale(project, change.getId()).get()).isFalse();
+ assertThat(indexer.reindexIfStale(project, change.getId())).isFalse();
}
@Test