mirror of https://github.com/apache/lucene.git
LUCENE-8049: IndexWriter.getMergingSegments() signature changed to return Set instead of Collection
This commit is contained in:
parent
7acccd5157
commit
64d95e6a6d
|
@ -23,6 +23,9 @@ API Changes
|
|||
* LUCENE-8014: Similarity.computeSlopFactor() and
|
||||
Similarity.computePayloadFactor() have been removed (Alan Woodward)
|
||||
|
||||
* LUCENE-8049: IndexWriter.getMergingSegments()'s return type was changed from
|
||||
Collection to Set to more accurately reflect it's nature. (David Smiley)
|
||||
|
||||
Changes in Runtime Behavior
|
||||
|
||||
* LUCENE-7837: Indices that were created before the previous major version
|
||||
|
|
|
@ -2285,9 +2285,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
|||
* (which you do when IndexWriter invokes the
|
||||
* MergePolicy).
|
||||
*
|
||||
* <p>Do not alter the returned collection! */
|
||||
public synchronized Collection<SegmentCommitInfo> getMergingSegments() {
|
||||
return mergingSegments;
|
||||
* <p>The Set is unmodifiable. */
|
||||
public synchronized Set<SegmentCommitInfo> getMergingSegments() {
|
||||
return Collections.unmodifiableSet(mergingSegments);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,10 +19,10 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* <p>This class implements a {@link MergePolicy} that tries
|
||||
|
@ -462,7 +462,7 @@ public abstract class LogMergePolicy extends MergePolicy {
|
|||
final List<SegmentInfoAndLevel> levels = new ArrayList<>(numSegments);
|
||||
final float norm = (float) Math.log(mergeFactor);
|
||||
|
||||
final Collection<SegmentCommitInfo> mergingSegments = writer.getMergingSegments();
|
||||
final Set<SegmentCommitInfo> mergingSegments = writer.getMergingSegments();
|
||||
|
||||
for(int i=0;i<numSegments;i++) {
|
||||
final SegmentCommitInfo info = infos.info(i);
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Merges segments of approximately equal size, subject to
|
||||
|
@ -287,8 +288,8 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
if (infos.size() == 0) {
|
||||
return null;
|
||||
}
|
||||
final Collection<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
final Collection<SegmentCommitInfo> toBeMerged = new HashSet<>();
|
||||
final Set<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
final Set<SegmentCommitInfo> toBeMerged = new HashSet<>();
|
||||
|
||||
final List<SegmentCommitInfo> infosSorted = new ArrayList<>(infos.asList());
|
||||
|
||||
|
@ -519,7 +520,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
|
||||
List<SegmentCommitInfo> eligible = new ArrayList<>();
|
||||
boolean forceMergeRunning = false;
|
||||
final Collection<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
final Set<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
boolean segmentIsOriginal = false;
|
||||
for(SegmentCommitInfo info : infos) {
|
||||
final Boolean isOriginal = segmentsToMerge.get(info);
|
||||
|
@ -594,7 +595,7 @@ public class TieredMergePolicy extends MergePolicy {
|
|||
message("findForcedDeletesMerges infos=" + writer.segString(infos) + " forceMergeDeletesPctAllowed=" + forceMergeDeletesPctAllowed, writer);
|
||||
}
|
||||
final List<SegmentCommitInfo> eligible = new ArrayList<>();
|
||||
final Collection<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
final Set<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
for(SegmentCommitInfo info : infos) {
|
||||
double pctDeletes = 100.*((double) writer.numDeletedDocs(info))/info.info.maxDoc();
|
||||
if (pctDeletes > forceMergeDeletesPctAllowed && !merging.contains(info)) {
|
||||
|
|
|
@ -18,11 +18,11 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
|
@ -56,7 +56,7 @@ public class MockRandomMergePolicy extends MergePolicy {
|
|||
int numSegments = segmentInfos.size();
|
||||
|
||||
List<SegmentCommitInfo> segments = new ArrayList<>();
|
||||
final Collection<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
final Set<SegmentCommitInfo> merging = writer.getMergingSegments();
|
||||
|
||||
for(SegmentCommitInfo sipc : segmentInfos) {
|
||||
if (!merging.contains(sipc)) {
|
||||
|
|
Loading…
Reference in New Issue