diff --git a/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java b/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java index 11e55734046..745117daec0 100644 --- a/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java +++ b/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java @@ -132,9 +132,9 @@ class BufferedDeletesStream { public final long gen; // If non-null, contains segments that are 100% deleted - public final SegmentInfos allDeleted; + public final List allDeleted; - ApplyDeletesResult(boolean anyDeletes, long gen, SegmentInfos allDeleted) { + ApplyDeletesResult(boolean anyDeletes, long gen, List allDeleted) { this.anyDeletes = anyDeletes; this.gen = gen; this.allDeleted = allDeleted; @@ -164,7 +164,7 @@ class BufferedDeletesStream { /** Resolves the buffered deleted Term/Query/docIDs, into * actual deleted docIDs in the deletedDocs BitVector for * each SegmentReader. */ - public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, SegmentInfos infos) throws IOException { + public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List infos) throws IOException { final long t0 = System.currentTimeMillis(); if (infos.size() == 0) { @@ -182,7 +182,7 @@ class BufferedDeletesStream { message("applyDeletes: infos=" + infos + " packetCount=" + deletes.size()); } - SegmentInfos infos2 = new SegmentInfos(); + List infos2 = new ArrayList(); infos2.addAll(infos); Collections.sort(infos2, sortSegInfoByDelGen); @@ -192,7 +192,7 @@ class BufferedDeletesStream { int infosIDX = infos2.size()-1; int delIDX = deletes.size()-1; - SegmentInfos allDeleted = null; + List allDeleted = null; while (infosIDX >= 0) { //System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX); @@ -245,7 +245,7 @@ class BufferedDeletesStream { if (segAllDeletes) { if (allDeleted == null) { - allDeleted = new SegmentInfos(); + allDeleted = new ArrayList(); } allDeleted.add(info); } @@ -287,7 +287,7 @@ class BufferedDeletesStream { if (segAllDeletes) { if (allDeleted == null) { - allDeleted = new SegmentInfos(); + allDeleted = new ArrayList(); } allDeleted.add(info); } diff --git a/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java b/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java index b9cafc7c5c2..bc29b35c241 100644 --- a/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java +++ b/lucene/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java @@ -135,8 +135,8 @@ public class ConcurrentMergeScheduler extends MergeScheduler { final MergePolicy.OneMerge m1 = t1.getCurrentMerge(); final MergePolicy.OneMerge m2 = t2.getCurrentMerge(); - final int c1 = m1 == null ? Integer.MAX_VALUE : m1.segments.totalDocCount(); - final int c2 = m2 == null ? Integer.MAX_VALUE : m2.segments.totalDocCount(); + final int c1 = m1 == null ? Integer.MAX_VALUE : m1.totalDocCount; + final int c2 = m2 == null ? Integer.MAX_VALUE : m2.totalDocCount; return c2 - c1; } diff --git a/lucene/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/src/java/org/apache/lucene/index/IndexWriter.java index 0f0ba3e85b6..4d924627828 100644 --- a/lucene/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/src/java/org/apache/lucene/index/IndexWriter.java @@ -421,7 +421,7 @@ public class IndexWriter implements Closeable { private final Map readerMap = new HashMap(); /** Forcefully clear changes for the specified segments. This is called on successful merge. */ - synchronized void clear(SegmentInfos infos) throws IOException { + synchronized void clear(List infos) throws IOException { if (infos == null) { for (Map.Entry ent: readerMap.entrySet()) { ent.getValue().hasChanges = false; @@ -511,7 +511,7 @@ public class IndexWriter implements Closeable { return false; } - public synchronized void drop(SegmentInfos infos) throws IOException { + public synchronized void drop(List infos) throws IOException { for(SegmentInfo info : infos) { drop(info); } @@ -2729,7 +2729,7 @@ public class IndexWriter implements Closeable { assert testPoint("startCommitMergeDeletes"); - final SegmentInfos sourceSegments = merge.segments; + final List sourceSegments = merge.segments; if (infoStream != null) message("commitMergeDeletes " + merge.segString(directory)); @@ -2741,7 +2741,7 @@ public class IndexWriter implements Closeable { long minGen = Long.MAX_VALUE; for(int i=0; i < sourceSegments.size(); i++) { - SegmentInfo info = sourceSegments.info(i); + SegmentInfo info = sourceSegments.get(i); minGen = Math.min(info.getBufferedDeletesGen(), minGen); int docCount = info.docCount; final SegmentReader previousReader = merge.readerClones.get(i); @@ -3184,7 +3184,7 @@ public class IndexWriter implements Closeable { // It's possible we are called twice, eg if there was an // exception inside mergeInit if (merge.registerDone) { - final SegmentInfos sourceSegments = merge.segments; + final List sourceSegments = merge.segments; for(SegmentInfo info : sourceSegments) { mergingSegments.remove(info); } @@ -3255,7 +3255,7 @@ public class IndexWriter implements Closeable { int mergedDocCount = 0; - SegmentInfos sourceSegments = merge.segments; + List sourceSegments = merge.segments; SegmentMerger merger = new SegmentMerger(directory, config.getTermIndexInterval(), mergedName, merge, codecs, payloadProcessorProvider, @@ -3276,7 +3276,7 @@ public class IndexWriter implements Closeable { int segUpto = 0; while(segUpto < sourceSegments.size()) { - final SegmentInfo info = sourceSegments.info(segUpto); + final SegmentInfo info = sourceSegments.get(segUpto); // Hold onto the "live" reader; we will use this to // commit merged deletes @@ -3470,14 +3470,14 @@ public class IndexWriter implements Closeable { } /** @lucene.internal */ - public synchronized String segString(SegmentInfos infos) throws IOException { + public synchronized String segString(List infos) throws IOException { StringBuilder buffer = new StringBuilder(); final int count = infos.size(); for(int i = 0; i < count; i++) { if (i > 0) { buffer.append(' '); } - buffer.append(segString(infos.info(i))); + buffer.append(segString(infos.get(i))); } return buffer.toString(); diff --git a/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java b/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java index 1be4f26b77f..fc419bd2f7c 100644 --- a/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java +++ b/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java @@ -20,7 +20,6 @@ package org.apache.lucene.index; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Set; @@ -595,7 +594,7 @@ public abstract class LogMergePolicy extends MergePolicy { } else if (!anyTooLarge) { if (spec == null) spec = new MergeSpecification(); - final SegmentInfos mergeInfos = new SegmentInfos(); + final List mergeInfos = new ArrayList(); for(int i=start;i 0; // Make sure it all adds up: - assert docShift == maxDocID - (newStarts[docMaps.length-1] + merge.segments.info(docMaps.length-1).docCount - delCounts[docMaps.length-1]); + assert docShift == maxDocID - (newStarts[docMaps.length-1] + merge.segments.get(docMaps.length-1).docCount - delCounts[docMaps.length-1]); } public int remap(int oldDocID) { diff --git a/lucene/src/java/org/apache/lucene/index/MergePolicy.java b/lucene/src/java/org/apache/lucene/index/MergePolicy.java index 31289bd18d6..bbced4e9cef 100644 --- a/lucene/src/java/org/apache/lucene/index/MergePolicy.java +++ b/lucene/src/java/org/apache/lucene/index/MergePolicy.java @@ -75,15 +75,21 @@ public abstract class MergePolicy implements java.io.Closeable { long estimatedMergeBytes; // used by IndexWriter List readers; // used by IndexWriter List readerClones; // used by IndexWriter - public final SegmentInfos segments; + public final List segments; + public final int totalDocCount; boolean aborted; Throwable error; boolean paused; - public OneMerge(SegmentInfos segments) { + public OneMerge(List segments) { if (0 == segments.size()) throw new RuntimeException("segments must include at least one segment"); this.segments = segments; + int count = 0; + for(SegmentInfo info : segments) { + count += info.docCount; + } + totalDocCount = count; } /** Record that an exception occurred while executing @@ -147,7 +153,7 @@ public abstract class MergePolicy implements java.io.Closeable { final int numSegments = segments.size(); for(int i=0;i 0) b.append(' '); - b.append(segments.info(i).toString(dir, 0)); + b.append(segments.get(i).toString(dir, 0)); } if (info != null) b.append(" into ").append(info.name); diff --git a/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java b/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java index a070ce0f8c4..e69f612553d 100644 --- a/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java +++ b/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java @@ -23,6 +23,8 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Comparator; +import java.util.List; +import java.util.ArrayList; /** * Merges segments of approximately equal size, subject to @@ -249,7 +251,7 @@ public class TieredMergePolicy extends MergePolicy { final Collection merging = writer.get().getMergingSegments(); final Collection toBeMerged = new HashSet(); - final SegmentInfos infosSorted = new SegmentInfos(); + final List infosSorted = new ArrayList(); infosSorted.addAll(infos); Collections.sort(infosSorted, segmentByteSizeDescending); @@ -277,7 +279,7 @@ public class TieredMergePolicy extends MergePolicy { // If we have too-large segments, grace them out // of the maxSegmentCount: int tooBigCount = 0; - while (tooBigCount < infosSorted.size() && size(infosSorted.info(tooBigCount)) >= maxMergedSegmentBytes/2.0) { + while (tooBigCount < infosSorted.size() && size(infosSorted.get(tooBigCount)) >= maxMergedSegmentBytes/2.0) { totIndexBytes -= size(infosSorted.get(tooBigCount)); tooBigCount++; } @@ -310,7 +312,7 @@ public class TieredMergePolicy extends MergePolicy { // Gather eligible segments for merging, ie segments // not already being merged and not already picked (by // prior iteration of this loop) for merging: - final SegmentInfos eligible = new SegmentInfos(); + final List eligible = new ArrayList(); for(int idx = tooBigCount; idx best = null; boolean bestTooLarge = false; long bestMergeBytes = 0; @@ -341,10 +343,10 @@ public class TieredMergePolicy extends MergePolicy { long totAfterMergeBytes = 0; - final SegmentInfos candidate = new SegmentInfos(); + final List candidate = new ArrayList(); boolean hitTooLarge = false; for(int idx = startIdx;idx maxMergedSegmentBytes) { @@ -398,7 +400,7 @@ public class TieredMergePolicy extends MergePolicy { } /** Expert: scores one merge; subclasses can override. */ - protected MergeScore score(SegmentInfos candidate, boolean hitTooLarge, long mergingBytes) throws IOException { + protected MergeScore score(List candidate, boolean hitTooLarge, long mergingBytes) throws IOException { long totBeforeMergeBytes = 0; long totAfterMergeBytes = 0; long totAfterMergeBytesFloored = 0; @@ -420,7 +422,7 @@ public class TieredMergePolicy extends MergePolicy { // over time: skew = 1.0/maxMergeAtOnce; } else { - skew = ((double) floorSize(size(candidate.info(0))))/totAfterMergeBytesFloored; + skew = ((double) floorSize(size(candidate.get(0))))/totAfterMergeBytesFloored; } // Strongly favor merges with less skew (smaller @@ -458,7 +460,8 @@ public class TieredMergePolicy extends MergePolicy { if (verbose()) { message("findMergesForOptimize maxSegmentCount=" + maxSegmentCount + " infos=" + writer.get().segString(infos) + " segmentsToOptimize=" + segmentsToOptimize); } - SegmentInfos eligible = new SegmentInfos(); + + List eligible = new ArrayList(); boolean optimizeMergeRunning = false; final Collection merging = writer.get().getMergingSegments(); for(SegmentInfo info : infos) { @@ -499,7 +502,7 @@ public class TieredMergePolicy extends MergePolicy { if (spec == null) { spec = new MergeSpecification(); } - final OneMerge merge = new OneMerge(eligible.range(end-maxMergeAtOnceExplicit, end)); + final OneMerge merge = new OneMerge(eligible.subList(end-maxMergeAtOnceExplicit, end)); if (verbose()) { message("add merge=" + writer.get().segString(merge.segments)); } @@ -510,7 +513,7 @@ public class TieredMergePolicy extends MergePolicy { if (spec == null && !optimizeMergeRunning) { // Do final merge final int numToMerge = end - maxSegmentCount + 1; - final OneMerge merge = new OneMerge(eligible.range(end-numToMerge, end)); + final OneMerge merge = new OneMerge(eligible.subList(end-numToMerge, end)); if (verbose()) { message("add final merge=" + merge.segString(writer.get().getDirectory())); } @@ -527,7 +530,7 @@ public class TieredMergePolicy extends MergePolicy { if (verbose()) { message("findMergesToExpungeDeletes infos=" + writer.get().segString(infos) + " expungeDeletesPctAllowed=" + expungeDeletesPctAllowed); } - final SegmentInfos eligible = new SegmentInfos(); + final List eligible = new ArrayList(); final Collection merging = writer.get().getMergingSegments(); for(SegmentInfo info : infos) { double pctDeletes = 100.*((double) writer.get().numDeletedDocs(info))/info.docCount; @@ -580,7 +583,7 @@ public class TieredMergePolicy extends MergePolicy { spec = new MergeSpecification(); } - final OneMerge merge = new OneMerge(eligible.range(start, upto)); + final OneMerge merge = new OneMerge(eligible.subList(start, upto)); if (verbose()) { message("add merge=" + writer.get().segString(merge.segments)); } diff --git a/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java b/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java index cfc42aaae11..7e57888461d 100644 --- a/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java +++ b/lucene/src/java/org/apache/lucene/index/UpgradeIndexMergePolicy.java @@ -20,7 +20,9 @@ package org.apache.lucene.index; import org.apache.lucene.util.Constants; import java.io.IOException; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Set; /** This {@link MergePolicy} is used for upgrading all existing segments of @@ -101,7 +103,7 @@ public class UpgradeIndexMergePolicy extends MergePolicy { if (verbose()) message("findMergesForOptimize: " + base.getClass().getSimpleName() + " does not want to merge all old segments, merge remaining ones into new segment: " + oldSegments); - final SegmentInfos newInfos = new SegmentInfos(); + final List newInfos = new ArrayList(); for (final SegmentInfo si : segmentInfos) { if (oldSegments.contains(si)) { newInfos.add(si); diff --git a/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java b/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java index f2ee32950c8..0cc621aff6d 100644 --- a/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java +++ b/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java @@ -18,7 +18,9 @@ package org.apache.lucene.index; */ import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Random; import java.util.Set; @@ -58,7 +60,7 @@ public class MockRandomMergePolicy extends MergePolicy { SegmentInfos segmentInfos, int maxSegmentCount, Set segmentsToOptimize) throws CorruptIndexException, IOException { - final SegmentInfos eligibleSegments = new SegmentInfos(); + final List eligibleSegments = new ArrayList(); for(SegmentInfo info : segmentInfos) { if (segmentsToOptimize.contains(info)) { eligibleSegments.add(info); @@ -76,7 +78,7 @@ public class MockRandomMergePolicy extends MergePolicy { while(upto < eligibleSegments.size()) { int max = Math.min(10, eligibleSegments.size()-upto); int inc = max <= 2 ? max : _TestUtil.nextInt(random, 2, max); - mergeSpec.add(new OneMerge(eligibleSegments.range(upto, upto+inc))); + mergeSpec.add(new OneMerge(eligibleSegments.subList(upto, upto+inc))); upto += inc; } }