You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2020/06/03 19:18:20 UTC
[lucene-solr] branch jira/lucene-8962 updated (107fc17 -> cab5ef5)
This is an automated email from the ASF dual-hosted git repository.
sokolov pushed a change to branch jira/lucene-8962
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.
discard 107fc17 Fix case where mergeOnCommit would attempt to delete files twice in the presence of deletions
discard 4e7c17e Revert "Revert "LUCENE-8962""
discard cacdc86 SOLR-14517 Obey "mm" local param on edismax queries with operators (#1540)
discard b20ceb2 LUCENE-9301: include build time and user name only in non-snapshot builds so that jars are not recompiled on each build in development.
discard 270bdc1 SOLR-14491: Intercepting internode requests in KerberosPlugin when HTTP/2 client is used
discard 545dcc1 SOLR-14519:Fix or suppress warnings in solr/cloud/autoscaling
discard 0f545d7 SOLR-14494: Refactor BlockJoin to not use Filter (#1523)
discard 793c1a1c LUCENE-9359: Address test failures when the codec version gets modified.
discard 2a224cb LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)
discard 8cfa6a0 Revert "LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)"
discard daf7160 LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)
discard 2b7d278 ref_guide - metrics reporting - small typo (#1544)
discard 6e2cdcc SOLR-14419: adding {param:ref} to Query DSL
discard d7762e9 SOLR-14511: Documented node.sysprop shard preference (#1536)
discard f833cb5 SOLR-11934: REVERT addition of collection to log message This reverts commit e4dc9e94
discard de15321 SOLR-14498: BlockCache gets stuck not accepting new stores. Fix gradle check
discard 8fc28ef SOLR-14237: Fix an error on admin UI due to improper variable handling
discard cebb441 SOLR-14498: BlockCache gets stuck not accepting new stores fixing checksums
discard abd1f6a SOLR-14498: BlockCache gets stuck not accepting new stores. Fix gradle :solr:core:validateJarChecksums
discard 607cc54 Add back-compat indices for 8.5.2
discard 3e00bf9 Add bugfix version 8.5.2
discard 3fe52dd LUCENE-9380: Fix auxiliary class warnings in Lucene
discard 09fa2a1 SOLR-14474: Fix remaining auxilliary class warnings in Solr
discard b93a1cd DOAP changes for release 8.5.2
discard aa78758 SOLR-14280: SolrConfig error handling improvements
discard 3731d71 SOLR-14498: Upgrade to Caffeine 2.8.4, which fixes the cache poisoning issue.
discard cf98a61 SOLR-14495: Fix or suppress warnings in solr/search/function
discard 8e475cc SOLR-14443: Make SolrLogPostTool resilient to odd requests (#1525)
discard 1ef077f LUCENE-9330: Make SortFields responsible for index sorting and serialization (#1440)
discard a00d7eb SOLR-13325: Add a collection selector to ComputePlanAction (#1512)
discard 57456a9 SOLR-14461: Replace commons-fileupload with Jetty (#1490)
discard 631f4a9 SOLR-13289: Add Refguide changes (#1501)
discard ea36cb5 SOLR-13289: Rename minExactHits to minExactCount (#1511)
discard dfa3a96 SOLR-13289: Use the final collector's scoreMode (#1517)
discard aec740d SOLR-14504: ZkController LiveNodesListener has NullPointerException in startup race.
discard 17592d2 LUCENE-9376: Fix or suppress 20 resource leak precommit warnings in lucene/search
discard 63d9cff SOLR-14482: Fix or suppress warnings in solr/search/facet
discard 46b617e SOLR-14492: Fix ArrayIndexOutOfBoundsException in json.facet 'terms' when FacetFieldProcessorByHashDV is used with aggregations over multivalued numeric fields
discard d00f79c LUCENE-9374: Add checkBrokenLinks gradle task (#1522)
discard cd6b9b9 SOLR-14484: avoid putting null into MDC Co-authored-by: Andras Salamon
discard 51c8e07 SOLR-14486: Autoscaling simulation framework should stop using /clusterstate.json.
discard 8f2fc5e Lucene-9371: Allow external access to RegExp's parsed structure (#1521)
discard d68673d SOLR-14476: Fix precommit
discard dd08642 SOLR-14476: Add percentiles and standard deviation aggregations to stats, facet and timeseries Streaming Expressions
discard e0e50b9 SOLR-14472: missed CHANGES.txt
discard 648e068 SOLR-14472: Autoscale "cores": use metrics to count Also counts all cores (lazy, transient), although currently impossible to use these in SolrCloud.
discard 3cb8ebb SOLR-14473: Improve Overseer Javadoc (#1510)
add d282528 SOLR-14473: Improve Overseer Javadoc (#1510)
add d3d0df9 SOLR-14472: Autoscale "cores": use metrics to count Also counts all cores (lazy, transient), although currently impossible to use these in SolrCloud.
add 9259f00 SOLR-14472: missed CHANGES.txt
add 3b09d30 Merge branch 'master' of github.com:apache/lucene-solr
add 16aad55 SOLR-14476: Add percentiles and standard deviation aggregations to stats, facet and timeseries Streaming Expressions
add a795047 SOLR-14476: Fix precommit
add 44fc5b9 Lucene-9371: Allow external access to RegExp's parsed structure (#1521)
add 57b7d8a SOLR-14486: Autoscaling simulation framework should stop using /clusterstate.json.
add 2ac640f SOLR-14484: avoid putting null into MDC Co-authored-by: Andras Salamon
add 84ea0cb LUCENE-9374: Add checkBrokenLinks gradle task (#1522)
add 28209cb SOLR-14492: Fix ArrayIndexOutOfBoundsException in json.facet 'terms' when FacetFieldProcessorByHashDV is used with aggregations over multivalued numeric fields
add 9c066f6 SOLR-14482: Fix or suppress warnings in solr/search/facet
add 21b08d5 LUCENE-9376: Fix or suppress 20 resource leak precommit warnings in lucene/search
add 0728ef0 SOLR-14504: ZkController LiveNodesListener has NullPointerException in startup race.
add 5e9483e SOLR-13289: Use the final collector's scoreMode (#1517)
add 3ca7628 SOLR-13289: Rename minExactHits to minExactCount (#1511)
add 16a22fc SOLR-13289: Add Refguide changes (#1501)
add 3db79ed Merge branch 'master' of https://gitbox.apache.org/repos/asf/lucene-solr
add 3fba3daa SOLR-14461: Replace commons-fileupload with Jetty (#1490)
add 338671e SOLR-13325: Add a collection selector to ComputePlanAction (#1512)
add de2bad9 LUCENE-9330: Make SortFields responsible for index sorting and serialization (#1440)
add 78f4a5b SOLR-14443: Make SolrLogPostTool resilient to odd requests (#1525)
add 675956c SOLR-14495: Fix or suppress warnings in solr/search/function
add 22044fc SOLR-14498: Upgrade to Caffeine 2.8.4, which fixes the cache poisoning issue.
add 46ca768 SOLR-14280: SolrConfig error handling improvements
add ac80fb9 DOAP changes for release 8.5.2
add 07a9b5d SOLR-14474: Fix remaining auxilliary class warnings in Solr
add b576ef6 LUCENE-9380: Fix auxiliary class warnings in Lucene
add a240f0b Add bugfix version 8.5.2
add 18519f3 Add back-compat indices for 8.5.2
add 598cbc5 SOLR-14498: BlockCache gets stuck not accepting new stores. Fix gradle :solr:core:validateJarChecksums
add 84c5dfc SOLR-14498: BlockCache gets stuck not accepting new stores fixing checksums
add 679cebe SOLR-14237: Fix an error on admin UI due to improper variable handling
add 724709c SOLR-14498: BlockCache gets stuck not accepting new stores. Fix gradle check
add 596c64a SOLR-11934: REVERT addition of collection to log message This reverts commit e4dc9e94
add 5251c40 SOLR-14511: Documented node.sysprop shard preference (#1536)
add 2c8cfa6 SOLR-14419: adding {param:ref} to Query DSL
add 90d84d4 ref_guide - metrics reporting - small typo (#1544)
add bfb6bf9 LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)
add fe07d9d Revert "LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)"
add f908f2c LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)
add 22cb4d4 LUCENE-9359: Address test failures when the codec version gets modified.
add a6a02ac SOLR-14494: Refactor BlockJoin to not use Filter (#1523)
add e1cc7a7 SOLR-14519:Fix or suppress warnings in solr/cloud/autoscaling
add 1dda684 SOLR-14491: Intercepting internode requests in KerberosPlugin when HTTP/2 client is used
add da3dbb1 LUCENE-9301: include build time and user name only in non-snapshot builds so that jars are not recompiled on each build in development.
add cb7e948 SOLR-14517 Obey "mm" local param on edismax queries with operators (#1540)
add e841d76 SOLR-14525 For components loaded from packages SolrCoreAware, ResourceLoaderAware are not honored (#1547)
add 552f194 SOLR-14525: Test failure
add 502f62c SOLR-14531: Refactor out internode requests from HttpShardHandler
add 3e8e5d8 LUCENE-9382: update gradle to 6.4.1. (#1549)
add f59f8fd Add license file to ShardRequestor.
add db9cd0c SOLR-14526: fix or suppress warnings in apache/solr/core
add 549c42d Ref Guide: Add docs for updating urlscheme on existing collections; reorg page; fix hover color on tabs
add a06f57c SOLR: Use absolute paths for server paths. (#1546)
add 2af82c8 SOLR-11334: Split some field lists better Used by HighlightComponent and TermVectorComponent Used to produce an empty string on comma-space, leading to an exception.
add fb58f43 SOLR-14520: Fixed server errors from the json.facet allBuckets:true option when combined with refine:true
add 90039fc SOLR-14476: Add percentiles and standard deviation aggregations to stats, facet and timeseries Streaming Expressions
new 642e56a Revert "Revert "LUCENE-8962""
new cab5ef5 Fix case where mergeOnCommit would attempt to delete files twice in the presence of deletions
This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version. This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:
* -- * -- B -- O -- O -- O (107fc17)
\
N -- N -- N refs/heads/jira/lucene-8962 (cab5ef5)
You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.
Any revisions marked "omit" are not gone; other references still
refer to them. Any revisions marked "discard" are gone forever.
The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails. The revisions
listed as "add" were already present in the repository and have only
been added to this reference.
Summary of changes:
gradle/validation/check-environment.gradle | 2 +-
gradle/validation/validate-log-calls.gradle | 41 ++--
gradle/wrapper/gradle-wrapper.jar | Bin 58702 -> 58910 bytes
gradle/wrapper/gradle-wrapper.jar.sha256 | 2 +-
gradle/wrapper/gradle-wrapper.jar.version | 2 +-
gradle/wrapper/gradle-wrapper.properties | 2 +-
solr/CHANGES.txt | 13 ++
.../solr/core/AbstractSolrEventListener.java | 5 +-
.../java/org/apache/solr/core/BlobRepository.java | 9 +-
.../apache/solr/core/CachingDirectoryFactory.java | 4 +-
.../java/org/apache/solr/core/CodecFactory.java | 2 +-
.../java/org/apache/solr/core/ConfigOverlay.java | 14 ++
.../src/java/org/apache/solr/core/ConfigSet.java | 3 +
.../org/apache/solr/core/ConfigSetProperties.java | 2 +
.../org/apache/solr/core/ConfigSetService.java | 3 +
.../java/org/apache/solr/core/CoreContainer.java | 13 +-
.../java/org/apache/solr/core/CoreDescriptor.java | 6 +-
.../apache/solr/core/CorePropertiesLocator.java | 6 +-
.../org/apache/solr/core/DirectoryFactory.java | 19 +-
.../org/apache/solr/core/HdfsDirectoryFactory.java | 2 +-
.../solr/core/IndexDeletionPolicyWrapper.java | 4 +-
.../org/apache/solr/core/IndexReaderFactory.java | 2 +-
.../src/java/org/apache/solr/core/InitParams.java | 12 +-
.../org/apache/solr/core/MMapDirectoryFactory.java | 1 +
.../java/org/apache/solr/core/MemClassLoader.java | 5 +-
.../solr/core/NRTCachingDirectoryFactory.java | 1 +
.../src/java/org/apache/solr/core/NodeConfig.java | 20 +-
.../src/java/org/apache/solr/core/PluginBag.java | 27 ++-
.../src/java/org/apache/solr/core/PluginInfo.java | 6 +-
.../org/apache/solr/core/QuerySenderListener.java | 1 +
.../java/org/apache/solr/core/RequestParams.java | 22 +-
.../org/apache/solr/core/SchemaCodecFactory.java | 1 +
.../apache/solr/core/SimpleTextCodecFactory.java | 1 +
.../src/java/org/apache/solr/core/SolrConfig.java | 7 +
.../src/java/org/apache/solr/core/SolrCore.java | 20 +-
.../org/apache/solr/core/SolrDeletionPolicy.java | 2 +-
.../org/apache/solr/core/SolrResourceLoader.java | 82 ++++---
.../solr/core/TransientSolrCoreCacheDefault.java | 6 +-
.../backup/repository/HdfsBackupRepository.java | 1 +
.../repository/LocalFileSystemRepository.java | 3 +-
.../solr/core/snapshots/SolrSnapshotManager.java | 4 +-
.../solr/core/snapshots/SolrSnapshotsTool.java | 1 +
.../solr/handler/admin/CoreAdminHandler.java | 2 +-
.../solr/handler/admin/CoreAdminOperation.java | 4 +-
.../solr/handler/component/HttpShardHandler.java | 140 +-----------
.../solr/handler/component/ShardRequestor.java | 178 +++++++++++++++
.../java/org/apache/solr/pkg/PackageListeners.java | 4 +-
.../java/org/apache/solr/pkg/PackageLoader.java | 41 +++-
.../org/apache/solr/pkg/PackagePluginHolder.java | 34 ++-
.../search/facet/FacetFieldProcessorByArray.java | 4 +
.../java/org/apache/solr/search/facet/SlotAcc.java | 57 +++++
.../java/org/apache/solr/update/SolrCoreState.java | 3 +-
.../java/org/apache/solr/util/SolrPluginUtils.java | 16 +-
.../org/apache/solr/core/TestCoreDiscovery.java | 2 +-
.../org/apache/solr/highlight/HighlighterTest.java | 18 ++
.../src/test/org/apache/solr/pkg/TestPackages.java | 146 +++++++++----
.../solr/search/facet/TestJsonFacetRefinement.java | 19 +-
.../src/common-query-parameters.adoc | 4 +-
solr/solr-ref-guide/src/css/decoration.css | 2 +-
solr/solr-ref-guide/src/css/ref-guide.css | 6 +-
solr/solr-ref-guide/src/enabling-ssl.adoc | 238 ++++++++++-----------
61 files changed, 841 insertions(+), 456 deletions(-)
create mode 100644 solr/core/src/java/org/apache/solr/handler/component/ShardRequestor.java
[lucene-solr] 01/02: Revert "Revert "LUCENE-8962""
Posted by so...@apache.org.
This is an automated email from the ASF dual-hosted git repository.
sokolov pushed a commit to branch jira/lucene-8962
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit 642e56aba85947ba280e30917b61bdf605c0aa51
Author: Michael Sokolov <so...@amazon.com>
AuthorDate: Mon Jun 1 14:45:00 2020 -0400
Revert "Revert "LUCENE-8962""
This reverts commit 4501b3d3fdbc35af99bde6abe7432cfc5e8b5547.
This reverts commit 075adac59865b3277adcf86052f2fae3e6d11135.
---
lucene/CHANGES.txt | 2 +
.../org/apache/lucene/index/FilterMergePolicy.java | 5 +
.../java/org/apache/lucene/index/IndexWriter.java | 114 ++++++++++++++++++++-
.../org/apache/lucene/index/IndexWriterConfig.java | 29 ++++++
.../org/apache/lucene/index/IndexWriterEvents.java | 57 +++++++++++
.../apache/lucene/index/LiveIndexWriterConfig.java | 26 +++++
.../java/org/apache/lucene/index/MergePolicy.java | 28 ++++-
.../java/org/apache/lucene/index/MergeTrigger.java | 7 +-
.../org/apache/lucene/index/NoMergePolicy.java | 3 +
.../lucene/index/OneMergeWrappingMergePolicy.java | 5 +
.../lucene/index/TestIndexWriterMergePolicy.java | 70 ++++++++++++-
.../apache/lucene/index/MockRandomMergePolicy.java | 32 ++++++
.../org/apache/lucene/util/LuceneTestCase.java | 1 +
13 files changed, 373 insertions(+), 6 deletions(-)
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 50b7f7b..6e63ad7 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -376,6 +376,8 @@ Improvements
* LUCENE-9253: KoreanTokenizer now supports custom dictionaries(system, unknown). (Namgyu Kim)
+* LUCENE-8962: Add ability to selectively merge on commit (Michael Froh)
+
* LUCENE-9171: QueryBuilder can now use BoostAttributes on input token streams to selectively
boost particular terms or synonyms in parsed queries. (Alessandro Benedetti, Alan Woodward)
diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
index eb634b4..b4e33f8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FilterMergePolicy.java
@@ -58,6 +58,11 @@ public class FilterMergePolicy extends MergePolicy {
}
@Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+ return in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext);
+ }
+
+ @Override
public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext)
throws IOException {
return in.useCompoundFile(infos, mergedInfo, mergeContext);
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 52adbef..88fdb90 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -33,6 +33,8 @@ import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -3152,6 +3154,42 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
}
}
+ private MergePolicy.OneMerge updateSegmentInfosOnMergeFinish(MergePolicy.OneMerge merge, final SegmentInfos toCommit,
+ AtomicReference<CountDownLatch> mergeLatchRef) {
+ return new MergePolicy.OneMerge(merge.segments) {
+ public void mergeFinished() throws IOException {
+ super.mergeFinished();
+ CountDownLatch mergeAwaitLatch = mergeLatchRef.get();
+ if (mergeAwaitLatch == null) {
+ // Commit thread timed out waiting for this merge and moved on. No need to manipulate toCommit.
+ return;
+ }
+ if (committed) {
+ deleter.incRef(this.info.files());
+ // Resolve "live" SegmentInfos segments to their toCommit cloned equivalents, based on segment name.
+ Set<String> mergedSegmentNames = new HashSet<>();
+ for (SegmentCommitInfo sci : this.segments) {
+ deleter.decRef(sci.files());
+ mergedSegmentNames.add(sci.info.name);
+ }
+ List<SegmentCommitInfo> toCommitMergedAwaySegments = new ArrayList<>();
+ for (SegmentCommitInfo sci : toCommit) {
+ if (mergedSegmentNames.contains(sci.info.name)) {
+ toCommitMergedAwaySegments.add(sci);
+ }
+ }
+ // Construct a OneMerge that applies to toCommit
+ MergePolicy.OneMerge applicableMerge = new MergePolicy.OneMerge(toCommitMergedAwaySegments);
+ applicableMerge.info = this.info.clone();
+ long segmentCounter = Long.parseLong(this.info.info.name.substring(1), Character.MAX_RADIX);
+ toCommit.counter = Math.max(toCommit.counter, segmentCounter + 1);
+ toCommit.applyMergeChanges(applicableMerge, false);
+ }
+ mergeAwaitLatch.countDown();
+ }
+ };
+ }
+
private long prepareCommitInternal() throws IOException {
startCommitTime = System.nanoTime();
synchronized(commitLock) {
@@ -3174,6 +3212,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
SegmentInfos toCommit = null;
boolean anyChanges = false;
long seqNo;
+ List<MergePolicy.OneMerge> commitMerges = null;
+ AtomicReference<CountDownLatch> mergeAwaitLatchRef = null;
// This is copied from doFlush, except it's modified to
// clone & incRef the flushed SegmentInfos inside the
@@ -3228,6 +3268,30 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// sneak into the commit point:
toCommit = segmentInfos.clone();
+ if (anyChanges) {
+ // Find any merges that can execute on commit (per MergePolicy).
+ MergePolicy.MergeSpecification mergeSpec =
+ config.getMergePolicy().findFullFlushMerges(MergeTrigger.COMMIT, segmentInfos, this);
+ if (mergeSpec != null && mergeSpec.merges.size() > 0) {
+ int mergeCount = mergeSpec.merges.size();
+ commitMerges = new ArrayList<>(mergeCount);
+ mergeAwaitLatchRef = new AtomicReference<>(new CountDownLatch(mergeCount));
+ for (MergePolicy.OneMerge oneMerge : mergeSpec.merges) {
+ MergePolicy.OneMerge trackedMerge =
+ updateSegmentInfosOnMergeFinish(oneMerge, toCommit, mergeAwaitLatchRef);
+ if (registerMerge(trackedMerge) == false) {
+ throw new IllegalStateException("MergePolicy " + config.getMergePolicy().getClass() +
+ " returned merging segments from findFullFlushMerges");
+ }
+ commitMerges.add(trackedMerge);
+ }
+ if (infoStream.isEnabled("IW")) {
+ infoStream.message("IW", "Registered " + mergeCount + " commit merges");
+ infoStream.message("IW", "Before executing commit merges, had " + toCommit.size() + " segments");
+ }
+ }
+ }
+
pendingCommitChangeCount = changeCount.get();
// This protects the segmentInfos we are now going
@@ -3235,8 +3299,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// we are trying to sync all referenced files, a
// merge completes which would otherwise have
// removed the files we are now syncing.
- filesToCommit = toCommit.files(false);
- deleter.incRef(filesToCommit);
+ deleter.incRef(toCommit.files(false));
}
success = true;
} finally {
@@ -3257,6 +3320,52 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
} finally {
maybeCloseOnTragicEvent();
}
+
+ if (mergeAwaitLatchRef != null) {
+ CountDownLatch mergeAwaitLatch = mergeAwaitLatchRef.get();
+ // If we found and registered any merges above, within the flushLock, then we want to ensure that they
+ // complete execution. Note that since we released the lock, other merges may have been scheduled. We will
+ // block until the merges that we registered complete. As they complete, they will update toCommit to
+ // replace merged segments with the result of each merge.
+ config.getIndexWriterEvents().beginMergeOnCommit();
+ mergeScheduler.merge(mergeSource, MergeTrigger.COMMIT);
+ long mergeWaitStart = System.nanoTime();
+ int abandonedCount = 0;
+ long waitTimeMillis = (long) (config.getMaxCommitMergeWaitSeconds() * 1000.0);
+ try {
+ if (mergeAwaitLatch.await(waitTimeMillis, TimeUnit.MILLISECONDS) == false) {
+ synchronized (this) {
+ // Need to do this in a synchronized block, to make sure none of our commit merges are currently
+ // executing mergeFinished (since mergeFinished itself is called from within the IndexWriter lock).
+ // After we clear the value from mergeAwaitLatchRef, the merges we schedule will still execute as
+ // usual, but when they finish, they won't attempt to update toCommit or modify segment reference
+ // counts.
+ mergeAwaitLatchRef.set(null);
+ for (MergePolicy.OneMerge commitMerge : commitMerges) {
+ if (runningMerges.contains(commitMerge) || pendingMerges.contains(commitMerge)) {
+ abandonedCount++;
+ }
+ }
+ }
+ }
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ } finally {
+ if (infoStream.isEnabled("IW")) {
+ infoStream.message("IW", String.format(Locale.ROOT, "Waited %.1f ms for commit merges",
+ (System.nanoTime() - mergeWaitStart)/1_000_000.0));
+ infoStream.message("IW", "After executing commit merges, had " + toCommit.size() + " segments");
+ if (abandonedCount > 0) {
+ infoStream.message("IW", "Abandoned " + abandonedCount + " commit merges after " + waitTimeMillis + " ms");
+ }
+ }
+ if (abandonedCount > 0) {
+ config.getIndexWriterEvents().abandonedMergesOnCommit(abandonedCount);
+ }
+ config.getIndexWriterEvents().finishMergeOnCommit();
+ }
+ }
+ filesToCommit = toCommit.files(false);
try {
if (anyChanges) {
@@ -3962,6 +4071,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
}
try (Closeable finalizer = this::checkpoint) {
+ merge.committed = true;
// Must close before checkpoint, otherwise IFD won't be
// able to delete the held-open files from the merge
// readers:
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
index 26e7e3d..629b1e8 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import java.util.Arrays;
+import java.util.EnumSet;
import java.util.stream.Collectors;
import org.apache.lucene.analysis.Analyzer;
@@ -109,6 +110,9 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
/** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
+
+ /** Default value for time to wait for merges on commit (when using a {@link MergePolicy} that implements findFullFlushMerges). */
+ public static final double DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS = 30.0;
// indicates whether this config instance is already attached to a writer.
// not final so that it can be cloned properly.
@@ -460,6 +464,31 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig {
}
/**
+ * Expert: sets the amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...).
+ * If this time is reached, we proceed with the commit based on segments merged up to that point.
+ * The merges are not cancelled, and may still run to completion independent of the commit.
+ */
+ public IndexWriterConfig setMaxCommitMergeWaitSeconds(double maxCommitMergeWaitSeconds) {
+ this.maxCommitMergeWaitSeconds = maxCommitMergeWaitSeconds;
+ return this;
+ }
+
+ /**
+ * Set the callback that gets invoked when IndexWriter performs various actions.
+ */
+ public IndexWriterConfig setIndexWriterEvents(IndexWriterEvents indexWriterEvents) {
+ this.indexWriterEvents = indexWriterEvents;
+ return this;
+ }
+
+ /** We only allow sorting on these types */
+ private static final EnumSet<SortField.Type> ALLOWED_INDEX_SORT_TYPES = EnumSet.of(SortField.Type.STRING,
+ SortField.Type.LONG,
+ SortField.Type.INT,
+ SortField.Type.DOUBLE,
+ SortField.Type.FLOAT);
+
+ /**
* Set the {@link Sort} order to use for all (flushed and merged) segments.
*/
public IndexWriterConfig setIndexSort(Sort sort) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterEvents.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterEvents.java
new file mode 100644
index 0000000..d36fb25
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterEvents.java
@@ -0,0 +1,57 @@
+/*
+ * 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.lucene.index;
+
+/**
+ * Callback interface to signal various actions taken by IndexWriter.
+ *
+ * @lucene.experimental
+ */
+public interface IndexWriterEvents {
+ /**
+ * A default implementation that ignores all events.
+ */
+ IndexWriterEvents NULL_EVENTS = new IndexWriterEvents() {
+ @Override
+ public void beginMergeOnCommit() { }
+
+ @Override
+ public void finishMergeOnCommit() { }
+
+ @Override
+ public void abandonedMergesOnCommit(int abandonedCount) { }
+ };
+
+ /**
+ * Signals the start of waiting for a merge on commit, returned from
+ * {@link MergePolicy#findFullFlushMerges(MergeTrigger, SegmentInfos, MergePolicy.MergeContext)}.
+ */
+ void beginMergeOnCommit();
+
+ /**
+ * Signals the end of waiting for merges on commit. This may be either because the merges completed, or because we timed out according
+ * to the limit set in {@link IndexWriterConfig#setMaxCommitMergeWaitSeconds(double)}.
+ */
+ void finishMergeOnCommit();
+
+ /**
+ * Called to signal that we abandoned some merges on commit upon reaching the timeout specified in
+ * {@link IndexWriterConfig#setMaxCommitMergeWaitSeconds(double)}.
+ */
+ void abandonedMergesOnCommit(int abandonedCount);
+}
diff --git a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
index 1f48acc..59a54c7 100644
--- a/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
+++ b/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
@@ -109,6 +109,12 @@ public class LiveIndexWriterConfig {
/** soft deletes field */
protected String softDeletesField = null;
+ /** Amount of time to wait for merges returned by MergePolicy.findFullFlushMerges(...) */
+ protected volatile double maxCommitMergeWaitSeconds;
+
+ /** Callback interface called on index writer actions. */
+ protected IndexWriterEvents indexWriterEvents;
+
// used by IndexWriterConfig
LiveIndexWriterConfig(Analyzer analyzer) {
@@ -132,6 +138,8 @@ public class LiveIndexWriterConfig {
flushPolicy = new FlushByRamOrCountsPolicy();
readerPooling = IndexWriterConfig.DEFAULT_READER_POOLING;
perThreadHardLimitMB = IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB;
+ maxCommitMergeWaitSeconds = IndexWriterConfig.DEFAULT_MAX_COMMIT_MERGE_WAIT_SECONDS;
+ indexWriterEvents = IndexWriterEvents.NULL_EVENTS;
}
/** Returns the default analyzer to use for indexing documents. */
@@ -461,6 +469,22 @@ public class LiveIndexWriterConfig {
return softDeletesField;
}
+ /**
+ * Expert: return the amount of time to wait for merges returned by by MergePolicy.findFullFlushMerges(...).
+ * If this time is reached, we proceed with the commit based on segments merged up to that point.
+ * The merges are not cancelled, and may still run to completion independent of the commit.
+ */
+ public double getMaxCommitMergeWaitSeconds() {
+ return maxCommitMergeWaitSeconds;
+ }
+
+ /**
+ * Returns a callback used to signal actions taken by the {@link IndexWriter}.
+ */
+ public IndexWriterEvents getIndexWriterEvents() {
+ return indexWriterEvents;
+ }
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@@ -484,6 +508,8 @@ public class LiveIndexWriterConfig {
sb.append("indexSort=").append(getIndexSort()).append("\n");
sb.append("checkPendingFlushOnUpdate=").append(isCheckPendingFlushOnUpdate()).append("\n");
sb.append("softDeletesField=").append(getSoftDeletesField()).append("\n");
+ sb.append("maxCommitMergeWaitSeconds=").append(getMaxCommitMergeWaitSeconds()).append("\n");
+ sb.append("indexWriterEvents=").append(getIndexWriterEvents().getClass().getName()).append("\n");
return sb.toString();
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
index 3ac3914..13fb2db 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java
@@ -225,6 +225,8 @@ public abstract class MergePolicy {
public final int totalMaxDoc;
Throwable error;
+ boolean committed; // Set by IndexWriter once the merge has been committed to disk
+
/** Sole constructor.
* @param segments List of {@link SegmentCommitInfo}s
* to be merged. */
@@ -500,7 +502,7 @@ public abstract class MergePolicy {
* an original segment present in the
* to-be-merged index; else, it was a segment
* produced by a cascaded merge.
- * @param mergeContext the IndexWriter to find the merges on
+ * @param mergeContext the MergeContext to find the merges on
*/
public abstract MergeSpecification findForcedMerges(
SegmentInfos segmentInfos, int maxSegmentCount, Map<SegmentCommitInfo,Boolean> segmentsToMerge, MergeContext mergeContext)
@@ -511,12 +513,34 @@ public abstract class MergePolicy {
* deletes from the index.
* @param segmentInfos
* the total set of segments in the index
- * @param mergeContext the IndexWriter to find the merges on
+ * @param mergeContext the MergeContext to find the merges on
*/
public abstract MergeSpecification findForcedDeletesMerges(
SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException;
/**
+ * Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit.
+ *
+ * Any merges returned here will make {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} block until
+ * the merges complete or until {@link IndexWriterConfig#getMaxCommitMergeWaitSeconds()} have elapsed. This may be
+ * used to merge small segments that have just been flushed as part of the commit, reducing the number of segments in
+ * the commit. If a merge does not complete in the allotted time, it will continue to execute, but will not be reflected
+ * in the commit.
+ *
+ * If a {@link OneMerge} in the returned {@link MergeSpecification} includes a segment already included in a registered
+ * merge, then {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} will throw a {@link IllegalStateException}.
+ * Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge.
+ *
+ * @param mergeTrigger the event that triggered the merge (COMMIT or FULL_FLUSH).
+ * @param segmentInfos the total set of segments in the index (while preparing the commit)
+ * @param mergeContext the MergeContext to find the merges on, which should be used to determine which segments are
+ * already in a registered merge (see {@link MergeContext#getMergingSegments()}).
+ */
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+ return null;
+ }
+
+ /**
* Returns true if a new segment (regardless of its origin) should use the
* compound file format. The default implementation returns <code>true</code>
* iff the size of the given mergedInfo is less or equal to
diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java b/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
index d165a27..01a6b15 100644
--- a/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
+++ b/lucene/core/src/java/org/apache/lucene/index/MergeTrigger.java
@@ -47,5 +47,10 @@ public enum MergeTrigger {
/**
* Merge was triggered by a closing IndexWriter.
*/
- CLOSING
+ CLOSING,
+
+ /**
+ * Merge was triggered on commit.
+ */
+ COMMIT,
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
index 1480ce4..b209e8ae 100644
--- a/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
@@ -46,6 +46,9 @@ public final class NoMergePolicy extends MergePolicy {
public MergeSpecification findForcedDeletesMerges(SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
@Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) { return null; }
+
+ @Override
public boolean useCompoundFile(SegmentInfos segments, SegmentCommitInfo newSegment, MergeContext mergeContext) {
return newSegment.info.getUseCompoundFile();
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java b/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
index d08711e..a5fd66a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/OneMergeWrappingMergePolicy.java
@@ -59,6 +59,11 @@ public class OneMergeWrappingMergePolicy extends FilterMergePolicy {
return wrapSpec(in.findForcedDeletesMerges(segmentInfos, mergeContext));
}
+ @Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+ return wrapSpec(in.findFullFlushMerges(mergeTrigger, segmentInfos, mergeContext));
+ }
+
private MergeSpecification wrapSpec(MergeSpecification spec) {
MergeSpecification wrapped = spec == null ? null : new MergeSpecification();
if (wrapped != null) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
index ce591a2..8a463ef 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java
@@ -18,17 +18,42 @@ package org.apache.lucene.index;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
public class TestIndexWriterMergePolicy extends LuceneTestCase {
-
+
+ private static final MergePolicy MERGE_ON_COMMIT_POLICY = new LogDocMergePolicy() {
+ @Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) {
+ // Optimize down to a single segment on commit
+ if (mergeTrigger == MergeTrigger.COMMIT && segmentInfos.size() > 1) {
+ List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
+ for (SegmentCommitInfo sci : segmentInfos) {
+ if (mergeContext.getMergingSegments().contains(sci) == false) {
+ nonMergingSegments.add(sci);
+ }
+ }
+ if (nonMergingSegments.size() > 1) {
+ MergeSpecification mergeSpecification = new MergeSpecification();
+ mergeSpecification.add(new OneMerge(nonMergingSegments));
+ return mergeSpecification;
+ }
+ }
+ return null;
+ }
+ };
+
// Test the normal case
public void testNormalCase() throws IOException {
Directory dir = newDirectory();
@@ -278,6 +303,49 @@ public class TestIndexWriterMergePolicy extends LuceneTestCase {
assertSetters(new LogDocMergePolicy());
}
+ // Test basic semantics of merge on commit
+ public void testMergeOnCommit() throws IOException {
+ Directory dir = newDirectory();
+
+ IndexWriter firstWriter = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+ .setMergePolicy(NoMergePolicy.INSTANCE));
+ for (int i = 0; i < 5; i++) {
+ TestIndexWriter.addDoc(firstWriter);
+ firstWriter.flush();
+ }
+ DirectoryReader firstReader = DirectoryReader.open(firstWriter);
+ assertEquals(5, firstReader.leaves().size());
+ firstReader.close();
+ firstWriter.close(); // When this writer closes, it does not merge on commit.
+
+ IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
+ .setMergePolicy(MERGE_ON_COMMIT_POLICY);
+
+ IndexWriter writerWithMergePolicy = new IndexWriter(dir, iwc);
+ writerWithMergePolicy.commit(); // No changes. Commit doesn't trigger a merge.
+
+ DirectoryReader unmergedReader = DirectoryReader.open(writerWithMergePolicy);
+ assertEquals(5, unmergedReader.leaves().size());
+ unmergedReader.close();
+
+ TestIndexWriter.addDoc(writerWithMergePolicy);
+ writerWithMergePolicy.commit(); // Doc added, do merge on commit.
+ assertEquals(1, writerWithMergePolicy.getSegmentCount()); //
+
+ DirectoryReader mergedReader = DirectoryReader.open(writerWithMergePolicy);
+ assertEquals(1, mergedReader.leaves().size());
+ mergedReader.close();
+
+ try (IndexReader reader = writerWithMergePolicy.getReader()) {
+ IndexSearcher searcher = new IndexSearcher(reader);
+ assertEquals(6, reader.numDocs());
+ assertEquals(6, searcher.count(new MatchAllDocsQuery()));
+ }
+
+ writerWithMergePolicy.close();
+ dir.close();
+ }
+
private void assertSetters(MergePolicy lmp) {
lmp.setMaxCFSSegmentSizeMB(2.0);
assertEquals(2.0, lmp.getMaxCFSSegmentSizeMB(), EPSILON);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java b/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
index beb4dad..92ffc73 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/MockRandomMergePolicy.java
@@ -129,6 +129,38 @@ public class MockRandomMergePolicy extends MergePolicy {
}
@Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException {
+ MergeSpecification mergeSpecification = findMerges(null, segmentInfos, mergeContext);
+ if (mergeSpecification == null) {
+ return null;
+ }
+ // Do not return any merges involving already-merging segments.
+ MergeSpecification filteredMergeSpecification = new MergeSpecification();
+ for (OneMerge oneMerge : mergeSpecification.merges) {
+ boolean filtered = false;
+ List<SegmentCommitInfo> nonMergingSegments = new ArrayList<>();
+ for (SegmentCommitInfo sci : oneMerge.segments) {
+ if (mergeContext.getMergingSegments().contains(sci) == false) {
+ nonMergingSegments.add(sci);
+ } else {
+ filtered = true;
+ }
+ }
+ if (filtered == true) {
+ if (nonMergingSegments.size() > 0) {
+ filteredMergeSpecification.add(new OneMerge(nonMergingSegments));
+ }
+ } else {
+ filteredMergeSpecification.add(oneMerge);
+ }
+ }
+ if (filteredMergeSpecification.merges.size() > 0) {
+ return filteredMergeSpecification;
+ }
+ return null;
+ }
+
+ @Override
public boolean useCompoundFile(SegmentInfos infos, SegmentCommitInfo mergedInfo, MergeContext mergeContext) throws IOException {
// 80% of the time we create CFS:
return random.nextInt(5) != 1;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 9f2cd27..cc779a0 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1003,6 +1003,7 @@ public abstract class LuceneTestCase extends Assert {
if (rarely(r)) {
c.setCheckPendingFlushUpdate(false);
}
+ c.setMaxCommitMergeWaitSeconds(atLeast(r, 1));
return c;
}
[lucene-solr] 02/02: Fix case where mergeOnCommit would attempt to
delete files twice in the presence of deletions
Posted by so...@apache.org.
This is an automated email from the ASF dual-hosted git repository.
sokolov pushed a commit to branch jira/lucene-8962
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
commit cab5ef5e6f2bdcda59fd669a298ec1377777af9d
Author: Michael Sokolov <so...@amazon.com>
AuthorDate: Wed Jun 3 15:12:02 2020 -0400
Fix case where mergeOnCommit would attempt to delete files twice in the presence of deletions
---
.../java/org/apache/lucene/index/IndexWriter.java | 3 +-
.../org/apache/lucene/index/TestIndexWriter.java | 141 +++++++++++++--------
2 files changed, 91 insertions(+), 53 deletions(-)
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 88fdb90..13e0443 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -3169,13 +3169,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// Resolve "live" SegmentInfos segments to their toCommit cloned equivalents, based on segment name.
Set<String> mergedSegmentNames = new HashSet<>();
for (SegmentCommitInfo sci : this.segments) {
- deleter.decRef(sci.files());
mergedSegmentNames.add(sci.info.name);
}
List<SegmentCommitInfo> toCommitMergedAwaySegments = new ArrayList<>();
for (SegmentCommitInfo sci : toCommit) {
if (mergedSegmentNames.contains(sci.info.name)) {
toCommitMergedAwaySegments.add(sci);
+ deleter.decRef(sci.files());
}
}
// Construct a OneMerge that applies to toCommit
@@ -4593,6 +4593,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
// Merge would produce a 0-doc segment, so we do nothing except commit the merge to remove all the 0-doc segments that we "merged":
assert merge.info.info.maxDoc() == 0;
commitMerge(merge, mergeState);
+ success = true;
return 0;
}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index 8fb1ce5..7590b1a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -344,7 +344,7 @@ public class TestIndexWriter extends LuceneTestCase {
// Make sure it's OK to change RAM buffer size and
// maxBufferedDocs in a write session
public void testChangingRAMBuffer() throws IOException {
- Directory dir = newDirectory();
+ Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
writer.getConfig().setMaxBufferedDocs(10);
writer.getConfig().setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH);
@@ -607,7 +607,7 @@ public class TestIndexWriter extends LuceneTestCase {
doc.add(newField("content4", contents, customType));
type = customType;
} else
- type = TextField.TYPE_NOT_STORED;
+ type = TextField.TYPE_NOT_STORED;
doc.add(newTextField("content1", contents, Field.Store.NO));
doc.add(newField("content3", "", customType));
doc.add(newField("content5", "", type));
@@ -663,13 +663,13 @@ public class TestIndexWriter extends LuceneTestCase {
writer.close();
dir.close();
}
-
+
public void testEmptyFieldNameTerms() throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
Document doc = new Document();
doc.add(newTextField("", "a b c", Field.Store.NO));
- writer.addDocument(doc);
+ writer.addDocument(doc);
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
LeafReader subreader = getOnlyLeafReader(reader);
@@ -681,7 +681,7 @@ public class TestIndexWriter extends LuceneTestCase {
reader.close();
dir.close();
}
-
+
public void testEmptyFieldNameWithEmptyTerm() throws IOException {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -690,7 +690,7 @@ public class TestIndexWriter extends LuceneTestCase {
doc.add(newStringField("", "a", Field.Store.NO));
doc.add(newStringField("", "b", Field.Store.NO));
doc.add(newStringField("", "c", Field.Store.NO));
- writer.addDocument(doc);
+ writer.addDocument(doc);
writer.close();
DirectoryReader reader = DirectoryReader.open(dir);
LeafReader subreader = getOnlyLeafReader(reader);
@@ -834,7 +834,7 @@ public class TestIndexWriter extends LuceneTestCase {
customType.setStoreTermVectors(true);
customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorOffsets(true);
-
+
doc.add(newField("content", "aaa bbb ccc ddd eee fff ggg hhh iii", customType));
writer.addDocument(doc);
writer.addDocument(doc);
@@ -922,7 +922,7 @@ public class TestIndexWriter extends LuceneTestCase {
// open/close slowly sometimes
dir.setUseSlowOpenClosers(true);
-
+
// throttle a little
dir.setThrottling(MockDirectoryWrapper.Throttling.SOMETIMES);
@@ -1148,7 +1148,7 @@ public class TestIndexWriter extends LuceneTestCase {
FieldType customType = new FieldType(StoredField.TYPE);
customType.setTokenized(true);
-
+
Field f = new Field("binary", b, 10, 17, customType);
// TODO: this is evil, changing the type after creating the field:
customType.setIndexOptions(IndexOptions.DOCS);
@@ -1157,7 +1157,7 @@ public class TestIndexWriter extends LuceneTestCase {
f.setTokenStream(doc1field1);
FieldType customType2 = new FieldType(TextField.TYPE_STORED);
-
+
Field f2 = newField("string", "value", customType2);
final MockTokenizer doc1field2 = new MockTokenizer(MockTokenizer.WHITESPACE, false);
doc1field2.setReader(new StringReader("doc1field2"));
@@ -1233,7 +1233,7 @@ public class TestIndexWriter extends LuceneTestCase {
public void testDeleteUnusedFiles() throws Exception {
assumeFalse("test relies on exact filenames", Codec.getDefault() instanceof SimpleTextCodec);
assumeWorkingMMapOnWindows();
-
+
for(int iter=0;iter<2;iter++) {
// relies on windows semantics
Path path = createTempDir();
@@ -1250,7 +1250,7 @@ public class TestIndexWriter extends LuceneTestCase {
}
MergePolicy mergePolicy = newLogMergePolicy(true);
-
+
// This test expects all of its segments to be in CFS
mergePolicy.setNoCFSRatio(1.0);
mergePolicy.setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
@@ -1338,7 +1338,7 @@ public class TestIndexWriter extends LuceneTestCase {
customType.setStoreTermVectors(true);
customType.setStoreTermVectorPositions(true);
customType.setStoreTermVectorOffsets(true);
-
+
doc.add(newField("c", "val", customType));
writer.addDocument(doc);
writer.commit();
@@ -1379,7 +1379,7 @@ public class TestIndexWriter extends LuceneTestCase {
// indexed, flushed (but not committed) and then IW rolls back, then no
// files are left in the Directory.
Directory dir = newDirectory();
-
+
String[] origFiles = dir.listAll();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
.setMaxBufferedDocs(2)
@@ -1409,8 +1409,8 @@ public class TestIndexWriter extends LuceneTestCase {
// Adding just one document does not call flush yet.
int computedExtraFileCount = 0;
for (String file : dir.listAll()) {
- if (IndexWriter.WRITE_LOCK_NAME.equals(file) ||
- file.startsWith(IndexFileNames.SEGMENTS) ||
+ if (IndexWriter.WRITE_LOCK_NAME.equals(file) ||
+ file.startsWith(IndexFileNames.SEGMENTS) ||
IndexFileNames.CODEC_FILE_PATTERN.matcher(file).matches()) {
if (file.lastIndexOf('.') < 0
// don't count stored fields and term vectors in, or any temporary files they might
@@ -1458,7 +1458,7 @@ public class TestIndexWriter extends LuceneTestCase {
FieldType customType3 = new FieldType(TextField.TYPE_STORED);
customType3.setTokenized(false);
customType3.setOmitNorms(true);
-
+
for (int i=0; i<2; i++) {
Document doc = new Document();
doc.add(new Field("id", Integer.toString(i)+BIG, customType3));
@@ -1478,7 +1478,7 @@ public class TestIndexWriter extends LuceneTestCase {
SegmentReader sr = (SegmentReader) ctx.reader();
assertFalse(sr.getFieldInfos().hasVectors());
}
-
+
r0.close();
dir.close();
}
@@ -1501,7 +1501,7 @@ public class TestIndexWriter extends LuceneTestCase {
@Override
public final boolean incrementToken() {
- clearAttributes();
+ clearAttributes();
if (upto < tokens.length) {
termAtt.setEmpty();
termAtt.append(tokens[upto]);
@@ -1724,7 +1724,7 @@ public class TestIndexWriter extends LuceneTestCase {
r.close();
dir.close();
}
-
+
public void testDontInvokeAnalyzerForUnAnalyzedFields() throws Exception {
Analyzer analyzer = new Analyzer() {
@Override
@@ -1759,13 +1759,13 @@ public class TestIndexWriter extends LuceneTestCase {
w.close();
dir.close();
}
-
+
//LUCENE-1468 -- make sure opening an IndexWriter with
// create=true does not remove non-index files
-
+
public void testOtherFiles() throws Throwable {
Directory dir = newDirectory();
- IndexWriter iw = new IndexWriter(dir,
+ IndexWriter iw = new IndexWriter(dir,
newIndexWriterConfig(new MockAnalyzer(random())));
iw.addDocument(new Document());
iw.close();
@@ -1774,15 +1774,15 @@ public class TestIndexWriter extends LuceneTestCase {
IndexOutput out = dir.createOutput("myrandomfile", newIOContext(random()));
out.writeByte((byte) 42);
out.close();
-
+
new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))).close();
-
+
assertTrue(slowFileExists(dir, "myrandomfile"));
} finally {
dir.close();
}
}
-
+
// LUCENE-3849
public void testStopwordsPosIncHole() throws Exception {
Directory dir = newDirectory();
@@ -1811,7 +1811,7 @@ public class TestIndexWriter extends LuceneTestCase {
ir.close();
dir.close();
}
-
+
// LUCENE-3849
public void testStopwordsPosIncHole2() throws Exception {
// use two stopfilters for testing here
@@ -1843,23 +1843,23 @@ public class TestIndexWriter extends LuceneTestCase {
ir.close();
dir.close();
}
-
+
// LUCENE-4575
public void testCommitWithUserDataOnly() throws Exception {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(null));
writer.commit(); // first commit to complete IW create transaction.
-
+
// this should store the commit data, even though no other changes were made
writer.setLiveCommitData(new HashMap<String,String>() {{
put("key", "value");
}}.entrySet());
writer.commit();
-
+
DirectoryReader r = DirectoryReader.open(dir);
assertEquals("value", r.getIndexCommit().getUserData().get("key"));
r.close();
-
+
// now check setCommitData and prepareCommit/commit sequence
writer.setLiveCommitData(new HashMap<String,String>() {{
put("key", "value1");
@@ -1873,7 +1873,7 @@ public class TestIndexWriter extends LuceneTestCase {
r = DirectoryReader.open(dir);
assertEquals("value1", r.getIndexCommit().getUserData().get("key"));
r.close();
-
+
// now should commit the second commitData - there was a bug where
// IndexWriter.finishCommit overrode the second commitData
writer.commit();
@@ -1881,7 +1881,7 @@ public class TestIndexWriter extends LuceneTestCase {
assertEquals("IndexWriter.finishCommit may have overridden the second commitData",
"value2", r.getIndexCommit().getUserData().get("key"));
r.close();
-
+
writer.close();
dir.close();
}
@@ -1896,7 +1896,7 @@ public class TestIndexWriter extends LuceneTestCase {
}
return data;
}
-
+
@Test
public void testGetCommitData() throws Exception {
Directory dir = newDirectory();
@@ -1906,16 +1906,16 @@ public class TestIndexWriter extends LuceneTestCase {
}}.entrySet());
assertEquals("value", getLiveCommitData(writer).get("key"));
writer.close();
-
+
// validate that it's also visible when opening a new IndexWriter
writer = new IndexWriter(dir, newIndexWriterConfig(null)
.setOpenMode(OpenMode.APPEND));
assertEquals("value", getLiveCommitData(writer).get("key"));
writer.close();
-
+
dir.close();
}
-
+
public void testNullAnalyzer() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwConf = newIndexWriterConfig(null);
@@ -1942,7 +1942,7 @@ public class TestIndexWriter extends LuceneTestCase {
iw.close();
dir.close();
}
-
+
public void testNullDocument() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
@@ -1967,7 +1967,7 @@ public class TestIndexWriter extends LuceneTestCase {
iw.close();
dir.close();
}
-
+
public void testNullDocuments() throws IOException {
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
@@ -1992,7 +1992,7 @@ public class TestIndexWriter extends LuceneTestCase {
iw.close();
dir.close();
}
-
+
public void testIterableFieldThrowsException() throws IOException {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2000,7 +2000,7 @@ public class TestIndexWriter extends LuceneTestCase {
int docCount = 0;
int docId = 0;
Set<String> liveIds = new HashSet<>();
- for (int i = 0; i < iters; i++) {
+ for (int i = 0; i < iters; i++) {
int numDocs = atLeast(4);
for (int j = 0; j < numDocs; j++) {
String id = Integer.toString(docId++);
@@ -2008,7 +2008,7 @@ public class TestIndexWriter extends LuceneTestCase {
fields.add(new StringField("id", id, Field.Store.YES));
fields.add(new StringField("foo", TestUtil.randomSimpleString(random()), Field.Store.NO));
docId++;
-
+
boolean success = false;
try {
w.addDocument(new RandomFailingIterable<IndexableField>(fields, random()));
@@ -2040,7 +2040,7 @@ public class TestIndexWriter extends LuceneTestCase {
w.close();
IOUtils.close(reader, dir);
}
-
+
public void testIterableThrowsException() throws IOException {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2088,7 +2088,7 @@ public class TestIndexWriter extends LuceneTestCase {
w.close();
IOUtils.close(reader, dir);
}
-
+
public void testIterableThrowsException2() throws IOException {
Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2128,7 +2128,7 @@ public class TestIndexWriter extends LuceneTestCase {
this.list = list;
this.failOn = random.nextInt(5);
}
-
+
@Override
public Iterator<T> iterator() {
final Iterator<? extends T> docIter = list.iterator();
@@ -2254,7 +2254,7 @@ public class TestIndexWriter extends LuceneTestCase {
writer.close();
dir.close();
}
-
+
public void testMergeAllDeleted() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
@@ -2477,12 +2477,12 @@ public class TestIndexWriter extends LuceneTestCase {
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
w.addDocument(new Document());
w.close();
-
+
SegmentInfos sis = SegmentInfos.readLatestCommit(d);
byte[] id1 = sis.getId();
assertNotNull(id1);
assertEquals(StringHelper.ID_LENGTH, id1.length);
-
+
byte[] id2 = sis.info(0).info.getId();
byte[] sciId2 = sis.info(0).getId();
assertNotNull(id2);
@@ -2514,7 +2514,7 @@ public class TestIndexWriter extends LuceneTestCase {
ids.add(id);
}
}
-
+
public void testEmptyNorm() throws Exception {
Directory d = newDirectory();
IndexWriter w = new IndexWriter(d, newIndexWriterConfig(new MockAnalyzer(random())));
@@ -2579,7 +2579,7 @@ public class TestIndexWriter extends LuceneTestCase {
assertEquals(1, r2.getIndexCommit().getGeneration());
assertEquals("segments_1", r2.getIndexCommit().getSegmentsFileName());
r2.close();
-
+
// make a change and another commit
w.addDocument(new Document());
w.commit();
@@ -2866,7 +2866,7 @@ public class TestIndexWriter extends LuceneTestCase {
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
IndexWriter w = new IndexWriter(dir, iwc);
w.close();
-
+
IndexOutput out = dir.createTempOutput("_0", "bkd", IOContext.DEFAULT);
String tempName = out.getName();
out.close();
@@ -3151,7 +3151,7 @@ public class TestIndexWriter extends LuceneTestCase {
expectThrows(IllegalArgumentException.class, () -> {
writer.softUpdateDocument(null, new Document(), new NumericDocValuesField("soft_delete", 1));
});
-
+
expectThrows(IllegalArgumentException.class, () -> {
writer.softUpdateDocument(new Term("id", "1"), new Document());
});
@@ -4167,4 +4167,41 @@ public class TestIndexWriter extends LuceneTestCase {
}
}
}
+
+ public void testMergeOnCommitKeepFullyDeletedSegments() throws Exception {
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ iwc.mergePolicy = new FilterMergePolicy(newMergePolicy()) {
+ @Override
+ public boolean keepFullyDeletedSegment(IOSupplier<CodecReader> readerIOSupplier) {
+ return true;
+ }
+
+ @Override
+ public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger,
+ SegmentInfos segmentInfos,
+ MergeContext mergeContext) {
+ List<SegmentCommitInfo> fullyDeletedSegments = segmentInfos.asList().stream()
+ .filter(s -> s.info.maxDoc() - s.getDelCount() == 0)
+ .collect(Collectors.toList());
+ if (fullyDeletedSegments.isEmpty()) {
+ return null;
+ }
+ MergeSpecification spec = new MergeSpecification();
+ spec.add(new OneMerge(fullyDeletedSegments));
+ return spec;
+ }
+ };
+ IndexWriter w = new IndexWriter(dir, iwc);
+ Document d = new Document();
+ d.add(new StringField("id", "1", Field.Store.YES));
+ w.addDocument(d);
+ w.commit();
+ w.updateDocument(new Term("id", "1"), d);
+ w.commit();
+ try (DirectoryReader reader = w.getReader()) {
+ assertEquals(1, reader.numDocs());
+ }
+ IOUtils.close(w, dir);
+ }
}