Index: lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java =================================================================== --- lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java (revision 1104493) +++ lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java (working copy) @@ -19,6 +19,7 @@ import java.io.IOException; +import java.util.Collections; import java.util.Set; /** @@ -135,7 +136,7 @@ if (last > 1 || !isOptimized(infos.info(0))) { spec = new MergeSpecification(); - spec.add(new OneMerge(infos.range(0, last))); + spec.add(new OneMerge(infos.asList().subList(0, last))); } } else if (last > maxNumSegments) { @@ -192,7 +193,7 @@ prev = backLink[i][prev]; int mergeStart = i + prev; if((mergeEnd - mergeStart) > 1) { - spec.add(new OneMerge(infos.range(mergeStart, mergeEnd))); + spec.add(new OneMerge(infos.asList().subList(mergeStart, mergeEnd))); } else { if(partialExpunge) { SegmentInfo info = infos.info(mergeStart); @@ -208,7 +209,7 @@ if(partialExpunge && maxDelCount > 0) { // expunge deletes - spec.add(new OneMerge(infos.range(expungeCandidate, expungeCandidate + 1))); + spec.add(new OneMerge(Collections.singletonList(infos.info(expungeCandidate)))); } return spec; @@ -250,7 +251,10 @@ MergeSpecification spec = null; if(numLargeSegs < numSegs) { - SegmentInfos smallSegments = infos.range(numLargeSegs, numSegs); + // hack to create a shallow sub-range as SegmentInfos instance, + // it does not clone all metadata, but LogMerge does not need it + final SegmentInfos smallSegments = new SegmentInfos(); + smallSegments.rollbackSegmentInfos(infos.asList().subList(numLargeSegs, numSegs)); spec = super.findMergesToExpungeDeletes(smallSegments); } @@ -258,7 +262,7 @@ for(int i = 0; i < numLargeSegs; i++) { SegmentInfo info = infos.info(i); if(info.hasDeletions()) { - spec.add(new OneMerge(infos.range(i, i + 1))); + spec.add(new OneMerge(Collections.singletonList(infos.info(i)))); } } return spec; @@ -296,7 +300,7 @@ if(totalSmallSegSize < targetSegSize * 2) { MergeSpecification spec = findBalancedMerges(infos, numLargeSegs, (numLargeSegs - 1), _partialExpunge); if(spec == null) spec = new MergeSpecification(); // should not happen - spec.add(new OneMerge(infos.range(numLargeSegs, numSegs))); + spec.add(new OneMerge(infos.asList().subList(numLargeSegs, numSegs))); return spec; } else { return findBalancedMerges(infos, numSegs, numLargeSegs, _partialExpunge); @@ -311,11 +315,13 @@ if(size(info) < sizeThreshold) break; startSeg++; } - spec.add(new OneMerge(infos.range(startSeg, numSegs))); + spec.add(new OneMerge(infos.asList().subList(startSeg, numSegs))); return spec; } else { - // apply the log merge policy to small segments. - SegmentInfos smallSegments = infos.range(numLargeSegs, numSegs); + // hack to create a shallow sub-range as SegmentInfos instance, + // it does not clone all metadata, but LogMerge does not need it + final SegmentInfos smallSegments = new SegmentInfos(); + smallSegments.rollbackSegmentInfos(infos.asList().subList(numLargeSegs, numSegs)); MergeSpecification spec = super.findMerges(smallSegments); if(_partialExpunge) { @@ -342,7 +348,7 @@ } } if (maxDelCount > 0) { - return new OneMerge(infos.range(expungeCandidate, expungeCandidate + 1)); + return new OneMerge(Collections.singletonList(infos.info(expungeCandidate))); } return null; } Index: lucene/src/java/org/apache/lucene/index/DirectoryReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/DirectoryReader.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/DirectoryReader.java (working copy) @@ -732,8 +732,7 @@ // case we have to roll back: startCommit(); - final SegmentInfos rollbackSegmentInfos = new SegmentInfos(); - rollbackSegmentInfos.addAll(segmentInfos); + final List rollbackSegments = segmentInfos.createBackupSegmentInfos(false); boolean success = false; try { @@ -765,8 +764,7 @@ deleter.refresh(); // Restore all SegmentInfos (in case we pruned some) - segmentInfos.clear(); - segmentInfos.addAll(rollbackSegmentInfos); + segmentInfos.rollbackSegmentInfos(rollbackSegments); } } Index: lucene/src/java/org/apache/lucene/index/IndexWriter.java =================================================================== --- lucene/src/java/org/apache/lucene/index/IndexWriter.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/IndexWriter.java (working copy) @@ -22,6 +22,7 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -221,7 +222,7 @@ private volatile long changeCount; // increments every time a change is completed private long lastCommitChangeCount; // last changeCount that was committed - private SegmentInfos rollbackSegmentInfos; // segmentInfos we will fallback to if the commit fails + private List rollbackSegments; // list of segmentInfo we will fallback to if the commit fails volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit()) volatile long pendingCommitChangeCount; @@ -440,14 +441,14 @@ public synchronized boolean infoIsLive(SegmentInfo info) { int idx = segmentInfos.indexOf(info); assert idx != -1: "info=" + info + " isn't in pool"; - assert segmentInfos.get(idx) == info: "info=" + info + " doesn't match live info in segmentInfos"; + assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos"; return true; } public synchronized SegmentInfo mapToLive(SegmentInfo info) { int idx = segmentInfos.indexOf(info); if (idx != -1) { - info = segmentInfos.get(idx); + info = segmentInfos.info(idx); } return info; } @@ -818,7 +819,7 @@ } } - setRollbackSegmentInfos(segmentInfos); + rollbackSegments = segmentInfos.createBackupSegmentInfos(true); // start with previous field numbers, but new FieldInfos globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory); @@ -862,10 +863,6 @@ } } - private synchronized void setRollbackSegmentInfos(SegmentInfos infos) { - rollbackSegmentInfos = (SegmentInfos) infos.clone(); - } - /** * Returns the private {@link IndexWriterConfig}, cloned * from the {@link IndexWriterConfig} passed to @@ -1126,8 +1123,7 @@ else count = 0; - for (int i = 0; i < segmentInfos.size(); i++) - count += segmentInfos.info(i).docCount; + count += segmentInfos.totalDocCount(); return count; } @@ -1144,8 +1140,7 @@ else count = 0; - for (int i = 0; i < segmentInfos.size(); i++) { - final SegmentInfo info = segmentInfos.info(i); + for (final SegmentInfo info : segmentInfos) { count += info.docCount - numDeletedDocs(info); } return count; @@ -1159,9 +1154,11 @@ if (docWriter.anyDeletions()) { return true; } - for (int i = 0; i < segmentInfos.size(); i++) - if (segmentInfos.info(i).hasDeletions()) + for (final SegmentInfo info : segmentInfos) { + if (info.hasDeletions()) { return true; + } + } return false; } @@ -1554,7 +1551,8 @@ synchronized(this) { resetMergeExceptions(); - segmentsToOptimize = new HashSet(segmentInfos); + segmentsToOptimize.clear(); + segmentsToOptimize.addAll(segmentInfos.asSet()); optimizeMaxNumSegments = maxNumSegments; // Now mark all pending & running merges as optimize @@ -1778,7 +1776,7 @@ final MergePolicy.MergeSpecification spec; if (optimize) { - spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, segmentsToOptimize); + spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, Collections.unmodifiableSet(segmentsToOptimize)); if (spec != null) { final int numMerges = spec.merges.size(); @@ -1889,8 +1887,7 @@ // attempt to commit using this instance of IndexWriter // will always write to a new generation ("write // once"). - segmentInfos.clear(); - segmentInfos.addAll(rollbackSegmentInfos); + segmentInfos.rollbackSegmentInfos(rollbackSegments); docWriter.abort(); @@ -2555,7 +2552,7 @@ lastCommitChangeCount = pendingCommitChangeCount; segmentInfos.updateGeneration(pendingCommit); segmentInfos.setUserData(pendingCommit.getUserData()); - setRollbackSegmentInfos(pendingCommit); + rollbackSegments = segmentInfos.createBackupSegmentInfos(true); deleter.checkpoint(pendingCommit, true); } finally { // Matches the incRef done in startCommit: @@ -2660,7 +2657,7 @@ final synchronized void applyAllDeletes() throws IOException { flushDeletesCount.incrementAndGet(); final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream - .applyDeletes(readerPool, segmentInfos); + .applyDeletes(readerPool, segmentInfos.asList()); if (result.anyDeletes) { checkpoint(); } @@ -2709,7 +2706,7 @@ private void ensureValidMerge(MergePolicy.OneMerge merge) throws IOException { for(SegmentInfo info : merge.segments) { - if (segmentInfos.indexOf(info) == -1) { + if (!segmentInfos.contains(info)) { throw new MergePolicy.MergeException("MergePolicy selected a segment (" + info.name + ") that is not in the current index " + segString(), directory); } } @@ -2847,39 +2844,13 @@ message("merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert")); } - final Set mergedAway = new HashSet(merge.segments); - int segIdx = 0; - int newSegIdx = 0; - boolean inserted = false; - final int curSegCount = segmentInfos.size(); - while(segIdx < curSegCount) { - final SegmentInfo info = segmentInfos.info(segIdx++); - if (mergedAway.contains(info)) { - if (!inserted && (!allDeleted || keepFullyDeletedSegments)) { - segmentInfos.set(segIdx-1, merge.info); - inserted = true; - newSegIdx++; - } - } else { - segmentInfos.set(newSegIdx++, info); - } + final boolean dropSegment = allDeleted && !keepFullyDeletedSegments; + segmentInfos.applyMergeChanges(merge, dropSegment); + + if (dropSegment) { + readerPool.drop(merge.info); } - - // Either we found place to insert segment, or, we did - // not, but only because all segments we merged became - // deleted while we are merging, in which case it should - // be the case that the new segment is also all deleted: - if (!inserted) { - assert allDeleted; - if (keepFullyDeletedSegments) { - segmentInfos.add(0, merge.info); - } else { - readerPool.drop(merge.info); - } - } - - segmentInfos.subList(newSegIdx, segmentInfos.size()).clear(); - + if (infoStream != null) { message("after commit: " + segString()); } @@ -3014,7 +2985,7 @@ if (mergingSegments.contains(info)) { return false; } - if (segmentInfos.indexOf(info) == -1) { + if (!segmentInfos.contains(info)) { return false; } if (info.dir != directory) { @@ -3462,7 +3433,7 @@ } // utility routines for tests - SegmentInfo newestSegment() { + synchronized SegmentInfo newestSegment() { return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size()-1) : null; } @@ -3472,16 +3443,14 @@ } /** @lucene.internal */ - 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) { + public synchronized String segString(Iterable infos) throws IOException { + final StringBuilder buffer = new StringBuilder(); + for(final SegmentInfo s : infos) { + if (buffer.length() > 0) { buffer.append(' '); } - buffer.append(segString(infos.get(i))); + buffer.append(segString(s)); } - return buffer.toString(); } Index: lucene/src/java/org/apache/lucene/index/LogMergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (working copy) @@ -242,6 +242,7 @@ private MergeSpecification findMergesForOptimizeSizeLimit( SegmentInfos infos, int maxNumSegments, int last) throws IOException { MergeSpecification spec = new MergeSpecification(); + final List segments = infos.asList(); int start = last - 1; while (start >= 0) { @@ -254,12 +255,12 @@ // unless there is only 1 which is optimized. if (last - start - 1 > 1 || (start != last - 1 && !isOptimized(infos.info(start + 1)))) { // there is more than 1 segment to the right of this one, or an unoptimized single segment. - spec.add(new OneMerge(infos.range(start + 1, last))); + spec.add(new OneMerge(segments.subList(start + 1, last))); } last = start; } else if (last - start == mergeFactor) { // mergeFactor eligible segments were found, add them as a merge. - spec.add(new OneMerge(infos.range(start, last))); + spec.add(new OneMerge(segments.subList(start, last))); last = start; } --start; @@ -267,7 +268,7 @@ // Add any left-over segments, unless there is just 1 already optimized. if (last > 0 && (++start + 1 < last || !isOptimized(infos.info(start)))) { - spec.add(new OneMerge(infos.range(start, last))); + spec.add(new OneMerge(segments.subList(start, last))); } return spec.merges.size() == 0 ? null : spec; @@ -280,11 +281,12 @@ */ private MergeSpecification findMergesForOptimizeMaxNumSegments(SegmentInfos infos, int maxNumSegments, int last) throws IOException { MergeSpecification spec = new MergeSpecification(); + final List segments = infos.asList(); // First, enroll all "full" merges (size // mergeFactor) to potentially be run concurrently: while (last - maxNumSegments + 1 >= mergeFactor) { - spec.add(new OneMerge(infos.range(last - mergeFactor, last))); + spec.add(new OneMerge(segments.subList(last - mergeFactor, last))); last -= mergeFactor; } @@ -296,7 +298,7 @@ // Since we must optimize down to 1 segment, the // choice is simple: if (last > 1 || !isOptimized(infos.info(0))) { - spec.add(new OneMerge(infos.range(0, last))); + spec.add(new OneMerge(segments.subList(0, last))); } } else if (last > maxNumSegments) { @@ -325,7 +327,7 @@ } } - spec.add(new OneMerge(infos.range(bestStart, bestStart + finalMergeSize))); + spec.add(new OneMerge(segments.subList(bestStart, bestStart + finalMergeSize))); } } return spec.merges.size() == 0 ? null : spec; @@ -412,7 +414,8 @@ @Override public MergeSpecification findMergesToExpungeDeletes(SegmentInfos segmentInfos) throws CorruptIndexException, IOException { - final int numSegments = segmentInfos.size(); + final List segments = segmentInfos.asList(); + final int numSegments = segments.size(); if (verbose()) message("findMergesToExpungeDeletes: " + numSegments + " segments"); @@ -434,7 +437,7 @@ // deletions, so force a merge now: if (verbose()) message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive"); - spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i))); + spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i))); firstSegmentWithDeletions = i; } } else if (firstSegmentWithDeletions != -1) { @@ -443,7 +446,7 @@ // mergeFactor segments if (verbose()) message(" add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive"); - spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i))); + spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i))); firstSegmentWithDeletions = -1; } } @@ -451,7 +454,7 @@ if (firstSegmentWithDeletions != -1) { if (verbose()) message(" add merge " + firstSegmentWithDeletions + " to " + (numSegments-1) + " inclusive"); - spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, numSegments))); + spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, numSegments))); } return spec; Index: lucene/src/java/org/apache/lucene/index/MergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/MergePolicy.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/MergePolicy.java (working copy) @@ -84,7 +84,8 @@ public OneMerge(List segments) { if (0 == segments.size()) throw new RuntimeException("segments must include at least one segment"); - this.segments = segments; + // clone the list, as the in list may be based off original SegmentInfos and may be modified + this.segments = new ArrayList(segments); int count = 0; for(SegmentInfo info : segments) { count += info.docCount; Index: lucene/src/java/org/apache/lucene/index/SegmentInfo.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentInfo.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/SegmentInfo.java (working copy) @@ -40,7 +40,7 @@ * * @lucene.experimental */ -public final class SegmentInfo { +public final class SegmentInfo implements Cloneable { // TODO: remove with hasVector and hasProx private static final int CHECK_FIELDINFO = -2; static final int NO = -1; // e.g. no norms; no deletes; Index: lucene/src/java/org/apache/lucene/index/SegmentInfos.java =================================================================== --- lucene/src/java/org/apache/lucene/index/SegmentInfos.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/SegmentInfos.java (working copy) @@ -20,13 +20,16 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.PrintStream; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Map; -import java.util.Vector; +import java.util.Set; import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.index.codecs.CodecProvider; @@ -45,7 +48,7 @@ * * @lucene.experimental */ -public final class SegmentInfos extends Vector { +public final class SegmentInfos implements Cloneable, Iterable { /* * The file format version, a negative number. @@ -84,7 +87,11 @@ private int format; private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand - + + private List segments = new ArrayList(); + private Map segMap = new HashMap(); + private boolean mapIndexesInvalid = false; + /** * If non-null, information about loading segments_N files * will be printed here. @see #setInfoStream. @@ -107,8 +114,8 @@ return format; } - public final SegmentInfo info(int i) { - return get(i); + public SegmentInfo info(int i) { + return segments.get(i); } /** @@ -237,7 +244,7 @@ boolean success = false; // Clear any previous segments: - clear(); + this.clear(); generation = generationFromSegmentsFileName(segmentFileName); @@ -252,7 +259,7 @@ if (!success) { // Clear any segment infos we had loaded so we // have a clean slate on retry: - clear(); + this.clear(); } } } @@ -349,15 +356,15 @@ /** Prunes any segment whose docs are all deleted. */ public void pruneDeletedSegments() { - int segIdx = 0; - while(segIdx < size()) { - final SegmentInfo info = info(segIdx); + for(final Iterator it = segments.iterator(); it.hasNext();) { + final SegmentInfo info = it.next(); if (info.getDelCount() == info.docCount) { - remove(segIdx); - } else { - segIdx++; + mapIndexesInvalid = true; + it.remove(); + segMap.remove(info); } } + assert segMap.size() == segments.size(); } /** @@ -367,14 +374,22 @@ @Override public Object clone() { - SegmentInfos sis = (SegmentInfos) super.clone(); - for(int i=0;i(size()); + sis.segMap = new HashMap(size()); + sis.mapIndexesInvalid = false; + for(final SegmentInfo info : this) { + assert info.getSegmentCodecs() != null; + // dont directly access segments, use add method!!! + sis.add((SegmentInfo) info.clone()); + } + sis.userData = new HashMap(userData); + return sis; + } catch (CloneNotSupportedException e) { + throw new RuntimeException("should not happen", e); } - sis.userData = new HashMap(userData); - return sis; } /** @@ -742,18 +757,6 @@ protected abstract Object doBody(String segmentFileName) throws CorruptIndexException, IOException; } - /** - * Returns a new SegmentInfos containing the SegmentInfo - * instances in the specified range first (inclusive) to - * last (exclusive), so total number of segments returned - * is last-first. - */ - public SegmentInfos range(int first, int last) { - SegmentInfos infos = new SegmentInfos(codecs); - infos.addAll(super.subList(first, last)); - return infos; - } - // Carry over generation numbers from another SegmentInfos void updateGeneration(SegmentInfos other) { lastGeneration = other.lastGeneration; @@ -960,7 +963,7 @@ } - public synchronized String toString(Directory directory) { + public String toString(Directory directory) { StringBuilder buffer = new StringBuilder(); buffer.append(getCurrentSegmentFileName()).append(": "); final int count = size(); @@ -991,8 +994,7 @@ * remain write once. */ void replace(SegmentInfos other) { - clear(); - addAll(other); + rollbackSegmentInfos(other.asList()); lastGeneration = other.lastGeneration; lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion; format = other.format; @@ -1018,7 +1020,7 @@ * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}. * If this {@link SegmentInfos} has no global field number map the returned instance is empty */ - synchronized FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException { + FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException { if (globalFieldNumberMap != null) { return globalFieldNumberMap; } @@ -1058,4 +1060,152 @@ long getLastGlobalFieldMapVersion() { return lastGlobalFieldMapVersion; } + + /** applies all changes caused by committing a merge to this SegmentInfos */ + void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) { + try { + final Set mergedAway = new HashSet(merge.segments); + boolean inserted = false; + int newSegIdx = 0; + for (int segIdx = 0, cnt = segments.size(); segIdx < cnt; segIdx++) { + assert segIdx >= newSegIdx; + final SegmentInfo info = segments.get(segIdx); + if (mergedAway.contains(info)) { + if (!inserted && !dropSegment) { + segments.set(segIdx, merge.info); + inserted = true; + newSegIdx++; + } + } else { + segments.set(newSegIdx, info); + newSegIdx++; + } + } + + // Either we found place to insert segment, or, we did + // not, but only because all segments we merged became + // deleted while we are merging, in which case it should + // be the case that the new segment is also all deleted, + // we insert it at the beginning if it should not be dropped: + if (!inserted && !dropSegment) { + segments.add(0, merge.info); + } + + // the rest of the segments in list are duplicates, so don't remove from map, only list! + segments.subList(newSegIdx, segments.size()).clear(); + + // update the Map + if (!dropSegment) { + segMap.put(merge.info, -1); // dummy index (mapIndexesInvalid:=true) + } + segMap.keySet().removeAll(mergedAway); + + assert segMap.size() == segments.size(); + } finally { + // all indexes are for sure wrong now + mapIndexesInvalid = true; + } + } + + List createBackupSegmentInfos(boolean cloneChilds) { + if (cloneChilds) { + final List list = new ArrayList(size()); + for(final SegmentInfo info : this) { + assert info.getSegmentCodecs() != null; + list.add((SegmentInfo) info.clone()); + } + return list; + } else { + return new ArrayList(segments); + } + } + + void rollbackSegmentInfos(List infos) { + this.clear(); + this.addAll(infos); + } + + /** Returns an unmodifiable {@link Iterator} of contained segments in order. */ + // @Override (comment out until Java 6) + public Iterator iterator() { + return asList().iterator(); + } + + /** Returns all contained segments as an unmodifiable {@link List} view. */ + public List asList() { + return Collections.unmodifiableList(segments); + } + + /** Returns all contained segments as an unmodifiable {@link Set} view. */ + public Set asSet() { + return Collections.unmodifiableSet(segMap.keySet()); + } + + public int size() { + return segments.size(); + } + + public void add(SegmentInfo si) { + if (segMap.containsKey(si)) { + throw new IllegalStateException("Cannot add the same segment two times to this SegmentInfos instance"); + } + segMap.put(si, segments.size()); + segments.add(si); + assert segMap.size() == segments.size(); + } + + public void addAll(Iterable sis) { + for (final SegmentInfo si : sis) { + this.add(si); + } + } + + public void clear() { + segments.clear(); + segMap.clear(); + // we know this for sure :-) + mapIndexesInvalid = false; + } + + public void remove(SegmentInfo si) { + final int index = this.indexOf(si); + if (index >= 0) { + this.remove(index); + } + } + + public void remove(int index) { + try { + segMap.remove(segments.remove(index)); + assert segMap.size() == segments.size(); + } finally { + mapIndexesInvalid = true; + } + } + + public boolean contains(SegmentInfo si) { + return segMap.containsKey(si); + } + + public int indexOf(SegmentInfo si) { + // if the indexes in the map are invalid (because inserts/deletes), rebuild + if (mapIndexesInvalid) { + segMap.clear(); + for (int i = 0, c = segments.size(); i < c; i++) { + segMap.put(segments.get(i), i); + } + assert segMap.size() == segments.size(); + mapIndexesInvalid = false; + } + + final Integer index = segMap.get(si); + if (index != null) { + final int i = index.intValue(); + assert si.equals(segments.get(i)); + return i; + } else { + return -1; + } + } + } Index: lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java =================================================================== --- lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (revision 1104493) +++ lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (working copy) @@ -251,9 +251,7 @@ final Collection merging = writer.get().getMergingSegments(); final Collection toBeMerged = new HashSet(); - final List infosSorted = new ArrayList(); - infosSorted.addAll(infos); - + final List infosSorted = new ArrayList(infos.asList()); Collections.sort(infosSorted, segmentByteSizeDescending); // Compute total index bytes & print details about the index Index: lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java =================================================================== --- lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java (revision 1104493) +++ lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java (working copy) @@ -42,14 +42,13 @@ if (segmentInfos.size() > 1 && random.nextInt(5) == 3) { - SegmentInfos segmentInfos2 = new SegmentInfos(); - segmentInfos2.addAll(segmentInfos); - Collections.shuffle(segmentInfos2, random); + List segments = new ArrayList(segmentInfos.asList()); + Collections.shuffle(segments, random); // TODO: sometimes make more than 1 merge? mergeSpec = new MergeSpecification(); final int segsToMerge = _TestUtil.nextInt(random, 1, segmentInfos.size()); - mergeSpec.add(new OneMerge(segmentInfos2.range(0, segsToMerge))); + mergeSpec.add(new OneMerge(segments.subList(0, segsToMerge))); } return mergeSpec; Index: lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java (revision 1104493) +++ lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java (working copy) @@ -363,7 +363,7 @@ w.close(); SegmentInfos sis = new SegmentInfos(); sis.read(base); - SegmentInfo segmentInfo = sis.get(sis.size() - 1);// last segment must + SegmentInfo segmentInfo = sis.info(sis.size() - 1);// last segment must // have all fields with // consistent numbers FieldInfos fieldInfos = segmentInfo.getFieldInfos(); Index: lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java =================================================================== --- lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (revision 1104493) +++ lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (working copy) @@ -128,8 +128,8 @@ fsmp.length = 2; System.out.println("maybeMerge "+writer.segmentInfos); - SegmentInfo info0 = writer.segmentInfos.get(0); - SegmentInfo info1 = writer.segmentInfos.get(1); + SegmentInfo info0 = writer.segmentInfos.info(0); + SegmentInfo info1 = writer.segmentInfos.info(1); writer.maybeMerge(); System.out.println("maybeMerge after "+writer.segmentInfos); @@ -199,7 +199,7 @@ // deletes for info1, the newly created segment from the // merge should have no deletes because they were applied in // the merge - //SegmentInfo info1 = writer.segmentInfos.get(1); + //SegmentInfo info1 = writer.segmentInfos.info(1); //assertFalse(exists(info1, writer.docWriter.segmentDeletes)); //System.out.println("infos4:"+writer.segmentInfos); @@ -261,11 +261,7 @@ throws CorruptIndexException, IOException { MergeSpecification ms = new MergeSpecification(); if (doMerge) { - SegmentInfos mergeInfos = new SegmentInfos(); - for (int x=start; x < (start+length); x++) { - mergeInfos.add(segmentInfos.get(x)); - } - OneMerge om = new OneMerge(mergeInfos); + OneMerge om = new OneMerge(segmentInfos.asList().subList(start, start + length)); ms.add(om); doMerge = false; return ms;