mirror of https://github.com/apache/lucene.git
LUCENE-4055: remove another redundant boolean; speed up file matching
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1339878 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
373dc4a2df
commit
67747b0955
|
@ -132,6 +132,7 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
|||
final Codec codec = Codec.forName("Lucene3x");
|
||||
final Map<String,String> diagnostics = input.readStringStringMap();
|
||||
|
||||
// nocommit cleane up
|
||||
final boolean hasVectors;
|
||||
if (format <= SegmentInfos.FORMAT_HAS_VECTORS) {
|
||||
hasVectors = input.readByte() == 1;
|
||||
|
@ -148,23 +149,10 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader {
|
|||
storeIsCompoundFile = isCompoundFile;
|
||||
ext = IndexFileNames.COMPOUND_FILE_EXTENSION;
|
||||
}
|
||||
final Directory dirToTest;
|
||||
if (storeIsCompoundFile) {
|
||||
dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
|
||||
} else {
|
||||
dirToTest = dir;
|
||||
}
|
||||
try {
|
||||
hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION));
|
||||
} finally {
|
||||
if (isCompoundFile) {
|
||||
dirToTest.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
|
||||
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
|
||||
delCount, hasProx, codec, diagnostics, hasVectors);
|
||||
delCount, hasProx, codec, diagnostics);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,12 +19,15 @@ package org.apache.lucene.codecs.lucene3x;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
import java.util.Arrays; //nocommit
|
||||
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
|
@ -38,8 +41,38 @@ import org.apache.lucene.store.IOContext;
|
|||
class Lucene3xTermVectorsFormat extends TermVectorsFormat {
|
||||
|
||||
@Override
|
||||
public TermVectorsReader vectorsReader(Directory directory,SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
|
||||
return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context);
|
||||
public TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException {
|
||||
final String fileName = IndexFileNames.segmentFileName(segmentInfo.getDocStoreSegment(), "", Lucene3xTermVectorsReader.VECTORS_FIELDS_EXTENSION);
|
||||
|
||||
// Unfortunately, for 3.x indices, each segment's
|
||||
// FieldInfos can lie about hasVectors (claim it's true
|
||||
// when really it's false).... so we have to carefully
|
||||
// check if the files really exist before trying to open
|
||||
// them (4.x has fixed this):
|
||||
final boolean exists;
|
||||
if (segmentInfo.getDocStoreOffset() != -1 && segmentInfo.getDocStoreIsCompoundFile()) {
|
||||
String cfxFileName = IndexFileNames.segmentFileName(segmentInfo.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION);
|
||||
if (segmentInfo.dir.fileExists(cfxFileName)) {
|
||||
Directory cfsDir = new CompoundFileDirectory(segmentInfo.dir, cfxFileName, context, false);
|
||||
try {
|
||||
exists = cfsDir.fileExists(fileName);
|
||||
} finally {
|
||||
cfsDir.close();
|
||||
}
|
||||
} else {
|
||||
exists = false;
|
||||
}
|
||||
} else {
|
||||
exists = directory.fileExists(fileName);
|
||||
}
|
||||
|
||||
if (!exists) {
|
||||
// 3x's FieldInfos sometimes lies and claims a segment
|
||||
// has vectors when it doesn't:
|
||||
return null;
|
||||
} else {
|
||||
return new Lucene3xTermVectorsReader(directory, segmentInfo, fieldInfos, context);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -693,21 +693,19 @@ class Lucene3xTermVectorsReader extends TermVectorsReader {
|
|||
// note: if there are shared docstores, we are also called by Lucene3xCodec even in
|
||||
// the CFS case. so logic here must handle this.
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
} else if (!info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
if (info.getDocStoreOffset() != -1) {
|
||||
assert info.getDocStoreSegment() != null;
|
||||
if (info.getDocStoreIsCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", Lucene3xCodec.COMPOUND_FILE_STORE_EXTENSION));
|
||||
} else {
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.getDocStoreSegment(), "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
} else if (!info.getUseCompoundFile()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -85,10 +85,9 @@ public class Lucene40SegmentInfosReader extends SegmentInfosReader {
|
|||
final boolean hasProx = input.readByte() == 1;
|
||||
final Codec codec = Codec.forName(input.readString());
|
||||
final Map<String,String> diagnostics = input.readStringStringMap();
|
||||
final boolean hasVectors = input.readByte() == 1;
|
||||
|
||||
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
|
||||
docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
|
||||
delCount, hasProx, codec, diagnostics, hasVectors);
|
||||
delCount, hasProx, codec, diagnostics);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -98,7 +98,6 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter {
|
|||
output.writeByte((byte) (si.getHasProx() ? 1 : 0));
|
||||
output.writeString(si.getCodec().getName());
|
||||
output.writeStringStringMap(si.getDiagnostics());
|
||||
output.writeByte((byte) (si.getHasVectors() ? 1 : 0));
|
||||
}
|
||||
|
||||
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
|
||||
|
|
|
@ -712,11 +712,9 @@ public class Lucene40TermVectorsReader extends TermVectorsReader {
|
|||
}
|
||||
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_INDEX_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_FIELDS_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_DOCUMENTS_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -67,6 +67,6 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
|||
public void abort() {
|
||||
Set<String> files = new HashSet<String>();
|
||||
files(segmentName, files);
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, SegmentInfo.findMatchingFiles(directory, files).toArray(new String[0]));
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, SegmentInfo.findMatchingFiles(segmentName, directory, files).toArray(new String[0]));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -131,7 +131,7 @@ public class SimpleTextNormsFormat extends NormsFormat {
|
|||
Set<String> files = new HashSet<String>();
|
||||
filesInternal(state.segmentName, files, segmentSuffix);
|
||||
IOUtils.deleteFilesIgnoringExceptions(state.directory,
|
||||
SegmentInfo.findMatchingFiles(state.directory, files).toArray(new String[0]));
|
||||
SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0]));
|
||||
}
|
||||
|
||||
public static void files(SegmentInfo segmentInfo, Set<String> files)
|
||||
|
@ -142,6 +142,9 @@ public class SimpleTextNormsFormat extends NormsFormat {
|
|||
|
||||
public static void filesInternal(String segmentName,
|
||||
Set<String> files, String segmentSuffix) {
|
||||
// nocommit simplify this: weird that we get suffix as
|
||||
// an arg... it's always a constant
|
||||
assert segmentSuffix.equals(NORMS_SEG_SUFFIX);
|
||||
String id = docValuesIdRegexp(segmentName);
|
||||
files.add(IndexFileNames.segmentFileName(id, "",
|
||||
segmentSuffix));
|
||||
|
|
|
@ -60,7 +60,7 @@ class SimpleTextPerDocConsumer extends PerDocConsumer {
|
|||
Set<String> files = new HashSet<String>();
|
||||
files(state.directory, state.segmentName, files, segmentSuffix);
|
||||
IOUtils.deleteFilesIgnoringExceptions(state.directory,
|
||||
SegmentInfo.findMatchingFiles(state.directory, files).toArray(new String[0]));
|
||||
SegmentInfo.findMatchingFiles(state.segmentName, state.directory, files).toArray(new String[0]));
|
||||
}
|
||||
|
||||
static void files(SegmentInfo info, Set<String> files, String segmentSuffix) {
|
||||
|
|
|
@ -106,10 +106,6 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
|
|||
assert StringHelper.startsWith(scratch, SI_HASPROX);
|
||||
final boolean hasProx = Boolean.parseBoolean(readString(SI_HASPROX.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, SI_HASVECTORS);
|
||||
final boolean hasVectors = Boolean.parseBoolean(readString(SI_HASVECTORS.length, scratch));
|
||||
|
||||
SimpleTextUtil.readLine(input, scratch);
|
||||
assert StringHelper.startsWith(scratch, SI_USECOMPOUND);
|
||||
final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
|
||||
|
@ -168,7 +164,7 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader {
|
|||
|
||||
return new SegmentInfo(directory, version, name, docCount, delGen, dsOffset,
|
||||
dsSegment, dsCompoundFile, normGen, isCompoundFile,
|
||||
delCount, hasProx, codec, diagnostics, hasVectors);
|
||||
delCount, hasProx, codec, diagnostics);
|
||||
}
|
||||
|
||||
private String readString(int offset, BytesRef scratch) {
|
||||
|
|
|
@ -52,7 +52,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
|
|||
final static BytesRef SI_DOCCOUNT = new BytesRef(" number of documents ");
|
||||
final static BytesRef SI_DELCOUNT = new BytesRef(" number of deletions ");
|
||||
final static BytesRef SI_HASPROX = new BytesRef(" has prox ");
|
||||
final static BytesRef SI_HASVECTORS = new BytesRef(" has vectors ");
|
||||
final static BytesRef SI_USECOMPOUND = new BytesRef(" uses compound file ");
|
||||
final static BytesRef SI_DSOFFSET = new BytesRef(" docstore offset ");
|
||||
final static BytesRef SI_DSSEGMENT = new BytesRef(" docstore segment ");
|
||||
|
@ -150,10 +149,6 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter {
|
|||
SimpleTextUtil.write(output, si.getHasProx() ? "true" : "false", scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
SimpleTextUtil.write(output, SI_HASVECTORS);
|
||||
SimpleTextUtil.write(output, si.getHasVectors() ? "true" : "false", scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
||||
SimpleTextUtil.write(output, SI_USECOMPOUND);
|
||||
SimpleTextUtil.write(output, Boolean.toString(si.getUseCompoundFile()), scratch);
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
|
|
|
@ -202,9 +202,7 @@ public class SimpleTextTermVectorsReader extends TermVectorsReader {
|
|||
}
|
||||
|
||||
public static void files(SegmentInfo info, Set<String> files) throws IOException {
|
||||
if (info.getHasVectors()) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
|
||||
}
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
|
||||
}
|
||||
|
||||
private void readLine() throws IOException {
|
||||
|
|
|
@ -706,8 +706,7 @@ public class CheckIndex {
|
|||
* checks Fields api is consistent with itself.
|
||||
* searcher is optional, to verify with queries. Can be null.
|
||||
*/
|
||||
// TODO: cutover term vectors to this!
|
||||
private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
|
||||
private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher, boolean doPrint) throws IOException {
|
||||
// TODO: we should probably return our own stats thing...?!
|
||||
|
||||
final Status.TermIndexStatus status = new Status.TermIndexStatus();
|
||||
|
@ -1131,8 +1130,10 @@ public class CheckIndex {
|
|||
if (status.termCount != uniqueTermCountAllFields) {
|
||||
throw new RuntimeException("termCount mismatch " + uniqueTermCountAllFields + " vs " + (status.termCount));
|
||||
}
|
||||
|
||||
msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
|
||||
|
||||
if (doPrint) {
|
||||
msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
|
||||
}
|
||||
|
||||
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
|
||||
for(Map.Entry<String,BlockTreeTermsReader.Stats> ent : status.blockTreeStats.entrySet()) {
|
||||
|
@ -1163,13 +1164,13 @@ public class CheckIndex {
|
|||
}
|
||||
|
||||
final Fields fields = reader.fields();
|
||||
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is);
|
||||
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is, true);
|
||||
if (liveDocs != null) {
|
||||
if (infoStream != null) {
|
||||
infoStream.print(" test (ignoring deletes): terms, freq, prox...");
|
||||
}
|
||||
// TODO: can we make a IS that ignores all deletes?
|
||||
checkFields(fields, null, maxDoc, fieldInfos, null);
|
||||
checkFields(fields, null, maxDoc, fieldInfos, null, true);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
msg("ERROR: " + e);
|
||||
|
@ -1386,10 +1387,10 @@ public class CheckIndex {
|
|||
|
||||
if (tfv != null) {
|
||||
// First run with no deletions:
|
||||
checkFields(tfv, null, 1, fieldInfos, null);
|
||||
checkFields(tfv, null, 1, fieldInfos, null, false);
|
||||
|
||||
// Again, with the one doc deleted:
|
||||
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, null);
|
||||
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, null, false);
|
||||
|
||||
// Only agg stats if the doc is live:
|
||||
final boolean doStats = liveDocs == null || liveDocs.get(j);
|
||||
|
|
|
@ -478,8 +478,7 @@ class DocumentsWriterPerThread {
|
|||
final SegmentInfo newSegment = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, segment, flushState.numDocs,
|
||||
SegmentInfo.NO, -1, segment, false, null, false, 0,
|
||||
flushState.fieldInfos.hasProx(), flushState.codec,
|
||||
null,
|
||||
flushState.fieldInfos.hasVectors());
|
||||
null);
|
||||
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 " +
|
||||
|
|
|
@ -2290,8 +2290,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
int docCount = mergeState.mergedDocCount;
|
||||
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, docCount,
|
||||
SegmentInfo.NO, -1, mergedName, false, null, false, 0,
|
||||
mergeState.fieldInfos.hasProx(), codec, null,
|
||||
mergeState.fieldInfos.hasVectors());
|
||||
mergeState.fieldInfos.hasProx(), codec, null);
|
||||
|
||||
setDiagnostics(info, "addIndexes(IndexReader...)");
|
||||
|
||||
|
@ -3376,7 +3375,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
payloadProcessorProvider, new MutableFieldInfos(globalFieldNumberMap), codec, context);
|
||||
|
||||
if (infoStream.isEnabled("IW")) {
|
||||
infoStream.message("IW", "merging " + segString(merge.segments) + " mergeVectors=" + merge.info.getHasVectors());
|
||||
infoStream.message("IW", "merging " + segString(merge.segments));
|
||||
}
|
||||
|
||||
merge.readers = new ArrayList<SegmentReader>();
|
||||
|
@ -3436,7 +3435,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit {
|
|||
mergedDocCount = merge.info.docCount = mergeState.mergedDocCount;
|
||||
|
||||
// LUCENE-3403: set hasVectors after merge(), so that it is properly set.
|
||||
merge.info.setHasVectors(mergeState.fieldInfos.hasVectors());
|
||||
merge.info.setHasProx(mergeState.fieldInfos.hasProx());
|
||||
|
||||
// Record which codec was used to write the segment
|
||||
|
|
|
@ -115,8 +115,8 @@ final class SegmentCoreReaders {
|
|||
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(cfsDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
|
||||
if (fieldInfos.hasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(cfsDir, si, fieldInfos, context);
|
||||
} else {
|
||||
termVectorsReaderOrig = null;
|
||||
|
|
|
@ -71,6 +71,9 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
private boolean isCompoundFile;
|
||||
|
||||
private volatile List<String> files; // Cached list of files that this segment uses
|
||||
// in the Directory
|
||||
|
||||
private volatile long sizeInBytes = -1; // total byte size of all files (computed on demand)
|
||||
|
||||
//TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0)
|
||||
|
@ -84,7 +87,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
private int delCount; // How many deleted docs in this segment
|
||||
|
||||
private boolean hasVectors; // True if this segment has any term vectors fields
|
||||
private boolean hasProx; // True if this segment has any fields with positional information
|
||||
|
||||
private Codec codec;
|
||||
|
@ -132,7 +134,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
*/
|
||||
public SegmentInfo(Directory dir, String version, String name, int docCount, long delGen, int docStoreOffset,
|
||||
String docStoreSegment, boolean docStoreIsCompoundFile, Map<Integer,Long> normGen, boolean isCompoundFile,
|
||||
int delCount, boolean hasProx, Codec codec, Map<String,String> diagnostics, boolean hasVectors) {
|
||||
int delCount, boolean hasProx, Codec codec, Map<String,String> diagnostics) {
|
||||
this.dir = dir;
|
||||
this.version = version;
|
||||
this.name = name;
|
||||
|
@ -144,11 +146,10 @@ public final class SegmentInfo implements Cloneable {
|
|||
this.normGen = normGen;
|
||||
this.isCompoundFile = isCompoundFile;
|
||||
this.delCount = delCount;
|
||||
// nocommit remove these now that we can do regexp instead!
|
||||
this.hasProx = hasProx;
|
||||
this.codec = codec;
|
||||
this.diagnostics = diagnostics;
|
||||
// nocommit remove these now that we can do regexp instead!
|
||||
this.hasVectors = hasVectors;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -175,16 +176,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
clearFilesCache();
|
||||
}
|
||||
|
||||
// nocommit: ideally codec stores this info privately:
|
||||
public boolean getHasVectors() throws IOException {
|
||||
return hasVectors;
|
||||
}
|
||||
|
||||
public void setHasVectors(boolean hasVectors) {
|
||||
this.hasVectors = hasVectors;
|
||||
clearFilesCache();
|
||||
}
|
||||
|
||||
public boolean hasDeletions() {
|
||||
// Cases:
|
||||
//
|
||||
|
@ -230,8 +221,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
|
||||
return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
|
||||
docStoreSegment, docStoreIsCompoundFile, clonedNormGen, isCompoundFile,
|
||||
delCount, hasProx, codec, new HashMap<String,String>(diagnostics),
|
||||
hasVectors);
|
||||
delCount, hasProx, codec, new HashMap<String,String>(diagnostics));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -333,7 +323,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
}
|
||||
|
||||
// nocommit move elsewhere? IndexFileNames?
|
||||
public static List<String> findMatchingFiles(Directory dir, Set<String> namesOrPatterns) {
|
||||
public static List<String> findMatchingFiles(String segmentName, Directory dir, Set<String> namesOrPatterns) {
|
||||
// nocommit need more efficient way to do this?
|
||||
List<String> files = new ArrayList<String>();
|
||||
final String[] existingFiles;
|
||||
|
@ -343,6 +333,7 @@ public final class SegmentInfo implements Cloneable {
|
|||
// nocommit maybe just throw IOE...? not sure how far up we'd have to change sigs...
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
List<Pattern> compiledPatterns = new ArrayList<Pattern>();
|
||||
for(String nameOrPattern : namesOrPatterns) {
|
||||
boolean exists = false;
|
||||
try {
|
||||
|
@ -354,9 +345,18 @@ public final class SegmentInfo implements Cloneable {
|
|||
if (exists) {
|
||||
files.add(nameOrPattern);
|
||||
} else {
|
||||
for(String file : existingFiles) {
|
||||
if (Pattern.matches(nameOrPattern, file)) {
|
||||
// nocommit can i test whether the regexp matches only 1 string...? maybe... make into autamaton and union them all....?
|
||||
compiledPatterns.add(Pattern.compile(nameOrPattern));
|
||||
}
|
||||
}
|
||||
|
||||
// nocommit this is DOG SLOW: try TestBoolean2 w/ seed 1F7F3638C719C665
|
||||
for(String file : existingFiles) {
|
||||
if (file.startsWith(segmentName)) {
|
||||
for(Pattern pattern : compiledPatterns) {
|
||||
if (pattern.matcher(file).matches()) {
|
||||
files.add(file);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -372,15 +372,20 @@ public final class SegmentInfo implements Cloneable {
|
|||
*/
|
||||
|
||||
public List<String> files() throws IOException {
|
||||
final Set<String> fileSet = new HashSet<String>();
|
||||
codec.files(this, fileSet);
|
||||
return findMatchingFiles(dir, fileSet);
|
||||
if (files == null) {
|
||||
// nocommit can we remove this again....?
|
||||
final Set<String> fileSet = new HashSet<String>();
|
||||
codec.files(this, fileSet);
|
||||
files = findMatchingFiles(name, dir, fileSet);
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
/* Called whenever any change is made that affects which
|
||||
* files this segment has. */
|
||||
private void clearFilesCache() {
|
||||
sizeInBytes = -1;
|
||||
files = null;
|
||||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
|
@ -411,19 +416,6 @@ public final class SegmentInfo implements Cloneable {
|
|||
if (this.dir != dir) {
|
||||
s.append('x');
|
||||
}
|
||||
try {
|
||||
if (getHasVectors()) {
|
||||
s.append('v');
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
// Messy: because getHasVectors may be used in an
|
||||
// thread-unsafe way, and may attempt to open an fnm
|
||||
// file that has since (legitimately) been deleted by
|
||||
// IndexWriter, instead of throwing these exceptions
|
||||
// up, just add v? to indicate we don't know if this
|
||||
// segment has vectors:
|
||||
s.append("v?");
|
||||
}
|
||||
s.append(docCount);
|
||||
|
||||
int delCount = getDelCount() + pendingDelCount;
|
||||
|
|
|
@ -323,11 +323,14 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
|
|||
assertEquals("field with non-ascii name", f.stringValue());
|
||||
}
|
||||
|
||||
Terms tfv = reader.getTermVectors(i).terms("utf8");
|
||||
Fields tfvFields = reader.getTermVectors(i);
|
||||
assertNotNull("i=" + i, tfvFields);
|
||||
Terms tfv = tfvFields.terms("utf8");
|
||||
assertNotNull("docID=" + i + " index=" + oldName, tfv);
|
||||
} else
|
||||
} else {
|
||||
// Only ID 7 is deleted
|
||||
assertEquals(7, i);
|
||||
}
|
||||
}
|
||||
|
||||
ScoreDoc[] hits = searcher.search(new TermQuery(new Term("content", "aaa")), null, 1000).scoreDocs;
|
||||
|
|
|
@ -314,7 +314,7 @@ public class TestCodecs extends LuceneTestCase {
|
|||
final SegmentInfo si = new SegmentInfo(dir, Constants.LUCENE_MAIN_VERSION, SEGMENT, 10000, -1, -1,
|
||||
SEGMENT, false, null, false, 0,
|
||||
fieldInfos.hasProx(),
|
||||
codec, null, fieldInfos.hasVectors());
|
||||
codec, null);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: now read postings");
|
||||
|
|
|
@ -202,8 +202,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
r2.close();
|
||||
final SegmentInfo info = new SegmentInfo(si1.dir, Constants.LUCENE_MAIN_VERSION, merged,
|
||||
si1.docCount + si2.docCount, -1, -1, merged,
|
||||
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null,
|
||||
mergeState.fieldInfos.hasVectors());
|
||||
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null);
|
||||
|
||||
if (useCompoundFile) {
|
||||
Collection<String> filesToDelete = IndexWriter.createCompoundFile(dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random()));
|
||||
|
|
|
@ -1547,10 +1547,11 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
DirectoryReader r0 = IndexReader.open(dir);
|
||||
for (IndexReader r : r0.getSequentialSubReaders()) {
|
||||
SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
|
||||
assertFalse(s.getHasVectors());
|
||||
assertFalse(((SegmentReader) r).getFieldInfos().hasVectors());
|
||||
Set<String> files = new HashSet<String>();
|
||||
s.getCodec().termVectorsFormat().files(s, files);
|
||||
assertTrue(files.isEmpty());
|
||||
List<String> filesExisting = SegmentInfo.findMatchingFiles(s.name, dir, files);
|
||||
assertTrue(filesExisting.isEmpty());
|
||||
}
|
||||
|
||||
r0.close();
|
||||
|
|
|
@ -1284,14 +1284,13 @@ public class TestIndexWriterExceptions extends LuceneTestCase {
|
|||
w.close();
|
||||
IndexReader reader = IndexReader.open(dir);
|
||||
assertTrue(reader.numDocs() > 0);
|
||||
reader.close();
|
||||
SegmentInfos sis = new SegmentInfos();
|
||||
sis.read(dir);
|
||||
for (SegmentInfo segmentInfo : sis) {
|
||||
assertFalse(segmentInfo.getHasVectors());
|
||||
for(AtomicReaderContext context : reader.getTopReaderContext().leaves()) {
|
||||
assertFalse(context.reader().getFieldInfos().hasVectors());
|
||||
}
|
||||
reader.close();
|
||||
dir.close();
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,8 +85,7 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
assertTrue(docsMerged == 2);
|
||||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, docsMerged, -1, -1, mergedSegment,
|
||||
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null,
|
||||
mergeState.fieldInfos.hasVectors()),
|
||||
false, null, false, 0, mergeState.fieldInfos.hasProx(), codec, null),
|
||||
DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random()));
|
||||
assertTrue(mergedReader != null);
|
||||
assertTrue(mergedReader.numDocs() == 2);
|
||||
|
|
|
@ -188,7 +188,7 @@ public class TestTermVectorsReader extends LuceneTestCase {
|
|||
DirectoryReader reader = IndexReader.open(dir);
|
||||
for (IndexReader r : reader.getSequentialSubReaders()) {
|
||||
SegmentInfo s = ((SegmentReader) r).getSegmentInfo();
|
||||
assertTrue(s.getHasVectors());
|
||||
assertTrue(((SegmentReader) r).getFieldInfos().hasVectors());
|
||||
Set<String> files = new HashSet<String>();
|
||||
s.getCodec().termVectorsFormat().files(s, files);
|
||||
assertFalse(files.isEmpty());
|
||||
|
|
|
@ -95,7 +95,7 @@ class PreFlexRWSegmentInfosWriter extends SegmentInfosWriter {
|
|||
output.writeInt(si.getDelCount());
|
||||
output.writeByte((byte) (si.getHasProx() ? 1 : 0));
|
||||
output.writeStringStringMap(si.getDiagnostics());
|
||||
output.writeByte((byte) (si.getHasVectors() ? 1: 0));
|
||||
output.writeByte((byte) 1);
|
||||
}
|
||||
|
||||
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
|
||||
|
|
Loading…
Reference in New Issue