From 7c9361ff9e4bec3e79c0782f7e9bc799dcfa2677 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Thu, 17 May 2012 17:00:29 +0000 Subject: [PATCH] LUCENE-4055: decouple SI/FI somewhat git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4055@1339711 13f79535-47bb-0310-9956-ffa450edef68 --- .../apache/lucene/codecs/DocValuesFormat.java | 7 + .../lucene/codecs/FieldInfosFormat.java | 8 + .../apache/lucene/codecs/LiveDocsFormat.java | 4 +- .../org/apache/lucene/codecs/NormsFormat.java | 9 + .../lucene/codecs/PerDocProducerBase.java | 4 + .../apache/lucene/codecs/PostingsFormat.java | 2 +- .../lucene/codecs/StoredFieldsFormat.java | 8 + .../lucene/codecs/TermVectorsFormat.java | 8 + .../lucene3x/Lucene3xSegmentInfosReader.java | 13 +- .../lucene40/Lucene40DocValuesConsumer.java | 16 +- .../codecs/lucene40/Lucene40NormsFormat.java | 14 +- .../lucene40/Lucene40SegmentInfosReader.java | 10 +- .../lucene40/Lucene40SegmentInfosWriter.java | 7 +- .../perfield/PerFieldPostingsFormat.java | 3 +- .../codecs/sep/SepDocValuesConsumer.java | 55 +---- .../lucene/codecs/sep/SepPostingsReader.java | 4 +- .../SimpleTextDocValuesConsumer.java | 5 +- .../simpletext/SimpleTextNormsFormat.java | 18 +- .../simpletext/SimpleTextPerDocConsumer.java | 34 +-- .../SimpleTextSegmentInfosReader.java | 33 ++- .../SimpleTextSegmentInfosWriter.java | 29 ++- .../org/apache/lucene/index/CheckIndex.java | 6 +- .../index/DocumentsWriterPerThread.java | 27 ++- .../org/apache/lucene/index/FieldInfo.java | 6 +- .../org/apache/lucene/index/FieldInfos.java | 24 +- .../apache/lucene/index/IndexFileDeleter.java | 26 +-- .../org/apache/lucene/index/IndexWriter.java | 92 +++++++- .../lucene/index/MutableFieldInfos.java | 5 +- .../lucene/index/ReadOnlyFieldInfos.java | 18 +- .../lucene/index/SegmentCoreReaders.java | 7 +- .../org/apache/lucene/index/SegmentInfo.java | 206 ++++++++++-------- .../org/apache/lucene/index/SegmentInfos.java | 24 -- .../lucene/util/PrintStreamInfoStream.java | 2 +- .../org/apache/lucene/index/TestCodecs.java | 15 +- .../index/TestConsistentFieldNumbers.java | 45 ++-- .../test/org/apache/lucene/index/TestDoc.java | 18 +- .../lucene/index/TestSegmentMerger.java | 10 +- .../lucene/index/TestTermVectorsReader.java | 2 +- .../lucene3x/PreFlexRWSegmentInfosWriter.java | 4 +- .../lucene/store/MockDirectoryWrapper.java | 3 +- .../util/TestRuleSetupAndRestoreClassEnv.java | 22 +- .../org/apache/lucene/util/_TestUtil.java | 25 +++ 42 files changed, 488 insertions(+), 390 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesFormat.java index 6de777361d2..637bbe47aea 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/DocValuesFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/DocValuesFormat.java @@ -30,7 +30,14 @@ import org.apache.lucene.index.SegmentReadState; * @lucene.experimental */ public abstract class DocValuesFormat { + + /** Consumes (writes) doc values during indexing. */ public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException; + + /** Produces (reads) doc values during reading/searching. */ public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; + + /** Gathers files (exact file name or a Pattern regex) + * associated with this segment. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java index a6876052aed..92f5f6ca829 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/FieldInfosFormat.java @@ -28,7 +28,15 @@ import org.apache.lucene.index.SegmentInfo; * @lucene.experimental */ public abstract class FieldInfosFormat { + /** Returns a {@link FieldInfosReader} to read field infos + * from the index */ public abstract FieldInfosReader getFieldInfosReader() throws IOException; + + /** Returns a {@link FieldInfosWriter} to write field infos + * to the index */ public abstract FieldInfosWriter getFieldInfosWriter() throws IOException; + + /** Gathers files (exact file name or a Pattern regex) + * associated with this segment. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java index e48aaf3b395..a016feb921f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/LiveDocsFormat.java @@ -43,7 +43,7 @@ public abstract class LiveDocsFormat { * generation of the deletes file you should write to. */ public abstract void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, int newDelCount, IOContext context) throws IOException; - /** Records all files in use by this {@link SegmentInfo} - * into the files argument. */ + /** Records all files (exact file name or a Pattern regex) + * in use by this {@link SegmentInfo} into the files argument. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/NormsFormat.java index e5924473d1f..03e62581154 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/NormsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/NormsFormat.java @@ -28,7 +28,16 @@ import org.apache.lucene.index.SegmentReadState; * format for normalization factors */ public abstract class NormsFormat { + + /** Returns a {@link PerDocConsumer} to write norms to the + * index. */ public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException; + + /** Returns a {@link PerDocProducer} to read norms from the + * index. */ public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException; + + /** Gathers files (exact file name or a Pattern regex) + * associated with this segment. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java b/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java index d2bfb46b9f6..ddc66f1436f 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PerDocProducerBase.java @@ -97,6 +97,10 @@ public abstract class PerDocProducerBase extends PerDocProducer { public static String docValuesId(String segmentsName, int fieldId) { return segmentsName + "_" + fieldId; } + + public static String docValuesRegex(String segmentsName) { + return segmentsName + "_\\d+"; + } /** * Loads a {@link DocValues} instance depending on the given {@link Type}. diff --git a/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java index 3ed7e6c6ef5..8a2ff11ff5c 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java @@ -57,7 +57,7 @@ public abstract class PostingsFormat implements NamedSPILoader.NamedSPI { public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException; /** - * Gathers files associated with this segment + * Gathers files (exact file name or a Pattern regex) associated with this segment * * @param segmentInfo the {@link SegmentInfo} for this segment * @param segmentSuffix the format's suffix within this segment diff --git a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsFormat.java index 1554b814101..ee7b198d616 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsFormat.java @@ -29,7 +29,15 @@ import org.apache.lucene.store.IOContext; * Controls the format of stored fields */ public abstract class StoredFieldsFormat { + /** Returns a {@link StoredFieldsReader} to load stored + * fields. */ public abstract StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException; + + /** Returns a {@link StoredFieldsWriter} to write stored + * fields. */ public abstract StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException; + + /** Gathers files (exact file name or a Pattern regex) + * associated with this segment. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsFormat.java index a5449f903b8..72af0366ebc 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsFormat.java @@ -29,7 +29,15 @@ import org.apache.lucene.store.IOContext; * Controls the format of term vectors */ public abstract class TermVectorsFormat { + /** Returns a {@link TermVectorsReader} to read term + * vectors. */ public abstract TermVectorsReader vectorsReader(Directory directory, SegmentInfo segmentInfo, FieldInfos fieldInfos, IOContext context) throws IOException; + + /** Returns a {@link TermVectorsWriter} to write term + * vectors. */ public abstract TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException; + + /** Gathers files (exact file name or a Pattern regex) + * associated with this segment. */ public abstract void files(SegmentInfo info, Set files) throws IOException; } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java index b7054ba4ce3..c448d353a4d 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xSegmentInfosReader.java @@ -127,14 +127,14 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader { final int delCount = input.readInt(); assert delCount <= docCount; - final int hasProx = input.readByte(); + final boolean hasProx = input.readByte() == 1; final Codec codec = Codec.forName("Lucene3x"); final Map diagnostics = input.readStringStringMap(); - final int hasVectors; + final boolean hasVectors; if (format <= SegmentInfos.FORMAT_HAS_VECTORS) { - hasVectors = input.readByte(); + hasVectors = input.readByte() == 1; } else { final String storesSegment; final String ext; @@ -155,16 +155,17 @@ class Lucene3xSegmentInfosReader extends SegmentInfosReader { dirToTest = dir; } try { - hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION)) ? SegmentInfo.YES : SegmentInfo.NO; + hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", Lucene3xTermVectorsReader.VECTORS_INDEX_EXTENSION)); } finally { if (isCompoundFile) { dirToTest.close(); } } } - + + // nocommit 3x always has norms? return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset, docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile, - delCount, hasProx, codec, diagnostics, hasVectors); + delCount, hasProx, codec, diagnostics, hasVectors, false, true, true); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesConsumer.java index 30fdcd087e9..e649d6bd387 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40DocValuesConsumer.java @@ -21,8 +21,6 @@ import java.io.IOException; import java.util.Set; import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.PerDocWriteState; import org.apache.lucene.index.SegmentInfo; @@ -69,15 +67,11 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase { } public static void files(SegmentInfo segmentInfo, Set files) throws IOException { - FieldInfos fieldInfos = segmentInfo.getFieldInfos(); - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.hasDocValues()) { - files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION)); - files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); - assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); - assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION)); - break; - } + if (segmentInfo.getHasDocValues()) { + files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION)); + files.add(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); + assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION)); + assert segmentInfo.dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION)); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java index 7e4bc732b31..ec3a9354e04 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40NormsFormat.java @@ -123,15 +123,11 @@ public class Lucene40NormsFormat extends NormsFormat { } public static void files(SegmentInfo segmentInfo, Set files) throws IOException { - final String normsFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION); - FieldInfos fieldInfos = segmentInfo.getFieldInfos(); - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.hasNorms()) { - final String normsEntriesFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION); - files.add(normsFileName); - files.add(normsEntriesFileName); - return; - } + if (segmentInfo.getHasNorms()) { + final String normsFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION); + final String normsEntriesFileName = IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION); + files.add(normsFileName); + files.add(normsEntriesFileName); } } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java index 7070ff7bb3f..71baf25c5ef 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosReader.java @@ -82,13 +82,17 @@ public class Lucene40SegmentInfosReader extends SegmentInfosReader { final int delCount = input.readInt(); assert delCount <= docCount; - final int hasProx = input.readByte(); + final boolean hasProx = input.readByte() == 1; final Codec codec = Codec.forName(input.readString()); final Map diagnostics = input.readStringStringMap(); - final int hasVectors = input.readByte(); + final boolean hasVectors = input.readByte() == 1; + final boolean hasDocValues = input.readByte() == 1; + final boolean hasNorms = input.readByte() == 1; + final boolean hasFreq = 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, hasVectors, hasDocValues, hasNorms, + hasFreq); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java index 670eae6de1c..3bdaba0d331 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40SegmentInfosWriter.java @@ -95,10 +95,13 @@ public class Lucene40SegmentInfosWriter extends SegmentInfosWriter { output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeInt(si.getDelCount()); - output.writeByte((byte) (si.getHasProxInternal())); + output.writeByte((byte) (si.getHasProx() ? 1 : 0)); output.writeString(si.getCodec().getName()); output.writeStringStringMap(si.getDiagnostics()); - output.writeByte((byte) (si.getHasVectorsInternal())); + output.writeByte((byte) (si.getHasVectors() ? 1 : 0)); + output.writeByte((byte) (si.getHasDocValues() ? 1 : 0)); + output.writeByte((byte) (si.getHasNorms() ? 1 : 0)); + output.writeByte((byte) (si.getHasFreq() ? 1 : 0)); } protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context) diff --git a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java index 901c290d441..ef9138ffbf8 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java @@ -336,6 +336,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat { final String mapFileName = IndexFileNames.segmentFileName(info.name, segmentSuffix, PER_FIELD_EXTENSION); files.add(mapFileName); + // nocommit can we use regexp to simplify this? try { new VisitPerFieldFile(dir, info.name, segmentSuffix) { @Override @@ -344,7 +345,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat { } @Override - protected void visitOneField(String field, PostingsFormat format) { + protected void visitOneField(String field, PostingsFormat format) { } }; } catch (FileNotFoundException fnfe) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java index b28b40e4581..44618bf3dca 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepDocValuesConsumer.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.IOUtils; * Implementation of PerDocConsumer that uses separate files. * @lucene.experimental */ + public class SepDocValuesConsumer extends DocValuesWriterBase { private final Directory directory; private final FieldInfos fieldInfos; @@ -52,59 +53,21 @@ public class SepDocValuesConsumer extends DocValuesWriterBase { public static void files(SegmentInfo segmentInfo, Set files) throws IOException { - files(segmentInfo.dir, segmentInfo.getFieldInfos(), segmentInfo.name, files); + files(segmentInfo, files); } @SuppressWarnings("fallthrough") - private static void files(Directory dir,FieldInfos fieldInfos, String segmentName, Set files) { - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.hasDocValues()) { - String filename = PerDocProducerBase.docValuesId(segmentName, fieldInfo.number); - switch (fieldInfo.getDocValuesType()) { - case BYTES_FIXED_DEREF: - case BYTES_VAR_DEREF: - case BYTES_VAR_STRAIGHT: - case BYTES_FIXED_SORTED: - case BYTES_VAR_SORTED: - files.add(IndexFileNames.segmentFileName(filename, "", - INDEX_EXTENSION)); - try { - assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", - INDEX_EXTENSION)); - } catch (IOException e) { - // don't throw checked exception - dir is only used in assert - throw new RuntimeException(e); - } - // until here all types use an index - case BYTES_FIXED_STRAIGHT: - case FLOAT_32: - case FLOAT_64: - case VAR_INTS: - case FIXED_INTS_16: - case FIXED_INTS_32: - case FIXED_INTS_64: - case FIXED_INTS_8: - files.add(IndexFileNames.segmentFileName(filename, "", - DATA_EXTENSION)); - try { - assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", - DATA_EXTENSION)); - } catch (IOException e) { - // don't throw checked exception - dir is only used in assert - throw new RuntimeException(e); - } - break; - default: - assert false; - } - } - } + private static void files(String segmentName, Set files) { + String filename = PerDocProducerBase.docValuesRegex(segmentName); + files.add(IndexFileNames.segmentFileName(filename, "", INDEX_EXTENSION)); + files.add(IndexFileNames.segmentFileName(filename, "", DATA_EXTENSION)); } @Override public void abort() { Set files = new HashSet(); - files(directory, fieldInfos, segmentName, files); - IOUtils.deleteFilesIgnoringExceptions(directory, files.toArray(new String[0])); + assert false: "sep is broken for now!!"; + files(segmentName, files); + IOUtils.deleteFilesIgnoringExceptions(directory, SegmentInfo.findMatchingFiles(directory, files).toArray(new String[0])); } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java index 7ce3b5d0c97..a1fbf09b743 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java @@ -69,7 +69,7 @@ public class SepPostingsReader extends PostingsReaderBase { skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION), context); - if (segmentInfo.getFieldInfos().hasFreq()) { + if (segmentInfo.getHasFreq()) { freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION), context); } else { freqIn = null; @@ -93,7 +93,7 @@ public class SepPostingsReader extends PostingsReaderBase { files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.DOC_EXTENSION)); files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.SKIP_EXTENSION)); - if (segmentInfo.getFieldInfos().hasFreq()) { + if (segmentInfo.getHasFreq()) { files.add(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SepPostingsWriter.FREQ_EXTENSION)); } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java index 89afc2a6932..c6a5e907f3b 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesConsumer.java @@ -286,7 +286,4 @@ public class SimpleTextDocValuesConsumer extends DocValuesConsumer { protected Type getType() { return type; } - - - -} \ No newline at end of file +} diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java index b241027812e..990cf7234e9 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java @@ -129,25 +129,23 @@ public class SimpleTextNormsFormat extends NormsFormat { @Override public void abort() { Set files = new HashSet(); - filesInternal(state.fieldInfos, state.segmentName, files, segmentSuffix); + filesInternal(state.fieldInfos.hasNorms(), state.segmentName, files, segmentSuffix); IOUtils.deleteFilesIgnoringExceptions(state.directory, - files.toArray(new String[0])); + SegmentInfo.findMatchingFiles(state.directory, files).toArray(new String[0])); } public static void files(SegmentInfo segmentInfo, Set files) throws IOException { - filesInternal(segmentInfo.getFieldInfos(), segmentInfo.name, files, + filesInternal(segmentInfo.getHasNorms(), segmentInfo.name, files, NORMS_SEG_SUFFIX); } - public static void filesInternal(FieldInfos fieldInfos, String segmentName, + public static void filesInternal(boolean hasNorms, String segmentName, Set files, String segmentSuffix) { - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.hasNorms()) { - String id = docValuesId(segmentName, fieldInfo.number); - files.add(IndexFileNames.segmentFileName(id, "", - segmentSuffix)); - } + if (hasNorms) { + String id = docValuesIdRegexp(segmentName); + files.add(IndexFileNames.segmentFileName(id, "", + segmentSuffix)); } } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java index d3dfd4485bb..bffc21d1ccd 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java @@ -58,37 +58,27 @@ class SimpleTextPerDocConsumer extends PerDocConsumer { @Override public void abort() { Set files = new HashSet(); - files(state.directory, state.fieldInfos, state.segmentName, files, segmentSuffix); + files(state.directory, state.segmentName, files, segmentSuffix); IOUtils.deleteFilesIgnoringExceptions(state.directory, - files.toArray(new String[0])); + SegmentInfo.findMatchingFiles(state.directory, files).toArray(new String[0])); } - - static void files(SegmentInfo info, Set files, String segmentSuffix) throws IOException { - files(info.dir, info.getFieldInfos(), info.name, files, segmentSuffix); + static void files(SegmentInfo info, Set files, String segmentSuffix) { + files(info.dir, info.name, files, segmentSuffix); } static String docValuesId(String segmentsName, int fieldId) { return segmentsName + "_" + fieldId; } - @SuppressWarnings("fallthrough") - private static void files(Directory dir, FieldInfos fieldInfos, - String segmentName, Set files, String segmentSuffix) { - for (FieldInfo fieldInfo : fieldInfos) { - if (fieldInfo.hasDocValues()) { - String filename = docValuesId(segmentName, fieldInfo.number); - files.add(IndexFileNames.segmentFileName(filename, "", - segmentSuffix)); - try { - assert dir.fileExists(IndexFileNames.segmentFileName(filename, "", - segmentSuffix)); - } catch (IOException e) { - // don't throw checked exception - dir is only used in assert - throw new RuntimeException(e); - } - } - } + static String docValuesIdRegexp(String segmentsName) { + return segmentsName + "_\\d+"; } + @SuppressWarnings("fallthrough") + private static void files(Directory dir, + String segmentName, Set files, String segmentSuffix) { + files.add(IndexFileNames.segmentFileName(docValuesIdRegexp(segmentName), "", + segmentSuffix)); + } } \ No newline at end of file diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java index 590cf6ac47b..a07469087b3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosReader.java @@ -104,11 +104,23 @@ public class SimpleTextSegmentInfosReader extends SegmentInfosReader { SimpleTextUtil.readLine(input, scratch); assert StringHelper.startsWith(scratch, SI_HASPROX); - final int hasProx = readTernary(SI_HASPROX.length, scratch); - + final boolean hasProx = Boolean.parseBoolean(readString(SI_HASPROX.length, scratch)); + SimpleTextUtil.readLine(input, scratch); assert StringHelper.startsWith(scratch, SI_HASVECTORS); - final int hasVectors = readTernary(SI_HASVECTORS.length, scratch); + final boolean hasVectors = Boolean.parseBoolean(readString(SI_HASVECTORS.length, scratch)); + + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, SI_HASDOCVALUES); + final boolean hasDocValues = Boolean.parseBoolean(readString(SI_HASDOCVALUES.length, scratch)); + + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, SI_HASNORMS); + final boolean hasNorms = Boolean.parseBoolean(readString(SI_HASNORMS.length, scratch)); + + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch, SI_HASFREQS); + final boolean hasFreqs = Boolean.parseBoolean(readString(SI_HASFREQS.length, scratch)); SimpleTextUtil.readLine(input, scratch); assert StringHelper.startsWith(scratch, SI_USECOMPOUND); @@ -168,23 +180,10 @@ 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, hasVectors, hasDocValues, hasNorms, hasFreqs); } private String readString(int offset, BytesRef scratch) { return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8); } - - private int readTernary(int offset, BytesRef scratch) throws IOException { - String s = readString(offset, scratch); - if ("check fieldinfo".equals(s)) { - return SegmentInfo.CHECK_FIELDINFO; - } else if ("true".equals(s)) { - return SegmentInfo.YES; - } else if ("false".equals(s)) { - return 0; - } else { - throw new CorruptIndexException("invalid ternary value: " + s); - } - } } diff --git a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java index 18c80cf2312..1ddc2a920f3 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfosWriter.java @@ -53,6 +53,9 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter { 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_HASDOCVALUES = new BytesRef(" has doc values "); + final static BytesRef SI_HASNORMS = new BytesRef(" has norms "); + final static BytesRef SI_HASFREQS = new BytesRef(" has freqs "); 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 "); @@ -147,21 +150,23 @@ public class SimpleTextSegmentInfosWriter extends SegmentInfosWriter { SimpleTextUtil.writeNewline(output); SimpleTextUtil.write(output, SI_HASPROX); - switch(si.getHasProxInternal()) { - case SegmentInfo.YES: SimpleTextUtil.write(output, "true", scratch); break; - case SegmentInfo.CHECK_FIELDINFO: SimpleTextUtil.write(output, "check fieldinfo", scratch); break; - // its "NO" if its 'anything but YES'... such as 0 - default: SimpleTextUtil.write(output, "false", scratch); break; - } + SimpleTextUtil.write(output, si.getHasProx() ? "true" : "false", scratch); SimpleTextUtil.writeNewline(output); SimpleTextUtil.write(output, SI_HASVECTORS); - switch(si.getHasVectorsInternal()) { - case SegmentInfo.YES: SimpleTextUtil.write(output, "true", scratch); break; - case SegmentInfo.CHECK_FIELDINFO: SimpleTextUtil.write(output, "check fieldinfo", scratch); break; - // its "NO" if its 'anything but YES'... such as 0 - default: SimpleTextUtil.write(output, "false", scratch); break; - } + SimpleTextUtil.write(output, si.getHasVectors() ? "true" : "false", scratch); + SimpleTextUtil.writeNewline(output); + + SimpleTextUtil.write(output, SI_HASDOCVALUES); + SimpleTextUtil.write(output, si.getHasDocValues() ? "true" : "false", scratch); + SimpleTextUtil.writeNewline(output); + + SimpleTextUtil.write(output, SI_HASNORMS); + SimpleTextUtil.write(output, si.getHasNorms() ? "true" : "false", scratch); + SimpleTextUtil.writeNewline(output); + + SimpleTextUtil.write(output, SI_HASFREQS); + SimpleTextUtil.write(output, si.getHasFreq() ? "true" : "false", scratch); SimpleTextUtil.writeNewline(output); SimpleTextUtil.write(output, SI_USECOMPOUND); diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index 2d354421da0..efdba5f6392 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -609,7 +609,7 @@ public class CheckIndex { // Test Term Vectors segInfoStat.termVectorStatus = testTermVectors(fieldInfos, info, reader, nf); - segInfoStat.docValuesStatus = testDocValues(info, reader); + segInfoStat.docValuesStatus = testDocValues(info, fieldInfos, reader); // Rethrow the first exception we encountered // This will cause stats for failed segments to be incremented properly @@ -1311,13 +1311,13 @@ public class CheckIndex { } private Status.DocValuesStatus testDocValues(SegmentInfo info, - SegmentReader reader) { + FieldInfos fieldInfos, + SegmentReader reader) { final Status.DocValuesStatus status = new Status.DocValuesStatus(); try { if (infoStream != null) { infoStream.print(" test: DocValues........"); } - final FieldInfos fieldInfos = info.getFieldInfos(); for (FieldInfo fieldInfo : fieldInfos) { if (fieldInfo.hasDocValues()) { status.totalValueFields++; diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index cfca6213cd5..4129f0cca97 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -17,25 +17,26 @@ package org.apache.lucene.index; * limitations under the License. */ -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; - import java.io.IOException; import java.text.NumberFormat; import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.util.Constants; import org.apache.lucene.codecs.Codec; import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FlushInfo; import org.apache.lucene.store.IOContext; -import org.apache.lucene.util.Counter; import org.apache.lucene.util.ByteBlockPool.Allocator; import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator; +import org.apache.lucene.util.Counter; import org.apache.lucene.util.InfoStream; -import org.apache.lucene.util.RamUsageEstimator; import org.apache.lucene.util.MutableBits; +import org.apache.lucene.util.RamUsageEstimator; + +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; class DocumentsWriterPerThread { @@ -474,10 +475,22 @@ class DocumentsWriterPerThread { try { consumer.flush(flushState); pendingDeletes.terms.clear(); - final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly()); + 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(), + flushState.fieldInfos.hasDocValues(), + flushState.fieldInfos.hasNorms(), + flushState.fieldInfos.hasFreq()); 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 " + (newSegment.getHasVectors() ? "vectors" : "no vectors")); + infoStream.message("DWPT", "new segment has " + + (newSegment.getHasVectors() ? "vectors" : "no vectors") + "; " + + (newSegment.getHasNorms() ? "norms" : "no norms") + "; " + + (newSegment.getHasDocValues() ? "docValues" : "no docValues") + "; " + + (newSegment.getHasProx() ? "prox" : "no prox") + "; " + + (newSegment.getHasProx() ? "freqs" : "no freqs")); infoStream.message("DWPT", "flushedFiles=" + newSegment.files()); infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec()); } diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java index 954d09b442c..d1cc1e3c679 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java @@ -1,8 +1,5 @@ package org.apache.lucene.index; -import org.apache.lucene.index.DocValues.Type; - - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -20,6 +17,8 @@ import org.apache.lucene.index.DocValues.Type; * limitations under the License. */ +import org.apache.lucene.index.DocValues.Type; + /** * Access to the Fieldable Info file that describes document fields and whether or * not they are indexed. Each segment has a separate Fieldable Info file. Objects @@ -27,6 +26,7 @@ import org.apache.lucene.index.DocValues.Type; * be adding documents at a time, with no other reader or writer threads * accessing this object. **/ + public final class FieldInfo { public final String name; public final int number; diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java index 2a7e4afb080..134f2d861e0 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/FieldInfos.java @@ -91,6 +91,18 @@ public abstract class FieldInfos implements Cloneable,Iterable { } return false; } + + /** + * @return true if at least one field has doc values + */ + public boolean hasDocValues() { + for (FieldInfo fi : this) { + if (fi.hasDocValues()) { + return true; + } + } + return false; + } /** * @return true if at least one field has any norms @@ -103,16 +115,4 @@ public abstract class FieldInfos implements Cloneable,Iterable { } return false; } - - /** - * @return true if at least one field has docValues - */ - public boolean hasDocValues() { - for (FieldInfo fi : this) { - if (fi.hasDocValues()) { - return true; - } - } - return false; - } } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java index 2f7ec499774..d231bedede3 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java @@ -188,30 +188,6 @@ final class IndexFileDeleter { sis = null; } } - if (sis != null) { - final SegmentInfos infos = sis; - for (SegmentInfo segmentInfo : infos) { - try { - /* - * Force FI to load for each segment since we could see a - * segments file and load successfully above if the files are - * still referenced when they are deleted and the os doesn't let - * you delete them. Yet its likely that fnm files are removed - * while seg file is still around Since LUCENE-2984 we need FI - * to find out if a seg has vectors and prox so we need those - * files to be opened for a commit point. - */ - segmentInfo.getFieldInfos(); - } catch (FileNotFoundException e) { - refresh(segmentInfo.name); - sis = null; - if (infoStream.isEnabled("IFD")) { - infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point"); - } - } - } - - } if (sis != null) { final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directory, sis); if (sis.getGeneration() == segmentInfos.getGeneration()) { @@ -488,7 +464,7 @@ final class IndexFileDeleter { assert locked(); // If this is a commit point, also incRef the // segments_N file: - for( final String fileName: segmentInfos.files(directory, isCommit) ) { + for(final String fileName: segmentInfos.files(directory, isCommit)) { incRef(fileName); } } diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index a205ec4f6e2..0ff98fc8d26 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -661,7 +661,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { rollbackSegments = segmentInfos.createBackupSegmentInfos(true); // start with previous field numbers, but new FieldInfos - globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(); + globalFieldNumberMap = getOrLoadGlobalFieldNumberMap(); docWriter = new DocumentsWriter(codec, config, directory, this, globalFieldNumberMap, bufferedDeletesStream); // Default deleter (for backwards compatibility) is @@ -702,6 +702,55 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { } } } + + private FieldInfos getFieldInfos(SegmentInfo info) throws IOException { + Directory cfsDir = null; + try { + if (info.getUseCompoundFile()) { + cfsDir = new CompoundFileDirectory(directory, + IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), + IOContext.READONCE, + false); + } else { + cfsDir = directory; + } + return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir, + info.name, + IOContext.READONCE); + } finally { + if (info.getUseCompoundFile() && cfsDir != null) { + cfsDir.close(); + } + } + } + + /** + * 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 + */ + private FieldNumberBiMap getOrLoadGlobalFieldNumberMap() throws IOException { + final FieldNumberBiMap map = new FieldNumberBiMap(); + + if (segmentInfos.size() > 0) { + if (segmentInfos.getFormat() > SegmentInfos.FORMAT_DIAGNOSTICS) { + // Pre-3.1 index. In this case we sweep all + // segments, merging their FieldInfos: + for(SegmentInfo info : segmentInfos) { + for(FieldInfo fi : getFieldInfos(info)) { + map.addOrGet(fi.name, fi.number); + } + } + } else { + // Already >= 3.1 index; just seed the FieldInfos + // from the last segment + for(FieldInfo fi : getFieldInfos(segmentInfos.info(segmentInfos.size()-1))) { + map.addOrGet(fi.name, fi.number); + } + } + } + + return map; + } /** * Returns the private {@link IndexWriterConfig}, cloned @@ -2233,14 +2282,20 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { mergedName, MergeState.CheckAbort.NONE, payloadProcessorProvider, new MutableFieldInfos(globalFieldNumberMap), codec, context); - for (IndexReader reader : readers) // add new indexes + for (IndexReader reader : readers) { // add new indexes merger.add(reader); + } + MergeState mergeState = merger.merge(); // merge 'em int docCount = mergeState.mergedDocCount; - final FieldInfos fieldInfos = mergeState.fieldInfos; - SegmentInfo info = new SegmentInfo(mergedName, docCount, directory, - false, codec, - fieldInfos); + 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.hasDocValues(), + mergeState.fieldInfos.hasNorms(), + mergeState.fieldInfos.hasFreq()); + setDiagnostics(info, "addIndexes(IndexReader...)"); boolean useCompoundFile; @@ -3157,7 +3212,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { // Bind a new segment name here so even with // ConcurrentMergePolicy we keep deterministic segment // names. - merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null, new MutableFieldInfos(globalFieldNumberMap)); + merge.info = new SegmentInfo(newSegmentName(), 0, directory, false, null); // TODO: in the non-pool'd case this is somewhat // wasteful, because we open these readers, close them, @@ -3320,11 +3375,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { final MergeState.CheckAbort checkAbort = new MergeState.CheckAbort(merge, directory); SegmentMerger merger = new SegmentMerger(infoStream, directory, config.getTermIndexInterval(), mergedName, checkAbort, - // nocommit - payloadProcessorProvider, (MutableFieldInfos)merge.info.getFieldInfos(), codec, context); + // nocommit + payloadProcessorProvider, new MutableFieldInfos(globalFieldNumberMap), codec, context); if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "merging " + segString(merge.segments) + " mergeVectors=" + merge.info.getFieldInfos().hasVectors()); + infoStream.message("IW", "merging " + segString(merge.segments) + " mergeVectors=" + merge.info.getHasVectors()); } merge.readers = new ArrayList(); @@ -3383,11 +3438,26 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { MergeState mergeState = merger.merge(); 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()); + merge.info.setHasFreq(mergeState.fieldInfos.hasFreq()); + merge.info.setHasDocValues(mergeState.fieldInfos.hasDocValues()); + merge.info.setHasNorms(mergeState.fieldInfos.hasNorms()); + // Record which codec was used to write the segment + + // nocommit stop doing this once we call non-wimpy + // ctor when we make the merge.info: merge.info.setCodec(codec); if (infoStream.isEnabled("IW")) { - infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount); + infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount + "; merged segment has " + + (merge.info.getHasVectors() ? "vectors" : "no vectors") + "; " + + (merge.info.getHasNorms() ? "norms" : "no norms") + "; " + + (merge.info.getHasDocValues() ? "docValues" : "no docValues") + "; " + + (merge.info.getHasProx() ? "prox" : "no prox") + "; " + + (merge.info.getHasProx() ? "freqs" : "no freqs")); } // Very important to do this before opening the reader diff --git a/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java index 1545c30686a..0bf303f4ab8 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/MutableFieldInfos.java @@ -50,8 +50,8 @@ final class MutableFieldInfos extends FieldInfos { final Integer preferredBoxed = Integer.valueOf(preferredFieldNumber); if (preferredFieldNumber != -1 && !numberToName.containsKey(preferredBoxed)) { - // cool - we can use this number globally - fieldNumber = preferredBoxed; + // cool - we can use this number globally + fieldNumber = preferredBoxed; } else { // find a new FieldNumber while (numberToName.containsKey(++lowestUnassignedFieldNumber)) { @@ -62,7 +62,6 @@ final class MutableFieldInfos extends FieldInfos { numberToName.put(fieldNumber, fieldName); nameToNumber.put(fieldName, fieldNumber); - } return fieldNumber.intValue(); diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java b/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java index ba291793483..cb1d5755c1c 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadOnlyFieldInfos.java @@ -1,14 +1,5 @@ package org.apache.lucene.index; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.SortedMap; -import java.util.TreeMap; - -import org.apache.lucene.index.FieldInfo.IndexOptions; - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -26,6 +17,15 @@ import org.apache.lucene.index.FieldInfo.IndexOptions; * limitations under the License. */ +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.lucene.index.FieldInfo.IndexOptions; + // nocommit: temporary public final class ReadOnlyFieldInfos extends FieldInfos { // nocommit diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java index 9c159872ead..26c801bb3a6 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java @@ -24,6 +24,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FieldInfosReader; import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.PerDocProducer; import org.apache.lucene.codecs.PostingsFormat; @@ -98,9 +99,8 @@ final class SegmentCoreReaders { cfsReader = null; cfsDir = dir; } - si.loadFieldInfos(cfsDir, false); // prevent opening the CFS to load fieldInfos - fieldInfos = si.getFieldInfos(); - + fieldInfos = codec.fieldInfosFormat().getFieldInfosReader().read(cfsDir, si.name, IOContext.READONCE); + this.termsIndexDivisor = termsIndexDivisor; final PostingsFormat format = codec.postingsFormat(); final SegmentReadState segmentReadState = new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor); @@ -110,6 +110,7 @@ final class SegmentCoreReaders { // ask codec for its Norms: // TODO: since we don't write any norms file if there are no norms, // kinda jaky to assume the codec handles the case of no norms file at all gracefully?! + // nocommit shouldn't we check si.getHasNorms()/si.getHasDocValues()...? norms = codec.normsFormat().docsProducer(segmentReadState); perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState); diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java index 44cf8e597df..eb29128e494 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java @@ -23,9 +23,10 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Map.Entry; +import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.FieldInfosReader; @@ -70,10 +71,11 @@ public final class SegmentInfo implements Cloneable { private boolean isCompoundFile; - private volatile List files; // cached list of files that this segment uses + // nocommit should we stop caching the files!? + private volatile List 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) + 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) private int docStoreOffset; // if this segment shares stored fields & vectors, this @@ -86,13 +88,11 @@ public final class SegmentInfo implements Cloneable { private int delCount; // How many deleted docs in this segment - //TODO: remove when we don't have to support old indexes anymore that had this field - private int hasVectors = CHECK_FIELDINFO; - //TODO: remove when we don't have to support old indexes anymore that had this field - private int hasProx = CHECK_FIELDINFO; // True if this segment has any fields with positional information - - - private FieldInfos fieldInfos; + private boolean hasVectors; // True if this segment has any term vectors fields + private boolean hasDocValues; // True if this segment has any doc values fields + private boolean hasFreq; // True if this segment has any fields with docFreq information + private boolean hasProx; // True if this segment has any fields with positional information + private boolean hasNorms; // True if this segment has any fields with norms enabled private Codec codec; @@ -109,11 +109,9 @@ public final class SegmentInfo implements Cloneable { // this is never written to/read from the Directory private long bufferedDeletesGen; - // holds the fieldInfos Version to refresh files() cache if FI has changed - private long fieldInfosVersion; - + // nocommit why do we have this wimpy ctor...? public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile, - Codec codec, FieldInfos fieldInfos) { + Codec codec) { this.name = name; this.docCount = docCount; this.dir = dir; @@ -124,7 +122,6 @@ public final class SegmentInfo implements Cloneable { this.codec = codec; delCount = 0; version = Constants.LUCENE_MAIN_VERSION; - this.fieldInfos = fieldInfos; } void setDiagnostics(Map diagnostics) { @@ -142,7 +139,8 @@ 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 normGen, boolean isCompoundFile, - int delCount, int hasProx, Codec codec, Map diagnostics, int hasVectors) { + int delCount, boolean hasProx, Codec codec, Map diagnostics, boolean hasVectors, boolean hasDocValues, + boolean hasNorms, boolean hasFreq) { this.dir = dir; this.version = version; this.name = name; @@ -157,48 +155,75 @@ public final class SegmentInfo implements Cloneable { this.hasProx = hasProx; this.codec = codec; this.diagnostics = diagnostics; + // nocommit remove these now that we can do regexp instead! this.hasVectors = hasVectors; - } - - synchronized void loadFieldInfos(Directory dir, boolean checkCompoundFile) throws IOException { - if (fieldInfos == null) { - Directory dir0 = dir; - if (isCompoundFile && checkCompoundFile) { - dir0 = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(name, - "", IndexFileNames.COMPOUND_FILE_EXTENSION), IOContext.READONCE, false); - } - try { - FieldInfosReader reader = codec.fieldInfosFormat().getFieldInfosReader(); - fieldInfos = reader.read(dir0, name, IOContext.READONCE); - } finally { - if (dir != dir0) { - dir0.close(); - } - } - } + this.hasDocValues = hasDocValues; + this.hasNorms = hasNorms; + this.hasFreq = hasFreq; } /** * Returns total size in bytes of all of files used by this segment */ public long sizeInBytes() throws IOException { - long sum = 0; - for (final String fileName : files()) { - sum += dir.fileLength(fileName); - } - sizeInBytes = sum; - return sizeInBytes; + long sum = 0; + for (final String fileName : files()) { + sum += dir.fileLength(fileName); + } + sizeInBytes = sum; + return sizeInBytes; } + // nocommit: ideally codec stores this info privately: + public boolean getHasFreq() throws IOException { + return hasFreq; + } + + public void setHasFreq(boolean hasFreq) { + this.hasFreq = hasFreq; + clearFilesCache(); + } + + // nocommit: ideally codec stores this info privately: + public boolean getHasProx() throws IOException { + return hasProx; + } + + public void setHasProx(boolean hasProx) { + this.hasProx = hasProx; + clearFilesCache(); + } + + // nocommit: ideally codec stores this info privately: public boolean getHasVectors() throws IOException { - return hasVectors == CHECK_FIELDINFO ? getFieldInfos().hasVectors() : hasVectors == YES; + return hasVectors; + } + + public void setHasVectors(boolean hasVectors) { + this.hasVectors = hasVectors; + clearFilesCache(); + } + + // nocommit: ideally codec stores this info privately: + public boolean getHasDocValues() throws IOException { + return hasDocValues; + } + + public void setHasDocValues(boolean hasDocValues) { + this.hasDocValues = hasDocValues; + clearFilesCache(); } - public FieldInfos getFieldInfos() throws IOException { - loadFieldInfos(dir, true); - return fieldInfos; + // nocommit: ideally codec stores this info privately: + public boolean getHasNorms() throws IOException { + return hasNorms; } + public void setHasNorms(boolean hasNorms) { + this.hasNorms = hasNorms; + clearFilesCache(); + } + public boolean hasDeletions() { // Cases: // @@ -232,24 +257,20 @@ public final class SegmentInfo implements Cloneable { @Override public SegmentInfo clone() { - final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, codec, - fieldInfos == null ? null : fieldInfos.clone()); - si.docStoreOffset = docStoreOffset; - si.docStoreSegment = docStoreSegment; - si.docStoreIsCompoundFile = docStoreIsCompoundFile; - si.delGen = delGen; - si.delCount = delCount; - si.diagnostics = new HashMap(diagnostics); + final HashMap clonedNormGen; if (normGen != null) { - si.normGen = new HashMap(); + clonedNormGen = new HashMap(); for (Entry entry : normGen.entrySet()) { - si.normGen.put(entry.getKey(), entry.getValue()); + clonedNormGen.put(entry.getKey(), entry.getValue()); } + } else { + clonedNormGen = null; } - si.version = version; - si.hasProx = hasProx; - si.hasVectors = hasVectors; - return si; + + return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset, + docStoreSegment, docStoreIsCompoundFile, clonedNormGen, isCompoundFile, + delCount, hasProx, codec, new HashMap(diagnostics), + hasVectors, hasDocValues, hasNorms, hasFreq); } /** @@ -337,10 +358,6 @@ public final class SegmentInfo implements Cloneable { return docStoreSegment; } - public boolean getHasProx() throws IOException { - return hasProx == CHECK_FIELDINFO ? getFieldInfos().hasProx() : hasProx == YES; - } - /** Can only be called once. */ public void setCodec(Codec codec) { assert this.codec == null; @@ -354,6 +371,39 @@ public final class SegmentInfo implements Cloneable { return codec; } + // nocommit move elsewhere? IndexFileNames? + public static List findMatchingFiles(Directory dir, Set namesOrPatterns) { + // nocommit need more efficient way to do this? + List files = new ArrayList(); + final String[] existingFiles; + try { + existingFiles = dir.listAll(); + } catch (IOException ioe) { + // nocommit maybe just throw IOE...? not sure how far up we'd have to change sigs... + throw new RuntimeException(ioe); + } + for(String nameOrPattern : namesOrPatterns) { + boolean exists = false; + try { + exists = dir.fileExists(nameOrPattern); + } catch (IOException ioe) { + // nocommit maybe just throw IOE...? + // Ignore + } + if (exists) { + files.add(nameOrPattern); + } else { + for(String file : existingFiles) { + if (Pattern.matches(nameOrPattern, file)) { + files.add(file); + } + } + } + } + + return files; + } + /* * Return all files referenced by this SegmentInfo. The * returns List is a locally cached List so you should not @@ -361,20 +411,13 @@ public final class SegmentInfo implements Cloneable { */ public List files() throws IOException { - final long fisVersion = fieldInfosVersion; - // nocommit - FieldInfos infos = getFieldInfos(); - if (infos instanceof MutableFieldInfos && fisVersion != (fieldInfosVersion = ((MutableFieldInfos)infos).getVersion())) { - clearFilesCache(); // FIS has modifications - need to recompute - } else if (files != null) { - // Already cached: - return files; + if (files == null) { + // nocommit maybe don't cache...? + // Cache + final Set fileSet = new HashSet(); + codec.files(this, fileSet); + files = findMatchingFiles(dir, fileSet); } - final Set fileSet = new HashSet(); - - codec.files(this, fileSet); - - files = new ArrayList(fileSet); return files; } @@ -502,17 +545,4 @@ public final class SegmentInfo implements Cloneable { public Map getNormGen() { return normGen; } - - // TODO: clean up this SI/FI stuff here - /** returns the 'real' value for hasProx (doesn't consult fieldinfos) - * @lucene.internal */ - public int getHasProxInternal() { - return hasProx; - } - - /** returns the 'real' value for hasVectors (doesn't consult fieldinfos) - * @lucene.internal */ - public int getHasVectorsInternal() { - return hasVectors; - } } diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java index 55aee19cf0f..be88e5a624a 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -130,8 +130,6 @@ public final class SegmentInfos implements Cloneable, Iterable { private int format; - private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand - private List segments = new ArrayList(); private Set segmentSet = new HashSet(); private transient List cachedUnmodifiableList; @@ -896,28 +894,6 @@ public final class SegmentInfos implements Cloneable, Iterable { version++; } - /** - * 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 - */ - FieldNumberBiMap getOrLoadGlobalFieldNumberMap() throws IOException { - if (globalFieldNumberMap != null) { - return globalFieldNumberMap; - } - final FieldNumberBiMap map = new FieldNumberBiMap(); - - if (size() > 0) { - // build the map up - for (SegmentInfo info : this) { - final FieldInfos segFieldInfos = info.getFieldInfos(); - for (FieldInfo fi : segFieldInfos) { - map.addOrGet(fi.name, fi.number); - } - } - } - return globalFieldNumberMap = map; - } - /** applies all changes caused by committing a merge to this SegmentInfos */ void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) { final Set mergedAway = new HashSet(merge.segments); diff --git a/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java b/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java index 186d7ba5000..aa5e689ffbc 100644 --- a/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java +++ b/lucene/core/src/java/org/apache/lucene/util/PrintStreamInfoStream.java @@ -33,7 +33,7 @@ public class PrintStreamInfoStream extends InfoStream { private static final AtomicInteger MESSAGE_ID = new AtomicInteger(); protected final int messageID; - private final PrintStream stream; + protected final PrintStream stream; public PrintStreamInfoStream(PrintStream stream) { this(stream, MESSAGE_ID.getAndIncrement()); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java index c4e7b228fd9..887d5ecd624 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java @@ -31,9 +31,9 @@ import org.apache.lucene.codecs.TermsConsumer; import org.apache.lucene.codecs.lucene3x.Lucene3xCodec; import org.apache.lucene.codecs.mocksep.MockSepPostingsFormat; import org.apache.lucene.document.Document; -import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.document.FieldType; import org.apache.lucene.document.TextField; +import org.apache.lucene.index.FieldInfo.IndexOptions; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.PhraseQuery; @@ -41,6 +41,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Constants; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.OpenBitSet; @@ -253,10 +254,9 @@ public class TestCodecs extends LuceneTestCase { final FieldData[] fields = new FieldData[] {field}; final Directory dir = newDirectory(); - FieldInfos clonedFieldInfos = fieldInfos.clone(); this.write(fieldInfos, dir, fields, true); Codec codec = Codec.getDefault(); - final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, codec, clonedFieldInfos); + final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, codec); final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR)); @@ -309,10 +309,15 @@ public class TestCodecs extends LuceneTestCase { System.out.println("TEST: now write postings"); } - FieldInfos clonedFieldInfos = fieldInfos.clone(); this.write(fieldInfos, dir, fields, false); Codec codec = Codec.getDefault(); - final SegmentInfo si = new SegmentInfo(SEGMENT, 10000, dir, false, codec, clonedFieldInfos); + 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(), + fieldInfos.hasDocValues(), + fieldInfos.hasNorms(), + fieldInfos.hasFreq()); if (VERBOSE) { System.out.println("TEST: now read postings"); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java b/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java index 18ed3280be0..b14d100dc9d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java @@ -26,9 +26,12 @@ import org.apache.lucene.document.FieldType; import org.apache.lucene.document.StoredField; import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; +import org.apache.lucene.store.CompoundFileDirectory; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.util.FailOnNonBulkMergesInfoStream; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util._TestUtil; import org.junit.Test; public class TestConsistentFieldNumbers extends LuceneTestCase { @@ -66,8 +69,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { sis.read(dir); assertEquals(2, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); - FieldInfos fis2 = sis.info(1).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); @@ -84,7 +87,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis3 = sis.info(0).getFieldInfos(); + FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis3.fieldInfo(0).name); assertEquals("f2", fis3.fieldInfo(1).name); @@ -129,8 +132,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { sis.read(dir1); assertEquals(2, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); - FieldInfos fis2 = sis.info(1).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); @@ -140,22 +143,6 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { assertEquals("f3", fis2.fieldInfo(2).name); assertEquals("f4", fis2.fieldInfo(3).name); - writer = new IndexWriter(dir1, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random()))); - writer.forceMerge(1); - writer.close(); - - sis = new SegmentInfos(); - sis.read(dir1); - assertEquals(1, sis.size()); - - FieldInfos fis3 = sis.info(0).getFieldInfos(); - - // after merging the ordering should be identical to the first segment - assertEquals("f1", fis3.fieldInfo(0).name); - assertEquals("f2", fis3.fieldInfo(1).name); - assertEquals("f3", fis3.fieldInfo(2).name); - assertEquals("f4", fis3.fieldInfo(3).name); - dir1.close(); dir2.close(); } @@ -176,7 +163,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); } @@ -195,8 +182,8 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(2, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); - FieldInfos fis2 = sis.info(1).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f1", fis2.fieldInfo(0).name); @@ -218,9 +205,9 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(3, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); - FieldInfos fis2 = sis.info(1).getFieldInfos(); - FieldInfos fis3 = sis.info(2).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); + FieldInfos fis2 = _TestUtil.getFieldInfos(sis.info(1)); + FieldInfos fis3 = _TestUtil.getFieldInfos(sis.info(2)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f1", fis2.fieldInfo(0).name); @@ -252,7 +239,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { SegmentInfos sis = new SegmentInfos(); sis.read(dir); assertEquals(1, sis.size()); - FieldInfos fis1 = sis.info(0).getFieldInfos(); + FieldInfos fis1 = _TestUtil.getFieldInfos(sis.info(0)); assertEquals("f1", fis1.fieldInfo(0).name); assertEquals("f2", fis1.fieldInfo(1).name); assertEquals("f3", fis1.fieldInfo(2).name); @@ -290,7 +277,7 @@ public class TestConsistentFieldNumbers extends LuceneTestCase { SegmentInfos sis = new SegmentInfos(); sis.read(dir); for (SegmentInfo si : sis) { - FieldInfos fis = si.getFieldInfos(); + FieldInfos fis = _TestUtil.getFieldInfos(si); for (FieldInfo fi : fis) { Field expected = getField(Integer.parseInt(fi.name)); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java index 7212aa0ed2b..ff0532f837c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java @@ -22,10 +22,8 @@ import java.io.FileWriter; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; - -import java.util.LinkedList; import java.util.Collection; - +import java.util.LinkedList; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.codecs.Codec; @@ -35,6 +33,7 @@ import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.Constants; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; @@ -201,15 +200,20 @@ public class TestDoc extends LuceneTestCase { MergeState mergeState = merger.merge(); r1.close(); r2.close(); - final FieldInfos fieldInfos = mergeState.fieldInfos; - final SegmentInfo info = new SegmentInfo(merged, si1.docCount + si2.docCount, si1.dir, - false, codec, fieldInfos); + 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(), + mergeState.fieldInfos.hasDocValues(), + mergeState.fieldInfos.hasNorms(), + mergeState.fieldInfos.hasFreq()); if (useCompoundFile) { Collection filesToDelete = IndexWriter.createCompoundFile(dir, merged + ".cfs", MergeState.CheckAbort.NONE, info, newIOContext(random())); info.setUseCompoundFile(true); - for (final String fileToDelete : filesToDelete) + for (final String fileToDelete : filesToDelete) { si1.dir.deleteFile(fileToDelete); + } } return info; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java index 243ab7b50c3..c18115356ec 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java @@ -24,6 +24,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Constants; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; @@ -82,10 +83,13 @@ public class TestSegmentMerger extends LuceneTestCase { MergeState mergeState = merger.merge(); int docsMerged = mergeState.mergedDocCount; assertTrue(docsMerged == 2); - final FieldInfos fieldInfos = mergeState.fieldInfos; //Should be able to open a new SegmentReader against the new directory - SegmentReader mergedReader = new SegmentReader(new SegmentInfo(mergedSegment, docsMerged, mergedDir, false, - codec, fieldInfos), + 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(), + mergeState.fieldInfos.hasDocValues(), + mergeState.fieldInfos.hasNorms(), + mergeState.fieldInfos.hasFreq()), DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random())); assertTrue(mergedReader != null); assertTrue(mergedReader.numDocs() == 2); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java index 03abc20a06d..ac3af7df67d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java @@ -128,7 +128,7 @@ public class TestTermVectorsReader extends LuceneTestCase { seg = writer.newestSegment(); writer.close(); - fieldInfos = seg.getFieldInfos(); //new FieldInfos(dir, IndexFileNames.segmentFileName(seg.name, "", IndexFileNames.FIELD_INFOS_EXTENSION)); + fieldInfos = _TestUtil.getFieldInfos(seg); } @Override diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java index eb1633d9f15..617339e503d 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene3x/PreFlexRWSegmentInfosWriter.java @@ -93,9 +93,9 @@ class PreFlexRWSegmentInfosWriter extends SegmentInfosWriter { output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); output.writeInt(si.getDelCount()); - output.writeByte((byte) (si.getHasProxInternal())); + output.writeByte((byte) (si.getHasProx() ? 1 : 0)); output.writeStringStringMap(si.getDiagnostics()); - output.writeByte((byte) (si.getHasVectorsInternal())); + output.writeByte((byte) (si.getHasVectors() ? 1: 0)); } protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context) diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java index 5bc0e48ce4f..40c4ea92356 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java +++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java @@ -784,8 +784,9 @@ public class MockDirectoryWrapper extends Directory { public IndexInputSlicer createSlicer(final String name, IOContext context) throws IOException { maybeYield(); - if (!delegate.fileExists(name)) + if (!delegate.fileExists(name)) { throw new FileNotFoundException(name); + } // cannot open a file for input if it's still open for // output, except for segments.gen and segments_N if (openFilesForWrite.contains(name) && !name.startsWith("segments")) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java index b1fa37a5642..254f634c6ad 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleSetupAndRestoreClassEnv.java @@ -1,6 +1,7 @@ package org.apache.lucene.util; import java.util.Arrays; +import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.Locale; @@ -20,14 +21,12 @@ import org.apache.lucene.search.RandomSimilarityProvider; import org.apache.lucene.search.similarities.DefaultSimilarity; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; - import com.carrotsearch.randomizedtesting.RandomizedContext; -import static org.apache.lucene.util.LuceneTestCase.VERBOSE; +import static org.apache.lucene.util.LuceneTestCase.*; import static org.apache.lucene.util.LuceneTestCase.INFOSTREAM; import static org.apache.lucene.util.LuceneTestCase.TEST_CODEC; - -import static org.apache.lucene.util.LuceneTestCase.*; +import static org.apache.lucene.util.LuceneTestCase.VERBOSE; @@ -126,7 +125,20 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule { final Random random = RandomizedContext.current().getRandom(); final boolean v = random.nextBoolean(); if (INFOSTREAM) { - InfoStream.setDefault(new PrintStreamInfoStream(System.out)); + InfoStream.setDefault(new PrintStreamInfoStream(System.out) { + @Override + public void message(String component, String message) { + final String name; + if (Thread.currentThread().getName().startsWith("TEST-")) { + // The name of the main thread is way too + // long when looking at IW verbose output...: + name = "main"; + } else { + name = Thread.currentThread().getName(); + } + stream.println(component + " " + messageID + " [" + new Date() + "; " + name + "]: " + message); + } + }); } else if (v) { InfoStream.setDefault(new NullInfoStream()); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java index d4eb5389fea..7d58802d40b 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/_TestUtil.java @@ -55,6 +55,7 @@ import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocsAndPositionsEnum; import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexableField; @@ -62,13 +63,16 @@ import org.apache.lucene.index.LogMergePolicy; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.MergeScheduler; import org.apache.lucene.index.MultiFields; +import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; +import org.apache.lucene.store.CompoundFileDirectory; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; import org.junit.Assert; /** @@ -893,4 +897,25 @@ public class _TestUtil { } } } + + public static FieldInfos getFieldInfos(SegmentInfo info) throws IOException { + Directory cfsDir = null; + try { + if (info.getUseCompoundFile()) { + cfsDir = new CompoundFileDirectory(info.dir, + IndexFileNames.segmentFileName(info.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), + IOContext.READONCE, + false); + } else { + cfsDir = info.dir; + } + return info.getCodec().fieldInfosFormat().getFieldInfosReader().read(cfsDir, + info.name, + IOContext.READONCE); + } finally { + if (info.getUseCompoundFile() && cfsDir != null) { + cfsDir.close(); + } + } + } }