You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2013/02/07 21:48:28 UTC
svn commit: r1443717 [2/14] - in /lucene/dev/trunk: ./ dev-tools/ lucene/
lucene/analysis/ lucene/analysis/common/
lucene/analysis/icu/src/java/org/apache/lucene/collation/
lucene/analysis/icu/src/test/org/apache/lucene/collation/ lucene/backwards/
luc...
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40FieldInfosReader.java Thu Feb 7 20:48:21 2013
@@ -27,8 +27,8 @@ import org.apache.lucene.index.CorruptIn
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.index.DocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@@ -39,8 +39,10 @@ import org.apache.lucene.util.IOUtils;
*
* @lucene.experimental
* @see Lucene40FieldInfosFormat
+ * @deprecated Only for reading old 4.0 and 4.1 segments
*/
-public class Lucene40FieldInfosReader extends FieldInfosReader {
+@Deprecated
+class Lucene40FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
public Lucene40FieldInfosReader() {
@@ -48,14 +50,14 @@ public class Lucene40FieldInfosReader ex
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
- final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION);
+ final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
boolean success = false;
try {
- CodecUtil.checkHeader(input, Lucene40FieldInfosWriter.CODEC_NAME,
- Lucene40FieldInfosWriter.FORMAT_START,
- Lucene40FieldInfosWriter.FORMAT_CURRENT);
+ CodecUtil.checkHeader(input, Lucene40FieldInfosFormat.CODEC_NAME,
+ Lucene40FieldInfosFormat.FORMAT_START,
+ Lucene40FieldInfosFormat.FORMAT_CURRENT);
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
@@ -64,18 +66,18 @@ public class Lucene40FieldInfosReader ex
String name = input.readString();
final int fieldNumber = input.readVInt();
byte bits = input.readByte();
- boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0;
- boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0;
- boolean omitNorms = (bits & Lucene40FieldInfosWriter.OMIT_NORMS) != 0;
- boolean storePayloads = (bits & Lucene40FieldInfosWriter.STORE_PAYLOADS) != 0;
+ boolean isIndexed = (bits & Lucene40FieldInfosFormat.IS_INDEXED) != 0;
+ boolean storeTermVector = (bits & Lucene40FieldInfosFormat.STORE_TERMVECTOR) != 0;
+ boolean omitNorms = (bits & Lucene40FieldInfosFormat.OMIT_NORMS) != 0;
+ boolean storePayloads = (bits & Lucene40FieldInfosFormat.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
- } else if ((bits & Lucene40FieldInfosWriter.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
+ } else if ((bits & Lucene40FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
- } else if ((bits & Lucene40FieldInfosWriter.OMIT_POSITIONS) != 0) {
+ } else if ((bits & Lucene40FieldInfosFormat.OMIT_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
- } else if ((bits & Lucene40FieldInfosWriter.STORE_OFFSETS_IN_POSTINGS) != 0) {
+ } else if ((bits & Lucene40FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
@@ -89,11 +91,20 @@ public class Lucene40FieldInfosReader ex
}
// DV Types are packed in one byte
byte val = input.readByte();
- final DocValues.Type docValuesType = getDocValuesType((byte) (val & 0x0F));
- final DocValues.Type normsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
- final Map<String,String> attributes = input.readStringStringMap();
+ final LegacyDocValuesType oldValuesType = getDocValuesType((byte) (val & 0x0F));
+ final LegacyDocValuesType oldNormsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
+ final Map<String,String> attributes = input.readStringStringMap();;
+ if (oldValuesType.mapping != null) {
+ attributes.put(LEGACY_DV_TYPE_KEY, oldValuesType.name());
+ }
+ if (oldNormsType.mapping != null) {
+ if (oldNormsType.mapping != DocValuesType.NUMERIC) {
+ throw new CorruptIndexException("invalid norm type: " + oldNormsType);
+ }
+ attributes.put(LEGACY_NORM_TYPE_KEY, oldNormsType.name());
+ }
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
- omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
+ omitNorms, storePayloads, indexOptions, oldValuesType.mapping, oldNormsType.mapping, Collections.unmodifiableMap(attributes));
}
if (input.getFilePointer() != input.length()) {
@@ -110,39 +121,35 @@ public class Lucene40FieldInfosReader ex
}
}
}
-
- private static DocValues.Type getDocValuesType(final byte b) {
- switch(b) {
- case 0:
- return null;
- case 1:
- return DocValues.Type.VAR_INTS;
- case 2:
- return DocValues.Type.FLOAT_32;
- case 3:
- return DocValues.Type.FLOAT_64;
- case 4:
- return DocValues.Type.BYTES_FIXED_STRAIGHT;
- case 5:
- return DocValues.Type.BYTES_FIXED_DEREF;
- case 6:
- return DocValues.Type.BYTES_VAR_STRAIGHT;
- case 7:
- return DocValues.Type.BYTES_VAR_DEREF;
- case 8:
- return DocValues.Type.FIXED_INTS_16;
- case 9:
- return DocValues.Type.FIXED_INTS_32;
- case 10:
- return DocValues.Type.FIXED_INTS_64;
- case 11:
- return DocValues.Type.FIXED_INTS_8;
- case 12:
- return DocValues.Type.BYTES_FIXED_SORTED;
- case 13:
- return DocValues.Type.BYTES_VAR_SORTED;
- default:
- throw new IllegalStateException("unhandled indexValues type " + b);
+
+ static final String LEGACY_DV_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".dvtype";
+ static final String LEGACY_NORM_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".normtype";
+
+ // mapping of 4.0 types -> 4.2 types
+ static enum LegacyDocValuesType {
+ NONE(null),
+ VAR_INTS(DocValuesType.NUMERIC),
+ FLOAT_32(DocValuesType.NUMERIC),
+ FLOAT_64(DocValuesType.NUMERIC),
+ BYTES_FIXED_STRAIGHT(DocValuesType.BINARY),
+ BYTES_FIXED_DEREF(DocValuesType.BINARY),
+ BYTES_VAR_STRAIGHT(DocValuesType.BINARY),
+ BYTES_VAR_DEREF(DocValuesType.BINARY),
+ FIXED_INTS_16(DocValuesType.NUMERIC),
+ FIXED_INTS_32(DocValuesType.NUMERIC),
+ FIXED_INTS_64(DocValuesType.NUMERIC),
+ FIXED_INTS_8(DocValuesType.NUMERIC),
+ BYTES_FIXED_SORTED(DocValuesType.SORTED),
+ BYTES_VAR_SORTED(DocValuesType.SORTED);
+
+ final DocValuesType mapping;
+ LegacyDocValuesType(DocValuesType mapping) {
+ this.mapping = mapping;
}
}
+
+ // decodes a 4.0 type
+ private static LegacyDocValuesType getDocValuesType(byte b) {
+ return LegacyDocValuesType.values()[b];
+ }
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsBaseFormat.java Thu Feb 7 20:48:21 2013
@@ -42,7 +42,7 @@ public final class Lucene40PostingsBaseF
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
- return new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ return new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Thu Feb 7 20:48:21 2013
@@ -248,12 +248,12 @@ public class Lucene40PostingsFormat exte
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ PostingsReaderBase postings = new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
- state.dir,
+ state.directory,
state.fieldInfos,
state.segmentInfo,
postings,
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html Thu Feb 7 20:48:21 2013
@@ -363,11 +363,11 @@ file, previously they were stored in tex
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
-({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
-factors need no longer be a single byte, they can be any DocValues
-{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
-strings, they can be any byte sequence. Term offsets can optionally be indexed
-into the postings lists. Payloads can be stored in the term vectors.</li>
+({@code DocValues}) was introduced. Normalization factors need no longer be a
+single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
+Terms need not be unicode strings, they can be any byte sequence. Term offsets
+can optionally be indexed into the postings lists. Payloads can be stored in the
+term vectors.</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Thu Feb 7 20:48:21 2013
@@ -20,19 +20,19 @@ package org.apache.lucene.codecs.lucene4
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
-import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsFormat;
-import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
@@ -64,9 +64,7 @@ public class Lucene41Codec extends Codec
};
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
- private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
- private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@@ -93,17 +91,12 @@ public class Lucene41Codec extends Codec
}
@Override
- public final DocValuesFormat docValuesFormat() {
- return docValuesFormat;
- }
-
- @Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
- public final FieldInfosFormat fieldInfosFormat() {
+ public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@@ -111,11 +104,6 @@ public class Lucene41Codec extends Codec
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
-
- @Override
- public final NormsFormat normsFormat() {
- return normsFormat;
- }
@Override
public final LiveDocsFormat liveDocsFormat() {
@@ -131,5 +119,17 @@ public class Lucene41Codec extends Codec
return defaultFormat;
}
+ @Override
+ public DocValuesFormat docValuesFormat() {
+ return dvFormat;
+ }
+
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
+ private final DocValuesFormat dvFormat = new Lucene40DocValuesFormat();
+ private final NormsFormat normsFormat = new Lucene40NormsFormat();
+
+ @Override
+ public NormsFormat normsFormat() {
+ return normsFormat;
+ }
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java Thu Feb 7 20:48:21 2013
@@ -41,7 +41,7 @@ public final class Lucene41PostingsBaseF
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
- return new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
+ return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Thu Feb 7 20:48:21 2013
@@ -427,14 +427,14 @@ public final class Lucene41PostingsForma
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
- PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.dir,
+ PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
boolean success = false;
try {
- FieldsProducer ret = new BlockTreeTermsReader(state.dir,
+ FieldsProducer ret = new BlockTreeTermsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
postingsReader,
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html Thu Feb 7 20:48:21 2013
@@ -368,11 +368,11 @@ file, previously they were stored in tex
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
-({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
-factors need no longer be a single byte, they can be any DocValues
-{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
-strings, they can be any byte sequence. Term offsets can optionally be indexed
-into the postings lists. Payloads can be stored in the term vectors.</li>
+({@code DocValues}) was introduced. Normalization factors need no longer be a
+single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
+Terms need not be unicode strings, they can be any byte sequence. Term offsets
+can optionally be indexed into the postings lists. Payloads can be stored in the
+term vectors.</li>
<li>In version 4.1, the format of the postings list changed to use either
of FOR compression or variable-byte encoding, depending upon the frequency
of the term.</li>
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Thu Feb 7 20:48:21 2013
@@ -37,7 +37,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.IOUtils;
/**
- * Enables per field format support.
+ * Enables per field postings support.
* <p>
* Note, when extending this class, the name ({@link #getName}) is
* written into the index. In order for the field to be read, the
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/DoubleDocValuesField.java Thu Feb 7 20:48:21 2013
@@ -17,34 +17,21 @@ package org.apache.lucene.document;
* limitations under the License.
*/
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.AtomicReader; // javadocs
+import org.apache.lucene.search.FieldCache; // javadocs
/**
+ * Syntactic sugar for encoding doubles as NumericDocValues
+ * via {@link Double#doubleToRawLongBits(double)}.
* <p>
- * Field that stores a per-document <code>double</code> value for scoring,
- * sorting or value retrieval. Here's an example usage:
- *
- * <pre class="prettyprint">
- * document.add(new DoubleDocValuesField(name, 22.0));
- * </pre>
- *
+ * Per-document double values can be retrieved via
+ * {@link FieldCache#getDoubles(AtomicReader, String, boolean)}.
* <p>
- * If you also need to store the value, you should add a
- * separate {@link StoredField} instance.
- *
- * @see DocValues
- * */
-
-public class DoubleDocValuesField extends StoredField {
-
- /**
- * Type for 64-bit double DocValues.
- */
- public static final FieldType TYPE = new FieldType();
- static {
- TYPE.setDocValueType(DocValues.Type.FLOAT_64);
- TYPE.freeze();
- }
+ * <b>NOTE</b>: In most all cases this will be rather inefficient,
+ * requiring eight bytes per document. Consider encoding double
+ * values yourself with only as much precision as you require.
+ */
+public class DoubleDocValuesField extends NumericDocValuesField {
/**
* Creates a new DocValues field with the specified 64-bit double value
@@ -53,7 +40,16 @@ public class DoubleDocValuesField extend
* @throws IllegalArgumentException if the field name is null
*/
public DoubleDocValuesField(String name, double value) {
- super(name, TYPE);
- fieldsData = Double.valueOf(value);
+ super(name, Double.doubleToRawLongBits(value));
+ }
+
+ @Override
+ public void setDoubleValue(double value) {
+ super.setLongValue(Double.doubleToRawLongBits(value));
+ }
+
+ @Override
+ public void setLongValue(long value) {
+ throw new IllegalArgumentException("cannot change value type from Double to Long");
}
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/Field.java Thu Feb 7 20:48:21 2013
@@ -29,7 +29,6 @@ import org.apache.lucene.document.FieldT
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.IndexableFieldType;
-import org.apache.lucene.index.Norm; // javadocs
import org.apache.lucene.index.StorableField;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.FieldInvertState; // javadocs
@@ -38,13 +37,8 @@ import org.apache.lucene.index.FieldInve
* Expert: directly create a field for a document. Most
* users should use one of the sugar subclasses: {@link
* IntField}, {@link LongField}, {@link FloatField}, {@link
- * DoubleField}, {@link ByteDocValuesField}, {@link
- * ShortDocValuesField}, {@link IntDocValuesField}, {@link
- * LongDocValuesField}, {@link PackedLongDocValuesField},
- * {@link FloatDocValuesField}, {@link
- * DoubleDocValuesField}, {@link SortedBytesDocValuesField},
- * {@link DerefBytesDocValuesField}, {@link
- * StraightBytesDocValuesField}, {@link
+ * DoubleField}, {@link BinaryDocValuesField}, {@link
+ * NumericDocValuesField}, {@link SortedDocValuesField}, {@link
* StringField}, {@link TextField}, {@link StoredField}.
*
* <p/> A field is a section of a Document. Each field has three
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FieldType.java Thu Feb 7 20:48:21 2013
@@ -18,7 +18,7 @@ package org.apache.lucene.document;
*/
import org.apache.lucene.analysis.Analyzer; // javadocs
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexableFieldType;
import org.apache.lucene.search.NumericRangeQuery; // javadocs
@@ -55,7 +55,7 @@ public class FieldType implements Indexa
private NumericType numericType;
private boolean frozen;
private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
- private DocValues.Type docValueType;
+ private DocValuesType docValueType;
/**
* Create a new mutable FieldType with all of the properties from <code>ref</code>
@@ -416,21 +416,21 @@ public class FieldType implements Indexa
* {@inheritDoc}
* <p>
* The default is <code>null</code> (no docValues)
- * @see #setDocValueType(DocValues.Type)
+ * @see #setDocValueType(org.apache.lucene.index.FieldInfo.DocValuesType)
*/
@Override
- public DocValues.Type docValueType() {
+ public DocValuesType docValueType() {
return docValueType;
}
/**
- * Set's the field's DocValues.Type
+ * Set's the field's DocValuesType
* @param type DocValues type, or null if no DocValues should be stored.
* @throws IllegalStateException if this FieldType is frozen against
* future modifications.
* @see #docValueType()
*/
- public void setDocValueType(DocValues.Type type) {
+ public void setDocValueType(DocValuesType type) {
checkIfFrozen();
docValueType = type;
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/document/FloatDocValuesField.java Thu Feb 7 20:48:21 2013
@@ -17,33 +17,21 @@ package org.apache.lucene.document;
* limitations under the License.
*/
-import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.AtomicReader; // javadocs
+import org.apache.lucene.search.FieldCache; // javadocs
/**
+ * Syntactic sugar for encoding floats as NumericDocValues
+ * via {@link Float#floatToRawIntBits(float)}.
* <p>
- * Field that stores a per-document <code>float</code> value for scoring,
- * sorting or value retrieval. Here's an example usage:
- *
- * <pre class="prettyprint">
- * document.add(new FloatDocValuesField(name, 22f));
- * </pre>
- *
+ * Per-document floating point values can be retrieved via
+ * {@link FieldCache#getFloats(AtomicReader, String, boolean)}.
* <p>
- * If you also need to store the value, you should add a
- * separate {@link StoredField} instance.
- * @see DocValues
- * */
-
-public class FloatDocValuesField extends StoredField {
-
- /**
- * Type for 32-bit float DocValues.
- */
- public static final FieldType TYPE = new FieldType();
- static {
- TYPE.setDocValueType(DocValues.Type.FLOAT_32);
- TYPE.freeze();
- }
+ * <b>NOTE</b>: In most all cases this will be rather inefficient,
+ * requiring four bytes per document. Consider encoding floating
+ * point values yourself with only as much precision as you require.
+ */
+public class FloatDocValuesField extends NumericDocValuesField {
/**
* Creates a new DocValues field with the specified 32-bit float value
@@ -52,7 +40,16 @@ public class FloatDocValuesField extends
* @throws IllegalArgumentException if the field name is null
*/
public FloatDocValuesField(String name, float value) {
- super(name, TYPE);
- fieldsData = Float.valueOf(value);
+ super(name, Float.floatToRawIntBits(value));
+ }
+
+ @Override
+ public void setFloatValue(float value) {
+ super.setLongValue(Float.floatToRawIntBits(value));
+ }
+
+ @Override
+ public void setLongValue(long value) {
+ throw new IllegalArgumentException("cannot change value type from Float to Long");
}
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Thu Feb 7 20:48:21 2013
@@ -156,19 +156,30 @@ public abstract class AtomicReader exten
}
return null;
}
-
- /**
- * Returns {@link DocValues} for this field.
- * This method may return null if the reader has no per-document
- * values stored.
- */
- public abstract DocValues docValues(String field) throws IOException;
-
- /**
- * Returns {@link DocValues} for this field's normalization values.
- * This method may return null if the field has no norms.
- */
- public abstract DocValues normValues(String field) throws IOException;
+
+ /** Returns {@link NumericDocValues} for this field, or
+ * null if no {@link NumericDocValues} were indexed for
+ * this field. The returned instance should only be
+ * used by a single thread. */
+ public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
+
+ /** Returns {@link BinaryDocValues} for this field, or
+ * null if no {@link BinaryDocValues} were indexed for
+ * this field. The returned instance should only be
+ * used by a single thread. */
+ public abstract BinaryDocValues getBinaryDocValues(String field) throws IOException;
+
+ /** Returns {@link SortedDocValues} for this field, or
+ * null if no {@link SortedDocValues} were indexed for
+ * this field. The returned instance should only be
+ * used by a single thread. */
+ public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
+
+ /** Returns {@link NumericDocValues} representing norms
+ * for this field, or null if no {@link NumericDocValues}
+ * were indexed. The returned instance should only be
+ * used by a single thread. */
+ public abstract NumericDocValues getNormValues(String field) throws IOException;
/**
* Get the {@link FieldInfos} describing all fields in
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Thu Feb 7 20:48:21 2013
@@ -31,10 +31,7 @@ import java.util.Map;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
-import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType; // for javadocs
-import org.apache.lucene.index.DocValues.SortedSource;
-import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
@@ -183,7 +180,7 @@ public class CheckIndex {
/** Number of deleted documents. */
public int numDeleted;
- /** True if we were able to open a SegmentReader on this
+ /** True if we were able to open an AtomicReader on this
* segment. */
public boolean openReaderPassed;
@@ -343,9 +340,9 @@ public class CheckIndex {
setInfoStream(out, false);
}
- private void msg(String msg) {
- if (infoStream != null)
- infoStream.println(msg);
+ private static void msg(PrintStream out, String msg) {
+ if (out != null)
+ out.println(msg);
}
/** Returns a {@link Status} instance detailing
@@ -381,7 +378,7 @@ public class CheckIndex {
try {
sis.read(dir);
} catch (Throwable t) {
- msg("ERROR: could not read any segments file in directory");
+ msg(infoStream, "ERROR: could not read any segments file in directory");
result.missingSegments = true;
if (infoStream != null)
t.printStackTrace(infoStream);
@@ -416,7 +413,7 @@ public class CheckIndex {
try {
input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
} catch (Throwable t) {
- msg("ERROR: could not open segments file in directory");
+ msg(infoStream, "ERROR: could not open segments file in directory");
if (infoStream != null)
t.printStackTrace(infoStream);
result.cantOpenSegments = true;
@@ -426,7 +423,7 @@ public class CheckIndex {
try {
format = input.readInt();
} catch (Throwable t) {
- msg("ERROR: could not read segment file version in directory");
+ msg(infoStream, "ERROR: could not read segment file version in directory");
if (infoStream != null)
t.printStackTrace(infoStream);
result.missingSegmentVersion = true;
@@ -460,7 +457,7 @@ public class CheckIndex {
versionString = oldest.equals(newest) ? ( "version=" + oldest ) : ("versions=[" + oldest + " .. " + newest + "]");
}
- msg("Segments file=" + segmentsFileName + " numSegments=" + numSegments
+ msg(infoStream, "Segments file=" + segmentsFileName + " numSegments=" + numSegments
+ " " + versionString + " format=" + sFormat + userDataString);
if (onlySegments != null) {
@@ -472,11 +469,11 @@ public class CheckIndex {
infoStream.print(" " + s);
}
result.segmentsChecked.addAll(onlySegments);
- msg(":");
+ msg(infoStream, ":");
}
if (skip) {
- msg("\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
+ msg(infoStream, "\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
result.toolOutOfDate = true;
return result;
}
@@ -497,38 +494,41 @@ public class CheckIndex {
}
Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
result.segmentInfos.add(segInfoStat);
- msg(" " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
+ msg(infoStream, " " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
segInfoStat.name = info.info.name;
segInfoStat.docCount = info.info.getDocCount();
int toLoseDocCount = info.info.getDocCount();
- SegmentReader reader = null;
+ AtomicReader reader = null;
try {
final Codec codec = info.info.getCodec();
- msg(" codec=" + codec);
+ msg(infoStream, " codec=" + codec);
segInfoStat.codec = codec;
- msg(" compound=" + info.info.getUseCompoundFile());
+ msg(infoStream, " compound=" + info.info.getUseCompoundFile());
segInfoStat.compound = info.info.getUseCompoundFile();
- msg(" numFiles=" + info.files().size());
+ msg(infoStream, " numFiles=" + info.files().size());
segInfoStat.numFiles = info.files().size();
segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
- msg(" size (MB)=" + nf.format(segInfoStat.sizeMB));
+ msg(infoStream, " size (MB)=" + nf.format(segInfoStat.sizeMB));
Map<String,String> diagnostics = info.info.getDiagnostics();
segInfoStat.diagnostics = diagnostics;
if (diagnostics.size() > 0) {
- msg(" diagnostics = " + diagnostics);
+ msg(infoStream, " diagnostics = " + diagnostics);
}
- // TODO: we could append the info attributes() to the msg?
+ Map<String,String> atts = info.info.attributes();
+ if (atts != null && !atts.isEmpty()) {
+ msg(infoStream, " attributes = " + atts);
+ }
if (!info.hasDeletions()) {
- msg(" no deletions");
+ msg(infoStream, " no deletions");
segInfoStat.hasDeletions = false;
}
else{
- msg(" has deletions [delGen=" + info.getDelGen() + "]");
+ msg(infoStream, " has deletions [delGen=" + info.getDelGen() + "]");
segInfoStat.hasDeletions = true;
segInfoStat.deletionsGen = info.getDelGen();
}
@@ -566,7 +566,7 @@ public class CheckIndex {
}
segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
- msg("OK [" + (segInfoStat.numDeleted) + " deleted docs]");
+ msg(infoStream, "OK [" + (segInfoStat.numDeleted) + " deleted docs]");
} else {
if (info.getDelCount() != 0) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
@@ -580,7 +580,7 @@ public class CheckIndex {
}
}
}
- msg("OK");
+ msg(infoStream, "OK");
}
if (reader.maxDoc() != info.info.getDocCount()) {
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
@@ -591,22 +591,22 @@ public class CheckIndex {
infoStream.print(" test: fields..............");
}
FieldInfos fieldInfos = reader.getFieldInfos();
- msg("OK [" + fieldInfos.size() + " fields]");
+ msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
segInfoStat.numFields = fieldInfos.size();
// Test Field Norms
- segInfoStat.fieldNormStatus = testFieldNorms(fieldInfos, reader);
+ segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream);
// Test the Term Index
- segInfoStat.termIndexStatus = testPostings(fieldInfos, reader);
+ segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose);
// Test Stored Fields
- segInfoStat.storedFieldStatus = testStoredFields(info, reader, nf);
+ segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream);
// Test Term Vectors
- segInfoStat.termVectorStatus = testTermVectors(fieldInfos, info, reader, nf);
-
- segInfoStat.docValuesStatus = testDocValues(info, fieldInfos, reader);
+ segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors);
+
+ segInfoStat.docValuesStatus = testDocValues(reader, infoStream);
// Rethrow the first exception we encountered
// This will cause stats for failed segments to be incremented properly
@@ -622,16 +622,16 @@ public class CheckIndex {
throw new RuntimeException("DocValues test failed");
}
- msg("");
+ msg(infoStream, "");
} catch (Throwable t) {
- msg("FAILED");
+ msg(infoStream, "FAILED");
String comment;
comment = "fixIndex() would remove reference to this segment";
- msg(" WARNING: " + comment + "; full exception:");
+ msg(infoStream, " WARNING: " + comment + "; full exception:");
if (infoStream != null)
t.printStackTrace(infoStream);
- msg("");
+ msg(infoStream, "");
result.totLoseDocCount += toLoseDocCount;
result.numBadSegments++;
continue;
@@ -647,16 +647,16 @@ public class CheckIndex {
if (0 == result.numBadSegments) {
result.clean = true;
} else
- msg("WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
+ msg(infoStream, "WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
if ( ! (result.validCounter = (result.maxSegmentName < sis.counter))) {
result.clean = false;
result.newSegments.counter = result.maxSegmentName + 1;
- msg("ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
+ msg(infoStream, "ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
}
if (result.clean) {
- msg("No problems were detected with this index.\n");
+ msg(infoStream, "No problems were detected with this index.\n");
}
return result;
@@ -664,8 +664,9 @@ public class CheckIndex {
/**
* Test field norms.
+ * @lucene.experimental
*/
- private Status.FieldNormStatus testFieldNorms(FieldInfos fieldInfos, SegmentReader reader) {
+ public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream) {
final Status.FieldNormStatus status = new Status.FieldNormStatus();
try {
@@ -673,21 +674,20 @@ public class CheckIndex {
if (infoStream != null) {
infoStream.print(" test: field norms.........");
}
- for (FieldInfo info : fieldInfos) {
+ for (FieldInfo info : reader.getFieldInfos()) {
if (info.hasNorms()) {
- DocValues dv = reader.normValues(info.name);
- checkDocValues(dv, info.name, info.getNormType(), reader.maxDoc());
+ checkNorms(info, reader, infoStream);
++status.totFields;
} else {
- if (reader.normValues(info.name) != null) {
+ if (reader.getNormValues(info.name) != null) {
throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
}
}
}
- msg("OK [" + status.totFields + " fields]");
+ msg(infoStream, "OK [" + status.totFields + " fields]");
} catch (Throwable e) {
- msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+ msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@@ -701,14 +701,14 @@ public class CheckIndex {
* checks Fields api is consistent with itself.
* searcher is optional, to verify with queries. Can be null.
*/
- private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors) throws IOException {
+ private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
// TODO: we should probably return our own stats thing...?!
final Status.TermIndexStatus status = new Status.TermIndexStatus();
int computedFieldCount = 0;
if (fields == null) {
- msg("OK [no fields/terms]");
+ msg(infoStream, "OK [no fields/terms]");
return status;
}
@@ -769,7 +769,7 @@ public class CheckIndex {
break;
}
- checkBounds(term);
+ assert term.isValid();
// make sure terms arrive in order according to
// the comp
@@ -859,7 +859,7 @@ public class CheckIndex {
lastPos = pos;
BytesRef payload = postings.getPayload();
if (payload != null) {
- checkBounds(payload);
+ assert payload.isValid();
}
if (payload != null && payload.length < 1) {
throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " payload length is out of bounds " + payload.length);
@@ -1155,7 +1155,7 @@ public class CheckIndex {
}
if (doPrint) {
- msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
+ msg(infoStream, "OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
}
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
@@ -1170,8 +1170,17 @@ public class CheckIndex {
/**
* Test the term index.
+ * @lucene.experimental
+ */
+ public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) {
+ return testPostings(reader, infoStream, false);
+ }
+
+ /**
+ * Test the term index.
+ * @lucene.experimental
*/
- private Status.TermIndexStatus testPostings(FieldInfos fieldInfos, SegmentReader reader) {
+ public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose) {
// TODO: we should go and verify term vectors match, if
// crossCheckTermVectors is on...
@@ -1186,15 +1195,16 @@ public class CheckIndex {
}
final Fields fields = reader.fields();
- status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false);
+ final FieldInfos fieldInfos = reader.getFieldInfos();
+ status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false, infoStream, verbose);
if (liveDocs != null) {
if (infoStream != null) {
infoStream.print(" test (ignoring deletes): terms, freq, prox...");
}
- checkFields(fields, null, maxDoc, fieldInfos, true, false);
+ checkFields(fields, null, maxDoc, fieldInfos, true, false, infoStream, verbose);
}
} catch (Throwable e) {
- msg("ERROR: " + e);
+ msg(infoStream, "ERROR: " + e);
status = new Status.TermIndexStatus();
status.error = e;
if (infoStream != null) {
@@ -1206,9 +1216,10 @@ public class CheckIndex {
}
/**
- * Test stored fields for a segment.
+ * Test stored fields.
+ * @lucene.experimental
*/
- private Status.StoredFieldStatus testStoredFields(SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
+ public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream) {
final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
try {
@@ -1218,7 +1229,7 @@ public class CheckIndex {
// Scan stored fields for all documents
final Bits liveDocs = reader.getLiveDocs();
- for (int j = 0; j < info.info.getDocCount(); ++j) {
+ for (int j = 0; j < reader.maxDoc(); ++j) {
// Intentionally pull even deleted documents to
// make sure they too are not corrupt:
StoredDocument doc = reader.document(j);
@@ -1233,10 +1244,10 @@ public class CheckIndex {
throw new RuntimeException("docCount=" + status.docCount + " but saw " + status.docCount + " undeleted docs");
}
- msg("OK [" + status.totFields + " total field count; avg " +
- format.format((((float) status.totFields)/status.docCount)) + " fields per doc]");
+ msg(infoStream, "OK [" + status.totFields + " total field count; avg " +
+ NumberFormat.getInstance(Locale.ROOT).format((((float) status.totFields)/status.docCount)) + " fields per doc]");
} catch (Throwable e) {
- msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+ msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@@ -1246,124 +1257,33 @@ public class CheckIndex {
return status;
}
- /** Helper method to verify values (either docvalues or norms), also checking
- * type and size against fieldinfos/segmentinfo
+ /**
+ * Test docvalues.
+ * @lucene.experimental
*/
- private void checkDocValues(DocValues docValues, String fieldName, DocValues.Type expectedType, int expectedDocs) throws IOException {
- if (docValues == null) {
- throw new RuntimeException("field: " + fieldName + " omits docvalues but should have them!");
- }
- DocValues.Type type = docValues.getType();
- if (type != expectedType) {
- throw new RuntimeException("field: " + fieldName + " has type: " + type + " but fieldInfos says:" + expectedType);
- }
- final Source values = docValues.getDirectSource();
- int size = docValues.getValueSize();
- for (int i = 0; i < expectedDocs; i++) {
- switch (type) {
- case BYTES_FIXED_SORTED:
- case BYTES_VAR_SORTED:
- case BYTES_FIXED_DEREF:
- case BYTES_FIXED_STRAIGHT:
- case BYTES_VAR_DEREF:
- case BYTES_VAR_STRAIGHT:
- BytesRef bytes = new BytesRef();
- values.getBytes(i, bytes);
- if (size != -1 && size != bytes.length) {
- throw new RuntimeException("field: " + fieldName + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
- }
- break;
- case FLOAT_32:
- assert size == 4;
- values.getFloat(i);
- break;
- case FLOAT_64:
- assert size == 8;
- values.getFloat(i);
- break;
- case VAR_INTS:
- assert size == -1;
- values.getInt(i);
- break;
- case FIXED_INTS_16:
- assert size == 2;
- values.getInt(i);
- break;
- case FIXED_INTS_32:
- assert size == 4;
- values.getInt(i);
- break;
- case FIXED_INTS_64:
- assert size == 8;
- values.getInt(i);
- break;
- case FIXED_INTS_8:
- assert size == 1;
- values.getInt(i);
- break;
- default:
- throw new IllegalArgumentException("Field: " + fieldName
- + " - no such DocValues type: " + type);
- }
- }
- if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
- // check sorted bytes
- SortedSource sortedValues = values.asSortedSource();
- Comparator<BytesRef> comparator = sortedValues.getComparator();
- int maxOrd = sortedValues.getValueCount() - 1;
- FixedBitSet seenOrds = new FixedBitSet(sortedValues.getValueCount());
- int lastOrd = -1;
- BytesRef lastBytes = new BytesRef();
- for (int i = 0; i < expectedDocs; i++) {
- int ord = sortedValues.ord(i);
- if (ord < 0 || ord > maxOrd) {
- throw new RuntimeException("field: " + fieldName + " ord is out of bounds: " + ord);
- }
- BytesRef bytes = new BytesRef();
- sortedValues.getByOrd(ord, bytes);
- if (lastOrd != -1) {
- int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
- int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
- if (ordComp != bytesComp) {
- throw new RuntimeException("field: " + fieldName + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
- }
- }
- lastOrd = ord;
- lastBytes = bytes;
- seenOrds.set(ord);
- }
- if (seenOrds.cardinality() != sortedValues.getValueCount()) {
- // TODO: find the bug here and figure out a workaround (we can implement in LUCENE-4547's back compat layer maybe)
- // basically ord 0 is unused by any docs: so the sortedbytes ords are all off-by-one
- // does it always happen? e.g. maybe only if there are missing values? or a bug in its merge optimizations?
- // throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + sortedValues.getValueCount() + " but only used: " + seenOrds.cardinality());
- }
- }
- }
-
- private Status.DocValuesStatus testDocValues(SegmentInfoPerCommit info,
- FieldInfos fieldInfos,
- SegmentReader reader) {
+ public static Status.DocValuesStatus testDocValues(AtomicReader reader,
+ PrintStream infoStream) {
final Status.DocValuesStatus status = new Status.DocValuesStatus();
try {
if (infoStream != null) {
infoStream.print(" test: docvalues...........");
}
- for (FieldInfo fieldInfo : fieldInfos) {
+ for (FieldInfo fieldInfo : reader.getFieldInfos()) {
if (fieldInfo.hasDocValues()) {
status.totalValueFields++;
- final DocValues docValues = reader.docValues(fieldInfo.name);
- checkDocValues(docValues, fieldInfo.name, fieldInfo.getDocValuesType(), reader.maxDoc());
+ checkDocValues(fieldInfo, reader, infoStream);
} else {
- if (reader.docValues(fieldInfo.name) != null) {
+ if (reader.getBinaryDocValues(fieldInfo.name) != null ||
+ reader.getNumericDocValues(fieldInfo.name) != null ||
+ reader.getSortedDocValues(fieldInfo.name) != null) {
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
}
}
}
- msg("OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
+ msg(infoStream, "OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
} catch (Throwable e) {
- msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+ msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@@ -1372,37 +1292,94 @@ public class CheckIndex {
return status;
}
- // basic value checks
- private static void checkBounds(BytesRef b) {
- if (b.bytes == null) {
- throw new RuntimeException("bytes is null");
- }
- if (b.length < 0) {
- throw new RuntimeException("length is negative: " + b.length);
- }
- if (b.length > b.bytes.length) {
- throw new RuntimeException("length is out of bounds: " + b.length + ", bytes.length=" + b.bytes.length);
+ private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
+ BytesRef scratch = new BytesRef();
+ for (int i = 0; i < reader.maxDoc(); i++) {
+ dv.get(i, scratch);
+ assert scratch.isValid();
}
- if (b.offset < 0) {
- throw new RuntimeException("offset is negative: " + b.offset);
+ }
+
+ private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
+ checkBinaryDocValues(fieldName, reader, dv);
+ final int maxOrd = dv.getValueCount()-1;
+ FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
+ int maxOrd2 = -1;
+ for (int i = 0; i < reader.maxDoc(); i++) {
+ int ord = dv.getOrd(i);
+ if (ord < 0 || ord > maxOrd) {
+ throw new RuntimeException("ord out of bounds: " + ord);
+ }
+ maxOrd2 = Math.max(maxOrd2, ord);
+ seenOrds.set(ord);
+ }
+ if (maxOrd != maxOrd2) {
+ throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
+ }
+ if (seenOrds.cardinality() != dv.getValueCount()) {
+ throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + dv.getValueCount() + " but only used: " + seenOrds.cardinality());
+ }
+ BytesRef lastValue = null;
+ BytesRef scratch = new BytesRef();
+ for (int i = 0; i <= maxOrd; i++) {
+ dv.lookupOrd(i, scratch);
+ assert scratch.isValid();
+ if (lastValue != null) {
+ if (scratch.compareTo(lastValue) <= 0) {
+ throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + scratch);
+ }
+ }
+ lastValue = BytesRef.deepCopyOf(scratch);
}
- if (b.offset > b.bytes.length) {
- throw new RuntimeException("offset out of bounds: " + b.offset + ", length=" + b.length);
+ }
+
+ private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
+ for (int i = 0; i < reader.maxDoc(); i++) {
+ ndv.get(i);
}
- if (b.offset + b.length < 0) {
- throw new RuntimeException("offset+length is negative: offset=" + b.offset + ",length=" + b.length);
+ }
+
+ private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws Exception {
+ switch(fi.getDocValuesType()) {
+ case SORTED:
+ checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
+ break;
+ case BINARY:
+ checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
+ break;
+ case NUMERIC:
+ checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
+ break;
+ default:
+ throw new AssertionError();
}
- if (b.offset + b.length > b.bytes.length) {
- throw new RuntimeException("offset+length out of bounds: offset=" + b.offset + ",length=" + b.length + ",bytes.length=" + b.bytes.length);
+ }
+
+ private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
+ switch(fi.getNormType()) {
+ case NUMERIC:
+ checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
+ break;
+ default:
+ throw new AssertionError("wtf: " + fi.getNormType());
}
}
/**
- * Test term vectors for a segment.
+ * Test term vectors.
+ * @lucene.experimental
*/
- private Status.TermVectorStatus testTermVectors(FieldInfos fieldInfos, SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
- final Status.TermVectorStatus status = new Status.TermVectorStatus();
+ public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) {
+ return testTermVectors(reader, infoStream, false, false);
+ }
+ /**
+ * Test term vectors.
+ * @lucene.experimental
+ */
+ public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors) {
+ final Status.TermVectorStatus status = new Status.TermVectorStatus();
+ final FieldInfos fieldInfos = reader.getFieldInfos();
final Bits onlyDocIsDeleted = new FixedBitSet(1);
try {
@@ -1430,7 +1407,7 @@ public class CheckIndex {
TermsEnum termsEnum = null;
TermsEnum postingsTermsEnum = null;
- for (int j = 0; j < info.info.getDocCount(); ++j) {
+ for (int j = 0; j < reader.maxDoc(); ++j) {
// Intentionally pull/visit (but don't count in
// stats) deleted documents to make sure they too
// are not corrupt:
@@ -1441,10 +1418,10 @@ public class CheckIndex {
if (tfv != null) {
// First run with no deletions:
- checkFields(tfv, null, 1, fieldInfos, false, true);
+ checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
// Again, with the one doc deleted:
- checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true);
+ checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
// Only agg stats if the doc is live:
final boolean doStats = liveDocs == null || liveDocs.get(j);
@@ -1608,10 +1585,10 @@ public class CheckIndex {
}
}
float vectorAvg = status.docCount == 0 ? 0 : status.totVectors / (float)status.docCount;
- msg("OK [" + status.totVectors + " total vector count; avg " +
- format.format(vectorAvg) + " term/freq vector fields per doc]");
+ msg(infoStream, "OK [" + status.totVectors + " total vector count; avg " +
+ NumberFormat.getInstance(Locale.ROOT).format(vectorAvg) + " term/freq vector fields per doc]");
} catch (Throwable e) {
- msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
+ msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Thu Feb 7 20:48:21 2013
@@ -24,6 +24,5 @@ abstract class DocConsumer {
abstract void finishDocument() throws IOException;
abstract void flush(final SegmentWriteState state) throws IOException;
abstract void abort();
- abstract boolean freeRAM();
abstract void doAfterFlush();
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java Thu Feb 7 20:48:21 2013
@@ -28,15 +28,9 @@ abstract class DocFieldConsumer {
/** Called when an aborting exception is hit */
abstract void abort();
- /** Called when DocumentsWriterPerThread is using too much RAM.
- * The consumer should free RAM, if possible, returning
- * true if any RAM was in fact freed. */
- abstract boolean freeRAM();
-
abstract void startDocument() throws IOException;
abstract DocFieldConsumerPerField addField(FieldInfo fi);
abstract void finishDocument() throws IOException;
-
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu Feb 7 20:48:21 2013
@@ -25,16 +25,10 @@ import java.util.HashSet;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
-import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.FieldInfosWriter;
-import org.apache.lucene.codecs.PerDocConsumer;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
-import org.apache.lucene.index.TypePromoter.TypeCompatibility;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.IOUtils;
-
+import org.apache.lucene.util.Counter;
/**
* This is a DocConsumer that gathers all fields under the
@@ -47,7 +41,7 @@ import org.apache.lucene.util.IOUtils;
final class DocFieldProcessor extends DocConsumer {
final DocFieldConsumer consumer;
- final StoredFieldsConsumer fieldsWriter;
+ final StoredFieldsConsumer storedConsumer;
final Codec codec;
// Holds all fields seen in current doc
@@ -62,11 +56,14 @@ final class DocFieldProcessor extends Do
int fieldGen;
final DocumentsWriterPerThread.DocState docState;
- public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
+ final Counter bytesUsed;
+
+ public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer, StoredFieldsConsumer storedConsumer) {
this.docState = docWriter.docState;
this.codec = docWriter.codec;
+ this.bytesUsed = docWriter.bytesUsed;
this.consumer = consumer;
- fieldsWriter = new StoredFieldsConsumer(docWriter);
+ this.storedConsumer = storedConsumer;
}
@Override
@@ -78,15 +75,10 @@ final class DocFieldProcessor extends Do
childFields.put(f.getFieldInfo().name, f);
}
- fieldsWriter.flush(state);
- consumer.flush(childFields, state);
+ assert fields.size() == totalFieldCount;
- for (DocValuesConsumerHolder consumer : docValues.values()) {
- consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
- }
-
- // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
- IOUtils.close(perDocConsumer);
+ storedConsumer.flush(state);
+ consumer.flush(childFields, state);
// Important to save after asking consumer to flush so
// consumer can alter the FieldInfo* if necessary. EG,
@@ -113,11 +105,9 @@ final class DocFieldProcessor extends Do
field = next;
}
}
- IOUtils.closeWhileHandlingException(perDocConsumer);
- // TODO add abort to PerDocConsumer!
try {
- fieldsWriter.abort();
+ storedConsumer.abort();
} catch (Throwable t) {
if (th == null) {
th = t;
@@ -132,16 +122,6 @@ final class DocFieldProcessor extends Do
}
}
- try {
- if (perDocConsumer != null) {
- perDocConsumer.abort();
- }
- } catch (Throwable t) {
- if (th == null) {
- th = t;
- }
- }
-
// If any errors occured, throw it.
if (th != null) {
if (th instanceof RuntimeException) throw (RuntimeException) th;
@@ -151,11 +131,6 @@ final class DocFieldProcessor extends Do
}
}
- @Override
- public boolean freeRAM() {
- return consumer.freeRAM();
- }
-
public Collection<DocFieldConsumerPerField> fields() {
Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
for(int i=0;i<fieldHash.length;i++) {
@@ -176,8 +151,6 @@ final class DocFieldProcessor extends Do
fieldHash = new DocFieldProcessorPerField[2];
hashMask = 1;
totalFieldCount = 0;
- perDocConsumer = null;
- docValues.clear();
}
private void rehash() {
@@ -207,7 +180,7 @@ final class DocFieldProcessor extends Do
public void processDocument(FieldInfos.Builder fieldInfos) throws IOException {
consumer.startDocument();
- fieldsWriter.startDocument();
+ storedConsumer.startDocument();
fieldCount = 0;
@@ -226,38 +199,12 @@ final class DocFieldProcessor extends Do
fp.addField(field);
}
+
for (StorableField field: docState.doc.storableFields()) {
final String fieldName = field.name();
IndexableFieldType ft = field.fieldType();
-
- DocFieldProcessorPerField fp = processField(fieldInfos, thisFieldGen, fieldName, ft);
- if (ft.stored()) {
- fieldsWriter.addField(field, fp.fieldInfo);
- }
-
- final DocValues.Type dvType = ft.docValueType();
- if (dvType != null) {
- DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
- docState, fp.fieldInfo);
- DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
- if (docValuesConsumer.compatibility == null) {
- consumer.add(docState.docID, field);
- docValuesConsumer.compatibility = new TypeCompatibility(dvType,
- consumer.getValueSize());
- } else if (docValuesConsumer.compatibility.isCompatible(dvType,
- TypePromoter.getValueSize(dvType, field.binaryValue()))) {
- consumer.add(docState.docID, field);
- } else {
- docValuesConsumer.compatibility.isCompatible(dvType,
- TypePromoter.getValueSize(dvType, field.binaryValue()));
- TypeCompatibility compatibility = docValuesConsumer.compatibility;
- throw new IllegalArgumentException("Incompatible DocValues type: "
- + dvType.name() + " size: "
- + TypePromoter.getValueSize(dvType, field.binaryValue())
- + " expected: " + " type: " + compatibility.getBaseType()
- + " size: " + compatibility.getBaseSize());
- }
- }
+ FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
+ storedConsumer.addField(docState.docID, field, fieldInfo);
}
// If we are writing vectors then we must visit
@@ -280,6 +227,7 @@ final class DocFieldProcessor extends Do
private DocFieldProcessorPerField processField(FieldInfos.Builder fieldInfos,
final int thisFieldGen, final String fieldName, IndexableFieldType ft) {
+
// Make sure we have a PerField allocated
final int hashPos = fieldName.hashCode() & hashMask;
DocFieldProcessorPerField fp = fieldHash[hashPos];
@@ -305,7 +253,7 @@ final class DocFieldProcessor extends Do
rehash();
}
} else {
- fieldInfos.addOrUpdate(fp.fieldInfo.name, ft);
+ fp.fieldInfo.update(ft);
}
if (thisFieldGen != fp.lastGen) {
@@ -336,54 +284,9 @@ final class DocFieldProcessor extends Do
@Override
void finishDocument() throws IOException {
try {
- fieldsWriter.finishDocument();
+ storedConsumer.finishDocument();
} finally {
consumer.finishDocument();
}
}
-
- private static class DocValuesConsumerHolder {
- // Only used to enforce that same DV field name is never
- // added more than once per doc:
- int docID;
- final DocValuesConsumer docValuesConsumer;
- TypeCompatibility compatibility;
-
- public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
- this.docValuesConsumer = docValuesConsumer;
- }
- }
-
- final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
- private PerDocConsumer perDocConsumer;
-
- DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo)
- throws IOException {
- DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
- if (docValuesConsumerAndDocID != null) {
- if (docState.docID == docValuesConsumerAndDocID.docID) {
- throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
- }
- assert docValuesConsumerAndDocID.docID < docState.docID;
- docValuesConsumerAndDocID.docID = docState.docID;
- return docValuesConsumerAndDocID;
- }
-
- if (perDocConsumer == null) {
- PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
- perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
- if (perDocConsumer == null) {
- throw new IllegalStateException("codec=" + docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
- }
- }
- DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
- assert fieldInfo.getDocValuesType() == null || fieldInfo.getDocValuesType() == valueType;
- fieldInfo.setDocValuesType(valueType);
-
- docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
- docValuesConsumerAndDocID.docID = docState.docID;
- docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
- return docValuesConsumerAndDocID;
- }
-
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Thu Feb 7 20:48:21 2013
@@ -17,7 +17,12 @@ package org.apache.lucene.index;
* limitations under the License.
*/
+import java.util.HashMap;
+import java.util.Map;
+
import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
/**
@@ -28,16 +33,19 @@ final class DocFieldProcessorPerField {
final DocFieldConsumerPerField consumer;
final FieldInfo fieldInfo;
+ private final Counter bytesUsed;
DocFieldProcessorPerField next;
int lastGen = -1;
int fieldCount;
IndexableField[] fields = new IndexableField[1];
+ private final Map<FieldInfo,String> dvFields = new HashMap<FieldInfo,String>();
public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
this.fieldInfo = fieldInfo;
+ this.bytesUsed = docFieldProcessor.bytesUsed;
}
public void addField(IndexableField field) {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocInverter.java Thu Feb 7 20:48:21 2013
@@ -78,13 +78,7 @@ final class DocInverter extends DocField
}
@Override
- public boolean freeRAM() {
- return consumer.freeRAM();
- }
-
- @Override
public DocFieldConsumerPerField addField(FieldInfo fi) {
return new DocInverterPerField(this, fi);
}
-
}
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Thu Feb 7 20:48:21 2013
@@ -63,18 +63,21 @@ class DocumentsWriterPerThread {
This is the current indexing chain:
DocConsumer / DocConsumerPerThread
- --> code: DocFieldProcessor / DocFieldProcessorPerThread
- --> DocFieldConsumer / DocFieldConsumerPerThread / DocFieldConsumerPerField
- --> code: DocFieldConsumers / DocFieldConsumersPerThread / DocFieldConsumersPerField
- --> code: DocInverter / DocInverterPerThread / DocInverterPerField
- --> InvertedDocConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField
- --> code: TermsHash / TermsHashPerThread / TermsHashPerField
- --> TermsHashConsumer / TermsHashConsumerPerThread / TermsHashConsumerPerField
- --> code: FreqProxTermsWriter / FreqProxTermsWriterPerThread / FreqProxTermsWriterPerField
- --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerThread / TermVectorsTermsWriterPerField
- --> InvertedDocEndConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField
- --> code: NormsWriter / NormsWriterPerThread / NormsWriterPerField
- --> code: StoredFieldsWriter / StoredFieldsWriterPerThread / StoredFieldsWriterPerField
+ --> code: DocFieldProcessor
+ --> DocFieldConsumer / DocFieldConsumerPerField
+ --> code: DocFieldConsumers / DocFieldConsumersPerField
+ --> code: DocInverter / DocInverterPerField
+ --> InvertedDocConsumer / InvertedDocConsumerPerField
+ --> code: TermsHash / TermsHashPerField
+ --> TermsHashConsumer / TermsHashConsumerPerField
+ --> code: FreqProxTermsWriter / FreqProxTermsWriterPerField
+ --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerField
+ --> InvertedDocEndConsumer / InvertedDocConsumerPerField
+ --> code: NormsConsumer / NormsConsumerPerField
+ --> StoredFieldsConsumer
+ --> TwoStoredFieldConsumers
+ -> code: StoredFieldsProcessor
+ -> code: DocValuesProcessor
*/
// Build up indexing chain:
@@ -82,11 +85,14 @@ class DocumentsWriterPerThread {
final TermsHashConsumer termVectorsWriter = new TermVectorsConsumer(documentsWriterPerThread);
final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter();
- final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
- new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
- final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
+ final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
+ new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
+ final NormsConsumer normsWriter = new NormsConsumer();
final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
- return new DocFieldProcessor(documentsWriterPerThread, docInverter);
+ final StoredFieldsConsumer storedFields = new TwoStoredFieldsConsumers(
+ new StoredFieldsProcessor(documentsWriterPerThread),
+ new DocValuesProcessor(documentsWriterPerThread.bytesUsed));
+ return new DocFieldProcessor(documentsWriterPerThread, docInverter, storedFields);
}
};
@@ -651,10 +657,6 @@ class DocumentsWriterPerThread {
}
}
- PerDocWriteState newPerDocWriteState(String segmentSuffix) {
- assert segmentInfo != null;
- return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);
- }
@Override
public String toString() {
Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java?rev=1443717&r1=1443716&r2=1443717&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FieldInfo.java Thu Feb 7 20:48:21 2013
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
import java.util.HashMap;
import java.util.Map;
-import org.apache.lucene.index.DocValues.Type;
-
/**
* Access to the Field Info file that describes document fields and whether or
* not they are indexed. Each segment has a separate Field Info file. Objects
@@ -37,12 +35,12 @@ public final class FieldInfo {
public final int number;
private boolean indexed;
- private DocValues.Type docValueType;
+ private DocValuesType docValueType;
// True if any document indexed term vectors
private boolean storeTermVector;
- private DocValues.Type normType;
+ private DocValuesType normType;
private boolean omitNorms; // omit norms associated with indexed fields
private IndexOptions indexOptions;
private boolean storePayloads; // whether this field stores payloads together with term positions
@@ -82,6 +80,29 @@ public final class FieldInfo {
*/
DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
};
+
+ /**
+ * DocValues types.
+ * Note that DocValues is strongly typed, so a field cannot have different types
+ * across different documents.
+ */
+ public static enum DocValuesType {
+ /**
+ * A per-document Number
+ */
+ NUMERIC,
+ /**
+ * A per-document byte[].
+ */
+ BINARY,
+ /**
+ * A pre-sorted byte[]. Fields with this type only store distinct byte values
+ * and store an additional offset pointer per document to dereference the shared
+ * byte[]. The stored byte[] is presorted and allows access via document id,
+ * ordinal and by-value.
+ */
+ SORTED
+ };
/**
* Sole Constructor.
@@ -89,7 +110,7 @@ public final class FieldInfo {
* @lucene.experimental
*/
public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector,
- boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normsType, Map<String,String> attributes) {
+ boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map<String,String> attributes) {
this.name = name;
this.indexed = indexed;
this.number = number;
@@ -130,9 +151,13 @@ public final class FieldInfo {
return true;
}
+ void update(IndexableFieldType ft) {
+ update(ft.indexed(), false, ft.omitNorms(), false, ft.indexOptions());
+ }
+
// should only be called by FieldInfos#addOrUpdate
void update(boolean indexed, boolean storeTermVector, boolean omitNorms, boolean storePayloads, IndexOptions indexOptions) {
-
+ //System.out.println("FI.update field=" + name + " indexed=" + indexed + " omitNorms=" + omitNorms + " this.omitNorms=" + this.omitNorms);
if (this.indexed != indexed) {
this.indexed = true; // once indexed, always index
}
@@ -163,7 +188,10 @@ public final class FieldInfo {
assert checkConsistency();
}
- void setDocValuesType(DocValues.Type type) {
+ void setDocValuesType(DocValuesType type) {
+ if (docValueType != null && docValueType != type) {
+ throw new IllegalArgumentException("cannot change DocValues type from " + docValueType + " to " + type + " for field \"" + name + "\"");
+ }
docValueType = type;
assert checkConsistency();
}
@@ -181,16 +209,16 @@ public final class FieldInfo {
}
/**
- * Returns {@link DocValues.Type} of the docValues. this may be null if the field has no docvalues.
+ * Returns {@link DocValuesType} of the docValues. this may be null if the field has no docvalues.
*/
- public DocValues.Type getDocValuesType() {
+ public DocValuesType getDocValuesType() {
return docValueType;
}
/**
- * Returns {@link DocValues.Type} of the norm. this may be null if the field has no norms.
+ * Returns {@link DocValuesType} of the norm. this may be null if the field has no norms.
*/
- public DocValues.Type getNormType() {
+ public DocValuesType getNormType() {
return normType;
}
@@ -206,7 +234,10 @@ public final class FieldInfo {
assert checkConsistency();
}
- void setNormValueType(Type type) {
+ void setNormValueType(DocValuesType type) {
+ if (normType != null && normType != type) {
+ throw new IllegalArgumentException("cannot change Norm type from " + normType + " to " + type + " for field \"" + name + "\"");
+ }
normType = type;
assert checkConsistency();
}