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: + * + *

+ * + * @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;