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/10/21 20:58:44 UTC
svn commit: r1534320 [10/39] - in /lucene/dev/branches/lucene4956: ./
dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/
dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/
dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/...
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Mon Oct 21 18:58:24 2013
@@ -20,7 +20,6 @@ package org.apache.lucene.codecs.lucene4
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
@@ -361,10 +360,8 @@ public class Lucene40TermVectorsReader e
}
@Override
- public Comparator<BytesRef> getComparator() {
- // TODO: really indexer hardwires
- // this...? I guess codec could buffer and re-sort...
- return BytesRef.getUTF8SortedAsUnicodeComparator();
+ public boolean hasFreqs() {
+ return true;
}
@Override
@@ -569,11 +566,6 @@ public class Lucene40TermVectorsReader e
docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets, payloadOffsets, payloadData);
return docsAndPositionsEnum;
}
-
- @Override
- public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
}
// NOTE: sort of a silly class, since you can get the
@@ -763,5 +755,10 @@ public class Lucene40TermVectorsReader e
return new Lucene40TermVectorsReader(fieldInfos, cloneTvx, cloneTvd, cloneTvf, size, numTotalDocs);
}
+
+ @Override
+ public long ramBytesUsed() {
+ return 0;
+ }
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.lucene4
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermVectorsReader;
@@ -443,9 +442,4 @@ public final class Lucene40TermVectorsWr
IOUtils.close(tvx, tvd, tvf);
tvx = tvd = tvf = null;
}
-
- @Override
- public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41Codec.java Mon Oct 21 18:58:24 2013
@@ -101,7 +101,7 @@ public class Lucene41Codec extends Codec
}
@Override
- public final SegmentInfoFormat segmentInfoFormat() {
+ public SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Mon Oct 21 18:58:24 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter.IntBlockTermState;
import java.io.IOException;
import java.util.Arrays;
@@ -34,7 +35,6 @@ import org.apache.lucene.index.FieldInfo
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.TermState;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
@@ -59,6 +59,7 @@ public final class Lucene41PostingsReade
private final IndexInput payIn;
private final ForUtil forUtil;
+ private int version;
// public static boolean DEBUG = false;
@@ -71,27 +72,21 @@ public final class Lucene41PostingsReade
try {
docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
ioContext);
- CodecUtil.checkHeader(docIn,
+ version = CodecUtil.checkHeader(docIn,
Lucene41PostingsWriter.DOC_CODEC,
- Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.VERSION_START,
Lucene41PostingsWriter.VERSION_CURRENT);
forUtil = new ForUtil(docIn);
if (fieldInfos.hasProx()) {
posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
ioContext);
- CodecUtil.checkHeader(posIn,
- Lucene41PostingsWriter.POS_CODEC,
- Lucene41PostingsWriter.VERSION_CURRENT,
- Lucene41PostingsWriter.VERSION_CURRENT);
+ CodecUtil.checkHeader(posIn, Lucene41PostingsWriter.POS_CODEC, version, version);
if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
ioContext);
- CodecUtil.checkHeader(payIn,
- Lucene41PostingsWriter.PAY_CODEC,
- Lucene41PostingsWriter.VERSION_CURRENT,
- Lucene41PostingsWriter.VERSION_CURRENT);
+ CodecUtil.checkHeader(payIn, Lucene41PostingsWriter.PAY_CODEC, version, version);
}
}
@@ -111,7 +106,7 @@ public final class Lucene41PostingsReade
// Make sure we are talking to the matching postings writer
CodecUtil.checkHeader(termsIn,
Lucene41PostingsWriter.TERMS_CODEC,
- Lucene41PostingsWriter.VERSION_CURRENT,
+ Lucene41PostingsWriter.VERSION_START,
Lucene41PostingsWriter.VERSION_CURRENT);
final int indexBlockSize = termsIn.readVInt();
if (indexBlockSize != BLOCK_SIZE) {
@@ -141,54 +136,8 @@ public final class Lucene41PostingsReade
}
}
- // Must keep final because we do non-standard clone
- private final static class IntBlockTermState extends BlockTermState {
- long docStartFP;
- long posStartFP;
- long payStartFP;
- long skipOffset;
- long lastPosBlockOffset;
- // docid when there is a single pulsed posting, otherwise -1
- // freq is always implicitly totalTermFreq in this case.
- int singletonDocID;
-
- // Only used by the "primary" TermState -- clones don't
- // copy this (basically they are "transient"):
- ByteArrayDataInput bytesReader; // TODO: should this NOT be in the TermState...?
- byte[] bytes;
-
- @Override
- public IntBlockTermState clone() {
- IntBlockTermState other = new IntBlockTermState();
- other.copyFrom(this);
- return other;
- }
-
- @Override
- public void copyFrom(TermState _other) {
- super.copyFrom(_other);
- IntBlockTermState other = (IntBlockTermState) _other;
- docStartFP = other.docStartFP;
- posStartFP = other.posStartFP;
- payStartFP = other.payStartFP;
- lastPosBlockOffset = other.lastPosBlockOffset;
- skipOffset = other.skipOffset;
- singletonDocID = other.singletonDocID;
-
- // Do not copy bytes, bytesReader (else TermState is
- // very heavy, ie drags around the entire block's
- // byte[]). On seek back, if next() is in fact used
- // (rare!), they will be re-read from disk.
- }
-
- @Override
- public String toString() {
- return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
- }
- }
-
@Override
- public IntBlockTermState newTermState() {
+ public BlockTermState newTermState() {
return new IntBlockTermState();
}
@@ -197,81 +146,69 @@ public final class Lucene41PostingsReade
IOUtils.close(docIn, posIn, payIn);
}
- /* Reads but does not decode the byte[] blob holding
- metadata for the current terms block */
- @Override
- public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
- final IntBlockTermState termState = (IntBlockTermState) _termState;
-
- final int numBytes = termsIn.readVInt();
-
- if (termState.bytes == null) {
- termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
- termState.bytesReader = new ByteArrayDataInput();
- } else if (termState.bytes.length < numBytes) {
- termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
- }
-
- termsIn.readBytes(termState.bytes, 0, numBytes);
- termState.bytesReader.reset(termState.bytes, 0, numBytes);
- }
-
@Override
- public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
+ public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
throws IOException {
final IntBlockTermState termState = (IntBlockTermState) _termState;
- final boolean isFirstTerm = termState.termBlockOrd == 0;
final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
final boolean fieldHasPayloads = fieldInfo.hasPayloads();
- final DataInput in = termState.bytesReader;
- if (isFirstTerm) {
- if (termState.docFreq == 1) {
- termState.singletonDocID = in.readVInt();
- termState.docStartFP = 0;
- } else {
- termState.singletonDocID = -1;
- termState.docStartFP = in.readVLong();
+ if (absolute) {
+ termState.docStartFP = 0;
+ termState.posStartFP = 0;
+ termState.payStartFP = 0;
+ }
+ if (version < Lucene41PostingsWriter.VERSION_META_ARRAY) { // backward compatibility
+ _decodeTerm(in, fieldInfo, termState);
+ return;
+ }
+ termState.docStartFP += longs[0];
+ if (fieldHasPositions) {
+ termState.posStartFP += longs[1];
+ if (fieldHasOffsets || fieldHasPayloads) {
+ termState.payStartFP += longs[2];
}
- if (fieldHasPositions) {
- termState.posStartFP = in.readVLong();
- if (termState.totalTermFreq > BLOCK_SIZE) {
- termState.lastPosBlockOffset = in.readVLong();
- } else {
- termState.lastPosBlockOffset = -1;
- }
- if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
- termState.payStartFP = in.readVLong();
- } else {
- termState.payStartFP = -1;
- }
+ }
+ if (termState.docFreq == 1) {
+ termState.singletonDocID = in.readVInt();
+ } else {
+ termState.singletonDocID = -1;
+ }
+ if (fieldHasPositions) {
+ if (termState.totalTermFreq > BLOCK_SIZE) {
+ termState.lastPosBlockOffset = in.readVLong();
+ } else {
+ termState.lastPosBlockOffset = -1;
}
+ }
+ if (termState.docFreq > BLOCK_SIZE) {
+ termState.skipOffset = in.readVLong();
+ } else {
+ termState.skipOffset = -1;
+ }
+ }
+ private void _decodeTerm(DataInput in, FieldInfo fieldInfo, IntBlockTermState termState) throws IOException {
+ final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+ final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+ final boolean fieldHasPayloads = fieldInfo.hasPayloads();
+ if (termState.docFreq == 1) {
+ termState.singletonDocID = in.readVInt();
} else {
- if (termState.docFreq == 1) {
- termState.singletonDocID = in.readVInt();
+ termState.singletonDocID = -1;
+ termState.docStartFP += in.readVLong();
+ }
+ if (fieldHasPositions) {
+ termState.posStartFP += in.readVLong();
+ if (termState.totalTermFreq > BLOCK_SIZE) {
+ termState.lastPosBlockOffset = in.readVLong();
} else {
- termState.singletonDocID = -1;
- termState.docStartFP += in.readVLong();
+ termState.lastPosBlockOffset = -1;
}
- if (fieldHasPositions) {
- termState.posStartFP += in.readVLong();
- if (termState.totalTermFreq > BLOCK_SIZE) {
- termState.lastPosBlockOffset = in.readVLong();
- } else {
- termState.lastPosBlockOffset = -1;
- }
- if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
- long delta = in.readVLong();
- if (termState.payStartFP == -1) {
- termState.payStartFP = delta;
- } else {
- termState.payStartFP += delta;
- }
- }
+ if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
+ termState.payStartFP += in.readVLong();
}
}
-
if (termState.docFreq > BLOCK_SIZE) {
termState.skipOffset = in.readVLong();
} else {
@@ -1604,4 +1541,10 @@ public final class Lucene41PostingsReade
return docFreq;
}
}
+
+ @Override
+ public long ramBytesUsed() {
+ return 0;
+ }
+
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41StoredFieldsFormat.java Mon Oct 21 18:58:24 2013
@@ -23,6 +23,7 @@ import org.apache.lucene.codecs.compress
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsIndexWriter;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.PackedInts;
@@ -88,6 +89,11 @@ import org.apache.lucene.util.packed.Pac
* <li>If documents are larger than 16KB then chunks will likely contain only
* one document. However, documents can never spread across several chunks (all
* fields of a single document are in the same chunk).</li>
+ * <li>When at least one document in a chunk is large enough so that the chunk
+ * is larger than 32KB, the chunk will actually be compressed in several LZ4
+ * blocks of 16KB. This allows {@link StoredFieldVisitor}s which are only
+ * interested in the first fields of a document to not have to decompress 10MB
+ * of data if the document is 10MB, but only 16KB.</li>
* <li>Given that the original lengths are written in the metadata of the chunk,
* the decompressor can leverage this information to stop decoding as soon as
* enough data has been decompressed.</li>
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42Codec.java Mon Oct 21 18:58:24 2013
@@ -17,7 +17,10 @@ package org.apache.lucene.codecs.lucene4
* limitations under the License.
*/
+import java.io.IOException;
+
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
@@ -32,6 +35,7 @@ import org.apache.lucene.codecs.lucene40
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
+import org.apache.lucene.index.SegmentWriteState;
/**
* Implements the Lucene 4.2 index format, with configurable per-field postings
@@ -42,10 +46,12 @@ import org.apache.lucene.codecs.perfield
*
* @see org.apache.lucene.codecs.lucene42 package documentation for file format details.
* @lucene.experimental
+ * @deprecated Only for reading old 4.2 segments
*/
// NOTE: if we make largish changes in a minor release, easier to just make Lucene43Codec or whatever
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
+@Deprecated
public class Lucene42Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
@@ -89,12 +95,12 @@ public class Lucene42Codec extends Codec
}
@Override
- public final FieldInfosFormat fieldInfosFormat() {
+ public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
- public final SegmentInfoFormat segmentInfoFormat() {
+ public SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@@ -129,10 +135,15 @@ public class Lucene42Codec extends Codec
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene42");
- private final NormsFormat normsFormat = new Lucene42NormsFormat();
+ private final NormsFormat normsFormat = new Lucene42NormsFormat() {
+ @Override
+ public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+ };
@Override
- public final NormsFormat normsFormat() {
+ public NormsFormat normsFormat() {
return normsFormat;
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -118,8 +118,19 @@ import org.apache.lucene.util.packed.Blo
* <p>SortedSet entries store the list of ordinals in their BinaryData as a
* sequences of increasing {@link DataOutput#writeVLong vLong}s, delta-encoded.</p>
* </ol>
+ * <p>
+ * Limitations:
+ * <ul>
+ * <li> Binary doc values can be at most {@link #MAX_BINARY_FIELD_LENGTH} in length.
+ * </ul>
+ * @deprecated Only for reading old 4.2 segments
*/
-public final class Lucene42DocValuesFormat extends DocValuesFormat {
+@Deprecated
+public class Lucene42DocValuesFormat extends DocValuesFormat {
+
+ /** Maximum length for each binary doc values field. */
+ public static final int MAX_BINARY_FIELD_LENGTH = (1 << 15) - 2;
+
final float acceptableOverheadRatio;
/**
@@ -145,8 +156,7 @@ public final class Lucene42DocValuesForm
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
- // note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
- return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+ throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
@@ -154,8 +164,8 @@ public final class Lucene42DocValuesForm
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
- private static final String DATA_CODEC = "Lucene42DocValuesData";
- private static final String DATA_EXTENSION = "dvd";
- private static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
- private static final String METADATA_EXTENSION = "dvm";
+ static final String DATA_CODEC = "Lucene42DocValuesData";
+ static final String DATA_EXTENSION = "dvd";
+ static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
+ static final String METADATA_EXTENSION = "dvm";
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java Mon Oct 21 18:58:24 2013
@@ -17,13 +17,7 @@ package org.apache.lucene.codecs.lucene4
* limitations under the License.
*/
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.TABLE_COMPRESSED;
-import static org.apache.lucene.codecs.lucene42.Lucene42DocValuesConsumer.UNCOMPRESSED;
-
import java.io.IOException;
-import java.util.Comparator;
import java.util.HashMap;
import java.util.Map;
@@ -48,6 +42,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.PagedBytes;
+import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST;
@@ -78,6 +73,22 @@ class Lucene42DocValuesProducer extends
new HashMap<Integer,FST<Long>>();
private final int maxDoc;
+
+
+ static final byte NUMBER = 0;
+ static final byte BYTES = 1;
+ static final byte FST = 2;
+
+ static final int BLOCK_SIZE = 4096;
+
+ static final byte DELTA_COMPRESSED = 0;
+ static final byte TABLE_COMPRESSED = 1;
+ static final byte UNCOMPRESSED = 2;
+ static final byte GCD_COMPRESSED = 3;
+
+ static final int VERSION_START = 0;
+ static final int VERSION_GCD_COMPRESSION = 1;
+ static final int VERSION_CURRENT = VERSION_GCD_COMPRESSION;
Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.getDocCount();
@@ -88,8 +99,8 @@ class Lucene42DocValuesProducer extends
final int version;
try {
version = CodecUtil.checkHeader(in, metaCodec,
- Lucene42DocValuesConsumer.VERSION_START,
- Lucene42DocValuesConsumer.VERSION_CURRENT);
+ VERSION_START,
+ VERSION_CURRENT);
numerics = new HashMap<Integer,NumericEntry>();
binaries = new HashMap<Integer,BinaryEntry>();
fsts = new HashMap<Integer,FSTEntry>();
@@ -109,8 +120,8 @@ class Lucene42DocValuesProducer extends
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.openInput(dataName, state.context);
final int version2 = CodecUtil.checkHeader(data, dataCodec,
- Lucene42DocValuesConsumer.VERSION_START,
- Lucene42DocValuesConsumer.VERSION_CURRENT);
+ VERSION_START,
+ VERSION_CURRENT);
if (version != version2) {
throw new CorruptIndexException("Format versions mismatch");
}
@@ -127,7 +138,7 @@ class Lucene42DocValuesProducer extends
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
int fieldType = meta.readByte();
- if (fieldType == Lucene42DocValuesConsumer.NUMBER) {
+ if (fieldType == NUMBER) {
NumericEntry entry = new NumericEntry();
entry.offset = meta.readLong();
entry.format = meta.readByte();
@@ -140,11 +151,11 @@ class Lucene42DocValuesProducer extends
default:
throw new CorruptIndexException("Unknown format: " + entry.format + ", input=" + meta);
}
- if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
+ if (entry.format != UNCOMPRESSED) {
entry.packedIntsVersion = meta.readVInt();
}
numerics.put(fieldNumber, entry);
- } else if (fieldType == Lucene42DocValuesConsumer.BYTES) {
+ } else if (fieldType == BYTES) {
BinaryEntry entry = new BinaryEntry();
entry.offset = meta.readLong();
entry.numBytes = meta.readLong();
@@ -155,7 +166,7 @@ class Lucene42DocValuesProducer extends
entry.blockSize = meta.readVInt();
}
binaries.put(fieldNumber, entry);
- } else if (fieldType == Lucene42DocValuesConsumer.FST) {
+ } else if (fieldType == FST) {
FSTEntry entry = new FSTEntry();
entry.offset = meta.readLong();
entry.numOrds = meta.readVLong();
@@ -177,6 +188,11 @@ class Lucene42DocValuesProducer extends
return instance;
}
+ @Override
+ public long ramBytesUsed() {
+ return RamUsageEstimator.sizeOf(this);
+ }
+
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
data.seek(entry.offset);
@@ -429,6 +445,15 @@ class Lucene42DocValuesProducer extends
}
};
}
+
+ @Override
+ public Bits getDocsWithField(FieldInfo field) throws IOException {
+ if (field.getDocValuesType() == FieldInfo.DocValuesType.SORTED_SET) {
+ return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+ } else {
+ return new Bits.MatchAllBits(maxDoc);
+ }
+ }
@Override
public void close() throws IOException {
@@ -485,11 +510,6 @@ class Lucene42DocValuesProducer extends
}
@Override
- public Comparator<BytesRef> getComparator() {
- return BytesRef.getUTF8SortedAsUnicodeComparator();
- }
-
- @Override
public SeekStatus seekCeil(BytesRef text) throws IOException {
if (in.seekCeil(text) == null) {
return SeekStatus.END;
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosFormat.java Mon Oct 21 18:58:24 2013
@@ -83,10 +83,11 @@ import org.apache.lucene.store.DataOutpu
* </ul>
*
* @lucene.experimental
+ * @deprecated Only for reading old 4.2-4.5 segments
*/
-public final class Lucene42FieldInfosFormat extends FieldInfosFormat {
+@Deprecated
+public class Lucene42FieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene42FieldInfosReader();
- private final FieldInfosWriter writer = new Lucene42FieldInfosWriter();
/** Sole constructor. */
public Lucene42FieldInfosFormat() {
@@ -99,7 +100,7 @@ public final class Lucene42FieldInfosFor
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
- return writer;
+ throw new UnsupportedOperationException("this codec can only be used for reading");
}
/** Extension of field infos */
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42FieldInfosReader.java Mon Oct 21 18:58:24 2013
@@ -38,8 +38,10 @@ import org.apache.lucene.util.IOUtils;
* Lucene 4.2 FieldInfos reader.
*
* @lucene.experimental
+ * @deprecated Only for reading old 4.2-4.5 segments
* @see Lucene42FieldInfosFormat
*/
+@Deprecated
final class Lucene42FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
@@ -47,7 +49,7 @@ final class Lucene42FieldInfosReader ext
}
@Override
- public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
+ public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42NormsFormat.java Mon Oct 21 18:58:24 2013
@@ -41,7 +41,7 @@ import org.apache.lucene.util.packed.Pac
* </ul>
* @see Lucene42DocValuesFormat
*/
-public final class Lucene42NormsFormat extends NormsFormat {
+public class Lucene42NormsFormat extends NormsFormat {
final float acceptableOverheadRatio;
/**
@@ -67,7 +67,7 @@ public final class Lucene42NormsFormat e
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
- return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
+ return new Lucene42NormsConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, acceptableOverheadRatio);
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/package.html Mon Oct 21 18:58:24 2013
@@ -178,7 +178,7 @@ For each field in each document, a value
that is multiplied into the score for hits on that field.
</li>
<li>
-{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vectors}.
+{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vectors}.
For each field in each document, the term vector (sometimes
called document vector) may be stored. A term vector consists of term text and
term frequency. To add Term Vectors to your index see the
@@ -299,17 +299,17 @@ systems that frequently run out of file
<td>Encodes additional scoring factors or other per-document information.</td>
</tr>
<tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Index}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Index}</td>
<td>.tvx</td>
<td>Stores offset into the document data file</td>
</tr>
<tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Documents}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Documents}</td>
<td>.tvd</td>
<td>Contains information about each document that has term vectors</td>
</tr>
<tr>
-<td>{@link org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat Term Vector Fields}</td>
+<td>{@link org.apache.lucene.codecs.lucene42.Lucene42TermVectorsFormat Term Vector Fields}</td>
<td>.tvf</td>
<td>The field level info about term vectors</td>
</tr>
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/package.html?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/package.html Mon Oct 21 18:58:24 2013
@@ -61,9 +61,13 @@ name of your codec.
If you just want to customise the {@link org.apache.lucene.codecs.PostingsFormat}, or use different postings
formats for different fields, then you can register your custom postings format in the same way (in
META-INF/services/org.apache.lucene.codecs.PostingsFormat), and then extend the default
- {@link org.apache.lucene.codecs.lucene42.Lucene42Codec} and override
- {@link org.apache.lucene.codecs.lucene42.Lucene42Codec#getPostingsFormatForField(String)} to return your custom
+ {@link org.apache.lucene.codecs.lucene46.Lucene46Codec} and override
+ {@link org.apache.lucene.codecs.lucene46.Lucene46Codec#getPostingsFormatForField(String)} to return your custom
postings format.
</p>
+<p>
+ Similarly, if you just want to customise the {@link org.apache.lucene.codecs.DocValuesFormat} per-field, have
+ a look at {@link org.apache.lucene.codecs.lucene46.Lucene46Codec#getDocValuesFormatForField(String)}.
+</p>
</body>
</html>
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldDocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -22,13 +22,13 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Map;
-import java.util.ServiceLoader; // javadocs
+import java.util.ServiceLoader;
import java.util.TreeMap;
-import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
@@ -36,8 +36,10 @@ import org.apache.lucene.index.SegmentRe
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
/**
* Enables per field docvalues support.
@@ -74,11 +76,10 @@ public abstract class PerFieldDocValuesF
}
@Override
- public final DocValuesConsumer fieldsConsumer(SegmentWriteState state)
- throws IOException {
+ public final DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new FieldsWriter(state);
}
-
+
static class ConsumerAndSuffix implements Closeable {
DocValuesConsumer consumer;
int suffix;
@@ -95,7 +96,7 @@ public abstract class PerFieldDocValuesF
private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
private final SegmentWriteState segmentWriteState;
-
+
public FieldsWriter(SegmentWriteState state) {
segmentWriteState = state;
}
@@ -121,32 +122,53 @@ public abstract class PerFieldDocValuesF
}
private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
- final DocValuesFormat format = getDocValuesFormatForField(field.name);
+ DocValuesFormat format = null;
+ if (field.getDocValuesGen() != -1) {
+ final String formatName = field.getAttribute(PER_FIELD_FORMAT_KEY);
+ // this means the field never existed in that segment, yet is applied updates
+ if (formatName != null) {
+ format = DocValuesFormat.forName(formatName);
+ }
+ }
+ if (format == null) {
+ format = getDocValuesFormatForField(field.name);
+ }
if (format == null) {
throw new IllegalStateException("invalid null DocValuesFormat for field=\"" + field.name + "\"");
}
final String formatName = format.getName();
String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
- assert previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
+ assert field.getDocValuesGen() != -1 || previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
- Integer suffix;
+ Integer suffix = null;
ConsumerAndSuffix consumer = formats.get(format);
if (consumer == null) {
// First time we are seeing this format; create a new instance
+
+ if (field.getDocValuesGen() != -1) {
+ final String suffixAtt = field.getAttribute(PER_FIELD_SUFFIX_KEY);
+ // even when dvGen is != -1, it can still be a new field, that never
+ // existed in the segment, and therefore doesn't have the recorded
+ // attributes yet.
+ if (suffixAtt != null) {
+ suffix = Integer.valueOf(suffixAtt);
+ }
+ }
- // bump the suffix
- suffix = suffixes.get(formatName);
if (suffix == null) {
- suffix = 0;
- } else {
- suffix = suffix + 1;
+ // bump the suffix
+ suffix = suffixes.get(formatName);
+ if (suffix == null) {
+ suffix = 0;
+ } else {
+ suffix = suffix + 1;
+ }
}
suffixes.put(formatName, suffix);
- final String segmentSuffix = getFullSegmentSuffix(field.name,
- segmentWriteState.segmentSuffix,
+ final String segmentSuffix = getFullSegmentSuffix(segmentWriteState.segmentSuffix,
getSuffix(formatName, Integer.toString(suffix)));
consumer = new ConsumerAndSuffix();
consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
@@ -159,10 +181,10 @@ public abstract class PerFieldDocValuesF
}
previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
- assert previousValue == null;
+ assert field.getDocValuesGen() != -1 || previousValue == null : "suffix=" + Integer.toString(suffix) + " prevValue=" + previousValue;
// TODO: we should only provide the "slice" of FIS
- // that this PF actually sees ...
+ // that this DVF actually sees ...
return consumer.consumer;
}
@@ -177,14 +199,11 @@ public abstract class PerFieldDocValuesF
return formatName + "_" + suffix;
}
- static String getFullSegmentSuffix(String fieldName, String outerSegmentSuffix, String segmentSuffix) {
+ static String getFullSegmentSuffix(String outerSegmentSuffix, String segmentSuffix) {
if (outerSegmentSuffix.length() == 0) {
return segmentSuffix;
} else {
- // TODO: support embedding; I think it should work but
- // we need a test confirm to confirm
- // return outerSegmentSuffix + "_" + segmentSuffix;
- throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
+ return outerSegmentSuffix + "_" + segmentSuffix;
}
}
@@ -208,7 +227,7 @@ public abstract class PerFieldDocValuesF
final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
assert suffix != null;
DocValuesFormat format = DocValuesFormat.forName(formatName);
- String segmentSuffix = getSuffix(formatName, suffix);
+ String segmentSuffix = getFullSegmentSuffix(readState.segmentSuffix, getSuffix(formatName, suffix));
if (!formats.containsKey(segmentSuffix)) {
formats.put(segmentSuffix, format.fieldsProducer(new SegmentReadState(readState, segmentSuffix)));
}
@@ -265,6 +284,12 @@ public abstract class PerFieldDocValuesF
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getSortedSet(field);
}
+
+ @Override
+ public Bits getDocsWithField(FieldInfo field) throws IOException {
+ DocValuesProducer producer = fields.get(field.name);
+ return producer == null ? null : producer.getDocsWithField(field);
+ }
@Override
public void close() throws IOException {
@@ -275,6 +300,16 @@ public abstract class PerFieldDocValuesF
public DocValuesProducer clone() {
return new FieldsReader(this);
}
+
+ @Override
+ public long ramBytesUsed() {
+ long size = 0;
+ for (Map.Entry<String,DocValuesProducer> entry : formats.entrySet()) {
+ size += (entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR) +
+ entry.getValue().ramBytesUsed();
+ }
+ return size;
+ }
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Mon Oct 21 18:58:24 2013
@@ -17,24 +17,28 @@ package org.apache.lucene.codecs.perfiel
* limitations under the License.
*/
-import java.io.Closeable;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.ServiceLoader; // javadocs
+import java.util.Set;
import java.util.TreeMap;
+import java.util.TreeSet;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.Fields;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import static org.apache.lucene.index.FilterAtomicReader.FilterFields;
/**
* Enables per field postings support.
@@ -64,96 +68,22 @@ public abstract class PerFieldPostingsFo
* segment suffix name for each field. */
public static final String PER_FIELD_SUFFIX_KEY = PerFieldPostingsFormat.class.getSimpleName() + ".suffix";
-
/** Sole constructor. */
public PerFieldPostingsFormat() {
super(PER_FIELD_NAME);
}
- @Override
- public final FieldsConsumer fieldsConsumer(SegmentWriteState state)
- throws IOException {
- return new FieldsWriter(state);
- }
-
- static class FieldsConsumerAndSuffix implements Closeable {
- FieldsConsumer consumer;
+ /** Group of fields written by one PostingsFormat */
+ static class FieldsGroup {
+ final Set<String> fields = new TreeSet<String>();
int suffix;
-
- @Override
- public void close() throws IOException {
- consumer.close();
- }
- }
-
- private class FieldsWriter extends FieldsConsumer {
- private final Map<PostingsFormat,FieldsConsumerAndSuffix> formats = new HashMap<PostingsFormat,FieldsConsumerAndSuffix>();
- private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
-
- private final SegmentWriteState segmentWriteState;
-
- public FieldsWriter(SegmentWriteState state) {
- segmentWriteState = state;
- }
+ /** Custom SegmentWriteState for this group of fields,
+ * with the segmentSuffix uniqueified for this
+ * PostingsFormat */
+ SegmentWriteState state;
+ };
- @Override
- public TermsConsumer addField(FieldInfo field) throws IOException {
- final PostingsFormat format = getPostingsFormatForField(field.name);
- if (format == null) {
- throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field.name + "\"");
- }
- final String formatName = format.getName();
-
- String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
- assert previousValue == null;
-
- Integer suffix;
-
- FieldsConsumerAndSuffix consumer = formats.get(format);
- if (consumer == null) {
- // First time we are seeing this format; create a new instance
-
- // bump the suffix
- suffix = suffixes.get(formatName);
- if (suffix == null) {
- suffix = 0;
- } else {
- suffix = suffix + 1;
- }
- suffixes.put(formatName, suffix);
-
- final String segmentSuffix = getFullSegmentSuffix(field.name,
- segmentWriteState.segmentSuffix,
- getSuffix(formatName, Integer.toString(suffix)));
- consumer = new FieldsConsumerAndSuffix();
- consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
- consumer.suffix = suffix;
- formats.put(format, consumer);
- } else {
- // we've already seen this format, so just grab its suffix
- assert suffixes.containsKey(formatName);
- suffix = consumer.suffix;
- }
-
- previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
- assert previousValue == null;
-
- // TODO: we should only provide the "slice" of FIS
- // that this PF actually sees ... then stuff like
- // .hasProx could work correctly?
- // NOTE: .hasProx is already broken in the same way for the non-perfield case,
- // if there is a fieldinfo with prox that has no postings, you get a 0 byte file.
- return consumer.consumer.addField(field);
- }
-
- @Override
- public void close() throws IOException {
- // Close all subs
- IOUtils.close(formats.values());
- }
- }
-
static String getSuffix(String formatName, String suffix) {
return formatName + "_" + suffix;
}
@@ -168,6 +98,87 @@ public abstract class PerFieldPostingsFo
throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
}
}
+
+ private class FieldsWriter extends FieldsConsumer {
+ final SegmentWriteState writeState;
+
+ public FieldsWriter(SegmentWriteState writeState) {
+ this.writeState = writeState;
+ }
+
+ @Override
+ public void write(Fields fields) throws IOException {
+
+ // Maps a PostingsFormat instance to the suffix it
+ // should use
+ Map<PostingsFormat,FieldsGroup> formatToGroups = new HashMap<PostingsFormat,FieldsGroup>();
+
+ // Holds last suffix of each PostingFormat name
+ Map<String,Integer> suffixes = new HashMap<String,Integer>();
+
+ // First pass: assign field -> PostingsFormat
+ for(String field : fields) {
+ FieldInfo fieldInfo = writeState.fieldInfos.fieldInfo(field);
+
+ final PostingsFormat format = getPostingsFormatForField(field);
+
+ if (format == null) {
+ throw new IllegalStateException("invalid null PostingsFormat for field=\"" + field + "\"");
+ }
+ String formatName = format.getName();
+
+ FieldsGroup group = formatToGroups.get(format);
+ if (group == null) {
+ // First time we are seeing this format; create a
+ // new instance
+
+ // bump the suffix
+ Integer suffix = suffixes.get(formatName);
+ if (suffix == null) {
+ suffix = 0;
+ } else {
+ suffix = suffix + 1;
+ }
+ suffixes.put(formatName, suffix);
+
+ String segmentSuffix = getFullSegmentSuffix(field,
+ writeState.segmentSuffix,
+ getSuffix(formatName, Integer.toString(suffix)));
+ group = new FieldsGroup();
+ group.state = new SegmentWriteState(writeState, segmentSuffix);
+ group.suffix = suffix;
+ formatToGroups.put(format, group);
+ } else {
+ // we've already seen this format, so just grab its suffix
+ assert suffixes.containsKey(formatName);
+ }
+
+ group.fields.add(field);
+
+ String previousValue = fieldInfo.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
+ assert previousValue == null;
+
+ previousValue = fieldInfo.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(group.suffix));
+ assert previousValue == null;
+ }
+
+ // Second pass: write postings
+ for(Map.Entry<PostingsFormat,FieldsGroup> ent : formatToGroups.entrySet()) {
+ PostingsFormat format = ent.getKey();
+ final FieldsGroup group = ent.getValue();
+
+ // Exposes only the fields from this group:
+ Fields maskedFields = new FilterFields(fields) {
+ @Override
+ public Iterator<String> iterator() {
+ return group.fields.iterator();
+ }
+ };
+
+ format.fieldsConsumer(group.state).write(maskedFields);
+ }
+ }
+ }
private class FieldsReader extends FieldsProducer {
@@ -225,6 +236,22 @@ public abstract class PerFieldPostingsFo
public void close() throws IOException {
IOUtils.close(formats.values());
}
+
+ @Override
+ public long ramBytesUsed() {
+ long sizeInBytes = 0;
+ for(Map.Entry<String,FieldsProducer> entry: formats.entrySet()) {
+ sizeInBytes += entry.getKey().length() * RamUsageEstimator.NUM_BYTES_CHAR;
+ sizeInBytes += entry.getValue().ramBytesUsed();
+ }
+ return sizeInBytes;
+ }
+ }
+
+ @Override
+ public final FieldsConsumer fieldsConsumer(SegmentWriteState state)
+ throws IOException {
+ return new FieldsWriter(state);
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Mon Oct 21 18:58:24 2013
@@ -207,6 +207,12 @@ public abstract class AtomicReader exten
* this field. The returned instance should only be
* used by a single thread. */
public abstract SortedSetDocValues getSortedSetDocValues(String field) throws IOException;
+
+ /** Returns a {@link Bits} at the size of <code>reader.maxDoc()</code>,
+ * with turned on bits for each docid that does have a value for this field,
+ * or null if no DocValues were indexed for this field. The
+ * returned instance should only be used by a single thread */
+ public abstract Bits getDocsWithField(String field) throws IOException;
/** Returns {@link NumericDocValues} representing norms
* for this field, or null if no {@link NumericDocValues}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/AutomatonTermsEnum.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.Comparator;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
@@ -65,7 +64,6 @@ class AutomatonTermsEnum extends Filtere
// of terms where we should simply do sequential reads instead.
private boolean linear = false;
private final BytesRef linearUpperBound = new BytesRef(10);
- private final Comparator<BytesRef> termComp;
/**
* Construct an enumerator based upon an automaton, enumerating the specified
@@ -85,8 +83,6 @@ class AutomatonTermsEnum extends Filtere
// used for path tracking, where each bit is a numbered state.
visited = new long[runAutomaton.getSize()];
-
- termComp = getComparator();
}
/**
@@ -99,10 +95,10 @@ class AutomatonTermsEnum extends Filtere
if (runAutomaton.run(term.bytes, term.offset, term.length))
return linear ? AcceptStatus.YES : AcceptStatus.YES_AND_SEEK;
else
- return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ return (linear && term.compareTo(linearUpperBound) < 0) ?
AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
} else {
- return (linear && termComp.compare(term, linearUpperBound) < 0) ?
+ return (linear && term.compareTo(linearUpperBound) < 0) ?
AcceptStatus.NO : AcceptStatus.NO_AND_SEEK;
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValues.java Mon Oct 21 18:58:24 2013
@@ -30,17 +30,12 @@ public abstract class BinaryDocValues {
/** Lookup the value for document. */
public abstract void get(int docID, BytesRef result);
-
- /**
- * Indicates the value was missing for the document.
- */
- public static final byte[] MISSING = new byte[0];
- /** An empty BinaryDocValues which returns {@link #MISSING} for every document */
+ /** An empty BinaryDocValues which returns {@link BytesRef#EMPTY_BYTES} for every document */
public static final BinaryDocValues EMPTY = new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
- result.bytes = MISSING;
+ result.bytes = BytesRef.EMPTY_BYTES;
result.offset = 0;
result.length = 0;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesWriter.java Mon Oct 21 18:58:24 2013
@@ -22,29 +22,49 @@ import java.util.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
-import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
import org.apache.lucene.util.packed.PackedInts;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
-
/** Buffers up pending byte[] per doc, then flushes when
* segment flushes. */
class BinaryDocValuesWriter extends DocValuesWriter {
- private final ByteBlockPool pool;
+ /** Maximum length for a binary field; we set this to "a
+ * bit" below Integer.MAX_VALUE because the exact max
+ * allowed byte[] is JVM dependent, so we want to avoid
+ * a case where a large value worked in one JVM but
+ * failed later at search time with a different JVM. */
+ private static final int MAX_LENGTH = Integer.MAX_VALUE-256;
+
+ // 32 KB block sizes for PagedBytes storage:
+ private final static int BLOCK_BITS = 15;
+
+ private final PagedBytes bytes;
+ private final DataOutput bytesOut;
+
+ private final Counter iwBytesUsed;
private final AppendingDeltaPackedLongBuffer lengths;
+ private final OpenBitSet docsWithField;
private final FieldInfo fieldInfo;
- private int addedValues = 0;
+ private int addedValues;
+ private long bytesUsed;
public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
- this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
+ this.bytes = new PagedBytes(BLOCK_BITS);
+ this.bytesOut = bytes.getDataOutput();
this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
+ this.iwBytesUsed = iwBytesUsed;
+ this.docsWithField = new OpenBitSet();
+ this.bytesUsed = docsWithFieldBytesUsed();
+ iwBytesUsed.addAndGet(bytesUsed);
}
public void addValue(int docID, BytesRef value) {
@@ -54,10 +74,10 @@ class BinaryDocValuesWriter extends DocV
if (value == null) {
throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\": null value not allowed");
}
- if (value.length > (BYTE_BLOCK_SIZE - 2)) {
- throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + (BYTE_BLOCK_SIZE - 2));
+ if (value.length > MAX_LENGTH) {
+ throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + MAX_LENGTH);
}
-
+
// Fill in any holes:
while(addedValues < docID) {
addedValues++;
@@ -65,7 +85,25 @@ class BinaryDocValuesWriter extends DocV
}
addedValues++;
lengths.add(value.length);
- pool.append(value);
+ try {
+ bytesOut.writeBytes(value.bytes, value.offset, value.length);
+ } catch (IOException ioe) {
+ // Should never happen!
+ throw new RuntimeException(ioe);
+ }
+ docsWithField.set(docID);
+ updateBytesUsed();
+ }
+
+ private long docsWithFieldBytesUsed() {
+ // size of the long[] + some overhead
+ return RamUsageEstimator.sizeOf(docsWithField.getBits()) + 64;
+ }
+
+ private void updateBytesUsed() {
+ final long newBytesUsed = lengths.ramBytesUsed() + bytes.ramBytesUsed() + docsWithFieldBytesUsed();
+ iwBytesUsed.addAndGet(newBytesUsed - bytesUsed);
+ bytesUsed = newBytesUsed;
}
@Override
@@ -75,6 +113,7 @@ class BinaryDocValuesWriter extends DocV
@Override
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
+ bytes.freeze(false);
dvConsumer.addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
@@ -92,10 +131,10 @@ class BinaryDocValuesWriter extends DocV
private class BytesIterator implements Iterator<BytesRef> {
final BytesRef value = new BytesRef();
final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
+ final DataInput bytesIterator = bytes.getDataInput();
final int size = (int) lengths.size();
final int maxDoc;
int upto;
- long byteOffset;
BytesIterator(int maxDoc) {
this.maxDoc = maxDoc;
@@ -111,19 +150,27 @@ class BinaryDocValuesWriter extends DocV
if (!hasNext()) {
throw new NoSuchElementException();
}
+ final BytesRef v;
if (upto < size) {
int length = (int) lengthsIterator.next();
value.grow(length);
value.length = length;
- pool.readBytes(byteOffset, value.bytes, value.offset, value.length);
- byteOffset += length;
+ try {
+ bytesIterator.readBytes(value.bytes, value.offset, value.length);
+ } catch (IOException ioe) {
+ // Should never happen!
+ throw new RuntimeException(ioe);
+ }
+ if (docsWithField.get(upto)) {
+ v = value;
+ } else {
+ v = null;
+ }
} else {
- // This is to handle last N documents not having
- // this DV field in the end of the segment:
- value.length = 0;
+ v = null;
}
upto++;
- return value;
+ return v;
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletes.java Mon Oct 21 18:58:24 2013
@@ -19,25 +19,26 @@ package org.apache.lucene.index;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.RamUsageEstimator;
-/* Holds buffered deletes, by docID, term or query for a
+/* Holds buffered deletes and updates, by docID, term or query for a
* single segment. This is used to hold buffered pending
- * deletes against the to-be-flushed segment. Once the
- * deletes are pushed (on flush in DocumentsWriter), these
- * deletes are converted to a FrozenDeletes instance. */
+ * deletes and updates against the to-be-flushed segment. Once the
+ * deletes and updates are pushed (on flush in DocumentsWriter), they
+ * are converted to a FrozenDeletes instance. */
// NOTE: instances of this class are accessed either via a private
// instance on DocumentWriterPerThread, or via sync'd code by
// DocumentsWriterDeleteQueue
-class BufferedDeletes {
+class BufferedDeletes { // TODO (DVU_RENAME) BufferedUpdates?
/* Rough logic: HashMap has an array[Entry] w/ varying
load factor (say 2 * POINTER). Entry is object w/ Term
@@ -63,11 +64,50 @@ class BufferedDeletes {
undercount (say 24 bytes). Integer is OBJ_HEADER + INT. */
final static int BYTES_PER_DEL_QUERY = 5*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 2*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_INT + 24;
+ /* Rough logic: NumericUpdate calculates its actual size,
+ * including the update Term and DV field (String). The
+ * per-field map holds a reference to the updated field, and
+ * therefore we only account for the object reference and
+ * map space itself. This is incremented when we first see
+ * an updated field.
+ *
+ * HashMap has an array[Entry] w/ varying load
+ * factor (say 2*POINTER). Entry is an object w/ String key,
+ * LinkedHashMap val, int hash, Entry next (OBJ_HEADER + 3*POINTER + INT).
+ *
+ * LinkedHashMap (val) is counted as OBJ_HEADER, array[Entry] ref + header, 4*INT, 1*FLOAT,
+ * Set (entrySet) (2*OBJ_HEADER + ARRAY_HEADER + 2*POINTER + 4*INT + FLOAT)
+ */
+ final static int BYTES_PER_NUMERIC_FIELD_ENTRY =
+ 7*RamUsageEstimator.NUM_BYTES_OBJECT_REF + 3*RamUsageEstimator.NUM_BYTES_OBJECT_HEADER +
+ RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + 5*RamUsageEstimator.NUM_BYTES_INT + RamUsageEstimator.NUM_BYTES_FLOAT;
+
+ /* Rough logic: Incremented when we see another Term for an already updated
+ * field.
+ * LinkedHashMap has an array[Entry] w/ varying load factor
+ * (say 2*POINTER). Entry is an object w/ Term key, NumericUpdate val,
+ * int hash, Entry next, Entry before, Entry after (OBJ_HEADER + 5*POINTER + INT).
+ *
+ * Term (key) is counted only as POINTER.
+ * NumericUpdate (val) counts its own size and isn't accounted for here.
+ */
+ final static int BYTES_PER_NUMERIC_UPDATE_ENTRY = 7*RamUsageEstimator.NUM_BYTES_OBJECT_REF + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + RamUsageEstimator.NUM_BYTES_INT;
+
final AtomicInteger numTermDeletes = new AtomicInteger();
+ final AtomicInteger numNumericUpdates = new AtomicInteger();
final Map<Term,Integer> terms = new HashMap<Term,Integer>();
final Map<Query,Integer> queries = new HashMap<Query,Integer>();
final List<Integer> docIDs = new ArrayList<Integer>();
+ // Map<dvField,Map<updateTerm,NumericUpdate>>
+ // For each field we keep an ordered list of NumericUpdates, key'd by the
+ // update Term. LinkedHashMap guarantees we will later traverse the map in
+ // insertion order (so that if two terms affect the same document, the last
+ // one that came in wins), and helps us detect faster if the same Term is
+ // used to update the same field multiple times (so we later traverse it
+ // only once).
+ final Map<String,LinkedHashMap<Term,NumericUpdate>> numericUpdates = new HashMap<String,LinkedHashMap<Term,NumericUpdate>>();
+
public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE);
final AtomicLong bytesUsed;
@@ -75,21 +115,17 @@ class BufferedDeletes {
private final static boolean VERBOSE_DELETES = false;
long gen;
+
public BufferedDeletes() {
- this(new AtomicLong());
- }
-
- BufferedDeletes(AtomicLong bytesUsed) {
- assert bytesUsed != null;
- this.bytesUsed = bytesUsed;
+ this.bytesUsed = new AtomicLong();
}
@Override
public String toString() {
if (VERBOSE_DELETES) {
return "gen=" + gen + " numTerms=" + numTermDeletes + ", terms=" + terms
- + ", queries=" + queries + ", docIDs=" + docIDs + ", bytesUsed="
- + bytesUsed;
+ + ", queries=" + queries + ", docIDs=" + docIDs + ", numericUpdates=" + numericUpdates
+ + ", bytesUsed=" + bytesUsed;
} else {
String s = "gen=" + gen;
if (numTermDeletes.get() != 0) {
@@ -101,6 +137,9 @@ class BufferedDeletes {
if (docIDs.size() != 0) {
s += " " + docIDs.size() + " deleted docIDs";
}
+ if (numNumericUpdates.get() != 0) {
+ s += " " + numNumericUpdates.get() + " numeric updates (unique count=" + numericUpdates.size() + ")";
+ }
if (bytesUsed.get() != 0) {
s += " bytesUsed=" + bytesUsed.get();
}
@@ -145,20 +184,46 @@ class BufferedDeletes {
}
}
+ public void addNumericUpdate(NumericUpdate update, int docIDUpto) {
+ LinkedHashMap<Term,NumericUpdate> fieldUpdates = numericUpdates.get(update.field);
+ if (fieldUpdates == null) {
+ fieldUpdates = new LinkedHashMap<Term,NumericUpdate>();
+ numericUpdates.put(update.field, fieldUpdates);
+ bytesUsed.addAndGet(BYTES_PER_NUMERIC_FIELD_ENTRY);
+ }
+ final NumericUpdate current = fieldUpdates.get(update.term);
+ if (current != null && docIDUpto < current.docIDUpto) {
+ // Only record the new number if it's greater than or equal to the current
+ // one. This is important because if multiple threads are replacing the
+ // same doc at nearly the same time, it's possible that one thread that
+ // got a higher docID is scheduled before the other threads.
+ return;
+ }
+
+ update.docIDUpto = docIDUpto;
+ // since it's a LinkedHashMap, we must first remove the Term entry so that
+ // it's added last (we're interested in insertion-order).
+ if (current != null) {
+ fieldUpdates.remove(update.term);
+ }
+ fieldUpdates.put(update.term, update);
+ numNumericUpdates.incrementAndGet();
+ if (current == null) {
+ bytesUsed.addAndGet(BYTES_PER_NUMERIC_UPDATE_ENTRY + update.sizeInBytes());
+ }
+ }
+
void clear() {
terms.clear();
queries.clear();
docIDs.clear();
+ numericUpdates.clear();
numTermDeletes.set(0);
+ numNumericUpdates.set(0);
bytesUsed.set(0);
}
- void clearDocIDs() {
- bytesUsed.addAndGet(-docIDs.size()*BYTES_PER_DEL_DOCID);
- docIDs.clear();
- }
-
boolean any() {
- return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0;
+ return terms.size() > 0 || docIDs.size() > 0 || queries.size() > 0 || numericUpdates.size() > 0;
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Mon Oct 21 18:58:24 2013
@@ -18,10 +18,13 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.List;
import java.util.ArrayList;
-import java.util.Comparator;
+import java.util.Arrays;
import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -35,12 +38,12 @@ import org.apache.lucene.util.InfoStream
/* Tracks the stream of {@link BufferedDeletes}.
* When DocumentsWriterPerThread flushes, its buffered
- * deletes are appended to this stream. We later
- * apply these deletes (resolve them to the actual
+ * deletes and updates are appended to this stream. We later
+ * apply them (resolve them to the actual
* docIDs, per segment) when a merge is started
* (only to the to-be-merged segments). We
* also apply to all segments when NRT reader is pulled,
- * commit/close is called, or when too many deletes are
+ * commit/close is called, or when too many deletes or updates are
* buffered and must be flushed (by RAM usage or by count).
*
* Each packet is assigned a generation, and each flushed or
@@ -48,7 +51,7 @@ import org.apache.lucene.util.InfoStream
* track which BufferedDeletes packets to apply to any given
* segment. */
-class BufferedDeletesStream {
+class BufferedDeletesStream { // TODO (DVU_RENAME) BufferedUpdatesStream
// TODO: maybe linked list?
private final List<FrozenBufferedDeletes> deletes = new ArrayList<FrozenBufferedDeletes>();
@@ -114,6 +117,7 @@ class BufferedDeletesStream {
}
public static class ApplyDeletesResult {
+
// True if any actual deletes took place:
public final boolean anyDeletes;
@@ -183,7 +187,7 @@ class BufferedDeletesStream {
final long segGen = info.getBufferedDeletesGen();
if (packet != null && segGen < packet.delGen()) {
- //System.out.println(" coalesce");
+// System.out.println(" coalesce");
if (coalescedDeletes == null) {
coalescedDeletes = new CoalescedDeletes();
}
@@ -210,15 +214,21 @@ class BufferedDeletesStream {
int delCount = 0;
final boolean segAllDeletes;
try {
+ Map<String,NumericFieldUpdates> fieldUpdates = null;
if (coalescedDeletes != null) {
//System.out.println(" del coalesced");
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
+ fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, fieldUpdates);
}
//System.out.println(" del exact");
// Don't delete by Term here; DocumentsWriterPerThread
// already did that on flush:
delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader);
+ fieldUpdates = applyNumericDocValuesUpdates(Arrays.asList(packet.updates), rld, reader, fieldUpdates);
+ if (!fieldUpdates.isEmpty()) {
+ rld.writeFieldUpdates(info.info.dir, fieldUpdates);
+ }
final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
assert fullDelCount <= rld.info.info.getDocCount();
segAllDeletes = fullDelCount == rld.info.info.getDocCount();
@@ -265,10 +275,14 @@ class BufferedDeletesStream {
try {
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
+ Map<String,NumericFieldUpdates> fieldUpdates = applyNumericDocValuesUpdates(coalescedDeletes.numericDVUpdates, rld, reader, null);
+ if (!fieldUpdates.isEmpty()) {
+ rld.writeFieldUpdates(info.info.dir, fieldUpdates);
+ }
final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
assert fullDelCount <= rld.info.info.getDocCount();
segAllDeletes = fullDelCount == rld.info.info.getDocCount();
- } finally {
+ } finally {
rld.release(reader);
readerPool.release(rld);
}
@@ -282,7 +296,7 @@ class BufferedDeletesStream {
}
if (infoStream.isEnabled("BD")) {
- infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
+ infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + coalescedDeletes + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
}
}
info.setBufferedDeletesGen(gen);
@@ -377,7 +391,7 @@ class BufferedDeletesStream {
currentField = term.field();
Terms terms = fields.terms(currentField);
if (terms != null) {
- termsEnum = terms.iterator(null);
+ termsEnum = terms.iterator(termsEnum);
} else {
termsEnum = null;
}
@@ -402,15 +416,15 @@ class BufferedDeletesStream {
if (docID == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
+ if (!any) {
+ rld.initWritableLiveDocs();
+ any = true;
+ }
// NOTE: there is no limit check on the docID
// when deleting by Term (unlike by Query)
// because on flush we apply all Term deletes to
// each segment. So all Term deleting here is
// against prior segments:
- if (!any) {
- rld.initWritableLiveDocs();
- any = true;
- }
if (rld.delete(docID)) {
delCount++;
}
@@ -422,6 +436,87 @@ class BufferedDeletesStream {
return delCount;
}
+ // NumericDocValues Updates
+ // If otherFieldUpdates != null, we need to merge the updates into them
+ private synchronized Map<String,NumericFieldUpdates> applyNumericDocValuesUpdates(Iterable<NumericUpdate> updates,
+ ReadersAndLiveDocs rld, SegmentReader reader, Map<String,NumericFieldUpdates> otherFieldUpdates) throws IOException {
+ Fields fields = reader.fields();
+ if (fields == null) {
+ // This reader has no postings
+ return Collections.emptyMap();
+ }
+
+ // TODO: we can process the updates per DV field, from last to first so that
+ // if multiple terms affect same document for the same field, we add an update
+ // only once (that of the last term). To do that, we can keep a bitset which
+ // marks which documents have already been updated. So e.g. if term T1
+ // updates doc 7, and then we process term T2 and it updates doc 7 as well,
+ // we don't apply the update since we know T1 came last and therefore wins
+ // the update.
+ // We can also use that bitset as 'liveDocs' to pass to TermEnum.docs(), so
+ // that these documents aren't even returned.
+
+ String currentField = null;
+ TermsEnum termsEnum = null;
+ DocsEnum docs = null;
+ final Map<String,NumericFieldUpdates> result = otherFieldUpdates == null ? new HashMap<String,NumericFieldUpdates>() : otherFieldUpdates;
+ //System.out.println(Thread.currentThread().getName() + " numericDVUpdate reader=" + reader);
+ for (NumericUpdate update : updates) {
+ Term term = update.term;
+ int limit = update.docIDUpto;
+
+ // TODO: we traverse the terms in update order (not term order) so that we
+ // apply the updates in the correct order, i.e. if two terms udpate the
+ // same document, the last one that came in wins, irrespective of the
+ // terms lexical order.
+ // we can apply the updates in terms order if we keep an updatesGen (and
+ // increment it with every update) and attach it to each NumericUpdate. Note
+ // that we cannot rely only on docIDUpto because an app may send two updates
+ // which will get same docIDUpto, yet will still need to respect the order
+ // those updates arrived.
+
+ if (!term.field().equals(currentField)) {
+ // if we change the code to process updates in terms order, enable this assert
+// assert currentField == null || currentField.compareTo(term.field()) < 0;
+ currentField = term.field();
+ Terms terms = fields.terms(currentField);
+ if (terms != null) {
+ termsEnum = terms.iterator(termsEnum);
+ } else {
+ termsEnum = null;
+ continue; // no terms in that field
+ }
+ }
+
+ if (termsEnum == null) {
+ continue;
+ }
+ // System.out.println(" term=" + term);
+
+ if (termsEnum.seekExact(term.bytes())) {
+ // we don't need term frequencies for this
+ DocsEnum docsEnum = termsEnum.docs(rld.getLiveDocs(), docs, DocsEnum.FLAG_NONE);
+
+ //System.out.println("BDS: got docsEnum=" + docsEnum);
+
+ NumericFieldUpdates fieldUpdates = result.get(update.field);
+ if (fieldUpdates == null) {
+ fieldUpdates = new NumericFieldUpdates.PackedNumericFieldUpdates(reader.maxDoc());
+ result.put(update.field, fieldUpdates);
+ }
+ int doc;
+ while ((doc = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+ //System.out.println(Thread.currentThread().getName() + " numericDVUpdate term=" + term + " doc=" + docID);
+ if (doc >= limit) {
+ break; // no more docs that can be updated for this term
+ }
+ fieldUpdates.add(doc, update.value);
+ }
+ }
+ }
+ return result;
+ }
+
public static class QueryAndLimit {
public final Query query;
public final int limit;