diff --git a/changelog/unreleased/SOLR-17725-CoreAdmin-API-to-upgrade-an-index-in-place.yml b/changelog/unreleased/SOLR-17725-CoreAdmin-API-to-upgrade-an-index-in-place.yml
new file mode 100644
index 000000000000..0ac334f8b566
--- /dev/null
+++ b/changelog/unreleased/SOLR-17725-CoreAdmin-API-to-upgrade-an-index-in-place.yml
@@ -0,0 +1,9 @@
+# See https://github.com/apache/solr/blob/main/dev-docs/changelog.adoc
+title: CoreAdmin API (/admin/cores?action=UPGRADECOREINDEX) to upgrade an index in-place
+type: added
+authors:
+ - name: Rahul Goswami
+links:
+ - name: SOLR-17725
+ url: https://issues.apache.org/jira/browse/SOLR-17725
+
diff --git a/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexRequestBody.java b/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexRequestBody.java
new file mode 100644
index 000000000000..ecc3081014e9
--- /dev/null
+++ b/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexRequestBody.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.solr.client.api.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.media.Schema;
+
+public class UpgradeCoreIndexRequestBody {
+
+ @Schema(description = "Request ID to track this action which will be processed asynchronously.")
+ @JsonProperty
+ public String async;
+
+ @Schema(
+ description =
+ "updateChain to be used for reindexing during index upgrade if you don't want to use the one used by /update by default")
+ @JsonProperty
+ public String updateChain;
+}
diff --git a/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexResponse.java b/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexResponse.java
new file mode 100644
index 000000000000..09e0ba1e8b22
--- /dev/null
+++ b/solr/api/src/java/org/apache/solr/client/api/model/UpgradeCoreIndexResponse.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.solr.client.api.model;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.media.Schema;
+
+public class UpgradeCoreIndexResponse extends SolrJerseyResponse {
+ @Schema(description = "The name of the core.")
+ @JsonProperty
+ public String core;
+
+ @Schema(description = "The total number of segments eligible for upgrade.")
+ @JsonProperty
+ public Integer numSegmentsEligibleForUpgrade;
+
+ @Schema(description = "The number of segments successfully upgraded.")
+ @JsonProperty
+ public Integer numSegmentsUpgraded;
+
+ @Schema(description = "Status of the core index upgrade operation.")
+ @JsonProperty
+ public String upgradeStatus;
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
index 0a15462f13fe..3cd2bb141e73 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminOperation.java
@@ -38,6 +38,7 @@
import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.STATUS;
import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.SWAP;
import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.UNLOAD;
+import static org.apache.solr.common.params.CoreAdminParams.CoreAdminAction.UPGRADECOREINDEX;
import static org.apache.solr.handler.admin.CoreAdminHandler.CallInfo;
import java.lang.invoke.MethodHandles;
@@ -256,7 +257,8 @@ public enum CoreAdminOperation implements CoreAdminOp {
final ListCoreSnapshotsResponse response = coreSnapshotAPI.listSnapshots(coreName);
V2ApiUtils.squashIntoSolrResponseWithoutHeader(it.rsp, response);
- });
+ }),
+ UPGRADECOREINDEX_OP(UPGRADECOREINDEX, new UpgradeCoreIndexOp());
final CoreAdminParams.CoreAdminAction action;
final CoreAdminOp fun;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/UpgradeCoreIndexOp.java b/solr/core/src/java/org/apache/solr/handler/admin/UpgradeCoreIndexOp.java
new file mode 100644
index 000000000000..8fff5c93d310
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/UpgradeCoreIndexOp.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.solr.handler.admin;
+
+import org.apache.solr.client.api.model.UpgradeCoreIndexRequestBody;
+import org.apache.solr.client.api.model.UpgradeCoreIndexResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.api.UpgradeCoreIndex;
+import org.apache.solr.handler.api.V2ApiUtils;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+class UpgradeCoreIndexOp implements CoreAdminHandler.CoreAdminOp {
+ @FunctionalInterface
+ public interface UpgradeCoreIndexFactory {
+ UpgradeCoreIndex create(
+ CoreContainer coreContainer,
+ CoreAdminHandler.CoreAdminAsyncTracker coreAdminAsyncTracker,
+ SolrQueryRequest req,
+ SolrQueryResponse rsp);
+ }
+
+ static UpgradeCoreIndexFactory UPGRADE_CORE_INDEX_FACTORY = UpgradeCoreIndex::new;
+
+ @Override
+ public boolean isExpensive() {
+ return true;
+ }
+
+ @Override
+ public void execute(CoreAdminHandler.CallInfo it) throws Exception {
+
+ assert it.handler.coreContainer != null;
+ if (it.handler.coreContainer.isZooKeeperAware()) {
+ throw new SolrException(
+ SolrException.ErrorCode.BAD_REQUEST,
+ "action=UPGRADECOREINDEX is not supported in SolrCloud mode. As an alternative, in order to upgrade index, configure LatestVersionMergePolicyFactory in solrconfig.xml and reindex the data in your collection.");
+ }
+
+ SolrParams params = it.req.getParams();
+ String cname = params.required().get(CoreAdminParams.CORE);
+ final boolean isAsync = params.get(CommonAdminParams.ASYNC) != null;
+ final var requestBody = new UpgradeCoreIndexRequestBody();
+ requestBody.updateChain = params.get(UpdateParams.UPDATE_CHAIN);
+
+ UpgradeCoreIndex upgradeCoreIndexApi =
+ UPGRADE_CORE_INDEX_FACTORY.create(
+ it.handler.coreContainer, it.handler.coreAdminAsyncTracker, it.req, it.rsp);
+ final UpgradeCoreIndexResponse response =
+ upgradeCoreIndexApi.upgradeCoreIndex(cname, requestBody);
+ V2ApiUtils.squashIntoSolrResponseWithoutHeader(it.rsp, response);
+
+ if (isAsync) {
+ final var opResponse = new NamedList<>();
+ V2ApiUtils.squashIntoNamedListWithoutHeader(opResponse, response);
+ // REQUESTSTATUS is returning the inner response NamedList as a positional array
+ // ([k1,v1,k2,v2...]).
+ // so converting to a map
+ it.rsp.addResponse(opResponse.asMap(1));
+ }
+ }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/UpgradeCoreIndex.java b/solr/core/src/java/org/apache/solr/handler/admin/api/UpgradeCoreIndex.java
new file mode 100644
index 000000000000..a0f69633b508
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/UpgradeCoreIndex.java
@@ -0,0 +1,420 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.solr.handler.admin.api;
+
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.List;
+import java.util.Set;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.index.StoredFields;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.Version;
+import org.apache.solr.client.api.model.UpgradeCoreIndexRequestBody;
+import org.apache.solr.client.api.model.UpgradeCoreIndexResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.admin.CoreAdminHandler;
+import org.apache.solr.index.LatestVersionMergePolicy;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.DocValuesIteratorCache;
+import org.apache.solr.search.SolrDocumentFetcher;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.util.RefCounted;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implements the UPGRADECOREINDEX CoreAdmin action, which upgrades an existing core's index
+ * in-place by reindexing documents from segments belonging to older Lucene versions, so that they
+ * get written into latest version segments.
+ *
+ *
The upgrade process:
+ *
+ *
+ * - Temporarily installs {@link LatestVersionMergePolicy} to prevent older-version segments
+ * from participating in merges during reindexing.
+ *
- Iterates each segment whose {@code minVersion} is older than the current Lucene major
+ * version. For each live document, rebuilds a {@link SolrInputDocument} from stored fields,
+ * decorates it with non-stored DocValues fields (excluding copyField targets), and re-adds it
+ * through Solr's update pipeline.
+ *
- Commits the changes and validates that no older-format segments remain.
+ *
- Restores the original merge policy.
+ *
+ *
+ * @see LatestVersionMergePolicy
+ * @see UpgradeCoreIndexRequestBody
+ * @see UpgradeCoreIndexResponse
+ */
+public class UpgradeCoreIndex extends CoreAdminAPIBase {
+ private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ public enum CoreIndexUpgradeStatus {
+ UPGRADE_SUCCESSFUL,
+ ERROR,
+ NO_UPGRADE_NEEDED;
+ }
+
+ private static final int RETRY_COUNT_FOR_SEGMENT_DELETION = 5;
+
+ public UpgradeCoreIndex(
+ CoreContainer coreContainer,
+ CoreAdminHandler.CoreAdminAsyncTracker coreAdminAsyncTracker,
+ SolrQueryRequest req,
+ SolrQueryResponse rsp) {
+ super(coreContainer, coreAdminAsyncTracker, req, rsp);
+ }
+
+ @Override
+ public boolean isExpensive() {
+ return true;
+ }
+
+ public UpgradeCoreIndexResponse upgradeCoreIndex(
+ String coreName, UpgradeCoreIndexRequestBody requestBody) throws Exception {
+ ensureRequiredParameterProvided("coreName", coreName);
+
+ final UpgradeCoreIndexResponse response =
+ instantiateJerseyResponse(UpgradeCoreIndexResponse.class);
+
+ return handlePotentiallyAsynchronousTask(
+ response,
+ coreName,
+ requestBody.async,
+ "upgrade-index",
+ () -> performUpgrade(coreName, requestBody, response));
+ }
+
+ private UpgradeCoreIndexResponse performUpgrade(
+ String coreName, UpgradeCoreIndexRequestBody requestBody, UpgradeCoreIndexResponse response) {
+
+ try (SolrCore core = coreContainer.getCore(coreName)) {
+
+ // Set LatestVersionMergePolicy to prevent older segments from
+ // participating in merges while we reindex. This is to prevent any older version
+ // segments from
+ // merging with any newly formed segments created due to reindexing and undoing the work
+ // we are doing.
+ RefCounted iwRef = null;
+ MergePolicy originalMergePolicy = null;
+ int numSegmentsEligibleForUpgrade = 0, numSegmentsUpgraded = 0;
+ try {
+ iwRef = core.getSolrCoreState().getIndexWriter(core);
+ IndexWriter iw = iwRef.get();
+
+ originalMergePolicy = iw.getConfig().getMergePolicy();
+ iw.getConfig()
+ .setMergePolicy(
+ new LatestVersionMergePolicy(
+ iw.getConfig().getMergePolicy())); // prevent older segments from merging
+
+ RefCounted searcherRef = core.getSearcher();
+ try {
+ List leafContexts = searcherRef.get().getIndexReader().leaves();
+ DocValuesIteratorCache dvICache = new DocValuesIteratorCache(searcherRef.get());
+
+ UpdateRequestProcessorChain updateProcessorChain =
+ getUpdateProcessorChain(core, requestBody.updateChain);
+
+ for (LeafReaderContext lrc : leafContexts) {
+ if (!shouldUpgradeSegment(lrc)) {
+ continue;
+ }
+ numSegmentsEligibleForUpgrade++;
+ processSegment(lrc, updateProcessorChain, core, searcherRef.get(), dvICache);
+ numSegmentsUpgraded++;
+ }
+
+ if (numSegmentsEligibleForUpgrade == 0) {
+ response.core = coreName;
+ response.upgradeStatus = CoreIndexUpgradeStatus.NO_UPGRADE_NEEDED.toString();
+ response.numSegmentsEligibleForUpgrade = 0;
+ return response;
+ }
+ } catch (Exception e) {
+ log.error("Error while processing core: [{}}]", coreName, e);
+ throw new CoreAdminAPIBaseException(e);
+ } finally {
+ // important to decrement searcher ref count after use since we obtained it via
+ // SolrCore.getSearcher()
+ searcherRef.decref();
+ }
+
+ try {
+ doCommit(core);
+ } catch (IOException e) {
+ throw new CoreAdminAPIBaseException(e);
+ }
+
+ boolean indexUpgraded = isIndexUpgraded(core);
+
+ if (!indexUpgraded) {
+ log.error(
+ "Validation failed for core '{}'. Some data is still present in the older (<{}.x) Lucene index format.",
+ coreName,
+ Version.LATEST.major);
+ throw new CoreAdminAPIBaseException(
+ new SolrException(
+ SolrException.ErrorCode.SERVER_ERROR,
+ "Validation failed for core '"
+ + coreName
+ + "'. Some data is still present in the older (<"
+ + Version.LATEST.major
+ + ".x) Lucene index format."));
+ }
+
+ response.core = coreName;
+ response.upgradeStatus = CoreIndexUpgradeStatus.UPGRADE_SUCCESSFUL.toString();
+ response.numSegmentsEligibleForUpgrade = numSegmentsEligibleForUpgrade;
+ response.numSegmentsUpgraded = numSegmentsUpgraded;
+ } catch (Exception ioEx) {
+ throw new CoreAdminAPIBaseException(ioEx);
+
+ } finally {
+ // Restore original merge policy
+ if (iwRef != null) {
+ IndexWriter iw = iwRef.get();
+ if (originalMergePolicy != null) {
+ iw.getConfig().setMergePolicy(originalMergePolicy);
+ }
+ iwRef.decref();
+ }
+ }
+ }
+
+ return response;
+ }
+
+ private boolean shouldUpgradeSegment(LeafReaderContext lrc) {
+ Version segmentMinVersion = null;
+
+ LeafReader leafReader = lrc.reader();
+ leafReader = FilterLeafReader.unwrap(leafReader);
+
+ SegmentCommitInfo si = ((SegmentReader) leafReader).getSegmentInfo();
+ segmentMinVersion = si.info.getMinVersion();
+
+ return (segmentMinVersion == null || segmentMinVersion.major < Version.LATEST.major);
+ }
+
+ @SuppressWarnings({"rawtypes"})
+ private UpdateRequestProcessorChain getUpdateProcessorChain(
+ SolrCore core, String requestedUpdateChain) {
+
+ // Try explicitly requested chain first
+ if (requestedUpdateChain != null) {
+ UpdateRequestProcessorChain resolvedChain =
+ core.getUpdateProcessingChain(requestedUpdateChain);
+ if (resolvedChain != null) {
+ return resolvedChain;
+ }
+ throw new SolrException(
+ BAD_REQUEST,
+ "Requested update chain '"
+ + requestedUpdateChain
+ + "' not found for core "
+ + core.getName());
+ }
+
+ // Try to find chain configured in /update handler
+ String updateChainName = null;
+ SolrRequestHandler reqHandler = core.getRequestHandler("/update");
+
+ NamedList initArgs = ((RequestHandlerBase) reqHandler).getInitArgs();
+
+ if (initArgs != null) {
+ // Check invariants first
+ Object invariants = initArgs.get("invariants");
+ if (invariants instanceof NamedList) {
+ updateChainName = (String) ((NamedList) invariants).get(UpdateParams.UPDATE_CHAIN);
+ }
+
+ // Check defaults if not found in invariants
+ if (updateChainName == null) {
+ Object defaults = initArgs.get("defaults");
+ if (defaults instanceof NamedList) {
+ updateChainName = (String) ((NamedList) defaults).get(UpdateParams.UPDATE_CHAIN);
+ }
+ }
+ }
+
+ // default chain is returned if updateChainName is null
+ return core.getUpdateProcessingChain(updateChainName);
+ }
+
+ private boolean isIndexUpgraded(SolrCore core) throws IOException {
+
+ Directory dir =
+ core.getDirectoryFactory()
+ .get(
+ core.getIndexDir(),
+ DirectoryFactory.DirContext.DEFAULT,
+ core.getSolrConfig().indexConfig.lockType);
+
+ try (IndexReader reader = DirectoryReader.open(dir)) {
+ List leaves = reader.leaves();
+ if (leaves == null || leaves.isEmpty()) {
+ // no segments to process/validate
+ return true;
+ }
+
+ for (LeafReaderContext lrc : leaves) {
+ LeafReader leafReader = lrc.reader();
+ leafReader = FilterLeafReader.unwrap(leafReader);
+ if (leafReader instanceof SegmentReader) {
+ SegmentReader segmentReader = (SegmentReader) leafReader;
+ SegmentCommitInfo si = segmentReader.getSegmentInfo();
+ Version segMinVersion = si.info.getMinVersion();
+ if (segMinVersion == null || segMinVersion.major != Version.LATEST.major) {
+ log.warn(
+ "isIndexUpgraded(): Core: {}, Segment [{}] is still at minVersion [{}] and is not updated to the latest version [{}]; numLiveDocs: [{}]",
+ core.getName(),
+ si.info.name,
+ (segMinVersion == null ? 6 : segMinVersion.major),
+ Version.LATEST.major,
+ segmentReader.numDocs());
+ return false;
+ }
+ }
+ }
+ return true;
+ } catch (Exception e) {
+ log.error("Error while opening segmentInfos for core [{}]", core.getName(), e);
+ throw e;
+ } finally {
+ if (dir != null) {
+ core.getDirectoryFactory().release(dir);
+ }
+ }
+ }
+
+ private void doCommit(SolrCore core) throws IOException {
+ try (LocalSolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams())) {
+ CommitUpdateCommand cmd = new CommitUpdateCommand(req, false); // optimize=false
+ core.getUpdateHandler().commit(cmd);
+ } catch (IOException ioEx) {
+ log.warn("Error committing on core [{}] during index upgrade", core.getName(), ioEx);
+ throw ioEx;
+ }
+ }
+
+ private void processSegment(
+ LeafReaderContext leafReaderContext,
+ UpdateRequestProcessorChain processorChain,
+ SolrCore core,
+ SolrIndexSearcher solrIndexSearcher,
+ DocValuesIteratorCache dvICache)
+ throws Exception {
+
+ String coreName = core.getName();
+ IndexSchema indexSchema = core.getLatestSchema();
+
+ LeafReader leafReader = leafReaderContext.reader();
+ Bits liveDocs = leafReader.getLiveDocs();
+ SolrDocumentFetcher docFetcher = solrIndexSearcher.getDocFetcher();
+
+ // Exclude copy field targets to avoid duplicating values on reindex
+ Set nonStoredDVFields = docFetcher.getNonStoredDVsWithoutCopyTargets();
+
+ try (LocalSolrQueryRequest solrRequest =
+ new LocalSolrQueryRequest(core, new ModifiableSolrParams())) {
+ SolrQueryResponse rsp = new SolrQueryResponse();
+ UpdateRequestProcessor processor = processorChain.createProcessor(solrRequest, rsp);
+ try {
+ StoredFields storedFields = leafReader.storedFields();
+ for (int luceneDocId = 0; luceneDocId < leafReader.maxDoc(); luceneDocId++) {
+ if (liveDocs != null && !liveDocs.get(luceneDocId)) {
+ continue;
+ }
+
+ Document doc = storedFields.document(luceneDocId);
+ SolrInputDocument solrDoc = toSolrInputDocument(doc, indexSchema);
+
+ docFetcher.decorateDocValueFields(
+ solrDoc, leafReaderContext.docBase + luceneDocId, nonStoredDVFields, dvICache);
+ solrDoc.removeField("_version_");
+ AddUpdateCommand currDocCmd = new AddUpdateCommand(solrRequest);
+ currDocCmd.solrDoc = solrDoc;
+ processor.processAdd(currDocCmd);
+ }
+ } finally {
+ // finish() must be called before close() to flush pending operations
+ processor.finish();
+ processor.close();
+ }
+ }
+ }
+
+ /** Convert a lucene Document to a SolrInputDocument */
+ protected SolrInputDocument toSolrInputDocument(
+ org.apache.lucene.document.Document doc, IndexSchema schema) {
+ SolrInputDocument out = new SolrInputDocument();
+ for (IndexableField f : doc.getFields()) {
+ String fname = f.name();
+ SchemaField sf = schema.getFieldOrNull(f.name());
+ Object val = null;
+ if (sf != null) {
+ if ((!sf.hasDocValues() && !sf.stored()) || schema.isCopyFieldTarget(sf)) {
+ continue;
+ }
+ val = sf.getType().toObject(f);
+ } else {
+ val = f.stringValue();
+ if (val == null) {
+ val = f.numericValue();
+ }
+ if (val == null) {
+ val = f.binaryValue();
+ }
+ if (val == null) {
+ val = f;
+ }
+ }
+ out.addField(fname, val);
+ }
+ return out;
+ }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/UpgradeCoreIndexActionTest.java b/solr/core/src/test/org/apache/solr/handler/admin/UpgradeCoreIndexActionTest.java
new file mode 100644
index 000000000000..ad5a8928bae0
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/admin/UpgradeCoreIndexActionTest.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.solr.handler.admin;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.util.Version;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.CommonAdminParams;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.RefCounted;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class UpgradeCoreIndexActionTest extends SolrTestCaseJ4 {
+ private static final int DOCS_PER_SEGMENT = 3;
+ private static final String DV_FIELD = "dvonly_i_dvo";
+
+ private static VarHandle segmentInfoMinVersionHandle;
+
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ initCore("solrconfig-nomergepolicyfactory.xml", "schema.xml");
+ segmentInfoMinVersionHandle =
+ MethodHandles.privateLookupIn(SegmentInfo.class, MethodHandles.lookup())
+ .findVarHandle(SegmentInfo.class, "minVersion", Version.class);
+ }
+
+ @Before
+ public void resetIndex() {
+ assertU(delQ("*:*"));
+ assertU(commit("openSearcher", "true"));
+ }
+
+ @Test
+ public void testUpgradeCoreIndexSelectiveReindexDeletesOldSegments() throws Exception {
+ final SolrCore core = h.getCore();
+ final String coreName = core.getName();
+
+ final SegmentLayout layout = buildThreeSegments(coreName);
+ final Version simulatedOldMinVersion = Version.fromBits(Version.LATEST.major - 1, 0, 0);
+
+ // Simulate:
+ // - seg1: "pure 9x" (minVersion=9)
+ // - seg2: "pure 10x" (minVersion=10)
+ // - seg3: "minVersion 9x, version 10x" (merged segment; minVersion=9)
+ setMinVersionForSegments(core, Set.of(layout.seg1, layout.seg3), simulatedOldMinVersion);
+
+ final Set segmentsBeforeUpgrade = listSegmentNames(core);
+
+ CoreAdminHandler admin = new CoreAdminHandler(h.getCoreContainer());
+ try {
+ final SolrQueryResponse resp = new SolrQueryResponse();
+ admin.handleRequestBody(
+ req(
+ CoreAdminParams.ACTION,
+ CoreAdminParams.CoreAdminAction.UPGRADECOREINDEX.toString(),
+ CoreAdminParams.CORE,
+ coreName),
+ resp);
+
+ assertNull("Unexpected exception: " + resp.getException(), resp.getException());
+ assertEquals(coreName, resp.getValues().get("core"));
+ assertEquals(2, resp.getValues().get("numSegmentsEligibleForUpgrade"));
+ assertEquals(2, resp.getValues().get("numSegmentsUpgraded"));
+ assertEquals("UPGRADE_SUCCESSFUL", resp.getValues().get("upgradeStatus"));
+ } finally {
+ admin.shutdown();
+ admin.close();
+ }
+
+ // The action commits internally and reopens the searcher; verify segments on disk.
+ final Set segmentsAfter = listSegmentNames(core);
+ final Set newSegments = new HashSet<>(segmentsAfter);
+ newSegments.removeAll(segmentsBeforeUpgrade);
+ assertFalse(
+ "Expected at least one new segment to be created by reindexing", newSegments.isEmpty());
+ assertTrue("Expected seg2 to remain", segmentsAfter.contains(layout.seg2));
+ assertFalse("Expected seg1 to be dropped", segmentsAfter.contains(layout.seg1));
+ assertFalse("Expected seg3 to be dropped", segmentsAfter.contains(layout.seg3));
+
+ // Searcher was reopened by the action's commit; verify document count and field values.
+ assertQ(req("q", "*:*"), "//result[@numFound='" + (3 * DOCS_PER_SEGMENT) + "']");
+
+ // Validate docValues-only (non-stored) fields were preserved for reindexed documents.
+ // seg1 and seg3 were reindexed; seg2 was not.
+ assertDocValuesOnlyFieldPreserved();
+ }
+
+ @Test
+ @SuppressWarnings({"unchecked"})
+ public void testUpgradeCoreIndexAsyncRequestStatusContainsOperationResponse() throws Exception {
+ final SolrCore core = h.getCore();
+ final String coreName = core.getName();
+
+ final SegmentLayout layout = buildThreeSegments(coreName);
+ final Version simulatedOldMinVersion = Version.fromBits(Version.LATEST.major - 1, 0, 0);
+ setMinVersionForSegments(core, Set.of(layout.seg1, layout.seg3), simulatedOldMinVersion);
+
+ final Set segmentsBeforeUpgrade = listSegmentNames(core);
+
+ final String requestId = "upgradecoreindex_async_1";
+ CoreAdminHandler admin = new CoreAdminHandler(h.getCoreContainer());
+ try {
+ SolrQueryResponse submitResp = new SolrQueryResponse();
+ admin.handleRequestBody(
+ req(
+ CoreAdminParams.ACTION,
+ CoreAdminParams.CoreAdminAction.UPGRADECOREINDEX.toString(),
+ CoreAdminParams.CORE,
+ coreName,
+ CommonAdminParams.ASYNC,
+ requestId),
+ submitResp);
+ assertNull(submitResp.getException());
+
+ SolrQueryResponse statusResp = new SolrQueryResponse();
+ int maxRetries = 60;
+ while (maxRetries-- > 0) {
+ statusResp = new SolrQueryResponse();
+ admin.handleRequestBody(
+ req(
+ CoreAdminParams.ACTION,
+ CoreAdminParams.CoreAdminAction.REQUESTSTATUS.toString(),
+ CoreAdminParams.REQUESTID,
+ requestId),
+ statusResp);
+
+ if ("completed".equals(statusResp.getValues().get("STATUS"))) {
+ break;
+ }
+ Thread.sleep(250);
+ }
+
+ assertEquals("completed", statusResp.getValues().get("STATUS"));
+ Object opResponse = statusResp.getValues().get("response");
+ assertNotNull(opResponse);
+ assertTrue("Expected map response, got: " + opResponse.getClass(), opResponse instanceof Map);
+
+ Map opResponseMap = (Map) opResponse;
+ assertEquals(coreName, opResponseMap.get("core"));
+ assertEquals(2, ((Number) opResponseMap.get("numSegmentsEligibleForUpgrade")).intValue());
+ assertEquals(2, ((Number) opResponseMap.get("numSegmentsUpgraded")).intValue());
+ assertEquals("UPGRADE_SUCCESSFUL", opResponseMap.get("upgradeStatus"));
+ } finally {
+ admin.shutdown();
+ admin.close();
+ }
+
+ final Set segmentsAfter = listSegmentNames(core);
+ final Set newSegments = new HashSet<>(segmentsAfter);
+ newSegments.removeAll(segmentsBeforeUpgrade);
+ assertFalse(
+ "Expected at least one new segment to be created by reindexing", newSegments.isEmpty());
+ assertTrue("Expected seg2 to remain", segmentsAfter.contains(layout.seg2));
+ assertFalse("Expected seg1 to be dropped", segmentsAfter.contains(layout.seg1));
+ assertFalse("Expected seg3 to be dropped", segmentsAfter.contains(layout.seg3));
+
+ // Validate docValues-only (non-stored) fields were preserved for reindexed documents.
+ assertDocValuesOnlyFieldPreserved();
+ }
+
+ @Test
+ public void testNoUpgradeNeededWhenAllSegmentsCurrent() throws Exception {
+ final SolrCore core = h.getCore();
+ final String coreName = core.getName();
+
+ // Index documents and commit - all segments will be at the current Lucene version
+ for (int i = 0; i < DOCS_PER_SEGMENT; i++) {
+ assertU(adoc("id", Integer.toString(i)));
+ }
+ assertU(commit("openSearcher", "true"));
+
+ final Set segmentsBefore = listSegmentNames(core);
+ assertFalse("Expected at least one segment", segmentsBefore.isEmpty());
+
+ CoreAdminHandler admin = new CoreAdminHandler(h.getCoreContainer());
+ try {
+ final SolrQueryResponse resp = new SolrQueryResponse();
+ admin.handleRequestBody(
+ req(
+ CoreAdminParams.ACTION,
+ CoreAdminParams.CoreAdminAction.UPGRADECOREINDEX.toString(),
+ CoreAdminParams.CORE,
+ coreName),
+ resp);
+
+ assertNull("Unexpected exception: " + resp.getException(), resp.getException());
+ assertEquals(coreName, resp.getValues().get("core"));
+ assertEquals(0, resp.getValues().get("numSegmentsEligibleForUpgrade"));
+ assertEquals("NO_UPGRADE_NEEDED", resp.getValues().get("upgradeStatus"));
+ } finally {
+ admin.shutdown();
+ admin.close();
+ }
+
+ // Verify no segments were modified
+ final Set segmentsAfter = listSegmentNames(core);
+ assertEquals("Segments should remain unchanged", segmentsBefore, segmentsAfter);
+
+ // Verify documents are still queryable
+ assertQ(req("q", "*:*"), "//result[@numFound='" + DOCS_PER_SEGMENT + "']");
+ }
+
+ private SegmentLayout buildThreeSegments(String coreName) throws Exception {
+ final SolrCore core = h.getCore();
+
+ Set segmentsBefore = listSegmentNames(core);
+ indexDocs(0);
+ final String seg1 = commitAndGetNewSegment(core, segmentsBefore);
+ segmentsBefore = listSegmentNames(core);
+
+ indexDocs(1000);
+ final String seg2 = commitAndGetNewSegment(core, segmentsBefore);
+ segmentsBefore = listSegmentNames(core);
+
+ indexDocs(2000);
+ final String seg3 = commitAndGetNewSegment(core, segmentsBefore);
+
+ Set allSegments = listSegmentNames(core);
+ assertTrue(allSegments.contains(seg1));
+ assertTrue(allSegments.contains(seg2));
+ assertTrue(allSegments.contains(seg3));
+
+ return new SegmentLayout(coreName, seg1, seg2, seg3);
+ }
+
+ private void indexDocs(int baseId) {
+ for (int i = 0; i < DOCS_PER_SEGMENT; i++) {
+ // schema.xml copies id into numeric fields; use numeric IDs to avoid parsing errors
+ final String id = Integer.toString(baseId + i);
+ assertU(adoc("id", id, DV_FIELD, Integer.toString(baseId + i + 10_000), "title", "t" + id));
+ }
+ }
+
+ private void assertDocValuesOnlyFieldPreserved() {
+ // Assert one doc that must have been reindexed (seg1) and one from seg3.
+ assertDocHasDvFieldValue(0, 10_000);
+ assertDocHasDvFieldValue(2000, 12_000);
+
+ // Also sanity-check a doc from the untouched segment (seg2) still has its value.
+ assertDocHasDvFieldValue(1000, 11_000);
+ }
+
+ private void assertDocHasDvFieldValue(int id, int expected) {
+ assertQ(
+ req("q", "id:" + id, "fl", "id," + DV_FIELD),
+ "//result[@numFound='1']",
+ "//result/doc/int[@name='" + DV_FIELD + "'][.='" + expected + "']");
+ }
+
+ private String commitAndGetNewSegment(SolrCore core, Set segmentsBefore)
+ throws Exception {
+ assertU(commit("openSearcher", "true"));
+ Set segmentsAfter = new HashSet<>(listSegmentNames(core));
+ segmentsAfter.removeAll(new HashSet<>(segmentsBefore));
+ assertEquals("Expected exactly one new segment", 1, segmentsAfter.size());
+ return segmentsAfter.iterator().next();
+ }
+
+ private Set listSegmentNames(SolrCore core) {
+ RefCounted searcherRef = core.getSearcher();
+ try {
+ final Set segmentNames = new HashSet<>();
+ for (LeafReaderContext ctx : searcherRef.get().getTopReaderContext().leaves()) {
+ SegmentReader segmentReader = (SegmentReader) FilterLeafReader.unwrap(ctx.reader());
+ segmentNames.add(segmentReader.getSegmentName());
+ }
+ return segmentNames;
+ } finally {
+ searcherRef.decref();
+ }
+ }
+
+ private void setMinVersionForSegments(SolrCore core, Set segments, Version minVersion)
+ throws Exception {
+ RefCounted searcherRef = core.getSearcher();
+ try {
+ final List leaves = searcherRef.get().getTopReaderContext().leaves();
+ for (LeafReaderContext ctx : leaves) {
+ SegmentReader segmentReader = (SegmentReader) FilterLeafReader.unwrap(ctx.reader());
+ if (!segments.contains(segmentReader.getSegmentName())) {
+ continue;
+ }
+ final SegmentInfo segmentInfo = segmentReader.getSegmentInfo().info;
+ segmentInfoMinVersionHandle.set(segmentInfo, minVersion);
+ }
+ } finally {
+ searcherRef.decref();
+ }
+ }
+
+ private record SegmentLayout(String coreName, String seg1, String seg2, String seg3) {}
+}
diff --git a/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc b/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
index 656310886cf9..4e0bb0af92d6 100644
--- a/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
+++ b/solr/solr-ref-guide/modules/configuration-guide/pages/coreadmin-api.adoc
@@ -780,6 +780,93 @@ This command is used as part of SolrCloud's xref:deployment-guide:shard-manageme
When used against a core in a user-managed cluster without `split.key` parameter, this action will split the source index and distribute its documents alternately so that each split piece contains an equal number of documents.
If the `split.key` parameter is specified then only documents having the same route key will be split from the source index.
+[[coreadmin-upgradecoreindex]]
+== UPGRADECOREINDEX
+
+The `UPGRADECOREINDEX` action upgrades an existing core's index in-place after a Solr major-version upgrade by reindexing documents from older-format segments.
+It temporarily adjusts merge behavior to keep older Lucene version segments from being merged with new segments while the operation reindexes all live documents from older-format segments, then commits and validates the result.
+If a core is upgraded by this action, it ensures index compatibility with the next Solr major version (upon a future Solr upgrade) without having to re-create the index from source.
+
+This action is expensive and is intended for user-managed clusters or single-node installations only. It is currently not supported in SolrCloud mode.
+
+[NOTE]
+====
+Only fields that are stored or have DocValues enabled can be preserved during upgrade.
+Fields that are neither stored nor docValues-backed will lose their data. However if a field is neither stored, nor docValues-enabled, but is a copyField target, it will be re-populated by the copy field mechanism during reindexing.
+Always test on a copy and ensure you have a backup before running on production data.
+====
+
+=== UPGRADECOREINDEX Parameters
+
+`core`::
++
+[%autowidth,frame=none]
+|===
+s|Required |Default: none
+|===
++
+The name of the core whose index should be upgraded.
+
+`async`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: none
+|===
++
+Request ID to track this action which will be processed asynchronously.
+Use <> with the provided `requestid` to poll for completion and retrieve the operation response.
+
+`update.chain`::
++
+[%autowidth,frame=none]
+|===
+|Optional |Default: none
+|===
++
+The update processor chain to use for reindexing.
+If omitted, Solr uses the chain configured for the `/update` handler, or the default update chain for the core (in that order).
+
+=== UPGRADECOREINDEX Response
+
+On success, the response includes:
+
+`core`::
+The core name.
+
+`numSegmentsEligibleForUpgrade`::
+The number of segments with an older Lucene format that were targeted.
+
+`numSegmentsUpgraded`::
+The number of segments successfully processed.
+
+`upgradeStatus`::
+One of `UPGRADE_SUCCESSFUL` or `NO_UPGRADE_NEEDED`.
+On failure, an exception is thrown with error details.
+
+=== UPGRADECOREINDEX Examples
+
+*Synchronous:*
+
+[source,bash]
+----
+http://localhost:8983/solr/admin/cores?action=UPGRADECOREINDEX&core=techproducts
+----
+
+*Asynchronous (recommended for large cores):*
+
+[source,bash]
+----
+http://localhost:8983/solr/admin/cores?action=UPGRADECOREINDEX&core=techproducts&async=upgrade_1
+----
+
+Then poll status:
+
+[source,bash]
+----
+http://localhost:8983/solr/admin/cores?action=REQUESTSTATUS&requestid=upgrade_1
+----
+
[[coreadmin-requeststatus]]
== REQUESTSTATUS
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
index ea2206421240..ad41867cc31b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CoreAdminParams.java
@@ -178,7 +178,8 @@ public enum CoreAdminAction {
INSTALLCOREDATA,
CREATESNAPSHOT,
DELETESNAPSHOT,
- LISTSNAPSHOTS;
+ LISTSNAPSHOTS,
+ UPGRADECOREINDEX;
public final boolean isRead;