You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/05/18 17:57:36 UTC
svn commit: r1124307 - in /lucene/dev/trunk/lucene: ./
contrib/misc/src/java/org/apache/lucene/index/
src/java/org/apache/lucene/index/ src/test-framework/org/apache/lucene/index/
src/test/org/apache/lucene/index/
Author: uschindler
Date: Wed May 18 15:57:36 2011
New Revision: 1124307
URL: http://svn.apache.org/viewvc?rev=1124307&view=rev
Log:
LUCENE-3084: Eliminate Vector subclassing in SegmentInfos, redesign API.
Modified:
lucene/dev/trunk/lucene/CHANGES.txt
lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed May 18 15:57:36 2011
@@ -478,9 +478,15 @@ Changes in backwards compatibility polic
(Mike McCandless, Shai Erera)
* LUCENE-3084: MergePolicy.OneMerge.segments was changed from
- SegmentInfos to a List<SegmentInfo>; this is actually a minor change
- because SegmentInfos itself extends Vector<SegmentInfo>. (Uwe
- Schindler, Mike McCandless)
+ SegmentInfos to a List<SegmentInfo>. SegmentInfos itsself was changed
+ to no longer extend Vector<SegmentInfo> (to update code that is using
+ Vector-API, use the new asList() and asSet() methods returning unmodifiable
+ collections; modifying SegmentInfos is now only possible through
+ the explicitely declared methods). IndexWriter.segString() now takes
+ Iterable<SegmentInfo> instead of List<SegmentInfo>. A simple recompile
+ should fix this. MergePolicy and SegmentInfos are internal/experimental
+ APIs not covered by the strict backwards compatibility policy.
+ (Uwe Schindler, Mike McCandless)
Changes in runtime behavior
Modified: lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java Wed May 18 15:57:36 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.IOException;
+import java.util.Collections;
import java.util.Set;
/**
@@ -135,7 +136,7 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
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 @@ public class BalancedSegmentMergePolicy
}
}
if (maxDelCount > 0) {
- return new OneMerge(infos.range(expungeCandidate, expungeCandidate + 1));
+ return new OneMerge(Collections.singletonList(infos.info(expungeCandidate)));
}
return null;
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Wed May 18 15:57:36 2011
@@ -732,8 +732,7 @@ class DirectoryReader extends IndexReade
// case we have to roll back:
startCommit();
- final SegmentInfos rollbackSegmentInfos = new SegmentInfos();
- rollbackSegmentInfos.addAll(segmentInfos);
+ final List<SegmentInfo> rollbackSegments = segmentInfos.createBackupSegmentInfos(false);
boolean success = false;
try {
@@ -765,8 +764,7 @@ class DirectoryReader extends IndexReade
deleter.refresh();
// Restore all SegmentInfos (in case we pruned some)
- segmentInfos.clear();
- segmentInfos.addAll(rollbackSegmentInfos);
+ segmentInfos.rollbackSegmentInfos(rollbackSegments);
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Wed May 18 15:57:36 2011
@@ -22,6 +22,7 @@ import java.io.IOException;
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 @@ public class IndexWriter implements Clos
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<SegmentInfo> 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 class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
}
}
- setRollbackSegmentInfos(segmentInfos);
+ rollbackSegments = segmentInfos.createBackupSegmentInfos(true);
// start with previous field numbers, but new FieldInfos
globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
@@ -862,10 +863,6 @@ public class IndexWriter implements Clos
}
}
- 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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
synchronized(this) {
resetMergeExceptions();
- segmentsToOptimize = new HashSet<SegmentInfo>(segmentInfos);
+ segmentsToOptimize.clear();
+ segmentsToOptimize.addAll(segmentInfos.asSet());
optimizeMaxNumSegments = maxNumSegments;
// Now mark all pending & running merges as optimize
@@ -1778,7 +1776,7 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
// 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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
message("merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert"));
}
- final Set<SegmentInfo> mergedAway = new HashSet<SegmentInfo>(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);
- }
- }
-
- // 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);
- }
+ final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
+ segmentInfos.applyMergeChanges(merge, dropSegment);
+
+ if (dropSegment) {
+ readerPool.drop(merge.info);
}
-
- segmentInfos.subList(newSegIdx, segmentInfos.size()).clear();
-
+
if (infoStream != null) {
message("after commit: " + segString());
}
@@ -3014,7 +2985,7 @@ public class IndexWriter implements Clos
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 @@ public class IndexWriter implements Clos
}
// utility routines for tests
- SegmentInfo newestSegment() {
+ synchronized SegmentInfo newestSegment() {
return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size()-1) : null;
}
@@ -3472,16 +3443,14 @@ public class IndexWriter implements Clos
}
/** @lucene.internal */
- public synchronized String segString(List<SegmentInfo> 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<SegmentInfo> 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();
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Wed May 18 15:57:36 2011
@@ -242,6 +242,7 @@ public abstract class LogMergePolicy ext
private MergeSpecification findMergesForOptimizeSizeLimit(
SegmentInfos infos, int maxNumSegments, int last) throws IOException {
MergeSpecification spec = new MergeSpecification();
+ final List<SegmentInfo> segments = infos.asList();
int start = last - 1;
while (start >= 0) {
@@ -254,12 +255,12 @@ public abstract class LogMergePolicy ext
// 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 @@ public abstract class LogMergePolicy ext
// 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 @@ public abstract class LogMergePolicy ext
*/
private MergeSpecification findMergesForOptimizeMaxNumSegments(SegmentInfos infos, int maxNumSegments, int last) throws IOException {
MergeSpecification spec = new MergeSpecification();
+ final List<SegmentInfo> 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 @@ public abstract class LogMergePolicy ext
// 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 @@ public abstract class LogMergePolicy ext
}
}
- 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 @@ public abstract class LogMergePolicy ext
@Override
public MergeSpecification findMergesToExpungeDeletes(SegmentInfos segmentInfos)
throws CorruptIndexException, IOException {
- final int numSegments = segmentInfos.size();
+ final List<SegmentInfo> segments = segmentInfos.asList();
+ final int numSegments = segments.size();
if (verbose())
message("findMergesToExpungeDeletes: " + numSegments + " segments");
@@ -434,7 +437,7 @@ public abstract class LogMergePolicy ext
// 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 @@ public abstract class LogMergePolicy ext
// 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 @@ public abstract class LogMergePolicy ext
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;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java Wed May 18 15:57:36 2011
@@ -84,7 +84,8 @@ public abstract class MergePolicy implem
public OneMerge(List<SegmentInfo> 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<SegmentInfo>(segments);
int count = 0;
for(SegmentInfo info : segments) {
count += info.docCount;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Wed May 18 15:57:36 2011
@@ -40,7 +40,7 @@ import org.apache.lucene.util.Constants;
*
* @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;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Wed May 18 15:57:36 2011
@@ -20,13 +20,16 @@ package org.apache.lucene.index;
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 @@ import org.apache.lucene.util.ThreadInte
*
* @lucene.experimental
*/
-public final class SegmentInfos extends Vector<SegmentInfo> {
+public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
/*
* The file format version, a negative number.
@@ -84,7 +87,12 @@ public final class SegmentInfos extends
private int format;
private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand
-
+
+ private List<SegmentInfo> segments = new ArrayList<SegmentInfo>();
+ private Set<SegmentInfo> segmentSet = new HashSet<SegmentInfo>();
+ private transient List<SegmentInfo> cachedUnmodifiableList;
+ private transient Set<SegmentInfo> cachedUnmodifiableSet;
+
/**
* If non-null, information about loading segments_N files
* will be printed here. @see #setInfoStream.
@@ -107,8 +115,8 @@ public final class SegmentInfos extends
return format;
}
- public final SegmentInfo info(int i) {
- return get(i);
+ public SegmentInfo info(int i) {
+ return segments.get(i);
}
/**
@@ -237,7 +245,7 @@ public final class SegmentInfos extends
boolean success = false;
// Clear any previous segments:
- clear();
+ this.clear();
generation = generationFromSegmentsFileName(segmentFileName);
@@ -252,7 +260,7 @@ public final class SegmentInfos extends
if (!success) {
// Clear any segment infos we had loaded so we
// have a clean slate on retry:
- clear();
+ this.clear();
}
}
}
@@ -349,15 +357,14 @@ public final class SegmentInfos extends
/** 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<SegmentInfo> it = segments.iterator(); it.hasNext();) {
+ final SegmentInfo info = it.next();
if (info.getDelCount() == info.docCount) {
- remove(segIdx);
- } else {
- segIdx++;
+ it.remove();
+ segmentSet.remove(info);
}
}
+ assert segmentSet.size() == segments.size();
}
/**
@@ -367,14 +374,23 @@ public final class SegmentInfos extends
@Override
public Object clone() {
- SegmentInfos sis = (SegmentInfos) super.clone();
- for(int i=0;i<sis.size();i++) {
- final SegmentInfo info = sis.info(i);
- assert info.getSegmentCodecs() != null;
- sis.set(i, (SegmentInfo) info.clone());
+ try {
+ final SegmentInfos sis = (SegmentInfos) super.clone();
+ // deep clone, first recreate all collections:
+ sis.segments = new ArrayList<SegmentInfo>(size());
+ sis.segmentSet = new HashSet<SegmentInfo>(size());
+ sis.cachedUnmodifiableList = null;
+ sis.cachedUnmodifiableSet = null;
+ 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<String,String>(userData);
+ return sis;
+ } catch (CloneNotSupportedException e) {
+ throw new RuntimeException("should not happen", e);
}
- sis.userData = new HashMap<String,String>(userData);
- return sis;
}
/**
@@ -742,18 +758,6 @@ public final class SegmentInfos extends
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 +964,7 @@ public final class SegmentInfos extends
}
- 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 +995,7 @@ public final class SegmentInfos extends
* remain write once.
*/
void replace(SegmentInfos other) {
- clear();
- addAll(other);
+ rollbackSegmentInfos(other.asList());
lastGeneration = other.lastGeneration;
lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
format = other.format;
@@ -1018,7 +1021,7 @@ public final class SegmentInfos extends
* 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 +1061,135 @@ public final class SegmentInfos extends
long getLastGlobalFieldMapVersion() {
return lastGlobalFieldMapVersion;
}
+
+ /** applies all changes caused by committing a merge to this SegmentInfos */
+ void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
+ final Set<SegmentInfo> mergedAway = new HashSet<SegmentInfo>(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 Set
+ if (!dropSegment) {
+ segmentSet.add(merge.info);
+ }
+ segmentSet.removeAll(mergedAway);
+
+ assert segmentSet.size() == segments.size();
+ }
+
+ List<SegmentInfo> createBackupSegmentInfos(boolean cloneChildren) {
+ if (cloneChildren) {
+ final List<SegmentInfo> list = new ArrayList<SegmentInfo>(size());
+ for(final SegmentInfo info : this) {
+ assert info.getSegmentCodecs() != null;
+ list.add((SegmentInfo) info.clone());
+ }
+ return list;
+ } else {
+ return new ArrayList<SegmentInfo>(segments);
+ }
+ }
+
+ void rollbackSegmentInfos(List<SegmentInfo> infos) {
+ this.clear();
+ this.addAll(infos);
+ }
+
+ /** Returns an <b>unmodifiable</b> {@link Iterator} of contained segments in order. */
+ // @Override (comment out until Java 6)
+ public Iterator<SegmentInfo> iterator() {
+ return asList().iterator();
+ }
+
+ /** Returns all contained segments as an <b>unmodifiable</b> {@link List} view. */
+ public List<SegmentInfo> asList() {
+ if (cachedUnmodifiableList == null) {
+ cachedUnmodifiableList = Collections.unmodifiableList(segments);
+ }
+ return cachedUnmodifiableList;
+ }
+
+ /** Returns all contained segments as an <b>unmodifiable</b> {@link Set} view.
+ * The iterator is not sorted, use {@link List} view or {@link #iterator} to get all segments in order. */
+ public Set<SegmentInfo> asSet() {
+ if (cachedUnmodifiableSet == null) {
+ cachedUnmodifiableSet = Collections.unmodifiableSet(segmentSet);
+ }
+ return cachedUnmodifiableSet;
+ }
+
+ public int size() {
+ return segments.size();
+ }
+
+ public void add(SegmentInfo si) {
+ if (segmentSet.contains(si)) {
+ throw new IllegalStateException("Cannot add the same segment two times to this SegmentInfos instance");
+ }
+ segments.add(si);
+ segmentSet.add(si);
+ assert segmentSet.size() == segments.size();
+ }
+
+ public void addAll(Iterable<SegmentInfo> sis) {
+ for (final SegmentInfo si : sis) {
+ this.add(si);
+ }
+ }
+
+ public void clear() {
+ segments.clear();
+ segmentSet.clear();
+ }
+
+ public void remove(SegmentInfo si) {
+ final int index = this.indexOf(si);
+ if (index >= 0) {
+ this.remove(index);
+ }
+ }
+
+ public void remove(int index) {
+ segmentSet.remove(segments.remove(index));
+ assert segmentSet.size() == segments.size();
+ }
+
+ public boolean contains(SegmentInfo si) {
+ return segmentSet.contains(si);
+ }
+
+ public int indexOf(SegmentInfo si) {
+ if (segmentSet.contains(si)) {
+ return segments.indexOf(si);
+ } else {
+ return -1;
+ }
+ }
+
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java Wed May 18 15:57:36 2011
@@ -251,9 +251,7 @@ public class TieredMergePolicy extends M
final Collection<SegmentInfo> merging = writer.get().getMergingSegments();
final Collection<SegmentInfo> toBeMerged = new HashSet<SegmentInfo>();
- final List<SegmentInfo> infosSorted = new ArrayList<SegmentInfo>();
- infosSorted.addAll(infos);
-
+ final List<SegmentInfo> infosSorted = new ArrayList<SegmentInfo>(infos.asList());
Collections.sort(infosSorted, segmentByteSizeDescending);
// Compute total index bytes & print details about the index
Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java Wed May 18 15:57:36 2011
@@ -42,14 +42,13 @@ public class MockRandomMergePolicy exten
if (segmentInfos.size() > 1 && random.nextInt(5) == 3) {
- SegmentInfos segmentInfos2 = new SegmentInfos();
- segmentInfos2.addAll(segmentInfos);
- Collections.shuffle(segmentInfos2, random);
+ List<SegmentInfo> segments = new ArrayList<SegmentInfo>(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;
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java Wed May 18 15:57:36 2011
@@ -363,7 +363,7 @@ public class TestGlobalFieldNumbers exte
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();
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java?rev=1124307&r1=1124306&r2=1124307&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java Wed May 18 15:57:36 2011
@@ -128,8 +128,8 @@ public class TestPerSegmentDeletes exten
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 @@ public class TestPerSegmentDeletes exten
// 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 @@ public class TestPerSegmentDeletes exten
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;