mirror of https://github.com/apache/lucene.git
LUCENE-4055: clean up nocommits
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1341944 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9cde5052a2
commit
7a1834c41f
|
@ -61,10 +61,14 @@ public class Lucene40PostingsReader extends PostingsReaderBase {
|
|||
public Lucene40PostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
|
||||
freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene40PostingsFormat.FREQ_EXTENSION),
|
||||
ioContext);
|
||||
// nocommit don't consult FieldInfos here... then nuke
|
||||
// fieldInfos arg...
|
||||
// nocommit we can assert FIS.hasProx == our hasProx here...
|
||||
// this.segment = segmentInfo.name;
|
||||
// TODO: hasProx should (somehow!) become codec private,
|
||||
// but it's tricky because 1) FIS.hasProx is global (it
|
||||
// could be all fields that have prox are written by a
|
||||
// different codec), 2) the field may have had prox in
|
||||
// the past but all docs w/ that field were deleted.
|
||||
// Really we'd need to init prxOut lazily on write, and
|
||||
// then somewhere record that we actually wrote it so we
|
||||
// know whether to open on read:
|
||||
if (fieldInfos.hasProx()) {
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -40,8 +40,7 @@ public class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
|
|||
@Override
|
||||
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene40SegmentInfoFormat.SI_EXTENSION);
|
||||
assert si.getFiles() != null;
|
||||
si.getFiles().add(fileName);
|
||||
si.addFile(fileName);
|
||||
|
||||
final IndexOutput output = dir.createOutput(fileName, ioContext);
|
||||
|
||||
|
@ -57,7 +56,7 @@ public class Lucene40SegmentInfoWriter extends SegmentInfoWriter {
|
|||
output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
|
||||
output.writeStringStringMap(si.getDiagnostics());
|
||||
output.writeStringStringMap(si.attributes());
|
||||
output.writeStringSet(si.getFiles());
|
||||
output.writeStringSet(si.files());
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -67,7 +67,6 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
|
|||
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and {@link Type#BYTES_VAR_SORTED}.
|
||||
*/
|
||||
protected DocValuesWriterBase(PerDocWriteState state, boolean fasterButMoreRam) {
|
||||
// nocommit save away SegmentInfo instead?
|
||||
this.segmentName = state.segmentInfo.name;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
this.context = state.context;
|
||||
|
|
|
@ -59,7 +59,7 @@ public class SimpleTextSegmentInfoWriter extends SegmentInfoWriter {
|
|||
public void write(Directory dir, SegmentInfo si, FieldInfos fis, IOContext ioContext) throws IOException {
|
||||
|
||||
String segFileName = IndexFileNames.segmentFileName(si.name, "", SimpleTextSegmentInfoFormat.SI_EXTENSION);
|
||||
si.getFiles().add(segFileName);
|
||||
si.addFile(segFileName);
|
||||
|
||||
boolean success = false;
|
||||
IndexOutput output = dir.createOutput(segFileName, ioContext);
|
||||
|
@ -115,7 +115,7 @@ public class SimpleTextSegmentInfoWriter extends SegmentInfoWriter {
|
|||
}
|
||||
}
|
||||
|
||||
Set<String> files = si.getFiles();
|
||||
Set<String> files = si.files();
|
||||
int numFiles = files == null ? 0 : files.size();
|
||||
SimpleTextUtil.write(output, SI_NUM_FILES);
|
||||
SimpleTextUtil.write(output, Integer.toString(numFiles), scratch);
|
||||
|
|
|
@ -479,8 +479,8 @@ public class CheckIndex {
|
|||
segInfoStat.codec = codec;
|
||||
msg(" compound=" + info.info.getUseCompoundFile());
|
||||
segInfoStat.compound = info.info.getUseCompoundFile();
|
||||
msg(" numFiles=" + info.info.files().size());
|
||||
segInfoStat.numFiles = info.info.files().size();
|
||||
msg(" numFiles=" + info.files().size());
|
||||
segInfoStat.numFiles = info.files().size();
|
||||
segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
|
||||
msg(" size (MB)=" + nf.format(segInfoStat.sizeMB));
|
||||
Map<String,String> diagnostics = info.info.getDiagnostics();
|
||||
|
|
|
@ -489,6 +489,7 @@ class DocumentsWriterPerThread {
|
|||
segmentInfo.docCount = flushState.numDocs;
|
||||
segmentInfo.setFiles(new HashSet<String>(directory.getCreatedFiles()));
|
||||
|
||||
final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L);
|
||||
if (infoStream.isEnabled("DWPT")) {
|
||||
infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");
|
||||
infoStream.message("DWPT", "new segment has " +
|
||||
|
@ -497,7 +498,7 @@ class DocumentsWriterPerThread {
|
|||
(flushState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " +
|
||||
(flushState.fieldInfos.hasProx() ? "prox" : "no prox") + "; " +
|
||||
(flushState.fieldInfos.hasFreq() ? "freqs" : "no freqs"));
|
||||
infoStream.message("DWPT", "flushedFiles=" + segmentInfo.files());
|
||||
infoStream.message("DWPT", "flushedFiles=" + segmentInfoPerCommit.files());
|
||||
infoStream.message("DWPT", "flushed codec=" + codec);
|
||||
}
|
||||
|
||||
|
@ -522,7 +523,7 @@ class DocumentsWriterPerThread {
|
|||
|
||||
assert segmentInfo != null;
|
||||
|
||||
FlushedSegment fs = new FlushedSegment(new SegmentInfoPerCommit(segmentInfo, 0, -1L), flushState.fieldInfos,
|
||||
FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
|
||||
segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
|
||||
doAfterFlush();
|
||||
success = true;
|
||||
|
|
|
@ -1505,15 +1505,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// merge:
|
||||
for(final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
merge.maxNumSegments = maxNumSegments;
|
||||
// nocommit: remove this, except it causes
|
||||
// TestExternalCodecs.testPerFieldCodec failures:
|
||||
segmentsToMerge.put(merge.info, Boolean.TRUE);
|
||||
}
|
||||
|
||||
for (final MergePolicy.OneMerge merge: runningMerges) {
|
||||
merge.maxNumSegments = maxNumSegments;
|
||||
// nocommit: remove this, except it causes
|
||||
// TestExternalCodecs.testPerFieldCodec failures:
|
||||
segmentsToMerge.put(merge.info, Boolean.TRUE);
|
||||
}
|
||||
}
|
||||
|
@ -2046,7 +2042,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// above:
|
||||
codec.segmentInfoFormat().getSegmentInfosWriter().write(directory, newSegment.info, flushedSegment.fieldInfos, context);
|
||||
|
||||
// nocommit ideally we would freeze newSegment here!!
|
||||
// TODO: ideally we would freeze newSegment here!!
|
||||
// because any changes after writing the .si will be
|
||||
// lost...
|
||||
|
||||
|
@ -2341,7 +2337,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
// and 2) .si reflects useCompoundFile=true change
|
||||
// above:
|
||||
codec.segmentInfoFormat().getSegmentInfosWriter().write(trackingDir, info, mergeState.fieldInfos, context);
|
||||
info.getFiles().addAll(trackingDir.getCreatedFiles());
|
||||
info.addFiles(trackingDir.getCreatedFiles());
|
||||
|
||||
// Register the new segment
|
||||
synchronized(this) {
|
||||
|
@ -2410,7 +2406,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
Set<String> segFiles = new HashSet<String>();
|
||||
|
||||
// Build up new segment's file names:
|
||||
// Build up new segment's file names. Must do this
|
||||
// before writing SegmentInfo:
|
||||
for (String file: info.files()) {
|
||||
final String newFileName;
|
||||
if (codecDocStoreFiles.contains(file)) {
|
||||
|
@ -2421,17 +2418,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
segFiles.add(newFileName);
|
||||
}
|
||||
newInfo.setFiles(segFiles);
|
||||
|
||||
|
||||
// We must rewrite the SI file because it references
|
||||
// segment name (its own name, if its 3.x, and doc
|
||||
// store segment name):
|
||||
TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory);
|
||||
try {
|
||||
newInfo.getCodec().segmentInfoFormat().getSegmentInfosWriter().write(directory, newInfo, null, context);
|
||||
newInfo.getCodec().segmentInfoFormat().getSegmentInfosWriter().write(trackingDir, newInfo, null, context);
|
||||
} catch (UnsupportedOperationException uoe) {
|
||||
// OK: 3x codec cannot write a new SI file;
|
||||
// SegmentInfos will write this on commit
|
||||
}
|
||||
|
||||
final Collection<String> siFiles = trackingDir.getCreatedFiles();
|
||||
|
||||
// Copy the segment's files
|
||||
for (String file: info.files()) {
|
||||
|
||||
|
@ -2446,14 +2446,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
newFileName = segName + IndexFileNames.stripSegmentName(file);
|
||||
}
|
||||
|
||||
// nocommit hack
|
||||
//if (siFileNames != null && siFileNames.contains(newFileName)) {
|
||||
if (newFileName.endsWith(".si")) {
|
||||
if (siFiles.contains(newFileName)) {
|
||||
// We already rewrote this above
|
||||
continue;
|
||||
}
|
||||
|
||||
assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
|
||||
assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists; siFiles=" + siFiles;
|
||||
assert !copiedFiles.contains(file): "file \"" + file + "\" is being copied more than once";
|
||||
copiedFiles.add(file);
|
||||
info.info.dir.copy(directory, file, newFileName, context);
|
||||
|
@ -3501,7 +3499,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
|
||||
// This is where all the work happens:
|
||||
MergeState mergeState = merger.merge();
|
||||
// nocommit use setter and make this a SetOnce:
|
||||
merge.info.info.docCount = mergeState.mergedDocCount;
|
||||
merge.info.info.setFiles(new HashSet<String>(dirWrapper.getCreatedFiles()));
|
||||
|
||||
|
@ -3597,7 +3594,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
}
|
||||
}
|
||||
|
||||
// nocommit ideally we would freeze merge.info here!!
|
||||
// TODO: ideally we would freeze merge.info here!!
|
||||
// because any changes after writing the .si will be
|
||||
// lost...
|
||||
|
||||
|
@ -4031,6 +4028,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
IOUtils.closeWhileHandlingException(prior, cfsDir);
|
||||
}
|
||||
|
||||
// Replace all previous files with the CFS/CFE files:
|
||||
Set<String> siFiles = new HashSet<String>();
|
||||
siFiles.add(fileName);
|
||||
siFiles.add(IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.lucene.index;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -28,9 +29,6 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.TrackingDirectoryWrapper;
|
||||
|
||||
// nocommit fix codec api to pass this around so they can
|
||||
// store attrs
|
||||
|
||||
/**
|
||||
* Information about a segment such as it's name, directory, and files related
|
||||
* to the segment.
|
||||
|
@ -44,12 +42,9 @@ public final class SegmentInfo {
|
|||
public static final int YES = 1; // e.g. have norms; have deletes;
|
||||
|
||||
public final String name; // unique name in dir
|
||||
// nocommit make me final:
|
||||
public int docCount; // number of docs in seg
|
||||
public final Directory dir; // where segment resides
|
||||
|
||||
// nocommit what other members can we make final?
|
||||
|
||||
/*
|
||||
* Current generation of each field's norm file. If this array is null,
|
||||
* means no separate norms. If this array is not null, its values mean:
|
||||
|
@ -82,7 +77,6 @@ public final class SegmentInfo {
|
|||
// The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
|
||||
// specific versions afterwards ("3.0", "3.1" etc.).
|
||||
// see Constants.LUCENE_MAIN_VERSION.
|
||||
// nocommit final?
|
||||
private String version;
|
||||
|
||||
void setDiagnostics(Map<String, String> diagnostics) {
|
||||
|
@ -117,9 +111,11 @@ public final class SegmentInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns total size in bytes of all of files used by this segment
|
||||
* Returns total size in bytes of all of files used by
|
||||
* this segment. Note that this will not include any live
|
||||
* docs for the segment; to include that use {@link
|
||||
* SegmentInfoPerCommit.sizeInBytes} instead.
|
||||
*/
|
||||
// nocommit fails to take live docs into account... hmmm
|
||||
public long sizeInBytes() throws IOException {
|
||||
if (sizeInBytes == -1) {
|
||||
long sum = 0;
|
||||
|
@ -131,10 +127,6 @@ public final class SegmentInfo {
|
|||
return sizeInBytes;
|
||||
}
|
||||
|
||||
void clearSizeInBytes() {
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated separate norms are not supported in >= 4.0
|
||||
*/
|
||||
|
@ -217,15 +209,11 @@ public final class SegmentInfo {
|
|||
* modify it.
|
||||
*/
|
||||
|
||||
// nocommit remove this temporarily to see who is calling
|
||||
// it ... very dangerous having this one AND SIPC.files()
|
||||
public Collection<String> files() throws IOException {
|
||||
// nocommit make sure when we are called we really have
|
||||
// files set ...
|
||||
public Set<String> files() throws IOException {
|
||||
if (setFiles == null) {
|
||||
throw new IllegalStateException("files were not computed yet");
|
||||
}
|
||||
return setFiles;
|
||||
return Collections.unmodifiableSet(setFiles);
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -320,19 +308,19 @@ public final class SegmentInfo {
|
|||
|
||||
private Set<String> setFiles;
|
||||
|
||||
// nocommit now on building a CFS we erase the files that
|
||||
// are in it... maybe we should somehow preserve it...
|
||||
public void setFiles(Set<String> files) {
|
||||
setFiles = files;
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
// nocommit remove this! it's only needed for
|
||||
// clearing/adding the files set...
|
||||
public Set<String> getFiles() {
|
||||
return setFiles;
|
||||
public void addFiles(Collection<String> files) {
|
||||
setFiles.addAll(files);
|
||||
}
|
||||
|
||||
|
||||
public void addFile(String file) {
|
||||
setFiles.add(file);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a codec attribute value, or null if it does not exist
|
||||
*/
|
||||
|
|
|
@ -41,6 +41,8 @@ public class SegmentInfoPerCommit {
|
|||
// are no deletes yet):
|
||||
private long delGen;
|
||||
|
||||
private volatile long sizeInBytes = -1;
|
||||
|
||||
public SegmentInfoPerCommit(SegmentInfo info, int delCount, long delGen) {
|
||||
this.info = info;
|
||||
this.delCount = delCount;
|
||||
|
@ -53,18 +55,26 @@ public class SegmentInfoPerCommit {
|
|||
} else {
|
||||
delGen++;
|
||||
}
|
||||
info.clearSizeInBytes();
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
public long sizeInBytes() throws IOException {
|
||||
// nocommit add in live docs size
|
||||
return info.sizeInBytes();
|
||||
if (sizeInBytes == -1) {
|
||||
final Collection<String> files = new HashSet<String>();
|
||||
info.getCodec().liveDocsFormat().files(this, files);
|
||||
long sum = info.sizeInBytes();
|
||||
for (final String fileName : files()) {
|
||||
sum += info.dir.fileLength(fileName);
|
||||
}
|
||||
sizeInBytes = sum;
|
||||
}
|
||||
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
public Collection<String> files() throws IOException {
|
||||
Collection<String> files = new HashSet<String>(info.files());
|
||||
|
||||
// nocommit make this take list instead...?
|
||||
// Must separately add any live docs files:
|
||||
info.getCodec().liveDocsFormat().files(this, files);
|
||||
|
||||
|
@ -81,16 +91,17 @@ public class SegmentInfoPerCommit {
|
|||
|
||||
void setBufferedDeletesGen(long v) {
|
||||
bufferedDeletesGen = v;
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
void clearDelGen() {
|
||||
delGen = -1;
|
||||
info.clearSizeInBytes();
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
public void setDelGen(long delGen) {
|
||||
this.delGen = delGen;
|
||||
info.clearSizeInBytes();
|
||||
sizeInBytes = -1;
|
||||
}
|
||||
|
||||
public boolean hasDeletions() {
|
||||
|
|
|
@ -411,7 +411,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
|
|||
|
||||
// NOTE: this is NOT how 3.x is really written...
|
||||
String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene3xSegmentInfoFormat.SI_EXTENSION);
|
||||
si.getFiles().add(fileName);
|
||||
si.addFile(fileName);
|
||||
|
||||
//System.out.println("UPGRADE write " + fileName);
|
||||
boolean success = false;
|
||||
|
@ -456,7 +456,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfoPerCom
|
|||
output.writeStringStringMap(si.getDiagnostics());
|
||||
// hasVectors (lie):
|
||||
output.writeByte((byte) 1);
|
||||
output.writeStringSet(si.getFiles());
|
||||
output.writeStringSet(si.files());
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -189,16 +189,6 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
for (IndexReader r : reader.getSequentialSubReaders()) {
|
||||
SegmentInfoPerCommit s = ((SegmentReader) r).getSegmentInfo();
|
||||
assertTrue(((SegmentReader) r).getFieldInfos().hasVectors());
|
||||
|
||||
// nocommit
|
||||
/*
|
||||
Set<String> files = new HashSet<String>();
|
||||
s.getCodec().termVectorsFormat().files(s, files);
|
||||
assertFalse(files.isEmpty());
|
||||
for (String file : files) {
|
||||
assertTrue(dir.fileExists(file));
|
||||
}
|
||||
*/
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue