You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC
svn commit: r1512909 [7/38] - in /lucene/dev/branches/lucene4956: ./
dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/
dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/
dev-tools/idea/solr/core/src/test/ dev-tool...
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Sun Aug 11 12:19:13 2013
@@ -102,20 +102,23 @@ public final class CompressingTermVector
numDocs = si.getDocCount();
IndexInput indexStream = null;
try {
- vectorsStream = d.openInput(IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION), context);
+ // Load the index into memory
final String indexStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_INDEX_EXTENSION);
indexStream = d.openInput(indexStreamFN, context);
-
final String codecNameIdx = formatName + CODEC_SFX_IDX;
- final String codecNameDat = formatName + CODEC_SFX_DAT;
CodecUtil.checkHeader(indexStream, codecNameIdx, VERSION_START, VERSION_CURRENT);
- CodecUtil.checkHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
- assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
assert CodecUtil.headerLength(codecNameIdx) == indexStream.getFilePointer();
-
indexReader = new CompressingStoredFieldsIndexReader(indexStream, si);
+ indexStream.close();
indexStream = null;
+ // Open the data file and read metadata
+ final String vectorsStreamFN = IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
+ vectorsStream = d.openInput(vectorsStreamFN, context);
+ final String codecNameDat = formatName + CODEC_SFX_DAT;
+ CodecUtil.checkHeader(vectorsStream, codecNameDat, VERSION_START, VERSION_CURRENT);
+ assert CodecUtil.headerLength(codecNameDat) == vectorsStream.getFilePointer();
+
packedIntsVersion = vectorsStream.readVInt();
chunkSize = vectorsStream.readVInt();
decompressor = compressionMode.newDecompressor();
@@ -161,7 +164,7 @@ public final class CompressingTermVector
@Override
public void close() throws IOException {
if (!closed) {
- IOUtils.close(vectorsStream, indexReader);
+ IOUtils.close(vectorsStream);
closed = true;
}
}
@@ -821,7 +824,7 @@ public final class CompressingTermVector
}
@Override
- public SeekStatus seekCeil(BytesRef text, boolean useCache)
+ public SeekStatus seekCeil(BytesRef text)
throws IOException {
if (ord < numTerms && ord >= 0) {
final int cmp = term().compareTo(text);
@@ -848,16 +851,7 @@ public final class CompressingTermVector
@Override
public void seekExact(long ord) throws IOException {
- if (ord < -1 || ord >= numTerms) {
- throw new IOException("ord is out of range: ord=" + ord + ", numTerms=" + numTerms);
- }
- if (ord < this.ord) {
- reset();
- }
- for (int i = this.ord; i < ord; ++i) {
- next();
- }
- assert ord == this.ord();
+ throw new UnsupportedOperationException();
}
@Override
@@ -867,7 +861,7 @@ public final class CompressingTermVector
@Override
public long ord() throws IOException {
- return ord;
+ throw new UnsupportedOperationException();
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java Sun Aug 11 12:19:13 2013
@@ -45,6 +45,7 @@ import org.apache.lucene.store.IndexOutp
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.GrowableByteArrayDataOutput;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.BlockPackedWriter;
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/BitVector.java Sun Aug 11 12:19:13 2013
@@ -26,6 +26,7 @@ import org.apache.lucene.store.Directory
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BitUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.MutableBits;
@@ -166,7 +167,7 @@ final class BitVector implements Cloneab
int c = 0;
int end = bits.length;
for (int i = 0; i < end; i++) {
- c += BYTE_COUNTS[bits[i] & 0xFF]; // sum bits per byte
+ c += BitUtil.bitCount(bits[i]); // sum bits per byte
}
count = c;
}
@@ -179,29 +180,12 @@ final class BitVector implements Cloneab
int c = 0;
int end = bits.length;
for (int i = 0; i < end; i++) {
- c += BYTE_COUNTS[bits[i] & 0xFF]; // sum bits per byte
+ c += BitUtil.bitCount(bits[i]); // sum bits per byte
}
return c;
}
- private static final byte[] BYTE_COUNTS = { // table of bits/byte
- 0, 1, 1, 2, 1, 2, 2, 3, 1, 2, 2, 3, 2, 3, 3, 4,
- 1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
- 1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
- 1, 2, 2, 3, 2, 3, 3, 4, 2, 3, 3, 4, 3, 4, 4, 5,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
- 2, 3, 3, 4, 3, 4, 4, 5, 3, 4, 4, 5, 4, 5, 5, 6,
- 3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
- 3, 4, 4, 5, 4, 5, 5, 6, 4, 5, 5, 6, 5, 6, 6, 7,
- 4, 5, 5, 6, 5, 6, 6, 7, 5, 6, 6, 7, 6, 7, 7, 8
- };
+
private static String CODEC = "BitVector";
@@ -294,7 +278,7 @@ final class BitVector implements Cloneab
output.writeVInt(i-last);
output.writeByte(bits[i]);
last = i;
- numCleared -= (8-BYTE_COUNTS[bits[i] & 0xFF]);
+ numCleared -= (8-BitUtil.bitCount(bits[i]));
assert numCleared >= 0 || (i == (bits.length-1) && numCleared == -(8-(size&7)));
}
}
@@ -399,7 +383,7 @@ final class BitVector implements Cloneab
while (n>0) {
last += input.readVInt();
bits[last] = input.readByte();
- n -= BYTE_COUNTS[bits[last] & 0xFF];
+ n -= BitUtil.bitCount(bits[last]);
assert n >= 0;
}
}
@@ -416,7 +400,7 @@ final class BitVector implements Cloneab
while (numCleared>0) {
last += input.readVInt();
bits[last] = input.readByte();
- numCleared -= 8-BYTE_COUNTS[bits[last] & 0xFF];
+ numCleared -= 8-BitUtil.bitCount(bits[last]);
assert numCleared >= 0 || (last == (bits.length-1) && numCleared == -(8-(size&7)));
}
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -258,8 +258,7 @@ public class Lucene40PostingsFormat exte
state.segmentInfo,
postings,
state.context,
- state.segmentSuffix,
- state.termsIndexDivisor);
+ state.segmentSuffix);
success = true;
return ret;
} finally {
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -433,7 +433,7 @@ public class Lucene40TermVectorsReader e
// NOTE: slow! (linear scan)
@Override
- public SeekStatus seekCeil(BytesRef text, boolean useCache)
+ public SeekStatus seekCeil(BytesRef text)
throws IOException {
if (nextTerm != 0) {
final int cmp = text.compareTo(term);
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -242,8 +242,8 @@ public final class Lucene40TermVectorsWr
if (payloads) {
tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
}
- for (int i = 0; i < bufferedIndex; i++) {
- if (offsets) {
+ if (offsets) {
+ for (int i = 0; i < bufferedIndex; i++) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html Sun Aug 11 12:19:13 2013
@@ -372,13 +372,7 @@ term vectors.</li>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
<div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
document numbers, and the index file format uses an <code>Int32</code>
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Sun Aug 11 12:19:13 2013
@@ -161,7 +161,7 @@ import org.apache.lucene.util.packed.Pac
* <li>SkipFPDelta determines the position of this term's SkipData within the .doc
* file. In particular, it is the length of the TermFreq data.
* SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
- * (i.e. 8 in Lucene41PostingsFormat).</li>
+ * (i.e. 128 in Lucene41PostingsFormat).</li>
* <li>SingletonDocID is an optimization when a term only appears in one document. In this case, instead
* of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the
* single document ID is written to the term dictionary.</li>
@@ -439,8 +439,7 @@ public final class Lucene41PostingsForma
state.segmentInfo,
postingsReader,
state.context,
- state.segmentSuffix,
- state.termsIndexDivisor);
+ state.segmentSuffix);
success = true;
return ret;
} finally {
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene41/package.html Sun Aug 11 12:19:13 2013
@@ -381,13 +381,7 @@ the term dictionary. Stored fields are c
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
<div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
document numbers, and the index file format uses an <code>Int32</code>
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesConsumer.java Sun Aug 11 12:19:13 2013
@@ -245,7 +245,7 @@ class Lucene42DocValuesConsumer extends
meta.writeVInt(field.number);
meta.writeByte(FST);
meta.writeLong(data.getFilePointer());
- PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+ PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
Builder<Long> builder = new Builder<Long>(INPUT_TYPE.BYTE1, outputs);
IntsRef scratch = new IntsRef();
long ord = 0;
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -120,16 +120,33 @@ import org.apache.lucene.util.packed.Blo
* </ol>
*/
public final class Lucene42DocValuesFormat extends DocValuesFormat {
-
- /** Sole constructor */
+ final float acceptableOverheadRatio;
+
+ /**
+ * Calls {@link #Lucene42DocValuesFormat(float)
+ * Lucene42DocValuesFormat(PackedInts.DEFAULT)}
+ */
public Lucene42DocValuesFormat() {
+ this(PackedInts.DEFAULT);
+ }
+
+ /**
+ * Creates a new Lucene42DocValuesFormat with the specified
+ * <code>acceptableOverheadRatio</code> for NumericDocValues.
+ * @param acceptableOverheadRatio compression parameter for numerics.
+ * Currently this is only used when the number of unique values is small.
+ *
+ * @lucene.experimental
+ */
+ public Lucene42DocValuesFormat(float acceptableOverheadRatio) {
super("Lucene42");
+ this.acceptableOverheadRatio = acceptableOverheadRatio;
}
@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, PackedInts.DEFAULT);
+ return new Lucene42DocValuesConsumer(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/Lucene42DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/codecs/lucene42/Lucene42DocValuesProducer.java?rev=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -278,7 +278,7 @@ class Lucene42DocValuesProducer extends
instance = fstInstances.get(field.number);
if (instance == null) {
data.seek(entry.offset);
- instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
+ instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
fstInstances.put(field.number, instance);
}
}
@@ -352,7 +352,7 @@ class Lucene42DocValuesProducer extends
instance = fstInstances.get(field.number);
if (instance == null) {
data.seek(entry.offset);
- instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
+ instance = new FST<Long>(data, PositiveIntOutputs.getSingleton());
fstInstances.put(field.number, instance);
}
}
@@ -490,7 +490,7 @@ class Lucene42DocValuesProducer extends
}
@Override
- public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
+ public SeekStatus seekCeil(BytesRef text) throws IOException {
if (in.seekCeil(text) == null) {
return SeekStatus.END;
} else if (term().equals(text)) {
@@ -503,7 +503,7 @@ class Lucene42DocValuesProducer extends
}
@Override
- public boolean seekExact(BytesRef text, boolean useCache) throws IOException {
+ public boolean seekExact(BytesRef text) throws IOException {
if (in.seekExact(text) == null) {
return false;
} else {
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -42,14 +42,32 @@ import org.apache.lucene.util.packed.Pac
* @see Lucene42DocValuesFormat
*/
public final class Lucene42NormsFormat extends NormsFormat {
+ final float acceptableOverheadRatio;
- /** Sole constructor */
- public Lucene42NormsFormat() {}
+ /**
+ * Calls {@link #Lucene42NormsFormat(float)
+ * Lucene42DocValuesFormat(PackedInts.FASTEST)}
+ */
+ public Lucene42NormsFormat() {
+ // note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene)
+ this(PackedInts.FASTEST);
+ }
+
+ /**
+ * Creates a new Lucene42DocValuesFormat with the specified
+ * <code>acceptableOverheadRatio</code> for NumericDocValues.
+ * @param acceptableOverheadRatio compression parameter for numerics.
+ * Currently this is only used when the number of unique values is small.
+ *
+ * @lucene.experimental
+ */
+ public Lucene42NormsFormat(float acceptableOverheadRatio) {
+ this.acceptableOverheadRatio = acceptableOverheadRatio;
+ }
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
- // note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene)
- return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.FASTEST);
+ return new Lucene42DocValuesConsumer(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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -384,13 +384,7 @@ on multi-valued fields.</li>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>
<div>
-<p>When referring to term numbers, Lucene's current implementation uses a Java
-<code>int</code> to hold the term index, which means the
-maximum number of unique terms in any single index segment is ~2.1 billion
-times the term index interval (default 128) = ~274 billion. This is technically
-not a limitation of the index file format, just of Lucene's current
-implementation.</p>
-<p>Similarly, Lucene uses a Java <code>int</code> to refer to
+<p>Lucene uses a Java <code>int</code> to refer to
document numbers, and the index file format uses an <code>Int32</code>
on-disk to store document numbers. This is a limitation
of both the index file format and the current implementation. Eventually these
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/document/Field.java Sun Aug 11 12:19:13 2013
@@ -75,7 +75,6 @@ public class Field implements IndexableF
protected TokenStream tokenStream;
private transient TokenStream internalTokenStream;
- private transient ReusableStringReader internalReader;
/**
* Field's boost
@@ -552,56 +551,12 @@ public class Field implements IndexableF
} else if (readerValue() != null) {
return analyzer.tokenStream(name(), readerValue());
} else if (stringValue() != null) {
- if (internalReader == null) {
- internalReader = new ReusableStringReader();
- }
- internalReader.setValue(stringValue());
- return analyzer.tokenStream(name(), internalReader);
+ return analyzer.tokenStream(name(), stringValue());
}
throw new IllegalArgumentException("Field must have either TokenStream, String, Reader or Number value");
}
- static final class ReusableStringReader extends Reader {
- private int pos = 0, size = 0;
- private String s = null;
-
- void setValue(String s) {
- this.s = s;
- this.size = s.length();
- this.pos = 0;
- }
-
- @Override
- public int read() {
- if (pos < size) {
- return s.charAt(pos++);
- } else {
- s = null;
- return -1;
- }
- }
-
- @Override
- public int read(char[] c, int off, int len) {
- if (pos < size) {
- len = Math.min(len, size-pos);
- s.getChars(pos, pos+len, c, off);
- pos += len;
- return len;
- } else {
- s = null;
- return -1;
- }
- }
-
- @Override
- public void close() {
- pos = size; // this prevents NPE when reading after close!
- s = null;
- }
- }
-
static final class StringTokenStream extends TokenStream {
private final CharTermAttribute termAttribute = addAttribute(CharTermAttribute.class);
private final OffsetAttribute offsetAttribute = addAttribute(OffsetAttribute.class);
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -78,7 +78,7 @@ public abstract class AtomicReader exten
return 0;
}
final TermsEnum termsEnum = terms.iterator(null);
- if (termsEnum.seekExact(term.bytes(), true)) {
+ if (termsEnum.seekExact(term.bytes())) {
return termsEnum.docFreq();
} else {
return 0;
@@ -101,7 +101,7 @@ public abstract class AtomicReader exten
return 0;
}
final TermsEnum termsEnum = terms.iterator(null);
- if (termsEnum.seekExact(term.bytes(), true)) {
+ if (termsEnum.seekExact(term.bytes())) {
return termsEnum.totalTermFreq();
} else {
return 0;
@@ -156,7 +156,7 @@ public abstract class AtomicReader exten
final Terms terms = fields.terms(term.field());
if (terms != null) {
final TermsEnum termsEnum = terms.iterator(null);
- if (termsEnum.seekExact(term.bytes(), true)) {
+ if (termsEnum.seekExact(term.bytes())) {
return termsEnum.docs(getLiveDocs(), null);
}
}
@@ -176,7 +176,7 @@ public abstract class AtomicReader exten
final Terms terms = fields.terms(term.field());
if (terms != null) {
final TermsEnum termsEnum = terms.iterator(null);
- if (termsEnum.seekExact(term.bytes(), true)) {
+ if (termsEnum.seekExact(term.bytes())) {
return termsEnum.docsAndPositions(getLiveDocs(), null);
}
}
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -26,7 +26,8 @@ import org.apache.lucene.util.ByteBlockP
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
-import org.apache.lucene.util.packed.AppendingLongBuffer;
+import org.apache.lucene.util.packed.AppendingDeltaPackedLongBuffer;
+import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
@@ -36,14 +37,14 @@ import static org.apache.lucene.util.Byt
class BinaryDocValuesWriter extends DocValuesWriter {
private final ByteBlockPool pool;
- private final AppendingLongBuffer lengths;
+ private final AppendingDeltaPackedLongBuffer lengths;
private final FieldInfo fieldInfo;
private int addedValues = 0;
public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
- this.lengths = new AppendingLongBuffer();
+ this.lengths = new AppendingDeltaPackedLongBuffer(PackedInts.COMPACT);
}
public void addValue(int docID, BytesRef value) {
@@ -90,7 +91,7 @@ class BinaryDocValuesWriter extends DocV
// iterates over the values we have in ram
private class BytesIterator implements Iterator<BytesRef> {
final BytesRef value = new BytesRef();
- final AppendingLongBuffer.Iterator lengthsIterator = lengths.iterator();
+ final AppendingDeltaPackedLongBuffer.Iterator lengthsIterator = lengths.iterator();
final int size = (int) lengths.size();
final int maxDoc;
int upto;
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -33,9 +33,9 @@ import org.apache.lucene.util.RamUsageEs
* deletes are pushed (on flush in DocumentsWriter), these
* deletes are converted to a FrozenDeletes instance. */
-// NOTE: we are sync'd by BufferedDeletes, ie, all access to
-// instances of this class is via sync'd methods on
-// BufferedDeletes
+// NOTE: instances of this class are accessed either via a private
+// instance on DocumentWriterPerThread, or via sync'd code by
+// DocumentsWriterDeleteQueue
class BufferedDeletes {
@@ -136,6 +136,9 @@ class BufferedDeletes {
}
terms.put(term, Integer.valueOf(docIDUpto));
+ // note that if current != null then it means there's already a buffered
+ // delete on that term, therefore we seem to over-count. this over-counting
+ // is done to respect IndexWriterConfig.setMaxBufferedDeleteTerms.
numTermDeletes.incrementAndGet();
if (current == null) {
bytesUsed.addAndGet(BYTES_PER_DEL_TERM + term.bytes.length + (RamUsageEstimator.NUM_BYTES_CHAR * term.field().length()));
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=1512909&r1=1512908&r2=1512909&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 Sun Aug 11 12:19:13 2013
@@ -390,7 +390,7 @@ class BufferedDeletesStream {
// System.out.println(" term=" + term);
- if (termsEnum.seekExact(term.bytes(), false)) {
+ 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);
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Sun Aug 11 12:19:13 2013
@@ -30,8 +30,8 @@ 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.FieldType; // for javadocs
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.index.CheckIndex.Status.DocValuesStatus;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
@@ -159,19 +159,6 @@ public class CheckIndex {
* segment. */
public double sizeMB;
- /** Doc store offset, if this segment shares the doc
- * store files (stored fields and term vectors) with
- * other segments. This is -1 if it does not share. */
- public int docStoreOffset = -1;
-
- /** String of the shared doc store segment, or null if
- * this segment does not share the doc store files. */
- public String docStoreSegment;
-
- /** True if the shared doc store files are compound file
- * format. */
- public boolean docStoreCompoundFile;
-
/** True if this segment has pending deletions. */
public boolean hasDeletions;
@@ -297,10 +284,21 @@ public class CheckIndex {
DocValuesStatus() {
}
- /** Number of documents tested. */
- public int docCount;
/** Total number of docValues tested. */
public long totalValueFields;
+
+ /** Total number of numeric fields */
+ public long totalNumericFields;
+
+ /** Total number of binary fields */
+ public long totalBinaryFields;
+
+ /** Total number of sorted fields */
+ public long totalSortedFields;
+
+ /** Total number of sortedset fields */
+ public long totalSortedSetFields;
+
/** Exception thrown during doc values test (null on success) */
public Throwable error = null;
}
@@ -463,11 +461,11 @@ public class CheckIndex {
if (onlySegments != null) {
result.partial = true;
- if (infoStream != null)
+ if (infoStream != null) {
infoStream.print("\nChecking only these segments:");
- for (String s : onlySegments) {
- if (infoStream != null)
+ for (String s : onlySegments) {
infoStream.print(" " + s);
+ }
}
result.segmentsChecked.addAll(onlySegments);
msg(infoStream, ":");
@@ -535,7 +533,7 @@ public class CheckIndex {
}
if (infoStream != null)
infoStream.print(" test: open reader.........");
- reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
+ reader = new SegmentReader(info, IOContext.DEFAULT);
segInfoStat.openReaderPassed = true;
@@ -1117,7 +1115,7 @@ public class CheckIndex {
long totDocCountNoDeletes = 0;
long totDocFreq = 0;
for(int i=0;i<seekCount;i++) {
- if (!termsEnum.seekExact(seekTerms[i], true)) {
+ if (!termsEnum.seekExact(seekTerms[i])) {
throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
}
@@ -1272,7 +1270,7 @@ public class CheckIndex {
for (FieldInfo fieldInfo : reader.getFieldInfos()) {
if (fieldInfo.hasDocValues()) {
status.totalValueFields++;
- checkDocValues(fieldInfo, reader, infoStream);
+ checkDocValues(fieldInfo, reader, infoStream, status);
} else {
if (reader.getBinaryDocValues(fieldInfo.name) != null ||
reader.getNumericDocValues(fieldInfo.name) != null ||
@@ -1283,7 +1281,11 @@ public class CheckIndex {
}
}
- msg(infoStream, "OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
+ msg(infoStream, "OK [" + status.totalValueFields + " docvalues fields; "
+ + status.totalBinaryFields + " BINARY; "
+ + status.totalNumericFields + " NUMERIC; "
+ + status.totalSortedFields + " SORTED; "
+ + status.totalSortedSetFields + " SORTED_SET]");
} catch (Throwable e) {
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
@@ -1382,9 +1384,10 @@ public class CheckIndex {
}
}
- private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws Exception {
+ private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream, DocValuesStatus status) throws Exception {
switch(fi.getDocValuesType()) {
case SORTED:
+ status.totalSortedFields++;
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getNumericDocValues(fi.name) != null ||
@@ -1393,6 +1396,7 @@ public class CheckIndex {
}
break;
case SORTED_SET:
+ status.totalSortedSetFields++;
checkSortedSetDocValues(fi.name, reader, reader.getSortedSetDocValues(fi.name));
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getNumericDocValues(fi.name) != null ||
@@ -1401,6 +1405,7 @@ public class CheckIndex {
}
break;
case BINARY:
+ status.totalBinaryFields++;
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
if (reader.getNumericDocValues(fi.name) != null ||
reader.getSortedDocValues(fi.name) != null ||
@@ -1409,6 +1414,7 @@ public class CheckIndex {
}
break;
case NUMERIC:
+ status.totalNumericFields++;
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
if (reader.getBinaryDocValues(fi.name) != null ||
reader.getSortedDocValues(fi.name) != null ||
@@ -1543,7 +1549,7 @@ public class CheckIndex {
}
final DocsEnum postingsDocs2;
- if (!postingsTermsEnum.seekExact(term, true)) {
+ if (!postingsTermsEnum.seekExact(term)) {
throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
}
postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings);
@@ -1677,7 +1683,7 @@ public class CheckIndex {
*
* <p><b>WARNING</b>: Make sure you only call this when the
* index is not opened by any writer. */
- public void fixIndex(Status result, Codec codec) throws IOException {
+ public void fixIndex(Status result) throws IOException {
if (result.partial)
throw new IllegalArgumentException("can only fix an index that was fully checked (this status checked a subset of segments)");
result.newSegments.changed();
@@ -1732,7 +1738,6 @@ public class CheckIndex {
boolean doFix = false;
boolean doCrossCheckTermVectors = false;
- Codec codec = Codec.getDefault(); // only used when fixing
boolean verbose = false;
List<String> onlySegments = new ArrayList<String>();
String indexPath = null;
@@ -1744,13 +1749,6 @@ public class CheckIndex {
doFix = true;
} else if ("-crossCheckTermVectors".equals(arg)) {
doCrossCheckTermVectors = true;
- } else if ("-codec".equals(arg)) {
- if (i == args.length-1) {
- System.out.println("ERROR: missing name for -codec option");
- System.exit(1);
- }
- i++;
- codec = Codec.forName(args[i]);
} else if (arg.equals("-verbose")) {
verbose = true;
} else if (arg.equals("-segment")) {
@@ -1851,7 +1849,7 @@ public class CheckIndex {
System.out.println(" " + (5-s) + "...");
}
System.out.println("Writing...");
- checker.fixIndex(result, codec);
+ checker.fixIndex(result);
System.out.println("OK");
System.out.println("Wrote new segments file \"" + result.newSegments.getSegmentsFileName() + "\"");
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Sun Aug 11 12:19:13 2013
@@ -30,12 +30,11 @@ import java.util.Comparator;
* separate thread.
*
* <p>Specify the max number of threads that may run at
- * once with {@link #setMaxThreadCount}.</p>
+ * once, and the maximum number of simultaneous merges
+ * with {@link #setMaxMergesAndThreads}.</p>
*
- * <p>Separately specify the maximum number of simultaneous
- * merges with {@link #setMaxMergeCount}. If the number of
- * merges exceeds the max number of threads then the
- * largest merges are paused until one of the smaller
+ * <p>If the number of merges exceeds the max number of threads
+ * then the largest merges are paused until one of the smaller
* merges completes.</p>
*
* <p>If more than {@link #getMaxMergeCount} merges are
@@ -49,21 +48,29 @@ public class ConcurrentMergeScheduler ex
/** List of currently active {@link MergeThread}s. */
protected List<MergeThread> mergeThreads = new ArrayList<MergeThread>();
+
+ /**
+ * Default {@code maxThreadCount}.
+ * We default to 1: tests on spinning-magnet drives showed slower
+ * indexing performance if more than one merge thread runs at
+ * once (though on an SSD it was faster)
+ */
+ public static final int DEFAULT_MAX_THREAD_COUNT = 1;
+
+ /** Default {@code maxMergeCount}. */
+ public static final int DEFAULT_MAX_MERGE_COUNT = 2;
// Max number of merge threads allowed to be running at
// once. When there are more merges then this, we
// forcefully pause the larger ones, letting the smaller
// ones run, up until maxMergeCount merges at which point
// we forcefully pause incoming threads (that presumably
- // are the ones causing so much merging). We default to 1
- // here: tests on spinning-magnet drives showed slower
- // indexing perf if more than one merge thread runs at
- // once (though on an SSD it was faster):
- private int maxThreadCount = 1;
+ // are the ones causing so much merging).
+ private int maxThreadCount = DEFAULT_MAX_THREAD_COUNT;
// Max number of merges we accept before forcefully
// throttling the incoming threads
- private int maxMergeCount = 2;
+ private int maxMergeCount = DEFAULT_MAX_MERGE_COUNT;
/** {@link Directory} that holds the index. */
protected Directory dir;
@@ -80,43 +87,40 @@ public class ConcurrentMergeScheduler ex
public ConcurrentMergeScheduler() {
}
- /** Sets the max # simultaneous merge threads that should
- * be running at once. This must be <= {@link
- * #setMaxMergeCount}. */
- public void setMaxThreadCount(int count) {
- if (count < 1) {
- throw new IllegalArgumentException("count should be at least 1");
+ /**
+ * Sets the maximum number of merge threads and simultaneous merges allowed.
+ *
+ * @param maxMergeCount the max # simultaneous merges that are allowed.
+ * If a merge is necessary yet we already have this many
+ * threads running, the incoming thread (that is calling
+ * add/updateDocument) will block until a merge thread
+ * has completed. Note that we will only run the
+ * smallest <code>maxThreadCount</code> merges at a time.
+ * @param maxThreadCount the max # simultaneous merge threads that should
+ * be running at once. This must be <= <code>maxMergeCount</code>
+ */
+ public void setMaxMergesAndThreads(int maxMergeCount, int maxThreadCount) {
+ if (maxThreadCount < 1) {
+ throw new IllegalArgumentException("maxThreadCount should be at least 1");
}
- if (count > maxMergeCount) {
- throw new IllegalArgumentException("count should be <= maxMergeCount (= " + maxMergeCount + ")");
+ if (maxMergeCount < 1) {
+ throw new IllegalArgumentException("maxMergeCount should be at least 1");
}
- maxThreadCount = count;
+ if (maxThreadCount > maxMergeCount) {
+ throw new IllegalArgumentException("maxThreadCount should be <= maxMergeCount (= " + maxMergeCount + ")");
+ }
+ this.maxThreadCount = maxThreadCount;
+ this.maxMergeCount = maxMergeCount;
}
/** Returns {@code maxThreadCount}.
*
- * @see #setMaxThreadCount(int) */
+ * @see #setMaxMergesAndThreads(int, int) */
public int getMaxThreadCount() {
return maxThreadCount;
}
- /** Sets the max # simultaneous merges that are allowed.
- * If a merge is necessary yet we already have this many
- * threads running, the incoming thread (that is calling
- * add/updateDocument) will block until a merge thread
- * has completed. Note that we will only run the
- * smallest {@link #setMaxThreadCount} merges at a time. */
- public void setMaxMergeCount(int count) {
- if (count < 1) {
- throw new IllegalArgumentException("count should be at least 1");
- }
- if (count < maxThreadCount) {
- throw new IllegalArgumentException("count should be >= maxThreadCount (= " + maxThreadCount + ")");
- }
- maxMergeCount = count;
- }
-
- /** See {@link #setMaxMergeCount}. */
+ /** See {@link #setMaxMergesAndThreads}. */
public int getMaxMergeCount() {
return maxMergeCount;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Sun Aug 11 12:19:13 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.nio.file.NoSuchFileException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@@ -51,9 +52,6 @@ import org.apache.lucene.store.NoSuchDir
*/
public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader> {
- /** Default termInfosIndexDivisor. */
- public static final int DEFAULT_TERMS_INDEX_DIVISOR = 1;
-
/** The index directory. */
protected final Directory directory;
@@ -63,29 +61,7 @@ public abstract class DirectoryReader ex
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final Directory directory) throws IOException {
- return StandardDirectoryReader.open(directory, null, DEFAULT_TERMS_INDEX_DIVISOR);
- }
-
- /** Expert: Returns a IndexReader reading the index in the given
- * Directory with the given termInfosIndexDivisor.
- * @param directory the index directory
- * @param termInfosIndexDivisor Subsamples which indexed
- * terms are loaded into RAM. This has the same effect as {@link
- * IndexWriterConfig#setTermIndexInterval} except that setting
- * must be done at indexing time while this setting can be
- * set per reader. When set to N, then one in every
- * N*termIndexInterval terms in the index is loaded into
- * memory. By setting this to a value > 1 you can reduce
- * memory usage, at the expense of higher latency when
- * loading a TermInfo. The default value is 1. Set this
- * to -1 to skip loading the terms index entirely.
- * <b>NOTE:</b> divisor settings > 1 do not apply to all PostingsFormat
- * implementations, including the default one in this release. It only makes
- * sense for terms indexes that can efficiently re-sample terms at load time.
- * @throws IOException if there is a low-level IO error
- */
- public static DirectoryReader open(final Directory directory, int termInfosIndexDivisor) throws IOException {
- return StandardDirectoryReader.open(directory, null, termInfosIndexDivisor);
+ return StandardDirectoryReader.open(directory, null);
}
/**
@@ -117,29 +93,7 @@ public abstract class DirectoryReader ex
* @throws IOException if there is a low-level IO error
*/
public static DirectoryReader open(final IndexCommit commit) throws IOException {
- return StandardDirectoryReader.open(commit.getDirectory(), commit, DEFAULT_TERMS_INDEX_DIVISOR);
- }
-
- /** Expert: returns an IndexReader reading the index in the given
- * {@link IndexCommit} and termInfosIndexDivisor.
- * @param commit the commit point to open
- * @param termInfosIndexDivisor Subsamples which indexed
- * terms are loaded into RAM. This has the same effect as {@link
- * IndexWriterConfig#setTermIndexInterval} except that setting
- * must be done at indexing time while this setting can be
- * set per reader. When set to N, then one in every
- * N*termIndexInterval terms in the index is loaded into
- * memory. By setting this to a value > 1 you can reduce
- * memory usage, at the expense of higher latency when
- * loading a TermInfo. The default value is 1. Set this
- * to -1 to skip loading the terms index entirely.
- * <b>NOTE:</b> divisor settings > 1 do not apply to all PostingsFormat
- * implementations, including the default one in this release. It only makes
- * sense for terms indexes that can efficiently re-sample terms at load time.
- * @throws IOException if there is a low-level IO error
- */
- public static DirectoryReader open(final IndexCommit commit, int termInfosIndexDivisor) throws IOException {
- return StandardDirectoryReader.open(commit.getDirectory(), commit, termInfosIndexDivisor);
+ return StandardDirectoryReader.open(commit.getDirectory(), commit);
}
/**
@@ -291,7 +245,7 @@ public abstract class DirectoryReader ex
// IOException allowed to throw there, in case
// segments_N is corrupt
sis.read(dir, fileName);
- } catch (FileNotFoundException fnfe) {
+ } catch (FileNotFoundException | NoSuchFileException fnfe) {
// LUCENE-948: on NFS (and maybe others), if
// you have writers switching back and forth
// between machines, it's very likely that the
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Sun Aug 11 12:19:13 2013
@@ -24,5 +24,4 @@ abstract class DocConsumer {
abstract void finishDocument() throws IOException;
abstract void flush(final SegmentWriteState state) throws IOException;
abstract void abort();
- abstract void doAfterFlush();
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Sun Aug 11 12:19:13 2013
@@ -144,15 +144,6 @@ final class DocFieldProcessor extends Do
return fields;
}
- /** In flush we reset the fieldHash to not maintain per-field state
- * across segments */
- @Override
- void doAfterFlush() {
- fieldHash = new DocFieldProcessorPerField[2];
- hashMask = 1;
- totalFieldCount = 0;
- }
-
private void rehash() {
final int newHashSize = (fieldHash.length*2);
assert newHashSize > fieldHash.length;
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java Sun Aug 11 12:19:13 2013
@@ -17,12 +17,7 @@ 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;
/**
@@ -33,19 +28,16 @@ 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/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Sun Aug 11 12:19:13 2013
@@ -659,7 +659,7 @@ public class DocTermOrds {
}
@Override
- public SeekStatus seekCeil(BytesRef target, boolean useCache) throws IOException {
+ public SeekStatus seekCeil(BytesRef target) throws IOException {
// already here
if (term != null && term.equals(target)) {
@@ -729,7 +729,7 @@ public class DocTermOrds {
//System.out.println(" do seek term=" + base.utf8ToString());
ord = idx << indexIntervalBits;
delta = (int) (targetOrd - ord);
- final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(base, true);
+ final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(base);
assert seekStatus == TermsEnum.SeekStatus.FOUND;
} else {
//System.out.println("seek w/in block");
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Sun Aug 11 12:19:13 2013
@@ -29,15 +29,11 @@ import org.apache.lucene.index.Documents
import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
import org.apache.lucene.index.FieldInfos.FieldNumbers;
-import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FlushInfo;
-import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.MutableBits;
/**
* This class accepts multiple added documents and directly
@@ -114,6 +110,7 @@ final class DocumentsWriter {
List<String> newFiles;
final IndexWriter indexWriter;
+ final LiveIndexWriterConfig indexWriterConfig;
private AtomicInteger numDocsInRAM = new AtomicInteger(0);
@@ -144,6 +141,7 @@ final class DocumentsWriter {
this.indexWriter = writer;
this.infoStream = config.getInfoStream();
this.similarity = config.getSimilarity();
+ this.indexWriterConfig = writer.getConfig();
this.perThreadPool = config.getIndexerThreadPool();
this.chain = config.getIndexingChain();
this.perThreadPool.initialize(this, globalFieldNumbers, config);
@@ -517,7 +515,7 @@ final class DocumentsWriter {
// buffer, force them all to apply now. This is to
// prevent too-frequent flushing of a long tail of
// tiny segments:
- final double ramBufferSizeMB = indexWriter.getConfig().getRAMBufferSizeMB();
+ final double ramBufferSizeMB = indexWriterConfig.getRAMBufferSizeMB();
if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH &&
flushControl.getDeleteBytesUsed() > (1024*1024*ramBufferSizeMB/2)) {
if (infoStream.isEnabled("DW")) {
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Aug 11 12:19:13 2013
@@ -112,7 +112,7 @@ class DocumentsWriterPerThread {
// Only called by asserts
public boolean testPoint(String name) {
- return docWriter.writer.testPoint(name);
+ return docWriter.testPoint(name);
}
public void clear() {
@@ -194,6 +194,7 @@ class DocumentsWriterPerThread {
private final NumberFormat nf = NumberFormat.getInstance(Locale.ROOT);
final Allocator byteBlockAllocator;
final IntBlockPool.Allocator intBlockAllocator;
+ private final LiveIndexWriterConfig indexWriterConfig;
public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
@@ -203,6 +204,7 @@ class DocumentsWriterPerThread {
this.parent = parent;
this.fieldInfos = fieldInfos;
this.writer = parent.indexWriter;
+ this.indexWriterConfig = parent.indexWriterConfig;
this.infoStream = parent.infoStream;
this.codec = parent.codec;
this.docState = new DocState(this, infoStream);
@@ -232,6 +234,13 @@ class DocumentsWriterPerThread {
aborting = true;
}
+ final boolean testPoint(String message) {
+ if (infoStream.isEnabled("TP")) {
+ infoStream.message("TP", message);
+ }
+ return true;
+ }
+
boolean checkAndResetHasAborted() {
final boolean retval = hasAborted;
hasAborted = false;
@@ -239,7 +248,7 @@ class DocumentsWriterPerThread {
}
public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException {
- assert writer.testPoint("DocumentsWriterPerThread addDocument start");
+ assert testPoint("DocumentsWriterPerThread addDocument start");
assert deleteQueue != null;
docState.doc = doc;
docState.analyzer = analyzer;
@@ -292,7 +301,7 @@ class DocumentsWriterPerThread {
}
public int updateDocuments(Iterable<? extends IndexDocument> docs, Analyzer analyzer, Term delTerm) throws IOException {
- assert writer.testPoint("DocumentsWriterPerThread addDocuments start");
+ assert testPoint("DocumentsWriterPerThread addDocuments start");
assert deleteQueue != null;
docState.analyzer = analyzer;
if (segmentInfo == null) {
@@ -394,8 +403,8 @@ class DocumentsWriterPerThread {
++numDocsInRAM;
}
- // Buffer a specific docID for deletion. Currently only
- // used when we hit a exception when adding a document
+ // Buffer a specific docID for deletion. Currently only
+ // used when we hit an exception when adding a document
void deleteDocID(int docIDUpto) {
pendingDeletes.addDocID(docIDUpto);
// NOTE: we do not trigger flush here. This is
@@ -428,7 +437,6 @@ class DocumentsWriterPerThread {
/** Reset after a flush */
private void doAfterFlush() {
segmentInfo = null;
- consumer.doAfterFlush();
directory.getCreatedFiles().clear();
fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers);
parent.subtractFlushedNumDocs(numDocsInRAM);
@@ -460,7 +468,6 @@ class DocumentsWriterPerThread {
assert deleteSlice == null : "all deletes must be applied in prepareFlush";
segmentInfo.setDocCount(numDocsInRAM);
flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
- writer.getConfig().getTermIndexInterval(),
pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
@@ -561,7 +568,7 @@ class DocumentsWriterPerThread {
boolean success = false;
try {
- if (writer.useCompoundFile(newSegment)) {
+ if (indexWriterConfig.getUseCompoundFile()) {
// Now build compound file
Collection<String> oldFiles = IndexWriter.createCompoundFile(infoStream, directory, MergeState.CheckAbort.NONE, newSegment.info, context);
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Sun Aug 11 12:19:13 2013
@@ -147,7 +147,10 @@ abstract class DocumentsWriterPerThreadP
@Override
public DocumentsWriterPerThreadPool clone() {
// We should only be cloned before being used:
- assert numThreadStatesActive == 0;
+ if (numThreadStatesActive != 0) {
+ throw new IllegalStateException("clone this object before it is used!");
+ }
+
DocumentsWriterPerThreadPool clone;
try {
clone = (DocumentsWriterPerThreadPool) super.clone();
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Sun Aug 11 12:19:13 2013
@@ -157,8 +157,8 @@ public class FilterAtomicReader extends
}
@Override
- public SeekStatus seekCeil(BytesRef text, boolean useCache) throws IOException {
- return in.seekCeil(text, useCache);
+ public SeekStatus seekCeil(BytesRef text) throws IOException {
+ return in.seekCeil(text);
}
@Override
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Sun Aug 11 12:19:13 2013
@@ -154,7 +154,7 @@ public abstract class FilteredTermsEnum
* support seeking.
*/
@Override
- public boolean seekExact(BytesRef term, boolean useCache) throws IOException {
+ public boolean seekExact(BytesRef term) throws IOException {
throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
}
@@ -163,7 +163,7 @@ public abstract class FilteredTermsEnum
* support seeking.
*/
@Override
- public SeekStatus seekCeil(BytesRef term, boolean useCache) throws IOException {
+ public SeekStatus seekCeil(BytesRef term) throws IOException {
throw new UnsupportedOperationException(getClass().getName()+" does not support seeking");
}
@@ -222,7 +222,7 @@ public abstract class FilteredTermsEnum
//System.out.println(" seek to t=" + (t == null ? "null" : t.utf8ToString()) + " tenum=" + tenum);
// Make sure we always seek forward:
assert actualTerm == null || t == null || getComparator().compare(t, actualTerm) > 0: "curTerm=" + actualTerm + " seekTerm=" + t;
- if (t == null || tenum.seekCeil(t, false) == SeekStatus.END) {
+ if (t == null || tenum.seekCeil(t) == SeekStatus.END) {
// no more terms to seek to or enum exhausted
//System.out.println(" return null");
return null;
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java Sun Aug 11 12:19:13 2013
@@ -20,23 +20,32 @@ package org.apache.lucene.index;
import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
/**
- * Default {@link FlushPolicy} implementation that flushes based on RAM used,
- * document count and number of buffered deletes depending on the IndexWriter's
- * {@link IndexWriterConfig}.
+ * Default {@link FlushPolicy} implementation that flushes new segments based on
+ * RAM used and document count depending on the IndexWriter's
+ * {@link IndexWriterConfig}. It also applies pending deletes based on the
+ * number of buffered delete terms.
*
* <ul>
- * <li>{@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - flushes
- * based on the global number of buffered delete terms iff
- * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is enabled</li>
- * <li>{@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - flushes
- * either on the number of documents per {@link DocumentsWriterPerThread} (
+ * <li>
+ * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - applies pending delete operations based on the global number of buffered
+ * delete terms iff {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is
+ * enabled</li>
+ * <li>
+ * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - flushes either on the number of documents per
+ * {@link DocumentsWriterPerThread} (
* {@link DocumentsWriterPerThread#getNumDocsInRAM()}) or on the global active
* memory consumption in the current indexing session iff
* {@link IndexWriterConfig#getMaxBufferedDocs()} or
* {@link IndexWriterConfig#getRAMBufferSizeMB()} is enabled respectively</li>
- * <li>{@link #onUpdate(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} - calls
- * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} and
- * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)} in order</li>
+ * <li>
+ * {@link #onUpdate(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * - calls
+ * {@link #onInsert(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * and
+ * {@link #onDelete(DocumentsWriterFlushControl, DocumentsWriterPerThreadPool.ThreadState)}
+ * in order</li>
* </ul>
* All {@link IndexWriterConfig} settings are used to mark
* {@link DocumentsWriterPerThread} as flush pending during indexing with
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java Sun Aug 11 12:19:13 2013
@@ -32,18 +32,19 @@ import org.apache.lucene.util.SetOnce;
* {@link IndexWriterConfig#setRAMBufferSizeMB(double)}</li>
* <li>Number of RAM resident documents - configured via
* {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
- * <li>Number of buffered delete terms/queries - configured via
- * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}</li>
* </ul>
- *
- * The {@link IndexWriter} consults a provided {@link FlushPolicy} to control the
- * flushing process. The policy is informed for each added or
- * updated document as well as for each delete term. Based on the
- * {@link FlushPolicy}, the information provided via {@link ThreadState} and
+ * The policy also applies pending delete operations (by term and/or query),
+ * given the threshold set in
+ * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}.
+ * <p>
+ * {@link IndexWriter} consults the provided {@link FlushPolicy} to control the
+ * flushing process. The policy is informed for each added or updated document
+ * as well as for each delete term. Based on the {@link FlushPolicy}, the
+ * information provided via {@link ThreadState} and
* {@link DocumentsWriterFlushControl}, the {@link FlushPolicy} decides if a
- * {@link DocumentsWriterPerThread} needs flushing and mark it as
- * flush-pending via
- * {@link DocumentsWriterFlushControl#setFlushPending(DocumentsWriterPerThreadPool.ThreadState)}.
+ * {@link DocumentsWriterPerThread} needs flushing and mark it as flush-pending
+ * via {@link DocumentsWriterFlushControl#setFlushPending}, or if deletes need
+ * to be applied.
*
* @see ThreadState
* @see DocumentsWriterFlushControl
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Sun Aug 11 12:19:13 2013
@@ -415,7 +415,7 @@ final class FreqProxTermsWriterPerField
// which all share the same term. Now we must
// interleave the docID streams.
int docFreq = 0;
- long totTF = 0;
+ long totalTermFreq = 0;
int docID = 0;
while(true) {
@@ -483,7 +483,7 @@ final class FreqProxTermsWriterPerField
}
}
- totTF += termFreq;
+ totalTermFreq += termFreq;
// Carefully copy over the prox + payload info,
// changing the format to match Lucene's segment
@@ -540,8 +540,8 @@ final class FreqProxTermsWriterPerField
}
postingsConsumer.finishDoc();
}
- termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totTF : -1));
- sumTotalTermFreq += totTF;
+ termsConsumer.finishTerm(text, new TermStats(docFreq, writeTermFreq ? totalTermFreq : -1));
+ sumTotalTermFreq += totalTermFreq;
sumDocFreq += docFreq;
}
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,8 @@ package org.apache.lucene.index;
import java.util.List;
import java.io.IOException;
+import org.apache.lucene.store.Directory;
+
/**
* <p>Expert: policy for deletion of stale {@link IndexCommit index commits}.
*
@@ -46,6 +48,10 @@ import java.io.IOException;
* target="top"
* href="http://issues.apache.org/jira/browse/LUCENE-710">LUCENE-710</a>
* for details.</p>
+ *
+ * <p>Implementers of sub-classes should make sure that {@link #clone()}
+ * returns an independent instance able to work with any other {@link IndexWriter}
+ * or {@link Directory} instance.</p>
*/
public abstract class IndexDeletionPolicy implements Cloneable {
@@ -72,6 +78,8 @@ public abstract class IndexDeletionPolic
* @param commits List of current
* {@link IndexCommit point-in-time commits},
* sorted by age (the 0th one is the oldest commit).
+ * Note that for a new index this method is invoked with
+ * an empty list.
*/
public abstract void onInit(List<? extends IndexCommit> commits) throws IOException;
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sun Aug 11 12:19:13 2013
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.nio.file.NoSuchFileException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -170,7 +171,7 @@ final class IndexFileDeleter implements
SegmentInfos sis = new SegmentInfos();
try {
sis.read(directory, fileName);
- } catch (FileNotFoundException e) {
+ } catch (FileNotFoundException | NoSuchFileException e) {
// LUCENE-948: on NFS (and maybe others), if
// you have writers switching back and forth
// between machines, it's very likely that the
Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sun Aug 11 12:19:13 2013
@@ -214,7 +214,7 @@ public class IndexWriter implements Clos
private final Analyzer analyzer; // how to analyze text
private volatile long changeCount; // increments every time a change is completed
- private long lastCommitChangeCount; // last changeCount that was committed
+ private volatile long lastCommitChangeCount; // last changeCount that was committed
private List<SegmentInfoPerCommit> rollbackSegments; // list of segmentInfo we will fallback to if the commit fails
@@ -630,10 +630,13 @@ public class IndexWriter implements Clos
/**
* Constructs a new IndexWriter per the settings given in <code>conf</code>.
- * Note that the passed in {@link IndexWriterConfig} is
- * privately cloned; if you need to make subsequent "live"
- * changes to the configuration use {@link #getConfig}.
+ * If you want to make "live" changes to this writer instance, use
+ * {@link #getConfig()}.
+ *
* <p>
+ * <b>NOTE:</b> after ths writer is created, the given configuration instance
+ * cannot be passed to another writer. If you intend to do so, you should
+ * {@link IndexWriterConfig#clone() clone} it beforehand.
*
* @param d
* the index directory. The index is either created or appended
@@ -648,7 +651,8 @@ public class IndexWriter implements Clos
* IO error
*/
public IndexWriter(Directory d, IndexWriterConfig conf) throws IOException {
- config = new LiveIndexWriterConfig(conf.clone());
+ conf.setIndexWriter(this); // prevent reuse by other instances
+ config = new LiveIndexWriterConfig(conf);
directory = d;
analyzer = config.getAnalyzer();
infoStream = config.getInfoStream();
@@ -2269,10 +2273,6 @@ public class IndexWriter implements Clos
}
}
- synchronized boolean useCompoundFile(SegmentInfoPerCommit segmentInfo) throws IOException {
- return mergePolicy.useCompoundFile(segmentInfos, segmentInfo);
- }
-
private synchronized void resetMergeExceptions() {
mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
mergeGen++;
@@ -2428,15 +2428,16 @@ public class IndexWriter implements Clos
* close the writer. See <a href="#OOME">above</a> for details.
*
* <p>
+ * <b>NOTE:</b> empty segments are dropped by this method and not added to this
+ * index.
+ *
+ * <p>
* <b>NOTE:</b> this method merges all given {@link IndexReader}s in one
* merge. If you intend to merge a large number of readers, it may be better
* to call this method multiple times, each time with a small set of readers.
* In principle, if you use a merge policy with a {@code mergeFactor} or
* {@code maxMergeAtOnce} parameter, you should pass that many readers in one
- * call. Also, if the given readers are {@link DirectoryReader}s, they can be
- * opened with {@code termIndexInterval=-1} to save RAM, since during merge
- * the in-memory structure is not used. See
- * {@link DirectoryReader#open(Directory, int)}.
+ * call.
*
* <p>
* <b>NOTE</b>: if you call {@link #close(boolean)} with <tt>false</tt>, which
@@ -2461,11 +2462,20 @@ public class IndexWriter implements Clos
String mergedName = newSegmentName();
final List<AtomicReader> mergeReaders = new ArrayList<AtomicReader>();
for (IndexReader indexReader : readers) {
- numDocs += indexReader.numDocs();
- for (AtomicReaderContext ctx : indexReader.leaves()) {
- mergeReaders.add(ctx.reader());
+ if (indexReader.numDocs() > 0) {
+ numDocs += indexReader.numDocs();
+ for (AtomicReaderContext ctx : indexReader.leaves()) {
+ if (ctx.reader().numDocs() > 0) { // drop empty (or all deleted) segments
+ mergeReaders.add(ctx.reader());
+ }
+ }
}
}
+
+ if (mergeReaders.isEmpty()) { // no segments with documents to add
+ return;
+ }
+
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, true, -1));
// TODO: somehow we should fix this merge so it's
@@ -2475,7 +2485,7 @@ public class IndexWriter implements Clos
SegmentInfo info = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergedName, -1,
false, codec, null, null);
- SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir, config.getTermIndexInterval(),
+ SegmentMerger merger = new SegmentMerger(mergeReaders, info, infoStream, trackingDir,
MergeState.CheckAbort.NONE, globalFieldNumberMap, context);
MergeState mergeState;
@@ -2826,6 +2836,11 @@ public class IndexWriter implements Clos
commitInternal();
}
+ /** Returns true if there are changes that have not been committed */
+ public final boolean hasUncommittedChanges() {
+ return changeCount != lastCommitChangeCount;
+ }
+
private final void commitInternal() throws IOException {
if (infoStream.isEnabled("IW")) {
@@ -2865,8 +2880,8 @@ public class IndexWriter implements Clos
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
}
- lastCommitChangeCount = pendingCommitChangeCount;
segmentInfos.updateGeneration(pendingCommit);
+ lastCommitChangeCount = pendingCommitChangeCount;
rollbackSegments = pendingCommit.createBackupSegmentInfos();
deleter.checkpoint(pendingCommit, true);
} finally {
@@ -3652,7 +3667,7 @@ public class IndexWriter implements Clos
// Hold onto the "live" reader; we will use this to
// commit merged deletes
final ReadersAndLiveDocs rld = readerPool.get(info, true);
- SegmentReader reader = rld.getMergeReader(context);
+ SegmentReader reader = rld.getReader(context);
assert reader != null;
// Carefully pull the most recent live docs:
@@ -3709,7 +3724,7 @@ public class IndexWriter implements Clos
// we pass merge.getMergeReaders() instead of merge.readers to allow the
// OneMerge to return a view over the actual segments to merge
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
- merge.info.info, infoStream, dirWrapper, config.getTermIndexInterval(),
+ merge.info.info, infoStream, dirWrapper,
checkAbort, globalFieldNumberMap, context);
merge.checkAborted(directory);
@@ -4162,7 +4177,10 @@ public class IndexWriter implements Clos
// startCommitMergeDeletes
// startMergeInit
// DocumentsWriter.ThreadState.init start
- boolean testPoint(String name) {
+ private final boolean testPoint(String message) {
+ if (infoStream.isEnabled("TP")) {
+ infoStream.message("TP", message);
+ }
return true;
}