You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2009/01/05 23:31:05 UTC
svn commit: r731747 - in /lucene/java/trunk/src:
java/org/apache/lucene/index/ test/org/apache/lucene/index/
Author: mikemccand
Date: Mon Jan 5 14:31:04 2009
New Revision: 731747
URL: http://svn.apache.org/viewvc?rev=731747&view=rev
Log:
LUCENE-1509: don't return dup file names from IndexCommit.getFileNames
Modified:
lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java
lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java
lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
Modified: lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/DirectoryIndexReader.java Mon Jan 5 14:31:04 2009
@@ -23,7 +23,8 @@
import java.util.HashSet;
import java.util.Collection;
import java.util.ArrayList;
-import java.util.List;
+import java.util.Iterator;
+import java.util.Collections;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.Lock;
@@ -64,12 +65,7 @@
if (!readOnly && segmentInfos != null) {
// We assume that this segments_N was previously
// properly sync'd:
- for(int i=0;i<segmentInfos.size();i++) {
- final SegmentInfo info = segmentInfos.info(i);
- List files = info.files();
- for(int j=0;j<files.size();j++)
- synced.add(files.get(j));
- }
+ synced.addAll(segmentInfos.files(directory, true));
}
}
@@ -284,16 +280,13 @@
commitChanges();
// Sync all files we just wrote
- for(int i=0;i<segmentInfos.size();i++) {
- final SegmentInfo info = segmentInfos.info(i);
- final List files = info.files();
- for(int j=0;j<files.size();j++) {
- final String fileName = (String) files.get(j);
- if (!synced.contains(fileName)) {
- assert directory.fileExists(fileName);
- directory.sync(fileName);
- synced.add(fileName);
- }
+ Iterator it = segmentInfos.files(directory, false).iterator();
+ while(it.hasNext()) {
+ final String fileName = (String) it.next();
+ if (!synced.contains(fileName)) {
+ assert directory.fileExists(fileName);
+ directory.sync(fileName);
+ synced.add(fileName);
}
}
@@ -426,15 +419,8 @@
ReaderCommit(SegmentInfos infos, Directory dir) throws IOException {
segmentsFileName = infos.getCurrentSegmentFileName();
this.dir = dir;
- final int size = infos.size();
userData = infos.getUserData();
- files = new ArrayList(size);
- files.add(segmentsFileName);
- for(int i=0;i<size;i++) {
- SegmentInfo info = infos.info(i);
- if (info.dir == dir)
- files.addAll(info.files());
- }
+ files = Collections.unmodifiableCollection(infos.files(dir, true));
version = infos.getVersion();
generation = infos.getGeneration();
isOptimized = infos.size() == 1 && !infos.info(0).hasDeletions();
Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexFileDeleter.java Mon Jan 5 14:31:04 2009
@@ -266,9 +266,9 @@
if (infoStream != null) {
message("deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
}
- int size2 = commit.files.size();
- for(int j=0;j<size2;j++) {
- decRef((String) commit.files.get(j));
+ Iterator it = commit.files.iterator();
+ while(it.hasNext()) {
+ decRef((String) it.next());
}
}
commitsToDelete.clear();
@@ -340,7 +340,7 @@
int size = lastFiles.size();
if (size > 0) {
for(int i=0;i<size;i++)
- decRef((List) lastFiles.get(i));
+ decRef((Collection) lastFiles.get(i));
lastFiles.clear();
}
@@ -419,55 +419,46 @@
int size = lastFiles.size();
if (size > 0) {
for(int i=0;i<size;i++)
- decRef((List) lastFiles.get(i));
+ decRef((Collection) lastFiles.get(i));
lastFiles.clear();
}
// Save files so we can decr on next checkpoint/commit:
- size = segmentInfos.size();
- for(int i=0;i<size;i++) {
- SegmentInfo segmentInfo = segmentInfos.info(i);
- if (segmentInfo.dir == directory) {
- lastFiles.add(segmentInfo.files());
- }
- }
+ lastFiles.add(segmentInfos.files(directory, false));
+
if (docWriterFiles != null)
lastFiles.add(docWriterFiles);
}
}
void incRef(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
- int size = segmentInfos.size();
- for(int i=0;i<size;i++) {
- SegmentInfo segmentInfo = segmentInfos.info(i);
- if (segmentInfo.dir == directory) {
- incRef(segmentInfo.files());
- }
- }
-
- if (isCommit) {
- // Since this is a commit point, also incref its
- // segments_N file:
- getRefCount(segmentInfos.getCurrentSegmentFileName()).IncRef();
+ // If this is a commit point, also incRef the
+ // segments_N file:
+ Iterator it = segmentInfos.files(directory, isCommit).iterator();
+ while(it.hasNext()) {
+ incRef((String) it.next());
}
}
void incRef(List files) throws IOException {
int size = files.size();
for(int i=0;i<size;i++) {
- String fileName = (String) files.get(i);
- RefCount rc = getRefCount(fileName);
- if (infoStream != null && VERBOSE_REF_COUNTS) {
- message(" IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
- }
- rc.IncRef();
+ incRef((String) files.get(i));
}
}
- void decRef(List files) throws IOException {
- int size = files.size();
- for(int i=0;i<size;i++) {
- decRef((String) files.get(i));
+ void incRef(String fileName) throws IOException {
+ RefCount rc = getRefCount(fileName);
+ if (infoStream != null && VERBOSE_REF_COUNTS) {
+ message(" IncRef \"" + fileName + "\": pre-incr count is " + rc.count);
+ }
+ rc.IncRef();
+ }
+
+ void decRef(Collection files) throws IOException {
+ Iterator it = files.iterator();
+ while(it.hasNext()) {
+ decRef((String) it.next());
}
}
@@ -485,12 +476,9 @@
}
void decRef(SegmentInfos segmentInfos) throws IOException {
- final int size = segmentInfos.size();
- for(int i=0;i<size;i++) {
- SegmentInfo segmentInfo = segmentInfos.info(i);
- if (segmentInfo.dir == directory) {
- decRef(segmentInfo.files());
- }
+ Iterator it = segmentInfos.files(directory, false).iterator();
+ while(it.hasNext()) {
+ decRef((String) it.next());
}
}
@@ -511,7 +499,7 @@
deleteFile((String) files.get(i));
}
- /** Delets the specified files, but only if they are new
+ /** Deletes the specified files, but only if they are new
* (have not yet been incref'd). */
void deleteNewFiles(Collection files) throws IOException {
final Iterator it = files.iterator();
@@ -577,7 +565,7 @@
final private static class CommitPoint extends IndexCommit implements Comparable {
long gen;
- List files;
+ Collection files;
String segmentsFileName;
boolean deleted;
Directory directory;
@@ -594,17 +582,11 @@
segmentsFileName = segmentInfos.getCurrentSegmentFileName();
version = segmentInfos.getVersion();
generation = segmentInfos.getGeneration();
- int size = segmentInfos.size();
- files = new ArrayList(size);
- files.add(segmentsFileName);
+ files = Collections.unmodifiableCollection(segmentInfos.files(directory, true));
gen = segmentInfos.getGeneration();
- for(int i=0;i<size;i++) {
- SegmentInfo segmentInfo = segmentInfos.info(i);
- if (segmentInfo.dir == directory) {
- files.addAll(segmentInfo.files());
- }
- }
isOptimized = segmentInfos.size() == 1 && !segmentInfos.info(0).hasDeletions();
+
+ assert !segmentInfos.hasExternalSegments(directory);
}
public boolean isOptimized() {
@@ -616,7 +598,7 @@
}
public Collection getFileNames() throws IOException {
- return Collections.unmodifiableCollection(files);
+ return files;
}
public Directory getDirectory() {
Modified: lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/IndexWriter.java Mon Jan 5 14:31:04 2009
@@ -1187,12 +1187,7 @@
// We assume that this segments_N was previously
// properly sync'd:
- for(int i=0;i<segmentInfos.size();i++) {
- final SegmentInfo info = segmentInfos.info(i);
- List files = info.files();
- for(int j=0;j<files.size();j++)
- synced.add(files.get(j));
- }
+ synced.addAll(segmentInfos.files(directory, true));
}
this.autoCommit = autoCommit;
@@ -1231,7 +1226,7 @@
private synchronized void setRollbackSegmentInfos(SegmentInfos infos) {
rollbackSegmentInfos = (SegmentInfos) infos.clone();
- assert !hasExternalSegments(rollbackSegmentInfos);
+ assert !rollbackSegmentInfos.hasExternalSegments(directory);
rollbackSegments = new HashMap();
final int size = rollbackSegmentInfos.size();
for(int i=0;i<size;i++)
@@ -2721,7 +2716,7 @@
try {
localRollbackSegmentInfos = (SegmentInfos) segmentInfos.clone();
- assert !hasExternalSegments(segmentInfos);
+ assert !hasExternalSegments();
localAutoCommit = autoCommit;
localFlushedDocCount = docWriter.getFlushedDocCount();
@@ -3244,15 +3239,7 @@
}
private boolean hasExternalSegments() {
- return hasExternalSegments(segmentInfos);
- }
-
- private boolean hasExternalSegments(SegmentInfos infos) {
- final int numSegments = infos.size();
- for(int i=0;i<numSegments;i++)
- if (infos.info(i).dir != directory)
- return true;
- return false;
+ return segmentInfos.hasExternalSegments(directory);
}
/* If any of our segments are using a directory != ours
@@ -4098,17 +4085,9 @@
}
private void decrefMergeSegments(MergePolicy.OneMerge merge) throws IOException {
- final SegmentInfos sourceSegmentsClone = merge.segmentsClone;
- final int numSegmentsToMerge = sourceSegmentsClone.size();
assert merge.increfDone;
merge.increfDone = false;
- for(int i=0;i<numSegmentsToMerge;i++) {
- final SegmentInfo previousInfo = sourceSegmentsClone.info(i);
- // Decref all files for this SegmentInfo (this
- // matches the incref in mergeInit):
- if (previousInfo.dir == directory)
- deleter.decRef(previousInfo.files());
- }
+ deleter.decRef(merge.segmentsClone);
}
final private void handleMergeException(Throwable t, MergePolicy.OneMerge merge) throws IOException {
@@ -4373,14 +4352,7 @@
// properly merge deletes in commitMerge()
merge.segmentsClone = (SegmentInfos) merge.segments.clone();
- for (int i = 0; i < end; i++) {
- SegmentInfo si = merge.segmentsClone.info(i);
-
- // IncRef all files for this segment info to make sure
- // they are not removed while we are trying to merge.
- if (si.dir == directory)
- deleter.incRef(si.files());
- }
+ deleter.incRef(merge.segmentsClone, false);
merge.increfDone = true;
@@ -4814,6 +4786,8 @@
// copied the segmentInfos we intend to sync:
blockAddIndexes(false);
+ // On commit the segmentInfos must never
+ // reference a segment in another directory:
assert !hasExternalSegments();
try {
@@ -4858,24 +4832,21 @@
final Collection pending = new ArrayList();
- for(int i=0;i<toSync.size();i++) {
- final SegmentInfo info = toSync.info(i);
- final List files = info.files();
- for(int j=0;j<files.size();j++) {
- final String fileName = (String) files.get(j);
- if (startSync(fileName, pending)) {
- boolean success = false;
- try {
- // Because we incRef'd this commit point, above,
- // the file had better exist:
- assert directory.fileExists(fileName): "file '" + fileName + "' does not exist dir=" + directory;
- if (infoStream != null)
- message("now sync " + fileName);
- directory.sync(fileName);
- success = true;
- } finally {
- finishSync(fileName, success);
- }
+ Iterator it = toSync.files(directory, false).iterator();
+ while(it.hasNext()) {
+ final String fileName = (String) it.next();
+ if (startSync(fileName, pending)) {
+ boolean success = false;
+ try {
+ // Because we incRef'd this commit point, above,
+ // the file had better exist:
+ assert directory.fileExists(fileName): "file '" + fileName + "' does not exist dir=" + directory;
+ if (infoStream != null)
+ message("now sync " + fileName);
+ directory.sync(fileName);
+ success = true;
+ } finally {
+ finishSync(fileName, success);
}
}
}
Modified: lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/trunk/src/java/org/apache/lucene/index/SegmentInfos.java Mon Jan 5 14:31:04 2009
@@ -30,6 +30,8 @@
import java.io.IOException;
import java.io.PrintStream;
import java.util.Vector;
+import java.util.Collection;
+import java.util.HashSet;
final class SegmentInfos extends Vector {
@@ -794,6 +796,26 @@
write(dir);
}
+ /** Returns all file names referenced by SegmentInfo
+ * instances matching the provided Directory (ie files
+ * associated with any "external" segments are skipped).
+ * The returned collection is recomputed on each
+ * invocation. */
+ public Collection files(Directory dir, boolean includeSegmentsFile) throws IOException {
+ HashSet files = new HashSet();
+ if (includeSegmentsFile) {
+ files.add(getCurrentSegmentFileName());
+ }
+ final int size = size();
+ for(int i=0;i<size;i++) {
+ final SegmentInfo info = info(i);
+ if (info.dir == dir) {
+ files.addAll(info(i).files());
+ }
+ }
+ return files;
+ }
+
public final void finishCommit(Directory dir) throws IOException {
if (pendingOutput == null)
throw new IllegalStateException("prepareCommit was not called");
@@ -891,4 +913,13 @@
addAll(other);
lastGeneration = other.lastGeneration;
}
+
+ // Used only for testing
+ boolean hasExternalSegments(Directory dir) {
+ final int numSegments = size();
+ for(int i=0;i<numSegments;i++)
+ if (info(i).dir != dir)
+ return true;
+ return false;
+ }
}
Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexReader.java Mon Jan 5 14:31:04 2009
@@ -1558,4 +1558,35 @@
// expected
}
}
+
+ // LUCENE-1509
+ public void testNoDupCommitFileNames() throws Throwable {
+
+ Directory dir = new MockRAMDirectory();
+
+ IndexWriter writer = new IndexWriter(dir, new StandardAnalyzer(),
+ IndexWriter.MaxFieldLength.LIMITED);
+
+ writer.setMaxBufferedDocs(2);
+ writer.addDocument(createDocument("a"));
+ writer.addDocument(createDocument("a"));
+ writer.addDocument(createDocument("a"));
+ writer.close();
+
+ Collection commits = IndexReader.listCommits(dir);
+ Iterator it = commits.iterator();
+ while(it.hasNext()) {
+ IndexCommit commit = (IndexCommit) it.next();
+ Collection files = commit.getFileNames();
+ HashSet seen = new HashSet();
+ Iterator it2 = files.iterator();
+ while(it2.hasNext()) {
+ String fileName = (String) it2.next();
+ assertTrue("file " + fileName + " was duplicated", !seen.contains(fileName));
+ seen.add(fileName);
+ }
+ }
+
+ dir.close();
+ }
}
Modified: lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=731747&r1=731746&r2=731747&view=diff
==============================================================================
--- lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/java/trunk/src/test/org/apache/lucene/index/TestIndexWriter.java Mon Jan 5 14:31:04 2009
@@ -1164,9 +1164,9 @@
// and it doesn't delete intermediate segments then it
// will exceed this 100X:
// System.out.println("start " + startDiskUsage + "; mid " + midDiskUsage + ";end " + endDiskUsage);
- assertTrue("writer used to much space while adding documents when autoCommit=false",
+ assertTrue("writer used too much space while adding documents when autoCommit=false: mid=" + midDiskUsage + " start=" + startDiskUsage + " end=" + endDiskUsage,
midDiskUsage < 100*startDiskUsage);
- assertTrue("writer used to much space after close when autoCommit=false endDiskUsage=" + endDiskUsage + " startDiskUsage=" + startDiskUsage,
+ assertTrue("writer used too much space after close when autoCommit=false endDiskUsage=" + endDiskUsage + " startDiskUsage=" + startDiskUsage,
endDiskUsage < 100*startDiskUsage);
}